This commit is contained in:
tuanpach 2024-09-18 22:41:34 +02:00 committed by GitHub
commit 7b29cfc010
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
26 changed files with 340 additions and 158 deletions

View File

@ -93,7 +93,7 @@ void BaseExternalTable::parseStructureFromStructureField(const std::string & arg
/*one_line=*/true,
/*show_secrets=*/true,
/*print_pretty_type_names=*/false,
/*always_quote_identifiers=*/false,
/*identifier_quoting_rule=*/IdentifierQuotingRule::WhenNecessary,
/*identifier_quoting_style=*/IdentifierQuotingStyle::Backticks));
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Error while parsing table structure: expected column definition, got {}", child->formatForErrorMessage());
@ -118,7 +118,7 @@ void BaseExternalTable::parseStructureFromTypesField(const std::string & argumen
/*one_line=*/true,
/*show_secrets=*/true,
/*print_pretty_type_names=*/false,
/*always_quote_identifiers=*/false,
/*identifier_quoting_rule=*/IdentifierQuotingRule::WhenNecessary,
/*identifier_quoting_style=*/IdentifierQuotingStyle::Backticks));
}

View File

@ -1303,7 +1303,7 @@ class IColumn;
M(DateTimeOverflowBehavior, date_time_overflow_behavior, "ignore", "Overflow mode for Date, Date32, DateTime, DateTime64 types. Possible values: 'ignore', 'throw', 'saturate'.", 0) \
M(Bool, validate_experimental_and_suspicious_types_inside_nested_types, true, "Validate usage of experimental and suspicious types inside nested types like Array/Map/Tuple", 0) \
\
M(Bool, output_format_always_quote_identifiers, false, "Always quote identifiers", 0) \
M(IdentifierQuotingRule, output_format_identifier_quoting_rule, IdentifierQuotingRule::WhenNecessary, "Set the quoting rule for identifiers", 0) \
M(IdentifierQuotingStyle, output_format_identifier_quoting_style, IdentifierQuotingStyle::Backticks, "Set the quoting style for identifiers", 0) \

View File

@ -77,7 +77,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
{"join_output_by_rowlist_perkey_rows_threshold", 0, 5, "The lower limit of per-key average rows in the right table to determine whether to output by row list in hash join."},
{"create_if_not_exists", false, false, "New setting."},
{"allow_materialized_view_with_bad_select", true, true, "Support (but not enable yet) stricter validation in CREATE MATERIALIZED VIEW"},
{"output_format_always_quote_identifiers", false, false, "New setting."},
{"output_format_identifier_quoting_rule", "when_necessary", "when_necessary", "New setting."},
{"output_format_identifier_quoting_style", "Backticks", "Backticks", "New setting."},
{"parallel_replicas_mark_segment_size", 128, 0, "Value for this setting now determined automatically"},
{"database_replicated_allow_replicated_engine_arguments", 1, 0, "Don't allow explicit arguments by default"},

View File

@ -250,4 +250,11 @@ IMPLEMENT_SETTING_ENUM(IdentifierQuotingStyle, ErrorCodes::BAD_ARGUMENTS,
{"Backticks", IdentifierQuotingStyle::Backticks},
{"DoubleQuotes", IdentifierQuotingStyle::DoubleQuotes},
{"BackticksMySQL", IdentifierQuotingStyle::BackticksMySQL}})
IMPLEMENT_SETTING_ENUM(
IdentifierQuotingRule,
ErrorCodes::BAD_ARGUMENTS,
{{"user_display", IdentifierQuotingRule::UserDisplay},
{"when_necessary", IdentifierQuotingRule::WhenNecessary},
{"always", IdentifierQuotingRule::Always}})
}

View File

@ -352,6 +352,7 @@ DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeOverflowBehavior, FormatSettings::DateT
DECLARE_SETTING_ENUM(SQLSecurityType)
DECLARE_SETTING_ENUM(IdentifierQuotingRule)
DECLARE_SETTING_ENUM(IdentifierQuotingStyle)
enum class GroupArrayActionWhenLimitReached : uint8_t

View File

@ -796,7 +796,13 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
/// Verify that AST formatting is consistent:
/// If you format AST, parse it back, and format it again, you get the same string.
String formatted1 = ast->formatWithPossiblyHidingSensitiveData(0, true, true, false, false, IdentifierQuotingStyle::Backticks);
String formatted1 = ast->formatWithPossiblyHidingSensitiveData(
/*max_length=*/0,
/*one_line=*/true,
/*show_secrets=*/true,
/*print_pretty_type_names=*/false,
/*identifier_quoting_rule=*/IdentifierQuotingRule::WhenNecessary,
/*identifier_quoting_style=*/IdentifierQuotingStyle::Backticks);
/// The query can become more verbose after formatting, so:
size_t new_max_query_size = max_query_size > 0 ? (1000 + 2 * max_query_size) : 0;
@ -821,7 +827,13 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
chassert(ast2);
String formatted2 = ast2->formatWithPossiblyHidingSensitiveData(0, true, true, false, false, IdentifierQuotingStyle::Backticks);
String formatted2 = ast2->formatWithPossiblyHidingSensitiveData(
/*max_length=*/0,
/*one_line=*/true,
/*show_secrets=*/true,
/*print_pretty_type_names=*/false,
/*identifier_quoting_rule=*/IdentifierQuotingRule::WhenNecessary,
/*identifier_quoting_style=*/IdentifierQuotingStyle::Backticks);
if (formatted1 != formatted2)
throw Exception(ErrorCodes::LOGICAL_ERROR,

View File

@ -30,7 +30,7 @@ inline String format(const SecretHidingFormatSettings & settings)
settings.one_line,
show_secrets,
settings.ctx->getSettingsRef().print_pretty_type_names,
settings.ctx->getSettingsRef().output_format_always_quote_identifiers,
settings.ctx->getSettingsRef().output_format_identifier_quoting_rule,
settings.ctx->getSettingsRef().output_format_identifier_quoting_style);
}

View File

@ -30,7 +30,7 @@ protected:
{
settings.ostr << (settings.hilite ? hilite_identifier : "");
settings.writeIdentifier(column_name);
settings.writeIdentifier(column_name, /*ambiguous=*/false);
settings.ostr << (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_operator : "") << " = " << (settings.hilite ? hilite_none : "");

View File

@ -66,8 +66,7 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & format_settings, Fo
{
frame.need_parens = false;
/// We have to always quote column names to avoid ambiguity with INDEX and other declarations in CREATE query.
format_settings.quoteIdentifier(name);
format_settings.writeIdentifier(name, /*ambiguous=*/true);
if (type)
{

View File

@ -35,7 +35,7 @@ void ASTDictionaryAttributeDeclaration::formatImpl(const FormatSettings & settin
{
frame.need_parens = false;
settings.quoteIdentifier(name);
settings.writeIdentifier(name, /*ambiguous=*/true);
if (type)
{

View File

@ -108,7 +108,7 @@ void ASTIdentifier::formatImplWithoutAlias(const FormatSettings & settings, Form
auto format_element = [&](const String & elem_name)
{
settings.ostr << (settings.hilite ? hilite_identifier : "");
settings.writeIdentifier(elem_name);
settings.writeIdentifier(elem_name, /*ambiguous=*/false);
settings.ostr << (settings.hilite ? hilite_none : "");
};

View File

@ -79,7 +79,7 @@ void ASTIndexDeclaration::formatImpl(const FormatSettings & s, FormatState & sta
}
else
{
s.writeIdentifier(name);
s.writeIdentifier(name, /*ambiguous=*/false);
s.ostr << " ";
expr->formatImpl(s, state, frame);
}

View File

@ -17,7 +17,7 @@ ASTPtr ASTProjectionDeclaration::clone() const
void ASTProjectionDeclaration::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.writeIdentifier(name);
settings.writeIdentifier(name, /*ambiguous=*/false);
std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' ');
std::string nl_or_nothing = settings.one_line ? "" : "\n";
settings.ostr << settings.nl_or_ws << indent_str << "(" << nl_or_nothing;

View File

@ -35,7 +35,7 @@ void ASTSubquery::formatImplWithoutAlias(const FormatSettings & settings, Format
if (!cte_name.empty())
{
settings.ostr << (settings.hilite ? hilite_identifier : "");
settings.writeIdentifier(cte_name);
settings.writeIdentifier(cte_name, /*ambiguous=*/false);
settings.ostr << (settings.hilite ? hilite_none : "");
return;
}

View File

@ -10,7 +10,7 @@ namespace DB
static void writeAlias(const String & name, const ASTWithAlias::FormatSettings & settings)
{
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " AS " << (settings.hilite ? IAST::hilite_alias : "");
settings.writeIdentifier(name);
settings.writeIdentifier(name, /*ambiguous=*/false);
settings.ostr << (settings.hilite ? IAST::hilite_none : "");
}
@ -22,7 +22,7 @@ void ASTWithAlias::formatImpl(const FormatSettings & settings, FormatState & sta
if (!alias.empty() && !state.printed_asts_with_alias.emplace(frame.current_select, alias, getTreeHash(/*ignore_aliases=*/ true)).second)
{
settings.ostr << (settings.hilite ? IAST::hilite_identifier : "");
settings.writeIdentifier(alias);
settings.writeIdentifier(alias, /*ambiguous=*/false);
settings.ostr << (settings.hilite ? IAST::hilite_none : "");
}
else

View File

@ -19,7 +19,7 @@ void ASTWithElement::formatImpl(const FormatSettings & settings, FormatState & s
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
settings.ostr << (settings.hilite ? hilite_alias : "");
settings.writeIdentifier(name);
settings.writeIdentifier(name, /*ambiguous=*/false);
settings.ostr << (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS" << (settings.hilite ? hilite_none : "");
settings.ostr << settings.nl_or_ws << indent_str;

View File

@ -1,11 +1,15 @@
#include <Parsers/IAST.h>
#include <IO/Operators.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/IdentifierQuotingStyle.h>
#include <Poco/String.h>
#include <Common/SensitiveDataMasker.h>
#include <Common/SipHash.h>
#include <algorithm>
namespace DB
{
@ -170,7 +174,7 @@ String IAST::formatWithPossiblyHidingSensitiveData(
bool one_line,
bool show_secrets,
bool print_pretty_type_names,
bool always_quote_identifiers,
IdentifierQuotingRule identifier_quoting_rule,
IdentifierQuotingStyle identifier_quoting_style) const
{
@ -178,7 +182,7 @@ String IAST::formatWithPossiblyHidingSensitiveData(
FormatSettings settings(buf, one_line);
settings.show_secrets = show_secrets;
settings.print_pretty_type_names = print_pretty_type_names;
settings.always_quote_identifiers = always_quote_identifiers;
settings.identifier_quoting_rule = identifier_quoting_rule;
settings.identifier_quoting_style = identifier_quoting_style;
format(settings);
return wipeSensitiveDataAndCutToLength(buf.str(), max_length);
@ -217,22 +221,35 @@ String IAST::getColumnNameWithoutAlias() const
}
void IAST::FormatSettings::writeIdentifier(const String & name) const
void IAST::FormatSettings::writeIdentifier(const String & name, bool ambiguous) const
{
if (!ambiguous)
{
// Identifier is ambiguous if it is one of the defined keywords, set `ambiguous`
const auto & keywords = getAllKeyWords();
ambiguous = std::find(keywords.begin(), keywords.end(), Poco::toUpper(name)) != keywords.end();
}
bool must_quote
= (identifier_quoting_rule == IdentifierQuotingRule::Always
|| (ambiguous && identifier_quoting_rule == IdentifierQuotingRule::WhenNecessary));
switch (identifier_quoting_style)
{
case IdentifierQuotingStyle::None:
{
if (always_quote_identifiers)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Incompatible arguments: always_quote_identifiers = true && "
"identifier_quoting_style == IdentifierQuotingStyle::None");
if (must_quote)
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Incompatible arguments: identifier_quoting_rule = {} && identifier_quoting_style == {} && ambiguous = {}",
identifier_quoting_rule,
identifier_quoting_style,
ambiguous);
writeString(name, ostr);
break;
}
case IdentifierQuotingStyle::Backticks:
{
if (always_quote_identifiers)
if (must_quote)
writeBackQuotedString(name, ostr);
else
writeProbablyBackQuotedString(name, ostr);
@ -240,7 +257,7 @@ void IAST::FormatSettings::writeIdentifier(const String & name) const
}
case IdentifierQuotingStyle::DoubleQuotes:
{
if (always_quote_identifiers)
if (must_quote)
writeDoubleQuotedString(name, ostr);
else
writeProbablyDoubleQuotedString(name, ostr);
@ -248,7 +265,7 @@ void IAST::FormatSettings::writeIdentifier(const String & name) const
}
case IdentifierQuotingStyle::BackticksMySQL:
{
if (always_quote_identifiers)
if (must_quote)
writeBackQuotedStringMySQL(name, ostr);
else
writeProbablyBackQuotedStringMySQL(name, ostr);
@ -257,34 +274,6 @@ void IAST::FormatSettings::writeIdentifier(const String & name) const
}
}
void IAST::FormatSettings::quoteIdentifier(const String & name) const
{
switch (identifier_quoting_style)
{
case IdentifierQuotingStyle::None:
{
writeBackQuotedString(name, ostr);
break;
}
case IdentifierQuotingStyle::Backticks:
{
writeBackQuotedString(name, ostr);
break;
}
case IdentifierQuotingStyle::DoubleQuotes:
{
writeDoubleQuotedString(name, ostr);
break;
}
case IdentifierQuotingStyle::BackticksMySQL:
{
writeBackQuotedStringMySQL(name, ostr);
break;
}
}
}
void IAST::dumpTree(WriteBuffer & ostr, size_t indent) const
{
String indent_str(indent, '-');

View File

@ -196,7 +196,7 @@ public:
WriteBuffer & ostr;
bool one_line;
bool hilite;
bool always_quote_identifiers;
IdentifierQuotingRule identifier_quoting_rule;
IdentifierQuotingStyle identifier_quoting_style;
bool show_secrets; /// Show secret parts of the AST (e.g. passwords, encryption keys).
char nl_or_ws; /// Newline or whitespace.
@ -207,7 +207,7 @@ public:
WriteBuffer & ostr_,
bool one_line_,
bool hilite_ = false,
bool always_quote_identifiers_ = false,
IdentifierQuotingRule identifier_quoting_rule_ = IdentifierQuotingRule::WhenNecessary,
IdentifierQuotingStyle identifier_quoting_style_ = IdentifierQuotingStyle::Backticks,
bool show_secrets_ = true,
LiteralEscapingStyle literal_escaping_style_ = LiteralEscapingStyle::Regular,
@ -215,7 +215,7 @@ public:
: ostr(ostr_)
, one_line(one_line_)
, hilite(hilite_)
, always_quote_identifiers(always_quote_identifiers_)
, identifier_quoting_rule(identifier_quoting_rule_)
, identifier_quoting_style(identifier_quoting_style_)
, show_secrets(show_secrets_)
, nl_or_ws(one_line ? ' ' : '\n')
@ -228,7 +228,7 @@ public:
: ostr(ostr_)
, one_line(other.one_line)
, hilite(other.hilite)
, always_quote_identifiers(other.always_quote_identifiers)
, identifier_quoting_rule(other.identifier_quoting_rule)
, identifier_quoting_style(other.identifier_quoting_style)
, show_secrets(other.show_secrets)
, nl_or_ws(other.nl_or_ws)
@ -237,10 +237,7 @@ public:
{
}
void writeIdentifier(const String & name) const;
// Quote identifier `name` even when `always_quote_identifiers` is false.
// If `identifier_quoting_style` is `IdentifierQuotingStyle::None`, quote it with `IdentifierQuotingStyle::Backticks`
void quoteIdentifier(const String & name) const;
void writeIdentifier(const String & name, bool ambiguous) const;
};
/// State. For example, a set of nodes can be remembered, which we already walk through.
@ -286,7 +283,7 @@ public:
bool one_line,
bool show_secrets,
bool print_pretty_type_names,
bool always_quote_identifiers,
IdentifierQuotingRule identifier_quoting_rule,
IdentifierQuotingStyle identifier_quoting_style) const;
/** formatForLogging and formatForErrorMessage always hide secrets. This inconsistent
@ -296,12 +293,24 @@ public:
*/
String formatForLogging(size_t max_length = 0) const
{
return formatWithPossiblyHidingSensitiveData(max_length, true, false, false, false, IdentifierQuotingStyle::Backticks);
return formatWithPossiblyHidingSensitiveData(
/*max_length=*/max_length,
/*one_line=*/true,
/*show_secrets=*/false,
/*print_pretty_type_names=*/false,
/*identifier_quoting_rule=*/IdentifierQuotingRule::WhenNecessary,
/*identifier_quoting_style=*/IdentifierQuotingStyle::Backticks);
}
String formatForErrorMessage() const
{
return formatWithPossiblyHidingSensitiveData(0, true, false, false, false, IdentifierQuotingStyle::Backticks);
return formatWithPossiblyHidingSensitiveData(
/*max_length=*/0,
/*one_line=*/true,
/*show_secrets=*/false,
/*print_pretty_type_names=*/false,
/*identifier_quoting_rule=*/IdentifierQuotingRule::WhenNecessary,
/*identifier_quoting_style=*/IdentifierQuotingStyle::Backticks);
}
virtual bool hasSecretParts() const { return childrenHaveSecretParts(); }

View File

@ -14,4 +14,10 @@ enum class IdentifierQuotingStyle : uint8_t
BackticksMySQL, /// `mysql` style, most same as Backticks, but it uses '``' to escape '`'
};
enum class IdentifierQuotingRule : uint8_t
{
WhenNecessary, /// When the identifiers is one of the certain keywords defined in `writeProbablyQuotedStringImpl`, and ambiguous identifiers passed to `writeIdentifier`
Always, /// Always quote identifiers
UserDisplay, /// When the identifiers is one of the certain keywords defined in `writeProbablyQuotedStringImpl`
};
}

View File

@ -19,7 +19,12 @@ std::string getInsertQuery(const std::string & db_name, const std::string & tabl
query.columns->children.emplace_back(std::make_shared<ASTIdentifier>(column.name));
WriteBufferFromOwnString buf;
IAST::FormatSettings settings(buf, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers*/ true, /*identifier_quoting_style*/ quoting);
IAST::FormatSettings settings(
/*ostr_=*/buf,
/*one_line=*/true,
/*hilite=*/false,
/*identifier_quoting_rule=*/IdentifierQuotingRule::WhenNecessary,
/*identifier_quoting_style=*/quoting);
query.IAST::format(settings);
return buf.str();
}

View File

@ -94,7 +94,8 @@ static String formattedAST(const ASTPtr & ast)
return {};
WriteBufferFromOwnString buf;
IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers*/ true);
IAST::FormatSettings ast_format_settings(
/*ostr_=*/buf, /*one_line=*/true, /*hilite=*/false, /*identifier_quoting_rule=*/IdentifierQuotingRule::Always);
ast->format(ast_format_settings);
return buf.str();
}

View File

@ -23,15 +23,15 @@
#include <Columns/ColumnConst.h>
#include <Common/threadPoolCallbackRunner.h>
#include <Common/CurrentMetrics.h>
#include <Common/Macros.h>
#include <Common/ProfileEvents.h>
#include <Common/escapeForFileName.h>
#include <Common/typeid_cast.h>
#include <Common/formatReadable.h>
#include <Common/quoteString.h>
#include <Common/randomSeed.h>
#include <Common/formatReadable.h>
#include <Common/CurrentMetrics.h>
#include <Common/threadPoolCallbackRunner.h>
#include <Common/typeid_cast.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
@ -40,8 +40,9 @@
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/parseQuery.h>
#include <Parsers/IAST.h>
#include <Parsers/IdentifierQuotingStyle.h>
#include <Parsers/parseQuery.h>
#include <Analyzer/ColumnNode.h>
#include <Analyzer/FunctionNode.h>
@ -982,7 +983,8 @@ std::optional<QueryPipeline> StorageDistributed::distributedWriteBetweenDistribu
String new_query_str;
{
WriteBufferFromOwnString buf;
IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers_=*/ true);
IAST::FormatSettings ast_format_settings(
/*ostr_=*/buf, /*one_line*/ true, /*hilite*/ false, /*identifier_quoting_rule_=*/IdentifierQuotingRule::Always);
new_query->IAST::format(ast_format_settings);
new_query_str = buf.str();
}
@ -1101,7 +1103,8 @@ std::optional<QueryPipeline> StorageDistributed::distributedWriteFromClusterStor
String new_query_str;
{
WriteBufferFromOwnString buf;
IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers*/ true);
IAST::FormatSettings ast_format_settings(
/*ostr_=*/buf, /*one_line=*/true, /*hilite=*/false, /*identifier_quoting_rule=*/IdentifierQuotingRule::Always);
new_query->IAST::format(ast_format_settings);
new_query_str = buf.str();
}

View File

@ -5690,7 +5690,8 @@ std::optional<QueryPipeline> StorageReplicatedMergeTree::distributedWriteFromClu
String query_str;
{
WriteBufferFromOwnString buf;
IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers*/ true);
IAST::FormatSettings ast_format_settings(
/*ostr_=*/buf, /*one_line=*/true, /*hilite=*/false, /*identifier_quoting_rule=*/IdentifierQuotingRule::Always);
query.IAST::format(ast_format_settings);
query_str = buf.str();
}

View File

@ -381,13 +381,20 @@ String transformQueryForExternalDatabaseImpl(
ASTPtr select_ptr = select;
dropAliases(select_ptr);
IdentifierQuotingRule identifier_quoting_rule = IdentifierQuotingRule::WhenNecessary;
if (identifier_quoting_style != IdentifierQuotingStyle::None)
{
identifier_quoting_rule = IdentifierQuotingRule::Always;
}
WriteBufferFromOwnString out;
IAST::FormatSettings settings(
out, /*one_line*/ true, /*hilite*/ false,
/*always_quote_identifiers*/ identifier_quoting_style != IdentifierQuotingStyle::None,
/*identifier_quoting_style*/ identifier_quoting_style, /*show_secrets_*/ true,
/*literal_escaping_style*/ literal_escaping_style);
/*ostr_=*/out,
/*one_line=*/true,
/*hilite=*/false,
/*identifier_quoting_rule=*/identifier_quoting_rule,
/*identifier_quoting_style=*/identifier_quoting_style,
/*show_secrets_=*/true,
/*literal_escaping_style=*/literal_escaping_style);
select->format(settings);

View File

@ -1,40 +1,67 @@
CREATE TABLE default.uk_price_paid\n(\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n date,\n price\n ORDER BY \n town,\n date\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, date)\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n price,\n date,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid
CREATE TABLE default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY month\nORDER BY month\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(date) AS month\nSELECT\n month,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY month
CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY date\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED())
CREATE TABLE default.uk_price_paid\n(\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n date,\n price\n ORDER BY \n town,\n date\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, date)\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n price,\n date,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid
CREATE TABLE default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY month\nORDER BY month\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(date) AS month\nSELECT\n month,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY month
CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY date\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED())
CREATE TABLE `default`.`uk_price_paid`\n(\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX `county_index` `county` TYPE set(10) GRANULARITY 1,\n PROJECTION `town_date_projection`\n (\n SELECT \n `town`,\n `date`,\n `price`\n ORDER BY \n `town`,\n `date`\n ),\n PROJECTION `handy_aggs_projection`\n (\n SELECT \n avg(`price`),\n max(`price`),\n sum(`price`)\n GROUP BY `town`\n )\n)\nENGINE = MergeTree\nORDER BY (`postcode1`, `postcode2`, `date`)\nSETTINGS index_granularity = 8192
Settings: default
CREATE TABLE default.uk_price_paid\n(\n `Table` String,\n `Engine` String,\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n `date`,\n price\n ORDER BY \n town,\n `date`\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, `date`)\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n price,\n `date`,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid
CREATE TABLE default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(`min`, UInt32),\n `max_price` SimpleAggregateFunction(`max`, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `month`\nORDER BY `month`\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(`min`, UInt32),\n `max_price` SimpleAggregateFunction(`max`, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(`date`) AS `month`\nSELECT\n `month`,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY `month`
CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY `date`\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED())
Settings: when_necessary & None
Settings: user_display & None
CREATE TABLE default.uk_price_paid\n(\n Table String,\n Engine String,\n price UInt32,\n date Date,\n postcode1 LowCardinality(String),\n postcode2 LowCardinality(String),\n type Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n is_new UInt8,\n duration Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n addr1 String,\n addr2 String,\n street LowCardinality(String),\n locality LowCardinality(String),\n town LowCardinality(String),\n district LowCardinality(String),\n county LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n date,\n price\n ORDER BY \n town,\n date\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, date)\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n price UInt32,\n date Date,\n addr1 String,\n addr2 String,\n street LowCardinality(String),\n town LowCardinality(String),\n district LowCardinality(String),\n county LowCardinality(String)\n)\nAS SELECT\n price,\n date,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid
CREATE TABLE default.uk_prices_aggs_dest\n(\n month Date,\n min_price SimpleAggregateFunction(min, UInt32),\n max_price SimpleAggregateFunction(max, UInt32),\n volume AggregateFunction(count, UInt32),\n avg_price AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY month\nORDER BY month\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n month Date,\n min_price SimpleAggregateFunction(min, UInt32),\n max_price SimpleAggregateFunction(max, UInt32),\n volume AggregateFunction(count, UInt32),\n avg_price AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(date) AS month\nSELECT\n month,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY month
CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n date DateTime64,\n variable Decimal32(2),\n fixed Decimal32(2),\n bank Decimal32(2)\n)\nPRIMARY KEY date\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED())
Settings: always & Backticks
CREATE TABLE `default`.`uk_price_paid`\n(\n `Table` String,\n `Engine` String,\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX `county_index` `county` TYPE set(10) GRANULARITY 1,\n PROJECTION `town_date_projection`\n (\n SELECT \n `town`,\n `date`,\n `price`\n ORDER BY \n `town`,\n `date`\n ),\n PROJECTION `handy_aggs_projection`\n (\n SELECT \n avg(`price`),\n max(`price`),\n sum(`price`)\n GROUP BY `town`\n )\n)\nENGINE = MergeTree\nORDER BY (`postcode1`, `postcode2`, `date`)\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW `default`.`prices_by_year_view` TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n `price`,\n `date`,\n `addr1`,\n `addr2`,\n `street`,\n `town`,\n `district`,\n `county`\nFROM `default`.`uk_price_paid`
CREATE TABLE `default`.`uk_prices_aggs_dest`\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(`min`, UInt32),\n `max_price` SimpleAggregateFunction(`max`, UInt32),\n `volume` AggregateFunction(`count`, UInt32),\n `avg_price` AggregateFunction(`avg`, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `month`\nORDER BY `month`\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW `default`.`uk_prices_aggs_view` TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(`min`, UInt32),\n `max_price` SimpleAggregateFunction(`max`, UInt32),\n `volume` AggregateFunction(`count`, UInt32),\n `avg_price` AggregateFunction(`avg`, UInt32)\n)\nAS WITH toStartOfMonth(`date`) AS `month`\nSELECT\n `month`,\n minSimpleState(`price`) AS `min_price`,\n maxSimpleState(`price`) AS `max_price`,\n countState(`price`) AS `volume`,\n avgState(`price`) AS `avg_price`\nFROM `default`.`uk_price_paid`\nGROUP BY `month`
CREATE DICTIONARY `default`.`uk_mortgage_rates_dict`\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY `date`\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED())
CREATE TABLE default.uk_price_paid\n(\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n date,\n price\n ORDER BY \n town,\n date\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, date)\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n price,\n date,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid
CREATE TABLE default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY month\nORDER BY month\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(date) AS month\nSELECT\n month,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY month
CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY date\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED())
CREATE TABLE "default"."uk_price_paid"\n(\n "price" UInt32,\n "date" Date,\n "postcode1" LowCardinality(String),\n "postcode2" LowCardinality(String),\n "type" Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n "is_new" UInt8,\n "duration" Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n "addr1" String,\n "addr2" String,\n "street" LowCardinality(String),\n "locality" LowCardinality(String),\n "town" LowCardinality(String),\n "district" LowCardinality(String),\n "county" LowCardinality(String),\n INDEX "county_index" "county" TYPE set(10) GRANULARITY 1,\n PROJECTION "town_date_projection"\n (\n SELECT \n "town",\n "date",\n "price"\n ORDER BY \n "town",\n "date"\n ),\n PROJECTION "handy_aggs_projection"\n (\n SELECT \n avg("price"),\n max("price"),\n sum("price")\n GROUP BY "town"\n )\n)\nENGINE = MergeTree\nORDER BY ("postcode1", "postcode2", "date")\nSETTINGS index_granularity = 8192
Settings: user_display & Backticks
CREATE TABLE default.uk_price_paid\n(\n `Table` String,\n Engine String,\n price UInt32,\n date Date,\n postcode1 LowCardinality(String),\n postcode2 LowCardinality(String),\n type Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n is_new UInt8,\n duration Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n addr1 String,\n addr2 String,\n street LowCardinality(String),\n locality LowCardinality(String),\n town LowCardinality(String),\n district LowCardinality(String),\n county LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n date,\n price\n ORDER BY \n town,\n date\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, date)\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n price UInt32,\n date Date,\n addr1 String,\n addr2 String,\n street LowCardinality(String),\n town LowCardinality(String),\n district LowCardinality(String),\n county LowCardinality(String)\n)\nAS SELECT\n price,\n date,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid
CREATE TABLE default.uk_prices_aggs_dest\n(\n month Date,\n min_price SimpleAggregateFunction(min, UInt32),\n max_price SimpleAggregateFunction(max, UInt32),\n volume AggregateFunction(count, UInt32),\n avg_price AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY month\nORDER BY month\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n month Date,\n min_price SimpleAggregateFunction(min, UInt32),\n max_price SimpleAggregateFunction(max, UInt32),\n volume AggregateFunction(count, UInt32),\n avg_price AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(date) AS month\nSELECT\n month,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY month
CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n date DateTime64,\n variable Decimal32(2),\n fixed Decimal32(2),\n bank Decimal32(2)\n)\nPRIMARY KEY date\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED())
Settings: when_necessary & Backticks
CREATE TABLE default.uk_price_paid\n(\n `Table` String,\n `Engine` String,\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n `date`,\n price\n ORDER BY \n town,\n `date`\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, `date`)\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n price,\n `date`,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid
CREATE TABLE default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(`min`, UInt32),\n `max_price` SimpleAggregateFunction(`max`, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `month`\nORDER BY `month`\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(`min`, UInt32),\n `max_price` SimpleAggregateFunction(`max`, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(`date`) AS `month`\nSELECT\n `month`,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY `month`
CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY `date`\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED())
Settings: always & DoubleQuotes
CREATE TABLE "default"."uk_price_paid"\n(\n "Table" String,\n "Engine" String,\n "price" UInt32,\n "date" Date,\n "postcode1" LowCardinality(String),\n "postcode2" LowCardinality(String),\n "type" Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n "is_new" UInt8,\n "duration" Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n "addr1" String,\n "addr2" String,\n "street" LowCardinality(String),\n "locality" LowCardinality(String),\n "town" LowCardinality(String),\n "district" LowCardinality(String),\n "county" LowCardinality(String),\n INDEX "county_index" "county" TYPE set(10) GRANULARITY 1,\n PROJECTION "town_date_projection"\n (\n SELECT \n "town",\n "date",\n "price"\n ORDER BY \n "town",\n "date"\n ),\n PROJECTION "handy_aggs_projection"\n (\n SELECT \n avg("price"),\n max("price"),\n sum("price")\n GROUP BY "town"\n )\n)\nENGINE = MergeTree\nORDER BY ("postcode1", "postcode2", "date")\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW "default"."prices_by_year_view" TO default.prices_by_year_dest\n(\n "price" UInt32,\n "date" Date,\n "addr1" String,\n "addr2" String,\n "street" LowCardinality(String),\n "town" LowCardinality(String),\n "district" LowCardinality(String),\n "county" LowCardinality(String)\n)\nAS SELECT\n "price",\n "date",\n "addr1",\n "addr2",\n "street",\n "town",\n "district",\n "county"\nFROM "default"."uk_price_paid"
CREATE TABLE "default"."uk_prices_aggs_dest"\n(\n "month" Date,\n "min_price" SimpleAggregateFunction("min", UInt32),\n "max_price" SimpleAggregateFunction("max", UInt32),\n "volume" AggregateFunction("count", UInt32),\n "avg_price" AggregateFunction("avg", UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY "month"\nORDER BY "month"\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW "default"."uk_prices_aggs_view" TO default.uk_prices_aggs_dest\n(\n "month" Date,\n "min_price" SimpleAggregateFunction("min", UInt32),\n "max_price" SimpleAggregateFunction("max", UInt32),\n "volume" AggregateFunction("count", UInt32),\n "avg_price" AggregateFunction("avg", UInt32)\n)\nAS WITH toStartOfMonth("date") AS "month"\nSELECT\n "month",\n minSimpleState("price") AS "min_price",\n maxSimpleState("price") AS "max_price",\n countState("price") AS "volume",\n avgState("price") AS "avg_price"\nFROM "default"."uk_price_paid"\nGROUP BY "month"
CREATE DICTIONARY "default"."uk_mortgage_rates_dict"\n(\n "date" DateTime64,\n "variable" Decimal32(2),\n "fixed" Decimal32(2),\n "bank" Decimal32(2)\n)\nPRIMARY KEY "date"\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED())
CREATE TABLE default.uk_price_paid\n(\n "price" UInt32,\n "date" Date,\n "postcode1" LowCardinality(String),\n "postcode2" LowCardinality(String),\n "type" Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n "is_new" UInt8,\n "duration" Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n "addr1" String,\n "addr2" String,\n "street" LowCardinality(String),\n "locality" LowCardinality(String),\n "town" LowCardinality(String),\n "district" LowCardinality(String),\n "county" LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n date,\n price\n ORDER BY \n town,\n date\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, date)\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n "price" UInt32,\n "date" Date,\n "addr1" String,\n "addr2" String,\n "street" LowCardinality(String),\n "town" LowCardinality(String),\n "district" LowCardinality(String),\n "county" LowCardinality(String)\n)\nAS SELECT\n price,\n date,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid
CREATE TABLE default.uk_prices_aggs_dest\n(\n "month" Date,\n "min_price" SimpleAggregateFunction(min, UInt32),\n "max_price" SimpleAggregateFunction(max, UInt32),\n "volume" AggregateFunction(count, UInt32),\n "avg_price" AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY month\nORDER BY month\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n "month" Date,\n "min_price" SimpleAggregateFunction(min, UInt32),\n "max_price" SimpleAggregateFunction(max, UInt32),\n "volume" AggregateFunction(count, UInt32),\n "avg_price" AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(date) AS month\nSELECT\n month,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY month
CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n "date" DateTime64,\n "variable" Decimal32(2),\n "fixed" Decimal32(2),\n "bank" Decimal32(2)\n)\nPRIMARY KEY date\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED())
CREATE TABLE `default`.`uk_price_paid`\n(\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX `county_index` `county` TYPE set(10) GRANULARITY 1,\n PROJECTION `town_date_projection`\n (\n SELECT \n `town`,\n `date`,\n `price`\n ORDER BY \n `town`,\n `date`\n ),\n PROJECTION `handy_aggs_projection`\n (\n SELECT \n avg(`price`),\n max(`price`),\n sum(`price`)\n GROUP BY `town`\n )\n)\nENGINE = MergeTree\nORDER BY (`postcode1`, `postcode2`, `date`)\nSETTINGS index_granularity = 8192
Settings: user_display & DoubleQuotes
CREATE TABLE default.uk_price_paid\n(\n "Table" String,\n Engine String,\n price UInt32,\n date Date,\n postcode1 LowCardinality(String),\n postcode2 LowCardinality(String),\n type Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n is_new UInt8,\n duration Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n addr1 String,\n addr2 String,\n street LowCardinality(String),\n locality LowCardinality(String),\n town LowCardinality(String),\n district LowCardinality(String),\n county LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n date,\n price\n ORDER BY \n town,\n date\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, date)\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n price UInt32,\n date Date,\n addr1 String,\n addr2 String,\n street LowCardinality(String),\n town LowCardinality(String),\n district LowCardinality(String),\n county LowCardinality(String)\n)\nAS SELECT\n price,\n date,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid
CREATE TABLE default.uk_prices_aggs_dest\n(\n month Date,\n min_price SimpleAggregateFunction(min, UInt32),\n max_price SimpleAggregateFunction(max, UInt32),\n volume AggregateFunction(count, UInt32),\n avg_price AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY month\nORDER BY month\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n month Date,\n min_price SimpleAggregateFunction(min, UInt32),\n max_price SimpleAggregateFunction(max, UInt32),\n volume AggregateFunction(count, UInt32),\n avg_price AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(date) AS month\nSELECT\n month,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY month
CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n date DateTime64,\n variable Decimal32(2),\n fixed Decimal32(2),\n bank Decimal32(2)\n)\nPRIMARY KEY date\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED())
Settings: when_necessary & DoubleQuotes
CREATE TABLE default.uk_price_paid\n(\n "Table" String,\n "Engine" String,\n "price" UInt32,\n "date" Date,\n "postcode1" LowCardinality(String),\n "postcode2" LowCardinality(String),\n "type" Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n "is_new" UInt8,\n "duration" Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n "addr1" String,\n "addr2" String,\n "street" LowCardinality(String),\n "locality" LowCardinality(String),\n "town" LowCardinality(String),\n "district" LowCardinality(String),\n "county" LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n "date",\n price\n ORDER BY \n town,\n "date"\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, "date")\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n "price" UInt32,\n "date" Date,\n "addr1" String,\n "addr2" String,\n "street" LowCardinality(String),\n "town" LowCardinality(String),\n "district" LowCardinality(String),\n "county" LowCardinality(String)\n)\nAS SELECT\n price,\n "date",\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid
CREATE TABLE default.uk_prices_aggs_dest\n(\n "month" Date,\n "min_price" SimpleAggregateFunction("min", UInt32),\n "max_price" SimpleAggregateFunction("max", UInt32),\n "volume" AggregateFunction(count, UInt32),\n "avg_price" AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY "month"\nORDER BY "month"\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n "month" Date,\n "min_price" SimpleAggregateFunction("min", UInt32),\n "max_price" SimpleAggregateFunction("max", UInt32),\n "volume" AggregateFunction(count, UInt32),\n "avg_price" AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth("date") AS "month"\nSELECT\n "month",\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY "month"
CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n "date" DateTime64,\n "variable" Decimal32(2),\n "fixed" Decimal32(2),\n "bank" Decimal32(2)\n)\nPRIMARY KEY "date"\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED())
Settings: always & BackticksMySQL
CREATE TABLE `default`.`uk_price_paid`\n(\n `Table` String,\n `Engine` String,\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX `county_index` `county` TYPE set(10) GRANULARITY 1,\n PROJECTION `town_date_projection`\n (\n SELECT \n `town`,\n `date`,\n `price`\n ORDER BY \n `town`,\n `date`\n ),\n PROJECTION `handy_aggs_projection`\n (\n SELECT \n avg(`price`),\n max(`price`),\n sum(`price`)\n GROUP BY `town`\n )\n)\nENGINE = MergeTree\nORDER BY (`postcode1`, `postcode2`, `date`)\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW `default`.`prices_by_year_view` TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n `price`,\n `date`,\n `addr1`,\n `addr2`,\n `street`,\n `town`,\n `district`,\n `county`\nFROM `default`.`uk_price_paid`
CREATE TABLE `default`.`uk_prices_aggs_dest`\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(`min`, UInt32),\n `max_price` SimpleAggregateFunction(`max`, UInt32),\n `volume` AggregateFunction(`count`, UInt32),\n `avg_price` AggregateFunction(`avg`, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `month`\nORDER BY `month`\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW `default`.`uk_prices_aggs_view` TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(`min`, UInt32),\n `max_price` SimpleAggregateFunction(`max`, UInt32),\n `volume` AggregateFunction(`count`, UInt32),\n `avg_price` AggregateFunction(`avg`, UInt32)\n)\nAS WITH toStartOfMonth(`date`) AS `month`\nSELECT\n `month`,\n minSimpleState(`price`) AS `min_price`,\n maxSimpleState(`price`) AS `max_price`,\n countState(`price`) AS `volume`,\n avgState(`price`) AS `avg_price`\nFROM `default`.`uk_price_paid`\nGROUP BY `month`
CREATE DICTIONARY `default`.`uk_mortgage_rates_dict`\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY `date`\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED())
CREATE TABLE default.uk_price_paid\n(\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n date,\n price\n ORDER BY \n town,\n date\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, date)\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n price,\n date,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid
CREATE TABLE default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY month\nORDER BY month\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(date) AS month\nSELECT\n month,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY month
CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY date\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED())
Settings: user_display & BackticksMySQL
CREATE TABLE default.uk_price_paid\n(\n `Table` String,\n Engine String,\n price UInt32,\n date Date,\n postcode1 LowCardinality(String),\n postcode2 LowCardinality(String),\n type Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n is_new UInt8,\n duration Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n addr1 String,\n addr2 String,\n street LowCardinality(String),\n locality LowCardinality(String),\n town LowCardinality(String),\n district LowCardinality(String),\n county LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n date,\n price\n ORDER BY \n town,\n date\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, date)\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n price UInt32,\n date Date,\n addr1 String,\n addr2 String,\n street LowCardinality(String),\n town LowCardinality(String),\n district LowCardinality(String),\n county LowCardinality(String)\n)\nAS SELECT\n price,\n date,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid
CREATE TABLE default.uk_prices_aggs_dest\n(\n month Date,\n min_price SimpleAggregateFunction(min, UInt32),\n max_price SimpleAggregateFunction(max, UInt32),\n volume AggregateFunction(count, UInt32),\n avg_price AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY month\nORDER BY month\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n month Date,\n min_price SimpleAggregateFunction(min, UInt32),\n max_price SimpleAggregateFunction(max, UInt32),\n volume AggregateFunction(count, UInt32),\n avg_price AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(date) AS month\nSELECT\n month,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY month
CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n date DateTime64,\n variable Decimal32(2),\n fixed Decimal32(2),\n bank Decimal32(2)\n)\nPRIMARY KEY date\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED())
Settings: when_necessary & BackticksMySQL
CREATE TABLE default.uk_price_paid\n(\n `Table` String,\n `Engine` String,\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n `date`,\n price\n ORDER BY \n town,\n `date`\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, `date`)\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n price,\n `date`,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid
CREATE TABLE default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(`min`, UInt32),\n `max_price` SimpleAggregateFunction(`max`, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `month`\nORDER BY `month`\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(`min`, UInt32),\n `max_price` SimpleAggregateFunction(`max`, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(`date`) AS `month`\nSELECT\n `month`,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY `month`
CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY `date`\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED())

View File

@ -10,6 +10,8 @@ DROP TABLE IF EXISTS uk_price_paid;
CREATE TABLE uk_price_paid
(
Table String, -- column name is a keyword `table`
Engine String, -- column name is a keyword `ENGINE`
price UInt32,
date Date,
postcode1 LowCardinality(String),
@ -48,6 +50,8 @@ ENGINE = MergeTree
ORDER BY (postcode1, postcode2, date);
CREATE TABLE prices_by_year_dest (
Table String, -- column name is a keyword `table`
Engine String, -- column name is a keyword `ENGINE`
price UInt32,
date Date,
addr1 String,
@ -125,198 +129,309 @@ LIFETIME(2628000000);
-- Show tables, views, dictionaries with default settings
SELECT('Settings: default');
SHOW CREATE TABLE uk_price_paid;
SHOW CREATE VIEW prices_by_year_view;
SHOW CREATE uk_prices_aggs_dest;
SHOW CREATE VIEW uk_prices_aggs_view;
SHOW CREATE DICTIONARY uk_mortgage_rates_dict;
-- Show tables, views, dictionaries with output_format_always_quote_identifiers=false, output_format_identifier_quoting_style='None'
-- Show tables, views, dictionaries with output_format_identifier_quoting_rule='when_necessary', output_format_identifier_quoting_style='None'
SELECT('Settings: when_necessary & None');
SHOW CREATE TABLE uk_price_paid
SETTINGS
output_format_always_quote_identifiers=false,
output_format_identifier_quoting_rule='when_necessary',
output_format_identifier_quoting_style='None'; -- { serverError BAD_ARGUMENTS}
SHOW CREATE VIEW prices_by_year_view
SETTINGS
output_format_identifier_quoting_rule='when_necessary',
output_format_identifier_quoting_style='None'; -- { serverError BAD_ARGUMENTS}
SHOW CREATE uk_prices_aggs_dest
SETTINGS
output_format_identifier_quoting_rule='when_necessary',
output_format_identifier_quoting_style='None'; -- { serverError BAD_ARGUMENTS}
SHOW CREATE VIEW uk_prices_aggs_view
SETTINGS
output_format_identifier_quoting_rule='when_necessary',
output_format_identifier_quoting_style='None'; -- { serverError BAD_ARGUMENTS}
SHOW CREATE DICTIONARY uk_mortgage_rates_dict
SETTINGS
output_format_identifier_quoting_rule='when_necessary',
output_format_identifier_quoting_style='None'; -- { serverError BAD_ARGUMENTS}
-- Show tables, views, dictionaries with output_format_identifier_quoting_rule='user_display', output_format_identifier_quoting_style='None'
SELECT('Settings: user_display & None');
SHOW CREATE TABLE uk_price_paid
SETTINGS
output_format_identifier_quoting_rule='user_display',
output_format_identifier_quoting_style='None';
SHOW CREATE VIEW prices_by_year_view
SETTINGS
output_format_always_quote_identifiers=false,
output_format_identifier_quoting_rule='user_display',
output_format_identifier_quoting_style='None';
SHOW CREATE uk_prices_aggs_dest
SETTINGS
output_format_always_quote_identifiers=false,
output_format_identifier_quoting_rule='user_display',
output_format_identifier_quoting_style='None';
SHOW CREATE VIEW uk_prices_aggs_view
SETTINGS
output_format_always_quote_identifiers=false,
output_format_identifier_quoting_rule='user_display',
output_format_identifier_quoting_style='None';
SHOW CREATE DICTIONARY uk_mortgage_rates_dict
SETTINGS
output_format_always_quote_identifiers=false,
output_format_identifier_quoting_rule='user_display',
output_format_identifier_quoting_style='None';
-- Show tables, views, dictionaries with output_format_always_quote_identifiers=true, output_format_identifier_quoting_style='Backticks'
-- Show tables, views, dictionaries with output_format_identifier_quoting_rule='always', output_format_identifier_quoting_style='Backticks'
SELECT('Settings: always & Backticks');
SHOW CREATE TABLE uk_price_paid
SETTINGS
output_format_always_quote_identifiers=true,
output_format_identifier_quoting_rule='always',
output_format_identifier_quoting_style='Backticks';
SHOW CREATE VIEW prices_by_year_view
SETTINGS
output_format_always_quote_identifiers=true,
output_format_identifier_quoting_rule='always',
output_format_identifier_quoting_style='Backticks';
SHOW CREATE uk_prices_aggs_dest
SETTINGS
output_format_always_quote_identifiers=true,
output_format_identifier_quoting_rule='always',
output_format_identifier_quoting_style='Backticks';
SHOW CREATE VIEW uk_prices_aggs_view
SETTINGS
output_format_always_quote_identifiers=true,
output_format_identifier_quoting_rule='always',
output_format_identifier_quoting_style='Backticks';
SHOW CREATE DICTIONARY uk_mortgage_rates_dict
SETTINGS
output_format_always_quote_identifiers=true,
output_format_identifier_quoting_rule='always',
output_format_identifier_quoting_style='Backticks';
-- Show tables, views, dictionaries with output_format_always_quote_identifiers=false, output_format_identifier_quoting_style='Backticks'
-- Show tables, views, dictionaries with output_format_identifier_quoting_rule='user_display', output_format_identifier_quoting_style='Backticks'
SELECT('Settings: user_display & Backticks');
SHOW CREATE TABLE uk_price_paid
SETTINGS
output_format_always_quote_identifiers=false,
output_format_identifier_quoting_rule='user_display',
output_format_identifier_quoting_style='Backticks';
SHOW CREATE VIEW prices_by_year_view
SETTINGS
output_format_always_quote_identifiers=false,
output_format_identifier_quoting_rule='user_display',
output_format_identifier_quoting_style='Backticks';
SHOW CREATE uk_prices_aggs_dest
SETTINGS
output_format_always_quote_identifiers=false,
output_format_identifier_quoting_rule='user_display',
output_format_identifier_quoting_style='Backticks';
SHOW CREATE VIEW uk_prices_aggs_view
SETTINGS
output_format_always_quote_identifiers=false,
output_format_identifier_quoting_rule='user_display',
output_format_identifier_quoting_style='Backticks';
SHOW CREATE DICTIONARY uk_mortgage_rates_dict
SETTINGS
output_format_always_quote_identifiers=false,
output_format_identifier_quoting_rule='user_display',
output_format_identifier_quoting_style='Backticks';
-- Show tables, views, dictionaries with output_format_always_quote_identifiers=true, output_format_identifier_quoting_style='DoubleQuotes'
-- Show tables, views, dictionaries with output_format_identifier_quoting_rule='when_necessary', output_format_identifier_quoting_style='Backticks'
SELECT('Settings: when_necessary & Backticks');
SHOW CREATE TABLE uk_price_paid
SETTINGS
output_format_always_quote_identifiers=true,
output_format_identifier_quoting_rule='when_necessary',
output_format_identifier_quoting_style='Backticks';
SHOW CREATE VIEW prices_by_year_view
SETTINGS
output_format_identifier_quoting_rule='when_necessary',
output_format_identifier_quoting_style='Backticks';
SHOW CREATE uk_prices_aggs_dest
SETTINGS
output_format_identifier_quoting_rule='when_necessary',
output_format_identifier_quoting_style='Backticks';
SHOW CREATE VIEW uk_prices_aggs_view
SETTINGS
output_format_identifier_quoting_rule='when_necessary',
output_format_identifier_quoting_style='Backticks';
SHOW CREATE DICTIONARY uk_mortgage_rates_dict
SETTINGS
output_format_identifier_quoting_rule='when_necessary',
output_format_identifier_quoting_style='Backticks';
-- Show tables, views, dictionaries with output_format_identifier_quoting_rule='always', output_format_identifier_quoting_style='DoubleQuotes'
SELECT('Settings: always & DoubleQuotes');
SHOW CREATE TABLE uk_price_paid
SETTINGS
output_format_identifier_quoting_rule='always',
output_format_identifier_quoting_style='DoubleQuotes';
SHOW CREATE VIEW prices_by_year_view
SETTINGS
output_format_always_quote_identifiers=true,
output_format_identifier_quoting_rule='always',
output_format_identifier_quoting_style='DoubleQuotes';
SHOW CREATE uk_prices_aggs_dest
SETTINGS
output_format_always_quote_identifiers=true,
output_format_identifier_quoting_rule='always',
output_format_identifier_quoting_style='DoubleQuotes';
SHOW CREATE VIEW uk_prices_aggs_view
SETTINGS
output_format_always_quote_identifiers=true,
output_format_identifier_quoting_rule='always',
output_format_identifier_quoting_style='DoubleQuotes';
SHOW CREATE DICTIONARY uk_mortgage_rates_dict
SETTINGS
output_format_always_quote_identifiers=true,
output_format_identifier_quoting_rule='always',
output_format_identifier_quoting_style='DoubleQuotes';
-- Show tables, views, dictionaries with output_format_always_quote_identifiers=false, output_format_identifier_quoting_style='DoubleQuotes'
-- Show tables, views, dictionaries with output_format_identifier_quoting_rule='user_display', output_format_identifier_quoting_style='DoubleQuotes'
SELECT('Settings: user_display & DoubleQuotes');
SHOW CREATE TABLE uk_price_paid
SETTINGS
output_format_always_quote_identifiers=false,
output_format_identifier_quoting_rule='user_display',
output_format_identifier_quoting_style='DoubleQuotes';
SHOW CREATE VIEW prices_by_year_view
SETTINGS
output_format_always_quote_identifiers=false,
output_format_identifier_quoting_rule='user_display',
output_format_identifier_quoting_style='DoubleQuotes';
SHOW CREATE uk_prices_aggs_dest
SETTINGS
output_format_always_quote_identifiers=false,
output_format_identifier_quoting_rule='user_display',
output_format_identifier_quoting_style='DoubleQuotes';
SHOW CREATE VIEW uk_prices_aggs_view
SETTINGS
output_format_always_quote_identifiers=false,
output_format_identifier_quoting_rule='user_display',
output_format_identifier_quoting_style='DoubleQuotes';
SHOW CREATE DICTIONARY uk_mortgage_rates_dict
SETTINGS
output_format_always_quote_identifiers=false,
output_format_identifier_quoting_rule='user_display',
output_format_identifier_quoting_style='DoubleQuotes';
-- Show tables, views, dictionaries with output_format_always_quote_identifiers=true, output_format_identifier_quoting_style='BackticksMySQL'
-- Show tables, views, dictionaries with output_format_identifier_quoting_rule='when_necessary', output_format_identifier_quoting_style='DoubleQuotes'
SELECT('Settings: when_necessary & DoubleQuotes');
SHOW CREATE TABLE uk_price_paid
SETTINGS
output_format_always_quote_identifiers=true,
output_format_identifier_quoting_rule='when_necessary',
output_format_identifier_quoting_style='DoubleQuotes';
SHOW CREATE VIEW prices_by_year_view
SETTINGS
output_format_identifier_quoting_rule='when_necessary',
output_format_identifier_quoting_style='DoubleQuotes';
SHOW CREATE uk_prices_aggs_dest
SETTINGS
output_format_identifier_quoting_rule='when_necessary',
output_format_identifier_quoting_style='DoubleQuotes';
SHOW CREATE VIEW uk_prices_aggs_view
SETTINGS
output_format_identifier_quoting_rule='when_necessary',
output_format_identifier_quoting_style='DoubleQuotes';
SHOW CREATE DICTIONARY uk_mortgage_rates_dict
SETTINGS
output_format_identifier_quoting_rule='when_necessary',
output_format_identifier_quoting_style='DoubleQuotes';
-- Show tables, views, dictionaries with output_format_identifier_quoting_rule='always', output_format_identifier_quoting_style='BackticksMySQL'
SELECT('Settings: always & BackticksMySQL');
SHOW CREATE TABLE uk_price_paid
SETTINGS
output_format_identifier_quoting_rule='always',
output_format_identifier_quoting_style='BackticksMySQL';
SHOW CREATE VIEW prices_by_year_view
SETTINGS
output_format_always_quote_identifiers=true,
output_format_identifier_quoting_rule='always',
output_format_identifier_quoting_style='BackticksMySQL';
SHOW CREATE uk_prices_aggs_dest
SETTINGS
output_format_always_quote_identifiers=true,
output_format_identifier_quoting_rule='always',
output_format_identifier_quoting_style='BackticksMySQL';
SHOW CREATE VIEW uk_prices_aggs_view
SETTINGS
output_format_always_quote_identifiers=true,
output_format_identifier_quoting_rule='always',
output_format_identifier_quoting_style='BackticksMySQL';
SHOW CREATE DICTIONARY uk_mortgage_rates_dict
SETTINGS
output_format_always_quote_identifiers=true,
output_format_identifier_quoting_rule='always',
output_format_identifier_quoting_style='BackticksMySQL';
-- Show tables, views, dictionaries with output_format_always_quote_identifiers=false, output_format_identifier_quoting_style='BackticksMySQL'
-- Show tables, views, dictionaries with output_format_identifier_quoting_rule='user_display', output_format_identifier_quoting_style='BackticksMySQL'
SELECT('Settings: user_display & BackticksMySQL');
SHOW CREATE TABLE uk_price_paid
SETTINGS
output_format_always_quote_identifiers=false,
output_format_identifier_quoting_rule='user_display',
output_format_identifier_quoting_style='BackticksMySQL';
SHOW CREATE VIEW prices_by_year_view
SETTINGS
output_format_always_quote_identifiers=false,
output_format_identifier_quoting_rule='user_display',
output_format_identifier_quoting_style='BackticksMySQL';
SHOW CREATE uk_prices_aggs_dest
SETTINGS
output_format_always_quote_identifiers=false,
output_format_identifier_quoting_rule='user_display',
output_format_identifier_quoting_style='BackticksMySQL';
SHOW CREATE VIEW uk_prices_aggs_view
SETTINGS
output_format_always_quote_identifiers=false,
output_format_identifier_quoting_rule='user_display',
output_format_identifier_quoting_style='BackticksMySQL';
SHOW CREATE DICTIONARY uk_mortgage_rates_dict
SETTINGS
output_format_always_quote_identifiers=false,
output_format_identifier_quoting_rule='user_display',
output_format_identifier_quoting_style='BackticksMySQL';
-- Show tables, views, dictionaries with output_format_identifier_quoting_rule='when_necessary', output_format_identifier_quoting_style='BackticksMySQL'
SELECT('Settings: when_necessary & BackticksMySQL');
SHOW CREATE TABLE uk_price_paid
SETTINGS
output_format_identifier_quoting_rule='when_necessary',
output_format_identifier_quoting_style='BackticksMySQL';
SHOW CREATE VIEW prices_by_year_view
SETTINGS
output_format_identifier_quoting_rule='when_necessary',
output_format_identifier_quoting_style='BackticksMySQL';
SHOW CREATE uk_prices_aggs_dest
SETTINGS
output_format_identifier_quoting_rule='when_necessary',
output_format_identifier_quoting_style='BackticksMySQL';
SHOW CREATE VIEW uk_prices_aggs_view
SETTINGS
output_format_identifier_quoting_rule='when_necessary',
output_format_identifier_quoting_style='BackticksMySQL';
SHOW CREATE DICTIONARY uk_mortgage_rates_dict
SETTINGS
output_format_identifier_quoting_rule='when_necessary',
output_format_identifier_quoting_style='BackticksMySQL';
DROP DICTIONARY uk_mortgage_rates_dict;