Merge branch 'master' into materialized-pg-allow-unique-identifier

This commit is contained in:
Kseniia Sumarokova 2023-10-16 22:18:23 +02:00 committed by GitHub
commit c31299d86c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
110 changed files with 1431 additions and 1025 deletions

View File

@ -131,29 +131,3 @@ void sort(RandomIt first, RandomIt last)
using comparator = std::less<value_type>; using comparator = std::less<value_type>;
::sort(first, last, comparator()); ::sort(first, last, comparator());
} }
/** Try to fast sort elements for common sorting patterns:
* 1. If elements are already sorted.
* 2. If elements are already almost sorted.
* 3. If elements are already sorted in reverse order.
*
* Returns true if fast sort was performed or elements were already sorted, false otherwise.
*/
template <typename RandomIt, typename Compare>
bool trySort(RandomIt first, RandomIt last, Compare compare)
{
#ifndef NDEBUG
::shuffle(first, last);
#endif
ComparatorWrapper<Compare> compare_wrapper = compare;
return ::pdqsort_try_sort(first, last, compare_wrapper);
}
template <typename RandomIt>
bool trySort(RandomIt first, RandomIt last)
{
using value_type = typename std::iterator_traits<RandomIt>::value_type;
using comparator = std::less<value_type>;
return ::trySort(first, last, comparator());
}

View File

@ -54,10 +54,8 @@ namespace pdqsort_detail {
block_size = 64, block_size = 64,
// Cacheline size, assumes power of two. // Cacheline size, assumes power of two.
cacheline_size = 64, cacheline_size = 64
/// Try sort allowed iterations
try_sort_iterations = 3,
}; };
#if __cplusplus >= 201103L #if __cplusplus >= 201103L
@ -503,167 +501,6 @@ namespace pdqsort_detail {
leftmost = false; leftmost = false;
} }
} }
template<class Iter, class Compare, bool Branchless>
inline bool pdqsort_try_sort_loop(Iter begin,
Iter end,
Compare comp,
size_t bad_allowed,
size_t iterations_allowed,
bool force_sort = false,
bool leftmost = true) {
typedef typename std::iterator_traits<Iter>::difference_type diff_t;
// Use a while loop for tail recursion elimination.
while (true) {
if (!force_sort && iterations_allowed == 0) {
return false;
}
diff_t size = end - begin;
// Insertion sort is faster for small arrays.
if (size < insertion_sort_threshold) {
if (leftmost) insertion_sort(begin, end, comp);
else unguarded_insertion_sort(begin, end, comp);
return true;
}
// Choose pivot as median of 3 or pseudomedian of 9.
diff_t s2 = size / 2;
if (size > ninther_threshold) {
sort3(begin, begin + s2, end - 1, comp);
sort3(begin + 1, begin + (s2 - 1), end - 2, comp);
sort3(begin + 2, begin + (s2 + 1), end - 3, comp);
sort3(begin + (s2 - 1), begin + s2, begin + (s2 + 1), comp);
std::iter_swap(begin, begin + s2);
} else sort3(begin + s2, begin, end - 1, comp);
// If *(begin - 1) is the end of the right partition of a previous partition operation
// there is no element in [begin, end) that is smaller than *(begin - 1). Then if our
// pivot compares equal to *(begin - 1) we change strategy, putting equal elements in
// the left partition, greater elements in the right partition. We do not have to
// recurse on the left partition, since it's sorted (all equal).
if (!leftmost && !comp(*(begin - 1), *begin)) {
begin = partition_left(begin, end, comp) + 1;
continue;
}
// Partition and get results.
std::pair<Iter, bool> part_result =
Branchless ? partition_right_branchless(begin, end, comp)
: partition_right(begin, end, comp);
Iter pivot_pos = part_result.first;
bool already_partitioned = part_result.second;
// Check for a highly unbalanced partition.
diff_t l_size = pivot_pos - begin;
diff_t r_size = end - (pivot_pos + 1);
bool highly_unbalanced = l_size < size / 8 || r_size < size / 8;
// If we got a highly unbalanced partition we shuffle elements to break many patterns.
if (highly_unbalanced) {
if (!force_sort) {
return false;
}
// If we had too many bad partitions, switch to heapsort to guarantee O(n log n).
if (--bad_allowed == 0) {
std::make_heap(begin, end, comp);
std::sort_heap(begin, end, comp);
return true;
}
if (l_size >= insertion_sort_threshold) {
std::iter_swap(begin, begin + l_size / 4);
std::iter_swap(pivot_pos - 1, pivot_pos - l_size / 4);
if (l_size > ninther_threshold) {
std::iter_swap(begin + 1, begin + (l_size / 4 + 1));
std::iter_swap(begin + 2, begin + (l_size / 4 + 2));
std::iter_swap(pivot_pos - 2, pivot_pos - (l_size / 4 + 1));
std::iter_swap(pivot_pos - 3, pivot_pos - (l_size / 4 + 2));
}
}
if (r_size >= insertion_sort_threshold) {
std::iter_swap(pivot_pos + 1, pivot_pos + (1 + r_size / 4));
std::iter_swap(end - 1, end - r_size / 4);
if (r_size > ninther_threshold) {
std::iter_swap(pivot_pos + 2, pivot_pos + (2 + r_size / 4));
std::iter_swap(pivot_pos + 3, pivot_pos + (3 + r_size / 4));
std::iter_swap(end - 2, end - (1 + r_size / 4));
std::iter_swap(end - 3, end - (2 + r_size / 4));
}
}
} else {
// If we were decently balanced and we tried to sort an already partitioned
// sequence try to use insertion sort.
if (already_partitioned && partial_insertion_sort(begin, pivot_pos, comp)
&& partial_insertion_sort(pivot_pos + 1, end, comp)) {
return true;
}
}
// Sort the left partition first using recursion and do tail recursion elimination for
// the right-hand partition.
if (pdqsort_try_sort_loop<Iter, Compare, Branchless>(begin,
pivot_pos,
comp,
bad_allowed,
iterations_allowed - 1,
force_sort,
leftmost)) {
force_sort = true;
} else {
return false;
}
--iterations_allowed;
begin = pivot_pos + 1;
leftmost = false;
}
return false;
}
template<class Iter, class Compare, bool Branchless>
inline bool pdqsort_try_sort_impl(Iter begin, Iter end, Compare comp, size_t bad_allowed)
{
typedef typename std::iterator_traits<Iter>::difference_type diff_t;
static constexpr size_t iterations_allowed = pdqsort_detail::try_sort_iterations;
static constexpr size_t num_to_try = 16;
diff_t size = end - begin;
if (size > num_to_try * 10)
{
size_t out_of_order_elements = 0;
for (size_t i = 1; i < num_to_try; ++i)
{
diff_t offset = size / num_to_try;
diff_t prev_position = offset * (i - 1);
diff_t curr_position = offset * i;
diff_t next_position = offset * (i + 1) - 1;
bool prev_less_than_curr = comp(*(begin + prev_position), *(begin + curr_position));
bool curr_less_than_next = comp(*(begin + curr_position), *(begin + next_position));
if ((prev_less_than_curr && curr_less_than_next) || (!prev_less_than_curr && !curr_less_than_next))
continue;
++out_of_order_elements;
if (out_of_order_elements > iterations_allowed)
return false;
}
}
return pdqsort_try_sort_loop<Iter, Compare, Branchless>(begin, end, comp, bad_allowed, iterations_allowed);
}
} }
@ -701,41 +538,6 @@ inline void pdqsort_branchless(Iter begin, Iter end) {
pdqsort_branchless(begin, end, std::less<T>()); pdqsort_branchless(begin, end, std::less<T>());
} }
template<class Iter, class Compare>
inline bool pdqsort_try_sort(Iter begin, Iter end, Compare comp) {
if (begin == end) return true;
#if __cplusplus >= 201103L
return pdqsort_detail::pdqsort_try_sort_impl<Iter, Compare,
pdqsort_detail::is_default_compare<typename std::decay<Compare>::type>::value &&
std::is_arithmetic<typename std::iterator_traits<Iter>::value_type>::value>(
begin, end, comp, pdqsort_detail::log2(end - begin));
#else
return pdqsort_detail::pdqsort_try_sort_impl<Iter, Compare, false>(
begin, end, comp, pdqsort_detail::log2(end - begin));
#endif
}
template<class Iter>
inline bool pdqsort_try_sort(Iter begin, Iter end) {
typedef typename std::iterator_traits<Iter>::value_type T;
return pdqsort_try_sort(begin, end, std::less<T>());
}
template<class Iter, class Compare>
inline bool pdqsort_try_sort_branchless(Iter begin, Iter end, Compare comp) {
if (begin == end) return true;
return pdqsort_detail::pdqsort_try_sort_impl<Iter, Compare, true>(
begin, end, comp, pdqsort_detail::log2(end - begin));
}
template<class Iter>
inline bool pdqsort_try_sort_branchless(Iter begin, Iter end) {
typedef typename std::iterator_traits<Iter>::value_type T;
return pdqsort_try_sort_branchless(begin, end, std::less<T>());
}
#undef PDQSORT_PREFER_MOVE #undef PDQSORT_PREFER_MOVE

View File

@ -104,66 +104,76 @@ if [ -n "$CLICKHOUSE_USER" ] && [ "$CLICKHOUSE_USER" != "default" ] || [ -n "$CL
EOT EOT
fi fi
if [ -n "$(ls /docker-entrypoint-initdb.d/)" ] || [ -n "$CLICKHOUSE_DB" ]; then # checking $DATA_DIR for initialization
# port is needed to check if clickhouse-server is ready for connections if [ -d "${DATA_DIR%/}/data" ]; then
HTTP_PORT="$(clickhouse extract-from-config --config-file "$CLICKHOUSE_CONFIG" --key=http_port --try)" DATABASE_ALREADY_EXISTS='true'
HTTPS_PORT="$(clickhouse extract-from-config --config-file "$CLICKHOUSE_CONFIG" --key=https_port --try)" fi
if [ -n "$HTTP_PORT" ]; then # only run initialization on an empty data directory
URL="http://127.0.0.1:$HTTP_PORT/ping" if [ -z "${DATABASE_ALREADY_EXISTS}" ]; then
else if [ -n "$(ls /docker-entrypoint-initdb.d/)" ] || [ -n "$CLICKHOUSE_DB" ]; then
URL="https://127.0.0.1:$HTTPS_PORT/ping" # port is needed to check if clickhouse-server is ready for connections
fi HTTP_PORT="$(clickhouse extract-from-config --config-file "$CLICKHOUSE_CONFIG" --key=http_port --try)"
HTTPS_PORT="$(clickhouse extract-from-config --config-file "$CLICKHOUSE_CONFIG" --key=https_port --try)"
# Listen only on localhost until the initialization is done if [ -n "$HTTP_PORT" ]; then
/usr/bin/clickhouse su "${USER}:${GROUP}" /usr/bin/clickhouse-server --config-file="$CLICKHOUSE_CONFIG" -- --listen_host=127.0.0.1 & URL="http://127.0.0.1:$HTTP_PORT/ping"
pid="$!" else
URL="https://127.0.0.1:$HTTPS_PORT/ping"
fi
# check if clickhouse is ready to accept connections # Listen only on localhost until the initialization is done
# will try to send ping clickhouse via http_port (max 1000 retries by default, with 1 sec timeout and 1 sec delay between retries) /usr/bin/clickhouse su "${USER}:${GROUP}" /usr/bin/clickhouse-server --config-file="$CLICKHOUSE_CONFIG" -- --listen_host=127.0.0.1 &
tries=${CLICKHOUSE_INIT_TIMEOUT:-1000} pid="$!"
while ! wget --spider --no-check-certificate -T 1 -q "$URL" 2>/dev/null; do
if [ "$tries" -le "0" ]; then # check if clickhouse is ready to accept connections
echo >&2 'ClickHouse init process failed.' # will try to send ping clickhouse via http_port (max 1000 retries by default, with 1 sec timeout and 1 sec delay between retries)
tries=${CLICKHOUSE_INIT_TIMEOUT:-1000}
while ! wget --spider --no-check-certificate -T 1 -q "$URL" 2>/dev/null; do
if [ "$tries" -le "0" ]; then
echo >&2 'ClickHouse init process failed.'
exit 1
fi
tries=$(( tries-1 ))
sleep 1
done
clickhouseclient=( clickhouse-client --multiquery --host "127.0.0.1" -u "$CLICKHOUSE_USER" --password "$CLICKHOUSE_PASSWORD" )
echo
# create default database, if defined
if [ -n "$CLICKHOUSE_DB" ]; then
echo "$0: create database '$CLICKHOUSE_DB'"
"${clickhouseclient[@]}" -q "CREATE DATABASE IF NOT EXISTS $CLICKHOUSE_DB";
fi
for f in /docker-entrypoint-initdb.d/*; do
case "$f" in
*.sh)
if [ -x "$f" ]; then
echo "$0: running $f"
"$f"
else
echo "$0: sourcing $f"
# shellcheck source=/dev/null
. "$f"
fi
;;
*.sql) echo "$0: running $f"; "${clickhouseclient[@]}" < "$f" ; echo ;;
*.sql.gz) echo "$0: running $f"; gunzip -c "$f" | "${clickhouseclient[@]}"; echo ;;
*) echo "$0: ignoring $f" ;;
esac
echo
done
if ! kill -s TERM "$pid" || ! wait "$pid"; then
echo >&2 'Finishing of ClickHouse init process failed.'
exit 1 exit 1
fi fi
tries=$(( tries-1 ))
sleep 1
done
clickhouseclient=( clickhouse-client --multiquery --host "127.0.0.1" -u "$CLICKHOUSE_USER" --password "$CLICKHOUSE_PASSWORD" )
echo
# create default database, if defined
if [ -n "$CLICKHOUSE_DB" ]; then
echo "$0: create database '$CLICKHOUSE_DB'"
"${clickhouseclient[@]}" -q "CREATE DATABASE IF NOT EXISTS $CLICKHOUSE_DB";
fi
for f in /docker-entrypoint-initdb.d/*; do
case "$f" in
*.sh)
if [ -x "$f" ]; then
echo "$0: running $f"
"$f"
else
echo "$0: sourcing $f"
# shellcheck source=/dev/null
. "$f"
fi
;;
*.sql) echo "$0: running $f"; "${clickhouseclient[@]}" < "$f" ; echo ;;
*.sql.gz) echo "$0: running $f"; gunzip -c "$f" | "${clickhouseclient[@]}"; echo ;;
*) echo "$0: ignoring $f" ;;
esac
echo
done
if ! kill -s TERM "$pid" || ! wait "$pid"; then
echo >&2 'Finishing of ClickHouse init process failed.'
exit 1
fi fi
else
echo "ClickHouse Database directory appears to contain a database; Skipping initialization"
fi fi
# if no args passed to `docker run` or first argument start with `--`, then the user is passing clickhouse-server arguments # if no args passed to `docker run` or first argument start with `--`, then the user is passing clickhouse-server arguments

View File

@ -55,6 +55,7 @@ keeper foo bar
- `rmr <path>` -- Recursively deletes path. Confirmation required - `rmr <path>` -- Recursively deletes path. Confirmation required
- `flwc <command>` -- Executes four-letter-word command - `flwc <command>` -- Executes four-letter-word command
- `help` -- Prints this message - `help` -- Prints this message
- `get_all_children_number [path]` -- Get all numbers of children nodes under a specific path
- `get_stat [path]` -- Returns the node's stat (default `.`) - `get_stat [path]` -- Returns the node's stat (default `.`)
- `find_super_nodes <threshold> [path]` -- Finds nodes with number of children larger than some threshold for the given path (default `.`) - `find_super_nodes <threshold> [path]` -- Finds nodes with number of children larger than some threshold for the given path (default `.`)
- `delete_stale_backups` -- Deletes ClickHouse nodes used for backups that are now inactive - `delete_stale_backups` -- Deletes ClickHouse nodes used for backups that are now inactive

View File

@ -587,20 +587,22 @@ The two-argument form of `toDayOfWeek()` enables you to specify whether the week
| 2 | Sunday | 0-6: Sunday = 0, Monday = 1, ..., Saturday = 6 | | 2 | Sunday | 0-6: Sunday = 0, Monday = 1, ..., Saturday = 6 |
| 3 | Sunday | 1-7: Sunday = 1, Monday = 2, ..., Saturday = 7 | | 3 | Sunday | 1-7: Sunday = 1, Monday = 2, ..., Saturday = 7 |
Alias: `DAYOFWEEK`.
**Syntax** **Syntax**
``` sql ``` sql
toDayOfWeek(t[, mode[, timezone]]) toDayOfWeek(t[, mode[, timezone]])
``` ```
Alias: `DAYOFWEEK`.
**Arguments** **Arguments**
- `t` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) - `t` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
- `mode` - determines what the first day of the week is. Possible values are 0, 1, 2 or 3. See the table above for the differences. - `mode` - determines what the first day of the week is. Possible values are 0, 1, 2 or 3. See the table above for the differences.
- `timezone` - optional parameter, it behaves like any other conversion function - `timezone` - optional parameter, it behaves like any other conversion function
The first argument can also be specified as [String](../data-types/string.md) in a format supported by [parseDateTime64BestEffort()](type-conversion-functions.md#parsedatetime64besteffort). Support for string arguments exists only for reasons of compatibility with MySQL which is expected by certain 3rd party tools. As string argument support may in future be made dependent on new MySQL-compatibility settings and because string parsing is generally slow, it is recommended to not use it.
**Returned value** **Returned value**
- The day of the month (1 - 31) of the given date/time - The day of the month (1 - 31) of the given date/time
@ -1489,12 +1491,16 @@ For mode values with a meaning of “contains January 1”, the week contains Ja
toWeek(t[, mode[, time_zone]]) toWeek(t[, mode[, time_zone]])
``` ```
Alias: `WEEK`
**Arguments** **Arguments**
- `t` Date or DateTime. - `t` Date or DateTime.
- `mode` Optional parameter, Range of values is \[0,9\], default is 0. - `mode` Optional parameter, Range of values is \[0,9\], default is 0.
- `Timezone` Optional parameter, it behaves like any other conversion function. - `Timezone` Optional parameter, it behaves like any other conversion function.
The first argument can also be specified as [String](../data-types/string.md) in a format supported by [parseDateTime64BestEffort()](type-conversion-functions.md#parsedatetime64besteffort). Support for string arguments exists only for reasons of compatibility with MySQL which is expected by certain 3rd party tools. As string argument support may in future be made dependent on new MySQL-compatibility settings and because string parsing is generally slow, it is recommended to not use it.
**Example** **Example**
``` sql ``` sql
@ -1525,6 +1531,10 @@ The week number returned by `toYearWeek()` can be different from what the `toWee
toYearWeek(t[, mode[, timezone]]) toYearWeek(t[, mode[, timezone]])
``` ```
Alias: `YEARWEEK`
The first argument can also be specified as [String](../data-types/string.md) in a format supported by [parseDateTime64BestEffort()](type-conversion-functions.md#parsedatetime64besteffort). Support for string arguments exists only for reasons of compatibility with MySQL which is expected by certain 3rd party tools. As string argument support may in future be made dependent on new MySQL-compatibility settings and because string parsing is generally slow, it is recommended to not use it.
**Example** **Example**
``` sql ``` sql

View File

@ -16,7 +16,7 @@ The `RENAME` query is supported by the [Atomic](../../engines/database-engines/a
**Syntax** **Syntax**
```sql ```sql
RENAME DATABASE|TABLE|DICTIONARY name TO new_name [,...] [ON CLUSTER cluster] RENAME [DATABASE|TABLE|DICTIONARY] name TO new_name [,...] [ON CLUSTER cluster]
``` ```
## RENAME DATABASE ## RENAME DATABASE
@ -48,6 +48,11 @@ RENAME TABLE [db1.]name1 TO [db2.]name2 [,...] [ON CLUSTER cluster]
RENAME TABLE table_A TO table_A_bak, table_B TO table_B_bak; RENAME TABLE table_A TO table_A_bak, table_B TO table_B_bak;
``` ```
And you can use a simpler sql:
```sql
RENAME table_A TO table_A_bak, table_B TO table_B_bak;
```
## RENAME DICTIONARY ## RENAME DICTIONARY
Renames one or several dictionaries. This query can be used to move dictionaries between databases. Renames one or several dictionaries. This query can be used to move dictionaries between databases.

View File

@ -324,8 +324,8 @@ void FindBigFamily::execute(const ASTKeeperQuery * query, KeeperClient * client)
queue.pop(); queue.pop();
auto children = client->zookeeper->getChildren(next_path); auto children = client->zookeeper->getChildren(next_path);
std::transform(children.cbegin(), children.cend(), children.begin(), [&](const String & child) { return next_path / child; }); for (auto & child : children)
child = next_path / child;
auto response = client->zookeeper->get(children); auto response = client->zookeeper->get(children);
for (size_t i = 0; i < response.size(); ++i) for (size_t i = 0; i < response.size(); ++i)
@ -475,4 +475,45 @@ void FourLetterWordCommand::execute(const ASTKeeperQuery * query, KeeperClient *
std::cout << client->executeFourLetterCommand(query->args[0].safeGet<String>()) << "\n"; std::cout << client->executeFourLetterCommand(query->args[0].safeGet<String>()) << "\n";
} }
bool GetAllChildrenNumberCommand::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const
{
String path;
if (!parseKeeperPath(pos, expected, path))
path = ".";
node->args.push_back(std::move(path));
return true;
}
void GetAllChildrenNumberCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const
{
auto path = client->getAbsolutePath(query->args[0].safeGet<String>());
std::queue<fs::path> queue;
queue.push(path);
Coordination::Stat stat;
client->zookeeper->get(path, &stat);
int totalNumChildren = stat.numChildren;
while (!queue.empty())
{
auto next_path = queue.front();
queue.pop();
auto children = client->zookeeper->getChildren(next_path);
for (auto & child : children)
child = next_path / child;
auto response = client->zookeeper->get(children);
for (size_t i = 0; i < response.size(); ++i)
{
totalNumChildren += response[i].stat.numChildren;
queue.push(children[i]);
}
}
std::cout << totalNumChildren << "\n";
}
} }

View File

@ -238,4 +238,18 @@ class FourLetterWordCommand : public IKeeperClientCommand
String getHelpMessage() const override { return "{} <command> -- Executes four-letter-word command"; } String getHelpMessage() const override { return "{} <command> -- Executes four-letter-word command"; }
}; };
class GetAllChildrenNumberCommand : public IKeeperClientCommand
{
String getName() const override { return "get_all_children_number"; }
bool parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const override;
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override
{
return "{} [path] -- Get all numbers of children nodes under a specific path";
}
};
} }

View File

@ -2,6 +2,7 @@
#include "Commands.h" #include "Commands.h"
#include <Client/ReplxxLineReader.h> #include <Client/ReplxxLineReader.h>
#include <Client/ClientBase.h> #include <Client/ClientBase.h>
#include "Common/VersionNumber.h"
#include <Common/Config/ConfigProcessor.h> #include <Common/Config/ConfigProcessor.h>
#include <Common/EventNotifier.h> #include <Common/EventNotifier.h>
#include <Common/filesystemHelpers.h> #include <Common/filesystemHelpers.h>
@ -206,6 +207,7 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */)
std::make_shared<SyncCommand>(), std::make_shared<SyncCommand>(),
std::make_shared<HelpCommand>(), std::make_shared<HelpCommand>(),
std::make_shared<FourLetterWordCommand>(), std::make_shared<FourLetterWordCommand>(),
std::make_shared<GetAllChildrenNumberCommand>(),
}); });
String home_path; String home_path;

View File

@ -1474,7 +1474,7 @@ try
{ {
std::lock_guard lock(servers_lock); std::lock_guard lock(servers_lock);
/// We should start interserver communications before (and more imporant shutdown after) tables. /// We should start interserver communications before (and more important shutdown after) tables.
/// Because server can wait for a long-running queries (for example in tcp_handler) after interserver handler was already shut down. /// Because server can wait for a long-running queries (for example in tcp_handler) after interserver handler was already shut down.
/// In this case we will have replicated tables which are unable to send any parts to other replicas, but still can /// In this case we will have replicated tables which are unable to send any parts to other replicas, but still can
/// communicate with zookeeper, execute merges, etc. /// communicate with zookeeper, execute merges, etc.

View File

@ -420,6 +420,10 @@
color: var(--auth-error-color); color: var(--auth-error-color);
} }
#charts > div:only-child .display-only-if-more-than-one-chart {
display: none;
}
/* Source: https://cdn.jsdelivr.net/npm/uplot@1.6.21/dist/uPlot.min.css /* Source: https://cdn.jsdelivr.net/npm/uplot@1.6.21/dist/uPlot.min.css
* It is copy-pasted to lower the number of requests. * It is copy-pasted to lower the number of requests.
*/ */
@ -931,6 +935,9 @@ function insertChart(i) {
saveState(); saveState();
}); });
move.classList.add('display-only-if-more-than-one-chart');
maximize.classList.add('display-only-if-more-than-one-chart');
edit_buttons.appendChild(move); edit_buttons.appendChild(move);
edit_buttons.appendChild(maximize); edit_buttons.appendChild(maximize);
edit_buttons.appendChild(edit); edit_buttons.appendChild(edit);

View File

@ -4,7 +4,6 @@
#include <Common/assert_cast.h> #include <Common/assert_cast.h>
#include <Common/WeakHash.h> #include <Common/WeakHash.h>
#include <Common/HashTable/Hash.h> #include <Common/HashTable/Hash.h>
#include <Common/RadixSort.h>
#include <base/unaligned.h> #include <base/unaligned.h>
#include <base/sort.h> #include <base/sort.h>
@ -16,7 +15,6 @@
#include <Columns/ColumnDecimal.h> #include <Columns/ColumnDecimal.h>
#include <Columns/ColumnCompressed.h> #include <Columns/ColumnCompressed.h>
#include <Columns/MaskOperations.h> #include <Columns/MaskOperations.h>
#include <Columns/RadixSortHelper.h>
#include <Processors/Transforms/ColumnGathererTransform.h> #include <Processors/Transforms/ColumnGathererTransform.h>
@ -161,59 +159,6 @@ void ColumnDecimal<T>::getPermutation(IColumn::PermutationSortDirection directio
return data[lhs] > data[rhs]; return data[lhs] > data[rhs];
}; };
size_t data_size = data.size();
res.resize(data_size);
if (limit >= data_size)
limit = 0;
for (size_t i = 0; i < data_size; ++i)
res[i] = i;
if constexpr (is_arithmetic_v<NativeT> && !is_big_int_v<NativeT>)
{
if (!limit)
{
/// A case for radix sort
/// LSD RadixSort is stable
bool reverse = direction == IColumn::PermutationSortDirection::Descending;
bool ascending = direction == IColumn::PermutationSortDirection::Ascending;
bool sort_is_stable = stability == IColumn::PermutationSortStability::Stable;
/// TODO: LSD RadixSort is currently not stable if direction is descending
bool use_radix_sort = (sort_is_stable && ascending) || !sort_is_stable;
/// Thresholds on size. Lower threshold is arbitrary. Upper threshold is chosen by the type for histogram counters.
if (data_size >= 256 && data_size <= std::numeric_limits<UInt32>::max() && use_radix_sort)
{
for (size_t i = 0; i < data_size; ++i)
res[i] = i;
bool try_sort = false;
if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable)
try_sort = trySort(res.begin(), res.end(), comparator_ascending);
else if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Stable)
try_sort = trySort(res.begin(), res.end(), comparator_ascending_stable);
else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable)
try_sort = trySort(res.begin(), res.end(), comparator_descending);
else
try_sort = trySort(res.begin(), res.end(), comparator_descending_stable);
if (try_sort)
return;
PaddedPODArray<ValueWithIndex<NativeT>> pairs(data_size);
for (UInt32 i = 0; i < static_cast<UInt32>(data_size); ++i)
pairs[i] = {data[i].value, i};
RadixSort<RadixSortTraits<NativeT>>::executeLSD(pairs.data(), data_size, reverse, res.data());
return;
}
}
}
if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable) if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable)
this->getPermutationImpl(limit, res, comparator_ascending, DefaultSort(), DefaultPartialSort()); this->getPermutationImpl(limit, res, comparator_ascending, DefaultSort(), DefaultPartialSort());
else if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Stable) else if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Stable)
@ -246,37 +191,7 @@ void ColumnDecimal<T>::updatePermutation(IColumn::PermutationSortDirection direc
return data[lhs] < data[rhs]; return data[lhs] < data[rhs];
}; };
auto equals_comparator = [this](size_t lhs, size_t rhs) { return data[lhs] == data[rhs]; }; auto equals_comparator = [this](size_t lhs, size_t rhs) { return data[lhs] == data[rhs]; };
auto sort = [&](auto begin, auto end, auto pred) auto sort = [](auto begin, auto end, auto pred) { ::sort(begin, end, pred); };
{
bool reverse = direction == IColumn::PermutationSortDirection::Descending;
bool ascending = direction == IColumn::PermutationSortDirection::Ascending;
bool sort_is_stable = stability == IColumn::PermutationSortStability::Stable;
/// TODO: LSD RadixSort is currently not stable if direction is descending
bool use_radix_sort = (sort_is_stable && ascending) || !sort_is_stable;
size_t size = end - begin;
if (size >= 256 && size <= std::numeric_limits<UInt32>::max() && use_radix_sort)
{
bool try_sort = trySort(begin, end, pred);
if (try_sort)
return;
PaddedPODArray<ValueWithIndex<NativeT>> pairs(size);
size_t index = 0;
for (auto * it = begin; it != end; ++it)
{
pairs[index] = {data[*it].value, static_cast<UInt32>(*it)};
++index;
}
RadixSort<RadixSortTraits<NativeT>>::executeLSD(pairs.data(), size, reverse, res.data());
return;
}
::sort(begin, end, pred);
};
auto partial_sort = [](auto begin, auto mid, auto end, auto pred) { ::partial_sort(begin, mid, end, pred); }; auto partial_sort = [](auto begin, auto mid, auto end, auto pred) { ::partial_sort(begin, mid, end, pred); };
if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable) if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable)

View File

@ -3,7 +3,6 @@
#include <Columns/ColumnsCommon.h> #include <Columns/ColumnsCommon.h>
#include <Columns/ColumnCompressed.h> #include <Columns/ColumnCompressed.h>
#include <Columns/MaskOperations.h> #include <Columns/MaskOperations.h>
#include <Columns/RadixSortHelper.h>
#include <Processors/Transforms/ColumnGathererTransform.h> #include <Processors/Transforms/ColumnGathererTransform.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <Common/Arena.h> #include <Common/Arena.h>
@ -193,6 +192,26 @@ struct ColumnVector<T>::equals
bool operator()(size_t lhs, size_t rhs) const { return CompareHelper<T>::equals(parent.data[lhs], parent.data[rhs], nan_direction_hint); } bool operator()(size_t lhs, size_t rhs) const { return CompareHelper<T>::equals(parent.data[lhs], parent.data[rhs], nan_direction_hint); }
}; };
namespace
{
template <typename T>
struct ValueWithIndex
{
T value;
UInt32 index;
};
template <typename T>
struct RadixSortTraits : RadixSortNumTraits<T>
{
using Element = ValueWithIndex<T>;
using Result = size_t;
static T & extractKey(Element & elem) { return elem.value; }
static size_t extractResult(Element & elem) { return elem.index; }
};
}
#if USE_EMBEDDED_COMPILER #if USE_EMBEDDED_COMPILER
template <typename T> template <typename T>
@ -235,25 +254,35 @@ template <typename T>
void ColumnVector<T>::getPermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, void ColumnVector<T>::getPermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability,
size_t limit, int nan_direction_hint, IColumn::Permutation & res) const size_t limit, int nan_direction_hint, IColumn::Permutation & res) const
{ {
size_t data_size = data.size(); size_t s = data.size();
res.resize(data_size); res.resize(s);
if (data_size == 0) if (s == 0)
return; return;
if (limit >= data_size) if (limit >= s)
limit = 0; limit = 0;
for (size_t i = 0; i < data_size; ++i) if (limit)
res[i] = i;
if constexpr (is_arithmetic_v<T> && !is_big_int_v<T>)
{ {
if (!limit) for (size_t i = 0; i < s; ++i)
{ res[i] = i;
/// A case for radix sort
/// LSD RadixSort is stable
if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable)
::partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this, nan_direction_hint));
else if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Stable)
::partial_sort(res.begin(), res.begin() + limit, res.end(), less_stable(*this, nan_direction_hint));
else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable)
::partial_sort(res.begin(), res.begin() + limit, res.end(), greater(*this, nan_direction_hint));
else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Stable)
::partial_sort(res.begin(), res.begin() + limit, res.end(), greater_stable(*this, nan_direction_hint));
}
else
{
/// A case for radix sort
/// LSD RadixSort is stable
if constexpr (is_arithmetic_v<T> && !is_big_int_v<T>)
{
bool reverse = direction == IColumn::PermutationSortDirection::Descending; bool reverse = direction == IColumn::PermutationSortDirection::Descending;
bool ascending = direction == IColumn::PermutationSortDirection::Ascending; bool ascending = direction == IColumn::PermutationSortDirection::Ascending;
bool sort_is_stable = stability == IColumn::PermutationSortStability::Stable; bool sort_is_stable = stability == IColumn::PermutationSortStability::Stable;
@ -262,27 +291,13 @@ void ColumnVector<T>::getPermutation(IColumn::PermutationSortDirection direction
bool use_radix_sort = (sort_is_stable && ascending && !std::is_floating_point_v<T>) || !sort_is_stable; bool use_radix_sort = (sort_is_stable && ascending && !std::is_floating_point_v<T>) || !sort_is_stable;
/// Thresholds on size. Lower threshold is arbitrary. Upper threshold is chosen by the type for histogram counters. /// Thresholds on size. Lower threshold is arbitrary. Upper threshold is chosen by the type for histogram counters.
if (data_size >= 256 && data_size <= std::numeric_limits<UInt32>::max() && use_radix_sort) if (s >= 256 && s <= std::numeric_limits<UInt32>::max() && use_radix_sort)
{ {
bool try_sort = false; PaddedPODArray<ValueWithIndex<T>> pairs(s);
for (UInt32 i = 0; i < static_cast<UInt32>(s); ++i)
if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable)
try_sort = trySort(res.begin(), res.end(), less(*this, nan_direction_hint));
else if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Stable)
try_sort = trySort(res.begin(), res.end(), less_stable(*this, nan_direction_hint));
else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable)
try_sort = trySort(res.begin(), res.end(), greater(*this, nan_direction_hint));
else
try_sort = trySort(res.begin(), res.end(), greater_stable(*this, nan_direction_hint));
if (try_sort)
return;
PaddedPODArray<ValueWithIndex<T>> pairs(data_size);
for (UInt32 i = 0; i < static_cast<UInt32>(data_size); ++i)
pairs[i] = {data[i], i}; pairs[i] = {data[i], i};
RadixSort<RadixSortTraits<T>>::executeLSD(pairs.data(), data_size, reverse, res.data()); RadixSort<RadixSortTraits<T>>::executeLSD(pairs.data(), s, reverse, res.data());
/// Radix sort treats all NaNs to be greater than all numbers. /// Radix sort treats all NaNs to be greater than all numbers.
/// If the user needs the opposite, we must move them accordingly. /// If the user needs the opposite, we must move them accordingly.
@ -290,9 +305,9 @@ void ColumnVector<T>::getPermutation(IColumn::PermutationSortDirection direction
{ {
size_t nans_to_move = 0; size_t nans_to_move = 0;
for (size_t i = 0; i < data_size; ++i) for (size_t i = 0; i < s; ++i)
{ {
if (isNaN(data[res[reverse ? i : data_size - 1 - i]])) if (isNaN(data[res[reverse ? i : s - 1 - i]]))
++nans_to_move; ++nans_to_move;
else else
break; break;
@ -300,35 +315,38 @@ void ColumnVector<T>::getPermutation(IColumn::PermutationSortDirection direction
if (nans_to_move) if (nans_to_move)
{ {
std::rotate(std::begin(res), std::begin(res) + (reverse ? nans_to_move : data_size - nans_to_move), std::end(res)); std::rotate(std::begin(res), std::begin(res) + (reverse ? nans_to_move : s - nans_to_move), std::end(res));
} }
} }
return; return;
} }
} }
}
if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable) /// Default sorting algorithm.
this->getPermutationImpl(limit, res, less(*this, nan_direction_hint), DefaultSort(), DefaultPartialSort()); for (size_t i = 0; i < s; ++i)
else if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Stable) res[i] = i;
this->getPermutationImpl(limit, res, less_stable(*this, nan_direction_hint), DefaultSort(), DefaultPartialSort());
else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable) if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable)
this->getPermutationImpl(limit, res, greater(*this, nan_direction_hint), DefaultSort(), DefaultPartialSort()); ::sort(res.begin(), res.end(), less(*this, nan_direction_hint));
else else if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Stable)
this->getPermutationImpl(limit, res, greater_stable(*this, nan_direction_hint), DefaultSort(), DefaultPartialSort()); ::sort(res.begin(), res.end(), less_stable(*this, nan_direction_hint));
else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable)
::sort(res.begin(), res.end(), greater(*this, nan_direction_hint));
else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Stable)
::sort(res.begin(), res.end(), greater_stable(*this, nan_direction_hint));
}
} }
template <typename T> template <typename T>
void ColumnVector<T>::updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, void ColumnVector<T>::updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability,
size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const
{ {
bool reverse = direction == IColumn::PermutationSortDirection::Descending;
bool ascending = direction == IColumn::PermutationSortDirection::Ascending;
bool sort_is_stable = stability == IColumn::PermutationSortStability::Stable;
auto sort = [&](auto begin, auto end, auto pred) auto sort = [&](auto begin, auto end, auto pred)
{ {
bool reverse = direction == IColumn::PermutationSortDirection::Descending;
bool ascending = direction == IColumn::PermutationSortDirection::Ascending;
bool sort_is_stable = stability == IColumn::PermutationSortStability::Stable;
/// A case for radix sort /// A case for radix sort
if constexpr (is_arithmetic_v<T> && !is_big_int_v<T>) if constexpr (is_arithmetic_v<T> && !is_big_int_v<T>)
{ {
@ -339,10 +357,6 @@ void ColumnVector<T>::updatePermutation(IColumn::PermutationSortDirection direct
/// Thresholds on size. Lower threshold is arbitrary. Upper threshold is chosen by the type for histogram counters. /// Thresholds on size. Lower threshold is arbitrary. Upper threshold is chosen by the type for histogram counters.
if (size >= 256 && size <= std::numeric_limits<UInt32>::max() && use_radix_sort) if (size >= 256 && size <= std::numeric_limits<UInt32>::max() && use_radix_sort)
{ {
bool try_sort = trySort(begin, end, pred);
if (try_sort)
return;
PaddedPODArray<ValueWithIndex<T>> pairs(size); PaddedPODArray<ValueWithIndex<T>> pairs(size);
size_t index = 0; size_t index = 0;

View File

@ -1,25 +0,0 @@
#pragma once
#include <Common/RadixSort.h>
namespace DB
{
template <typename T>
struct ValueWithIndex
{
T value;
UInt32 index;
};
template <typename T>
struct RadixSortTraits : RadixSortNumTraits<T>
{
using Element = ValueWithIndex<T>;
using Result = size_t;
static T & extractKey(Element & elem) { return elem.value; }
static size_t extractResult(Element & elem) { return elem.index; }
};
}

View File

@ -223,7 +223,7 @@ namespace DB
void CaresPTRResolver::process_possible_timeout(ares_channel channel) void CaresPTRResolver::process_possible_timeout(ares_channel channel)
{ {
/* Call ares_process() unconditonally here, even if we simply timed out /* Call ares_process() unconditionally here, even if we simply timed out
above, as otherwise the ares name resolve won't timeout! */ above, as otherwise the ares name resolve won't timeout! */
ares_process_fd(channel, ARES_SOCKET_BAD, ARES_SOCKET_BAD); ares_process_fd(channel, ARES_SOCKET_BAD, ARES_SOCKET_BAD);
} }

View File

@ -158,6 +158,10 @@ public:
{ {
allow_use_jemalloc_memory.store(value, std::memory_order_relaxed); allow_use_jemalloc_memory.store(value, std::memory_order_relaxed);
} }
bool getAllowUseJemallocMmemory() const
{
return allow_use_jemalloc_memory.load(std::memory_order_relaxed);
}
/** Set limit if it was not set. /** Set limit if it was not set.
* Otherwise, set limit to new value, if new value is greater than previous limit. * Otherwise, set limit to new value, if new value is greater than previous limit.

View File

@ -418,7 +418,7 @@ finish:
/// this two vals are useless, xxx|xxx cannot be trivial nor prefix. /// this two vals are useless, xxx|xxx cannot be trivial nor prefix.
bool next_is_trivial = true; bool next_is_trivial = true;
pos = analyzeImpl(regexp, pos, required_substring, next_is_trivial, next_alternatives); pos = analyzeImpl(regexp, pos, required_substring, next_is_trivial, next_alternatives);
/// For xxx|xxx|xxx, we only conbine the alternatives and return a empty required_substring. /// For xxx|xxx|xxx, we only combine the alternatives and return a empty required_substring.
if (next_alternatives.empty() || shortest_literal_length(next_alternatives) < required_substring.literal.size()) if (next_alternatives.empty() || shortest_literal_length(next_alternatives) < required_substring.literal.size())
{ {
global_alternatives.push_back(required_substring); global_alternatives.push_back(required_substring);

View File

@ -321,7 +321,7 @@ protected:
percolate(ptr); percolate(ptr);
} }
// This is equivallent to one step of bubble sort // This is equivalent to one step of bubble sort
void percolate(Counter * counter) void percolate(Counter * counter)
{ {
while (counter->slot > 0) while (counter->slot > 0)

View File

@ -3,6 +3,7 @@
#include <Common/Config/ConfigProcessor.h> #include <Common/Config/ConfigProcessor.h>
#include <Common/Macros.h> #include <Common/Macros.h>
#include <Common/ThreadPool.h> #include <Common/ThreadPool.h>
#include <Common/callOnce.h>
#include <Core/ServerSettings.h> #include <Core/ServerSettings.h>
@ -14,6 +15,7 @@
namespace ProfileEvents namespace ProfileEvents
{ {
extern const Event ContextLock; extern const Event ContextLock;
extern const Event ContextLockWaitMicroseconds;
} }
namespace CurrentMetrics namespace CurrentMetrics
@ -39,8 +41,8 @@ struct ContextSharedPart : boost::noncopyable
: macros(std::make_unique<Macros>()) : macros(std::make_unique<Macros>())
{} {}
/// For access of most of shared objects. Recursive mutex. /// For access of most of shared objects.
mutable std::recursive_mutex mutex; mutable SharedMutex mutex;
mutable std::mutex keeper_dispatcher_mutex; mutable std::mutex keeper_dispatcher_mutex;
mutable std::shared_ptr<KeeperDispatcher> keeper_dispatcher TSA_GUARDED_BY(keeper_dispatcher_mutex); mutable std::shared_ptr<KeeperDispatcher> keeper_dispatcher TSA_GUARDED_BY(keeper_dispatcher_mutex);
@ -50,13 +52,16 @@ struct ContextSharedPart : boost::noncopyable
String path; /// Path to the data directory, with a slash at the end. String path; /// Path to the data directory, with a slash at the end.
ConfigurationPtr config; /// Global configuration settings. ConfigurationPtr config; /// Global configuration settings.
MultiVersion<Macros> macros; /// Substitutions extracted from config. MultiVersion<Macros> macros; /// Substitutions extracted from config.
OnceFlag schedule_pool_initialized;
mutable std::unique_ptr<BackgroundSchedulePool> schedule_pool; /// A thread pool that can run different jobs in background mutable std::unique_ptr<BackgroundSchedulePool> schedule_pool; /// A thread pool that can run different jobs in background
RemoteHostFilter remote_host_filter; /// Allowed URL from config.xml RemoteHostFilter remote_host_filter; /// Allowed URL from config.xml
///
mutable OnceFlag readers_initialized;
mutable std::unique_ptr<IAsynchronousReader> asynchronous_remote_fs_reader; mutable std::unique_ptr<IAsynchronousReader> asynchronous_remote_fs_reader;
mutable std::unique_ptr<IAsynchronousReader> asynchronous_local_fs_reader; mutable std::unique_ptr<IAsynchronousReader> asynchronous_local_fs_reader;
mutable std::unique_ptr<IAsynchronousReader> synchronous_local_fs_reader; mutable std::unique_ptr<IAsynchronousReader> synchronous_local_fs_reader;
mutable OnceFlag threadpool_writer_initialized;
mutable std::unique_ptr<ThreadPool> threadpool_writer; mutable std::unique_ptr<ThreadPool> threadpool_writer;
mutable ThrottlerPtr remote_read_throttler; /// A server-wide throttler for remote IO reads mutable ThrottlerPtr remote_read_throttler; /// A server-wide throttler for remote IO reads
@ -64,13 +69,14 @@ struct ContextSharedPart : boost::noncopyable
mutable ThrottlerPtr local_read_throttler; /// A server-wide throttler for local IO reads mutable ThrottlerPtr local_read_throttler; /// A server-wide throttler for local IO reads
mutable ThrottlerPtr local_write_throttler; /// A server-wide throttler for local IO writes mutable ThrottlerPtr local_write_throttler; /// A server-wide throttler for local IO writes
}; };
ContextData::ContextData() = default;
ContextData::ContextData(const ContextData &) = default;
Context::Context() = default; Context::Context() = default;
Context::Context(const Context & rhs) : ContextData(rhs), std::enable_shared_from_this<Context>(rhs) {}
Context::~Context() = default; Context::~Context() = default;
Context::Context(const Context &) = default;
Context & Context::operator=(const Context &) = default;
SharedContextHolder::SharedContextHolder(SharedContextHolder &&) noexcept = default; SharedContextHolder::SharedContextHolder(SharedContextHolder &&) noexcept = default;
SharedContextHolder & SharedContextHolder::operator=(SharedContextHolder &&) noexcept = default; SharedContextHolder & SharedContextHolder::operator=(SharedContextHolder &&) noexcept = default;
@ -87,10 +93,10 @@ void Context::makeGlobalContext()
global_context = shared_from_this(); global_context = shared_from_this();
} }
ContextMutablePtr Context::createGlobal(ContextSharedPart * shared) ContextMutablePtr Context::createGlobal(ContextSharedPart * shared_part)
{ {
auto res = std::shared_ptr<Context>(new Context); auto res = std::shared_ptr<Context>(new Context);
res->shared = shared; res->shared = shared_part;
return res; return res;
} }
@ -105,6 +111,7 @@ SharedContextHolder Context::createShared()
return SharedContextHolder(std::make_unique<ContextSharedPart>()); return SharedContextHolder(std::make_unique<ContextSharedPart>());
} }
ContextMutablePtr Context::getGlobalContext() const ContextMutablePtr Context::getGlobalContext() const
{ {
auto ptr = global_context.lock(); auto ptr = global_context.lock();
@ -112,22 +119,55 @@ ContextMutablePtr Context::getGlobalContext() const
return ptr; return ptr;
} }
std::unique_lock<std::recursive_mutex> Context::getLock() const std::unique_lock<SharedMutex> Context::getGlobalLock() const
{ {
ProfileEvents::increment(ProfileEvents::ContextLock); ProfileEvents::increment(ProfileEvents::ContextLock);
CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait}; CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait};
return std::unique_lock(shared->mutex); Stopwatch watch;
auto lock = std::unique_lock(shared->mutex);
ProfileEvents::increment(ProfileEvents::ContextLockWaitMicroseconds, watch.elapsedMicroseconds());
return lock;
}
std::shared_lock<SharedMutex> Context::getGlobalSharedLock() const
{
ProfileEvents::increment(ProfileEvents::ContextLock);
CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait};
Stopwatch watch;
auto lock = std::shared_lock(shared->mutex);
ProfileEvents::increment(ProfileEvents::ContextLockWaitMicroseconds, watch.elapsedMicroseconds());
return lock;
}
std::unique_lock<SharedMutex> Context::getLocalLock() const
{
ProfileEvents::increment(ProfileEvents::ContextLock);
CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait};
Stopwatch watch;
auto lock = std::unique_lock(mutex);
ProfileEvents::increment(ProfileEvents::ContextLockWaitMicroseconds, watch.elapsedMicroseconds());
return lock;
}
std::shared_lock<SharedMutex> Context::getLocalSharedLock() const
{
ProfileEvents::increment(ProfileEvents::ContextLock);
CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait};
Stopwatch watch;
auto lock = std::shared_lock(mutex);
ProfileEvents::increment(ProfileEvents::ContextLockWaitMicroseconds, watch.elapsedMicroseconds());
return lock;
} }
String Context::getPath() const String Context::getPath() const
{ {
auto lock = getLock(); auto lock = getGlobalSharedLock();
return shared->path; return shared->path;
} }
void Context::setPath(const String & path) void Context::setPath(const String & path)
{ {
auto lock = getLock(); auto lock = getGlobalLock();
shared->path = path; shared->path = path;
} }
@ -143,15 +183,13 @@ void Context::setMacros(std::unique_ptr<Macros> && macros)
BackgroundSchedulePool & Context::getSchedulePool() const BackgroundSchedulePool & Context::getSchedulePool() const
{ {
auto lock = getLock(); callOnce(shared->schedule_pool_initialized, [&] {
if (!shared->schedule_pool)
{
shared->schedule_pool = std::make_unique<BackgroundSchedulePool>( shared->schedule_pool = std::make_unique<BackgroundSchedulePool>(
shared->server_settings.background_schedule_pool_size, shared->server_settings.background_schedule_pool_size,
CurrentMetrics::BackgroundSchedulePoolTask, CurrentMetrics::BackgroundSchedulePoolTask,
CurrentMetrics::BackgroundSchedulePoolSize, CurrentMetrics::BackgroundSchedulePoolSize,
"BgSchPool"); "BgSchPool");
} });
return *shared->schedule_pool; return *shared->schedule_pool;
} }
@ -168,30 +206,21 @@ const RemoteHostFilter & Context::getRemoteHostFilter() const
IAsynchronousReader & Context::getThreadPoolReader(FilesystemReaderType type) const IAsynchronousReader & Context::getThreadPoolReader(FilesystemReaderType type) const
{ {
auto lock = getLock(); callOnce(shared->readers_initialized, [&] {
const auto & config = getConfigRef();
shared->asynchronous_remote_fs_reader = createThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER, config);
shared->asynchronous_local_fs_reader = createThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER, config);
shared->synchronous_local_fs_reader = createThreadPoolReader(FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER, config);
});
switch (type) switch (type)
{ {
case FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER: case FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER:
{
if (!shared->asynchronous_remote_fs_reader)
shared->asynchronous_remote_fs_reader = createThreadPoolReader(type, getConfigRef());
return *shared->asynchronous_remote_fs_reader; return *shared->asynchronous_remote_fs_reader;
}
case FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER: case FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER:
{
if (!shared->asynchronous_local_fs_reader)
shared->asynchronous_local_fs_reader = createThreadPoolReader(type, getConfigRef());
return *shared->asynchronous_local_fs_reader; return *shared->asynchronous_local_fs_reader;
}
case FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER: case FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER:
{
if (!shared->synchronous_local_fs_reader)
shared->synchronous_local_fs_reader = createThreadPoolReader(type, getConfigRef());
return *shared->synchronous_local_fs_reader; return *shared->synchronous_local_fs_reader;
}
} }
} }
@ -207,19 +236,19 @@ std::shared_ptr<FilesystemReadPrefetchesLog> Context::getFilesystemReadPrefetche
void Context::setConfig(const ConfigurationPtr & config) void Context::setConfig(const ConfigurationPtr & config)
{ {
auto lock = getLock(); auto lock = getGlobalLock();
shared->config = config; shared->config = config;
} }
const Poco::Util::AbstractConfiguration & Context::getConfigRef() const const Poco::Util::AbstractConfiguration & Context::getConfigRef() const
{ {
auto lock = getLock(); auto lock = getGlobalSharedLock();
return shared->config ? *shared->config : Poco::Util::Application::instance().config(); return shared->config ? *shared->config : Poco::Util::Application::instance().config();
} }
std::shared_ptr<AsyncReadCounters> Context::getAsyncReadCounters() const std::shared_ptr<AsyncReadCounters> Context::getAsyncReadCounters() const
{ {
auto lock = getLock(); auto lock = getLocalLock();
if (!async_read_counters) if (!async_read_counters)
async_read_counters = std::make_shared<AsyncReadCounters>(); async_read_counters = std::make_shared<AsyncReadCounters>();
return async_read_counters; return async_read_counters;
@ -227,18 +256,14 @@ std::shared_ptr<AsyncReadCounters> Context::getAsyncReadCounters() const
ThreadPool & Context::getThreadPoolWriter() const ThreadPool & Context::getThreadPoolWriter() const
{ {
const auto & config = getConfigRef(); callOnce(shared->threadpool_writer_initialized, [&] {
const auto & config = getConfigRef();
auto lock = getLock();
if (!shared->threadpool_writer)
{
auto pool_size = config.getUInt(".threadpool_writer_pool_size", 100); auto pool_size = config.getUInt(".threadpool_writer_pool_size", 100);
auto queue_size = config.getUInt(".threadpool_writer_queue_size", 1000000); auto queue_size = config.getUInt(".threadpool_writer_queue_size", 1000000);
shared->threadpool_writer = std::make_unique<ThreadPool>( shared->threadpool_writer = std::make_unique<ThreadPool>(
CurrentMetrics::IOWriterThreads, CurrentMetrics::IOWriterThreadsActive, pool_size, pool_size, queue_size); CurrentMetrics::IOWriterThreads, CurrentMetrics::IOWriterThreadsActive, pool_size, pool_size, queue_size);
} });
return *shared->threadpool_writer; return *shared->threadpool_writer;
} }

View File

@ -6,6 +6,7 @@
#include <Common/MultiVersion.h> #include <Common/MultiVersion.h>
#include <Common/RemoteHostFilter.h> #include <Common/RemoteHostFilter.h>
#include <Common/SharedMutex.h>
#include <Disks/IO/getThreadPoolReader.h> #include <Disks/IO/getThreadPoolReader.h>
@ -44,17 +45,9 @@ private:
std::unique_ptr<ContextSharedPart> shared; std::unique_ptr<ContextSharedPart> shared;
}; };
class ContextData
class Context : public std::enable_shared_from_this<Context>
{ {
private: protected:
/// Use copy constructor or createGlobal() instead
Context();
Context(const Context &);
Context & operator=(const Context &);
std::unique_lock<std::recursive_mutex> getLock() const;
ContextWeakMutablePtr global_context; ContextWeakMutablePtr global_context;
inline static ContextPtr global_context_instance; inline static ContextPtr global_context_instance;
ContextSharedPart * shared; ContextSharedPart * shared;
@ -63,9 +56,33 @@ private:
mutable std::shared_ptr<AsyncReadCounters> async_read_counters; mutable std::shared_ptr<AsyncReadCounters> async_read_counters;
Settings settings; /// Setting for query execution. Settings settings; /// Setting for query execution.
public:
/// Use copy constructor or createGlobal() instead
ContextData();
ContextData(const ContextData &);
};
class Context : public ContextData, public std::enable_shared_from_this<Context>
{
private:
/// ContextData mutex
mutable SharedMutex mutex;
Context();
Context(const Context &);
std::unique_lock<SharedMutex> getGlobalLock() const;
std::shared_lock<SharedMutex> getGlobalSharedLock() const;
std::unique_lock<SharedMutex> getLocalLock() const;
std::shared_lock<SharedMutex> getLocalSharedLock() const;
public: public:
/// Create initial Context with ContextShared and etc. /// Create initial Context with ContextShared and etc.
static ContextMutablePtr createGlobal(ContextSharedPart * shared); static ContextMutablePtr createGlobal(ContextSharedPart * shared_part);
static SharedContextHolder createShared(); static SharedContextHolder createShared();
ContextMutablePtr getGlobalContext() const; ContextMutablePtr getGlobalContext() const;

View File

@ -284,7 +284,7 @@ void deserializeLogMagic(ReadBuffer & in)
/// strange, that this 550 bytes obviously was a part of Create transaction, /// strange, that this 550 bytes obviously was a part of Create transaction,
/// but the operation code was -1. We have added debug prints to original /// but the operation code was -1. We have added debug prints to original
/// zookeeper (3.6.3) and found that it just reads 550 bytes of this "Error" /// zookeeper (3.6.3) and found that it just reads 550 bytes of this "Error"
/// transaction, tooks the first 4 bytes as an error code (it was 79, non /// transaction, took the first 4 bytes as an error code (it was 79, non
/// existing code) and skip all remaining 546 bytes. NOTE: it looks like a bug /// existing code) and skip all remaining 546 bytes. NOTE: it looks like a bug
/// in ZooKeeper. /// in ZooKeeper.
/// ///

View File

@ -339,7 +339,7 @@ static DataTypePtr getLeastCommonTypeForObject(const DataTypes & types, bool che
for (size_t i = 1; i < subtypes.size(); ++i) for (size_t i = 1; i < subtypes.size(); ++i)
if (first_dim != getNumberOfDimensions(*subtypes[i])) if (first_dim != getNumberOfDimensions(*subtypes[i]))
throw Exception(ErrorCodes::TYPE_MISMATCH, throw Exception(ErrorCodes::TYPE_MISMATCH,
"Uncompatible types of subcolumn '{}': {} and {}", "Incompatible types of subcolumn '{}': {} and {}",
key.getPath(), subtypes[0]->getName(), subtypes[i]->getName()); key.getPath(), subtypes[0]->getName(), subtypes[i]->getName());
tuple_paths.emplace_back(key); tuple_paths.emplace_back(key);

View File

@ -10,10 +10,8 @@
#include <Common/Arena.h> #include <Common/Arena.h>
#include <Formats/FormatSettings.h> #include <Formats/FormatSettings.h>
#include <Formats/ProtobufReader.h>
#include <Formats/ProtobufWriter.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h> #include <IO/Operators.h>
#include <IO/WriteBufferFromString.h>
namespace DB namespace DB
{ {

View File

@ -11,7 +11,6 @@
#include <IO/WriteBufferFromString.h> #include <IO/WriteBufferFromString.h>
#include <Formats/FormatSettings.h> #include <Formats/FormatSettings.h>
#include <Formats/ProtobufReader.h>
namespace DB namespace DB
{ {

View File

@ -4,7 +4,6 @@
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <Columns/ColumnsNumber.h> #include <Columns/ColumnsNumber.h>
#include <Formats/ProtobufReader.h>
#include <Common/assert_cast.h> #include <Common/assert_cast.h>

View File

@ -1,16 +1,14 @@
#include <DataTypes/Serializations/SerializationDateTime.h> #include <DataTypes/Serializations/SerializationDateTime.h>
#include <Columns/ColumnVector.h> #include <Columns/ColumnVector.h>
#include <Common/assert_cast.h>
#include <Common/DateLUT.h>
#include <Formats/FormatSettings.h> #include <Formats/FormatSettings.h>
#include <Formats/ProtobufReader.h>
#include <Formats/ProtobufWriter.h>
#include <IO/Operators.h> #include <IO/Operators.h>
#include <IO/ReadBufferFromString.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <IO/parseDateTimeBestEffort.h> #include <IO/parseDateTimeBestEffort.h>
#include <IO/ReadBufferFromString.h> #include <Common/DateLUT.h>
#include <Common/assert_cast.h>
namespace DB namespace DB
{ {

View File

@ -1,15 +1,14 @@
#include <DataTypes/Serializations/SerializationDateTime64.h> #include <DataTypes/Serializations/SerializationDateTime64.h>
#include <Columns/ColumnVector.h> #include <Columns/ColumnVector.h>
#include <Common/assert_cast.h>
#include <Common/DateLUT.h>
#include <Formats/FormatSettings.h> #include <Formats/FormatSettings.h>
#include <Formats/ProtobufReader.h> #include <IO/ReadBufferFromString.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromString.h> #include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <IO/parseDateTimeBestEffort.h> #include <IO/parseDateTimeBestEffort.h>
#include <IO/ReadBufferFromString.h> #include <Common/DateLUT.h>
#include <Common/assert_cast.h>
namespace DB namespace DB
{ {

View File

@ -1,13 +1,11 @@
#include <DataTypes/Serializations/SerializationDecimal.h> #include <DataTypes/Serializations/SerializationDecimal.h>
#include <Columns/ColumnVector.h> #include <Columns/ColumnVector.h>
#include <Common/assert_cast.h>
#include <Common/typeid_cast.h>
#include <Formats/ProtobufReader.h>
#include <Formats/ProtobufWriter.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <IO/readDecimalText.h> #include <IO/readDecimalText.h>
#include <Common/assert_cast.h>
#include <Common/typeid_cast.h>
namespace DB namespace DB
{ {

View File

@ -1,11 +1,9 @@
#include <DataTypes/Serializations/SerializationDecimalBase.h> #include <DataTypes/Serializations/SerializationDecimalBase.h>
#include <Common/assert_cast.h>
#include <Common/typeid_cast.h>
#include <Formats/ProtobufReader.h>
#include <Formats/ProtobufWriter.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <Common/assert_cast.h>
#include <Common/typeid_cast.h>
#include <ranges> #include <ranges>

View File

@ -1,11 +1,9 @@
#include <DataTypes/Serializations/SerializationEnum.h> #include <DataTypes/Serializations/SerializationEnum.h>
#include <Columns/ColumnVector.h> #include <Columns/ColumnVector.h>
#include <Common/assert_cast.h>
#include <IO/WriteBufferFromString.h>
#include <Formats/FormatSettings.h> #include <Formats/FormatSettings.h>
#include <Formats/ProtobufReader.h> #include <IO/WriteBufferFromString.h>
#include <Formats/ProtobufWriter.h> #include <Common/assert_cast.h>
namespace DB namespace DB
{ {

View File

@ -4,8 +4,6 @@
#include <Columns/ColumnConst.h> #include <Columns/ColumnConst.h>
#include <Formats/FormatSettings.h> #include <Formats/FormatSettings.h>
#include <Formats/ProtobufReader.h>
#include <Formats/ProtobufWriter.h>
#include <IO/WriteBuffer.h> #include <IO/WriteBuffer.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>

View File

@ -1,14 +1,14 @@
#include <DataTypes/Serializations/SerializationNumber.h> #include <DataTypes/Serializations/SerializationNumber.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnConst.h> #include <Columns/ColumnConst.h>
#include <Columns/ColumnVector.h>
#include <Core/Field.h>
#include <Formats/FormatSettings.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <Common/NaNUtils.h> #include <Common/NaNUtils.h>
#include <Common/typeid_cast.h>
#include <Common/assert_cast.h> #include <Common/assert_cast.h>
#include <Formats/FormatSettings.h> #include <Common/typeid_cast.h>
#include <Formats/ProtobufReader.h>
#include <Core/Field.h>
#include <ranges> #include <ranges>

View File

@ -1,7 +1,5 @@
#include <Columns/ColumnsNumber.h> #include <Columns/ColumnsNumber.h>
#include <DataTypes/Serializations/SerializationUUID.h> #include <DataTypes/Serializations/SerializationUUID.h>
#include <Formats/ProtobufReader.h>
#include <Formats/ProtobufWriter.h>
#include <IO/ReadBufferFromString.h> #include <IO/ReadBufferFromString.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>

View File

@ -33,7 +33,7 @@ private:
/// Number of references (hardlinks) to this metadata file. /// Number of references (hardlinks) to this metadata file.
/// ///
/// FIXME: Why we are tracking it explicetly, without /// FIXME: Why we are tracking it explicitly, without
/// info from filesystem???? /// info from filesystem????
uint32_t ref_count = 0; uint32_t ref_count = 0;

View File

@ -1,18 +1,19 @@
#pragma once #pragma once
#include <Columns/ColumnString.h>
#include <Columns/ColumnVector.h> #include <Columns/ColumnVector.h>
#include <Columns/ColumnsNumber.h> #include <Columns/ColumnsNumber.h>
#include <base/types.h> #include <Common/DateLUTImpl.h>
#include <Common/Exception.h>
#include <Core/DecimalFunctions.h> #include <Core/DecimalFunctions.h>
#include <DataTypes/DataTypeString.h>
#include <Functions/DateTimeTransforms.h> #include <Functions/DateTimeTransforms.h>
#include <Functions/FunctionHelpers.h> #include <Functions/FunctionHelpers.h>
#include <Functions/extractTimeZoneFromFunctionArguments.h>
#include <Functions/IFunction.h> #include <Functions/IFunction.h>
#include <Common/Exception.h> #include <Functions/extractTimeZoneFromFunctionArguments.h>
#include <Common/DateLUTImpl.h> #include <IO/ReadBufferFromString.h>
#include <IO/parseDateTimeBestEffort.h>
/// The default mode value to use for the WEEK() function #include <base/types.h>
#define DEFAULT_WEEK_MODE 0
namespace DB namespace DB
@ -22,132 +23,6 @@ namespace ErrorCodes
extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_COLUMN;
} }
/**
* CustomWeek Transformations.
*/
struct ToYearWeekImpl
{
static constexpr auto name = "toYearWeek";
static inline UInt32 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
// TODO: ditch toDayNum()
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode | static_cast<UInt32>(WeekModeFlag::YEAR));
return yw.first * 100 + yw.second;
}
static inline UInt32 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode | static_cast<UInt32>(WeekModeFlag::YEAR));
return yw.first * 100 + yw.second;
}
static inline UInt32 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
YearWeek yw = time_zone.toYearWeek(ExtendedDayNum (d), week_mode | static_cast<UInt32>(WeekModeFlag::YEAR));
return yw.first * 100 + yw.second;
}
static inline UInt32 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
YearWeek yw = time_zone.toYearWeek(DayNum(d), week_mode | static_cast<UInt32>(WeekModeFlag::YEAR));
return yw.first * 100 + yw.second;
}
using FactorTransform = ZeroTransform;
};
struct ToStartOfWeekImpl
{
static constexpr auto name = "toStartOfWeek";
static inline UInt16 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode);
}
static inline UInt16 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode);
}
static inline UInt16 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfWeek(ExtendedDayNum(d), week_mode);
}
static inline UInt16 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfWeek(DayNum(d), week_mode);
}
static inline Int64 executeExtendedResult(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode);
}
static inline Int32 executeExtendedResult(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfWeek(ExtendedDayNum(d), week_mode);
}
using FactorTransform = ZeroTransform;
};
struct ToLastDayOfWeekImpl
{
static constexpr auto name = "toLastDayOfWeek";
static inline UInt16 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toLastDayNumOfWeek(time_zone.toDayNum(t), week_mode);
}
static inline UInt16 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toLastDayNumOfWeek(time_zone.toDayNum(t), week_mode);
}
static inline UInt16 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toLastDayNumOfWeek(ExtendedDayNum(d), week_mode);
}
static inline UInt16 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toLastDayNumOfWeek(DayNum(d), week_mode);
}
static inline Int64 executeExtendedResult(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toLastDayNumOfWeek(time_zone.toDayNum(t), week_mode);
}
static inline Int32 executeExtendedResult(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toLastDayNumOfWeek(ExtendedDayNum(d), week_mode);
}
using FactorTransform = ZeroTransform;
};
struct ToWeekImpl
{
static constexpr auto name = "toWeek";
static inline UInt8 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
// TODO: ditch conversion to DayNum, since it doesn't support extended range.
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode);
return yw.second;
}
static inline UInt8 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode);
return yw.second;
}
static inline UInt8 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
YearWeek yw = time_zone.toYearWeek(ExtendedDayNum(d), week_mode);
return yw.second;
}
static inline UInt8 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
YearWeek yw = time_zone.toYearWeek(DayNum(d), week_mode);
return yw.second;
}
using FactorTransform = ToStartOfYearImpl;
};
template <typename FromType, typename ToType, typename Transform, bool is_extended_result = false> template <typename FromType, typename ToType, typename Transform, bool is_extended_result = false>
struct WeekTransformer struct WeekTransformer
@ -157,8 +32,7 @@ struct WeekTransformer
{} {}
template <typename FromVectorType, typename ToVectorType> template <typename FromVectorType, typename ToVectorType>
void void vector(const FromVectorType & vec_from, ToVectorType & vec_to, UInt8 week_mode, const DateLUTImpl & time_zone) const
vector(const FromVectorType & vec_from, ToVectorType & vec_to, UInt8 week_mode, const DateLUTImpl & time_zone) const
{ {
using ValueType = typename ToVectorType::value_type; using ValueType = typename ToVectorType::value_type;
size_t size = vec_from.size(); size_t size = vec_from.size();
@ -186,7 +60,8 @@ struct CustomWeekTransformImpl
{ {
const auto op = WeekTransformer<typename FromDataType::FieldType, typename ToDataType::FieldType, Transform, is_extended_result>{std::move(transform)}; const auto op = WeekTransformer<typename FromDataType::FieldType, typename ToDataType::FieldType, Transform, is_extended_result>{std::move(transform)};
UInt8 week_mode = DEFAULT_WEEK_MODE; static constexpr UInt8 default_week_mode = 0;
UInt8 week_mode = default_week_mode;
if (arguments.size() > 1) if (arguments.size() > 1)
{ {
if (const auto * week_mode_column = checkAndGetColumnConst<ColumnUInt8>(arguments[1].column.get())) if (const auto * week_mode_column = checkAndGetColumnConst<ColumnUInt8>(arguments[1].column.get()))
@ -195,7 +70,26 @@ struct CustomWeekTransformImpl
const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(arguments, 2, 0); const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(arguments, 2, 0);
const ColumnPtr source_col = arguments[0].column; const ColumnPtr source_col = arguments[0].column;
if (const auto * sources = checkAndGetColumn<typename FromDataType::ColumnType>(source_col.get()))
if constexpr (std::is_same_v<FromDataType, DataTypeString>)
{
static const DateLUTImpl & utc_time_zone = DateLUT::instance("UTC");
const auto * sources = checkAndGetColumn<DataTypeString::ColumnType>(source_col.get());
auto col_to = ToDataType::ColumnType::create();
col_to->getData().resize(sources->size());
for (size_t i = 0; i < sources->size(); ++i)
{
DateTime64 dt64;
ReadBufferFromString buf(sources->getDataAt(i).toView());
parseDateTime64BestEffort(dt64, 0, buf, time_zone, utc_time_zone);
col_to->getData()[i] = static_cast<ToDataType::FieldType>(transform.execute(dt64, week_mode, time_zone));
}
return col_to;
}
else if (const auto * sources = checkAndGetColumn<typename FromDataType::ColumnType>(source_col.get()))
{ {
auto col_to = ToDataType::ColumnType::create(); auto col_to = ToDataType::ColumnType::create();
op.vector(sources->getData(), col_to->getData(), week_mode, time_zone); op.vector(sources->getData(), col_to->getData(), week_mode, time_zone);

View File

@ -305,6 +305,132 @@ struct ToStartOfYearImpl
using FactorTransform = ZeroTransform; using FactorTransform = ZeroTransform;
}; };
struct ToYearWeekImpl
{
static constexpr auto name = "toYearWeek";
static constexpr bool value_may_be_string = true;
static UInt32 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
// TODO: ditch toDayNum()
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode | static_cast<UInt32>(WeekModeFlag::YEAR));
return yw.first * 100 + yw.second;
}
static UInt32 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode | static_cast<UInt32>(WeekModeFlag::YEAR));
return yw.first * 100 + yw.second;
}
static UInt32 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
YearWeek yw = time_zone.toYearWeek(ExtendedDayNum (d), week_mode | static_cast<UInt32>(WeekModeFlag::YEAR));
return yw.first * 100 + yw.second;
}
static UInt32 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
YearWeek yw = time_zone.toYearWeek(DayNum(d), week_mode | static_cast<UInt32>(WeekModeFlag::YEAR));
return yw.first * 100 + yw.second;
}
using FactorTransform = ZeroTransform;
};
struct ToStartOfWeekImpl
{
static constexpr auto name = "toStartOfWeek";
static constexpr bool value_may_be_string = false;
static UInt16 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode);
}
static UInt16 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode);
}
static UInt16 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfWeek(ExtendedDayNum(d), week_mode);
}
static UInt16 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfWeek(DayNum(d), week_mode);
}
static Int64 executeExtendedResult(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode);
}
static Int32 executeExtendedResult(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfWeek(ExtendedDayNum(d), week_mode);
}
using FactorTransform = ZeroTransform;
};
struct ToLastDayOfWeekImpl
{
static constexpr auto name = "toLastDayOfWeek";
static constexpr bool value_may_be_string = false;
static UInt16 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toLastDayNumOfWeek(time_zone.toDayNum(t), week_mode);
}
static UInt16 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toLastDayNumOfWeek(time_zone.toDayNum(t), week_mode);
}
static UInt16 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toLastDayNumOfWeek(ExtendedDayNum(d), week_mode);
}
static UInt16 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toLastDayNumOfWeek(DayNum(d), week_mode);
}
static Int64 executeExtendedResult(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toLastDayNumOfWeek(time_zone.toDayNum(t), week_mode);
}
static Int32 executeExtendedResult(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toLastDayNumOfWeek(ExtendedDayNum(d), week_mode);
}
using FactorTransform = ZeroTransform;
};
struct ToWeekImpl
{
static constexpr auto name = "toWeek";
static constexpr bool value_may_be_string = true;
static UInt8 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
// TODO: ditch conversion to DayNum, since it doesn't support extended range.
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode);
return yw.second;
}
static UInt8 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode);
return yw.second;
}
static UInt8 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
YearWeek yw = time_zone.toYearWeek(ExtendedDayNum(d), week_mode);
return yw.second;
}
static UInt8 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
YearWeek yw = time_zone.toYearWeek(DayNum(d), week_mode);
return yw.second;
}
using FactorTransform = ToStartOfYearImpl;
};
template <IntervalKind::Kind unit> template <IntervalKind::Kind unit>
struct ToStartOfInterval; struct ToStartOfInterval;
@ -1176,6 +1302,7 @@ struct ToDayOfMonthImpl
struct ToDayOfWeekImpl struct ToDayOfWeekImpl
{ {
static constexpr auto name = "toDayOfWeek"; static constexpr auto name = "toDayOfWeek";
static constexpr bool value_may_be_string = true;
static UInt8 execute(Int64 t, UInt8 mode, const DateLUTImpl & time_zone) static UInt8 execute(Int64 t, UInt8 mode, const DateLUTImpl & time_zone)
{ {

View File

@ -16,19 +16,19 @@ private:
const bool enable_extended_results_for_datetime_functions = false; const bool enable_extended_results_for_datetime_functions = false;
public: public:
static FunctionPtr create(ContextPtr context_) static FunctionPtr create(ContextPtr context)
{ {
return std::make_shared<FunctionCustomWeekToDateOrDate32>(context_); return std::make_shared<FunctionCustomWeekToDateOrDate32>(context);
} }
explicit FunctionCustomWeekToDateOrDate32(ContextPtr context_) explicit FunctionCustomWeekToDateOrDate32(ContextPtr context)
: enable_extended_results_for_datetime_functions(context_->getSettingsRef().enable_extended_results_for_datetime_functions) : enable_extended_results_for_datetime_functions(context->getSettingsRef().enable_extended_results_for_datetime_functions)
{ {
} }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{ {
this->checkArguments(arguments, /*is_result_type_date_or_date32*/ true); this->checkArguments(arguments, /*is_result_type_date_or_date32*/ true, Transform::value_may_be_string);
const IDataType * from_type = arguments[0].type.get(); const IDataType * from_type = arguments[0].type.get();
WhichDataType which(from_type); WhichDataType which(from_type);
@ -44,16 +44,13 @@ public:
WhichDataType which(from_type); WhichDataType which(from_type);
if (which.isDate()) if (which.isDate())
return CustomWeekTransformImpl<DataTypeDate, DataTypeDate>::execute( return CustomWeekTransformImpl<DataTypeDate, DataTypeDate>::execute(arguments, result_type, input_rows_count, Transform{});
arguments, result_type, input_rows_count, Transform{});
else if (which.isDate32()) else if (which.isDate32())
{ {
if (enable_extended_results_for_datetime_functions) if (enable_extended_results_for_datetime_functions)
return CustomWeekTransformImpl<DataTypeDate32, DataTypeDate32, /*is_extended_result*/ true>::execute( return CustomWeekTransformImpl<DataTypeDate32, DataTypeDate32, /*is_extended_result*/ true>::execute(arguments, result_type, input_rows_count, Transform{});
arguments, result_type, input_rows_count, Transform{});
else else
return CustomWeekTransformImpl<DataTypeDate32, DataTypeDate>::execute( return CustomWeekTransformImpl<DataTypeDate32, DataTypeDate>::execute(arguments, result_type, input_rows_count, Transform{});
arguments, result_type, input_rows_count, Transform{});
} }
else if (which.isDateTime()) else if (which.isDateTime())
return CustomWeekTransformImpl<DataTypeDateTime, DataTypeDate>::execute( return CustomWeekTransformImpl<DataTypeDateTime, DataTypeDate>::execute(
@ -61,14 +58,14 @@ public:
else if (which.isDateTime64()) else if (which.isDateTime64())
{ {
if (enable_extended_results_for_datetime_functions) if (enable_extended_results_for_datetime_functions)
return CustomWeekTransformImpl<DataTypeDateTime64, DataTypeDate32, /*is_extended_result*/ true>::execute( return CustomWeekTransformImpl<DataTypeDateTime64, DataTypeDate32, /*is_extended_result*/ true>::execute(arguments, result_type, input_rows_count,
arguments, result_type, input_rows_count,
TransformDateTime64<Transform>{assert_cast<const DataTypeDateTime64 *>(from_type)->getScale()}); TransformDateTime64<Transform>{assert_cast<const DataTypeDateTime64 *>(from_type)->getScale()});
else else
return CustomWeekTransformImpl<DataTypeDateTime64, DataTypeDate>::execute( return CustomWeekTransformImpl<DataTypeDateTime64, DataTypeDate>::execute(arguments, result_type, input_rows_count,
arguments, result_type, input_rows_count,
TransformDateTime64<Transform>{assert_cast<const DataTypeDateTime64 *>(from_type)->getScale()}); TransformDateTime64<Transform>{assert_cast<const DataTypeDateTime64 *>(from_type)->getScale()});
} }
else if (Transform::value_may_be_string && which.isString())
return CustomWeekTransformImpl<DataTypeString, DataTypeDate>::execute(arguments, result_type, input_rows_count, Transform{}); // TODO
else else
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument of function {}", "Illegal type {} of argument of function {}",

View File

@ -19,8 +19,7 @@ public:
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{ {
this->checkArguments(arguments); this->checkArguments(arguments, /*is_result_type_date_or_date32*/ false, Transform::value_may_be_string);
return std::make_shared<ToDataType>(); return std::make_shared<ToDataType>();
} }
@ -30,20 +29,16 @@ public:
WhichDataType which(from_type); WhichDataType which(from_type);
if (which.isDate()) if (which.isDate())
return CustomWeekTransformImpl<DataTypeDate, ToDataType>::execute( return CustomWeekTransformImpl<DataTypeDate, ToDataType>::execute(arguments, result_type, input_rows_count, Transform{});
arguments, result_type, input_rows_count, Transform{});
else if (which.isDate32()) else if (which.isDate32())
return CustomWeekTransformImpl<DataTypeDate32, ToDataType>::execute( return CustomWeekTransformImpl<DataTypeDate32, ToDataType>::execute(arguments, result_type, input_rows_count, Transform{});
arguments, result_type, input_rows_count, Transform{});
else if (which.isDateTime()) else if (which.isDateTime())
return CustomWeekTransformImpl<DataTypeDateTime, ToDataType>::execute( return CustomWeekTransformImpl<DataTypeDateTime, ToDataType>::execute(arguments, result_type, input_rows_count, Transform{});
arguments, result_type, input_rows_count, Transform{});
else if (which.isDateTime64()) else if (which.isDateTime64())
{ return CustomWeekTransformImpl<DataTypeDateTime64, ToDataType>::execute(arguments, result_type, input_rows_count,
return CustomWeekTransformImpl<DataTypeDateTime64, ToDataType>::execute(
arguments, result_type, input_rows_count,
TransformDateTime64<Transform>{assert_cast<const DataTypeDateTime64 *>(from_type)->getScale()}); TransformDateTime64<Transform>{assert_cast<const DataTypeDateTime64 *>(from_type)->getScale()});
} else if (Transform::value_may_be_string && which.isString())
return CustomWeekTransformImpl<DataTypeString, ToDataType>::execute(arguments, result_type, input_rows_count, Transform{});
else else
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument of function {}", "Illegal type {} of argument of function {}",

View File

@ -65,41 +65,47 @@ public:
} }
protected: protected:
void checkArguments(const ColumnsWithTypeAndName & arguments, bool is_result_type_date_or_date32 = false) const void checkArguments(const ColumnsWithTypeAndName & arguments, bool is_result_type_date_or_date32, bool value_may_be_string) const
{ {
if (arguments.size() == 1) if (arguments.size() == 1)
{ {
if (!isDate(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) auto type0 = arguments[0].type;
if (!isDate(type0) && !isDate32(type0) && !isDateTime(type0) && !isDateTime64(type0) && !(value_may_be_string && isString(type0)))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument of function {}. Must be Date, Date32, DateTime or DateTime64.", "Illegal type {} of argument of function {}. Must be Date, Date32, DateTime or DateTime64.",
arguments[0].type->getName(), getName()); type0->getName(), getName());
} }
else if (arguments.size() == 2) else if (arguments.size() == 2)
{ {
if (!isDate(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) auto type0 = arguments[0].type;
auto type1 = arguments[1].type;
if (!isDate(type0) && !isDate32(type0) && !isDateTime(type0) && !isDateTime64(type0) && !(value_may_be_string && isString(type0)))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of 1st argument of function {}. Must be Date, Date32, DateTime or DateTime64.", "Illegal type {} of 1st argument of function {}. Must be Date, Date32, DateTime or DateTime64.",
arguments[0].type->getName(), getName()); type0->getName(), getName());
if (!isUInt8(arguments[1].type)) if (!isUInt8(type1))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of 2nd (optional) argument of function {}. Must be constant UInt8 (week mode).", "Illegal type {} of 2nd (optional) argument of function {}. Must be constant UInt8 (week mode).",
arguments[1].type->getName(), getName()); type1->getName(), getName());
} }
else if (arguments.size() == 3) else if (arguments.size() == 3)
{ {
if (!isDate(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) auto type0 = arguments[0].type;
auto type1 = arguments[1].type;
auto type2 = arguments[2].type;
if (!isDate(type0) && !isDate32(type0) && !isDateTime(type0) && !isDateTime64(type0) && !(value_may_be_string && isString(type0)))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument of function {}. Must be Date, Date32, DateTime or DateTime64", "Illegal type {} of argument of function {}. Must be Date, Date32, DateTime or DateTime64",
arguments[0].type->getName(), getName()); type0->getName(), getName());
if (!isUInt8(arguments[1].type)) if (!isUInt8(type1))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of 2nd (optional) argument of function {}. Must be constant UInt8 (week mode).", "Illegal type {} of 2nd (optional) argument of function {}. Must be constant UInt8 (week mode).",
arguments[1].type->getName(), getName()); type1->getName(), getName());
if (!isString(arguments[2].type)) if (!isString(type2))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of 3rd (optional) argument of function {}. Must be constant string (timezone name).", "Illegal type {} of 3rd (optional) argument of function {}. Must be constant string (timezone name).",
arguments[2].type->getName(), getName()); type2->getName(), getName());
if ((isDate(arguments[0].type) || isDate32(arguments[0].type)) && is_result_type_date_or_date32) if (is_result_type_date_or_date32 && (isDate(type0) || isDate32(type0)))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"The timezone argument of function {} is allowed only when the 1st argument is DateTime or DateTime64.", "The timezone argument of function {} is allowed only when the 1st argument is DateTime or DateTime64.",
getName()); getName());

View File

@ -182,7 +182,7 @@ INSTANTIATE_TEST_SUITE_P(Basic,
DateLUT::instance("Europe/Minsk") DateLUT::instance("Europe/Minsk")
}, },
{ {
"When scale is 0, subsecond part (and separtor) is missing from string", "When scale is 0, subsecond part (and separator) is missing from string",
"2019-09-16 19:20:17", "2019-09-16 19:20:17",
1568650817ULL, 1568650817ULL,
0, 0,
@ -197,4 +197,3 @@ INSTANTIATE_TEST_SUITE_P(Basic,
} }
}) })
); );

View File

@ -166,7 +166,7 @@ namespace
access_to_revoke.grant(elements_to_revoke); access_to_revoke.grant(elements_to_revoke);
access_to_revoke.makeIntersection(all_granted_access); access_to_revoke.makeIntersection(all_granted_access);
/// Build more accurate list of elements to revoke, now we use an intesection of the initial list of elements to revoke /// Build more accurate list of elements to revoke, now we use an intersection of the initial list of elements to revoke
/// and all the granted access rights to these grantees. /// and all the granted access rights to these grantees.
bool grant_option = !elements_to_revoke.empty() && elements_to_revoke[0].grant_option; bool grant_option = !elements_to_revoke.empty() && elements_to_revoke[0].grant_option;
elements_to_revoke.clear(); elements_to_revoke.clear();

View File

@ -508,7 +508,8 @@ bool FileSegment::reserve(size_t size_to_reserve, FileCacheReserveStat * reserve
/// This (resizable file segments) is allowed only for single threaded use of file segment. /// This (resizable file segments) is allowed only for single threaded use of file segment.
/// Currently it is used only for temporary files through cache. /// Currently it is used only for temporary files through cache.
if (is_unbound && is_file_segment_size_exceeded) if (is_unbound && is_file_segment_size_exceeded)
segment_range.right = range().left + expected_downloaded_size + size_to_reserve; /// Note: segment_range.right is inclusive.
segment_range.right = range().left + expected_downloaded_size + size_to_reserve - 1;
/// if reserve_stat is not passed then use dummy stat and discard the result. /// if reserve_stat is not passed then use dummy stat and discard the result.
FileCacheReserveStat dummy_stat; FileCacheReserveStat dummy_stat;

View File

@ -3868,6 +3868,10 @@ void Context::setMaxTableSizeToDrop(size_t max_size)
shared->max_table_size_to_drop.store(max_size, std::memory_order_relaxed); shared->max_table_size_to_drop.store(max_size, std::memory_order_relaxed);
} }
size_t Context::getMaxTableSizeToDrop() const
{
return shared->max_table_size_to_drop.load(std::memory_order_relaxed);
}
void Context::checkTableCanBeDropped(const String & database, const String & table, const size_t & table_size) const void Context::checkTableCanBeDropped(const String & database, const String & table, const size_t & table_size) const
{ {
@ -3883,6 +3887,10 @@ void Context::setMaxPartitionSizeToDrop(size_t max_size)
shared->max_partition_size_to_drop.store(max_size, std::memory_order_relaxed); shared->max_partition_size_to_drop.store(max_size, std::memory_order_relaxed);
} }
size_t Context::getMaxPartitionSizeToDrop() const
{
return shared->max_partition_size_to_drop.load(std::memory_order_relaxed);
}
void Context::checkPartitionCanBeDropped(const String & database, const String & table, const size_t & partition_size) const void Context::checkPartitionCanBeDropped(const String & database, const String & table, const size_t & partition_size) const
{ {

View File

@ -1057,10 +1057,12 @@ public:
/// Prevents DROP TABLE if its size is greater than max_size (50GB by default, max_size=0 turn off this check) /// Prevents DROP TABLE if its size is greater than max_size (50GB by default, max_size=0 turn off this check)
void setMaxTableSizeToDrop(size_t max_size); void setMaxTableSizeToDrop(size_t max_size);
size_t getMaxTableSizeToDrop() const;
void checkTableCanBeDropped(const String & database, const String & table, const size_t & table_size) const; void checkTableCanBeDropped(const String & database, const String & table, const size_t & table_size) const;
/// Prevents DROP PARTITION if its size is greater than max_size (50GB by default, max_size=0 turn off this check) /// Prevents DROP PARTITION if its size is greater than max_size (50GB by default, max_size=0 turn off this check)
void setMaxPartitionSizeToDrop(size_t max_size); void setMaxPartitionSizeToDrop(size_t max_size);
size_t getMaxPartitionSizeToDrop() const;
void checkPartitionCanBeDropped(const String & database, const String & table, const size_t & partition_size) const; void checkPartitionCanBeDropped(const String & database, const String & table, const size_t & partition_size) const;
/// Lets you select the compression codec according to the conditions described in the configuration file. /// Lets you select the compression codec according to the conditions described in the configuration file.

View File

@ -11,7 +11,7 @@ namespace DB
namespace namespace
{ {
/// @note We place strings in ascending order here under the assumption it colud speed up String to Enum conversion. /// @note We place strings in ascending order here under the assumption it could speed up String to Enum conversion.
String makeStringsEnum(const std::set<String> & values) String makeStringsEnum(const std::set<String> & values)
{ {
String enum_string = "Enum8("; String enum_string = "Enum8(";

View File

@ -172,7 +172,7 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter)
if (!partition_commands.empty()) if (!partition_commands.empty())
{ {
table->checkAlterPartitionIsPossible(partition_commands, metadata_snapshot, getContext()->getSettingsRef()); table->checkAlterPartitionIsPossible(partition_commands, metadata_snapshot, getContext()->getSettingsRef(), getContext());
auto partition_commands_pipe = table->alterPartition(metadata_snapshot, partition_commands, getContext()); auto partition_commands_pipe = table->alterPartition(metadata_snapshot, partition_commands, getContext());
if (!partition_commands_pipe.empty()) if (!partition_commands_pipe.empty())
res.pipeline = QueryPipeline(std::move(partition_commands_pipe)); res.pipeline = QueryPipeline(std::move(partition_commands_pipe));

View File

@ -684,7 +684,15 @@ InterpreterSelectQuery::InterpreterSelectQuery(
if (!options.only_analyze) if (!options.only_analyze)
{ {
if (query.sampleSize() && (input_pipe || !storage || !storage->supportsSampling())) if (query.sampleSize() && (input_pipe || !storage || !storage->supportsSampling()))
throw Exception(ErrorCodes::SAMPLING_NOT_SUPPORTED, "Illegal SAMPLE: table {} doesn't support sampling", storage->getStorageID().getNameForLogs()); {
if (storage)
throw Exception(
ErrorCodes::SAMPLING_NOT_SUPPORTED,
"Storage {} doesn't support sampling",
storage->getStorageID().getNameForLogs());
else
throw Exception(ErrorCodes::SAMPLING_NOT_SUPPORTED, "Illegal SAMPLE: sampling is only allowed with the table engines that support it");
}
if (query.final() && (input_pipe || !storage || !storage->supportsFinal())) if (query.final() && (input_pipe || !storage || !storage->supportsFinal()))
{ {

View File

@ -413,18 +413,36 @@ public:
max_size = max_size_; max_size = max_size_;
} }
size_t getMaxSize() const
{
auto lock = unsafeLock();
return max_size;
}
void setMaxInsertQueriesAmount(size_t max_insert_queries_amount_) void setMaxInsertQueriesAmount(size_t max_insert_queries_amount_)
{ {
auto lock = unsafeLock(); auto lock = unsafeLock();
max_insert_queries_amount = max_insert_queries_amount_; max_insert_queries_amount = max_insert_queries_amount_;
} }
size_t getMaxInsertQueriesAmount() const
{
auto lock = unsafeLock();
return max_insert_queries_amount;
}
void setMaxSelectQueriesAmount(size_t max_select_queries_amount_) void setMaxSelectQueriesAmount(size_t max_select_queries_amount_)
{ {
auto lock = unsafeLock(); auto lock = unsafeLock();
max_select_queries_amount = max_select_queries_amount_; max_select_queries_amount = max_select_queries_amount_;
} }
size_t getMaxSelectQueriesAmount() const
{
auto lock = unsafeLock();
return max_select_queries_amount;
}
/// Try call cancel() for input and output streams of query with specified id and user /// Try call cancel() for input and output streams of query with specified id and user
CancellationCode sendCancelToQuery(const String & current_query_id, const String & current_user, bool kill = false); CancellationCode sendCancelToQuery(const String & current_query_id, const String & current_user, bool kill = false);

View File

@ -140,7 +140,7 @@ void QueryAliasesMatcher<T>::visitOther(const ASTPtr & ast, Data & data)
information for our ast node with query string. And this alias will be dropped because prefer_alias_to_column_name for ASTWIthAlias information for our ast node with query string. And this alias will be dropped because prefer_alias_to_column_name for ASTWIthAlias
by default is false. by default is false.
It is imporant that subquery can be converted to literal during ExecuteScalarSubqueriesVisitor. It is important that subquery can be converted to literal during ExecuteScalarSubqueriesVisitor.
And code below check if we previously set for subquery alias as _subquery, and if it is true And code below check if we previously set for subquery alias as _subquery, and if it is true
then set prefer_alias_to_column_name = true for node that was optimized during ExecuteScalarSubqueriesVisitor. then set prefer_alias_to_column_name = true for node that was optimized during ExecuteScalarSubqueriesVisitor.
*/ */

View File

@ -1,11 +1,21 @@
#include <gtest/gtest.h>
#include <filesystem> #include <filesystem>
#include <iomanip> #include <iomanip>
#include <iostream> #include <iostream>
#include <algorithm>
#include <numeric>
#include <random>
#include <memory> #include <memory>
#include <thread> #include <thread>
#include <Common/randomSeed.h>
#include <DataTypes/DataTypesNumber.h> #include <DataTypes/DataTypesNumber.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <Interpreters/Cache/FileCache.h> #include <Interpreters/Cache/FileCache.h>
#include <Interpreters/Cache/FileCacheSettings.h> #include <Interpreters/Cache/FileCacheSettings.h>
#include <Interpreters/Cache/FileSegment.h> #include <Interpreters/Cache/FileSegment.h>
@ -13,7 +23,6 @@
#include <Interpreters/TemporaryDataOnDisk.h> #include <Interpreters/TemporaryDataOnDisk.h>
#include <base/hex.h> #include <base/hex.h>
#include <base/sleep.h> #include <base/sleep.h>
#include <gtest/gtest.h>
#include <Poco/DOM/DOMParser.h> #include <Poco/DOM/DOMParser.h>
#include <Poco/Util/XMLConfiguration.h> #include <Poco/Util/XMLConfiguration.h>
#include <Common/CurrentThread.h> #include <Common/CurrentThread.h>
@ -187,6 +196,12 @@ public:
else else
setupLogs(TEST_LOG_LEVEL); setupLogs(TEST_LOG_LEVEL);
UInt64 seed = randomSeed();
if (const char * random_seed = std::getenv("TEST_RANDOM_SEED")) // NOLINT(concurrency-mt-unsafe)
seed = std::stoull(random_seed);
std::cout << "TEST_RANDOM_SEED=" << seed << std::endl;
rng = pcg64(seed);
if (fs::exists(cache_base_path)) if (fs::exists(cache_base_path))
fs::remove_all(cache_base_path); fs::remove_all(cache_base_path);
fs::create_directories(cache_base_path); fs::create_directories(cache_base_path);
@ -198,6 +213,7 @@ public:
fs::remove_all(cache_base_path); fs::remove_all(cache_base_path);
} }
pcg64 rng;
}; };
TEST_F(FileCacheTest, get) TEST_F(FileCacheTest, get)
@ -679,7 +695,7 @@ TEST_F(FileCacheTest, writeBuffer)
FileCache cache("6", settings); FileCache cache("6", settings);
cache.initialize(); cache.initialize();
auto write_to_cache = [&cache](const String & key, const Strings & data, bool flush) auto write_to_cache = [&cache, this](const String & key, const Strings & data, bool flush, ReadBufferPtr * out_read_buffer = nullptr)
{ {
CreateFileSegmentSettings segment_settings; CreateFileSegmentSettings segment_settings;
segment_settings.kind = FileSegmentKind::Temporary; segment_settings.kind = FileSegmentKind::Temporary;
@ -694,24 +710,32 @@ TEST_F(FileCacheTest, writeBuffer)
WriteBufferToFileSegment out(&segment); WriteBufferToFileSegment out(&segment);
std::list<std::thread> threads; std::list<std::thread> threads;
std::mutex mu; std::mutex mu;
for (const auto & s : data)
/// get random permutation of indexes
std::vector<size_t> indexes(data.size());
std::iota(indexes.begin(), indexes.end(), 0);
std::shuffle(indexes.begin(), indexes.end(), rng);
for (auto i : indexes)
{ {
/// Write from diffetent threads to check /// Write from diffetent threads to check
/// that no assertions inside cache related to downloaderId are triggered /// that no assertions inside cache related to downloaderId are triggered
const auto & s = data[i];
threads.emplace_back([&] threads.emplace_back([&]
{ {
std::unique_lock lock(mu); std::unique_lock lock(mu);
out.write(s.data(), s.size()); out.write(s.data(), s.size());
/// test different buffering scenarios /// test different buffering scenarios
if (flush) if (flush)
{
out.next(); out.next();
}
}); });
} }
for (auto & t : threads) for (auto & t : threads)
t.join(); t.join();
out.finalize(); out.finalize();
if (out_read_buffer)
*out_read_buffer = out.tryGetReadBuffer();
return holder; return holder;
}; };
@ -721,17 +745,31 @@ TEST_F(FileCacheTest, writeBuffer)
file_segment_paths.emplace_back(holder->front().getPathInLocalCache()); file_segment_paths.emplace_back(holder->front().getPathInLocalCache());
ASSERT_EQ(fs::file_size(file_segment_paths.back()), 7); ASSERT_EQ(fs::file_size(file_segment_paths.back()), 7);
ASSERT_TRUE(holder->front().range() == FileSegment::Range(0, 7)); EXPECT_EQ(holder->front().range().size(), 7);
EXPECT_EQ(holder->front().range().left, 0);
ASSERT_EQ(cache.getUsedCacheSize(), 7); ASSERT_EQ(cache.getUsedCacheSize(), 7);
{ {
auto holder2 = write_to_cache("key2", {"1", "22", "333", "4444", "55555"}, true); ReadBufferPtr reader = nullptr;
auto holder2 = write_to_cache("key2", {"22", "333", "4444", "55555", "1"}, true, &reader);
file_segment_paths.emplace_back(holder2->front().getPathInLocalCache()); file_segment_paths.emplace_back(holder2->front().getPathInLocalCache());
std::cerr << "\nFile segments: " << holder2->toString() << "\n"; std::cerr << "\nFile segments: " << holder2->toString() << "\n";
ASSERT_EQ(fs::file_size(file_segment_paths.back()), 15); ASSERT_EQ(fs::file_size(file_segment_paths.back()), 15);
ASSERT_EQ(holder2->front().range(), FileSegment::Range(0, 15)); EXPECT_TRUE(reader);
if (reader)
{
String result;
readStringUntilEOF(result, *reader);
/// sort result to make it independent of the order of writes
std::sort(result.begin(), result.end());
EXPECT_EQ(result, "122333444455555");
}
EXPECT_EQ(holder2->front().range().size(), 15);
EXPECT_EQ(holder2->front().range().left, 0);
ASSERT_EQ(cache.getUsedCacheSize(), 22); ASSERT_EQ(cache.getUsedCacheSize(), 22);
} }
ASSERT_FALSE(fs::exists(file_segment_paths.back())); ASSERT_FALSE(fs::exists(file_segment_paths.back()));

View File

@ -5,12 +5,39 @@
namespace DB namespace DB
{ {
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
void ASTPartition::setPartitionID(const ASTPtr & ast)
{
if (children.empty())
{
children.push_back(ast);
id = children[0].get();
}
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot have multiple children for partition AST");
}
void ASTPartition::setPartitionValue(const ASTPtr & ast)
{
if (children.empty())
{
children.push_back(ast);
value = children[0].get();
}
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot have multiple children for partition AST");
}
String ASTPartition::getID(char delim) const String ASTPartition::getID(char delim) const
{ {
if (value) if (value)
return "Partition"; return "Partition";
else else
return "Partition_ID" + (delim + id); return "Partition_ID" + (delim + id->getID());
} }
ASTPtr ASTPartition::clone() const ASTPtr ASTPartition::clone() const
@ -20,8 +47,14 @@ ASTPtr ASTPartition::clone() const
if (value) if (value)
{ {
res->value = value->clone(); res->children.push_back(children[0]->clone());
res->children.push_back(res->value); res->value = res->children[0].get();
}
if (id)
{
res->children.push_back(children[0]->clone());
res->id = res->children[0].get();
} }
return res; return res;
@ -33,18 +66,14 @@ void ASTPartition::formatImpl(const FormatSettings & settings, FormatState & sta
{ {
value->formatImpl(settings, state, frame); value->formatImpl(settings, state, frame);
} }
else if (all)
{
settings.ostr << "ALL";
}
else else
{ {
if (all) settings.ostr << (settings.hilite ? hilite_keyword : "") << "ID " << (settings.hilite ? hilite_none : "");
settings.ostr << "ALL"; id->formatImpl(settings, state, frame);
else
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "ID " << (settings.hilite ? hilite_none : "");
WriteBufferFromOwnString id_buf;
writeQuoted(id, id_buf);
settings.ostr << id_buf.str();
}
} }
} }
} }

View File

@ -1,7 +1,7 @@
#pragma once #pragma once
#include <Parsers/IAST.h> #include <Parsers/IAST.h>
#include <optional>
namespace DB namespace DB
{ {
@ -10,15 +10,24 @@ namespace DB
class ASTPartition : public IAST class ASTPartition : public IAST
{ {
public: public:
ASTPtr value; IAST * value{nullptr};
size_t fields_count = 0; std::optional<size_t> fields_count;
String id; IAST * id{nullptr};
bool all = false; bool all = false;
String getID(char) const override; String getID(char) const override;
ASTPtr clone() const override; ASTPtr clone() const override;
void setPartitionID(const ASTPtr & ast);
void setPartitionValue(const ASTPtr & ast);
void forEachPointerToChild(std::function<void(void **)> f) override
{
f(reinterpret_cast<void **>(&value));
f(reinterpret_cast<void **>(&id));
}
protected: protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
}; };

View File

@ -173,7 +173,7 @@ protected:
class ParserExpression : public IParserBase class ParserExpression : public IParserBase
{ {
public: public:
ParserExpression(bool allow_trailing_commas_ = false) : allow_trailing_commas(allow_trailing_commas_) {} explicit 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"; }

View File

@ -7,6 +7,7 @@
#include <Parsers/ASTFunction.h> #include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <Parsers/ASTQueryParameter.h>
namespace DB namespace DB
{ {
@ -16,19 +17,25 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserKeyword s_id("ID"); ParserKeyword s_id("ID");
ParserKeyword s_all("ALL"); ParserKeyword s_all("ALL");
ParserStringLiteral parser_string_literal; ParserStringLiteral parser_string_literal;
ParserSubstitution parser_substitution;
ParserLiteral literal_parser;
ParserTupleOfLiterals tuple_of_literals;
ParserExpression parser_expr; ParserExpression parser_expr;
Pos begin = pos;
auto partition = std::make_shared<ASTPartition>(); auto partition = std::make_shared<ASTPartition>();
if (s_id.ignore(pos, expected)) if (s_id.ignore(pos, expected))
{ {
ASTPtr partition_id; ASTPtr partition_id;
if (!parser_string_literal.parse(pos, partition_id, expected)) if (!parser_string_literal.parse(pos, partition_id, expected) && !parser_substitution.parse(pos, partition_id, expected))
return false; return false;
partition->id = partition_id->as<ASTLiteral &>().value.get<String>(); if (auto * partition_id_literal = partition_id->as<ASTLiteral>(); partition_id_literal != nullptr)
partition->setPartitionID(partition_id);
else if (auto * partition_id_query_parameter = partition_id->as<ASTQueryParameter>(); partition_id_query_parameter != nullptr)
partition->setPartitionID(partition_id);
else
return false;
} }
else if (s_all.ignore(pos, expected)) else if (s_all.ignore(pos, expected))
{ {
@ -37,27 +44,12 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
else else
{ {
ASTPtr value; ASTPtr value;
if (!parser_expr.parse(pos, value, expected)) std::optional<size_t> fields_count;
return false; if (literal_parser.parse(pos, value, expected) || tuple_of_literals.parse(pos, value, expected))
size_t fields_count;
const auto * tuple_ast = value->as<ASTFunction>();
bool surrounded_by_parens = false;
if (tuple_ast && tuple_ast->name == "tuple")
{
surrounded_by_parens = true;
const auto * arguments_ast = tuple_ast->arguments->as<ASTExpressionList>();
if (arguments_ast)
fields_count = arguments_ast->children.size();
else
fields_count = 0;
}
else if (const auto * literal = value->as<ASTLiteral>())
{ {
auto * literal = value->as<ASTLiteral>();
if (literal->value.getType() == Field::Types::Tuple) if (literal->value.getType() == Field::Types::Tuple)
{ {
surrounded_by_parens = true;
fields_count = literal->value.get<const Tuple &>().size(); fields_count = literal->value.get<const Tuple &>().size();
} }
else else
@ -65,27 +57,31 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
fields_count = 1; fields_count = 1;
} }
} }
else else if (parser_substitution.parse(pos, value, expected))
return false;
if (surrounded_by_parens)
{ {
Pos left_paren = begin; /// It can be tuple substitution
Pos right_paren = pos; fields_count = std::nullopt;
}
while (left_paren != right_paren && left_paren->type != TokenType::OpeningRoundBracket) else if (parser_expr.parse(pos, value, expected))
++left_paren; {
if (left_paren->type != TokenType::OpeningRoundBracket) const auto * tuple_ast = value->as<ASTFunction>();
return false; if (tuple_ast && tuple_ast->name == "tuple")
{
while (right_paren != left_paren && right_paren->type != TokenType::ClosingRoundBracket) const auto * arguments_ast = tuple_ast->arguments->as<ASTExpressionList>();
--right_paren; if (arguments_ast)
if (right_paren->type != TokenType::ClosingRoundBracket) fields_count = arguments_ast->children.size();
else
fields_count = 0;
}
else
return false; return false;
} }
else
{
return false;
}
partition->value = value; partition->setPartitionValue(value);
partition->children.push_back(value);
partition->fields_count = fields_count; partition->fields_count = fields_count;
} }

View File

@ -11,6 +11,7 @@ namespace DB
bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{ {
ParserKeyword s_rename("RENAME");
ParserKeyword s_rename_table("RENAME TABLE"); ParserKeyword s_rename_table("RENAME TABLE");
ParserKeyword s_exchange_tables("EXCHANGE TABLES"); ParserKeyword s_exchange_tables("EXCHANGE TABLES");
ParserKeyword s_rename_dictionary("RENAME DICTIONARY"); ParserKeyword s_rename_dictionary("RENAME DICTIONARY");
@ -24,18 +25,7 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
bool exchange = false; bool exchange = false;
bool dictionary = false; bool dictionary = false;
if (s_rename_table.ignore(pos, expected)) if (s_rename_database.ignore(pos, expected))
;
else if (s_exchange_tables.ignore(pos, expected))
exchange = true;
else if (s_rename_dictionary.ignore(pos, expected))
dictionary = true;
else if (s_exchange_dictionaries.ignore(pos, expected))
{
exchange = true;
dictionary = true;
}
else if (s_rename_database.ignore(pos, expected))
{ {
ASTPtr from_db; ASTPtr from_db;
ASTPtr to_db; ASTPtr to_db;
@ -67,6 +57,19 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
node = query; node = query;
return true; return true;
} }
else if (s_rename_table.ignore(pos, expected))
;
else if (s_exchange_tables.ignore(pos, expected))
exchange = true;
else if (s_rename_dictionary.ignore(pos, expected))
dictionary = true;
else if (s_exchange_dictionaries.ignore(pos, expected))
{
exchange = true;
dictionary = true;
}
else if (s_rename.ignore(pos, expected))
;
else else
return false; return false;

View File

@ -42,6 +42,7 @@
#include <Storages/ColumnsDescription.h> #include <Storages/ColumnsDescription.h>
#include <Storages/SelectQueryInfo.h> #include <Storages/SelectQueryInfo.h>
#include <Storages/StorageDummy.h> #include <Storages/StorageDummy.h>
#include <Storages/StorageDistributed.h>
#include <Storages/IStorage.h> #include <Storages/IStorage.h>
#include <Analyzer/Utils.h> #include <Analyzer/Utils.h>
@ -138,6 +139,84 @@ void checkStoragesSupportTransactions(const PlannerContextPtr & planner_context)
} }
} }
/** Storages can rely that filters that for storage will be available for analysis before
* getQueryProcessingStage method will be called.
*
* StorageDistributed skip unused shards optimization relies on this.
*
* To collect filters that will be applied to specific table in case we have JOINs requires
* to run query plan optimization pipeline.
*
* Algorithm:
* 1. Replace all table expressions in query tree with dummy tables.
* 2. Build query plan.
* 3. Optimize query plan.
* 4. Extract filters from ReadFromDummy query plan steps from query plan leaf nodes.
*/
void collectFiltersForAnalysis(const QueryTreeNodePtr & query_tree, const PlannerContextPtr & planner_context)
{
bool collect_filters = false;
for (auto & [table_expression, table_expression_data] : planner_context->getTableExpressionNodeToData())
{
auto * table_node = table_expression->as<TableNode>();
auto * table_function_node = table_expression->as<TableFunctionNode>();
if (!table_node && !table_function_node)
continue;
const auto & storage = table_node ? table_node->getStorage() : table_function_node->getStorage();
if (typeid_cast<const StorageDistributed *>(storage.get()))
{
collect_filters = true;
break;
}
}
if (!collect_filters)
return;
ResultReplacementMap replacement_map;
auto updated_query_tree = replaceTableExpressionsWithDummyTables(query_tree, planner_context->getQueryContext(), &replacement_map);
std::unordered_map<const IStorage *, TableExpressionData *> dummy_storage_to_table_expression_data;
for (auto & [from_table_expression, dummy_table_expression] : replacement_map)
{
auto * dummy_storage = dummy_table_expression->as<TableNode &>().getStorage().get();
auto * table_expression_data = &planner_context->getTableExpressionDataOrThrow(from_table_expression);
dummy_storage_to_table_expression_data.emplace(dummy_storage, table_expression_data);
}
const auto & query_context = planner_context->getQueryContext();
SelectQueryOptions select_query_options;
Planner planner(updated_query_tree, select_query_options);
planner.buildQueryPlanIfNeeded();
auto & result_query_plan = planner.getQueryPlan();
auto optimization_settings = QueryPlanOptimizationSettings::fromContext(query_context);
result_query_plan.optimize(optimization_settings);
std::vector<QueryPlan::Node *> nodes_to_process;
nodes_to_process.push_back(result_query_plan.getRootNode());
while (!nodes_to_process.empty())
{
const auto * node_to_process = nodes_to_process.back();
nodes_to_process.pop_back();
nodes_to_process.insert(nodes_to_process.end(), node_to_process->children.begin(), node_to_process->children.end());
auto * read_from_dummy = typeid_cast<ReadFromDummy *>(node_to_process->step.get());
if (!read_from_dummy)
continue;
auto filter_actions = ActionsDAG::buildFilterActionsDAG(read_from_dummy->getFilterNodes().nodes, {}, query_context);
auto & table_expression_data = dummy_storage_to_table_expression_data.at(&read_from_dummy->getStorage());
table_expression_data->setFilterActions(std::move(filter_actions));
}
}
/// Extend lifetime of query context, storages, and table locks /// Extend lifetime of query context, storages, and table locks
void extendQueryContextAndStoragesLifetime(QueryPlan & query_plan, const PlannerContextPtr & planner_context) void extendQueryContextAndStoragesLifetime(QueryPlan & query_plan, const PlannerContextPtr & planner_context)
{ {
@ -1227,6 +1306,9 @@ void Planner::buildPlanForQueryNode()
collectSets(query_tree, *planner_context); collectSets(query_tree, *planner_context);
collectTableExpressionData(query_tree, planner_context); collectTableExpressionData(query_tree, planner_context);
if (!select_query_options.only_analyze)
collectFiltersForAnalysis(query_tree, planner_context);
const auto & settings = query_context->getSettingsRef(); const auto & settings = query_context->getSettingsRef();
/// Check support for JOIN for parallel replicas with custom key /// Check support for JOIN for parallel replicas with custom key

View File

@ -46,6 +46,8 @@
#include <Processors/QueryPlan/ArrayJoinStep.h> #include <Processors/QueryPlan/ArrayJoinStep.h>
#include <Processors/Sources/SourceFromSingleChunk.h> #include <Processors/Sources/SourceFromSingleChunk.h>
#include <Storages/StorageDummy.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Interpreters/IJoin.h> #include <Interpreters/IJoin.h>
#include <Interpreters/TableJoin.h> #include <Interpreters/TableJoin.h>
@ -84,6 +86,10 @@ namespace
/// Check if current user has privileges to SELECT columns from table /// Check if current user has privileges to SELECT columns from table
void checkAccessRights(const TableNode & table_node, const Names & column_names, const ContextPtr & query_context) void checkAccessRights(const TableNode & table_node, const Names & column_names, const ContextPtr & query_context)
{ {
/// StorageDummy is created on preliminary stage, ignore access check for it.
if (typeid_cast<const StorageDummy *>(table_node.getStorage().get()))
return;
const auto & storage_id = table_node.getStorageID(); const auto & storage_id = table_node.getStorageID();
const auto & storage_snapshot = table_node.getStorageSnapshot(); const auto & storage_snapshot = table_node.getStorageSnapshot();
@ -553,6 +559,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
auto table_expression_query_info = select_query_info; auto table_expression_query_info = select_query_info;
table_expression_query_info.table_expression = table_expression; table_expression_query_info.table_expression = table_expression;
table_expression_query_info.filter_actions_dag = table_expression_data.getFilterActions();
size_t max_streams = settings.max_threads; size_t max_streams = settings.max_threads;
size_t max_threads_execute_query = settings.max_threads; size_t max_threads_execute_query = settings.max_threads;

View File

@ -355,24 +355,52 @@ QueryTreeNodePtr mergeConditionNodes(const QueryTreeNodes & condition_nodes, con
return function_node; return function_node;
} }
QueryTreeNodePtr replaceTablesAndTableFunctionsWithDummyTables(const QueryTreeNodePtr & query_node, QueryTreeNodePtr replaceTableExpressionsWithDummyTables(const QueryTreeNodePtr & query_node,
const ContextPtr & context, const ContextPtr & context,
ResultReplacementMap * result_replacement_map) ResultReplacementMap * result_replacement_map)
{ {
auto & query_node_typed = query_node->as<QueryNode &>(); auto & query_node_typed = query_node->as<QueryNode &>();
auto table_expressions = extractTableExpressions(query_node_typed.getJoinTree()); auto table_expressions = extractTableExpressions(query_node_typed.getJoinTree());
std::unordered_map<const IQueryTreeNode *, QueryTreeNodePtr> replacement_map; std::unordered_map<const IQueryTreeNode *, QueryTreeNodePtr> replacement_map;
size_t subquery_index = 0;
for (auto & table_expression : table_expressions) for (auto & table_expression : table_expressions)
{ {
auto * table_node = table_expression->as<TableNode>(); auto * table_node = table_expression->as<TableNode>();
auto * table_function_node = table_expression->as<TableFunctionNode>(); auto * table_function_node = table_expression->as<TableFunctionNode>();
if (!table_node && !table_function_node) auto * subquery_node = table_expression->as<QueryNode>();
continue; auto * union_node = table_expression->as<UnionNode>();
StoragePtr storage_dummy;
if (table_node || table_function_node)
{
const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot();
auto get_column_options = GetColumnsOptions(GetColumnsOptions::All).withExtendedObjects().withVirtuals();
storage_dummy
= std::make_shared<StorageDummy>(storage_snapshot->storage.getStorageID(), ColumnsDescription(storage_snapshot->getColumns(get_column_options)));
}
else if (subquery_node || union_node)
{
const auto & subquery_projection_columns
= subquery_node ? subquery_node->getProjectionColumns() : union_node->computeProjectionColumns();
NameSet unique_column_names;
NamesAndTypes storage_dummy_columns;
storage_dummy_columns.reserve(subquery_projection_columns.size());
for (const auto & projection_column : subquery_projection_columns)
{
auto [_, inserted] = unique_column_names.insert(projection_column.name);
if (inserted)
storage_dummy_columns.emplace_back(projection_column);
}
storage_dummy = std::make_shared<StorageDummy>(StorageID{"dummy", "subquery_" + std::to_string(subquery_index)}, ColumnsDescription(storage_dummy_columns));
++subquery_index;
}
const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot();
auto storage_dummy = std::make_shared<StorageDummy>(storage_snapshot->storage.getStorageID(),
storage_snapshot->metadata->getColumns());
auto dummy_table_node = std::make_shared<TableNode>(std::move(storage_dummy), context); auto dummy_table_node = std::make_shared<TableNode>(std::move(storage_dummy), context);
if (result_replacement_map) if (result_replacement_map)

View File

@ -65,9 +65,9 @@ bool queryHasWithTotalsInAnySubqueryInJoinTree(const QueryTreeNodePtr & query_no
/// Returns `and` function node that has condition nodes as its arguments /// Returns `and` function node that has condition nodes as its arguments
QueryTreeNodePtr mergeConditionNodes(const QueryTreeNodes & condition_nodes, const ContextPtr & context); QueryTreeNodePtr mergeConditionNodes(const QueryTreeNodes & condition_nodes, const ContextPtr & context);
/// Replace tables nodes and table function nodes with dummy table nodes /// Replace table expressions from query JOIN TREE with dummy tables
using ResultReplacementMap = std::unordered_map<QueryTreeNodePtr, QueryTreeNodePtr>; using ResultReplacementMap = std::unordered_map<QueryTreeNodePtr, QueryTreeNodePtr>;
QueryTreeNodePtr replaceTablesAndTableFunctionsWithDummyTables(const QueryTreeNodePtr & query_node, QueryTreeNodePtr replaceTableExpressionsWithDummyTables(const QueryTreeNodePtr & query_node,
const ContextPtr & context, const ContextPtr & context,
ResultReplacementMap * result_replacement_map = nullptr); ResultReplacementMap * result_replacement_map = nullptr);

View File

@ -226,7 +226,7 @@ struct StatisticsStringRef
/// or [element of ColumnString] -> std::string_view. /// or [element of ColumnString] -> std::string_view.
/// We do this conversion in small batches rather than all at once, just before encoding the batch, /// We do this conversion in small batches rather than all at once, just before encoding the batch,
/// in hopes of getting better performance through cache locality. /// in hopes of getting better performance through cache locality.
/// The Coverter* structs below are responsible for that. /// The Converter* structs below are responsible for that.
/// When conversion is not needed, getBatch() will just return pointer into original data. /// When conversion is not needed, getBatch() will just return pointer into original data.
template <typename Col, typename To, typename MinMaxType = typename std::conditional_t< template <typename Col, typename To, typename MinMaxType = typename std::conditional_t<

View File

@ -718,7 +718,29 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams(RangesInDataParts && parts_
{ {
/// Reduce the number of num_streams if the data is small. /// Reduce the number of num_streams if the data is small.
if (info.sum_marks < num_streams * info.min_marks_for_concurrent_read && parts_with_ranges.size() < num_streams) if (info.sum_marks < num_streams * info.min_marks_for_concurrent_read && parts_with_ranges.size() < num_streams)
num_streams = std::max((info.sum_marks + info.min_marks_for_concurrent_read - 1) / info.min_marks_for_concurrent_read, parts_with_ranges.size()); {
/*
If the data is fragmented, then allocate the size of parts to num_streams. If the data is not fragmented, besides the sum_marks and
min_marks_for_concurrent_read, involve the system cores to get the num_streams. Increase the num_streams and decrease the min_marks_for_concurrent_read
if the data is small but system has plentiful cores. It helps to improve the parallel performance of `MergeTreeRead` significantly.
Make sure the new num_streams `num_streams * increase_num_streams_ratio` will not exceed the previous calculated prev_num_streams.
The new info.min_marks_for_concurrent_read `info.min_marks_for_concurrent_read / increase_num_streams_ratio` should be larger than 8.
https://github.com/ClickHouse/ClickHouse/pull/53867
*/
if ((info.sum_marks + info.min_marks_for_concurrent_read - 1) / info.min_marks_for_concurrent_read > parts_with_ranges.size())
{
const size_t prev_num_streams = num_streams;
num_streams = (info.sum_marks + info.min_marks_for_concurrent_read - 1) / info.min_marks_for_concurrent_read;
const size_t increase_num_streams_ratio = std::min(prev_num_streams / num_streams, info.min_marks_for_concurrent_read / 8);
if (increase_num_streams_ratio > 1)
{
num_streams = num_streams * increase_num_streams_ratio;
info.min_marks_for_concurrent_read = (info.sum_marks + num_streams - 1) / num_streams;
}
}
else
num_streams = parts_with_ranges.size();
}
} }
auto read_type = is_parallel_reading_from_replicas ? ReadType::ParallelReplicas : ReadType::Default; auto read_type = is_parallel_reading_from_replicas ? ReadType::ParallelReplicas : ReadType::Default;

View File

@ -45,7 +45,8 @@ static void limitProgressingSpeed(size_t total_progress_size, size_t max_speed_i
void ExecutionSpeedLimits::throttle( void ExecutionSpeedLimits::throttle(
size_t read_rows, size_t read_bytes, size_t read_rows, size_t read_bytes,
size_t total_rows_to_read, UInt64 total_elapsed_microseconds) const size_t total_rows_to_read, UInt64 total_elapsed_microseconds,
OverflowMode timeout_overflow_mode) const
{ {
if ((min_execution_rps != 0 || max_execution_rps != 0 if ((min_execution_rps != 0 || max_execution_rps != 0
|| min_execution_bps != 0 || max_execution_bps != 0 || min_execution_bps != 0 || max_execution_bps != 0
@ -82,7 +83,7 @@ void ExecutionSpeedLimits::throttle(
{ {
double estimated_execution_time_seconds = elapsed_seconds * (static_cast<double>(total_rows_to_read) / read_rows); double estimated_execution_time_seconds = elapsed_seconds * (static_cast<double>(total_rows_to_read) / read_rows);
if (estimated_execution_time_seconds > max_execution_time.totalSeconds()) if (timeout_overflow_mode == OverflowMode::THROW && estimated_execution_time_seconds > max_execution_time.totalSeconds())
throw Exception( throw Exception(
ErrorCodes::TOO_SLOW, ErrorCodes::TOO_SLOW,
"Estimated query execution time ({} seconds) is too long. Maximum: {}. Estimated rows to process: {}", "Estimated query execution time ({} seconds) is too long. Maximum: {}. Estimated rows to process: {}",

View File

@ -25,7 +25,8 @@ public:
Poco::Timespan timeout_before_checking_execution_speed = 0; Poco::Timespan timeout_before_checking_execution_speed = 0;
/// Pause execution in case if speed limits were exceeded. /// Pause execution in case if speed limits were exceeded.
void throttle(size_t read_rows, size_t read_bytes, size_t total_rows_to_read, UInt64 total_elapsed_microseconds) const; void throttle(size_t read_rows, size_t read_bytes, size_t total_rows_to_read, UInt64 total_elapsed_microseconds,
OverflowMode timeout_overflow_mode) const;
bool checkTimeLimit(const Stopwatch & stopwatch, OverflowMode overflow_mode) const; bool checkTimeLimit(const Stopwatch & stopwatch, OverflowMode overflow_mode) const;
}; };

View File

@ -130,7 +130,7 @@ bool ReadProgressCallback::onProgress(uint64_t read_rows, uint64_t read_bytes, c
/// TODO: Should be done in PipelineExecutor. /// TODO: Should be done in PipelineExecutor.
for (const auto & limits : storage_limits) for (const auto & limits : storage_limits)
limits.local_limits.speed_limits.throttle(progress.read_rows, progress.read_bytes, total_rows, total_stopwatch.elapsedMicroseconds()); limits.local_limits.speed_limits.throttle(progress.read_rows, progress.read_bytes, total_rows, total_stopwatch.elapsedMicroseconds(), limits.local_limits.timeout_overflow_mode);
if (quota) if (quota)
quota->used({QuotaType::READ_ROWS, value.read_rows}, {QuotaType::READ_BYTES, value.read_bytes}); quota->used({QuotaType::READ_ROWS, value.read_rows}, {QuotaType::READ_BYTES, value.read_bytes});

View File

@ -17,9 +17,11 @@ void printPipeline(const Processors & processors, const Statuses & statuses, Wri
out << " rankdir=\"LR\";\n"; out << " rankdir=\"LR\";\n";
out << " { node [shape = rect]\n"; out << " { node [shape = rect]\n";
auto get_proc_id = [](const IProcessor & proc) -> UInt64 std::unordered_map<const void *, std::size_t> pointer_to_id;
auto get_proc_id = [&](const IProcessor & proc) -> std::size_t
{ {
return reinterpret_cast<std::uintptr_t>(&proc); auto [it, inserted] = pointer_to_id.try_emplace(&proc, pointer_to_id.size());
return it->second;
}; };
auto statuses_iter = statuses.begin(); auto statuses_iter = statuses.begin();

View File

@ -144,6 +144,17 @@ void ColumnDescription::readText(ReadBuffer & buf)
} }
} }
ColumnsDescription::ColumnsDescription(std::initializer_list<NameAndTypePair> ordinary)
{
for (const auto & elem : ordinary)
add(ColumnDescription(elem.name, elem.type));
}
ColumnsDescription::ColumnsDescription(NamesAndTypes ordinary)
{
for (auto & elem : ordinary)
add(ColumnDescription(std::move(elem.name), std::move(elem.type)));
}
ColumnsDescription::ColumnsDescription(NamesAndTypesList ordinary) ColumnsDescription::ColumnsDescription(NamesAndTypesList ordinary)
{ {

View File

@ -102,6 +102,11 @@ class ColumnsDescription : public IHints<>
{ {
public: public:
ColumnsDescription() = default; ColumnsDescription() = default;
ColumnsDescription(std::initializer_list<NameAndTypePair> ordinary);
explicit ColumnsDescription(NamesAndTypes ordinary);
explicit ColumnsDescription(NamesAndTypesList ordinary); explicit ColumnsDescription(NamesAndTypesList ordinary);
explicit ColumnsDescription(NamesAndTypesList ordinary, NamesAndAliases aliases); explicit ColumnsDescription(NamesAndTypesList ordinary, NamesAndAliases aliases);

View File

@ -211,7 +211,10 @@ void IStorage::checkMutationIsPossible(const MutationCommands & /*commands*/, co
} }
void IStorage::checkAlterPartitionIsPossible( void IStorage::checkAlterPartitionIsPossible(
const PartitionCommands & /*commands*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Settings & /*settings*/) const const PartitionCommands & /*commands*/,
const StorageMetadataPtr & /*metadata_snapshot*/,
const Settings & /*settings*/,
ContextPtr /*context*/) const
{ {
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitioning", getName()); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitioning", getName());
} }

View File

@ -495,7 +495,11 @@ public:
ContextPtr /* context */); ContextPtr /* context */);
/// Checks that partition commands can be applied to storage. /// Checks that partition commands can be applied to storage.
virtual void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const; virtual void checkAlterPartitionIsPossible(
const PartitionCommands & commands,
const StorageMetadataPtr & metadata_snapshot,
const Settings & settings,
ContextPtr context) const;
/** Perform any background work. For example, combining parts in a MergeTree type table. /** Perform any background work. For example, combining parts in a MergeTree type table.
* Returns whether any work has been done. * Returns whether any work has been done.

View File

@ -451,7 +451,8 @@ const KeyCondition::AtomMap KeyCondition::atom_map
}; };
static const std::map<std::string, std::string> inverse_relations = { static const std::map<std::string, std::string> inverse_relations =
{
{"equals", "notEquals"}, {"equals", "notEquals"},
{"notEquals", "equals"}, {"notEquals", "equals"},
{"less", "greaterOrEquals"}, {"less", "greaterOrEquals"},
@ -475,7 +476,7 @@ static const std::map<std::string, std::string> inverse_relations = {
}; };
bool isLogicalOperator(const String & func_name) static bool isLogicalOperator(const String & func_name)
{ {
return (func_name == "and" || func_name == "or" || func_name == "not" || func_name == "indexHint"); return (func_name == "and" || func_name == "or" || func_name == "not" || func_name == "indexHint");
} }
@ -954,10 +955,10 @@ static std::set<std::string_view> date_time_parsing_functions = {
"toDate32", "toDate32",
"toDateTime", "toDateTime",
"toDateTime64", "toDateTime64",
"ParseDateTimeBestEffort", "parseDateTimeBestEffort",
"ParseDateTimeBestEffortUS", "parseDateTimeBestEffortUS",
"ParseDateTime32BestEffort", "parseDateTime32BestEffort",
"ParseDateTime64BestEffort", "parseDateTime64BestEffort",
"parseDateTime", "parseDateTime",
"parseDateTimeInJodaSyntax", "parseDateTimeInJodaSyntax",
}; };
@ -1482,7 +1483,7 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl(
*/ */
const auto & sample_block = key_expr->getSampleBlock(); const auto & sample_block = key_expr->getSampleBlock();
// Key columns should use canonical names for index analysis /// Key columns should use canonical names for the index analysis.
String name = node.getColumnName(); String name = node.getColumnName();
if (array_joined_column_names.contains(name)) if (array_joined_column_names.contains(name))
@ -1685,7 +1686,7 @@ bool KeyCondition::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNEleme
func_name = "greaterOrEquals"; func_name = "greaterOrEquals";
else if (func_name == "in" || func_name == "notIn" || else if (func_name == "in" || func_name == "notIn" ||
func_name == "like" || func_name == "notLike" || func_name == "like" || func_name == "notLike" ||
func_name == "ilike" || func_name == "notIlike" || func_name == "ilike" || func_name == "notILike" ||
func_name == "startsWith" || func_name == "match") func_name == "startsWith" || func_name == "match")
{ {
/// "const IN data_column" doesn't make sense (unlike "data_column IN const") /// "const IN data_column" doesn't make sense (unlike "data_column IN const")

View File

@ -112,6 +112,13 @@ void MergeTreeBackgroundExecutor<Queue>::increaseThreadsAndMaxTasksCount(size_t
threads_count = new_threads_count; threads_count = new_threads_count;
} }
template <class Queue>
size_t MergeTreeBackgroundExecutor<Queue>::getMaxThreads() const
{
std::lock_guard lock(mutex);
return threads_count;
}
template <class Queue> template <class Queue>
size_t MergeTreeBackgroundExecutor<Queue>::getMaxTasksCount() const size_t MergeTreeBackgroundExecutor<Queue>::getMaxTasksCount() const
{ {

View File

@ -270,6 +270,8 @@ public:
/// implementing tasks eviction will definitely be too error-prone and buggy. /// implementing tasks eviction will definitely be too error-prone and buggy.
void increaseThreadsAndMaxTasksCount(size_t new_threads_count, size_t new_max_tasks_count); void increaseThreadsAndMaxTasksCount(size_t new_threads_count, size_t new_max_tasks_count);
size_t getMaxThreads() const;
/// This method can return stale value of max_tasks_count (no mutex locking). /// This method can return stale value of max_tasks_count (no mutex locking).
/// It's okay because amount of tasks can be only increased and getting stale value /// It's okay because amount of tasks can be only increased and getting stale value
/// can lead only to some postponing, not logical error. /// can lead only to some postponing, not logical error.

View File

@ -37,6 +37,7 @@
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Interpreters/convertFieldToType.h> #include <Interpreters/convertFieldToType.h>
#include <Interpreters/evaluateConstantExpression.h> #include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/ReplaceQueryParameterVisitor.h>
#include <Interpreters/ExpressionAnalyzer.h> #include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/inplaceBlockConversions.h> #include <Interpreters/inplaceBlockConversions.h>
#include <Interpreters/InterpreterSelectQuery.h> #include <Interpreters/InterpreterSelectQuery.h>
@ -52,6 +53,7 @@
#include <IO/SharedThreadPools.h> #include <IO/SharedThreadPools.h>
#include <Parsers/ASTExpressionList.h> #include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTIndexDeclaration.h> #include <Parsers/ASTIndexDeclaration.h>
#include <Parsers/ASTHelpers.h>
#include <Parsers/ASTFunction.h> #include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h> #include <Parsers/ASTLiteral.h>
#include <Parsers/ASTNameTypePair.h> #include <Parsers/ASTNameTypePair.h>
@ -62,6 +64,7 @@
#include <Parsers/parseQuery.h> #include <Parsers/parseQuery.h>
#include <Parsers/queryToString.h> #include <Parsers/queryToString.h>
#include <Parsers/ASTAlterQuery.h> #include <Parsers/ASTAlterQuery.h>
#include <Parsers/ASTQueryParameter.h>
#include <Processors/Formats/IInputFormat.h> #include <Processors/Formats/IInputFormat.h>
#include <Processors/QueryPlan/QueryIdHolder.h> #include <Processors/QueryPlan/QueryIdHolder.h>
#include <Processors/QueryPlan/ReadFromMergeTree.h> #include <Processors/QueryPlan/ReadFromMergeTree.h>
@ -2967,7 +2970,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
/// Just validate partition expression /// Just validate partition expression
if (command.partition) if (command.partition)
{ {
getPartitionIDFromQuery(command.partition, getContext()); getPartitionIDFromQuery(command.partition, local_context);
} }
if (command.column_name == merging_params.version_column) if (command.column_name == merging_params.version_column)
@ -4635,7 +4638,7 @@ void MergeTreeData::removePartContributionToColumnAndSecondaryIndexSizes(const D
} }
void MergeTreeData::checkAlterPartitionIsPossible( void MergeTreeData::checkAlterPartitionIsPossible(
const PartitionCommands & commands, const StorageMetadataPtr & /*metadata_snapshot*/, const Settings & settings) const const PartitionCommands & commands, const StorageMetadataPtr & /*metadata_snapshot*/, const Settings & settings, ContextPtr local_context) const
{ {
for (const auto & command : commands) for (const auto & command : commands)
{ {
@ -4663,7 +4666,7 @@ void MergeTreeData::checkAlterPartitionIsPossible(
throw DB::Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only support DROP/DETACH PARTITION ALL currently"); throw DB::Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only support DROP/DETACH PARTITION ALL currently");
} }
else else
getPartitionIDFromQuery(command.partition, getContext()); getPartitionIDFromQuery(command.partition, local_context);
} }
} }
} }
@ -5342,8 +5345,71 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc
if (!partition_ast.value) if (!partition_ast.value)
{ {
MergeTreePartInfo::validatePartitionID(partition_ast.id, format_version); MergeTreePartInfo::validatePartitionID(partition_ast.id->clone(), format_version);
return partition_ast.id; return partition_ast.id->as<ASTLiteral>()->value.safeGet<String>();
}
size_t partition_ast_fields_count = 0;
ASTPtr partition_value_ast = partition_ast.value->clone();
if (!partition_ast.fields_count.has_value())
{
if (partition_value_ast->as<ASTLiteral>())
{
partition_ast_fields_count = 1;
}
else if (const auto * tuple_ast = partition_value_ast->as<ASTFunction>())
{
if (tuple_ast->name != "tuple")
{
if (isFunctionCast(tuple_ast))
{
if (tuple_ast->arguments->as<ASTExpressionList>()->children.empty())
{
throw Exception(
ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name);
}
auto first_arg = tuple_ast->arguments->as<ASTExpressionList>()->children.at(0);
if (const auto * inner_tuple = first_arg->as<ASTFunction>(); inner_tuple && inner_tuple->name == "tuple")
{
const auto * arguments_ast = tuple_ast->arguments->as<ASTExpressionList>();
if (arguments_ast)
partition_ast_fields_count = arguments_ast->children.size();
else
partition_ast_fields_count = 0;
}
else if (const auto * inner_literal_tuple = first_arg->as<ASTLiteral>(); inner_literal_tuple)
{
if (inner_literal_tuple->value.getType() == Field::Types::Tuple)
partition_ast_fields_count = inner_literal_tuple->value.safeGet<Tuple>().size();
else
partition_ast_fields_count = 1;
}
else
{
throw Exception(
ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name);
}
}
else
throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name);
}
else
{
const auto * arguments_ast = tuple_ast->arguments->as<ASTExpressionList>();
if (arguments_ast)
partition_ast_fields_count = arguments_ast->children.size();
else
partition_ast_fields_count = 0;
}
}
else
{
throw Exception(
ErrorCodes::INVALID_PARTITION_VALUE, "Expected literal or tuple for partition key, got {}", partition_value_ast->getID());
}
}
else
{
partition_ast_fields_count = *partition_ast.fields_count;
} }
if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
@ -5352,9 +5418,8 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc
const auto * partition_lit = partition_ast.value->as<ASTLiteral>(); const auto * partition_lit = partition_ast.value->as<ASTLiteral>();
if (partition_lit && partition_lit->value.getType() == Field::Types::String) if (partition_lit && partition_lit->value.getType() == Field::Types::String)
{ {
String partition_id = partition_lit->value.get<String>(); MergeTreePartInfo::validatePartitionID(partition_ast.value->clone(), format_version);
MergeTreePartInfo::validatePartitionID(partition_id, format_version); return partition_lit->value.get<String>();
return partition_id;
} }
} }
@ -5362,35 +5427,48 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc
auto metadata_snapshot = getInMemoryMetadataPtr(); auto metadata_snapshot = getInMemoryMetadataPtr();
const Block & key_sample_block = metadata_snapshot->getPartitionKey().sample_block; const Block & key_sample_block = metadata_snapshot->getPartitionKey().sample_block;
size_t fields_count = key_sample_block.columns(); size_t fields_count = key_sample_block.columns();
if (partition_ast.fields_count != fields_count) if (partition_ast_fields_count != fields_count)
throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, throw Exception(ErrorCodes::INVALID_PARTITION_VALUE,
"Wrong number of fields in the partition expression: {}, must be: {}", "Wrong number of fields in the partition expression: {}, must be: {}",
partition_ast.fields_count, fields_count); partition_ast_fields_count, fields_count);
Row partition_row(fields_count); Row partition_row(fields_count);
if (fields_count == 0) if (fields_count == 0)
{ {
/// Function tuple(...) requires at least one argument, so empty key is a special case /// Function tuple(...) requires at least one argument, so empty key is a special case
assert(!partition_ast.fields_count); assert(!partition_ast_fields_count);
assert(typeid_cast<ASTFunction *>(partition_ast.value.get())); assert(typeid_cast<ASTFunction *>(partition_value_ast.get()));
assert(partition_ast.value->as<ASTFunction>()->name == "tuple"); assert(partition_value_ast->as<ASTFunction>()->name == "tuple");
assert(partition_ast.value->as<ASTFunction>()->arguments); assert(partition_value_ast->as<ASTFunction>()->arguments);
auto args = partition_ast.value->as<ASTFunction>()->arguments; auto args = partition_value_ast->as<ASTFunction>()->arguments;
if (!args) if (!args)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected at least one argument in partition AST"); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected at least one argument in partition AST");
bool empty_tuple = partition_ast.value->as<ASTFunction>()->arguments->children.empty(); bool empty_tuple = partition_value_ast->as<ASTFunction>()->arguments->children.empty();
if (!empty_tuple) if (!empty_tuple)
throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Partition key is empty, expected 'tuple()' as partition key"); throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Partition key is empty, expected 'tuple()' as partition key");
} }
else if (fields_count == 1) else if (fields_count == 1)
{ {
ASTPtr partition_value_ast = partition_ast.value; if (auto * tuple = partition_value_ast->as<ASTFunction>(); tuple)
if (auto * tuple = partition_value_ast->as<ASTFunction>())
{ {
assert(tuple->name == "tuple"); if (tuple->name == "tuple")
assert(tuple->arguments); {
assert(tuple->arguments->children.size() == 1); assert(tuple->arguments);
partition_value_ast = tuple->arguments->children[0]; assert(tuple->arguments->children.size() == 1);
partition_value_ast = tuple->arguments->children[0];
}
else if (isFunctionCast(tuple))
{
assert(tuple->arguments);
assert(tuple->arguments->children.size() == 2);
}
else
{
throw Exception(
ErrorCodes::INVALID_PARTITION_VALUE,
"Expected literal or tuple for partition key, got {}",
partition_value_ast->getID());
}
} }
/// Simple partition key, need to evaluate and cast /// Simple partition key, need to evaluate and cast
Field partition_key_value = evaluateConstantExpression(partition_value_ast, local_context).first; Field partition_key_value = evaluateConstantExpression(partition_value_ast, local_context).first;
@ -5399,7 +5477,7 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc
else else
{ {
/// Complex key, need to evaluate, untuple and cast /// Complex key, need to evaluate, untuple and cast
Field partition_key_value = evaluateConstantExpression(partition_ast.value, local_context).first; Field partition_key_value = evaluateConstantExpression(partition_value_ast, local_context).first;
if (partition_key_value.getType() != Field::Types::Tuple) if (partition_key_value.getType() != Field::Types::Tuple)
throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, throw Exception(ErrorCodes::INVALID_PARTITION_VALUE,
"Expected tuple for complex partition key, got {}", partition_key_value.getTypeName()); "Expected tuple for complex partition key, got {}", partition_key_value.getTypeName());

View File

@ -723,7 +723,11 @@ public:
void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override; void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override;
/// Checks that partition name in all commands is valid /// Checks that partition name in all commands is valid
void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const override; void checkAlterPartitionIsPossible(
const PartitionCommands & commands,
const StorageMetadataPtr & metadata_snapshot,
const Settings & settings,
ContextPtr local_context) const override;
/// Change MergeTreeSettings /// Change MergeTreeSettings
void changeSettings( void changeSettings(

View File

@ -1335,6 +1335,10 @@ QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts(
selectColumnNames(column_names_to_return, data, real_column_names, virt_column_names, sample_factor_column_queried); selectColumnNames(column_names_to_return, data, real_column_names, virt_column_names, sample_factor_column_queried);
/// Do not keep data parts in snapshot.
/// They are stored separately, and some could be released after PK analysis.
auto storage_snapshot_copy = storage_snapshot->clone(std::make_unique<MergeTreeData::SnapshotData>());
return std::make_unique<ReadFromMergeTree>( return std::make_unique<ReadFromMergeTree>(
std::move(parts), std::move(parts),
std::move(alter_conversions), std::move(alter_conversions),
@ -1342,7 +1346,7 @@ QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts(
virt_column_names, virt_column_names,
data, data,
query_info, query_info,
storage_snapshot, storage_snapshot_copy,
context, context,
max_block_size, max_block_size,
num_streams, num_streams,

View File

@ -2,7 +2,8 @@
#include <IO/ReadBufferFromString.h> #include <IO/ReadBufferFromString.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include "Core/ProtocolDefines.h" #include <Core/ProtocolDefines.h>
#include <Parsers/ASTLiteral.h>
namespace DB namespace DB
{ {
@ -23,8 +24,15 @@ MergeTreePartInfo MergeTreePartInfo::fromPartName(const String & part_name, Merg
throw Exception(ErrorCodes::BAD_DATA_PART_NAME, "Unexpected part name: {} for format version: {}", part_name, format_version); throw Exception(ErrorCodes::BAD_DATA_PART_NAME, "Unexpected part name: {} for format version: {}", part_name, format_version);
} }
void MergeTreePartInfo::validatePartitionID(const String & partition_id, MergeTreeDataFormatVersion format_version) void MergeTreePartInfo::validatePartitionID(const ASTPtr & partition_id_ast, MergeTreeDataFormatVersion format_version)
{ {
std::string partition_id;
if (auto * literal = partition_id_ast->as<ASTLiteral>(); literal != nullptr && literal->value.getType() == Field::Types::String)
partition_id = literal->value.safeGet<String>();
else
throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Partition id must be string literal");
if (partition_id.empty()) if (partition_id.empty())
throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Partition id is empty"); throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Partition id is empty");

View File

@ -9,6 +9,7 @@
#include <base/DayNum.h> #include <base/DayNum.h>
#include <IO/ReadBuffer.h> #include <IO/ReadBuffer.h>
#include <IO/WriteBuffer.h> #include <IO/WriteBuffer.h>
#include <Parsers/IAST_fwd.h>
#include <Storages/MergeTree/MergeTreeDataFormatVersion.h> #include <Storages/MergeTree/MergeTreeDataFormatVersion.h>
@ -119,7 +120,7 @@ struct MergeTreePartInfo
void deserialize(ReadBuffer & in); void deserialize(ReadBuffer & in);
/// Simple sanity check for partition ID. Checking that it's not too long or too short, doesn't contain a lot of '_'. /// Simple sanity check for partition ID. Checking that it's not too long or too short, doesn't contain a lot of '_'.
static void validatePartitionID(const String & partition_id, MergeTreeDataFormatVersion format_version); static void validatePartitionID(const ASTPtr & partition_id_ast, MergeTreeDataFormatVersion format_version);
static MergeTreePartInfo fromPartName(const String & part_name, MergeTreeDataFormatVersion format_version); static MergeTreePartInfo fromPartName(const String & part_name, MergeTreeDataFormatVersion format_version);

View File

@ -1174,7 +1174,7 @@ void MergeTreeRangeReader::fillPartOffsetColumn(ReadResult & result, UInt64 lead
UInt64 * pos = vec.data(); UInt64 * pos = vec.data();
UInt64 * end = &vec[num_rows]; UInt64 * end = &vec[num_rows];
/// Fill the reamining part of the previous range (it was started in the previous read request). /// Fill the remaining part of the previous range (it was started in the previous read request).
while (pos < end && leading_begin_part_offset < leading_end_part_offset) while (pos < end && leading_begin_part_offset < leading_end_part_offset)
*pos++ = leading_begin_part_offset++; *pos++ = leading_begin_part_offset++;

View File

@ -152,7 +152,7 @@ void MergeTreeReaderCompact::fillColumnPositions()
auto [name_in_storage, subcolumn_name] = Nested::splitName(column_to_read.name); auto [name_in_storage, subcolumn_name] = Nested::splitName(column_to_read.name);
/// If it is a part of Nested, we need to get the column from /// If it is a part of Nested, we need to get the column from
/// storage metatadata which is converted to Nested type with subcolumns. /// storage metadata which is converted to Nested type with subcolumns.
/// It is required for proper counting of shared streams. /// It is required for proper counting of shared streams.
if (!subcolumn_name.empty()) if (!subcolumn_name.empty())
{ {

View File

@ -749,7 +749,7 @@ QueueRepresentation getQueueRepresentation(const std::list<ReplicatedMergeTreeLo
const auto & key = entry->znode_name; const auto & key = entry->znode_name;
switch (entry->type) switch (entry->type)
{ {
/// explicetely specify all types of entries without default, so if /// explicitly specify all types of entries without default, so if
/// someone decide to add new type it will produce a compiler warning (error in our case) /// someone decide to add new type it will produce a compiler warning (error in our case)
case LogEntryType::GET_PART: case LogEntryType::GET_PART:
case LogEntryType::ATTACH_PART: case LogEntryType::ATTACH_PART:

View File

@ -240,7 +240,7 @@ private:
/// by first argument. If remove_part == true, than also remove part itself. /// by first argument. If remove_part == true, than also remove part itself.
/// Both negative flags will throw exception. /// Both negative flags will throw exception.
/// ///
/// Part removed from mutations which satisfy contitions: /// Part removed from mutations which satisfy conditions:
/// block_number > part.getDataVersion() /// block_number > part.getDataVersion()
/// or block_number == part.getDataVersion() /// or block_number == part.getDataVersion()
/// ^ (this may happen if we downloaded mutated part from other replica) /// ^ (this may happen if we downloaded mutated part from other replica)

View File

@ -847,6 +847,8 @@ StorageAzureBlobSource::GlobIterator::GlobIterator(
blobs_with_metadata.emplace_back(blob_path_with_globs, object_metadata); blobs_with_metadata.emplace_back(blob_path_with_globs, object_metadata);
if (outer_blobs) if (outer_blobs)
outer_blobs->emplace_back(blobs_with_metadata.back()); outer_blobs->emplace_back(blobs_with_metadata.back());
if (file_progress_callback)
file_progress_callback(FileProgress(0, object_metadata.size_bytes));
is_finished = true; is_finished = true;
return; return;
} }

View File

@ -8,7 +8,7 @@
namespace DB namespace DB
{ {
class StorageDummy : public IStorage class StorageDummy final : public IStorage
{ {
public: public:
StorageDummy(const StorageID & table_id_, const ColumnsDescription & columns_, ColumnsDescription object_columns_ = {}); StorageDummy(const StorageID & table_id_, const ColumnsDescription & columns_, ColumnsDescription object_columns_ = {});
@ -46,7 +46,7 @@ private:
const ColumnsDescription object_columns; const ColumnsDescription object_columns;
}; };
class ReadFromDummy : public SourceStepWithFilter class ReadFromDummy final : public SourceStepWithFilter
{ {
public: public:
explicit ReadFromDummy(const StorageDummy & storage_, explicit ReadFromDummy(const StorageDummy & storage_,

View File

@ -330,10 +330,11 @@ Pipe StorageMaterializedView::alterPartition(
} }
void StorageMaterializedView::checkAlterPartitionIsPossible( void StorageMaterializedView::checkAlterPartitionIsPossible(
const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot,
const Settings & settings, ContextPtr local_context) const
{ {
checkStatementCanBeForwarded(); checkStatementCanBeForwarded();
getTargetTable()->checkAlterPartitionIsPossible(commands, metadata_snapshot, settings); getTargetTable()->checkAlterPartitionIsPossible(commands, metadata_snapshot, settings, local_context);
} }
void StorageMaterializedView::mutate(const MutationCommands & commands, ContextPtr local_context) void StorageMaterializedView::mutate(const MutationCommands & commands, ContextPtr local_context)

View File

@ -64,7 +64,7 @@ public:
Pipe alterPartition(const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, ContextPtr context) override; Pipe alterPartition(const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, ContextPtr context) override;
void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const override; void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings, ContextPtr local_context) const override;
void mutate(const MutationCommands & commands, ContextPtr context) override; void mutate(const MutationCommands & commands, ContextPtr context) override;

View File

@ -257,13 +257,6 @@ void StorageMergeTree::read(
processed_stage, nullptr, enable_parallel_reading)) processed_stage, nullptr, enable_parallel_reading))
query_plan = std::move(*plan); query_plan = std::move(*plan);
} }
/// Now, copy of parts that is required for the query, stored in the processors,
/// while snapshot_data.parts includes all parts, even one that had been filtered out with partition pruning,
/// reset them to avoid holding them.
auto & snapshot_data = assert_cast<MergeTreeData::SnapshotData &>(*storage_snapshot->data);
snapshot_data.parts = {};
snapshot_data.alter_conversions = {};
} }
std::optional<UInt64> StorageMergeTree::totalRows(const Settings &) const std::optional<UInt64> StorageMergeTree::totalRows(const Settings &) const

View File

@ -115,9 +115,9 @@ public:
return getNested()->alterPartition(metadata_snapshot, commands, context); return getNested()->alterPartition(metadata_snapshot, commands, context);
} }
void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const override void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings, ContextPtr context) const override
{ {
getNested()->checkAlterPartitionIsPossible(commands, metadata_snapshot, settings); getNested()->checkAlterPartitionIsPossible(commands, metadata_snapshot, settings, context);
} }
bool optimize( bool optimize(

View File

@ -5134,15 +5134,6 @@ void StorageReplicatedMergeTree::read(
const size_t max_block_size, const size_t max_block_size,
const size_t num_streams) const size_t num_streams)
{ {
SCOPE_EXIT({
/// Now, copy of parts that is required for the query, stored in the processors,
/// while snapshot_data.parts includes all parts, even one that had been filtered out with partition pruning,
/// reset them to avoid holding them.
auto & snapshot_data = assert_cast<MergeTreeData::SnapshotData &>(*storage_snapshot->data);
snapshot_data.parts = {};
snapshot_data.alter_conversions = {};
});
const auto & settings = local_context->getSettingsRef(); const auto & settings = local_context->getSettingsRef();
/// The `select_sequential_consistency` setting has two meanings: /// The `select_sequential_consistency` setting has two meanings:

View File

@ -207,6 +207,16 @@ private:
{ {
auto answer = *buffer_iter; auto answer = *buffer_iter;
++buffer_iter; ++buffer_iter;
/// If url doesn't contain globs, we didn't list s3 bucket and didn't get object info for the key.
/// So we get object info lazily here on 'next()' request.
if (!answer.info)
{
answer.info = S3::getObjectInfo(*client, globbed_uri.bucket, answer.key, globbed_uri.version_id, request_settings);
if (file_progress_callback)
file_progress_callback(FileProgress(0, answer.info->size));
}
return answer; return answer;
} }

View File

@ -18,6 +18,16 @@ namespace ErrorCodes
extern const int COLUMN_QUERIED_MORE_THAN_ONCE; extern const int COLUMN_QUERIED_MORE_THAN_ONCE;
} }
std::shared_ptr<StorageSnapshot> StorageSnapshot::clone(DataPtr data_) const
{
auto res = std::make_shared<StorageSnapshot>(storage, metadata, object_columns);
res->projection = projection;
res->data = std::move(data_);
return res;
}
void StorageSnapshot::init() void StorageSnapshot::init()
{ {
for (const auto & [name, type] : storage.getVirtuals()) for (const auto & [name, type] : storage.getVirtuals())

View File

@ -60,6 +60,8 @@ struct StorageSnapshot
init(); init();
} }
std::shared_ptr<StorageSnapshot> clone(DataPtr data_) const;
/// Get all available columns with types according to options. /// Get all available columns with types according to options.
NamesAndTypesList getColumns(const GetColumnsOptions & options) const; NamesAndTypesList getColumns(const GetColumnsOptions & options) const;

View File

@ -1,8 +1,20 @@
#include <Storages/System/StorageSystemServerSettings.h> #include <Storages/System/StorageSystemServerSettings.h>
#include <Core/BackgroundSchedulePool.h>
#include <DataTypes/DataTypeString.h> #include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h> #include <DataTypes/DataTypesNumber.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Interpreters/ProcessList.h>
#include <Core/ServerSettings.h> #include <Core/ServerSettings.h>
#include <Storages/MergeTree/MergeTreeBackgroundExecutor.h>
namespace CurrentMetrics
{
extern const Metric BackgroundSchedulePoolSize;
extern const Metric BackgroundBufferFlushSchedulePoolSize;
extern const Metric BackgroundDistributedSchedulePoolSize;
extern const Metric BackgroundMessageBrokerSchedulePoolSize;
}
namespace DB namespace DB
{ {
@ -16,11 +28,35 @@ NamesAndTypesList StorageSystemServerSettings::getNamesAndTypes()
{"description", std::make_shared<DataTypeString>()}, {"description", std::make_shared<DataTypeString>()},
{"type", std::make_shared<DataTypeString>()}, {"type", std::make_shared<DataTypeString>()},
{"is_obsolete", std::make_shared<DataTypeUInt8>()}, {"is_obsolete", std::make_shared<DataTypeUInt8>()},
{"is_hot_reloadable", std::make_shared<DataTypeUInt8>()}
}; };
} }
void StorageSystemServerSettings::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const void StorageSystemServerSettings::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{ {
// Server settings that have been reloaded from the config file.
std::unordered_map<std::string, std::string> updated = {
{"max_server_memory_usage", std::to_string(total_memory_tracker.getHardLimit())},
{"allow_use_jemalloc_memory", std::to_string(total_memory_tracker.getAllowUseJemallocMmemory())},
{"max_table_size_to_drop", std::to_string(context->getMaxTableSizeToDrop())},
{"max_partition_size_to_drop", std::to_string(context->getMaxPartitionSizeToDrop())},
{"max_concurrent_queries", std::to_string(context->getProcessList().getMaxSize())},
{"max_concurrent_insert_queries", std::to_string(context->getProcessList().getMaxInsertQueriesAmount())},
{"max_concurrent_select_queries", std::to_string(context->getProcessList().getMaxSelectQueriesAmount())},
{"background_pool_size", std::to_string(context->getMergeMutateExecutor()->getMaxThreads())},
{"background_move_pool_size", std::to_string(context->getMovesExecutor()->getMaxThreads())},
{"background_fetches_pool_size", std::to_string(context->getFetchesExecutor()->getMaxThreads())},
{"background_common_pool_size", std::to_string(context->getCommonExecutor()->getMaxThreads())},
{"background_buffer_flush_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundBufferFlushSchedulePoolSize))},
{"background_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundSchedulePoolSize))},
{"background_message_broker_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundMessageBrokerSchedulePoolSize))},
{"background_distributed_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundDistributedSchedulePoolSize))}
};
const auto & config = context->getConfigRef(); const auto & config = context->getConfigRef();
ServerSettings settings; ServerSettings settings;
settings.loadSettingsFromConfig(config); settings.loadSettingsFromConfig(config);
@ -28,13 +64,16 @@ void StorageSystemServerSettings::fillData(MutableColumns & res_columns, Context
for (const auto & setting : settings.all()) for (const auto & setting : settings.all())
{ {
const auto & setting_name = setting.getName(); const auto & setting_name = setting.getName();
const auto & it = updated.find(setting_name);
res_columns[0]->insert(setting_name); res_columns[0]->insert(setting_name);
res_columns[1]->insert(setting.getValueString()); res_columns[1]->insert((it != updated.end()) ? it->second : setting.getValueString());
res_columns[2]->insert(setting.getDefaultValueString()); res_columns[2]->insert(setting.getDefaultValueString());
res_columns[3]->insert(setting.isValueChanged()); res_columns[3]->insert(setting.isValueChanged());
res_columns[4]->insert(setting.getDescription()); res_columns[4]->insert(setting.getDescription());
res_columns[5]->insert(setting.getTypeName()); res_columns[5]->insert(setting.getTypeName());
res_columns[6]->insert(setting.isObsolete()); res_columns[6]->insert(setting.isObsolete());
res_columns[7]->insert((it != updated.end()) ? true : false);
} }
} }

View File

@ -1,4 +1,3 @@
#include <algorithm>
#include <memory> #include <memory>
#include <Core/NamesAndTypes.h> #include <Core/NamesAndTypes.h>
#include <Core/TypeId.h> #include <Core/TypeId.h>
@ -82,33 +81,14 @@ bool extractFunctions(const ASTPtr & expression, const std::function<bool(const
} }
else if (function->name == "or") else if (function->name == "or")
{ {
bool ret = false; bool ret = true;
ASTs or_args; ASTs or_args;
for (const auto & child : function->arguments->children) for (const auto & child : function->arguments->children)
ret |= extractFunctions(child, is_constant, or_args); ret &= extractFunctions(child, is_constant, or_args);
/// We can keep condition only if it still OR condition (i.e. we
if (!or_args.empty()) /// have dependent conditions for columns at both sides)
{ if (or_args.size() == 2)
/// In case of there are less number of arguments for which
/// is_constant() == true, we need to add always-true
/// implicitly to avoid breaking AND invariant.
///
/// Consider the following:
///
/// ((value = 10) OR (_table = 'v2')) AND ((_table = 'v1') OR (value = 20))
///
/// Without implicit always-true:
///
/// (_table = 'v2') AND (_table = 'v1')
///
/// With:
///
/// (_table = 'v2' OR 1) AND (_table = 'v1' OR 1) -> (_table = 'v2') OR (_table = 'v1')
///
if (or_args.size() != function->arguments->children.size())
or_args.push_back(std::make_shared<ASTLiteral>(Field(1)));
result.push_back(makeASTForLogicalOr(std::move(or_args))); result.push_back(makeASTForLogicalOr(std::move(or_args)));
}
return ret; return ret;
} }
} }
@ -185,10 +165,8 @@ bool prepareFilterBlockWithQuery(const ASTPtr & query, ContextPtr context, Block
if (!select.where() && !select.prewhere()) if (!select.where() && !select.prewhere())
return unmodified; return unmodified;
// Provide input columns as constant columns to check if an expression is // Provide input columns as constant columns to check if an expression is constant.
// constant and depends on the columns from provided block (the last is std::function<bool(const ASTPtr &)> is_constant = [&block, &context](const ASTPtr & node)
// required to allow skipping some conditions for handling OR).
std::function<bool(const ASTPtr &)> is_constant = [&block, &context](const ASTPtr & expr)
{ {
auto actions = std::make_shared<ActionsDAG>(block.getColumnsWithTypeAndName()); auto actions = std::make_shared<ActionsDAG>(block.getColumnsWithTypeAndName());
PreparedSetsPtr prepared_sets = std::make_shared<PreparedSets>(); PreparedSetsPtr prepared_sets = std::make_shared<PreparedSets>();
@ -200,26 +178,13 @@ bool prepareFilterBlockWithQuery(const ASTPtr & query, ContextPtr context, Block
context, SizeLimits{}, 1, source_columns, std::move(actions), prepared_sets, true, true, true, context, SizeLimits{}, 1, source_columns, std::move(actions), prepared_sets, true, true, true,
{ aggregation_keys, grouping_set_keys, GroupByKind::NONE }); { aggregation_keys, grouping_set_keys, GroupByKind::NONE });
ActionsVisitor(visitor_data).visit(expr); ActionsVisitor(visitor_data).visit(node);
actions = visitor_data.getActions(); actions = visitor_data.getActions();
auto expr_column_name = expr->getColumnName();
const auto * expr_const_node = actions->tryFindInOutputs(expr_column_name);
if (!expr_const_node)
return false;
auto filter_actions = ActionsDAG::buildFilterActionsDAG({expr_const_node}, {}, context);
const auto & nodes = filter_actions->getNodes();
bool has_dependent_columns = std::any_of(nodes.begin(), nodes.end(), [&](const auto & node)
{
return block.has(node.result_name);
});
if (!has_dependent_columns)
return false;
auto expression_actions = std::make_shared<ExpressionActions>(actions); auto expression_actions = std::make_shared<ExpressionActions>(actions);
auto block_with_constants = block; auto block_with_constants = block;
expression_actions->execute(block_with_constants); expression_actions->execute(block_with_constants);
return block_with_constants.has(expr_column_name) && isColumnConst(*block_with_constants.getByName(expr_column_name).column); auto column_name = node->getColumnName();
return block_with_constants.has(column_name) && isColumnConst(*block_with_constants.getByName(column_name).column);
}; };
/// Create an expression that evaluates the expressions in WHERE and PREWHERE, depending only on the existing columns. /// Create an expression that evaluates the expressions in WHERE and PREWHERE, depending only on the existing columns.

View File

@ -143,6 +143,9 @@ class KeeperClient(object):
def find_super_nodes(self, threshold: int, timeout: float = 60.0) -> str: def find_super_nodes(self, threshold: int, timeout: float = 60.0) -> str:
return self.execute_query(f"find_super_nodes {threshold}", timeout) return self.execute_query(f"find_super_nodes {threshold}", timeout)
def get_all_children_number(self, path: str, timeout: float = 60.0) -> str:
return self.execute_query(f"get_all_children_number {path}", timeout)
def delete_stale_backups(self, timeout: float = 60.0) -> str: def delete_stale_backups(self, timeout: float = 60.0) -> str:
return self.execute_query("delete_stale_backups", timeout) return self.execute_query("delete_stale_backups", timeout)

View File

@ -216,3 +216,20 @@ def test_quoted_argument_parsing(client: KeeperClient):
client.execute_query(f"set '{node_path}' \"value4 with some whitespace\" 3") client.execute_query(f"set '{node_path}' \"value4 with some whitespace\" 3")
assert client.get(node_path) == "value4 with some whitespace" assert client.get(node_path) == "value4 with some whitespace"
def test_get_all_children_number(client: KeeperClient):
client.touch("/test_get_all_children_number")
client.touch("/test_get_all_children_number/1")
client.touch("/test_get_all_children_number/1/1")
client.touch("/test_get_all_children_number/1/2")
client.touch("/test_get_all_children_number/1/3")
client.touch("/test_get_all_children_number/1/4")
client.touch("/test_get_all_children_number/1/5")
client.touch("/test_get_all_children_number/2")
client.touch("/test_get_all_children_number/2/1")
client.touch("/test_get_all_children_number/2/2")
client.touch("/test_get_all_children_number/2/3")
client.touch("/test_get_all_children_number/2/4")
assert client.get_all_children_number("/test_get_all_children_number") == "11"

View File

@ -18,29 +18,15 @@
<value>merge_tree_insert_6</value> <value>merge_tree_insert_6</value>
</values> </values>
</substitution> </substitution>
<substitution>
<name>decimal_primary_key_table_name</name>
<values>
<value>merge_tree_insert_7</value>
<value>merge_tree_insert_8</value>
<value>merge_tree_insert_9</value>
</values>
</substitution>
</substitutions> </substitutions>
<create_query>CREATE TABLE merge_tree_insert_1 (value_1 UInt64, value_2 UInt64, value_3 UInt64) ENGINE = MergeTree ORDER BY (value_1)</create_query> <create_query>CREATE TABLE merge_tree_insert_1 (value_1 UInt64, value_2 UInt64, value_3 UInt64) ENGINE = MergeTree ORDER BY (value_1)</create_query>
<create_query>CREATE TABLE merge_tree_insert_2 (value_1 UInt64, value_2 UInt64, value_3 UInt64) ENGINE = MergeTree ORDER BY (value_1, value_2)</create_query> <create_query>CREATE TABLE merge_tree_insert_2 (value_1 UInt64, value_2 UInt64, value_3 UInt64) ENGINE = MergeTree ORDER BY (value_1, value_2)</create_query>
<create_query>CREATE TABLE merge_tree_insert_3 (value_1 UInt64, value_2 UInt64, value_3 UInt64) ENGINE = MergeTree ORDER BY (value_1, value_2, value_3)</create_query> <create_query>CREATE TABLE merge_tree_insert_3 (value_1 UInt64, value_2 UInt64, value_3 UInt64) ENGINE = MergeTree ORDER BY (value_1, value_2, value_3)</create_query>
<create_query>CREATE TABLE merge_tree_insert_4 (value_1 String, value_2 String, value_3 String) ENGINE = MergeTree ORDER BY (value_1)</create_query> <create_query>CREATE TABLE merge_tree_insert_4 (value_1 String, value_2 String, value_3 String) ENGINE = MergeTree ORDER BY (value_1)</create_query>
<create_query>CREATE TABLE merge_tree_insert_5 (value_1 String, value_2 String, value_3 String) ENGINE = MergeTree ORDER BY (value_1, value_2)</create_query> <create_query>CREATE TABLE merge_tree_insert_5 (value_1 String, value_2 String, value_3 String) ENGINE = MergeTree ORDER BY (value_1, value_2)</create_query>
<create_query>CREATE TABLE merge_tree_insert_6 (value_1 String, value_2 String, value_3 String) ENGINE = MergeTree ORDER BY (value_1, value_2, value_3)</create_query> <create_query>CREATE TABLE merge_tree_insert_6 (value_1 String, value_2 String, value_3 String) ENGINE = MergeTree ORDER BY (value_1, value_2, value_3)</create_query>
<create_query>CREATE TABLE merge_tree_insert_7 (value_1 Decimal64(8), value_2 Decimal64(8), value_3 Decimal64(8)) ENGINE = MergeTree ORDER BY (value_1)</create_query>
<create_query>CREATE TABLE merge_tree_insert_8 (value_1 Decimal64(8), value_2 Decimal64(8), value_3 Decimal64(8)) ENGINE = MergeTree ORDER BY (value_1, value_2)</create_query>
<create_query>CREATE TABLE merge_tree_insert_9 (value_1 Decimal64(8), value_2 Decimal64(8), value_3 Decimal64(8)) ENGINE = MergeTree ORDER BY (value_1, value_2, value_3)</create_query>
<query>INSERT INTO {integer_primary_key_table_name} SELECT rand64(0), rand64(1), rand64(2) FROM system.numbers LIMIT 500000</query> <query>INSERT INTO {integer_primary_key_table_name} SELECT rand64(0), rand64(1), rand64(2) FROM system.numbers LIMIT 500000</query>
<query>INSERT INTO {integer_primary_key_table_name} SELECT rand64(0), rand64(1), rand64(2) FROM system.numbers LIMIT 1000000</query> <query>INSERT INTO {integer_primary_key_table_name} SELECT rand64(0), rand64(1), rand64(2) FROM system.numbers LIMIT 1000000</query>
<query>INSERT INTO {integer_primary_key_table_name} SELECT rand64(0), rand64(1), rand64(2) FROM system.numbers LIMIT 1500000</query> <query>INSERT INTO {integer_primary_key_table_name} SELECT rand64(0), rand64(1), rand64(2) FROM system.numbers LIMIT 1500000</query>
@ -49,12 +35,7 @@
<query>INSERT INTO {string_primary_key_table_name} SELECT toString(rand64(0)), toString(rand64(1)), toString(rand64(2)) FROM system.numbers LIMIT 1000000</query> <query>INSERT INTO {string_primary_key_table_name} SELECT toString(rand64(0)), toString(rand64(1)), toString(rand64(2)) FROM system.numbers LIMIT 1000000</query>
<query>INSERT INTO {string_primary_key_table_name} SELECT toString(rand64(0)), toString(rand64(1)), toString(rand64(2)) FROM system.numbers LIMIT 1500000</query> <query>INSERT INTO {string_primary_key_table_name} SELECT toString(rand64(0)), toString(rand64(1)), toString(rand64(2)) FROM system.numbers LIMIT 1500000</query>
<query>INSERT INTO {decimal_primary_key_table_name} SELECT rand64(0) % 1000000, rand64(1) % 1500000, rand64(2) % 2000000 FROM system.numbers LIMIT 500000</query>
<query>INSERT INTO {decimal_primary_key_table_name} SELECT rand64(0) % 1000000, rand64(1) % 1500000, rand64(2) % 2000000 FROM system.numbers LIMIT 1000000</query>
<query>INSERT INTO {decimal_primary_key_table_name} SELECT rand64(0) % 1000000, rand64(1) % 1500000, rand64(2) % 2000000 FROM system.numbers LIMIT 1500000</query>
<drop_query>DROP TABLE IF EXISTS {integer_primary_key_table_name}</drop_query> <drop_query>DROP TABLE IF EXISTS {integer_primary_key_table_name}</drop_query>
<drop_query>DROP TABLE IF EXISTS {string_primary_key_table_name}</drop_query> <drop_query>DROP TABLE IF EXISTS {string_primary_key_table_name}</drop_query>
<drop_query>DROP TABLE IF EXISTS {decimal_primary_key_table_name}</drop_query>
</test> </test>

View File

@ -1,28 +0,0 @@
<test>
<substitutions>
<substitution>
<name>integer_type</name>
<values>
<value>UInt32</value>
<value>UInt64</value>
</values>
</substitution>
<substitution>
<name>sort_expression</name>
<values>
<value>key</value>
<value>key, value</value>
<value>key DESC</value>
<value>key DESC, value DESC</value>
</values>
</substitution>
</substitutions>
<create_query>CREATE TABLE sequential_{integer_type} (key {integer_type}, value {integer_type}) Engine = Memory</create_query>
<fill_query>INSERT INTO sequential_{integer_type} SELECT number, number FROM numbers(500000000)</fill_query>
<query>SELECT key, value FROM sequential_{integer_type} ORDER BY {sort_expression} FORMAT Null</query>
<drop_query>DROP TABLE IF EXISTS sequential_{integer_type}</drop_query>
</test>

View File

@ -0,0 +1,17 @@
-- Constant argument
3 3 3 3
24 24 24 24
201624 201624 201624 201624
-- Non-constant argument
3 3 3 3 3 3
7 7 7 7 7 7
1 1 1 1 1 1
4 4 4 4 4 4
37 37 37 37 37 37
39 39 39 39 39 39
4 4 4 4 4 4
7 7 7 7 7 7
201737 201737 201737 201737 201737 201737
201739 201739 201739 201739 201739 201739
201804 201804 201804 201804 201804 201804
201907 201907 201907 201907 201907 201907

View File

@ -0,0 +1,40 @@
-- Tests that functions `toDayOfWeek()`, 'toWeek()' and 'toYearWeek()' accepts a date given as string (for compatibility with MySQL)
SELECT '-- Constant argument';
SELECT toDayOfWeek(toDateTime('2016-06-15 23:00:00')), toDayOfWeek('2016-06-15'), toDayOfWeek('2016-06-15 23:00:00'), toDayOfWeek('2016-06-15 23:00:00.123456');
SELECT toWeek(toDateTime('2016-06-15 23:00:00')), toWeek('2016-06-15'), toWeek('2016-06-15 23:00:00'), toWeek('2016-06-15 23:00:00.123456');
SELECT toYearWeek(toDateTime('2016-06-15 23:00:00')), toYearWeek('2016-06-15'), toYearWeek('2016-06-15 23:00:00'), toYearWeek('2016-06-15 23:00:00.123456');
SELECT toDayOfWeek('invalid'); -- { serverError CANNOT_PARSE_DATETIME }
SELECT toWeek('invalid'); -- { serverError CANNOT_PARSE_DATETIME }
SELECT toYearWeek('invalid'); -- { serverError CANNOT_PARSE_DATETIME }
SELECT '-- Non-constant argument';
DROP TABLE IF EXISTS tab;
CREATE TABLE tab
(
d Date,
dt DateTime('UTC'),
dt64 DateTime64(6, 'UTC'),
str_d String,
str_dt String,
str_dt64 String,
invalid String
) ENGINE MergeTree ORDER BY dt;
INSERT INTO `tab` VALUES (toDate('2017-09-13'), toDateTime('2017-09-13 19:10:22', 'UTC'), toDateTime64('2017-09-13 19:10:22.123456', 6, 'UTC'), '2017-09-13', '2017-09-13 19:10:22', '2017-09-13 19:10:22.123456', 'foo');
INSERT INTO `tab` VALUES (toDate('2017-09-24'), toDateTime('2017-09-24 12:05:34', 'UTC'), toDateTime64('2017-09-24 12:05:34.123456', 6, 'UTC'), '2017-09-24', '2017-09-24 12:05:34', '2017-09-24 12:05:34.123456', 'bar');
INSERT INTO `tab` VALUES (toDate('2018-01-29'), toDateTime('2018-01-29 02:09:48', 'UTC'), toDateTime64('2018-01-29 02:09:48.123456', 6, 'UTC'), '2018-01-29', '2018-01-29 02:09:48', '2018-01-29 02:09:48.123456', 'qaz');
INSERT INTO `tab` VALUES (toDate('2019-02-21'), toDateTime('2019-02-21 15:07:43', 'UTC'), toDateTime64('2019-02-21 15:07:43.123456', 6, 'UTC'), '2019-02-21', '2019-02-21 15:07:43', '2019-02-21 15:07:43.123456', 'qux');
SELECT toDayOfWeek(d), toDayOfWeek(dt), toDayOfWeek(dt64), toDayOfWeek(str_d), toDayOfWeek(str_dt), toDayOfWeek(str_dt64) FROM tab ORDER BY d;
SELECT toWeek(d), toWeek(dt), toWeek(dt64), toWeek(str_d), toWeek(str_dt), toWeek(str_dt64) FROM tab ORDER BY d;
SELECT toYearWeek(d), toYearWeek(dt), toYearWeek(dt64), toYearWeek(str_d), toYearWeek(str_dt), toYearWeek(str_dt64) FROM tab ORDER BY d;
SELECT toDayOfWeek(invalid) FROM `tab`; -- { serverError CANNOT_PARSE_DATETIME }
SELECT toWeek(invalid) FROM `tab`; -- { serverError CANNOT_PARSE_DATETIME }
SELECT toYearWeek(invalid) FROM `tab`; -- { serverError CANNOT_PARSE_DATETIME }
DROP TABLE tab;

View File

@ -18,11 +18,6 @@ create view v2 as select * from d2;
create table m as v1 engine=Merge(currentDatabase(), '^(v1|v2)$'); create table m as v1 engine=Merge(currentDatabase(), '^(v1|v2)$');
{# -- FIXME:
select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v1') or 0 or 0 settings {{ settings }};
select _table, key from m where (value = 10 and _table = 'v3') or (value = 20 and _table = 'v3') or 0 or 0 settings {{ settings }};
#}
-- avoid reorder -- avoid reorder
set max_threads=1; set max_threads=1;
-- { echoOn } -- { echoOn }

Some files were not shown because too many files have changed in this diff Show More