mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 10:02:01 +00:00
Merge branch 'master' into test_soundex
This commit is contained in:
commit
e327371297
@ -1264,7 +1264,7 @@ Using replacement fields, you can define a pattern for the resulting string. “
|
|||||||
| %d | day of the month, zero-padded (01-31) | 02 |
|
| %d | day of the month, zero-padded (01-31) | 02 |
|
||||||
| %D | Short MM/DD/YY date, equivalent to %m/%d/%y | 01/02/18 |
|
| %D | Short MM/DD/YY date, equivalent to %m/%d/%y | 01/02/18 |
|
||||||
| %e | day of the month, space-padded (1-31) | 2 |
|
| %e | day of the month, space-padded (1-31) | 2 |
|
||||||
| %f | fractional second from the fractional part of DateTime64 | 1234560 |
|
| %f | fractional second, see 'Note 1' below | 1234560 |
|
||||||
| %F | short YYYY-MM-DD date, equivalent to %Y-%m-%d | 2018-01-02 |
|
| %F | short YYYY-MM-DD date, equivalent to %Y-%m-%d | 2018-01-02 |
|
||||||
| %g | two-digit year format, aligned to ISO 8601, abbreviated from four-digit notation | 18 |
|
| %g | two-digit year format, aligned to ISO 8601, abbreviated from four-digit notation | 18 |
|
||||||
| %G | four-digit year format for ISO week number, calculated from the week-based year [defined by the ISO 8601](https://en.wikipedia.org/wiki/ISO_8601#Week_dates) standard, normally useful only with %V | 2018 |
|
| %G | four-digit year format for ISO week number, calculated from the week-based year [defined by the ISO 8601](https://en.wikipedia.org/wiki/ISO_8601#Week_dates) standard, normally useful only with %V | 2018 |
|
||||||
@ -1276,7 +1276,7 @@ Using replacement fields, you can define a pattern for the resulting string. “
|
|||||||
| %k | hour in 24h format (00-23) | 22 |
|
| %k | hour in 24h format (00-23) | 22 |
|
||||||
| %l | hour in 12h format (01-12) | 09 |
|
| %l | hour in 12h format (01-12) | 09 |
|
||||||
| %m | month as an integer number (01-12) | 01 |
|
| %m | month as an integer number (01-12) | 01 |
|
||||||
| %M | full month name (January-December), see (*) below | January |
|
| %M | full month name (January-December), see 'Note 2' below | January |
|
||||||
| %n | new-line character (‘’) | |
|
| %n | new-line character (‘’) | |
|
||||||
| %p | AM or PM designation | PM |
|
| %p | AM or PM designation | PM |
|
||||||
| %Q | Quarter (1-4) | 1 |
|
| %Q | Quarter (1-4) | 1 |
|
||||||
@ -1295,7 +1295,9 @@ Using replacement fields, you can define a pattern for the resulting string. “
|
|||||||
| %z | Time offset from UTC as +HHMM or -HHMM | -0500 |
|
| %z | Time offset from UTC as +HHMM or -HHMM | -0500 |
|
||||||
| %% | a % sign | % |
|
| %% | a % sign | % |
|
||||||
|
|
||||||
(*) In ClickHouse versions earlier than v23.4, `%M` prints the minute (00-59) instead of the full month name (January-December). The previous behavior can be restored using setting `formatdatetime_parsedatetime_m_is_month_name = 0`.
|
Note 1: In ClickHouse versions earlier than v23.4, `%f` prints a single zero (0) if the formatted value is a Date, Date32 or DateTime (which have no fractional seconds) or a DateTime64 with a precision of 0. The previous behavior can be restored using setting `formatdatetime_f_prints_single_zero = 1`.
|
||||||
|
|
||||||
|
Note 2: In ClickHouse versions earlier than v23.4, `%M` prints the minute (00-59) instead of the full month name (January-December). The previous behavior can be restored using setting `formatdatetime_parsedatetime_m_is_month_name = 0`.
|
||||||
|
|
||||||
**Example**
|
**Example**
|
||||||
|
|
||||||
|
@ -1245,7 +1245,6 @@ Returns DateTime values parsed from input string according to a MySQL style form
|
|||||||
**Supported format specifiers**
|
**Supported format specifiers**
|
||||||
|
|
||||||
All format specifiers listed in [formatDateTime](/docs/en/sql-reference/functions/date-time-functions.md#date_time_functions-formatDateTime) except:
|
All format specifiers listed in [formatDateTime](/docs/en/sql-reference/functions/date-time-functions.md#date_time_functions-formatDateTime) except:
|
||||||
- %f: fractional second
|
|
||||||
- %Q: Quarter (1-4)
|
- %Q: Quarter (1-4)
|
||||||
|
|
||||||
**Example**
|
**Example**
|
||||||
|
@ -62,16 +62,10 @@ String getSHA1(const String & userdata)
|
|||||||
return String{digest_id.begin(), digest_id.end()};
|
return String{digest_id.begin(), digest_id.end()};
|
||||||
}
|
}
|
||||||
|
|
||||||
String generateDigest(const String & userdata)
|
|
||||||
{
|
|
||||||
std::vector<String> user_password;
|
|
||||||
boost::split(user_password, userdata, [](char character) { return character == ':'; });
|
|
||||||
return user_password[0] + ":" + base64Encode(getSHA1(userdata));
|
|
||||||
}
|
|
||||||
|
|
||||||
bool fixupACL(
|
bool fixupACL(
|
||||||
const std::vector<Coordination::ACL> & request_acls,
|
const std::vector<Coordination::ACL> & request_acls,
|
||||||
const std::vector<KeeperStorage::AuthID> & current_ids,
|
int64_t session_id,
|
||||||
|
const KeeperStorage::UncommittedState & uncommitted_state,
|
||||||
std::vector<Coordination::ACL> & result_acls)
|
std::vector<Coordination::ACL> & result_acls)
|
||||||
{
|
{
|
||||||
if (request_acls.empty())
|
if (request_acls.empty())
|
||||||
@ -82,14 +76,18 @@ bool fixupACL(
|
|||||||
{
|
{
|
||||||
if (request_acl.scheme == "auth")
|
if (request_acl.scheme == "auth")
|
||||||
{
|
{
|
||||||
for (const auto & current_id : current_ids)
|
uncommitted_state.forEachAuthInSession(
|
||||||
{
|
session_id,
|
||||||
valid_found = true;
|
[&](const KeeperStorage::AuthID & auth_id)
|
||||||
Coordination::ACL new_acl = request_acl;
|
{
|
||||||
new_acl.scheme = current_id.scheme;
|
valid_found = true;
|
||||||
new_acl.id = current_id.id;
|
Coordination::ACL new_acl = request_acl;
|
||||||
result_acls.push_back(new_acl);
|
|
||||||
}
|
new_acl.scheme = auth_id.scheme;
|
||||||
|
new_acl.id = auth_id.id;
|
||||||
|
|
||||||
|
result_acls.push_back(new_acl);
|
||||||
|
});
|
||||||
}
|
}
|
||||||
else if (request_acl.scheme == "world" && request_acl.id == "anyone")
|
else if (request_acl.scheme == "world" && request_acl.id == "anyone")
|
||||||
{
|
{
|
||||||
@ -565,6 +563,32 @@ Coordination::ACLs KeeperStorage::UncommittedState::getACLs(StringRef path) cons
|
|||||||
return storage.acl_map.convertNumber(node_it->value.acl_id);
|
return storage.acl_map.convertNumber(node_it->value.acl_id);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void KeeperStorage::UncommittedState::forEachAuthInSession(int64_t session_id, std::function<void(const AuthID &)> func) const
|
||||||
|
{
|
||||||
|
const auto call_for_each_auth = [&func](const auto & auth_ids)
|
||||||
|
{
|
||||||
|
for (const auto & auth : auth_ids)
|
||||||
|
{
|
||||||
|
using TAuth = std::remove_reference_t<decltype(auth)>;
|
||||||
|
|
||||||
|
const AuthID * auth_ptr = nullptr;
|
||||||
|
if constexpr (std::is_pointer_v<TAuth>)
|
||||||
|
auth_ptr = auth;
|
||||||
|
else
|
||||||
|
auth_ptr = &auth;
|
||||||
|
|
||||||
|
func(*auth_ptr);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
// for committed
|
||||||
|
if (storage.session_and_auth.contains(session_id))
|
||||||
|
call_for_each_auth(storage.session_and_auth.at(session_id));
|
||||||
|
// for uncommitted
|
||||||
|
if (session_and_auth.contains(session_id))
|
||||||
|
call_for_each_auth(session_and_auth.at(session_id));
|
||||||
|
}
|
||||||
|
|
||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -928,7 +952,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr
|
|||||||
return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}};
|
return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}};
|
||||||
|
|
||||||
Coordination::ACLs node_acls;
|
Coordination::ACLs node_acls;
|
||||||
if (!fixupACL(request.acls, storage.session_and_auth[session_id], node_acls))
|
if (!fixupACL(request.acls, session_id, storage.uncommitted_state, node_acls))
|
||||||
return {KeeperStorage::Delta{zxid, Coordination::Error::ZINVALIDACL}};
|
return {KeeperStorage::Delta{zxid, Coordination::Error::ZINVALIDACL}};
|
||||||
|
|
||||||
if (request.is_ephemeral)
|
if (request.is_ephemeral)
|
||||||
@ -1534,10 +1558,8 @@ struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestPr
|
|||||||
return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}};
|
return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}};
|
||||||
|
|
||||||
|
|
||||||
auto & session_auth_ids = storage.session_and_auth[session_id];
|
|
||||||
Coordination::ACLs node_acls;
|
Coordination::ACLs node_acls;
|
||||||
|
if (!fixupACL(request.acls, session_id, uncommitted_state, node_acls))
|
||||||
if (!fixupACL(request.acls, session_auth_ids, node_acls))
|
|
||||||
return {KeeperStorage::Delta{zxid, Coordination::Error::ZINVALIDACL}};
|
return {KeeperStorage::Delta{zxid, Coordination::Error::ZINVALIDACL}};
|
||||||
|
|
||||||
std::vector<KeeperStorage::Delta> new_deltas
|
std::vector<KeeperStorage::Delta> new_deltas
|
||||||
@ -1841,7 +1863,7 @@ struct KeeperStorageAuthRequestProcessor final : public KeeperStorageRequestProc
|
|||||||
return {KeeperStorage::Delta{zxid, Coordination::Error::ZAUTHFAILED}};
|
return {KeeperStorage::Delta{zxid, Coordination::Error::ZAUTHFAILED}};
|
||||||
|
|
||||||
std::vector<KeeperStorage::Delta> new_deltas;
|
std::vector<KeeperStorage::Delta> new_deltas;
|
||||||
auto auth_digest = generateDigest(auth_request.data);
|
auto auth_digest = KeeperStorage::generateDigest(auth_request.data);
|
||||||
if (auth_digest == storage.superdigest)
|
if (auth_digest == storage.superdigest)
|
||||||
{
|
{
|
||||||
KeeperStorage::AuthID auth{"super", ""};
|
KeeperStorage::AuthID auth{"super", ""};
|
||||||
@ -2421,5 +2443,12 @@ void KeeperStorage::recalculateStats()
|
|||||||
container.recalculateDataSize();
|
container.recalculateDataSize();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
String KeeperStorage::generateDigest(const String & userdata)
|
||||||
|
{
|
||||||
|
std::vector<String> user_password;
|
||||||
|
boost::split(user_password, userdata, [](char character) { return character == ':'; });
|
||||||
|
return user_password[0] + ":" + base64Encode(getSHA1(userdata));
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -105,6 +105,8 @@ public:
|
|||||||
return first.value == second.value;
|
return first.value == second.value;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
static String generateDigest(const String & userdata);
|
||||||
|
|
||||||
struct RequestForSession
|
struct RequestForSession
|
||||||
{
|
{
|
||||||
int64_t session_id;
|
int64_t session_id;
|
||||||
@ -263,6 +265,8 @@ public:
|
|||||||
return check_auth(auth_it->second);
|
return check_auth(auth_it->second);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void forEachAuthInSession(int64_t session_id, std::function<void(const AuthID &)> func) const;
|
||||||
|
|
||||||
std::shared_ptr<Node> tryGetNodeFromStorage(StringRef path) const;
|
std::shared_ptr<Node> tryGetNodeFromStorage(StringRef path) const;
|
||||||
|
|
||||||
std::unordered_map<int64_t, std::list<const AuthID *>> session_and_auth;
|
std::unordered_map<int64_t, std::list<const AuthID *>> session_and_auth;
|
||||||
|
@ -1579,6 +1579,113 @@ TEST_P(CoordinationTest, TestEphemeralNodeRemove)
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
TEST_P(CoordinationTest, TestCreateNodeWithAuthSchemeForAclWhenAuthIsPrecommitted)
|
||||||
|
{
|
||||||
|
using namespace Coordination;
|
||||||
|
using namespace DB;
|
||||||
|
|
||||||
|
ChangelogDirTest snapshots("./snapshots");
|
||||||
|
CoordinationSettingsPtr settings = std::make_shared<CoordinationSettings>();
|
||||||
|
ResponsesQueue queue(std::numeric_limits<size_t>::max());
|
||||||
|
SnapshotsQueue snapshots_queue{1};
|
||||||
|
|
||||||
|
auto state_machine = std::make_shared<KeeperStateMachine>(queue, snapshots_queue, "./snapshots", settings, keeper_context, nullptr);
|
||||||
|
state_machine->init();
|
||||||
|
|
||||||
|
String user_auth_data = "test_user:test_password";
|
||||||
|
String digest = KeeperStorage::generateDigest(user_auth_data);
|
||||||
|
|
||||||
|
std::shared_ptr<ZooKeeperAuthRequest> auth_req = std::make_shared<ZooKeeperAuthRequest>();
|
||||||
|
auth_req->scheme = "digest";
|
||||||
|
auth_req->data = user_auth_data;
|
||||||
|
|
||||||
|
// Add auth data to the session
|
||||||
|
auto auth_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), auth_req);
|
||||||
|
state_machine->pre_commit(1, auth_entry->get_buf());
|
||||||
|
|
||||||
|
// Create a node with 'auth' scheme for ACL
|
||||||
|
String node_path = "/hello";
|
||||||
|
std::shared_ptr<ZooKeeperCreateRequest> create_req = std::make_shared<ZooKeeperCreateRequest>();
|
||||||
|
create_req->path = node_path;
|
||||||
|
// When 'auth' scheme is used the creator must have been authenticated by the server (for example, using 'digest' scheme) before it can
|
||||||
|
// create nodes with this ACL.
|
||||||
|
create_req->acls = {{.permissions = 31, .scheme = "auth", .id = ""}};
|
||||||
|
auto create_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), create_req);
|
||||||
|
state_machine->pre_commit(2, create_entry->get_buf());
|
||||||
|
|
||||||
|
const auto & uncommitted_state = state_machine->getStorage().uncommitted_state;
|
||||||
|
ASSERT_TRUE(uncommitted_state.nodes.contains(node_path));
|
||||||
|
|
||||||
|
// commit log entries
|
||||||
|
state_machine->commit(1, auth_entry->get_buf());
|
||||||
|
state_machine->commit(2, create_entry->get_buf());
|
||||||
|
|
||||||
|
auto node = uncommitted_state.getNode(node_path);
|
||||||
|
ASSERT_NE(node, nullptr);
|
||||||
|
auto acls = uncommitted_state.getACLs(node_path);
|
||||||
|
ASSERT_EQ(acls.size(), 1);
|
||||||
|
EXPECT_EQ(acls[0].scheme, "digest");
|
||||||
|
EXPECT_EQ(acls[0].id, digest);
|
||||||
|
EXPECT_EQ(acls[0].permissions, 31);
|
||||||
|
}
|
||||||
|
|
||||||
|
TEST_P(CoordinationTest, TestSetACLWithAuthSchemeForAclWhenAuthIsPrecommitted)
|
||||||
|
{
|
||||||
|
using namespace Coordination;
|
||||||
|
using namespace DB;
|
||||||
|
|
||||||
|
ChangelogDirTest snapshots("./snapshots");
|
||||||
|
CoordinationSettingsPtr settings = std::make_shared<CoordinationSettings>();
|
||||||
|
ResponsesQueue queue(std::numeric_limits<size_t>::max());
|
||||||
|
SnapshotsQueue snapshots_queue{1};
|
||||||
|
|
||||||
|
auto state_machine = std::make_shared<KeeperStateMachine>(queue, snapshots_queue, "./snapshots", settings, keeper_context, nullptr);
|
||||||
|
state_machine->init();
|
||||||
|
|
||||||
|
String user_auth_data = "test_user:test_password";
|
||||||
|
String digest = KeeperStorage::generateDigest(user_auth_data);
|
||||||
|
|
||||||
|
std::shared_ptr<ZooKeeperAuthRequest> auth_req = std::make_shared<ZooKeeperAuthRequest>();
|
||||||
|
auth_req->scheme = "digest";
|
||||||
|
auth_req->data = user_auth_data;
|
||||||
|
|
||||||
|
// Add auth data to the session
|
||||||
|
auto auth_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), auth_req);
|
||||||
|
state_machine->pre_commit(1, auth_entry->get_buf());
|
||||||
|
|
||||||
|
// Create a node
|
||||||
|
String node_path = "/hello";
|
||||||
|
std::shared_ptr<ZooKeeperCreateRequest> create_req = std::make_shared<ZooKeeperCreateRequest>();
|
||||||
|
create_req->path = node_path;
|
||||||
|
auto create_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), create_req);
|
||||||
|
state_machine->pre_commit(2, create_entry->get_buf());
|
||||||
|
|
||||||
|
// Set ACL with 'auth' scheme for ACL
|
||||||
|
std::shared_ptr<ZooKeeperSetACLRequest> set_acl_req = std::make_shared<ZooKeeperSetACLRequest>();
|
||||||
|
set_acl_req->path = node_path;
|
||||||
|
// When 'auth' scheme is used the creator must have been authenticated by the server (for example, using 'digest' scheme) before it can
|
||||||
|
// set this ACL.
|
||||||
|
set_acl_req->acls = {{.permissions = 31, .scheme = "auth", .id = ""}};
|
||||||
|
auto set_acl_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), set_acl_req);
|
||||||
|
state_machine->pre_commit(3, set_acl_entry->get_buf());
|
||||||
|
|
||||||
|
// commit all entries
|
||||||
|
state_machine->commit(1, auth_entry->get_buf());
|
||||||
|
state_machine->commit(2, create_entry->get_buf());
|
||||||
|
state_machine->commit(3, set_acl_entry->get_buf());
|
||||||
|
|
||||||
|
const auto & uncommitted_state = state_machine->getStorage().uncommitted_state;
|
||||||
|
auto node = uncommitted_state.getNode(node_path);
|
||||||
|
|
||||||
|
ASSERT_NE(node, nullptr);
|
||||||
|
auto acls = uncommitted_state.getACLs(node_path);
|
||||||
|
ASSERT_EQ(acls.size(), 1);
|
||||||
|
EXPECT_EQ(acls[0].scheme, "digest");
|
||||||
|
EXPECT_EQ(acls[0].id, digest);
|
||||||
|
EXPECT_EQ(acls[0].permissions, 31);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
TEST_P(CoordinationTest, TestRotateIntervalChanges)
|
TEST_P(CoordinationTest, TestRotateIntervalChanges)
|
||||||
{
|
{
|
||||||
using namespace Coordination;
|
using namespace Coordination;
|
||||||
|
@ -467,7 +467,8 @@ class IColumn;
|
|||||||
M(Bool, allow_introspection_functions, false, "Allow functions for introspection of ELF and DWARF for query profiling. These functions are slow and may impose security considerations.", 0) \
|
M(Bool, allow_introspection_functions, false, "Allow functions for introspection of ELF and DWARF for query profiling. These functions are slow and may impose security considerations.", 0) \
|
||||||
\
|
\
|
||||||
M(Bool, allow_execute_multiif_columnar, true, "Allow execute multiIf function columnar", 0) \
|
M(Bool, allow_execute_multiif_columnar, true, "Allow execute multiIf function columnar", 0) \
|
||||||
M(Bool, formatdatetime_parsedatetime_m_is_month_name, true, "Formatter '%M' in function 'formatDateTime' produces the month name instead of minutes.", 0) \
|
M(Bool, formatdatetime_f_prints_single_zero, false, "Formatter '%f' in function 'formatDateTime()' produces a single zero instead of six zeros if the formatted value has no fractional seconds.", 0) \
|
||||||
|
M(Bool, formatdatetime_parsedatetime_m_is_month_name, true, "Formatter '%M' in functions 'formatDateTime()' and 'parseDateTime()' produces the month name instead of minutes.", 0) \
|
||||||
\
|
\
|
||||||
M(UInt64, max_partitions_per_insert_block, 100, "Limit maximum number of partitions in single INSERTed block. Zero means unlimited. Throw exception if the block contains too many partitions. This setting is a safety threshold, because using large number of partitions is a common misconception.", 0) \
|
M(UInt64, max_partitions_per_insert_block, 100, "Limit maximum number of partitions in single INSERTed block. Zero means unlimited. Throw exception if the block contains too many partitions. This setting is a safety threshold, because using large number of partitions is a common misconception.", 0) \
|
||||||
M(Int64, max_partitions_to_read, -1, "Limit the max number of partitions that can be accessed in one query. <= 0 means unlimited.", 0) \
|
M(Int64, max_partitions_to_read, -1, "Limit the max number of partitions that can be accessed in one query. <= 0 means unlimited.", 0) \
|
||||||
|
@ -101,6 +101,7 @@ static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> sett
|
|||||||
{"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"},
|
{"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"},
|
||||||
{"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}},
|
{"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}},
|
||||||
{"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}},
|
{"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}},
|
||||||
|
{"23.4", {{"formatdatetime_f_prints_single_zero", true, false, "Improved compatibility with MySQL DATE_FORMAT()/STR_TO_DATE()"}}},
|
||||||
{"23.4", {{"formatdatetime_parsedatetime_m_is_month_name", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}},
|
{"23.4", {{"formatdatetime_parsedatetime_m_is_month_name", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}},
|
||||||
{"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}},
|
{"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}},
|
||||||
{"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"},
|
{"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"},
|
||||||
|
@ -272,7 +272,7 @@ void RegExpTreeDictionary::initGraph()
|
|||||||
if (value->parent_id == 0) // this is root node.
|
if (value->parent_id == 0) // this is root node.
|
||||||
initTopologyOrder(id, visited, topology_id);
|
initTopologyOrder(id, visited, topology_id);
|
||||||
if (topology_order.size() != regex_nodes.size())
|
if (topology_order.size() != regex_nodes.size())
|
||||||
throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Invalid Regex tree");
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "The topology order cannot match the number of regex nodes. This is likely a internal bug.");
|
||||||
}
|
}
|
||||||
|
|
||||||
void RegExpTreeDictionary::initTopologyOrder(UInt64 node_idx, std::set<UInt64> & visited, UInt64 & topology_id)
|
void RegExpTreeDictionary::initTopologyOrder(UInt64 node_idx, std::set<UInt64> & visited, UInt64 & topology_id)
|
||||||
@ -280,7 +280,7 @@ void RegExpTreeDictionary::initTopologyOrder(UInt64 node_idx, std::set<UInt64> &
|
|||||||
visited.insert(node_idx);
|
visited.insert(node_idx);
|
||||||
for (UInt64 child_idx : regex_nodes[node_idx]->children)
|
for (UInt64 child_idx : regex_nodes[node_idx]->children)
|
||||||
if (visited.contains(child_idx))
|
if (visited.contains(child_idx))
|
||||||
throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Invalid Regex tree. The input tree is cyclical");
|
throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "The regexp tree is cyclical. Please check your config.");
|
||||||
else
|
else
|
||||||
initTopologyOrder(child_idx, visited, topology_id);
|
initTopologyOrder(child_idx, visited, topology_id);
|
||||||
topology_order[node_idx] = topology_id++;
|
topology_order[node_idx] = topology_id++;
|
||||||
|
@ -449,6 +449,20 @@ private:
|
|||||||
}
|
}
|
||||||
|
|
||||||
size_t mysqlFractionalSecond(char * dest, Time /*source*/, UInt64 fractional_second, UInt32 scale, const DateLUTImpl & /*timezone*/)
|
size_t mysqlFractionalSecond(char * dest, Time /*source*/, UInt64 fractional_second, UInt32 scale, const DateLUTImpl & /*timezone*/)
|
||||||
|
{
|
||||||
|
if (scale == 0)
|
||||||
|
scale = 6;
|
||||||
|
|
||||||
|
for (Int64 i = scale, value = fractional_second; i > 0; --i)
|
||||||
|
{
|
||||||
|
dest[i - 1] += value % 10;
|
||||||
|
value /= 10;
|
||||||
|
}
|
||||||
|
return scale;
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Same as mysqlFractionalSecond but prints a single zero if the value has no fractional seconds
|
||||||
|
size_t mysqlFractionalSecondSingleZero(char * dest, Time /*source*/, UInt64 fractional_second, UInt32 scale, const DateLUTImpl & /*timezone*/)
|
||||||
{
|
{
|
||||||
if (scale == 0)
|
if (scale == 0)
|
||||||
scale = 1;
|
scale = 1;
|
||||||
@ -710,6 +724,7 @@ private:
|
|||||||
}
|
}
|
||||||
|
|
||||||
const bool mysql_M_is_month_name;
|
const bool mysql_M_is_month_name;
|
||||||
|
const bool mysql_f_prints_single_zero;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
static constexpr auto name = Name::name;
|
static constexpr auto name = Name::name;
|
||||||
@ -718,6 +733,7 @@ public:
|
|||||||
|
|
||||||
explicit FunctionFormatDateTimeImpl(ContextPtr context)
|
explicit FunctionFormatDateTimeImpl(ContextPtr context)
|
||||||
: mysql_M_is_month_name(context->getSettings().formatdatetime_parsedatetime_m_is_month_name)
|
: mysql_M_is_month_name(context->getSettings().formatdatetime_parsedatetime_m_is_month_name)
|
||||||
|
, mysql_f_prints_single_zero(context->getSettings().formatdatetime_f_prints_single_zero)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1116,11 +1132,21 @@ public:
|
|||||||
// Fractional seconds
|
// Fractional seconds
|
||||||
case 'f':
|
case 'f':
|
||||||
{
|
{
|
||||||
/// If the time data type has no fractional part, then we print '0' as the fractional part.
|
/// If the time data type has no fractional part, we print (default) '000000' or (deprecated) '0' as fractional part.
|
||||||
Instruction<T> instruction;
|
if (mysql_f_prints_single_zero)
|
||||||
instruction.setMysqlFunc(&Instruction<T>::mysqlFractionalSecond);
|
{
|
||||||
instructions.push_back(std::move(instruction));
|
Instruction<T> instruction;
|
||||||
out_template += String(std::max<UInt32>(1, scale), '0');
|
instruction.setMysqlFunc(&Instruction<T>::mysqlFractionalSecondSingleZero);
|
||||||
|
instructions.push_back(std::move(instruction));
|
||||||
|
out_template += String(scale == 0 ? 1 : scale, '0');
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
Instruction<T> instruction;
|
||||||
|
instruction.setMysqlFunc(&Instruction<T>::mysqlFractionalSecond);
|
||||||
|
instructions.push_back(std::move(instruction));
|
||||||
|
out_template += String(scale == 0 ? 6 : scale, '0');
|
||||||
|
}
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -723,7 +723,7 @@ namespace
|
|||||||
if constexpr (need_check_space == NeedCheckSpace::Yes)
|
if constexpr (need_check_space == NeedCheckSpace::Yes)
|
||||||
checkSpace(cur, end, 1, "assertChar requires size >= 1", fragment);
|
checkSpace(cur, end, 1, "assertChar requires size >= 1", fragment);
|
||||||
|
|
||||||
if (*cur != expected)
|
if (*cur != expected) [[unlikely]]
|
||||||
throw Exception(
|
throw Exception(
|
||||||
ErrorCodes::CANNOT_PARSE_DATETIME,
|
ErrorCodes::CANNOT_PARSE_DATETIME,
|
||||||
"Unable to parse fragment {} from {} because char {} is expected but {} provided",
|
"Unable to parse fragment {} from {} because char {} is expected but {} provided",
|
||||||
@ -736,6 +736,24 @@ namespace
|
|||||||
return cur;
|
return cur;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
template <NeedCheckSpace need_check_space>
|
||||||
|
static Pos assertNumber(Pos cur, Pos end, const String & fragment)
|
||||||
|
{
|
||||||
|
if constexpr (need_check_space == NeedCheckSpace::Yes)
|
||||||
|
checkSpace(cur, end, 1, "assertChar requires size >= 1", fragment);
|
||||||
|
|
||||||
|
if (*cur < '0' || *cur > '9') [[unlikely]]
|
||||||
|
throw Exception(
|
||||||
|
ErrorCodes::CANNOT_PARSE_DATETIME,
|
||||||
|
"Unable to parse fragment {} from {} because {} is not a number",
|
||||||
|
fragment,
|
||||||
|
std::string_view(cur, end - cur),
|
||||||
|
String(*cur, 1));
|
||||||
|
|
||||||
|
++cur;
|
||||||
|
return cur;
|
||||||
|
}
|
||||||
|
|
||||||
static Pos mysqlDayOfWeekTextShort(Pos cur, Pos end, const String & fragment, DateTime & date)
|
static Pos mysqlDayOfWeekTextShort(Pos cur, Pos end, const String & fragment, DateTime & date)
|
||||||
{
|
{
|
||||||
checkSpace(cur, end, 3, "mysqlDayOfWeekTextShort requires size >= 3", fragment);
|
checkSpace(cur, end, 3, "mysqlDayOfWeekTextShort requires size >= 3", fragment);
|
||||||
@ -1074,6 +1092,16 @@ namespace
|
|||||||
return cur;
|
return cur;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
static Pos mysqlMicrosecond(Pos cur, Pos end, const String & fragment, DateTime & /*date*/)
|
||||||
|
{
|
||||||
|
checkSpace(cur, end, 6, "mysqlMicrosecond requires size >= 6", fragment);
|
||||||
|
|
||||||
|
for (size_t i = 0; i < 6; ++i)
|
||||||
|
cur = assertNumber<NeedCheckSpace::No>(cur, end, fragment);
|
||||||
|
|
||||||
|
return cur;
|
||||||
|
}
|
||||||
|
|
||||||
static Pos mysqlISO8601Time(Pos cur, Pos end, const String & fragment, DateTime & date)
|
static Pos mysqlISO8601Time(Pos cur, Pos end, const String & fragment, DateTime & date)
|
||||||
{
|
{
|
||||||
checkSpace(cur, end, 8, "mysqlISO8601Time requires size >= 8", fragment);
|
checkSpace(cur, end, 8, "mysqlISO8601Time requires size >= 8", fragment);
|
||||||
@ -1485,6 +1513,10 @@ namespace
|
|||||||
instructions.emplace_back(ACTION_ARGS(Instruction::mysqlDayOfMonthSpacePadded));
|
instructions.emplace_back(ACTION_ARGS(Instruction::mysqlDayOfMonthSpacePadded));
|
||||||
break;
|
break;
|
||||||
|
|
||||||
|
// Fractional seconds
|
||||||
|
case 'f':
|
||||||
|
instructions.emplace_back(ACTION_ARGS(Instruction::mysqlMicrosecond));
|
||||||
|
break;
|
||||||
|
|
||||||
// Short YYYY-MM-DD date, equivalent to %Y-%m-%d 2001-08-23
|
// Short YYYY-MM-DD date, equivalent to %Y-%m-%d 2001-08-23
|
||||||
case 'F':
|
case 'F':
|
||||||
@ -1637,8 +1669,6 @@ namespace
|
|||||||
/// Unimplemented
|
/// Unimplemented
|
||||||
|
|
||||||
/// Fractional seconds
|
/// Fractional seconds
|
||||||
case 'f':
|
|
||||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for fractional seconds");
|
|
||||||
case 'U':
|
case 'U':
|
||||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for WEEK (Sun-Sat)");
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for WEEK (Sun-Sat)");
|
||||||
case 'v':
|
case 'v':
|
||||||
|
@ -1028,12 +1028,15 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re
|
|||||||
|
|
||||||
bool is_ok = true;
|
bool is_ok = true;
|
||||||
if constexpr (std::is_same_v<ReturnType, void>)
|
if constexpr (std::is_same_v<ReturnType, void>)
|
||||||
datetime64 = DecimalUtils::decimalFromComponents<DateTime64>(components, scale);
|
{
|
||||||
|
datetime64 = DecimalUtils::decimalFromComponents<DateTime64>(components, scale) * negative_multiplier;
|
||||||
|
}
|
||||||
else
|
else
|
||||||
|
{
|
||||||
is_ok = DecimalUtils::tryGetDecimalFromComponents<DateTime64>(components, scale, datetime64);
|
is_ok = DecimalUtils::tryGetDecimalFromComponents<DateTime64>(components, scale, datetime64);
|
||||||
|
if (is_ok)
|
||||||
datetime64 *= negative_multiplier;
|
datetime64 *= negative_multiplier;
|
||||||
|
}
|
||||||
|
|
||||||
return ReturnType(is_ok);
|
return ReturnType(is_ok);
|
||||||
}
|
}
|
||||||
|
@ -4,6 +4,7 @@
|
|||||||
#include <Interpreters/IInterpreter.h>
|
#include <Interpreters/IInterpreter.h>
|
||||||
#include <Parsers/ASTInsertQuery.h>
|
#include <Parsers/ASTInsertQuery.h>
|
||||||
#include <Storages/StorageInMemoryMetadata.h>
|
#include <Storages/StorageInMemoryMetadata.h>
|
||||||
|
#include <Common/ThreadStatus.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
@ -303,9 +303,9 @@ ASTPtr makeBetweenOperator(bool negative, ASTs arguments)
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
ParserExpressionWithOptionalAlias::ParserExpressionWithOptionalAlias(bool allow_alias_without_as_keyword, bool is_table_function)
|
ParserExpressionWithOptionalAlias::ParserExpressionWithOptionalAlias(bool allow_alias_without_as_keyword, bool is_table_function, bool allow_trailing_commas)
|
||||||
: impl(std::make_unique<ParserWithOptionalAlias>(
|
: impl(std::make_unique<ParserWithOptionalAlias>(
|
||||||
is_table_function ? ParserPtr(std::make_unique<ParserTableFunctionExpression>()) : ParserPtr(std::make_unique<ParserExpression>()),
|
is_table_function ? ParserPtr(std::make_unique<ParserTableFunctionExpression>()) : ParserPtr(std::make_unique<ParserExpression>(allow_trailing_commas)),
|
||||||
allow_alias_without_as_keyword))
|
allow_alias_without_as_keyword))
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
@ -314,7 +314,7 @@ ParserExpressionWithOptionalAlias::ParserExpressionWithOptionalAlias(bool allow_
|
|||||||
bool ParserExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
bool ParserExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||||
{
|
{
|
||||||
return ParserList(
|
return ParserList(
|
||||||
std::make_unique<ParserExpressionWithOptionalAlias>(allow_alias_without_as_keyword, is_table_function),
|
std::make_unique<ParserExpressionWithOptionalAlias>(allow_alias_without_as_keyword, is_table_function, allow_trailing_commas),
|
||||||
std::make_unique<ParserToken>(TokenType::Comma))
|
std::make_unique<ParserToken>(TokenType::Comma))
|
||||||
.parse(pos, node, expected);
|
.parse(pos, node, expected);
|
||||||
}
|
}
|
||||||
@ -779,13 +779,50 @@ protected:
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
struct ParserExpressionImpl
|
||||||
|
{
|
||||||
|
static std::vector<std::pair<const char *, Operator>> operators_table;
|
||||||
|
static std::vector<std::pair<const char *, Operator>> unary_operators_table;
|
||||||
|
static const char * overlapping_operators_to_skip[];
|
||||||
|
|
||||||
|
static Operator finish_between_operator;
|
||||||
|
|
||||||
|
ParserCompoundIdentifier identifier_parser{false, true};
|
||||||
|
ParserNumber number_parser;
|
||||||
|
ParserAsterisk asterisk_parser;
|
||||||
|
ParserLiteral literal_parser;
|
||||||
|
ParserTupleOfLiterals tuple_literal_parser;
|
||||||
|
ParserArrayOfLiterals array_literal_parser;
|
||||||
|
ParserSubstitution substitution_parser;
|
||||||
|
ParserMySQLGlobalVariable mysql_global_variable_parser;
|
||||||
|
|
||||||
|
ParserKeyword any_parser{"ANY"};
|
||||||
|
ParserKeyword all_parser{"ALL"};
|
||||||
|
|
||||||
|
// Recursion
|
||||||
|
ParserQualifiedAsterisk qualified_asterisk_parser;
|
||||||
|
ParserColumnsMatcher columns_matcher_parser;
|
||||||
|
ParserQualifiedColumnsMatcher qualified_columns_matcher_parser;
|
||||||
|
ParserSubquery subquery_parser;
|
||||||
|
|
||||||
|
bool parse(std::unique_ptr<Layer> start, IParser::Pos & pos, ASTPtr & node, Expected & expected);
|
||||||
|
|
||||||
|
using Layers = std::vector<std::unique_ptr<Layer>>;
|
||||||
|
|
||||||
|
Action tryParseOperand(Layers & layers, IParser::Pos & pos, Expected & expected);
|
||||||
|
Action tryParseOperator(Layers & layers, IParser::Pos & pos, Expected & expected);
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
class ExpressionLayer : public Layer
|
class ExpressionLayer : public Layer
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
|
||||||
explicit ExpressionLayer(bool is_table_function_) : Layer(false, false)
|
explicit ExpressionLayer(bool is_table_function_, bool allow_trailing_commas_ = false)
|
||||||
|
: Layer(false, false)
|
||||||
{
|
{
|
||||||
is_table_function = is_table_function_;
|
is_table_function = is_table_function_;
|
||||||
|
allow_trailing_commas = allow_trailing_commas_;
|
||||||
}
|
}
|
||||||
|
|
||||||
bool getResult(ASTPtr & node) override
|
bool getResult(ASTPtr & node) override
|
||||||
@ -802,10 +839,62 @@ public:
|
|||||||
bool parse(IParser::Pos & pos, Expected & /*expected*/, Action & /*action*/) override
|
bool parse(IParser::Pos & pos, Expected & /*expected*/, Action & /*action*/) override
|
||||||
{
|
{
|
||||||
if (pos->type == TokenType::Comma)
|
if (pos->type == TokenType::Comma)
|
||||||
|
{
|
||||||
finished = true;
|
finished = true;
|
||||||
|
|
||||||
|
if (!allow_trailing_commas)
|
||||||
|
return true;
|
||||||
|
|
||||||
|
/// We support trailing commas at the end of the column declaration:
|
||||||
|
/// - SELECT a, b, c, FROM table
|
||||||
|
/// - SELECT 1,
|
||||||
|
|
||||||
|
/// For this purpose we need to eliminate the following cases:
|
||||||
|
/// 1. WITH 1 AS from SELECT 2, from
|
||||||
|
/// 2. SELECT to, from FROM table
|
||||||
|
/// 3. SELECT to, from AS alias FROM table
|
||||||
|
/// 4. SELECT to, from + to, from IN [1,2,3], FROM table
|
||||||
|
|
||||||
|
Expected test_expected;
|
||||||
|
auto test_pos = pos;
|
||||||
|
++test_pos;
|
||||||
|
|
||||||
|
/// End of query
|
||||||
|
if (test_pos.isValid() && test_pos->type != TokenType::Semicolon)
|
||||||
|
{
|
||||||
|
/// If we can't parse FROM then return
|
||||||
|
if (!ParserKeyword("FROM").ignore(test_pos, test_expected))
|
||||||
|
return true;
|
||||||
|
|
||||||
|
/// If we parse a second FROM then the first one was a name of a column
|
||||||
|
if (ParserKeyword("FROM").ignore(test_pos, test_expected))
|
||||||
|
return true;
|
||||||
|
|
||||||
|
/// If we parse an explicit alias to FROM, then it was a name of a column
|
||||||
|
if (ParserAlias(false).ignore(test_pos, test_expected))
|
||||||
|
return true;
|
||||||
|
|
||||||
|
/// If we parse an operator after FROM then it was a name of a column
|
||||||
|
auto cur_op = ParserExpressionImpl::operators_table.begin();
|
||||||
|
for (; cur_op != ParserExpressionImpl::operators_table.end(); ++cur_op)
|
||||||
|
{
|
||||||
|
if (parseOperator(test_pos, cur_op->first, test_expected))
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (cur_op != ParserExpressionImpl::operators_table.end())
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
++pos;
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
bool allow_trailing_commas;
|
||||||
};
|
};
|
||||||
|
|
||||||
/// Basic layer for a function with certain separator and end tokens:
|
/// Basic layer for a function with certain separator and end tokens:
|
||||||
@ -2164,44 +2253,10 @@ bool ParseTimestampOperatorExpression(IParser::Pos & pos, ASTPtr & node, Expecte
|
|||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
struct ParserExpressionImpl
|
|
||||||
{
|
|
||||||
static std::vector<std::pair<const char *, Operator>> operators_table;
|
|
||||||
static std::vector<std::pair<const char *, Operator>> unary_operators_table;
|
|
||||||
static const char * overlapping_operators_to_skip[];
|
|
||||||
|
|
||||||
static Operator finish_between_operator;
|
|
||||||
|
|
||||||
ParserCompoundIdentifier identifier_parser{false, true};
|
|
||||||
ParserNumber number_parser;
|
|
||||||
ParserAsterisk asterisk_parser;
|
|
||||||
ParserLiteral literal_parser;
|
|
||||||
ParserTupleOfLiterals tuple_literal_parser;
|
|
||||||
ParserArrayOfLiterals array_literal_parser;
|
|
||||||
ParserSubstitution substitution_parser;
|
|
||||||
ParserMySQLGlobalVariable mysql_global_variable_parser;
|
|
||||||
|
|
||||||
ParserKeyword any_parser{"ANY"};
|
|
||||||
ParserKeyword all_parser{"ALL"};
|
|
||||||
|
|
||||||
// Recursion
|
|
||||||
ParserQualifiedAsterisk qualified_asterisk_parser;
|
|
||||||
ParserColumnsMatcher columns_matcher_parser;
|
|
||||||
ParserQualifiedColumnsMatcher qualified_columns_matcher_parser;
|
|
||||||
ParserSubquery subquery_parser;
|
|
||||||
|
|
||||||
bool parse(std::unique_ptr<Layer> start, IParser::Pos & pos, ASTPtr & node, Expected & expected);
|
|
||||||
|
|
||||||
using Layers = std::vector<std::unique_ptr<Layer>>;
|
|
||||||
|
|
||||||
Action tryParseOperand(Layers & layers, IParser::Pos & pos, Expected & expected);
|
|
||||||
Action tryParseOperator(Layers & layers, IParser::Pos & pos, Expected & expected);
|
|
||||||
};
|
|
||||||
|
|
||||||
|
|
||||||
bool ParserExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
bool ParserExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||||
{
|
{
|
||||||
auto start = std::make_unique<ExpressionLayer>(false);
|
auto start = std::make_unique<ExpressionLayer>(false, allow_trailing_commas);
|
||||||
return ParserExpressionImpl().parse(std::move(start), pos, node, expected);
|
return ParserExpressionImpl().parse(std::move(start), pos, node, expected);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -2544,18 +2599,17 @@ Action ParserExpressionImpl::tryParseOperator(Layers & layers, IParser::Pos & po
|
|||||||
|
|
||||||
if (cur_op == operators_table.end())
|
if (cur_op == operators_table.end())
|
||||||
{
|
{
|
||||||
|
if (!layers.back()->allow_alias || layers.back()->parsed_alias)
|
||||||
|
return Action::NONE;
|
||||||
|
|
||||||
ASTPtr alias;
|
ASTPtr alias;
|
||||||
ParserAlias alias_parser(layers.back()->allow_alias_without_as_keyword);
|
ParserAlias alias_parser(layers.back()->allow_alias_without_as_keyword);
|
||||||
|
|
||||||
if (layers.back()->allow_alias &&
|
if (!alias_parser.parse(pos, alias, expected) || !layers.back()->insertAlias(alias))
|
||||||
!layers.back()->parsed_alias &&
|
return Action::NONE;
|
||||||
alias_parser.parse(pos, alias, expected) &&
|
|
||||||
layers.back()->insertAlias(alias))
|
layers.back()->parsed_alias = true;
|
||||||
{
|
return Action::OPERATOR;
|
||||||
layers.back()->parsed_alias = true;
|
|
||||||
return Action::OPERATOR;
|
|
||||||
}
|
|
||||||
return Action::NONE;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
auto op = cur_op->second;
|
auto op = cur_op->second;
|
||||||
|
@ -172,10 +172,15 @@ protected:
|
|||||||
|
|
||||||
class ParserExpression : public IParserBase
|
class ParserExpression : public IParserBase
|
||||||
{
|
{
|
||||||
|
public:
|
||||||
|
ParserExpression(bool allow_trailing_commas_ = false) : allow_trailing_commas(allow_trailing_commas_) {}
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
const char * getName() const override { return "lambda expression"; }
|
const char * getName() const override { return "lambda expression"; }
|
||||||
|
|
||||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||||
|
|
||||||
|
bool allow_trailing_commas;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
@ -192,7 +197,7 @@ protected:
|
|||||||
class ParserExpressionWithOptionalAlias : public IParserBase
|
class ParserExpressionWithOptionalAlias : public IParserBase
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
explicit ParserExpressionWithOptionalAlias(bool allow_alias_without_as_keyword_, bool is_table_function_ = false);
|
explicit ParserExpressionWithOptionalAlias(bool allow_alias_without_as_keyword_, bool is_table_function_ = false, bool allow_trailing_commas_ = false);
|
||||||
protected:
|
protected:
|
||||||
ParserPtr impl;
|
ParserPtr impl;
|
||||||
|
|
||||||
@ -209,12 +214,15 @@ protected:
|
|||||||
class ParserExpressionList : public IParserBase
|
class ParserExpressionList : public IParserBase
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
explicit ParserExpressionList(bool allow_alias_without_as_keyword_, bool is_table_function_ = false)
|
explicit ParserExpressionList(bool allow_alias_without_as_keyword_, bool is_table_function_ = false, bool allow_trailing_commas_ = false)
|
||||||
: allow_alias_without_as_keyword(allow_alias_without_as_keyword_), is_table_function(is_table_function_) {}
|
: allow_alias_without_as_keyword(allow_alias_without_as_keyword_)
|
||||||
|
, is_table_function(is_table_function_)
|
||||||
|
, allow_trailing_commas(allow_trailing_commas_) {}
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
bool allow_alias_without_as_keyword;
|
bool allow_alias_without_as_keyword;
|
||||||
bool is_table_function; // This expression list is used by a table function
|
bool is_table_function; // This expression list is used by a table function
|
||||||
|
bool allow_trailing_commas;
|
||||||
|
|
||||||
const char * getName() const override { return "list of expressions"; }
|
const char * getName() const override { return "list of expressions"; }
|
||||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||||
@ -224,8 +232,8 @@ protected:
|
|||||||
class ParserNotEmptyExpressionList : public IParserBase
|
class ParserNotEmptyExpressionList : public IParserBase
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
explicit ParserNotEmptyExpressionList(bool allow_alias_without_as_keyword)
|
explicit ParserNotEmptyExpressionList(bool allow_alias_without_as_keyword_, bool allow_trailing_commas_ = false)
|
||||||
: nested_parser(allow_alias_without_as_keyword) {}
|
: nested_parser(allow_alias_without_as_keyword_, false, allow_trailing_commas_) {}
|
||||||
private:
|
private:
|
||||||
ParserExpressionList nested_parser;
|
ParserExpressionList nested_parser;
|
||||||
protected:
|
protected:
|
||||||
|
@ -68,7 +68,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
|||||||
|
|
||||||
ParserNotEmptyExpressionList exp_list(false);
|
ParserNotEmptyExpressionList exp_list(false);
|
||||||
ParserNotEmptyExpressionList exp_list_for_with_clause(false);
|
ParserNotEmptyExpressionList exp_list_for_with_clause(false);
|
||||||
ParserNotEmptyExpressionList exp_list_for_select_clause(true); /// Allows aliases without AS keyword.
|
ParserNotEmptyExpressionList exp_list_for_select_clause(/*allow_alias_without_as_keyword*/ true, /*allow_trailing_commas*/ true);
|
||||||
ParserExpressionWithOptionalAlias exp_elem(false);
|
ParserExpressionWithOptionalAlias exp_elem(false);
|
||||||
ParserOrderByExpressionList order_list;
|
ParserOrderByExpressionList order_list;
|
||||||
ParserGroupingSetsExpressionList grouping_sets_list;
|
ParserGroupingSetsExpressionList grouping_sets_list;
|
||||||
|
@ -225,7 +225,6 @@ bool ParserTablesInSelectQueryElement::parseImpl(Pos & pos, ASTPtr & node, Expec
|
|||||||
}
|
}
|
||||||
else if (ParserKeyword("ON").ignore(pos, expected))
|
else if (ParserKeyword("ON").ignore(pos, expected))
|
||||||
{
|
{
|
||||||
/// OR is operator with lowest priority, so start parsing from it.
|
|
||||||
if (!ParserExpression().parse(pos, table_join->on_expression, expected))
|
if (!ParserExpression().parse(pos, table_join->on_expression, expected))
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
@ -7,6 +7,7 @@
|
|||||||
#include <Storages/IStorage.h>
|
#include <Storages/IStorage.h>
|
||||||
#include <Processors/Sinks/SinkToStorage.h>
|
#include <Processors/Sinks/SinkToStorage.h>
|
||||||
#include <Common/Stopwatch.h>
|
#include <Common/Stopwatch.h>
|
||||||
|
#include <Common/ThreadStatus.h>
|
||||||
|
|
||||||
namespace Poco
|
namespace Poco
|
||||||
{
|
{
|
||||||
|
@ -5,6 +5,7 @@ Jan Jan
|
|||||||
02 02
|
02 02
|
||||||
01/02/18 01/02/18
|
01/02/18 01/02/18
|
||||||
2 2
|
2 2
|
||||||
|
000000 000000
|
||||||
2018-01-02 2018-01-02
|
2018-01-02 2018-01-02
|
||||||
10 12
|
10 12
|
||||||
22 00
|
22 00
|
||||||
@ -43,11 +44,23 @@ no formatting pattern no formatting pattern
|
|||||||
-1100
|
-1100
|
||||||
+0300
|
+0300
|
||||||
+0530
|
+0530
|
||||||
1234560
|
000000
|
||||||
000340
|
000000
|
||||||
|
000000
|
||||||
|
000000
|
||||||
|
123
|
||||||
|
123456
|
||||||
|
123456789
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
123
|
||||||
|
123456
|
||||||
|
123456789
|
||||||
2022-12-08 18:11:29.123400000
|
2022-12-08 18:11:29.123400000
|
||||||
2022-12-08 18:11:29.1
|
2022-12-08 18:11:29.1
|
||||||
2022-12-08 18:11:29.0
|
2022-12-08 18:11:29.000000
|
||||||
2022-12-08 18:11:29.0
|
2022-12-08 18:11:29.000000
|
||||||
2022-12-08 00:00:00.0
|
2022-12-08 00:00:00.000000
|
||||||
2022-12-08 00:00:00.0
|
2022-12-08 00:00:00.000000
|
||||||
|
@ -17,6 +17,7 @@ SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%C'), formatDateTime(t
|
|||||||
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%d'), formatDateTime(toDate32('2018-01-02'), '%d');
|
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%d'), formatDateTime(toDate32('2018-01-02'), '%d');
|
||||||
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%D'), formatDateTime(toDate32('2018-01-02'), '%D');
|
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%D'), formatDateTime(toDate32('2018-01-02'), '%D');
|
||||||
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%e'), formatDateTime(toDate32('2018-01-02'), '%e');
|
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%e'), formatDateTime(toDate32('2018-01-02'), '%e');
|
||||||
|
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%f'), formatDateTime(toDate32('2018-01-02'), '%f');
|
||||||
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%F'), formatDateTime(toDate32('2018-01-02'), '%F');
|
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%F'), formatDateTime(toDate32('2018-01-02'), '%F');
|
||||||
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%h'), formatDateTime(toDate32('2018-01-02'), '%h');
|
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%h'), formatDateTime(toDate32('2018-01-02'), '%h');
|
||||||
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%H'), formatDateTime(toDate32('2018-01-02'), '%H');
|
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%H'), formatDateTime(toDate32('2018-01-02'), '%H');
|
||||||
@ -66,8 +67,22 @@ SELECT formatDateTime(toDateTime('2020-01-01 01:00:00', 'US/Samoa'), '%z');
|
|||||||
SELECT formatDateTime(toDateTime('2020-01-01 01:00:00', 'Europe/Moscow'), '%z');
|
SELECT formatDateTime(toDateTime('2020-01-01 01:00:00', 'Europe/Moscow'), '%z');
|
||||||
SELECT formatDateTime(toDateTime('1970-01-01 00:00:00', 'Asia/Kolkata'), '%z');
|
SELECT formatDateTime(toDateTime('1970-01-01 00:00:00', 'Asia/Kolkata'), '%z');
|
||||||
|
|
||||||
select formatDateTime(toDateTime64('2010-01-04 12:34:56.123456', 7), '%f');
|
-- %f (default settings)
|
||||||
select formatDateTime(toDateTime64('2022-12-08 18:11:29.00034', 6, 'UTC'), '%f');
|
select formatDateTime(toDate('2010-01-04'), '%f') SETTINGS formatdatetime_f_prints_single_zero = 0;
|
||||||
|
select formatDateTime(toDate32('2010-01-04'), '%f') SETTINGS formatdatetime_f_prints_single_zero = 0;
|
||||||
|
select formatDateTime(toDateTime('2010-01-04 12:34:56'), '%f') SETTINGS formatdatetime_f_prints_single_zero = 0;
|
||||||
|
select formatDateTime(toDateTime64('2010-01-04 12:34:56', 0), '%f') SETTINGS formatdatetime_f_prints_single_zero = 0;
|
||||||
|
select formatDateTime(toDateTime64('2010-01-04 12:34:56.123', 3), '%f') SETTINGS formatdatetime_f_prints_single_zero = 0;
|
||||||
|
select formatDateTime(toDateTime64('2010-01-04 12:34:56.123456', 6), '%f') SETTINGS formatdatetime_f_prints_single_zero = 0;
|
||||||
|
select formatDateTime(toDateTime64('2010-01-04 12:34:56.123456789', 9), '%f') SETTINGS formatdatetime_f_prints_single_zero = 0;
|
||||||
|
-- %f (legacy settings)
|
||||||
|
select formatDateTime(toDate('2010-01-04'), '%f') SETTINGS formatdatetime_f_prints_single_zero = 1;
|
||||||
|
select formatDateTime(toDate32('2010-01-04'), '%f') SETTINGS formatdatetime_f_prints_single_zero = 1;
|
||||||
|
select formatDateTime(toDateTime('2010-01-04 12:34:56'), '%f') SETTINGS formatdatetime_f_prints_single_zero = 1;
|
||||||
|
select formatDateTime(toDateTime64('2010-01-04 12:34:56', 0), '%f') SETTINGS formatdatetime_f_prints_single_zero = 1;
|
||||||
|
select formatDateTime(toDateTime64('2010-01-04 12:34:56.123', 3), '%f') SETTINGS formatdatetime_f_prints_single_zero = 1;
|
||||||
|
select formatDateTime(toDateTime64('2010-01-04 12:34:56.123456', 6), '%f') SETTINGS formatdatetime_f_prints_single_zero = 0;
|
||||||
|
select formatDateTime(toDateTime64('2010-01-04 12:34:56.123456789', 9), '%f') SETTINGS formatdatetime_f_prints_single_zero = 1;
|
||||||
|
|
||||||
select formatDateTime(toDateTime64('2022-12-08 18:11:29.1234', 9, 'UTC'), '%F %T.%f');
|
select formatDateTime(toDateTime64('2022-12-08 18:11:29.1234', 9, 'UTC'), '%F %T.%f');
|
||||||
select formatDateTime(toDateTime64('2022-12-08 18:11:29.1234', 1, 'UTC'), '%F %T.%f');
|
select formatDateTime(toDateTime64('2022-12-08 18:11:29.1234', 1, 'UTC'), '%F %T.%f');
|
||||||
|
@ -209,12 +209,19 @@ select parseDateTime('00/', '%s/', 'UTC') = toDateTime('1970-01-01 00:00:00', 'U
|
|||||||
select parseDateTime('60', '%s', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME }
|
select parseDateTime('60', '%s', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME }
|
||||||
select parseDateTime('-1', '%s', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME }
|
select parseDateTime('-1', '%s', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME }
|
||||||
select parseDateTime('123456789', '%s', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME }
|
select parseDateTime('123456789', '%s', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME }
|
||||||
|
-- microsecond
|
||||||
|
select parseDateTime('000000', '%f', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC');
|
||||||
|
1
|
||||||
|
select parseDateTime('456789', '%f', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC');
|
||||||
|
1
|
||||||
|
select parseDateTime('42', '%f', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -- { serverError NOT_ENOUGH_SPACE }
|
||||||
|
select parseDateTime('12ABCD', '%f', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME }
|
||||||
-- mixed YMD format
|
-- mixed YMD format
|
||||||
select parseDateTime('2021-01-04+23:00:00', '%Y-%m-%d+%H:%i:%s', 'UTC') = toDateTime('2021-01-04 23:00:00', 'UTC');
|
select parseDateTime('2021-01-04+23:00:00.654321', '%Y-%m-%d+%H:%i:%s.%f', 'UTC') = toDateTime('2021-01-04 23:00:00', 'UTC');
|
||||||
1
|
1
|
||||||
select parseDateTime('2019-07-03 11:04:10', '%Y-%m-%d %H:%i:%s', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC');
|
select parseDateTime('2019-07-03 11:04:10.975319', '%Y-%m-%d %H:%i:%s.%f', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC');
|
||||||
1
|
1
|
||||||
select parseDateTime('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC');
|
select parseDateTime('10:04:11 03-07-2019.242424', '%s:%i:%H %d-%m-%Y.%f', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC');
|
||||||
1
|
1
|
||||||
-- *OrZero, *OrNull, str_to_date
|
-- *OrZero, *OrNull, str_to_date
|
||||||
select parseDateTimeOrZero('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC');
|
select parseDateTimeOrZero('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC');
|
||||||
|
@ -140,10 +140,17 @@ select parseDateTime('00/', '%s/', 'UTC') = toDateTime('1970-01-01 00:00:00', 'U
|
|||||||
select parseDateTime('60', '%s', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME }
|
select parseDateTime('60', '%s', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME }
|
||||||
select parseDateTime('-1', '%s', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME }
|
select parseDateTime('-1', '%s', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME }
|
||||||
select parseDateTime('123456789', '%s', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME }
|
select parseDateTime('123456789', '%s', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME }
|
||||||
|
|
||||||
|
-- microsecond
|
||||||
|
select parseDateTime('000000', '%f', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC');
|
||||||
|
select parseDateTime('456789', '%f', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC');
|
||||||
|
select parseDateTime('42', '%f', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -- { serverError NOT_ENOUGH_SPACE }
|
||||||
|
select parseDateTime('12ABCD', '%f', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME }
|
||||||
|
|
||||||
-- mixed YMD format
|
-- mixed YMD format
|
||||||
select parseDateTime('2021-01-04+23:00:00', '%Y-%m-%d+%H:%i:%s', 'UTC') = toDateTime('2021-01-04 23:00:00', 'UTC');
|
select parseDateTime('2021-01-04+23:00:00.654321', '%Y-%m-%d+%H:%i:%s.%f', 'UTC') = toDateTime('2021-01-04 23:00:00', 'UTC');
|
||||||
select parseDateTime('2019-07-03 11:04:10', '%Y-%m-%d %H:%i:%s', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC');
|
select parseDateTime('2019-07-03 11:04:10.975319', '%Y-%m-%d %H:%i:%s.%f', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC');
|
||||||
select parseDateTime('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC');
|
select parseDateTime('10:04:11 03-07-2019.242424', '%s:%i:%H %d-%m-%Y.%f', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC');
|
||||||
|
|
||||||
-- *OrZero, *OrNull, str_to_date
|
-- *OrZero, *OrNull, str_to_date
|
||||||
select parseDateTimeOrZero('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC');
|
select parseDateTimeOrZero('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC');
|
||||||
|
@ -0,0 +1,5 @@
|
|||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1 2 0
|
||||||
|
1
|
5
tests/queries/0_stateless/02676_trailing_commas.sql
Normal file
5
tests/queries/0_stateless/02676_trailing_commas.sql
Normal file
@ -0,0 +1,5 @@
|
|||||||
|
SELECT 1,;
|
||||||
|
SELECT 1, FROM numbers(1);
|
||||||
|
WITH 1 as a SELECT a, FROM numbers(1);
|
||||||
|
WITH 1 as from SELECT from, from + from, from in [0], FROM numbers(1);
|
||||||
|
SELECT n, FROM (SELECT 1 AS n);
|
Loading…
Reference in New Issue
Block a user