Merge branch 'master' into expression-dag

This commit is contained in:
Nikolai Kochetov 2020-09-29 12:18:29 +03:00
commit de56c02525
101 changed files with 2453 additions and 323 deletions

View File

@ -173,7 +173,7 @@ endif ()
# Make sure the final executable has symbols exported
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -rdynamic")
find_program (OBJCOPY_PATH NAMES "llvm-objcopy" "llvm-objcopy-10" "llvm-objcopy-9" "llvm-objcopy-8" "objcopy")
find_program (OBJCOPY_PATH NAMES "llvm-objcopy" "llvm-objcopy-11" "llvm-objcopy-10" "llvm-objcopy-9" "llvm-objcopy-8" "objcopy")
if (OBJCOPY_PATH)
message(STATUS "Using objcopy: ${OBJCOPY_PATH}.")
@ -313,7 +313,7 @@ if (COMPILER_CLANG)
endif ()
# Always prefer llvm tools when using clang. For instance, we cannot use GNU ar when llvm LTO is enabled
find_program (LLVM_AR_PATH NAMES "llvm-ar" "llvm-ar-10" "llvm-ar-9" "llvm-ar-8")
find_program (LLVM_AR_PATH NAMES "llvm-ar" "llvm-ar-11" "llvm-ar-10" "llvm-ar-9" "llvm-ar-8")
if (LLVM_AR_PATH)
message(STATUS "Using llvm-ar: ${LLVM_AR_PATH}.")
@ -322,7 +322,7 @@ if (COMPILER_CLANG)
message(WARNING "Cannot find llvm-ar. System ar will be used instead. It does not work with ThinLTO.")
endif ()
find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib" "llvm-ranlib-10" "llvm-ranlib-9" "llvm-ranlib-8")
find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib" "llvm-ranlib-11" "llvm-ranlib-10" "llvm-ranlib-9" "llvm-ranlib-8")
if (LLVM_RANLIB_PATH)
message(STATUS "Using llvm-ranlib: ${LLVM_RANLIB_PATH}.")

View File

@ -38,10 +38,10 @@ bool hasInputData()
}
LineReader::Suggest::WordsRange LineReader::Suggest::getCompletions(const String & prefix, size_t prefix_length) const
std::optional<LineReader::Suggest::WordsRange> LineReader::Suggest::getCompletions(const String & prefix, size_t prefix_length) const
{
if (!ready)
return std::make_pair(words.end(), words.end());
return std::nullopt;
std::string_view last_word;

View File

@ -4,6 +4,7 @@
#include <atomic>
#include <vector>
#include <optional>
class LineReader
{
@ -18,7 +19,7 @@ public:
std::atomic<bool> ready{false};
/// Get iterators for the matched range of words if any.
WordsRange getCompletions(const String & prefix, size_t prefix_length) const;
std::optional<WordsRange> getCompletions(const String & prefix, size_t prefix_length) const;
};
using Patterns = std::vector<const char *>;

View File

@ -30,7 +30,8 @@ static LineReader::Suggest::Words::const_iterator end;
static void findRange(const char * prefix, size_t prefix_length)
{
std::string prefix_str(prefix);
std::tie(pos, end) = suggest->getCompletions(prefix_str, prefix_length);
if (auto completions = suggest->getCompletions(prefix_str, prefix_length))
std::tie(pos, end) = *completions;
}
/// Iterates through matched range.

View File

@ -70,8 +70,9 @@ ReplxxLineReader::ReplxxLineReader(
auto callback = [&suggest] (const String & context, size_t context_size)
{
auto range = suggest.getCompletions(context, context_size);
return Replxx::completions_t(range.first, range.second);
if (auto range = suggest.getCompletions(context, context_size))
return Replxx::completions_t(range->first, range->second);
return Replxx::completions_t();
};
rx.set_completion_callback(callback);

View File

@ -358,7 +358,12 @@ private:
}
else
{
if (number * sizeof(base_type) < sizeof(T))
if constexpr (sizeof(T) <= sizeof(base_type))
{
if (!number)
return x;
}
else if (number * sizeof(base_type) < sizeof(T))
return x >> (number * base_bits); // & std::numeric_limits<base_type>::max()
return 0;
}
@ -366,26 +371,32 @@ private:
template <typename T>
constexpr static integer<Bits, Signed>
op_minus(const integer<Bits, Signed> & lhs, T rhs)
minus(const integer<Bits, Signed> & lhs, T rhs)
{
integer<Bits, Signed> res;
constexpr const unsigned rhs_items = (sizeof(T) > sizeof(base_type)) ? (sizeof(T) / sizeof(base_type)) : 1;
constexpr const unsigned op_items = (item_count < rhs_items) ? item_count : rhs_items;
bool is_underflow = false;
for (unsigned i = 0; i < item_count; ++i)
integer<Bits, Signed> res(lhs);
bool underflows[item_count] = {};
for (unsigned i = 0; i < op_items; ++i)
{
base_type lhs_item = lhs.items[little(i)];
base_type rhs_item = get_item(rhs, i);
base_type & res_item = res.items[little(i)];
if (is_underflow)
underflows[i] = res_item < rhs_item;
res_item -= rhs_item;
}
for (unsigned i = 1; i < item_count; ++i)
{
if (underflows[i-1])
{
is_underflow = (lhs_item == 0);
--lhs_item;
base_type & res_item = res.items[little(i)];
if (res_item == 0)
underflows[i] = true;
--res_item;
}
if (lhs_item < rhs_item)
is_underflow = true;
res.items[little(i)] = lhs_item - rhs_item;
}
return res;
@ -393,39 +404,44 @@ private:
template <typename T>
constexpr static integer<Bits, Signed>
op_plus(const integer<Bits, Signed> & lhs, T rhs)
plus(const integer<Bits, Signed> & lhs, T rhs)
{
integer<Bits, Signed> res;
constexpr const unsigned rhs_items = (sizeof(T) > sizeof(base_type)) ? (sizeof(T) / sizeof(base_type)) : 1;
constexpr const unsigned op_items = (item_count < rhs_items) ? item_count : rhs_items;
bool is_overflow = false;
for (unsigned i = 0; i < item_count; ++i)
integer<Bits, Signed> res(lhs);
bool overflows[item_count] = {};
for (unsigned i = 0; i < op_items; ++i)
{
base_type lhs_item = lhs.items[little(i)];
base_type rhs_item = get_item(rhs, i);
if (is_overflow)
{
++lhs_item;
is_overflow = (lhs_item == 0);
}
base_type & res_item = res.items[little(i)];
res_item = lhs_item + rhs_item;
if (res_item < rhs_item)
is_overflow = true;
res_item += rhs_item;
overflows[i] = res_item < rhs_item;
}
for (unsigned i = 1; i < item_count; ++i)
{
if (overflows[i-1])
{
base_type & res_item = res.items[little(i)];
++res_item;
if (res_item == 0)
overflows[i] = true;
}
}
return res;
}
template <typename T>
constexpr static auto op_multiply(const integer<Bits, Signed> & lhs, const T & rhs)
constexpr static auto multiply(const integer<Bits, Signed> & lhs, const T & rhs)
{
integer<Bits, Signed> res{};
#if 1
integer<Bits, Signed> lhs2 = op_plus(lhs, shift_left(lhs, 1));
integer<Bits, Signed> lhs3 = op_plus(lhs2, shift_left(lhs, 2));
integer<Bits, Signed> lhs2 = plus(lhs, shift_left(lhs, 1));
integer<Bits, Signed> lhs3 = plus(lhs2, shift_left(lhs, 2));
#endif
for (unsigned i = 0; i < item_count; ++i)
{
@ -437,7 +453,7 @@ private:
#if 1 /// optimization
if ((rhs_item & 0x7) == 0x7)
{
res = op_plus(res, shift_left(lhs3, pos));
res = plus(res, shift_left(lhs3, pos));
rhs_item >>= 3;
pos += 3;
continue;
@ -445,14 +461,14 @@ private:
if ((rhs_item & 0x3) == 0x3)
{
res = op_plus(res, shift_left(lhs2, pos));
res = plus(res, shift_left(lhs2, pos));
rhs_item >>= 2;
pos += 2;
continue;
}
#endif
if (rhs_item & 1)
res = op_plus(res, shift_left(lhs, pos));
res = plus(res, shift_left(lhs, pos));
rhs_item >>= 1;
++pos;
@ -475,7 +491,7 @@ public:
constexpr static integer<Bits, Signed>
operator_unary_minus(const integer<Bits, Signed> & lhs) noexcept(std::is_same_v<Signed, unsigned>)
{
return op_plus(operator_unary_tilda(lhs), 1);
return plus(operator_unary_tilda(lhs), 1);
}
template <typename T>
@ -484,9 +500,9 @@ public:
if constexpr (should_keep_size<T>())
{
if (is_negative(rhs))
return op_minus(lhs, -rhs);
return minus(lhs, -rhs);
else
return op_plus(lhs, rhs);
return plus(lhs, rhs);
}
else
{
@ -502,9 +518,9 @@ public:
if constexpr (should_keep_size<T>())
{
if (is_negative(rhs))
return op_plus(lhs, -rhs);
return plus(lhs, -rhs);
else
return op_minus(lhs, rhs);
return minus(lhs, rhs);
}
else
{
@ -523,12 +539,12 @@ public:
if constexpr (std::is_signed_v<Signed>)
{
res = op_multiply((is_negative(lhs) ? make_positive(lhs) : lhs),
res = multiply((is_negative(lhs) ? make_positive(lhs) : lhs),
(is_negative(rhs) ? make_positive(rhs) : rhs));
}
else
{
res = op_multiply(lhs, (is_negative(rhs) ? make_positive(rhs) : rhs));
res = multiply(lhs, (is_negative(rhs) ? make_positive(rhs) : rhs));
}
if (std::is_same_v<Signed, signed> && is_negative(lhs) != is_negative(rhs))
@ -775,20 +791,20 @@ public:
{
if (*c >= '0' && *c <= '9')
{
res = op_multiply(res, 16U);
res = op_plus(res, *c - '0');
res = multiply(res, 16U);
res = plus(res, *c - '0');
++c;
}
else if (*c >= 'a' && *c <= 'f')
{
res = op_multiply(res, 16U);
res = op_plus(res, *c - 'a' + 10U);
res = multiply(res, 16U);
res = plus(res, *c - 'a' + 10U);
++c;
}
else if (*c >= 'A' && *c <= 'F')
{ // tolower must be used, but it is not constexpr
res = op_multiply(res, 16U);
res = op_plus(res, *c - 'A' + 10U);
res = multiply(res, 16U);
res = plus(res, *c - 'A' + 10U);
++c;
}
else
@ -802,8 +818,8 @@ public:
if (*c < '0' || *c > '9')
throwError("invalid char from");
res = op_multiply(res, 10U);
res = op_plus(res, *c - '0');
res = multiply(res, 10U);
res = plus(res, *c - '0');
++c;
}
}

View File

@ -23,6 +23,7 @@ ExtendedLogMessage ExtendedLogMessage::getFrom(const Poco::Message & base)
msg_ext.time_seconds = static_cast<UInt32>(tv.tv_sec);
msg_ext.time_microseconds = static_cast<UInt32>(tv.tv_usec);
msg_ext.time_in_microseconds = static_cast<UInt64>((tv.tv_sec) * 1000000U + (tv.tv_usec));
if (current_thread)
{

View File

@ -23,6 +23,7 @@ public:
uint32_t time_seconds = 0;
uint32_t time_microseconds = 0;
uint64_t time_in_microseconds = 0;
uint64_t thread_id = 0;
std::string query_id;

View File

@ -76,6 +76,7 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg)
TextLogElement elem;
elem.event_time = msg_ext.time_seconds;
elem.event_time_microseconds = msg_ext.time_in_microseconds;
elem.microseconds = msg_ext.time_microseconds;
elem.thread_name = getThreadName();

View File

@ -48,10 +48,15 @@ RUN apt-get update \
tzdata \
--yes --no-install-recommends
# Sanitizer options
# Sanitizer options for services (clickhouse-server)
RUN echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 history_size=7'" >> /etc/environment; \
echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment; \
echo "MSAN_OPTIONS='abort_on_error=1'" >> /etc/environment; \
ln -s /usr/lib/llvm-${LLVM_VERSION}/bin/llvm-symbolizer /usr/bin/llvm-symbolizer;
# Sanitizer options for current shell (not current, but the one that will be spawned on "docker run")
# (but w/o verbosity for TSAN, otherwise test.reference will not match)
ENV TSAN_OPTIONS='halt_on_error=1 history_size=7'
ENV UBSAN_OPTIONS='print_stacktrace=1'
ENV MSAN_OPTIONS='abort_on_error=1'
CMD sleep 1

View File

@ -78,11 +78,16 @@ for e in root.findall('query'):
assert(len(test_queries) == len(is_short))
# If we're given a list of queries to run, check that it makes sense.
for i in args.queries_to_run or []:
if i < 0 or i >= len(test_queries):
print(f'There is no query no. {i} in this test, only [{0}-{len(test_queries) - 1}] are present')
exit(1)
# If we're only asked to print the queries, do that and exit
# If we're only asked to print the queries, do that and exit.
if args.print_queries:
for q in test_queries:
print(q)
for i in args.queries_to_run or range(0, len(test_queries)):
print(test_queries[i])
exit(0)
# Print short queries
@ -196,12 +201,7 @@ if args.max_queries:
queries_to_run = random.sample(range(0, len(test_queries)), min(len(test_queries), args.max_queries))
if args.queries_to_run:
# Run the specified queries, with some sanity check.
for i in args.queries_to_run:
if i < 0 or i >= len(test_queries):
print(f'There is no query no. "{i}" in this test, only [{0}-{len(test_queries) - 1}] are present')
exit(1)
# Run the specified queries.
queries_to_run = args.queries_to_run
# Run test queries.

View File

@ -42,7 +42,10 @@ function start()
# install test configs
/usr/share/clickhouse-test/config/install.sh
# for clickhouse-server (via service)
echo "ASAN_OPTIONS='malloc_context_size=10 verbosity=1 allocator_release_to_os_interval_ms=10000'" >> /etc/environment
# for clickhouse-client
export ASAN_OPTIONS='malloc_context_size=10 verbosity=1 allocator_release_to_os_interval_ms=10000'
start

View File

@ -0,0 +1,10 @@
---
toc_priority: 100
---
# Information support {#information-support}
- Email address: <a class="feedback-email"></a>
- Phone: <a href="tel:+74957806510">+7-495-780-6510</a>
[Original article](https://clickhouse.tech/docs/en/introduction/info/) <!--hide-->

View File

@ -33,6 +33,7 @@ Columns:
- `'ExceptionWhileProcessing' = 4` — Exception during the query execution.
- `event_date` ([Date](../../sql-reference/data-types/date.md)) — Query starting date.
- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Query starting time.
- `event_time_microseconds` ([DateTime](../../sql-reference/data-types/datetime.md)) — Query starting time with microseconds precision.
- `query_start_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Start time of query execution.
- `query_start_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Start time of query execution with microsecond precision.
- `query_duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Duration of query execution in milliseconds.
@ -84,54 +85,57 @@ Columns:
**Example**
``` sql
SELECT * FROM system.query_log LIMIT 1 FORMAT Vertical;
SELECT * FROM system.query_log LIMIT 1 \G
```
``` text
Row 1:
──────
type: QueryStart
event_date: 2020-05-13
event_time: 2020-05-13 14:02:28
query_start_time: 2020-05-13 14:02:28
query_duration_ms: 0
read_rows: 0
read_bytes: 0
written_rows: 0
written_bytes: 0
result_rows: 0
result_bytes: 0
memory_usage: 0
query: SELECT 1
exception_code: 0
exception:
stack_trace:
is_initial_query: 1
user: default
query_id: 5e834082-6f6d-4e34-b47b-cd1934f4002a
address: ::ffff:127.0.0.1
port: 57720
initial_user: default
initial_query_id: 5e834082-6f6d-4e34-b47b-cd1934f4002a
initial_address: ::ffff:127.0.0.1
initial_port: 57720
interface: 1
os_user: bayonet
client_hostname: clickhouse.ru-central1.internal
client_name: ClickHouse client
client_revision: 54434
client_version_major: 20
client_version_minor: 4
client_version_patch: 1
http_method: 0
http_user_agent:
quota_key:
revision: 54434
thread_ids: []
ProfileEvents.Names: []
ProfileEvents.Values: []
Settings.Names: ['use_uncompressed_cache','load_balancing','log_queries','max_memory_usage']
Settings.Values: ['0','random','1','10000000000']
type: QueryStart
event_date: 2020-09-11
event_time: 2020-09-11 10:08:17
event_time_microseconds: 2020-09-11 10:08:17.063321
query_start_time: 2020-09-11 10:08:17
query_start_time_microseconds: 2020-09-11 10:08:17.063321
query_duration_ms: 0
read_rows: 0
read_bytes: 0
written_rows: 0
written_bytes: 0
result_rows: 0
result_bytes: 0
memory_usage: 0
current_database: default
query: INSERT INTO test1 VALUES
exception_code: 0
exception:
stack_trace:
is_initial_query: 1
user: default
query_id: 50a320fd-85a8-49b8-8761-98a86bcbacef
address: ::ffff:127.0.0.1
port: 33452
initial_user: default
initial_query_id: 50a320fd-85a8-49b8-8761-98a86bcbacef
initial_address: ::ffff:127.0.0.1
initial_port: 33452
interface: 1
os_user: bharatnc
client_hostname: tower
client_name: ClickHouse
client_revision: 54437
client_version_major: 20
client_version_minor: 7
client_version_patch: 2
http_method: 0
http_user_agent:
quota_key:
revision: 54440
thread_ids: []
ProfileEvents.Names: []
ProfileEvents.Values: []
Settings.Names: ['use_uncompressed_cache','load_balancing','log_queries','max_memory_usage','allow_introspection_functions']
Settings.Values: ['0','random','1','10000000000','1']
```
**See Also**

View File

@ -15,6 +15,7 @@ Columns:
- `event_date` ([Date](../../sql-reference/data-types/date.md)) — The date when the thread has finished execution of the query.
- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — The date and time when the thread has finished execution of the query.
- `event_time_microsecinds` ([DateTime](../../sql-reference/data-types/datetime.md)) — The date and time when the thread has finished execution of the query with microseconds precision.
- `query_start_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Start time of query execution.
- `query_start_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Start time of query execution with microsecond precision.
- `query_duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Duration of query execution.
@ -63,50 +64,51 @@ Columns:
**Example**
``` sql
SELECT * FROM system.query_thread_log LIMIT 1 FORMAT Vertical
SELECT * FROM system.query_thread_log LIMIT 1 \G
```
``` text
Row 1:
──────
event_date: 2020-05-13
event_time: 2020-05-13 14:02:28
query_start_time: 2020-05-13 14:02:28
query_duration_ms: 0
read_rows: 1
read_bytes: 1
written_rows: 0
written_bytes: 0
memory_usage: 0
peak_memory_usage: 0
thread_name: QueryPipelineEx
thread_id: 28952
master_thread_id: 28924
query: SELECT 1
is_initial_query: 1
user: default
query_id: 5e834082-6f6d-4e34-b47b-cd1934f4002a
address: ::ffff:127.0.0.1
port: 57720
initial_user: default
initial_query_id: 5e834082-6f6d-4e34-b47b-cd1934f4002a
initial_address: ::ffff:127.0.0.1
initial_port: 57720
interface: 1
os_user: bayonet
client_hostname: clickhouse.ru-central1.internal
client_name: ClickHouse client
client_revision: 54434
client_version_major: 20
client_version_minor: 4
client_version_patch: 1
http_method: 0
http_user_agent:
quota_key:
revision: 54434
ProfileEvents.Names: ['ContextLock','RealTimeMicroseconds','UserTimeMicroseconds','OSCPUWaitMicroseconds','OSCPUVirtualTimeMicroseconds']
ProfileEvents.Values: [1,97,81,5,81]
...
event_date: 2020-09-11
event_time: 2020-09-11 10:08:17
event_time_microseconds: 2020-09-11 10:08:17.134042
query_start_time: 2020-09-11 10:08:17
query_start_time_microseconds: 2020-09-11 10:08:17.063150
query_duration_ms: 70
read_rows: 0
read_bytes: 0
written_rows: 1
written_bytes: 12
memory_usage: 4300844
peak_memory_usage: 4300844
thread_name: TCPHandler
thread_id: 638133
master_thread_id: 638133
query: INSERT INTO test1 VALUES
is_initial_query: 1
user: default
query_id: 50a320fd-85a8-49b8-8761-98a86bcbacef
address: ::ffff:127.0.0.1
port: 33452
initial_user: default
initial_query_id: 50a320fd-85a8-49b8-8761-98a86bcbacef
initial_address: ::ffff:127.0.0.1
initial_port: 33452
interface: 1
os_user: bharatnc
client_hostname: tower
client_name: ClickHouse
client_revision: 54437
client_version_major: 20
client_version_minor: 7
client_version_patch: 2
http_method: 0
http_user_agent:
quota_key:
revision: 54440
ProfileEvents.Names: ['Query','InsertQuery','FileOpen','WriteBufferFromFileDescriptorWrite','WriteBufferFromFileDescriptorWriteBytes','ReadCompressedBytes','CompressedReadBufferBlocks','CompressedReadBufferBytes','IOBufferAllocs','IOBufferAllocBytes','FunctionExecute','CreatedWriteBufferOrdinary','DiskWriteElapsedMicroseconds','NetworkReceiveElapsedMicroseconds','NetworkSendElapsedMicroseconds','InsertedRows','InsertedBytes','SelectedRows','SelectedBytes','MergeTreeDataWriterRows','MergeTreeDataWriterUncompressedBytes','MergeTreeDataWriterCompressedBytes','MergeTreeDataWriterBlocks','MergeTreeDataWriterBlocksAlreadySorted','ContextLock','RWLockAcquiredReadLocks','RealTimeMicroseconds','UserTimeMicroseconds','SoftPageFaults','OSCPUVirtualTimeMicroseconds','OSWriteBytes','OSReadChars','OSWriteChars']
ProfileEvents.Values: [1,1,11,11,591,148,3,71,29,6533808,1,11,72,18,47,1,12,1,12,1,12,189,1,1,10,2,70853,2748,49,2747,45056,422,1520]
```
**See Also**

View File

@ -6,6 +6,7 @@ Columns:
- `event_date` (Date) — Date of the entry.
- `event_time` (DateTime) — Time of the entry.
- `event_time_microseconds` (DateTime) — Time of the entry with microseconds precision.
- `microseconds` (UInt32) — Microseconds of the entry.
- `thread_name` (String) — Name of the thread from which the logging was done.
- `thread_id` (UInt64) — OS thread ID.
@ -25,4 +26,28 @@ Columns:
- `source_file` (LowCardinality(String)) — Source file from which the logging was done.
- `source_line` (UInt64) — Source line from which the logging was done.
**Example**
``` sql
SELECT * FROM system.text_log LIMIT 1 \G
```
``` text
Row 1:
──────
event_date: 2020-09-10
event_time: 2020-09-10 11:23:07
event_time_microseconds: 2020-09-10 11:23:07.871397
microseconds: 871397
thread_name: clickhouse-serv
thread_id: 564917
level: Information
query_id:
logger_name: DNSCacheUpdater
message: Update period 15 seconds
revision: 54440
source_file: /ClickHouse/src/Interpreters/DNSCacheUpdater.cpp; void DB::DNSCacheUpdater::start()
source_line: 45
```
[Original article](https://clickhouse.tech/docs/en/operations/system_tables/text_log) <!--hide-->

View File

@ -12,6 +12,8 @@ Columns:
- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Timestamp of the sampling moment.
- `event_time_microseconds` ([DateTime](../../sql-reference/data-types/datetime.md)) — Timestamp of the sampling moment with microseconds precision.
- `timestamp_ns` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Timestamp of the sampling moment in nanoseconds.
- `revision` ([UInt32](../../sql-reference/data-types/int-uint.md)) — ClickHouse server build revision.
@ -38,13 +40,16 @@ SELECT * FROM system.trace_log LIMIT 1 \G
``` text
Row 1:
──────
event_date: 2019-11-15
event_time: 2019-11-15 15:09:38
revision: 54428
timer_type: Real
thread_number: 48
query_id: acc4d61f-5bd1-4a3e-bc91-2180be37c915
trace: [94222141367858,94222152240175,94222152325351,94222152329944,94222152330796,94222151449980,94222144088167,94222151682763,94222144088167,94222151682763,94222144088167,94222144058283,94222144059248,94222091840750,94222091842302,94222091831228,94222189631488,140509950166747,140509942945935]
event_date: 2020-09-10
event_time: 2020-09-10 11:23:09
event_time_microseconds: 2020-09-10 11:23:09.872924
timestamp_ns: 1599762189872924510
revision: 54440
trace_type: Memory
thread_id: 564963
query_id:
trace: [371912858,371912789,371798468,371799717,371801313,371790250,624462773,566365041,566440261,566445834,566460071,566459914,566459842,566459580,566459469,566459389,566459341,566455774,371993941,371988245,372158848,372187428,372187309,372187093,372185478,140222123165193,140222122205443]
size: 5244400
```
[Original article](https://clickhouse.tech/docs/en/operations/system_tables/trace_log) <!--hide-->

View File

@ -7,6 +7,6 @@ toc_priority: 100
Информационная поддержка ClickHouse осуществляется на всей территории Российской Федерации без ограничений посредством использования телефонной связи и средств электронной почты на русском языке в круглосуточном режиме:
- Адрес электронной почты: <a class="feedback-email"></a>
- Телефон: <a href="tel:88002509639">8-800-250-96-39</a> (звонки бесплатны из всех регионов России)
- Телефон: <a href="tel:+74957806510">+7-495-780-6510</a>
[Оригинальная статья](https://clickhouse.tech/docs/ru/introduction/info/) <!--hide-->

View File

@ -1,6 +1,6 @@
# Секция INTO OUTFILE {#into-outfile-clause}
Чтобы перенаправить вывод `SELECT` запроса в указанный файл на стороне клиента, добавьте к нему секцию `INTO OUTFILE filename` (где filenam — строковый литерал).
Чтобы перенаправить вывод `SELECT` запроса в указанный файл на стороне клиента, добавьте к нему секцию `INTO OUTFILE filename` (где filename — строковый литерал).
## Детали реализации {#implementation-details}

View File

@ -49,13 +49,14 @@ def translate_impl(text, target_language=None):
def translate(text, target_language=None):
result = []
for part in re.split(curly_braces_re, text):
if part.startswith('{') and part.endswith('}'):
result.append(part)
else:
result.append(translate_impl(part, target_language=target_language))
return ''.join(result)
return "".join(
[
part
if part.startswith("{") and part.endswith("}")
else translate_impl(part, target_language=target_language)
for part in re.split(curly_braces_re, text)
]
)
def translate_toc(root, lang):

View File

@ -80,7 +80,7 @@ Suggest::Suggest()
"WITH", "TOTALS", "HAVING", "ORDER", "COLLATE", "LIMIT", "UNION", "AND", "OR", "ASC",
"IN", "KILL", "QUERY", "SYNC", "ASYNC", "TEST", "BETWEEN", "TRUNCATE", "USER", "ROLE",
"PROFILE", "QUOTA", "POLICY", "ROW", "GRANT", "REVOKE", "OPTION", "ADMIN", "EXCEPT", "REPLACE",
"IDENTIFIED", "HOST", "NAME", "READONLY", "WRITABLE", "PERMISSIVE", "FOR", "RESTRICTIVE", "FOR", "RANDOMIZED",
"IDENTIFIED", "HOST", "NAME", "READONLY", "WRITABLE", "PERMISSIVE", "FOR", "RESTRICTIVE", "RANDOMIZED",
"INTERVAL", "LIMITS", "ONLY", "TRACKING", "IP", "REGEXP", "ILIKE"};
}

View File

@ -5,6 +5,9 @@ set (CLICKHOUSE_FORMAT_LINK
boost::program_options
clickhouse_common_io
clickhouse_parsers
clickhouse_functions
clickhouse_aggregate_functions
clickhouse_table_functions
dbms
)

View File

@ -1,13 +1,29 @@
#include <iostream>
#include <string_view>
#include <functional>
#include <boost/program_options.hpp>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <Parsers/ParserQuery.h>
#include <Parsers/parseQuery.h>
#include <Parsers/formatAST.h>
#include <Parsers/obfuscateQueries.h>
#include <Common/TerminalSize.h>
#include <Interpreters/Context.h>
#include <Functions/FunctionFactory.h>
#include <Functions/registerFunctions.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/registerAggregateFunctions.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <TableFunctions/registerTableFunctions.h>
#include <Storages/StorageFactory.h>
#include <Storages/registerStorages.h>
#include <DataTypes/DataTypeFactory.h>
#pragma GCC diagnostic ignored "-Wunused-function"
#pragma GCC diagnostic ignored "-Wmissing-declarations"
@ -22,6 +38,8 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
("oneline", "format in single line")
("quiet,q", "just check syntax, no output on success")
("multiquery,n", "allow multiple queries in the same file")
("obfuscate", "obfuscate instead of formatting")
("seed", po::value<std::string>(), "seed (arbitrary string) that determines the result of obfuscation")
;
boost::program_options::variables_map options;
@ -40,10 +58,17 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
bool oneline = options.count("oneline");
bool quiet = options.count("quiet");
bool multiple = options.count("multiquery");
bool obfuscate = options.count("obfuscate");
if (quiet && (hilite || oneline))
if (quiet && (hilite || oneline || obfuscate))
{
std::cerr << "Options 'hilite' or 'oneline' have no sense in 'quiet' mode." << std::endl;
std::cerr << "Options 'hilite' or 'oneline' or 'obfuscate' have no sense in 'quiet' mode." << std::endl;
return 2;
}
if (obfuscate && (hilite || oneline || quiet))
{
std::cerr << "Options 'hilite' or 'oneline' or 'quiet' have no sense in 'obfuscate' mode." << std::endl;
return 2;
}
@ -51,21 +76,66 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
ReadBufferFromFileDescriptor in(STDIN_FILENO);
readStringUntilEOF(query, in);
const char * pos = query.data();
const char * end = pos + query.size();
ParserQuery parser(end);
do
if (obfuscate)
{
ASTPtr res = parseQueryAndMovePosition(parser, pos, end, "query", multiple, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
if (!quiet)
WordMap obfuscated_words_map;
WordSet used_nouns;
SipHash hash_func;
if (options.count("seed"))
{
formatAST(*res, std::cout, hilite, oneline);
if (multiple)
std::cout << "\n;\n";
std::cout << std::endl;
std::string seed;
hash_func.update(options["seed"].as<std::string>());
}
} while (multiple && pos != end);
SharedContextHolder shared_context = Context::createShared();
Context context = Context::createGlobal(shared_context.get());
context.makeGlobalContext();
registerFunctions();
registerAggregateFunctions();
registerTableFunctions();
registerStorages();
std::unordered_set<std::string> additional_names;
auto all_known_storage_names = StorageFactory::instance().getAllRegisteredNames();
auto all_known_data_type_names = DataTypeFactory::instance().getAllRegisteredNames();
additional_names.insert(all_known_storage_names.begin(), all_known_storage_names.end());
additional_names.insert(all_known_data_type_names.begin(), all_known_data_type_names.end());
KnownIdentifierFunc is_known_identifier = [&](std::string_view name)
{
std::string what(name);
return FunctionFactory::instance().tryGet(what, context) != nullptr
|| AggregateFunctionFactory::instance().isAggregateFunctionName(what)
|| TableFunctionFactory::instance().isTableFunctionName(what)
|| additional_names.count(what);
};
WriteBufferFromFileDescriptor out(STDOUT_FILENO);
obfuscateQueries(query, out, obfuscated_words_map, used_nouns, hash_func, is_known_identifier);
}
else
{
const char * pos = query.data();
const char * end = pos + query.size();
ParserQuery parser(end);
do
{
ASTPtr res = parseQueryAndMovePosition(parser, pos, end, "query", multiple, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
if (!quiet)
{
formatAST(*res, std::cout, hilite, oneline);
if (multiple)
std::cout << "\n;\n";
std::cout << std::endl;
}
} while (multiple && pos != end);
}
}
catch (...)
{

View File

@ -552,7 +552,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
#if defined(__linux__)
fmt::print("Setting capabilities for clickhouse binary. This is optional.\n");
std::string command = fmt::format("command setcap && setcap 'cap_net_admin,cap_ipc_lock,cap_sys_nice+ep' {}", main_bin_path.string());
std::string command = fmt::format("command -v setcap && setcap 'cap_net_admin,cap_ipc_lock,cap_sys_nice+ep' {}", main_bin_path.string());
fmt::print(" {}\n", command);
executeScript(command);
#endif

View File

@ -210,7 +210,7 @@ try
/// Maybe useless
if (config().has("macros"))
context->setMacros(std::make_unique<Macros>(config(), "macros"));
context->setMacros(std::make_unique<Macros>(config(), "macros", log));
/// Skip networking

View File

@ -534,7 +534,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
}
if (config().has("macros"))
global_context->setMacros(std::make_unique<Macros>(config(), "macros"));
global_context->setMacros(std::make_unique<Macros>(config(), "macros", log));
/// Initialize main config reloader.
std::string include_from_path = config().getString("include_from", "/etc/metrika.xml");
@ -559,7 +559,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
//setTextLog(global_context->getTextLog());
//buildLoggers(*config, logger());
global_context->setClustersConfig(config);
global_context->setMacros(std::make_unique<Macros>(*config, "macros"));
global_context->setMacros(std::make_unique<Macros>(*config, "macros", log));
global_context->setExternalAuthenticatorsConfig(*config);
/// Setup protection to avoid accidental DROP for big tables (that are greater than 50 GB by default)
@ -671,6 +671,10 @@ int Server::main(const std::vector<std::string> & /*args*/)
total_memory_tracker.setDescription("(total)");
total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking);
/// Set current database name before loading tables and databases because
/// system logs may copy global context.
global_context->setCurrentDatabaseNameInGlobalContext(default_database);
LOG_INFO(log, "Loading metadata from {}", path);
try
@ -678,11 +682,14 @@ int Server::main(const std::vector<std::string> & /*args*/)
loadMetadataSystem(*global_context);
/// After attaching system databases we can initialize system log.
global_context->initializeSystemLogs();
auto & database_catalog = DatabaseCatalog::instance();
/// After the system database is created, attach virtual system tables (in addition to query_log and part_log)
attachSystemTablesServer(*DatabaseCatalog::instance().getSystemDatabase(), has_zookeeper);
attachSystemTablesServer(*database_catalog.getSystemDatabase(), has_zookeeper);
/// Then, load remaining databases
loadMetadata(*global_context, default_database);
DatabaseCatalog::instance().loadDatabases();
database_catalog.loadDatabases();
/// After loading validate that default database exists
database_catalog.assertDatabaseExists(default_database);
}
catch (...)
{
@ -745,8 +752,6 @@ int Server::main(const std::vector<std::string> & /*args*/)
LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they require PHDR cache to be created"
" (otherwise the function 'dl_iterate_phdr' is not lock free and not async-signal safe).");
global_context->setCurrentDatabase(default_database);
if (has_zookeeper && config().has("distributed_ddl"))
{
/// DDL worker should be started after all tables were loaded

View File

@ -1,22 +1,12 @@
#pragma once
#include <cstddef>
#include <cstdint>
#include <cassert>
#include <type_traits>
#include <common/defines.h>
/** Returns log2 of number, rounded down.
* Compiles to single 'bsr' instruction on x86.
* For zero argument, result is unspecified.
*/
inline unsigned int bitScanReverse(unsigned int x)
{
assert(x != 0);
return sizeof(unsigned int) * 8 - 1 - __builtin_clz(x);
}
/** For zero argument, result is zero.
* For arguments with most significand bit set, result is n.
* For other arguments, returns value, rounded up to power of two.
@ -41,10 +31,9 @@ inline size_t roundUpToPowerOfTwoOrZero(size_t n)
template <typename T>
inline size_t getLeadingZeroBits(T x)
inline size_t getLeadingZeroBitsUnsafe(T x)
{
if (!x)
return sizeof(x) * 8;
assert(x != 0);
if constexpr (sizeof(T) <= sizeof(unsigned int))
{
@ -60,10 +49,32 @@ inline size_t getLeadingZeroBits(T x)
}
}
template <typename T>
inline size_t getLeadingZeroBits(T x)
{
if (!x)
return sizeof(x) * 8;
return getLeadingZeroBitsUnsafe(x);
}
/** Returns log2 of number, rounded down.
* Compiles to single 'bsr' instruction on x86.
* For zero argument, result is unspecified.
*/
template <typename T>
inline uint32_t bitScanReverse(T x)
{
return (std::max<size_t>(sizeof(T), sizeof(unsigned int))) * 8 - 1 - getLeadingZeroBitsUnsafe(x);
}
// Unsafe since __builtin_ctz()-family explicitly state that result is undefined on x == 0
template <typename T>
inline size_t getTrailingZeroBitsUnsafe(T x)
{
assert(x != 0);
if constexpr (sizeof(T) <= sizeof(unsigned int))
{
return __builtin_ctz(x);
@ -88,8 +99,8 @@ inline size_t getTrailingZeroBits(T x)
}
/** Returns a mask that has '1' for `bits` LSB set:
* maskLowBits<UInt8>(3) => 00000111
*/
* maskLowBits<UInt8>(3) => 00000111
*/
template <typename T>
inline T maskLowBits(unsigned char bits)
{

View File

@ -2,6 +2,7 @@
#include <Common/Macros.h>
#include <Common/Exception.h>
#include <IO/WriteHelpers.h>
#include <common/logger_useful.h>
namespace DB
@ -12,19 +13,32 @@ namespace ErrorCodes
extern const int SYNTAX_ERROR;
}
Macros::Macros(const Poco::Util::AbstractConfiguration & config, const String & root_key)
Macros::Macros(const Poco::Util::AbstractConfiguration & config, const String & root_key, Poco::Logger * log)
{
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(root_key, keys);
for (const String & key : keys)
{
macros[key] = config.getString(root_key + "." + key);
if (key == "database" || key == "table" || key == "uuid")
{
if (log)
LOG_WARNING(log,
"Config file contains '{}' macro. This macro has special meaning "
"and it's explicit definition is not recommended. Implicit unfolding for "
"'database', 'table' and 'uuid' macros will be disabled.",
key);
enable_special_macros = false;
}
}
}
String Macros::expand(const String & s,
MacroExpansionInfo & info) const
{
/// Do not allow recursion if we expand only special macros, because it will be infinite recursion
assert(info.level == 0 || !info.expand_special_macros_only);
if (s.find('{') == String::npos)
return s;
@ -34,6 +48,10 @@ String Macros::expand(const String & s,
if (info.level >= 10)
throw Exception("Too deep recursion while expanding macros: '" + s + "'", ErrorCodes::SYNTAX_ERROR);
/// If config file contains explicit special macro, then we do not expand it in this mode.
if (!enable_special_macros && info.expand_special_macros_only)
return s;
String res;
size_t pos = 0;
while (true)
@ -59,15 +77,21 @@ String Macros::expand(const String & s,
auto it = macros.find(macro_name);
/// Prefer explicit macros over implicit.
if (it != macros.end())
if (it != macros.end() && !info.expand_special_macros_only)
res += it->second;
else if (macro_name == "database" && !info.database_name.empty())
res += info.database_name;
else if (macro_name == "table" && !info.table_name.empty())
res += info.table_name;
else if (macro_name == "database" && !info.table_id.database_name.empty())
{
res += info.table_id.database_name;
info.expanded_database = true;
}
else if (macro_name == "table" && !info.table_id.table_name.empty())
{
res += info.table_id.table_name;
info.expanded_table = true;
}
else if (macro_name == "uuid")
{
if (info.uuid == UUIDHelpers::Nil)
if (info.table_id.uuid == UUIDHelpers::Nil)
throw Exception("Macro 'uuid' and empty arguments of ReplicatedMergeTree "
"are supported only for ON CLUSTER queries with Atomic database engine",
ErrorCodes::SYNTAX_ERROR);
@ -76,12 +100,16 @@ String Macros::expand(const String & s,
/// It becomes impossible to check if {uuid} is contained inside some unknown macro.
if (info.level)
throw Exception("Macro 'uuid' should not be inside another macro", ErrorCodes::SYNTAX_ERROR);
res += toString(info.uuid);
res += toString(info.table_id.uuid);
info.expanded_uuid = true;
}
else if (info.ignore_unknown)
else if (info.ignore_unknown || info.expand_special_macros_only)
{
if (info.expand_special_macros_only)
res += '{';
res += macro_name;
if (info.expand_special_macros_only)
res += '}';
info.has_unknown = true;
}
else
@ -93,6 +121,9 @@ String Macros::expand(const String & s,
}
++info.level;
if (info.expand_special_macros_only)
return res;
return expand(res, info);
}
@ -113,9 +144,9 @@ String Macros::expand(const String & s) const
String Macros::expand(const String & s, const StorageID & table_id, bool allow_uuid) const
{
MacroExpansionInfo info;
info.database_name = table_id.database_name;
info.table_name = table_id.table_name;
info.uuid = allow_uuid ? table_id.uuid : UUIDHelpers::Nil;
info.table_id = table_id;
if (!allow_uuid)
info.table_id.uuid = UUIDHelpers::Nil;
return expand(s, info);
}

View File

@ -13,6 +13,7 @@ namespace Poco
{
class AbstractConfiguration;
}
class Logger;
}
@ -25,18 +26,19 @@ class Macros
{
public:
Macros() = default;
Macros(const Poco::Util::AbstractConfiguration & config, const String & key);
Macros(const Poco::Util::AbstractConfiguration & config, const String & key, Poco::Logger * log = nullptr);
struct MacroExpansionInfo
{
/// Settings
String database_name;
String table_name;
UUID uuid = UUIDHelpers::Nil;
StorageID table_id = StorageID::createEmpty();
bool ignore_unknown = false;
bool expand_special_macros_only = false;
/// Information about macro expansion
size_t level = 0;
bool expanded_database = false;
bool expanded_table = false;
bool expanded_uuid = false;
bool has_unknown = false;
};
@ -64,6 +66,7 @@ public:
private:
MacroMap macros;
bool enable_special_macros = true;
};

View File

@ -13,7 +13,6 @@ inline UInt64 clock_gettime_ns(clockid_t clock_type = CLOCK_MONOTONIC)
return UInt64(ts.tv_sec * 1000000000LL + ts.tv_nsec);
}
/** Differs from Poco::Stopwatch only by using 'clock_gettime' instead of 'gettimeofday',
* returns nanoseconds instead of microseconds, and also by other minor differencies.
*/

View File

@ -67,10 +67,19 @@ inline bool isASCII(char c)
return static_cast<unsigned char>(c) < 0x80;
}
inline bool isLowerAlphaASCII(char c)
{
return (c >= 'a' && c <= 'z');
}
inline bool isUpperAlphaASCII(char c)
{
return (c >= 'A' && c <= 'Z');
}
inline bool isAlphaASCII(char c)
{
return (c >= 'a' && c <= 'z')
|| (c >= 'A' && c <= 'Z');
return isLowerAlphaASCII(c) || isUpperAlphaASCII(c);
}
inline bool isNumericASCII(char c)
@ -122,6 +131,16 @@ inline bool isPrintableASCII(char c)
return uc >= 32 && uc <= 126; /// 127 is ASCII DEL.
}
inline bool isPunctuationASCII(char c)
{
uint8_t uc = c;
return (uc >= 33 && uc <= 47)
|| (uc >= 58 && uc <= 64)
|| (uc >= 91 && uc <= 96)
|| (uc >= 123 && uc <= 125);
}
inline bool isValidIdentifier(const std::string_view & str)
{
return !str.empty() && isValidIdentifierBegin(str[0]) && std::all_of(str.begin() + 1, str.end(), isWordCharASCII);

View File

@ -141,8 +141,14 @@ void TraceCollector::run()
if (trace_log)
{
UInt64 time = clock_gettime_ns(CLOCK_REALTIME);
TraceLogElement element{time_t(time / 1000000000), time, trace_type, thread_id, query_id, trace, size};
// time and time_in_microseconds are both being constructed from the same timespec so that the
// times will be equal upto the precision of a second.
struct timespec ts;
clock_gettime(CLOCK_REALTIME, &ts);
UInt64 time = UInt64(ts.tv_sec * 1000000000LL + ts.tv_nsec);
UInt64 time_in_microseconds = UInt64((ts.tv_sec * 1000000LL) + (ts.tv_nsec / 1000));
TraceLogElement element{time_t(time / 1000000000), time_in_microseconds, time, trace_type, thread_id, query_id, trace, size};
trace_log->add(element);
}
}

View File

@ -774,7 +774,7 @@ void DataTypeLowCardinality::deserializeTextQuoted(IColumn & column, ReadBuffer
void DataTypeLowCardinality::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
deserializeImpl(column, &IDataType::deserializeAsTextEscaped, istr, settings);
deserializeImpl(column, &IDataType::deserializeAsWholeText, istr, settings);
}
void DataTypeLowCardinality::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const

View File

@ -207,11 +207,13 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot move dictionary to other database");
StoragePtr table = getTableUnlocked(table_name, db_lock);
table->checkTableCanBeRenamed();
assert_can_move_mat_view(table);
StoragePtr other_table;
if (exchange)
{
other_table = other_db.getTableUnlocked(to_table_name, other_db_lock);
other_table->checkTableCanBeRenamed();
assert_can_move_mat_view(other_table);
}

View File

@ -331,6 +331,7 @@ void registerFileSegmentationEngineCSV(FormatFactory & factory);
void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory);
void registerFileSegmentationEngineRegexp(FormatFactory & factory);
void registerFileSegmentationEngineJSONAsString(FormatFactory & factory);
void registerFileSegmentationEngineLineAsString(FormatFactory & factory);
/// Formats for both input/output.
@ -400,6 +401,7 @@ FormatFactory::FormatFactory()
registerFileSegmentationEngineJSONEachRow(*this);
registerFileSegmentationEngineRegexp(*this);
registerFileSegmentationEngineJSONAsString(*this);
registerFileSegmentationEngineLineAsString(*this);
registerInputFormatNative(*this);
registerOutputFormatNative(*this);

View File

@ -23,13 +23,27 @@ namespace ErrorCodes
extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH;
}
MySQLBlockInputStream::Connection::Connection(
const mysqlxx::PoolWithFailover::Entry & entry_,
const std::string & query_str)
: entry(entry_)
, query{entry->query(query_str)}
, result{query.use()}
{
}
MySQLBlockInputStream::MySQLBlockInputStream(
const mysqlxx::PoolWithFailover::Entry & entry_, const std::string & query_str, const Block & sample_block, const UInt64 max_block_size_, const bool auto_close_)
: entry{entry_}, query{this->entry->query(query_str)}, result{query.use()}, max_block_size{max_block_size_}, auto_close{auto_close_}
const mysqlxx::PoolWithFailover::Entry & entry,
const std::string & query_str,
const Block & sample_block,
const UInt64 max_block_size_,
const bool auto_close_)
: connection{std::make_unique<Connection>(entry, query_str)}
, max_block_size{max_block_size_}
, auto_close{auto_close_}
{
if (sample_block.columns() != result.getNumFields())
throw Exception{"mysqlxx::UseQueryResult contains " + toString(result.getNumFields()) + " columns while "
if (sample_block.columns() != connection->result.getNumFields())
throw Exception{"mysqlxx::UseQueryResult contains " + toString(connection->result.getNumFields()) + " columns while "
+ toString(sample_block.columns()) + " expected",
ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH};
@ -106,11 +120,11 @@ namespace
Block MySQLBlockInputStream::readImpl()
{
auto row = result.fetch();
auto row = connection->result.fetch();
if (!row)
{
if (auto_close)
entry.disconnect();
connection->entry.disconnect();
return {};
}
@ -145,11 +159,42 @@ Block MySQLBlockInputStream::readImpl()
if (num_rows == max_block_size)
break;
row = result.fetch();
row = connection->result.fetch();
}
return description.sample_block.cloneWithColumns(std::move(columns));
}
MySQLBlockInputStream::MySQLBlockInputStream(
const Block & sample_block_,
UInt64 max_block_size_,
bool auto_close_)
: max_block_size(max_block_size_)
, auto_close(auto_close_)
{
description.init(sample_block_);
}
MySQLLazyBlockInputStream::MySQLLazyBlockInputStream(
mysqlxx::Pool & pool_,
const std::string & query_str_,
const Block & sample_block_,
const UInt64 max_block_size_,
const bool auto_close_)
: MySQLBlockInputStream(sample_block_, max_block_size_, auto_close_)
, pool(pool_)
, query_str(query_str_)
{
}
void MySQLLazyBlockInputStream::readPrefix()
{
connection = std::make_unique<Connection>(pool.get(), query_str);
if (description.sample_block.columns() != connection->result.getNumFields())
throw Exception{"mysqlxx::UseQueryResult contains " + toString(connection->result.getNumFields()) + " columns while "
+ toString(description.sample_block.columns()) + " expected",
ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH};
}
}
#endif

View File

@ -10,12 +10,13 @@
namespace DB
{
/// Allows processing results of a MySQL query as a sequence of Blocks, simplifies chaining
class MySQLBlockInputStream final : public IBlockInputStream
class MySQLBlockInputStream : public IBlockInputStream
{
public:
MySQLBlockInputStream(
const mysqlxx::PoolWithFailover::Entry & entry_,
const mysqlxx::PoolWithFailover::Entry & entry,
const std::string & query_str,
const Block & sample_block,
const UInt64 max_block_size_,
@ -25,15 +26,43 @@ public:
Block getHeader() const override { return description.sample_block.cloneEmpty(); }
private:
protected:
MySQLBlockInputStream(const Block & sample_block_, UInt64 max_block_size_, bool auto_close_);
Block readImpl() override;
mysqlxx::PoolWithFailover::Entry entry;
mysqlxx::Query query;
mysqlxx::UseQueryResult result;
struct Connection
{
Connection(const mysqlxx::PoolWithFailover::Entry & entry_, const std::string & query_str);
mysqlxx::PoolWithFailover::Entry entry;
mysqlxx::Query query;
mysqlxx::UseQueryResult result;
};
std::unique_ptr<Connection> connection;
const UInt64 max_block_size;
const bool auto_close;
ExternalResultDescription description;
};
/// Like MySQLBlockInputStream, but allocates connection only when reading is starting.
/// It allows to create a lot of stream objects without occupation of all connection pool.
class MySQLLazyBlockInputStream final : public MySQLBlockInputStream
{
public:
MySQLLazyBlockInputStream(
mysqlxx::Pool & pool_,
const std::string & query_str_,
const Block & sample_block_,
const UInt64 max_block_size_,
const bool auto_close_ = false);
private:
void readPrefix() override;
mysqlxx::Pool & pool;
std::string query_str;
};
}

View File

@ -236,11 +236,7 @@ static Block createBlockFromAST(const ASTPtr & node, const DataTypes & types, co
return header.cloneWithColumns(std::move(columns));
}
/** Create a block for set from literal.
* 'set_element_types' - types of what are on the left hand side of IN.
* 'right_arg' - Literal - Tuple or Array.
*/
static Block createBlockForSet(
Block createBlockForSet(
const DataTypePtr & left_arg_type,
const ASTPtr & right_arg,
const DataTypes & set_element_types,
@ -281,14 +277,7 @@ static Block createBlockForSet(
return block;
}
/** Create a block for set from expression.
* 'set_element_types' - types of what are on the left hand side of IN.
* 'right_arg' - list of values: 1, 2, 3 or list of tuples: (1, 2), (3, 4), (5, 6).
*
* We need special implementation for ASTFunction, because in case, when we interpret
* large tuple or array as function, `evaluateConstantExpression` works extremely slow.
*/
static Block createBlockForSet(
Block createBlockForSet(
const DataTypePtr & left_arg_type,
const std::shared_ptr<ASTFunction> & right_arg,
const DataTypes & set_element_types,

View File

@ -19,11 +19,37 @@ using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
class ActionsDAG;
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
/// The case of an explicit enumeration of values.
/// The case of an explicit enumeration of values.
SetPtr makeExplicitSet(
const ASTFunction * node, const ActionsDAG::Index & index, bool create_ordered_set,
const Context & context, const SizeLimits & limits, PreparedSets & prepared_sets);
/** Create a block for set from expression.
* 'set_element_types' - types of what are on the left hand side of IN.
* 'right_arg' - list of values: 1, 2, 3 or list of tuples: (1, 2), (3, 4), (5, 6).
*
* We need special implementation for ASTFunction, because in case, when we interpret
* large tuple or array as function, `evaluateConstantExpression` works extremely slow.
*
* Note: this and following functions are used in third-party applications in Arcadia, so
* they should be declared in header file.
*
*/
Block createBlockForSet(
const DataTypePtr & left_arg_type,
const std::shared_ptr<ASTFunction> & right_arg,
const DataTypes & set_element_types,
const Context & context);
/** Create a block for set from literal.
* 'set_element_types' - types of what are on the left hand side of IN.
* 'right_arg' - Literal - Tuple or Array.
*/
Block createBlockForSet(
const DataTypePtr & left_arg_type,
const ASTPtr & right_arg,
const DataTypes & set_element_types,
const Context & context);
/** For ActionsVisitor
* A stack of ExpressionActions corresponding to nested lambda expressions.

View File

@ -968,6 +968,7 @@ StoragePtr Context::getViewSource()
Settings Context::getSettings() const
{
auto lock = getLock();
return settings;
}
@ -1088,6 +1089,18 @@ String Context::getInitialQueryId() const
}
void Context::setCurrentDatabaseNameInGlobalContext(const String & name)
{
if (global_context != this)
throw Exception("Cannot set current database for non global context, this method should be used during server initialization", ErrorCodes::LOGICAL_ERROR);
auto lock = getLock();
if (!current_database.empty())
throw Exception("Default database name cannot be changed in global context without server restart", ErrorCodes::LOGICAL_ERROR);
current_database = name;
}
void Context::setCurrentDatabase(const String & name)
{
DatabaseCatalog::instance().assertDatabaseExists(name);

View File

@ -359,6 +359,9 @@ public:
String getInitialQueryId() const;
void setCurrentDatabase(const String & name);
/// Set current_database for global context. We don't validate that database
/// exists because it should be set before databases loading.
void setCurrentDatabaseNameInGlobalContext(const String & name);
void setCurrentQueryId(const String & query_id);
void killCurrentQuery();

View File

@ -888,7 +888,7 @@ void InterpreterCreateQuery::prepareOnClusterQuery(ASTCreateQuery & create, cons
{
String zk_path = create.storage->engine->arguments->children[0]->as<ASTLiteral>()->value.get<String>();
Macros::MacroExpansionInfo info;
info.uuid = create.uuid;
info.table_id.uuid = create.uuid;
info.ignore_unknown = true;
context.getMacros()->expand(zk_path, info);
if (!info.expanded_uuid)

View File

@ -431,7 +431,7 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even
}
if (get_settings && query_context)
res.query_settings = std::make_shared<Settings>(query_context->getSettingsRef());
res.query_settings = std::make_shared<Settings>(query_context->getSettings());
return res;
}

View File

@ -39,6 +39,7 @@ Block QueryLogElement::createBlock()
{std::move(query_status_datatype), "type"},
{std::make_shared<DataTypeDate>(), "event_date"},
{std::make_shared<DataTypeDateTime>(), "event_time"},
{std::make_shared<DataTypeDateTime64>(6), "event_time_microseconds"},
{std::make_shared<DataTypeDateTime>(), "query_start_time"},
{std::make_shared<DataTypeDateTime64>(6), "query_start_time_microseconds"},
{std::make_shared<DataTypeUInt64>(), "query_duration_ms"},
@ -97,6 +98,7 @@ void QueryLogElement::appendToBlock(MutableColumns & columns) const
columns[i++]->insert(type);
columns[i++]->insert(DateLUT::instance().toDayNum(event_time));
columns[i++]->insert(event_time);
columns[i++]->insert(event_time_microseconds);
columns[i++]->insert(query_start_time);
columns[i++]->insert(query_start_time_microseconds);
columns[i++]->insert(query_duration_ms);

View File

@ -30,6 +30,7 @@ struct QueryLogElement
/// Depending on the type of query and type of stage, not all the fields may be filled.
time_t event_time{};
UInt64 event_time_microseconds{};
time_t query_start_time{};
UInt64 query_start_time_microseconds{};
UInt64 query_duration_ms{};

View File

@ -23,6 +23,7 @@ Block QueryThreadLogElement::createBlock()
return {
{std::make_shared<DataTypeDate>(), "event_date"},
{std::make_shared<DataTypeDateTime>(), "event_time"},
{std::make_shared<DataTypeDateTime64>(6), "event_time_microseconds"},
{std::make_shared<DataTypeDateTime>(), "query_start_time"},
{std::make_shared<DataTypeDateTime64>(6), "query_start_time_microseconds"},
{std::make_shared<DataTypeUInt64>(), "query_duration_ms"},
@ -73,6 +74,7 @@ void QueryThreadLogElement::appendToBlock(MutableColumns & columns) const
columns[i++]->insert(DateLUT::instance().toDayNum(event_time));
columns[i++]->insert(event_time);
columns[i++]->insert(event_time_microseconds);
columns[i++]->insert(query_start_time);
columns[i++]->insert(query_start_time_microseconds);
columns[i++]->insert(query_duration_ms);

View File

@ -16,6 +16,7 @@ namespace DB
struct QueryThreadLogElement
{
time_t event_time{};
UInt64 event_time_microseconds{};
/// When query was attached to current thread
time_t query_start_time{};
/// same as above but adds microsecond precision

View File

@ -438,7 +438,7 @@ void SystemLog<LogElement>::flushImpl(const std::vector<LogElement> & to_flush,
ASTPtr query_ptr(insert.release());
// we need query context to do inserts to target table with MV containing subqueries or joins
auto insert_context = Context(context);
Context insert_context(context);
insert_context.makeQueryContext();
InterpreterInsertQuery interpreter(query_ptr, insert_context);

View File

@ -2,6 +2,7 @@
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeString.h>
#include <Common/ClickHouseRevision.h>
@ -29,6 +30,7 @@ Block TextLogElement::createBlock()
{
{std::make_shared<DataTypeDate>(), "event_date"},
{std::make_shared<DataTypeDateTime>(), "event_time"},
{std::make_shared<DataTypeDateTime64>(6), "event_time_microseconds"},
{std::make_shared<DataTypeUInt32>(), "microseconds"},
{std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()), "thread_name"},
@ -52,6 +54,7 @@ void TextLogElement::appendToBlock(MutableColumns & columns) const
columns[i++]->insert(DateLUT::instance().toDayNum(event_time));
columns[i++]->insert(event_time);
columns[i++]->insert(event_time_microseconds);
columns[i++]->insert(microseconds);
columns[i++]->insertData(thread_name.data(), thread_name.size());

View File

@ -9,6 +9,7 @@ using Poco::Message;
struct TextLogElement
{
time_t event_time{};
UInt64 event_time_microseconds{};
UInt32 microseconds;
String thread_name;

View File

@ -322,7 +322,14 @@ void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log)
{
QueryThreadLogElement elem;
elem.event_time = time(nullptr);
// construct current_time and current_time_microseconds using the same time point
// so that the two times will always be equal up to a precision of a second.
const auto now = std::chrono::system_clock::now();
auto current_time = time_in_seconds(now);
auto current_time_microseconds = time_in_microseconds(now);
elem.event_time = current_time;
elem.event_time_microseconds = current_time_microseconds;
elem.query_start_time = query_start_time;
elem.query_start_time_microseconds = query_start_time_microseconds;
elem.query_duration_ms = (getCurrentTimeNanoseconds() - query_start_time_nanoseconds) / 1000000U;

View File

@ -4,6 +4,7 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <Common/ClickHouseRevision.h>
@ -26,6 +27,7 @@ Block TraceLogElement::createBlock()
{
{std::make_shared<DataTypeDate>(), "event_date"},
{std::make_shared<DataTypeDateTime>(), "event_time"},
{std::make_shared<DataTypeDateTime64>(6), "event_time_microseconds"},
{std::make_shared<DataTypeUInt64>(), "timestamp_ns"},
{std::make_shared<DataTypeUInt32>(), "revision"},
{std::make_shared<TraceDataType>(trace_values), "trace_type"},
@ -42,6 +44,7 @@ void TraceLogElement::appendToBlock(MutableColumns & columns) const
columns[i++]->insert(DateLUT::instance().toDayNum(event_time));
columns[i++]->insert(event_time);
columns[i++]->insert(event_time_microseconds);
columns[i++]->insert(timestamp_ns);
columns[i++]->insert(ClickHouseRevision::getVersionRevision());
columns[i++]->insert(static_cast<UInt8>(trace_type));

View File

@ -18,6 +18,7 @@ struct TraceLogElement
static const TraceDataType::Values trace_values;
time_t event_time{};
UInt64 event_time_microseconds{};
UInt64 timestamp_ns{};
TraceType trace_type{};
UInt64 thread_id{};

View File

@ -206,10 +206,11 @@ static void onExceptionBeforeStart(const String & query_for_logging, Context & c
elem.type = QueryLogElementType::EXCEPTION_BEFORE_START;
// all callers to onExceptionBeforeStart upstream construct the timespec for event_time and
// event_time_microseconds from the same timespec. So it can be assumed that both of these
// all callers to onExceptionBeforeStart method construct the timespec for event_time and
// event_time_microseconds from the same time point. So, it can be assumed that both of these
// times are equal upto the precision of a second.
elem.event_time = current_time;
elem.event_time_microseconds = current_time_microseconds;
elem.query_start_time = current_time;
elem.query_start_time_microseconds = current_time_microseconds;
@ -319,7 +320,9 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
logQuery(query_for_logging, context, internal);
if (!internal)
{
onExceptionBeforeStart(query_for_logging, context, current_time, current_time_microseconds, ast);
}
throw;
}
@ -484,6 +487,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
elem.type = QueryLogElementType::QUERY_START;
elem.event_time = current_time;
elem.event_time_microseconds = current_time_microseconds;
elem.query_start_time = current_time;
elem.query_start_time_microseconds = current_time_microseconds;
@ -554,8 +558,11 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
elem.type = QueryLogElementType::QUERY_FINISH;
elem.event_time = time(nullptr);
// construct event_time and event_time_microseconds using the same time point
// so that the two times will always be equal up to a precision of a second.
const auto time_now = std::chrono::system_clock::now();
elem.event_time = time_in_seconds(time_now);
elem.event_time_microseconds = time_in_microseconds(time_now);
status_info_to_query_log(elem, info, ast);
auto progress_callback = context.getProgressCallback();
@ -615,7 +622,12 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
elem.type = QueryLogElementType::EXCEPTION_WHILE_PROCESSING;
elem.event_time = time(nullptr);
// event_time and event_time_microseconds are being constructed from the same time point
// to ensure that both the times will be equal upto the precision of a second.
const auto time_now = std::chrono::system_clock::now();
elem.event_time = time_in_seconds(time_now);
elem.event_time_microseconds = time_in_microseconds(time_now);
elem.query_duration_ms = 1000 * (elem.event_time - elem.query_start_time);
elem.exception_code = getCurrentExceptionCode();
elem.exception = getCurrentExceptionMessage(false);

View File

@ -53,7 +53,7 @@ public:
ASTPtr clone() const override
{
auto replacement = std::make_shared<Replacement>(*this);
replacement->name = name;
replacement->children.clear();
replacement->expr = expr->clone();
replacement->children.push_back(replacement->expr);
return replacement;

View File

@ -0,0 +1,937 @@
#include <cassert>
#include <Parsers/obfuscateQueries.h>
#include <Parsers/Lexer.h>
#include <Poco/String.h>
#include <Common/Exception.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/BitHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/WriteBuffer.h>
#include <IO/ReadHelpers.h>
#include <IO/ReadBufferFromMemory.h>
namespace DB
{
namespace ErrorCodes
{
extern const int TOO_MANY_TEMPORARY_COLUMNS;
}
namespace
{
const std::unordered_set<std::string_view> keywords
{
"CREATE", "DATABASE", "IF", "NOT", "EXISTS", "TEMPORARY", "TABLE", "ON", "CLUSTER", "DEFAULT",
"MATERIALIZED", "ALIAS", "ENGINE", "AS", "VIEW", "POPULATE", "SETTINGS", "ATTACH", "DETACH", "DROP",
"RENAME", "TO", "ALTER", "ADD", "MODIFY", "CLEAR", "COLUMN", "AFTER", "COPY", "PROJECT",
"PRIMARY", "KEY", "CHECK", "PARTITION", "PART", "FREEZE", "FETCH", "FROM", "SHOW", "INTO",
"OUTFILE", "FORMAT", "TABLES", "DATABASES", "LIKE", "PROCESSLIST", "CASE", "WHEN", "THEN", "ELSE",
"END", "DESCRIBE", "DESC", "USE", "SET", "OPTIMIZE", "FINAL", "DEDUPLICATE", "INSERT", "VALUES",
"SELECT", "DISTINCT", "SAMPLE", "ARRAY", "JOIN", "GLOBAL", "LOCAL", "ANY", "ALL", "INNER",
"LEFT", "RIGHT", "FULL", "OUTER", "CROSS", "USING", "PREWHERE", "WHERE", "GROUP", "BY",
"WITH", "TOTALS", "HAVING", "ORDER", "COLLATE", "LIMIT", "UNION", "AND", "OR", "ASC",
"IN", "KILL", "QUERY", "SYNC", "ASYNC", "TEST", "BETWEEN", "TRUNCATE", "USER", "ROLE",
"PROFILE", "QUOTA", "POLICY", "ROW", "GRANT", "REVOKE", "OPTION", "ADMIN", "EXCEPT", "REPLACE",
"IDENTIFIED", "HOST", "NAME", "READONLY", "WRITABLE", "PERMISSIVE", "FOR", "RESTRICTIVE", "RANDOMIZED",
"INTERVAL", "LIMITS", "ONLY", "TRACKING", "IP", "REGEXP", "ILIKE", "DICTIONARY"
};
const std::unordered_set<std::string_view> keep_words
{
"id", "name", "value", "num",
"Id", "Name", "Value", "Num",
"ID", "NAME", "VALUE", "NUM",
};
/// The list of nouns collected from here: http://www.desiquintans.com/nounlist, Public domain.
std::initializer_list<std::string_view> nouns
{
"aardvark", "abacus", "abbey", "abbreviation", "abdomen", "ability", "abnormality", "abolishment", "abortion",
"abrogation", "absence", "abundance", "abuse", "academics", "academy", "accelerant", "accelerator", "accent", "acceptance", "access",
"accessory", "accident", "accommodation", "accompanist", "accomplishment", "accord", "accordance", "accordion", "account", "accountability",
"accountant", "accounting", "accuracy", "accusation", "acetate", "achievement", "achiever", "acid", "acknowledgment", "acorn", "acoustics",
"acquaintance", "acquisition", "acre", "acrylic", "act", "action", "activation", "activist", "activity", "actor", "actress", "acupuncture",
"ad", "adaptation", "adapter", "addiction", "addition", "address", "adjective", "adjustment", "admin", "administration", "administrator",
"admire", "admission", "adobe", "adoption", "adrenalin", "adrenaline", "adult", "adulthood", "advance", "advancement", "advantage", "advent",
"adverb", "advertisement", "advertising", "advice", "adviser", "advocacy", "advocate", "affair", "affect", "affidavit", "affiliate",
"affinity", "afoul", "afterlife", "aftermath", "afternoon", "aftershave", "aftershock", "afterthought", "age", "agency", "agenda", "agent",
"aggradation", "aggression", "aglet", "agony", "agreement", "agriculture", "aid", "aide", "aim", "air", "airbag", "airbus", "aircraft",
"airfare", "airfield", "airforce", "airline", "airmail", "airman", "airplane", "airport", "airship", "airspace", "alarm", "alb", "albatross",
"album", "alcohol", "alcove", "alder", "ale", "alert", "alfalfa", "algebra", "algorithm", "alias", "alibi", "alien", "allegation", "allergist",
"alley", "alliance", "alligator", "allocation", "allowance", "alloy", "alluvium", "almanac", "almighty", "almond", "alpaca", "alpenglow",
"alpenhorn", "alpha", "alphabet", "altar", "alteration", "alternative", "altitude", "alto", "aluminium", "aluminum", "amazement", "amazon",
"ambassador", "amber", "ambience", "ambiguity", "ambition", "ambulance", "amendment", "amenity", "ammunition", "amnesty", "amount", "amusement",
"anagram", "analgesia", "analog", "analogue", "analogy", "analysis", "analyst", "analytics", "anarchist", "anarchy", "anatomy", "ancestor",
"anchovy", "android", "anesthesiologist", "anesthesiology", "angel", "anger", "angina", "angiosperm", "angle", "angora", "angstrom",
"anguish", "animal", "anime", "anise", "ankle", "anklet", "anniversary", "announcement", "annual", "anorak", "answer", "ant", "anteater",
"antecedent", "antechamber", "antelope", "antennae", "anterior", "anthropology", "antibody", "anticipation", "anticodon", "antigen",
"antique", "antiquity", "antler", "antling", "anxiety", "anybody", "anyone", "anything", "anywhere", "apartment", "ape", "aperitif",
"apology", "app", "apparatus", "apparel", "appeal", "appearance", "appellation", "appendix", "appetiser", "appetite", "appetizer", "applause",
"apple", "applewood", "appliance", "application", "appointment", "appreciation", "apprehension", "approach", "appropriation", "approval",
"apricot", "apron", "apse", "aquarium", "aquifer", "arcade", "arch", "archaeologist", "archaeology", "archeology", "archer",
"architect", "architecture", "archives", "area", "arena", "argument", "arithmetic", "ark", "arm", "armadillo", "armament",
"armchair", "armoire", "armor", "armour", "armpit", "armrest", "army", "arrangement", "array", "arrest", "arrival", "arrogance", "arrow",
"art", "artery", "arthur", "artichoke", "article", "artifact", "artificer", "artist", "ascend", "ascent", "ascot", "ash", "ashram", "ashtray",
"aside", "asparagus", "aspect", "asphalt", "aspic", "assassination", "assault", "assembly", "assertion", "assessment", "asset",
"assignment", "assist", "assistance", "assistant", "associate", "association", "assumption", "assurance", "asterisk", "astrakhan", "astrolabe",
"astrologer", "astrology", "astronomy", "asymmetry", "atelier", "atheist", "athlete", "athletics", "atmosphere", "atom", "atrium", "attachment",
"attack", "attacker", "attainment", "attempt", "attendance", "attendant", "attention", "attenuation", "attic", "attitude", "attorney",
"attraction", "attribute", "auction", "audience", "audit", "auditorium", "aunt", "authentication", "authenticity", "author", "authorisation",
"authority", "authorization", "auto", "autoimmunity", "automation", "automaton", "autumn", "availability", "avalanche", "avenue", "average",
"avocado", "award", "awareness", "awe", "axis", "azimuth", "babe", "baboon", "babushka", "baby", "bachelor", "back", "backbone",
"backburn", "backdrop", "background", "backpack", "backup", "backyard", "bacon", "bacterium", "badge", "badger", "bafflement", "bag",
"bagel", "baggage", "baggie", "baggy", "bagpipe", "bail", "bait", "bake", "baker", "bakery", "bakeware", "balaclava", "balalaika", "balance",
"balcony", "ball", "ballet", "balloon", "balloonist", "ballot", "ballpark", "bamboo", "ban", "banana", "band", "bandana", "bandanna",
"bandolier", "bandwidth", "bangle", "banjo", "bank", "bankbook", "banker", "banking", "bankruptcy", "banner", "banquette", "banyan",
"baobab", "bar", "barbecue", "barbeque", "barber", "barbiturate", "bargain", "barge", "baritone", "barium", "bark", "barley", "barn",
"barometer", "barracks", "barrage", "barrel", "barrier", "barstool", "bartender", "base", "baseball", "baseboard", "baseline", "basement",
"basics", "basil", "basin", "basis", "basket", "basketball", "bass", "bassinet", "bassoon", "bat", "bath", "bather", "bathhouse", "bathrobe",
"bathroom", "bathtub", "battalion", "batter", "battery", "batting", "battle", "battleship", "bay", "bayou", "beach", "bead", "beak",
"beam", "bean", "beancurd", "beanie", "beanstalk", "bear", "beard", "beast", "beastie", "beat", "beating", "beauty", "beaver", "beck",
"bed", "bedrock", "bedroom", "bee", "beech", "beef", "beer", "beet", "beetle", "beggar", "beginner", "beginning", "begonia", "behalf",
"behavior", "behaviour", "beheading", "behest", "behold", "being", "belfry", "belief", "believer", "bell", "belligerency", "bellows",
"belly", "belt", "bench", "bend", "beneficiary", "benefit", "beret", "berry", "bestseller", "bet", "beverage", "beyond",
"bias", "bibliography", "bicycle", "bid", "bidder", "bidding", "bidet", "bifocals", "bijou", "bike", "bikini", "bill", "billboard", "billing",
"billion", "bin", "binoculars", "biology", "biopsy", "biosphere", "biplane", "birch", "bird", "birdbath", "birdcage",
"birdhouse", "birth", "birthday", "biscuit", "bit", "bite", "bitten", "bitter", "black", "blackberry", "blackbird", "blackboard", "blackfish",
"blackness", "bladder", "blade", "blame", "blank", "blanket", "blast", "blazer", "blend", "blessing", "blight", "blind", "blinker", "blister",
"blizzard", "block", "blocker", "blog", "blogger", "blood", "bloodflow", "bloom", "bloomer", "blossom", "blouse", "blow", "blowgun",
"blowhole", "blue", "blueberry", "blush", "boar", "board", "boat", "boatload", "boatyard", "bob", "bobcat", "body", "bog", "bolero",
"bolt", "bomb", "bomber", "bombing", "bond", "bonding", "bondsman", "bone", "bonfire", "bongo", "bonnet", "bonsai", "bonus", "boogeyman",
"book", "bookcase", "bookend", "booking", "booklet", "bookmark", "boolean", "boom", "boon", "boost", "booster", "boot", "bootee", "bootie",
"booty", "border", "bore", "borrower", "borrowing", "bosom", "boss", "botany", "bother", "bottle", "bottling", "bottom",
"boudoir", "bough", "boulder", "boulevard", "boundary", "bouquet", "bourgeoisie", "bout", "boutique", "bow", "bower", "bowl", "bowler",
"bowling", "bowtie", "box", "boxer", "boxspring", "boy", "boycott", "boyfriend", "boyhood", "boysenberry", "bra", "brace", "bracelet",
"bracket", "brain", "brake", "bran", "branch", "brand", "brandy", "brass", "brassiere", "bratwurst", "bread", "breadcrumb", "breadfruit",
"break", "breakdown", "breakfast", "breakpoint", "breakthrough", "breast", "breastplate", "breath", "breeze", "brewer", "bribery", "brick",
"bricklaying", "bride", "bridge", "brief", "briefing", "briefly", "briefs", "brilliant", "brink", "brisket", "broad", "broadcast", "broccoli",
"brochure", "brocolli", "broiler", "broker", "bronchitis", "bronco", "bronze", "brooch", "brood", "brook", "broom", "brother",
"brow", "brown", "brownie", "browser", "browsing", "brunch", "brush", "brushfire", "brushing", "bubble", "buck", "bucket", "buckle",
"buckwheat", "bud", "buddy", "budget", "buffalo", "buffer", "buffet", "bug", "buggy", "bugle", "builder", "building", "bulb", "bulk",
"bull", "bulldozer", "bullet", "bump", "bumper", "bun", "bunch", "bungalow", "bunghole", "bunkhouse", "burden", "bureau",
"burglar", "burial", "burlesque", "burn", "burning", "burrito", "burro", "burrow", "burst", "bus", "bush", "business", "businessman",
"bust", "bustle", "butane", "butcher", "butler", "butter", "butterfly", "button", "buy", "buyer", "buying", "buzz", "buzzard",
"cabana", "cabbage", "cabin", "cabinet", "cable", "caboose", "cacao", "cactus", "caddy", "cadet", "cafe", "caffeine", "caftan", "cage",
"cake", "calcification", "calculation", "calculator", "calculus", "calendar", "calf", "caliber", "calibre", "calico", "call", "calm",
"calorie", "camel", "cameo", "camera", "camp", "campaign", "campaigning", "campanile", "camper", "campus", "can", "canal", "cancer",
"candelabra", "candidacy", "candidate", "candle", "candy", "cane", "cannibal", "cannon", "canoe", "canon", "canopy", "cantaloupe", "canteen",
"canvas", "cap", "capability", "capacity", "cape", "caper", "capital", "capitalism", "capitulation", "capon", "cappelletti", "cappuccino",
"captain", "caption", "captor", "car", "carabao", "caramel", "caravan", "carbohydrate", "carbon", "carboxyl", "card", "cardboard", "cardigan",
"care", "career", "cargo", "caribou", "carload", "carnation", "carnival", "carol", "carotene", "carp", "carpenter", "carpet", "carpeting",
"carport", "carriage", "carrier", "carrot", "carry", "cart", "cartel", "carter", "cartilage", "cartload", "cartoon", "cartridge", "carving",
"cascade", "case", "casement", "cash", "cashew", "cashier", "casino", "casket", "cassava", "casserole", "cassock", "cast", "castanet",
"castle", "casualty", "cat", "catacomb", "catalogue", "catalysis", "catalyst", "catamaran", "catastrophe", "catch", "catcher", "category",
"caterpillar", "cathedral", "cation", "catsup", "cattle", "cauliflower", "causal", "cause", "causeway", "caution", "cave", "caviar",
"cayenne", "ceiling", "celebration", "celebrity", "celeriac", "celery", "cell", "cellar", "cello", "celsius", "cement", "cemetery", "cenotaph",
"census", "cent", "center", "centimeter", "centre", "centurion", "century", "cephalopod", "ceramic", "ceramics", "cereal", "ceremony",
"certainty", "certificate", "certification", "cesspool", "chafe", "chain", "chainstay", "chair", "chairlift", "chairman", "chairperson",
"chaise", "chalet", "chalice", "chalk", "challenge", "chamber", "champagne", "champion", "championship", "chance", "chandelier", "change",
"channel", "chaos", "chap", "chapel", "chaplain", "chapter", "character", "characteristic", "characterization", "chard", "charge", "charger",
"charity", "charlatan", "charm", "charset", "chart", "charter", "chasm", "chassis", "chastity", "chasuble", "chateau", "chatter", "chauffeur",
"chauvinist", "check", "checkbook", "checking", "checkout", "checkroom", "cheddar", "cheek", "cheer", "cheese", "cheesecake", "cheetah",
"chef", "chem", "chemical", "chemistry", "chemotaxis", "cheque", "cherry", "chess", "chest", "chestnut", "chick", "chicken", "chicory",
"chief", "chiffonier", "child", "childbirth", "childhood", "chili", "chill", "chime", "chimpanzee", "chin", "chinchilla", "chino", "chip",
"chipmunk", "chivalry", "chive", "chives", "chocolate", "choice", "choir", "choker", "cholesterol", "choosing", "chop",
"chops", "chopstick", "chopsticks", "chord", "chorus", "chow", "chowder", "chrome", "chromolithograph", "chronicle", "chronograph", "chronometer",
"chrysalis", "chub", "chuck", "chug", "church", "churn", "chutney", "cicada", "cigarette", "cilantro", "cinder", "cinema", "cinnamon",
"circadian", "circle", "circuit", "circulation", "circumference", "circumstance", "cirrhosis", "cirrus", "citizen", "citizenship", "citron",
"citrus", "city", "civilian", "civilisation", "civilization", "claim", "clam", "clamp", "clan", "clank", "clapboard", "clarification",
"clarinet", "clarity", "clasp", "class", "classic", "classification", "classmate", "classroom", "clause", "clave", "clavicle", "clavier",
"claw", "clay", "cleaner", "clearance", "clearing", "cleat", "cleavage", "clef", "cleft", "clergyman", "cleric", "clerk", "click", "client",
"cliff", "climate", "climb", "clinic", "clip", "clipboard", "clipper", "cloak", "cloakroom", "clock", "clockwork", "clogs", "cloister",
"clone", "close", "closet", "closing", "closure", "cloth", "clothes", "clothing", "cloud", "cloudburst", "clove", "clover", "cloves",
"club", "clue", "cluster", "clutch", "coach", "coal", "coalition", "coast", "coaster", "coat", "cob", "cobbler", "cobweb",
"cock", "cockpit", "cockroach", "cocktail", "cocoa", "coconut", "cod", "code", "codepage", "codling", "codon", "codpiece", "coevolution",
"cofactor", "coffee", "coffin", "cohesion", "cohort", "coil", "coin", "coincidence", "coinsurance", "coke", "cold", "coleslaw", "coliseum",
"collaboration", "collagen", "collapse", "collar", "collard", "collateral", "colleague", "collection", "collectivisation", "collectivization",
"collector", "college", "collision", "colloquy", "colon", "colonial", "colonialism", "colonisation", "colonization", "colony", "color",
"colorlessness", "colt", "column", "columnist", "comb", "combat", "combination", "combine", "comeback", "comedy", "comestible", "comfort",
"comfortable", "comic", "comics", "comma", "command", "commander", "commandment", "comment", "commerce", "commercial", "commission",
"commitment", "committee", "commodity", "common", "commonsense", "commotion", "communicant", "communication", "communion", "communist",
"community", "commuter", "company", "comparison", "compass", "compassion", "compassionate", "compensation", "competence", "competition",
"competitor", "complaint", "complement", "completion", "complex", "complexity", "compliance", "complication", "complicity", "compliment",
"component", "comportment", "composer", "composite", "composition", "compost", "comprehension", "compress", "compromise", "comptroller",
"compulsion", "computer", "comradeship", "con", "concentrate", "concentration", "concept", "conception", "concern", "concert", "conclusion",
"concrete", "condition", "conditioner", "condominium", "condor", "conduct", "conductor", "cone", "confectionery", "conference", "confidence",
"confidentiality", "configuration", "confirmation", "conflict", "conformation", "confusion", "conga", "congo", "congregation", "congress",
"congressman", "congressperson", "conifer", "connection", "connotation", "conscience", "consciousness", "consensus", "consent", "consequence",
"conservation", "conservative", "consideration", "consignment", "consist", "consistency", "console", "consonant", "conspiracy", "conspirator",
"constant", "constellation", "constitution", "constraint", "construction", "consul", "consulate", "consulting", "consumer", "consumption",
"contact", "contact lens", "contagion", "container", "content", "contention", "contest", "context", "continent", "contingency", "continuity",
"contour", "contract", "contractor", "contrail", "contrary", "contrast", "contribution", "contributor", "control", "controller", "controversy",
"convection", "convenience", "convention", "conversation", "conversion", "convert", "convertible", "conviction", "cook", "cookbook",
"cookie", "cooking", "coonskin", "cooperation", "coordination", "coordinator", "cop", "cope", "copper", "copy", "copying",
"copyright", "copywriter", "coral", "cord", "corduroy", "core", "cork", "cormorant", "corn", "corner", "cornerstone", "cornet", "cornflakes",
"cornmeal", "corporal", "corporation", "corporatism", "corps", "corral", "correspondence", "correspondent", "corridor", "corruption",
"corsage", "cosset", "cost", "costume", "cot", "cottage", "cotton", "couch", "cougar", "cough", "council", "councilman", "councilor",
"councilperson", "counsel", "counseling", "counselling", "counsellor", "counselor", "count", "counter", "counterpart",
"counterterrorism", "countess", "country", "countryside", "county", "couple", "coupon", "courage", "course", "court", "courthouse", "courtroom",
"cousin", "covariate", "cover", "coverage", "coverall", "cow", "cowbell", "cowboy", "coyote", "crab", "crack", "cracker", "crackers",
"cradle", "craft", "craftsman", "cranberry", "crane", "cranky", "crash", "crate", "cravat", "craw", "crawdad", "crayfish", "crayon",
"crazy", "cream", "creation", "creationism", "creationist", "creative", "creativity", "creator", "creature", "creche", "credential",
"credenza", "credibility", "credit", "creditor", "creek", "creme brulee", "crepe", "crest", "crew", "crewman", "crewmate", "crewmember",
"crewmen", "cria", "crib", "cribbage", "cricket", "cricketer", "crime", "criminal", "crinoline", "crisis", "crisp", "criteria", "criterion",
"critic", "criticism", "crocodile", "crocus", "croissant", "crook", "crop", "cross", "crotch",
"croup", "crow", "crowd", "crown", "crucifixion", "crude", "cruelty", "cruise", "crumb", "crunch", "crusader", "crush", "crust", "cry",
"crystal", "crystallography", "cub", "cube", "cuckoo", "cucumber", "cue", "cuisine", "cultivar", "cultivator", "culture",
"culvert", "cummerbund", "cup", "cupboard", "cupcake", "cupola", "curd", "cure", "curio", "curiosity", "curl", "curler", "currant", "currency",
"current", "curriculum", "curry", "curse", "cursor", "curtailment", "curtain", "curve", "cushion", "custard", "custody", "custom", "customer",
"cut", "cuticle", "cutlet", "cutover", "cutting", "cyclamen", "cycle", "cyclone", "cyclooxygenase", "cygnet", "cylinder", "cymbal", "cynic",
"cyst", "cytokine", "cytoplasm", "dad", "daddy", "daffodil", "dagger", "dahlia", "daikon", "daily", "dairy", "daisy", "dam", "damage",
"dame", "dance", "dancer", "dancing", "dandelion", "danger", "dare", "dark", "darkness", "darn", "dart", "dash", "dashboard",
"data", "database", "date", "daughter", "dawn", "day", "daybed", "daylight", "dead", "deadline", "deal", "dealer", "dealing", "dearest",
"death", "deathwatch", "debate", "debris", "debt", "debtor", "decade", "decadence", "decency", "decimal", "decision",
"deck", "declaration", "declination", "decline", "decoder", "decongestant", "decoration", "decrease", "decryption", "dedication", "deduce",
"deduction", "deed", "deep", "deer", "default", "defeat", "defendant", "defender", "defense", "deficit", "definition", "deformation",
"degradation", "degree", "delay", "deliberation", "delight", "delivery", "demand", "democracy", "democrat", "demon", "demur", "den",
"denim", "denominator", "density", "dentist", "deodorant", "department", "departure", "dependency", "dependent", "deployment", "deposit",
"deposition", "depot", "depression", "depressive", "depth", "deputy", "derby", "derivation", "derivative", "derrick", "descendant", "descent",
"description", "desert", "design", "designation", "designer", "desire", "desk", "desktop", "dessert", "destination", "destiny", "destroyer",
"destruction", "detail", "detainee", "detainment", "detection", "detective", "detector", "detention", "determination", "detour", "devastation",
"developer", "developing", "development", "developmental", "deviance", "deviation", "device", "devil", "dew", "dhow", "diabetes", "diadem",
"diagnosis", "diagram", "dial", "dialect", "dialogue", "diam", "diamond", "diaper", "diaphragm", "diarist", "diary", "dibble", "dickey", "dictaphone", "dictator", "diction", "dictionary", "die", "diesel", "diet", "difference", "differential", "difficulty", "diffuse",
"dig", "digestion", "digestive", "digger", "digging", "digit", "dignity", "dilapidation", "dill", "dilution", "dime", "dimension", "dimple",
"diner", "dinghy", "dining", "dinner", "dinosaur", "dioxide", "dip", "diploma", "diplomacy", "dipstick", "direction", "directive", "director",
"directory", "dirndl", "dirt", "disability", "disadvantage", "disagreement", "disappointment", "disarmament", "disaster", "discharge",
"discipline", "disclaimer", "disclosure", "disco", "disconnection", "discount", "discourse", "discovery", "discrepancy", "discretion",
"discrimination", "discussion", "disdain", "disease", "disembodiment", "disengagement", "disguise", "disgust", "dish", "dishwasher",
"disk", "disparity", "dispatch", "displacement", "display", "disposal", "disposer", "disposition", "dispute", "disregard", "disruption",
"dissemination", "dissonance", "distance", "distinction", "distortion", "distribution", "distributor", "district", "divalent", "divan",
"diver", "diversity", "divide", "dividend", "divider", "divine", "diving", "division", "divorce", "doc", "dock", "doctor", "doctorate",
"doctrine", "document", "documentary", "documentation", "doe", "dog", "doggie", "dogsled", "dogwood", "doing", "doll", "dollar", "dollop",
"dolman", "dolor", "dolphin", "domain", "dome", "domination", "donation", "donkey", "donor", "donut", "door", "doorbell", "doorknob",
"doorpost", "doorway", "dory", "dose", "dot", "double", "doubling", "doubt", "doubter", "dough", "doughnut", "down", "downfall", "downforce",
"downgrade", "download", "downstairs", "downtown", "downturn", "dozen", "draft", "drag", "dragon", "dragonfly", "dragonfruit", "dragster",
"drain", "drainage", "drake", "drama", "dramaturge", "drapes", "draw", "drawbridge", "drawer", "drawing", "dream", "dreamer", "dredger",
"dress", "dresser", "dressing", "drill", "drink", "drinking", "drive", "driver", "driveway", "driving", "drizzle", "dromedary", "drop",
"drudgery", "drug", "drum", "drummer", "drunk", "dryer", "duck", "duckling", "dud", "dude", "due", "duel", "dueling", "duffel", "dugout",
"dulcimer", "dumbwaiter", "dump", "dump truck", "dune", "dune buggy", "dungarees", "dungeon", "duplexer", "duration", "durian", "dusk",
"dust", "dust storm", "duster", "duty", "dwarf", "dwell", "dwelling", "dynamics", "dynamite", "dynamo", "dynasty", "dysfunction",
"eagle", "eaglet", "ear", "eardrum", "earmuffs", "earnings", "earplug", "earring", "earrings", "earth", "earthquake",
"earthworm", "ease", "easel", "east", "eating", "eaves", "eavesdropper", "ecclesia", "echidna", "eclipse", "ecliptic", "ecology", "economics",
"economy", "ecosystem", "ectoderm", "ectodermal", "ecumenist", "eddy", "edge", "edger", "edible", "editing", "edition", "editor", "editorial",
"education", "eel", "effacement", "effect", "effective", "effectiveness", "effector", "efficacy", "efficiency", "effort", "egg", "egghead",
"eggnog", "eggplant", "ego", "eicosanoid", "ejector", "elbow", "elderberry", "election", "electricity", "electrocardiogram", "electronics",
"element", "elephant", "elevation", "elevator", "eleventh", "elf", "elicit", "eligibility", "elimination", "elite", "elixir", "elk",
"ellipse", "elm", "elongation", "elver", "email", "emanate", "embarrassment", "embassy", "embellishment", "embossing", "embryo", "emerald",
"emergence", "emergency", "emergent", "emery", "emission", "emitter", "emotion", "emphasis", "empire", "employ", "employee", "employer",
"employment", "empowerment", "emu", "enactment", "encirclement", "enclave", "enclosure", "encounter", "encouragement", "encyclopedia",
"end", "endive", "endoderm", "endorsement", "endothelium", "endpoint", "enemy", "energy", "enforcement", "engagement", "engine", "engineer",
"engineering", "enigma", "enjoyment", "enquiry", "enrollment", "enterprise", "entertainment", "enthusiasm", "entirety", "entity", "entrance",
"entree", "entrepreneur", "entry", "envelope", "environment", "envy", "enzyme", "epauliere", "epee", "ephemera", "ephemeris", "ephyra",
"epic", "episode", "epithelium", "epoch", "eponym", "epoxy", "equal", "equality", "equation", "equinox", "equipment", "equity", "equivalent",
"era", "eraser", "erection", "erosion", "error", "escalator", "escape", "escort", "espadrille", "espalier", "essay", "essence", "essential",
"establishment", "estate", "estimate", "estrogen", "estuary", "eternity", "ethernet", "ethics", "ethnicity", "ethyl", "euphonium", "eurocentrism",
"evaluation", "evaluator", "evaporation", "eve", "evening", "event", "everybody", "everyone", "everything", "eviction",
"evidence", "evil", "evocation", "evolution", "exaggeration", "exam", "examination", "examiner", "example",
"exasperation", "excellence", "exception", "excerpt", "excess", "exchange", "excitement", "exclamation", "excursion", "excuse", "execution",
"executive", "executor", "exercise", "exhaust", "exhaustion", "exhibit", "exhibition", "exile", "existence", "exit", "exocrine", "expansion",
"expansionism", "expectancy", "expectation", "expedition", "expense", "experience", "experiment", "experimentation", "expert", "expertise",
"explanation", "exploration", "explorer", "explosion", "export", "expose", "exposition", "exposure", "expression", "extension", "extent",
"exterior", "external", "extinction", "extreme", "extremist", "eye", "eyeball", "eyebrow", "eyebrows", "eyeglasses", "eyelash", "eyelashes",
"eyelid", "eyelids", "eyeliner", "eyestrain", "eyrie", "fabric", "face", "facelift", "facet", "facility", "facsimile", "fact", "factor",
"factory", "faculty", "fahrenheit", "fail", "failure", "fairness", "fairy", "faith", "faithful", "fall", "fallacy", "fame",
"familiar", "familiarity", "family", "fan", "fang", "fanlight", "fanny", "fantasy", "farm", "farmer", "farming", "farmland",
"farrow", "fascia", "fashion", "fat", "fate", "father", "fatigue", "fatigues", "faucet", "fault", "fav", "fava", "favor",
"favorite", "fawn", "fax", "fear", "feast", "feather", "feature", "fedelini", "federation", "fedora", "fee", "feed", "feedback", "feeding",
"feel", "feeling", "fellow", "felony", "female", "fen", "fence", "fencing", "fender", "feng", "fennel", "ferret", "ferry", "ferryboat",
"fertilizer", "festival", "fetus", "few", "fiber", "fiberglass", "fibre", "fibroblast", "fibrosis", "ficlet", "fiction", "fiddle", "field",
"fiery", "fiesta", "fifth", "fig", "fight", "fighter", "figure", "figurine", "file", "filing", "fill", "fillet", "filly", "film", "filter",
"filth", "final", "finance", "financing", "finding", "fine", "finer", "finger", "fingerling", "fingernail", "finish", "finisher", "fir",
"fire", "fireman", "fireplace", "firewall", "firm", "first", "fish", "fishbone", "fisherman", "fishery", "fishing", "fishmonger", "fishnet",
"fisting", "fit", "fitness", "fix", "fixture", "flag", "flair", "flame", "flan", "flanker", "flare", "flash", "flat", "flatboat", "flavor",
"flax", "fleck", "fledgling", "fleece", "flesh", "flexibility", "flick", "flicker", "flight", "flint", "flintlock", "flock",
"flood", "floodplain", "floor", "floozie", "flour", "flow", "flower", "flu", "flugelhorn", "fluke", "flume", "flung", "flute", "fly",
"flytrap", "foal", "foam", "fob", "focus", "fog", "fold", "folder", "folk", "folklore", "follower", "following", "fondue", "font", "food",
"foodstuffs", "fool", "foot", "footage", "football", "footnote", "footprint", "footrest", "footstep", "footstool", "footwear", "forage",
"forager", "foray", "force", "ford", "forearm", "forebear", "forecast", "forehead", "foreigner", "forelimb", "forest", "forestry", "forever",
"forgery", "fork", "form", "formal", "formamide", "format", "formation", "former", "formicarium", "formula", "fort", "forte", "fortnight",
"fortress", "fortune", "forum", "foundation", "founder", "founding", "fountain", "fourths", "fowl", "fox", "foxglove", "fraction", "fragrance",
"frame", "framework", "fratricide", "fraud", "fraudster", "freak", "freckle", "freedom", "freelance", "freezer", "freezing", "freight",
"freighter", "frenzy", "freon", "frequency", "fresco", "friction", "fridge", "friend", "friendship", "fries", "frigate", "fright", "fringe",
"fritter", "frock", "frog", "front", "frontier", "frost", "frosting", "frown", "fruit", "frustration", "fry", "fuel", "fugato",
"fulfillment", "full", "fun", "function", "functionality", "fund", "funding", "fundraising", "funeral", "fur", "furnace", "furniture",
"furry", "fusarium", "futon", "future", "gadget", "gaffe", "gaffer", "gain", "gaiters", "gale", "gallery", "galley",
"gallon", "galoshes", "gambling", "game", "gamebird", "gaming", "gander", "gang", "gap", "garage", "garb", "garbage", "garden",
"garlic", "garment", "garter", "gas", "gasket", "gasoline", "gasp", "gastronomy", "gastropod", "gate", "gateway", "gather", "gathering",
"gator", "gauge", "gauntlet", "gavel", "gazebo", "gazelle", "gear", "gearshift", "geek", "gel", "gelatin", "gelding", "gem", "gemsbok",
"gender", "gene", "general", "generation", "generator", "generosity", "genetics", "genie", "genius", "genocide", "genre", "gentleman",
"geography", "geology", "geometry", "geranium", "gerbil", "gesture", "geyser", "gherkin", "ghost", "giant", "gift", "gig", "gigantism",
"giggle", "ginger", "gingerbread", "ginseng", "giraffe", "girdle", "girl", "girlfriend", "git", "glacier", "gladiolus", "glance", "gland",
"glass", "glasses", "glee", "glen", "glider", "gliding", "glimpse", "globe", "glockenspiel", "gloom", "glory", "glove", "glow", "glucose",
"glue", "glut", "glutamate", "gnat", "gnu", "goal", "goat", "gobbler", "god", "goddess", "godfather", "godmother", "godparent",
"goggles", "going", "gold", "goldfish", "golf", "gondola", "gong", "good", "goodbye", "goodie", "goodness", "goodnight",
"goodwill", "goose", "gopher", "gorilla", "gosling", "gossip", "governance", "government", "governor", "gown", "grace", "grade",
"gradient", "graduate", "graduation", "graffiti", "graft", "grain", "gram", "grammar", "gran", "grand", "grandchild", "granddaughter",
"grandfather", "grandma", "grandmom", "grandmother", "grandpa", "grandparent", "grandson", "granny", "granola", "grant", "grape", "grapefruit",
"graph", "graphic", "grasp", "grass", "grasshopper", "grassland", "gratitude", "gravel", "gravitas", "gravity", "gravy", "gray", "grease",
"greatness", "greed", "green", "greenhouse", "greens", "grenade", "grey", "grid", "grief",
"grill", "grin", "grip", "gripper", "grit", "grocery", "ground", "group", "grouper", "grouse", "grove", "growth", "grub", "guacamole",
"guarantee", "guard", "guava", "guerrilla", "guess", "guest", "guestbook", "guidance", "guide", "guideline", "guilder", "guilt", "guilty",
"guinea", "guitar", "guitarist", "gum", "gumshoe", "gun", "gunpowder", "gutter", "guy", "gym", "gymnast", "gymnastics", "gynaecology",
"gyro", "habit", "habitat", "hacienda", "hacksaw", "hackwork", "hail", "hair", "haircut", "hake", "half",
"halibut", "hall", "halloween", "hallway", "halt", "ham", "hamburger", "hammer", "hammock", "hamster", "hand", "handball",
"handful", "handgun", "handicap", "handle", "handlebar", "handmaiden", "handover", "handrail", "handsaw", "hanger", "happening", "happiness",
"harald", "harbor", "harbour", "hardboard", "hardcover", "hardening", "hardhat", "hardship", "hardware", "hare", "harm",
"harmonica", "harmonise", "harmonize", "harmony", "harp", "harpooner", "harpsichord", "harvest", "harvester", "hash", "hashtag", "hassock",
"haste", "hat", "hatbox", "hatchet", "hatchling", "hate", "hatred", "haunt", "haven", "haversack", "havoc", "hawk", "hay", "haze", "hazel",
"hazelnut", "head", "headache", "headlight", "headline", "headphones", "headquarters", "headrest", "health", "hearing",
"hearsay", "heart", "heartache", "heartbeat", "hearth", "hearthside", "heartwood", "heat", "heater", "heating", "heaven",
"heavy", "hectare", "hedge", "hedgehog", "heel", "heifer", "height", "heir", "heirloom", "helicopter", "helium", "hell", "hellcat", "hello",
"helmet", "helo", "help", "hemisphere", "hemp", "hen", "hepatitis", "herb", "herbs", "heritage", "hermit", "hero", "heroine", "heron",
"herring", "hesitation", "hexagon", "heyday", "hiccups", "hide", "hierarchy", "high", "highland", "highlight",
"highway", "hike", "hiking", "hill", "hint", "hip", "hippodrome", "hippopotamus", "hire", "hiring", "historian", "history", "hit", "hive",
"hobbit", "hobby", "hockey", "hoe", "hog", "hold", "holder", "hole", "holiday", "home", "homeland", "homeownership", "hometown", "homework",
"homicide", "homogenate", "homonym", "honesty", "honey", "honeybee", "honeydew", "honor", "honoree", "hood",
"hoof", "hook", "hop", "hope", "hops", "horde", "horizon", "hormone", "horn", "hornet", "horror", "horse", "horseradish", "horst", "hose",
"hosiery", "hospice", "hospital", "hospitalisation", "hospitality", "hospitalization", "host", "hostel", "hostess", "hotdog", "hotel",
"hound", "hour", "hourglass", "house", "houseboat", "household", "housewife", "housework", "housing", "hovel", "hovercraft", "howard",
"howitzer", "hub", "hubcap", "hubris", "hug", "hugger", "hull", "human", "humanity", "humidity", "hummus", "humor", "humour", "hunchback",
"hundred", "hunger", "hunt", "hunter", "hunting", "hurdle", "hurdler", "hurricane", "hurry", "hurt", "husband", "hut", "hutch", "hyacinth",
"hybridisation", "hybridization", "hydrant", "hydraulics", "hydrocarb", "hydrocarbon", "hydrofoil", "hydrogen", "hydrolyse", "hydrolysis",
"hydrolyze", "hydroxyl", "hyena", "hygienic", "hype", "hyphenation", "hypochondria", "hypothermia", "hypothesis", "ice",
"iceberg", "icebreaker", "icecream", "icicle", "icing", "icon", "icy", "id", "idea", "ideal", "identification", "identity", "ideology",
"idiom", "idiot", "igloo", "ignorance", "ignorant", "ikebana", "illegal", "illiteracy", "illness", "illusion", "illustration", "image",
"imagination", "imbalance", "imitation", "immigrant", "immigration", "immortal", "impact", "impairment", "impala", "impediment", "implement",
"implementation", "implication", "import", "importance", "impostor", "impress", "impression", "imprisonment", "impropriety", "improvement",
"impudence", "impulse", "inability", "inauguration", "inbox", "incandescence", "incarnation", "incense", "incentive",
"inch", "incidence", "incident", "incision", "inclusion", "income", "incompetence", "inconvenience", "increase", "incubation", "independence",
"independent", "index", "indication", "indicator", "indigence", "individual", "industrialisation", "industrialization", "industry", "inequality",
"inevitable", "infancy", "infant", "infarction", "infection", "infiltration", "infinite", "infix", "inflammation", "inflation", "influence",
"influx", "info", "information", "infrastructure", "infusion", "inglenook", "ingrate", "ingredient", "inhabitant", "inheritance", "inhibition",
"inhibitor", "initial", "initialise", "initialize", "initiative", "injunction", "injury", "injustice", "ink", "inlay", "inn", "innervation",
"innocence", "innocent", "innovation", "input", "inquiry", "inscription", "insect", "insectarium", "insert", "inside", "insight", "insolence",
"insomnia", "inspection", "inspector", "inspiration", "installation", "instance", "instant", "instinct", "institute", "institution",
"instruction", "instructor", "instrument", "instrumentalist", "instrumentation", "insulation", "insurance", "insurgence", "insurrection",
"integer", "integral", "integration", "integrity", "intellect", "intelligence", "intensity", "intent", "intention", "intentionality",
"interaction", "interchange", "interconnection", "intercourse", "interest", "interface", "interferometer", "interior", "interject", "interloper",
"internet", "interpretation", "interpreter", "interval", "intervenor", "intervention", "interview", "interviewer", "intestine", "introduction",
"intuition", "invader", "invasion", "invention", "inventor", "inventory", "inverse", "inversion", "investigation", "investigator", "investment",
"investor", "invitation", "invite", "invoice", "involvement", "iridescence", "iris", "iron", "ironclad", "irony", "irrigation", "ischemia",
"island", "isogloss", "isolation", "issue", "item", "itinerary", "ivory", "jack", "jackal", "jacket", "jackfruit", "jade", "jaguar",
"jail", "jailhouse", "jalapeño", "jam", "jar", "jasmine", "jaw", "jazz", "jealousy", "jeans", "jeep", "jelly", "jellybeans", "jellyfish",
"jerk", "jet", "jewel", "jeweller", "jewellery", "jewelry", "jicama", "jiffy", "job", "jockey", "jodhpurs", "joey", "jogging", "joint",
"joke", "jot", "journal", "journalism", "journalist", "journey", "joy", "judge", "judgment", "judo", "jug", "juggernaut", "juice", "julienne",
"jumbo", "jump", "jumper", "jumpsuit", "jungle", "junior", "junk", "junker", "junket", "jury", "justice", "justification", "jute", "kale",
"kamikaze", "kangaroo", "karate", "kayak", "kazoo", "kebab", "keep", "keeper", "kendo", "kennel", "ketch", "ketchup", "kettle", "kettledrum",
"key", "keyboard", "keyboarding", "keystone", "kick", "kid", "kidney", "kielbasa", "kill", "killer", "killing", "kilogram",
"kilometer", "kilt", "kimono", "kinase", "kind", "kindness", "king", "kingdom", "kingfish", "kiosk", "kiss", "kit", "kitchen", "kite",
"kitsch", "kitten", "kitty", "kiwi", "knee", "kneejerk", "knickers", "knife", "knight", "knitting", "knock", "knot",
"knowledge", "knuckle", "koala", "kohlrabi", "kumquat", "lab", "label", "labor", "laboratory", "laborer", "labour", "labourer", "lace",
"lack", "lacquerware", "lad", "ladder", "ladle", "lady", "ladybug", "lag", "lake", "lamb", "lambkin", "lament", "lamp", "lanai", "land",
"landform", "landing", "landmine", "landscape", "lane", "language", "lantern", "lap", "laparoscope", "lapdog", "laptop", "larch", "lard",
"larder", "lark", "larva", "laryngitis", "lasagna", "lashes", "last", "latency", "latex", "lathe", "latitude", "latte", "latter", "laugh",
"laughter", "laundry", "lava", "law", "lawmaker", "lawn", "lawsuit", "lawyer", "lay", "layer", "layout", "lead", "leader", "leadership",
"leading", "leaf", "league", "leaker", "leap", "learning", "leash", "leather", "leave", "leaver", "lecture", "leek", "leeway", "left",
"leg", "legacy", "legal", "legend", "legging", "legislation", "legislator", "legislature", "legitimacy", "legume", "leisure", "lemon",
"lemonade", "lemur", "lender", "lending", "length", "lens", "lentil", "leopard", "leprosy", "leptocephalus", "lesson", "letter",
"lettuce", "level", "lever", "leverage", "leveret", "liability", "liar", "liberty", "libido", "library", "licence", "license", "licensing",
"licorice", "lid", "lie", "lieu", "lieutenant", "life", "lifestyle", "lifetime", "lift", "ligand", "light", "lighting", "lightning",
"lightscreen", "ligula", "likelihood", "likeness", "lilac", "lily", "limb", "lime", "limestone", "limit", "limitation", "limo", "line",
"linen", "liner", "linguist", "linguistics", "lining", "link", "linkage", "linseed", "lion", "lip", "lipid", "lipoprotein", "lipstick",
"liquid", "liquidity", "liquor", "list", "listening", "listing", "literate", "literature", "litigation", "litmus", "litter", "littleneck",
"liver", "livestock", "living", "lizard", "llama", "load", "loading", "loaf", "loafer", "loan", "lobby", "lobotomy", "lobster", "local",
"locality", "location", "lock", "locker", "locket", "locomotive", "locust", "lode", "loft", "log", "loggia", "logic", "login", "logistics",
"logo", "loincloth", "lollipop", "loneliness", "longboat", "longitude", "look", "lookout", "loop", "loophole", "loquat", "lord", "loss",
"lot", "lotion", "lottery", "lounge", "louse", "lout", "love", "lover", "lox", "loyalty", "luck", "luggage", "lumber", "lumberman", "lunch",
"luncheonette", "lunchmeat", "lunchroom", "lung", "lunge", "lust", "lute", "luxury", "lychee", "lycra", "lye", "lymphocyte", "lynx",
"lyocell", "lyre", "lyrics", "lysine", "mRNA", "macadamia", "macaroni", "macaroon", "macaw", "machine", "machinery", "macrame", "macro",
"macrofauna", "madam", "maelstrom", "maestro", "magazine", "maggot", "magic", "magnet", "magnitude", "maid", "maiden", "mail", "mailbox",
"mailer", "mailing", "mailman", "main", "mainland", "mainstream", "maintainer", "maintenance", "maize", "major", "majority",
"makeover", "maker", "makeup", "making", "male", "malice", "mall", "mallard", "mallet", "malnutrition", "mama", "mambo", "mammoth", "man",
"manacle", "management", "manager", "manatee", "mandarin", "mandate", "mandolin", "mangle", "mango", "mangrove", "manhunt", "maniac",
"manicure", "manifestation", "manipulation", "mankind", "manner", "manor", "mansard", "manservant", "mansion", "mantel", "mantle", "mantua",
"manufacturer", "manufacturing", "many", "map", "maple", "mapping", "maracas", "marathon", "marble", "march", "mare", "margarine", "margin",
"mariachi", "marimba", "marines", "marionberry", "mark", "marker", "market", "marketer", "marketing", "marketplace", "marksman", "markup",
"marmalade", "marriage", "marsh", "marshland", "marshmallow", "marten", "marxism", "mascara", "mask", "masonry", "mass", "massage", "mast",
"master", "masterpiece", "mastication", "mastoid", "mat", "match", "matchmaker", "mate", "material", "maternity", "math", "mathematics",
"matrix", "matter", "mattock", "mattress", "max", "maximum", "maybe", "mayonnaise", "mayor", "meadow", "meal", "mean", "meander", "meaning",
"means", "meantime", "measles", "measure", "measurement", "meat", "meatball", "meatloaf", "mecca", "mechanic", "mechanism", "med", "medal",
"media", "median", "medication", "medicine", "medium", "meet", "meeting", "melatonin", "melody", "melon", "member", "membership", "membrane",
"meme", "memo", "memorial", "memory", "men", "menopause", "menorah", "mention", "mentor", "menu", "merchandise", "merchant", "mercury",
"meridian", "meringue", "merit", "mesenchyme", "mess", "message", "messenger", "messy", "metabolite", "metal", "metallurgist", "metaphor",
"meteor", "meteorology", "meter", "methane", "method", "methodology", "metric", "metro", "metronome", "mezzanine", "microlending", "micronutrient",
"microphone", "microwave", "midden", "middle", "middleman", "midline", "midnight", "midwife", "might", "migrant", "migration",
"mile", "mileage", "milepost", "milestone", "military", "milk", "milkshake", "mill", "millennium", "millet", "millimeter", "million",
"millisecond", "millstone", "mime", "mimosa", "min", "mincemeat", "mind", "mine", "mineral", "mineshaft", "mini", "minibus",
"minimalism", "minimum", "mining", "minion", "minister", "mink", "minnow", "minor", "minority", "mint", "minute", "miracle",
"mirror", "miscarriage", "miscommunication", "misfit", "misnomer", "misogyny", "misplacement", "misreading", "misrepresentation", "miss",
"missile", "mission", "missionary", "mist", "mistake", "mister", "misunderstand", "miter", "mitten", "mix", "mixer", "mixture", "moai",
"moat", "mob", "mobile", "mobility", "mobster", "moccasins", "mocha", "mochi", "mode", "model", "modeling", "modem", "modernist", "modernity",
"modification", "molar", "molasses", "molding", "mole", "molecule", "mom", "moment", "monastery", "monasticism", "money", "monger", "monitor",
"monitoring", "monk", "monkey", "monocle", "monopoly", "monotheism", "monsoon", "monster", "month", "monument", "mood", "moody", "moon",
"moonlight", "moonscape", "moonshine", "moose", "mop", "morale", "morbid", "morbidity", "morning", "moron", "morphology", "morsel", "mortal",
"mortality", "mortgage", "mortise", "mosque", "mosquito", "most", "motel", "moth", "mother", "motion", "motivation",
"motive", "motor", "motorboat", "motorcar", "motorcycle", "mound", "mountain", "mouse", "mouser", "mousse", "moustache", "mouth", "mouton",
"movement", "mover", "movie", "mower", "mozzarella", "mud", "muffin", "mug", "mukluk", "mule", "multimedia", "murder", "muscat", "muscatel",
"muscle", "musculature", "museum", "mushroom", "music", "musician", "muskrat", "mussel", "mustache", "mustard",
"mutation", "mutt", "mutton", "mycoplasma", "mystery", "myth", "mythology", "nail", "name", "naming", "nanoparticle", "napkin", "narrative",
"nasal", "nation", "nationality", "native", "naturalisation", "nature", "navigation", "necessity", "neck", "necklace", "necktie", "nectar",
"nectarine", "need", "needle", "neglect", "negligee", "negotiation", "neighbor", "neighborhood", "neighbour", "neighbourhood", "neologism",
"neon", "neonate", "nephew", "nerve", "nest", "nestling", "nestmate", "net", "netball", "netbook", "netsuke", "network", "networking",
"neurobiologist", "neuron", "neuropathologist", "neuropsychiatry", "news", "newsletter", "newspaper", "newsprint", "newsstand", "nexus",
"nibble", "nicety", "niche", "nick", "nickel", "nickname", "niece", "night", "nightclub", "nightgown", "nightingale", "nightlife", "nightlight",
"nightmare", "ninja", "nit", "nitrogen", "nobody", "nod", "node", "noir", "noise", "nonbeliever", "nonconformist", "nondisclosure", "nonsense",
"noodle", "noodles", "noon", "norm", "normal", "normalisation", "normalization", "north", "nose", "notation", "note", "notebook", "notepad",
"nothing", "notice", "notion", "notoriety", "nougat", "noun", "nourishment", "novel", "nucleotidase", "nucleotide", "nudge", "nuke",
"number", "numeracy", "numeric", "numismatist", "nun", "nurse", "nursery", "nursing", "nurture", "nut", "nutmeg", "nutrient", "nutrition",
"nylon", "nymph", "oak", "oar", "oasis", "oat", "oatmeal", "oats", "obedience", "obesity", "obi", "object", "objection", "objective",
"obligation", "oboe", "observation", "observatory", "obsession", "obsidian", "obstacle", "occasion", "occupation", "occurrence", "ocean",
"ocelot", "octagon", "octave", "octavo", "octet", "octopus", "odometer", "odyssey", "oeuvre", "offence", "offense", "offer",
"offering", "office", "officer", "official", "offset", "oil", "okra", "oldie", "oleo", "olive", "omega", "omelet", "omission", "omnivore",
"oncology", "onion", "online", "onset", "opening", "opera", "operating", "operation", "operator", "ophthalmologist", "opinion", "opium",
"opossum", "opponent", "opportunist", "opportunity", "opposite", "opposition", "optimal", "optimisation", "optimist", "optimization",
"option", "orange", "orangutan", "orator", "orchard", "orchestra", "orchid", "order", "ordinary", "ordination", "ore", "oregano", "organ",
"organisation", "organising", "organization", "organizing", "orient", "orientation", "origin", "original", "originality", "ornament",
"osmosis", "osprey", "ostrich", "other", "otter", "ottoman", "ounce", "outback", "outcome", "outfielder", "outfit", "outhouse", "outlaw",
"outlay", "outlet", "outline", "outlook", "output", "outrage", "outrigger", "outrun", "outset", "outside", "oval", "ovary", "oven", "overcharge",
"overclocking", "overcoat", "overexertion", "overflight", "overhead", "overheard", "overload", "overnighter", "overshoot", "oversight",
"overview", "overweight", "owl", "owner", "ownership", "ox", "oxford", "oxygen", "oyster", "ozone", "pace", "pacemaker", "pack", "package",
"packaging", "packet", "pad", "paddle", "paddock", "pagan", "page", "pagoda", "pail", "pain", "paint", "painter", "painting", "paintwork",
"pair", "pajamas", "palace", "palate", "palm", "pamphlet", "pan", "pancake", "pancreas", "panda", "panel", "panic", "pannier", "panpipe",
"pansy", "panther", "panties", "pantologist", "pantology", "pantry", "pants", "pantsuit", "panty", "pantyhose", "papa", "papaya", "paper",
"paperback", "paperwork", "parable", "parachute", "parade", "paradise", "paragraph", "parallelogram", "paramecium", "paramedic", "parameter",
"paranoia", "parcel", "parchment", "pard", "pardon", "parent", "parenthesis", "parenting", "park", "parka", "parking", "parliament",
"parole", "parrot", "parser", "parsley", "parsnip", "part", "participant", "participation", "particle", "particular", "partner", "partnership",
"partridge", "party", "pass", "passage", "passbook", "passenger", "passing", "passion", "passive", "passport", "password", "past", "pasta",
"paste", "pastor", "pastoralist", "pastry", "pasture", "pat", "patch", "pate", "patent", "patentee", "path", "pathogenesis", "pathology",
"pathway", "patience", "patient", "patina", "patio", "patriarch", "patrimony", "patriot", "patrol", "patroller", "patrolling", "patron",
"pattern", "patty", "pattypan", "pause", "pavement", "pavilion", "paw", "pawnshop", "pay", "payee", "payment", "payoff", "pea", "peace",
"peach", "peacoat", "peacock", "peak", "peanut", "pear", "pearl", "peasant", "pecan", "pecker", "pedal", "peek", "peen", "peer",
"pegboard", "pelican", "pelt", "pen", "penalty", "pence", "pencil", "pendant", "pendulum", "penguin", "penicillin", "peninsula", "penis",
"pennant", "penny", "pension", "pentagon", "peony", "people", "pepper", "pepperoni", "percent", "percentage", "perception", "perch",
"perennial", "perfection", "performance", "perfume", "period", "periodical", "peripheral", "permafrost", "permission", "permit", "perp",
"perpendicular", "persimmon", "person", "personal", "personality", "personnel", "perspective", "pest", "pet", "petal", "petition", "petitioner",
"petticoat", "pew", "pharmacist", "pharmacopoeia", "phase", "pheasant", "phenomenon", "phenotype", "pheromone", "philanthropy", "philosopher",
"philosophy", "phone", "phosphate", "photo", "photodiode", "photograph", "photographer", "photography", "photoreceptor", "phrase", "phrasing",
"physical", "physics", "physiology", "pianist", "piano", "piccolo", "pick", "pickax", "pickaxe", "picket", "pickle", "pickup", "picnic",
"picture", "picturesque", "pie", "piece", "pier", "piety", "pig", "pigeon", "piglet", "pigpen", "pigsty", "pike", "pilaf", "pile", "pilgrim",
"pilgrimage", "pill", "pillar", "pillbox", "pillow", "pilot", "pimp", "pimple", "pin", "pinafore", "pine", "pineapple",
"pinecone", "ping", "pink", "pinkie", "pinot", "pinstripe", "pint", "pinto", "pinworm", "pioneer", "pipe", "pipeline", "piracy", "pirate",
"pistol", "pit", "pita", "pitch", "pitcher", "pitching", "pith", "pizza", "place", "placebo", "placement", "placode", "plagiarism",
"plain", "plaintiff", "plan", "plane", "planet", "planning", "plant", "plantation", "planter", "planula", "plaster", "plasterboard",
"plastic", "plate", "platelet", "platform", "platinum", "platter", "platypus", "play", "player", "playground", "playroom", "playwright",
"plea", "pleasure", "pleat", "pledge", "plenty", "plier", "pliers", "plight", "plot", "plough", "plover", "plow", "plowman", "plug",
"plugin", "plum", "plumber", "plume", "plunger", "plywood", "pneumonia", "pocket", "pocketbook", "pod", "podcast", "poem",
"poet", "poetry", "poignance", "point", "poison", "poisoning", "poker", "polarisation", "polarization", "pole", "polenta", "police",
"policeman", "policy", "polish", "politician", "politics", "poll", "polliwog", "pollutant", "pollution", "polo", "polyester", "polyp",
"pomegranate", "pomelo", "pompom", "poncho", "pond", "pony", "pool", "poor", "pop", "popcorn", "poppy", "popsicle", "popularity", "population",
"populist", "porcelain", "porch", "porcupine", "pork", "porpoise", "port", "porter", "portfolio", "porthole", "portion", "portrait",
"position", "possession", "possibility", "possible", "post", "postage", "postbox", "poster", "posterior", "postfix", "pot", "potato",
"potential", "pottery", "potty", "pouch", "poultry", "pound", "pounding", "poverty", "powder", "power", "practice", "practitioner", "prairie",
"praise", "pray", "prayer", "precedence", "precedent", "precipitation", "precision", "predecessor", "preface", "preference", "prefix",
"pregnancy", "prejudice", "prelude", "premeditation", "premier", "premise", "premium", "preoccupation", "preparation", "prescription",
"presence", "present", "presentation", "preservation", "preserves", "presidency", "president", "press", "pressroom", "pressure", "pressurisation",
"pressurization", "prestige", "presume", "pretzel", "prevalence", "prevention", "prey", "price", "pricing", "pride", "priest", "priesthood",
"primary", "primate", "prince", "princess", "principal", "principle", "print", "printer", "printing", "prior", "priority", "prison",
"prisoner", "privacy", "private", "privilege", "prize", "prizefight", "probability", "probation", "probe", "problem", "procedure", "proceedings",
"process", "processing", "processor", "proctor", "procurement", "produce", "producer", "product", "production", "productivity", "profession",
"professional", "professor", "profile", "profit", "progenitor", "program", "programme", "programming", "progress", "progression", "prohibition",
"project", "proliferation", "promenade", "promise", "promotion", "prompt", "pronoun", "pronunciation", "proof", "propaganda",
"propane", "property", "prophet", "proponent", "proportion", "proposal", "proposition", "proprietor", "prose", "prosecution", "prosecutor",
"prospect", "prosperity", "prostacyclin", "prostanoid", "prostrate", "protection", "protein", "protest", "protocol", "providence", "provider",
"province", "provision", "prow", "proximal", "proximity", "prune", "pruner", "pseudocode", "pseudoscience", "psychiatrist", "psychoanalyst",
"psychologist", "psychology", "ptarmigan", "pub", "public", "publication", "publicity", "publisher", "publishing", "pudding", "puddle",
"puffin", "pug", "puggle", "pulley", "pulse", "puma", "pump", "pumpernickel", "pumpkin", "pumpkinseed", "pun", "punch", "punctuation",
"punishment", "pup", "pupa", "pupil", "puppet", "puppy", "purchase", "puritan", "purity", "purple", "purpose", "purr", "purse", "pursuit",
"push", "pusher", "put", "puzzle", "pyramid", "pyridine", "quadrant", "quail", "qualification", "quality", "quantity", "quart", "quarter",
"quartet", "quartz", "queen", "query", "quest", "question", "questioner", "questionnaire", "quiche", "quicksand", "quiet", "quill", "quilt",
"quince", "quinoa", "quit", "quiver", "quota", "quotation", "quote", "rabbi", "rabbit", "raccoon", "race", "racer", "racing", "racism",
"racist", "rack", "radar", "radiator", "radio", "radiosonde", "radish", "raffle", "raft", "rag", "rage", "raid", "rail", "railing", "railroad",
"railway", "raiment", "rain", "rainbow", "raincoat", "rainmaker", "rainstorm", "rainy", "raise", "raisin", "rake", "rally", "ram", "rambler",
"ramen", "ramie", "ranch", "rancher", "randomisation", "randomization", "range", "ranger", "rank", "rap", "rape", "raspberry", "rat",
"rate", "ratepayer", "rating", "ratio", "rationale", "rations", "raven", "ravioli", "rawhide", "ray", "rayon", "razor", "reach", "reactant",
"reaction", "read", "reader", "readiness", "reading", "real", "reality", "realization", "realm", "reamer", "rear", "reason", "reasoning",
"rebel", "rebellion", "reboot", "recall", "recapitulation", "receipt", "receiver", "reception", "receptor", "recess", "recession", "recipe",
"recipient", "reciprocity", "reclamation", "recliner", "recognition", "recollection", "recommendation", "reconsideration", "record",
"recorder", "recording", "recovery", "recreation", "recruit", "rectangle", "red", "redesign", "redhead", "redirect", "rediscovery", "reduction",
"reef", "refectory", "reference", "referendum", "reflection", "reform", "refreshments", "refrigerator", "refuge", "refund", "refusal",
"refuse", "regard", "regime", "region", "regionalism", "register", "registration", "registry", "regret", "regulation", "regulator",
"rehospitalization", "reindeer", "reinscription", "reject", "relation", "relationship", "relative", "relaxation", "relay", "release",
"reliability", "relief", "religion", "relish", "reluctance", "remains", "remark", "reminder", "remnant", "remote", "removal", "renaissance",
"rent", "reorganisation", "reorganization", "repair", "reparation", "repayment", "repeat", "replacement", "replica", "replication", "reply",
"report", "reporter", "reporting", "repository", "representation", "representative", "reprocessing", "republic", "republican", "reputation",
"request", "requirement", "resale", "rescue", "research", "researcher", "resemblance", "reservation", "reserve", "reservoir", "reset",
"residence", "resident", "residue", "resist", "resistance", "resolution", "resolve", "resort", "resource", "respect", "respite", "response",
"responsibility", "rest", "restaurant", "restoration", "restriction", "restroom", "restructuring", "result", "resume", "retailer", "retention",
"rethinking", "retina", "retirement", "retouching", "retreat", "retrospect", "retrospective", "retrospectivity", "return", "reunion",
"revascularisation", "revascularization", "reveal", "revelation", "revenant", "revenge", "revenue", "reversal", "reverse", "review",
"revitalisation", "revitalization", "revival", "revolution", "revolver", "reward", "rhetoric", "rheumatism", "rhinoceros", "rhubarb",
"rhyme", "rhythm", "rib", "ribbon", "rice", "riddle", "ride", "rider", "ridge", "riding", "rifle", "right", "rim", "ring", "ringworm",
"riot", "rip", "ripple", "rise", "riser", "risk", "rite", "ritual", "river", "riverbed", "rivulet", "road", "roadway", "roar", "roast",
"robe", "robin", "robot", "robotics", "rock", "rocker", "rocket", "rod", "role", "roll", "roller", "romaine", "romance",
"roof", "room", "roommate", "rooster", "root", "rope", "rose", "rosemary", "roster", "rostrum", "rotation", "round", "roundabout", "route",
"router", "routine", "row", "rowboat", "rowing", "rubber", "rubric", "ruby", "ruckus", "rudiment", "ruffle", "rug", "rugby",
"ruin", "rule", "ruler", "ruling", "rum", "rumor", "run", "runaway", "runner", "running", "runway", "rush", "rust", "rutabaga", "rye",
"sabre", "sac", "sack", "saddle", "sadness", "safari", "safe", "safeguard", "safety", "saffron", "sage", "sail", "sailboat", "sailing",
"sailor", "saint", "sake", "salad", "salami", "salary", "sale", "salesman", "salmon", "salon", "saloon", "salsa", "salt", "salute", "samovar",
"sampan", "sample", "samurai", "sanction", "sanctity", "sanctuary", "sand", "sandal", "sandbar", "sandpaper", "sandwich", "sanity", "sardine",
"sari", "sarong", "sash", "satellite", "satin", "satire", "satisfaction", "sauce", "saucer", "sauerkraut", "sausage", "savage", "savannah",
"saving", "savings", "savior", "saviour", "savory", "saw", "saxophone", "scaffold", "scale", "scallion", "scallops", "scalp", "scam",
"scanner", "scarecrow", "scarf", "scarification", "scenario", "scene", "scenery", "scent", "schedule", "scheduling", "schema", "scheme",
"schizophrenic", "schnitzel", "scholar", "scholarship", "school", "schoolhouse", "schooner", "science", "scientist", "scimitar", "scissors",
"scooter", "scope", "score", "scorn", "scorpion", "scotch", "scout", "scow", "scrambled", "scrap", "scraper", "scratch", "screamer",
"screen", "screening", "screenwriting", "screw", "screwdriver", "scrim", "scrip", "script", "scripture", "scrutiny", "sculpting",
"sculptural", "sculpture", "sea", "seabass", "seafood", "seagull", "seal", "seaplane", "search", "seashore", "seaside", "season", "seat",
"seaweed", "second", "secrecy", "secret", "secretariat", "secretary", "secretion", "section", "sectional", "sector", "security", "sediment",
"seed", "seeder", "seeker", "seep", "segment", "seizure", "selection", "self", "seller",
"selling", "semantics", "semester", "semicircle", "semicolon", "semiconductor", "seminar", "senate", "senator", "sender", "senior", "sense",
"sensibility", "sensitive", "sensitivity", "sensor", "sentence", "sentencing", "sentiment", "sepal", "separation", "septicaemia", "sequel",
"sequence", "serial", "series", "sermon", "serum", "serval", "servant", "server", "service", "servitude", "sesame", "session", "set",
"setback", "setting", "settlement", "settler", "severity", "sewer", "sex", "sexuality", "shack", "shackle", "shade", "shadow", "shadowbox",
"shakedown", "shaker", "shallot", "shallows", "shame", "shampoo", "shanty", "shape", "share", "shareholder", "shark", "shaw", "shawl",
"shear", "shearling", "sheath", "shed", "sheep", "sheet", "shelf", "shell", "shelter", "sherbet", "sherry", "shield", "shift", "shin",
"shine", "shingle", "ship", "shipper", "shipping", "shipyard", "shirt", "shirtdress", "shoat", "shock", "shoe",
"shoehorn", "shoelace", "shoemaker", "shoes", "shoestring", "shofar", "shoot", "shootdown", "shop", "shopper", "shopping", "shore", "shoreline",
"short", "shortage", "shorts", "shortwave", "shot", "shoulder", "shout", "shovel", "show", "shower", "shred", "shrimp",
"shrine", "shutdown", "sibling", "sick", "sickness", "side", "sideboard", "sideburns", "sidecar", "sidestream", "sidewalk", "siding",
"siege", "sigh", "sight", "sightseeing", "sign", "signal", "signature", "signet", "significance", "signify", "signup", "silence", "silica",
"silicon", "silk", "silkworm", "sill", "silly", "silo", "silver", "similarity", "simple", "simplicity", "simplification", "simvastatin",
"sin", "singer", "singing", "singular", "sink", "sinuosity", "sip", "sir", "sister", "sitar", "site", "situation", "size",
"skate", "skating", "skean", "skeleton", "ski", "skiing", "skill", "skin", "skirt", "skull", "skullcap", "skullduggery", "skunk", "sky",
"skylight", "skyline", "skyscraper", "skywalk", "slang", "slapstick", "slash", "slate", "slavery", "slaw", "sled", "sledge",
"sleep", "sleepiness", "sleeping", "sleet", "sleuth", "slice", "slide", "slider", "slime", "slip", "slipper", "slippers", "slope", "slot",
"sloth", "slump", "smell", "smelting", "smile", "smith", "smock", "smog", "smoke", "smoking", "smolt", "smuggling", "snack", "snail",
"snake", "snakebite", "snap", "snarl", "sneaker", "sneakers", "sneeze", "sniffle", "snob", "snorer", "snow", "snowboarding", "snowflake",
"snowman", "snowmobiling", "snowplow", "snowstorm", "snowsuit", "snuck", "snug", "snuggle", "soap", "soccer", "socialism", "socialist",
"society", "sociology", "sock", "socks", "soda", "sofa", "softball", "softdrink", "softening", "software", "soil", "soldier", "sole",
"solicitation", "solicitor", "solidarity", "solidity", "soliloquy", "solitaire", "solution", "solvency", "sombrero", "somebody", "someone",
"someplace", "somersault", "something", "somewhere", "son", "sonar", "sonata", "song", "songbird", "sonnet", "soot", "sophomore", "soprano",
"sorbet", "sorghum", "sorrel", "sorrow", "sort", "soul", "soulmate", "sound", "soundness", "soup", "source", "sourwood", "sousaphone",
"south", "southeast", "souvenir", "sovereignty", "sow", "soy", "soybean", "space", "spacing", "spade", "spaghetti", "span", "spandex",
"spank", "sparerib", "spark", "sparrow", "spasm", "spat", "spatula", "spawn", "speaker", "speakerphone", "speaking", "spear", "spec",
"special", "specialist", "specialty", "species", "specification", "spectacle", "spectacles", "spectrograph", "spectrum", "speculation",
"speech", "speed", "speedboat", "spell", "spelling", "spelt", "spending", "sphere", "sphynx", "spice", "spider", "spiderling", "spike",
"spill", "spinach", "spine", "spiral", "spirit", "spiritual", "spirituality", "spit", "spite", "spleen", "splendor", "split", "spokesman",
"spokeswoman", "sponge", "sponsor", "sponsorship", "spool", "spoon", "spork", "sport", "sportsman", "spot", "spotlight", "spouse", "sprag",
"sprat", "spray", "spread", "spreadsheet", "spree", "spring", "sprinkles", "sprinter", "sprout", "spruce", "spud", "spume", "spur", "spy",
"spyglass", "square", "squash", "squatter", "squeegee", "squid", "squirrel", "stab", "stability", "stable", "stack", "stacking", "stadium",
"staff", "stag", "stage", "stain", "stair", "staircase", "stake", "stalk", "stall", "stallion", "stamen", "stamina", "stamp", "stance",
"stand", "standard", "standardisation", "standardization", "standing", "standoff", "standpoint", "star", "starboard", "start", "starter",
"state", "statement", "statin", "station", "statistic", "statistics", "statue", "status", "statute", "stay", "steak",
"stealth", "steam", "steamroller", "steel", "steeple", "stem", "stench", "stencil", "step",
"stepdaughter", "stepmother",
"stepson", "stereo", "stew", "steward", "stick", "sticker", "stiletto", "still", "stimulation", "stimulus", "sting",
"stinger", "stitch", "stitcher", "stock", "stockings", "stole", "stomach", "stone", "stonework", "stool",
"stop", "stopsign", "stopwatch", "storage", "store", "storey", "storm", "story", "storyboard", "stot", "stove", "strait",
"strand", "stranger", "strap", "strategy", "straw", "strawberry", "strawman", "stream", "street", "streetcar", "strength", "stress",
"stretch", "strife", "strike", "string", "strip", "stripe", "strobe", "stroke", "structure", "strudel", "struggle", "stucco", "stud",
"student", "studio", "study", "stuff", "stumbling", "stump", "stupidity", "sturgeon", "sty", "style", "styling", "stylus", "sub", "subcomponent",
"subconscious", "subcontractor", "subexpression", "subgroup", "subject", "submarine", "submitter", "subprime", "subroutine", "subscription",
"subsection", "subset", "subsidence", "subsidiary", "subsidy", "substance", "substitution", "subtitle", "suburb", "subway", "success",
"succotash", "suck", "sucker", "suede", "suet", "suffocation", "sugar", "suggestion", "suicide", "suit", "suitcase", "suite", "sulfur",
"sultan", "sum", "summary", "summer", "summit", "sun", "sunbeam", "sunbonnet", "sundae", "sunday", "sundial", "sunflower", "sunglasses",
"sunlamp", "sunlight", "sunrise", "sunroom", "sunset", "sunshine", "superiority", "supermarket", "supernatural", "supervision", "supervisor",
"supper", "supplement", "supplier", "supply", "support", "supporter", "suppression", "supreme", "surface", "surfboard", "surge", "surgeon",
"surgery", "surname", "surplus", "surprise", "surround", "surroundings", "surrounds", "survey", "survival", "survivor", "sushi", "suspect",
"suspenders", "suspension", "sustainment", "sustenance", "swallow", "swamp", "swan", "swanling", "swath", "sweat", "sweater", "sweatshirt",
"sweatshop", "sweatsuit", "sweets", "swell", "swim", "swimming", "swimsuit", "swine", "swing", "switch", "switchboard", "switching",
"swivel", "sword", "swordfight", "swordfish", "sycamore", "symbol", "symmetry", "sympathy", "symptom", "syndicate", "syndrome", "synergy",
"synod", "synonym", "synthesis", "syrup", "system", "tab", "tabby", "tabernacle", "table", "tablecloth", "tablet", "tabletop",
"tachometer", "tackle", "taco", "tactics", "tactile", "tadpole", "tag", "tail", "tailbud", "tailor", "tailspin", "takeover",
"tale", "talent", "talk", "talking", "tamale", "tambour", "tambourine", "tan", "tandem", "tangerine", "tank",
"tanker", "tankful", "tap", "tape", "tapioca", "target", "taro", "tarragon", "tart", "task", "tassel", "taste", "tatami", "tattler",
"tattoo", "tavern", "tax", "taxi", "taxicab", "taxpayer", "tea", "teacher", "teaching", "team", "teammate", "teapot", "tear", "tech",
"technician", "technique", "technologist", "technology", "tectonics", "teen", "teenager", "teepee", "telephone", "telescreen", "teletype",
"television", "tell", "teller", "temp", "temper", "temperature", "temple", "tempo", "temporariness", "temporary", "temptation", "temptress",
"tenant", "tendency", "tender", "tenement", "tenet", "tennis", "tenor", "tension", "tensor", "tent", "tentacle", "tenth", "tepee", "teriyaki",
"term", "terminal", "termination", "terminology", "termite", "terrace", "terracotta", "terrapin", "terrarium", "territory", "terror",
"terrorism", "terrorist", "test", "testament", "testimonial", "testimony", "testing", "text", "textbook", "textual", "texture", "thanks",
"thaw", "theater", "theft", "theism", "theme", "theology", "theory", "therapist", "therapy", "thermals", "thermometer", "thermostat",
"thesis", "thickness", "thief", "thigh", "thing", "thinking", "thirst", "thistle", "thong", "thongs", "thorn", "thought", "thousand",
"thread", "threat", "threshold", "thrift", "thrill", "throat", "throne", "thrush", "thrust", "thug", "thumb", "thump", "thunder", "thunderbolt",
"thunderhead", "thunderstorm", "thyme", "tiara", "tic", "tick", "ticket", "tide", "tie", "tiger", "tights", "tile", "till", "tilt", "timbale",
"timber", "time", "timeline", "timeout", "timer", "timetable", "timing", "timpani", "tin", "tinderbox", "tinkle", "tintype", "tip", "tire",
"tissue", "titanium", "title", "toad", "toast", "toaster", "tobacco", "today", "toe", "toenail", "toffee", "tofu", "tog", "toga", "toilet",
"tolerance", "tolerant", "toll", "tomatillo", "tomato", "tomb", "tomography", "tomorrow", "ton", "tonality", "tone", "tongue",
"tonic", "tonight", "tool", "toot", "tooth", "toothbrush", "toothpaste", "toothpick", "top", "topic", "topsail", "toque",
"toreador", "tornado", "torso", "torte", "tortellini", "tortilla", "tortoise", "tosser", "total", "tote", "touch", "tour",
"tourism", "tourist", "tournament", "towel", "tower", "town", "townhouse", "township", "toy", "trace", "trachoma", "track",
"tracking", "tracksuit", "tract", "tractor", "trade", "trader", "trading", "tradition", "traditionalism", "traffic", "trafficker", "tragedy",
"trail", "trailer", "trailpatrol", "train", "trainer", "training", "trait", "tram", "tramp", "trance", "transaction", "transcript", "transfer",
"transformation", "transit", "transition", "translation", "transmission", "transom", "transparency", "transplantation", "transport",
"transportation", "trap", "trapdoor", "trapezium", "trapezoid", "trash", "travel", "traveler", "tray", "treasure", "treasury", "treat",
"treatment", "treaty", "tree", "trek", "trellis", "tremor", "trench", "trend", "triad", "trial", "triangle", "tribe", "tributary", "trick",
"trigger", "trigonometry", "trillion", "trim", "trinket", "trip", "tripod", "tritone", "triumph", "trolley", "trombone", "troop", "trooper",
"trophy", "trouble", "trousers", "trout", "trove", "trowel", "truck", "trumpet", "trunk", "trust", "trustee", "truth", "try", "tsunami",
"tub", "tuba", "tube", "tuber", "tug", "tugboat", "tuition", "tulip", "tumbler", "tummy", "tuna", "tune", "tunic", "tunnel",
"turban", "turf", "turkey", "turmeric", "turn", "turning", "turnip", "turnover", "turnstile", "turret", "turtle", "tusk", "tussle", "tutu",
"tuxedo", "tweet", "tweezers", "twig", "twilight", "twine", "twins", "twist", "twister", "twitter", "type", "typeface", "typewriter",
"typhoon", "ukulele", "ultimatum", "umbrella", "unblinking", "uncertainty", "uncle", "underclothes", "underestimate", "underground",
"underneath", "underpants", "underpass", "undershirt", "understanding", "understatement", "undertaker", "underwear", "underweight", "underwire",
"underwriting", "unemployment", "unibody", "uniform", "uniformity", "union", "unique", "unit", "unity", "universe", "university", "update",
"upgrade", "uplift", "upper", "upstairs", "upward", "urge", "urgency", "urn", "usage", "use", "user", "usher", "usual", "utensil", "utilisation",
"utility", "utilization", "vacation", "vaccine", "vacuum", "vagrant", "valance", "valentine", "validate", "validity", "valley", "valuable",
"value", "vampire", "van", "vanadyl", "vane", "vanilla", "vanity", "variability", "variable", "variant", "variation", "variety", "vascular",
"vase", "vault", "vaulting", "veal", "vector", "vegetable", "vegetarian", "vegetarianism", "vegetation", "vehicle", "veil", "vein", "veldt",
"vellum", "velocity", "velodrome", "velvet", "vendor", "veneer", "vengeance", "venison", "venom", "venti", "venture", "venue", "veranda",
"verb", "verdict", "verification", "vermicelli", "vernacular", "verse", "version", "vertigo", "verve", "vessel", "vest", "vestment",
"vet", "veteran", "veterinarian", "veto", "viability", "vibe", "vibraphone", "vibration", "vibrissae", "vice", "vicinity", "victim",
"victory", "video", "view", "viewer", "vignette", "villa", "village", "vine", "vinegar", "vineyard", "vintage", "vintner", "vinyl", "viola",
"violation", "violence", "violet", "violin", "virginal", "virtue", "virus", "visa", "viscose", "vise", "vision", "visit", "visitor",
"visor", "vista", "visual", "vitality", "vitamin", "vitro", "vivo", "vixen", "vodka", "vogue", "voice", "void", "vol", "volatility",
"volcano", "volleyball", "volume", "volunteer", "volunteering", "vomit", "vote", "voter", "voting", "voyage", "vulture", "wad", "wafer",
"waffle", "wage", "wagon", "waist", "waistband", "wait", "waiter", "waiting", "waitress", "waiver", "wake", "walk", "walker", "walking",
"walkway", "wall", "wallaby", "wallet", "walnut", "walrus", "wampum", "wannabe", "want", "war", "warden", "wardrobe", "warfare", "warlock",
"warlord", "warming", "warmth", "warning", "warrant", "warren", "warrior", "wasabi", "wash", "washbasin", "washcloth", "washer",
"washtub", "wasp", "waste", "wastebasket", "wasting", "watch", "watcher", "watchmaker", "water", "waterbed", "watercress", "waterfall",
"waterfront", "watermelon", "waterskiing", "waterspout", "waterwheel", "wave", "waveform", "wax", "way", "weakness", "wealth", "weapon",
"wear", "weasel", "weather", "web", "webinar", "webmail", "webpage", "website", "wedding", "wedge", "weed", "weeder", "weedkiller", "week",
"weekend", "weekender", "weight", "weird", "welcome", "welfare", "well", "west", "western", "wetland", "wetsuit",
"whack", "whale", "wharf", "wheat", "wheel", "whelp", "whey", "whip", "whirlpool", "whirlwind", "whisker", "whiskey", "whisper", "whistle",
"white", "whole", "wholesale", "wholesaler", "whorl", "wick", "widget", "widow", "width", "wife", "wifi", "wild", "wildebeest", "wilderness",
"wildlife", "will", "willingness", "willow", "win", "wind", "windage", "window", "windscreen", "windshield", "wine", "winery",
"wing", "wingman", "wingtip", "wink", "winner", "winter", "wire", "wiretap", "wiring", "wisdom", "wiseguy", "wish", "wisteria", "wit",
"witch", "withdrawal", "witness", "wok", "wolf", "woman", "wombat", "wonder", "wont", "wood", "woodchuck", "woodland",
"woodshed", "woodwind", "wool", "woolens", "word", "wording", "work", "workbench", "worker", "workforce", "workhorse", "working", "workout",
"workplace", "workshop", "world", "worm", "worry", "worship", "worshiper", "worth", "wound", "wrap", "wraparound", "wrapper", "wrapping",
"wreck", "wrecker", "wren", "wrench", "wrestler", "wriggler", "wrinkle", "wrist", "writer", "writing", "wrong", "xylophone", "yacht",
"yahoo", "yak", "yam", "yang", "yard", "yarmulke", "yarn", "yawl", "year", "yeast", "yellow", "yellowjacket", "yesterday", "yew", "yin",
"yoga", "yogurt", "yoke", "yolk", "young", "youngster", "yourself", "youth", "yoyo", "yurt", "zampone", "zebra", "zebrafish", "zen",
"zephyr", "zero", "ziggurat", "zinc", "zipper", "zither", "zombie", "zone", "zoo", "zoologist", "zoology", "zucchini"
};
std::string_view obfuscateWord(std::string_view src, WordMap & obfuscate_map, WordSet & used_nouns, SipHash hash_func)
{
/// Prevent using too many nouns
if (obfuscate_map.size() * 2 > nouns.size())
throw Exception("Too many unique identifiers in queries", ErrorCodes::TOO_MANY_TEMPORARY_COLUMNS);
std::string_view & mapped = obfuscate_map[src];
if (!mapped.empty())
return mapped;
hash_func.update(src.data(), src.size());
std::string_view noun = nouns.begin()[hash_func.get64() % nouns.size()];
/// Prevent collisions
while (!used_nouns.insert(noun).second)
{
hash_func.update('\0');
noun = nouns.begin()[hash_func.get64() % nouns.size()];
}
mapped = noun;
return mapped;
}
void obfuscateIdentifier(std::string_view src, WriteBuffer & result, WordMap & obfuscate_map, WordSet & used_nouns, SipHash hash_func)
{
/// Find words in form 'snake_case', 'CamelCase' or 'ALL_CAPS'.
const char * src_pos = src.data();
const char * src_end = src_pos + src.size();
const char * word_begin = src_pos;
bool word_has_alphanumerics = false;
auto append_word = [&]
{
std::string_view word(word_begin, src_pos - word_begin);
if (keep_words.count(word))
{
result.write(word.data(), word.size());
}
else
{
std::string_view obfuscated_word = obfuscateWord(word, obfuscate_map, used_nouns, hash_func);
/// Match the style of source word.
bool first_caps = !word.empty() && isUpperAlphaASCII(word[0]);
bool all_caps = first_caps && word.size() >= 2 && isUpperAlphaASCII(word[1]);
for (size_t i = 0, size = obfuscated_word.size(); i < size; ++i)
{
if (all_caps || (i == 0 && first_caps))
result.write(toUpperIfAlphaASCII(obfuscated_word[i]));
else
result.write(obfuscated_word[i]);
}
}
word_begin = src_pos;
word_has_alphanumerics = false;
};
while (src_pos < src_end)
{
if (isAlphaNumericASCII(src_pos[0]))
word_has_alphanumerics = true;
if (word_has_alphanumerics && src_pos[0] == '_')
{
append_word();
result.write('_');
++word_begin;
}
else if (word_has_alphanumerics && isUpperAlphaASCII(src_pos[0]) && isLowerAlphaASCII(src_pos[-1])) /// xX
{
append_word();
}
++src_pos;
}
if (word_begin < src_pos)
append_word();
}
void obfuscateLiteral(std::string_view src, WriteBuffer & result, SipHash hash_func)
{
const char * src_pos = src.data();
const char * src_end = src_pos + src.size();
while (src_pos < src_end)
{
/// Date
if (src_pos + strlen("0000-00-00") <= src_end
&& isNumericASCII(src_pos[0])
&& isNumericASCII(src_pos[1])
&& isNumericASCII(src_pos[2])
&& isNumericASCII(src_pos[3])
&& src_pos[4] == '-'
&& isNumericASCII(src_pos[5])
&& isNumericASCII(src_pos[6])
&& src_pos[7] == '-'
&& isNumericASCII(src_pos[8])
&& isNumericASCII(src_pos[9]))
{
DayNum date;
ReadBufferFromMemory in(src_pos, strlen("0000-00-00"));
readDateText(date, in);
SipHash hash_func_date = hash_func;
if (date != 0)
{
date += hash_func_date.get64() % 256;
}
writeDateText(date, result);
src_pos += strlen("0000-00-00");
/// DateTime
if (src_pos + strlen(" 00:00:00") <= src_end
&& isNumericASCII(src_pos[1])
&& isNumericASCII(src_pos[2])
&& src_pos[3] == ':'
&& isNumericASCII(src_pos[4])
&& isNumericASCII(src_pos[5])
&& src_pos[6] == ':'
&& isNumericASCII(src_pos[7])
&& isNumericASCII(src_pos[8]))
{
result.write(src_pos[0]);
hash_func_date.update(src_pos + 1, strlen("00:00:00"));
uint64_t hash_value = hash_func_date.get64();
uint32_t new_hour = hash_value % 24;
hash_value /= 24;
uint32_t new_minute = hash_value % 60;
hash_value /= 60;
uint32_t new_second = hash_value % 60;
result.write('0' + (new_hour / 10));
result.write('0' + (new_hour % 10));
result.write(':');
result.write('0' + (new_minute / 10));
result.write('0' + (new_minute % 10));
result.write(':');
result.write('0' + (new_second / 10));
result.write('0' + (new_second % 10));
src_pos += strlen(" 00:00:00");
}
}
else if (isNumericASCII(src_pos[0]))
{
/// Number
if (src_pos[0] == '0' || src_pos[0] == '1')
{
/// Keep zero and one as is.
result.write(src_pos[0]);
++src_pos;
}
else
{
ReadBufferFromMemory in(src_pos, src_end - src_pos);
uint64_t num;
readIntText(num, in);
SipHash hash_func_num = hash_func;
hash_func_num.update(src_pos, in.count());
src_pos += in.count();
/// Obfuscate number but keep it within same power of two range.
uint64_t obfuscated = hash_func_num.get64();
uint64_t log2 = bitScanReverse(num);
obfuscated = (1ULL << log2) + obfuscated % (1ULL << log2);
writeIntText(obfuscated, result);
}
}
else if (src_pos + 1 < src_end
&& (src_pos[0] == 'e' || src_pos[0] == 'E')
&& (isNumericASCII(src_pos[1]) || (src_pos[1] == '-' && src_pos + 2 < src_end && isNumericASCII(src_pos[2]))))
{
/// Something like an exponent of floating point number. Keep it as is.
/// But if it looks like a large number, overflow it into 16 bit.
result.write(src_pos[0]);
++src_pos;
ReadBufferFromMemory in(src_pos, src_end - src_pos);
int16_t num;
readIntText(num, in);
writeIntText(num, result);
src_pos += in.count();
}
else if (isAlphaASCII(src_pos[0]))
{
/// Alphabetial characters
const char * alpha_end = src_pos + 1;
while (alpha_end < src_end && isAlphaASCII(*alpha_end))
++alpha_end;
hash_func.update(src_pos, alpha_end - src_pos);
pcg64 rng(hash_func.get64());
while (src_pos < alpha_end)
{
auto random = rng();
if (isLowerAlphaASCII(*src_pos))
result.write('a' + random % 26);
else
result.write('A' + random % 26);
++src_pos;
}
}
else if (isASCII(src_pos[0]))
{
/// Punctuation, whitespace and control characters - keep as is.
result.write(src_pos[0]);
++src_pos;
}
else if (src_pos[0] <= '\xBF')
{
/// Continuation of UTF-8 sequence.
hash_func.update(src_pos[0]);
uint64_t hash = hash_func.get64();
char c = 0x80 + hash % (0xC0 - 0x80);
result.write(c);
++src_pos;
}
else
{
/// Start of UTF-8 sequence.
hash_func.update(src_pos[0]);
uint64_t hash = hash_func.get64();
if (src_pos[0] < '\xE0')
{
char c = 0xC0 + hash % 32;
result.write(c);
}
else if (src_pos[0] < '\xF0')
{
char c = 0xE0 + hash % 16;
result.write(c);
}
else
{
char c = 0xF0 + hash % 8;
result.write(c);
}
++src_pos;
}
}
}
}
void obfuscateQueries(
std::string_view src,
WriteBuffer & result,
WordMap & obfuscate_map,
WordSet & used_nouns,
SipHash hash_func,
KnownIdentifierFunc known_identifier_func)
{
Lexer lexer(src.data(), src.data() + src.size());
while (true)
{
Token token = lexer.nextToken();
std::string_view whole_token(token.begin, token.size());
if (token.isEnd())
break;
if (token.type == TokenType::BareWord)
{
std::string whole_token_uppercase(whole_token);
Poco::toUpperInPlace(whole_token_uppercase);
if (keywords.count(whole_token_uppercase)
|| known_identifier_func(whole_token))
{
/// Keep keywords as is.
result.write(token.begin, token.size());
}
else
{
/// Obfuscate identifiers
obfuscateIdentifier(whole_token, result, obfuscate_map, used_nouns, hash_func);
}
}
else if (token.type == TokenType::QuotedIdentifier)
{
assert(token.size() >= 2);
/// Write quotes and the obfuscated content inside.
result.write(*token.begin);
obfuscateIdentifier({token.begin + 1, token.size() - 2}, result, obfuscate_map, used_nouns, hash_func);
result.write(token.end[-1]);
}
else if (token.type == TokenType::Number)
{
obfuscateLiteral(whole_token, result, hash_func);
}
else if (token.type == TokenType::StringLiteral)
{
assert(token.size() >= 2);
result.write(*token.begin);
obfuscateLiteral({token.begin + 1, token.size() - 2}, result, hash_func);
result.write(token.end[-1]);
}
else if (token.type == TokenType::Comment)
{
/// Skip comments - they may contain confidential info.
}
else
{
/// Everyting else is kept as is.
result.write(token.begin, token.size());
}
}
}
}

View File

@ -0,0 +1,50 @@
#pragma once
#include <string>
#include <unordered_set>
#include <unordered_map>
#include <string_view>
#include <functional>
#include <Common/SipHash.h>
namespace DB
{
class WriteBuffer;
using WordMap = std::unordered_map<std::string_view, std::string_view>;
using WordSet = std::unordered_set<std::string_view>;
using KnownIdentifierFunc = std::function<bool(std::string_view)>;
/** Takes one or multiple queries and obfuscates them by replacing identifiers to pseudorandom words
* and replacing literals to random values, while preserving the structure of the queries and the general sense.
*
* Its intended use case is when the user wants to share their queries for testing and debugging
* but is afraid to disclose the details about their column names, domain area and values of constants.
*
* It can obfuscate multiple queries in consistent fashion - identical names will be transformed to identical results.
*
* The function is not guaranteed to always give correct result or to be secure. It's implemented in "best effort" fashion.
*
* @param src - a string with source queries.
* @param result - where the obfuscated queries will be written.
* @param obfuscate_map - information about substituted identifiers
* (pass empty map at the beginning or reuse it from previous invocation to get consistent result)
* @param used_nouns - information about words used for substitution
* (pass empty set at the beginning or reuse it from previous invocation to get consistent result)
* @param hash_func - hash function that will be used as a pseudorandom source,
* it's recommended to preseed the function before passing here.
* @param known_identifier_func - a function that returns true if identifier is known name
* (of function, aggregate function, etc. that should be kept as is). If it returns false, identifier will be obfuscated.
*/
void obfuscateQueries(
std::string_view src,
WriteBuffer & result,
WordMap & obfuscate_map,
WordSet & used_nouns,
SipHash hash_func,
KnownIdentifierFunc known_identifier_func);
}

View File

@ -85,6 +85,7 @@ SRCS(
MySQL/ASTDeclareReference.cpp
MySQL/ASTDeclareSubPartition.cpp
MySQL/ASTDeclareTableOptions.cpp
obfuscateQueries.cpp
parseDatabaseAndTableName.cpp
parseIdentifierOrStringLiteral.cpp
parseIntervalKind.cpp

View File

@ -10,7 +10,6 @@ ForkProcessor::Status ForkProcessor::prepare()
/// Check can output.
bool all_finished = true;
bool all_can_push = true;
size_t num_active_outputs = 0;
@ -18,7 +17,6 @@ ForkProcessor::Status ForkProcessor::prepare()
{
if (!output.isFinished())
{
all_finished = false;
++num_active_outputs;
/// The order is important.
@ -27,7 +25,7 @@ ForkProcessor::Status ForkProcessor::prepare()
}
}
if (all_finished)
if (0 == num_active_outputs)
{
input.close();
return Status::Finished;

View File

@ -36,7 +36,7 @@ void LineAsStringRowInputFormat::readLineObject(IColumn & column)
while (newline)
{
pos = find_first_symbols<'\n', '\\'>(buf.position(), buf.buffer().end());
pos = find_first_symbols<'\n'>(buf.position(), buf.buffer().end());
buf.position() = pos;
if (buf.position() == buf.buffer().end())
{
@ -47,12 +47,6 @@ void LineAsStringRowInputFormat::readLineObject(IColumn & column)
{
newline = false;
}
else if (*buf.position() == '\\')
{
++buf.position();
if (!buf.eof())
++buf.position();
}
}
buf.makeContinuousMemoryFromCheckpointToPos();
@ -64,10 +58,12 @@ void LineAsStringRowInputFormat::readLineObject(IColumn & column)
bool LineAsStringRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &)
{
if (!buf.eof())
readLineObject(*columns[0]);
if (buf.eof())
return false;
return !buf.eof();
readLineObject(*columns[0]);
return true;
}
void registerInputFormatProcessorLineAsString(FormatFactory & factory)
@ -82,4 +78,30 @@ void registerInputFormatProcessorLineAsString(FormatFactory & factory)
});
}
static bool fileSegmentationEngineLineAsStringpImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size)
{
char * pos = in.position();
bool need_more_data = true;
while (loadAtPosition(in, memory, pos) && need_more_data)
{
pos = find_first_symbols<'\n'>(pos, in.buffer().end());
if (pos == in.buffer().end())
continue;
if (memory.size() + static_cast<size_t>(pos - in.position()) >= min_chunk_size)
need_more_data = false;
++pos;
}
saveUpToPosition(in, memory, pos);
return loadAtPosition(in, memory, pos);
}
void registerFileSegmentationEngineLineAsString(FormatFactory & factory)
{
factory.registerFileSegmentationEngine("LineAsString", &fileSegmentationEngineLineAsStringpImpl);
}
}

View File

@ -51,6 +51,8 @@ RegexpRowInputFormat::ColumnFormat RegexpRowInputFormat::stringToFormat(const St
return ColumnFormat::Csv;
if (format == "JSON")
return ColumnFormat::Json;
if (format == "Raw")
return ColumnFormat::Raw;
throw Exception("Unsupported column format \"" + format + "\".", ErrorCodes::BAD_ARGUMENTS);
}
@ -88,6 +90,12 @@ bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns)
else
type->deserializeAsTextJSON(*columns[index], field_buf, format_settings);
break;
case ColumnFormat::Raw:
if (parse_as_nullable)
read = DataTypeNullable::deserializeWholeText(*columns[index], field_buf, format_settings, type);
else
type->deserializeAsWholeText(*columns[index], field_buf, format_settings);
break;
default:
break;
}

View File

@ -337,6 +337,8 @@ public:
throw Exception("Truncate is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
virtual void checkTableCanBeRenamed() const {}
/** Rename the table.
* Renaming a name in a file with metadata, the name in the list of tables in the RAM, is done separately.
* In this function, you need to rename the directory with the data, if any.

View File

@ -44,7 +44,7 @@ public:
/// Information about different TTLs for part. Can be used by
/// TTLSelector to assign merges with TTL.
const MergeTreeDataPartTTLInfos * ttl_infos;
const MergeTreeDataPartTTLInfos * ttl_infos = nullptr;
/// Part compression codec definition.
ASTPtr compression_codec_desc;

View File

@ -717,6 +717,8 @@ protected:
bool require_part_metadata;
/// Relative path data, changes during rename for ordinary databases use
/// under lockForShare if rename is possible.
String relative_data_path;

View File

@ -223,7 +223,7 @@ Pipe MergeTreeDataSelectExecutor::readFromParts(
if (part_column_queried)
VirtualColumnUtils::filterBlockWithQuery(query_info.query, virtual_columns_block, context);
std::multiset<String> part_values = VirtualColumnUtils::extractSingleValueFromBlock<String>(virtual_columns_block, "_part");
auto part_values = VirtualColumnUtils::extractSingleValueFromBlock<String>(virtual_columns_block, "_part");
metadata_snapshot->check(real_column_names, data.getVirtuals(), data.getStorageID());

View File

@ -56,10 +56,12 @@ void ReplicatedMergeTreeCleanupThread::run()
void ReplicatedMergeTreeCleanupThread::iterate()
{
storage.clearOldPartsAndRemoveFromZK();
storage.clearOldWriteAheadLogs();
{
auto lock = storage.lockForShare(RWLockImpl::NO_QUERY, storage.getSettings()->lock_acquire_timeout_for_background_operations);
/// Both use relative_data_path which changes during rename, so we
/// do it under share lock
storage.clearOldWriteAheadLogs();
storage.clearOldTemporaryDirectories();
}

View File

@ -412,26 +412,31 @@ static StoragePtr create(const StorageFactory::Arguments & args)
/// For Replicated.
String zookeeper_path;
String replica_name;
bool allow_renaming = true;
if (replicated)
{
bool has_arguments = arg_num + 2 <= arg_cnt;
bool has_valid_arguments = has_arguments && engine_args[arg_num]->as<ASTLiteral>() && engine_args[arg_num + 1]->as<ASTLiteral>();
ASTLiteral * ast_zk_path;
ASTLiteral * ast_replica_name;
if (has_valid_arguments)
{
const auto * ast = engine_args[arg_num]->as<ASTLiteral>();
if (ast && ast->value.getType() == Field::Types::String)
zookeeper_path = safeGet<String>(ast->value);
/// Get path and name from engine arguments
ast_zk_path = engine_args[arg_num]->as<ASTLiteral>();
if (ast_zk_path && ast_zk_path->value.getType() == Field::Types::String)
zookeeper_path = safeGet<String>(ast_zk_path->value);
else
throw Exception(
"Path in ZooKeeper must be a string literal" + getMergeTreeVerboseHelp(is_extended_storage_def),
ErrorCodes::BAD_ARGUMENTS);
++arg_num;
ast = engine_args[arg_num]->as<ASTLiteral>();
if (ast && ast->value.getType() == Field::Types::String)
replica_name = safeGet<String>(ast->value);
ast_replica_name = engine_args[arg_num]->as<ASTLiteral>();
if (ast_replica_name && ast_replica_name->value.getType() == Field::Types::String)
replica_name = safeGet<String>(ast_replica_name->value);
else
throw Exception(
"Replica name must be a string literal" + getMergeTreeVerboseHelp(is_extended_storage_def), ErrorCodes::BAD_ARGUMENTS);
@ -444,9 +449,24 @@ static StoragePtr create(const StorageFactory::Arguments & args)
else if (is_extended_storage_def && !has_arguments)
{
/// Try use default values if arguments are not specified.
/// It works for ON CLUSTER queries when database engine is Atomic and there are {shard} and {replica} in config.
zookeeper_path = "/clickhouse/tables/{uuid}/{shard}";
replica_name = "{replica}"; /// TODO maybe use hostname if {replica} is not defined?
/// Note: {uuid} macro works for ON CLUSTER queries when database engine is Atomic.
zookeeper_path = args.context.getConfigRef().getString("default_replica_path", "/clickhouse/tables/{uuid}/{shard}");
/// TODO maybe use hostname if {replica} is not defined?
replica_name = args.context.getConfigRef().getString("default_replica_name", "{replica}");
/// Modify query, so default values will be written to metadata
assert(arg_num == 0);
ASTs old_args;
std::swap(engine_args, old_args);
auto path_arg = std::make_shared<ASTLiteral>(zookeeper_path);
auto name_arg = std::make_shared<ASTLiteral>(replica_name);
ast_zk_path = path_arg.get();
ast_replica_name = name_arg.get();
engine_args.emplace_back(std::move(path_arg));
engine_args.emplace_back(std::move(name_arg));
std::move(std::begin(old_args), std::end(old_args), std::back_inserter(engine_args));
arg_num = 2;
arg_cnt += 2;
}
else
throw Exception("Expected two string literal arguments: zookeper_path and replica_name", ErrorCodes::BAD_ARGUMENTS);
@ -454,8 +474,44 @@ static StoragePtr create(const StorageFactory::Arguments & args)
/// Allow implicit {uuid} macros only for zookeeper_path in ON CLUSTER queries
bool is_on_cluster = args.local_context.getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY;
bool allow_uuid_macro = is_on_cluster || args.query.attach;
zookeeper_path = args.context.getMacros()->expand(zookeeper_path, args.table_id, allow_uuid_macro);
replica_name = args.context.getMacros()->expand(replica_name, args.table_id, false);
/// Unfold {database} and {table} macro on table creation, so table can be renamed.
/// We also unfold {uuid} macro, so path will not be broken after moving table from Atomic to Ordinary database.
if (!args.attach)
{
Macros::MacroExpansionInfo info;
/// NOTE: it's not recursive
info.expand_special_macros_only = true;
info.table_id = args.table_id;
if (!allow_uuid_macro)
info.table_id.uuid = UUIDHelpers::Nil;
zookeeper_path = args.context.getMacros()->expand(zookeeper_path, info);
info.level = 0;
info.table_id.uuid = UUIDHelpers::Nil;
replica_name = args.context.getMacros()->expand(replica_name, info);
}
ast_zk_path->value = zookeeper_path;
ast_replica_name->value = replica_name;
/// Expand other macros (such as {shard} and {replica}). We do not expand them on previous step
/// to make possible copying metadata files between replicas.
Macros::MacroExpansionInfo info;
info.table_id = args.table_id;
if (!allow_uuid_macro)
info.table_id.uuid = UUIDHelpers::Nil;
zookeeper_path = args.context.getMacros()->expand(zookeeper_path, info);
info.level = 0;
info.table_id.uuid = UUIDHelpers::Nil;
replica_name = args.context.getMacros()->expand(replica_name, info);
/// We do not allow renaming table with these macros in metadata, because zookeeper_path will be broken after RENAME TABLE.
/// NOTE: it may happen if table was created by older version of ClickHouse (< 20.10) and macros was not unfolded on table creation
/// or if one of these macros is recursively expanded from some other macro.
if (info.expanded_database || info.expanded_table)
allow_renaming = false;
}
/// This merging param maybe used as part of sorting key
@ -706,7 +762,8 @@ static StoragePtr create(const StorageFactory::Arguments & args)
date_column_name,
merging_params,
std::move(storage_settings),
args.has_force_restore_data_flag);
args.has_force_restore_data_flag,
allow_renaming);
else
return StorageMergeTree::create(
args.table_id,

View File

@ -180,7 +180,16 @@ StoragePtr StorageFactory::get(
.has_force_restore_data_flag = has_force_restore_data_flag
};
return storages.at(name).creator_fn(arguments);
auto res = storages.at(name).creator_fn(arguments);
if (!empty_engine_args.empty())
{
/// Storage creator modified empty arguments list, so we should modify the query
assert(storage_def && storage_def->engine && !storage_def->engine->arguments);
storage_def->engine->arguments = std::make_shared<ASTExpressionList>();
storage_def->engine->children.push_back(storage_def->engine->arguments);
storage_def->engine->arguments->children = empty_engine_args;
}
return res;
}
StorageFactory & StorageFactory::instance()

View File

@ -919,11 +919,13 @@ BackgroundProcessingPoolTaskResult StorageMergeTree::mergeMutateTask()
{
{
auto share_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations);
/// All use relative_data_path which changes during rename
/// so execute under share lock.
clearOldPartsFromFilesystem();
clearOldTemporaryDirectories();
clearOldWriteAheadLogs();
}
clearOldMutations();
clearOldWriteAheadLogs();
}
///TODO: read deduplicate option from table config

View File

@ -96,7 +96,7 @@ Pipe StorageMySQL::read(
/// TODO: rewrite MySQLBlockInputStream
return Pipe(std::make_shared<SourceFromInputStream>(
std::make_shared<MySQLBlockInputStream>(pool.get(), query, sample_block, max_block_size_)));
std::make_shared<MySQLLazyBlockInputStream>(pool, query, sample_block, max_block_size_, /* auto_close = */ true)));
}

View File

@ -178,7 +178,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
const String & date_column_name,
const MergingParams & merging_params_,
std::unique_ptr<MergeTreeSettings> settings_,
bool has_force_restore_data_flag)
bool has_force_restore_data_flag,
bool allow_renaming_)
: MergeTreeData(table_id_,
relative_data_path_,
metadata_,
@ -200,6 +201,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
, cleanup_thread(*this)
, part_check_thread(*this)
, restarting_thread(*this)
, allow_renaming(allow_renaming_)
{
queue_updating_task = global_context.getSchedulePool().createTask(
getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::queueUpdatingTask)", [this]{ queueUpdatingTask(); });
@ -4187,8 +4189,17 @@ void StorageReplicatedMergeTree::checkTableCanBeDropped() const
global_context.checkTableCanBeDropped(table_id.database_name, table_id.table_name, getTotalActiveSizeInBytes());
}
void StorageReplicatedMergeTree::checkTableCanBeRenamed() const
{
if (!allow_renaming)
throw Exception("Cannot rename Replicated table, because zookeeper_path contains implicit 'database' or 'table' macro. "
"We cannot rename path in ZooKeeper, so path may become inconsistent with table name. If you really want to rename table, "
"you should edit metadata file first and restart server or reattach the table.", ErrorCodes::NOT_IMPLEMENTED);
}
void StorageReplicatedMergeTree::rename(const String & new_path_to_table_data, const StorageID & new_table_id)
{
checkTableCanBeRenamed();
MergeTreeData::rename(new_path_to_table_data, new_table_id);
/// Update table name in zookeeper

View File

@ -128,6 +128,8 @@ public:
void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) override;
void checkTableCanBeRenamed() const override;
void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override;
bool supportsIndexForIn() const override { return true; }
@ -304,6 +306,9 @@ private:
/// True if replica was created for existing table with fixed granularity
bool other_replicas_fixed_granularity = false;
/// Do not allow RENAME TABLE if zookeeper_path contains {database} or {table} macro
const bool allow_renaming;
template <class Func>
void foreachCommittedParts(const Func & func) const;
@ -571,7 +576,8 @@ protected:
const String & date_column_name,
const MergingParams & merging_params_,
std::unique_ptr<MergeTreeSettings> settings_,
bool has_force_restore_data_flag);
bool has_force_restore_data_flag,
bool allow_renaming_);
};

View File

@ -1,6 +1,6 @@
#pragma once
#include <set>
#include <unordered_set>
#include <Core/Block.h>
#include <Parsers/IAST_fwd.h>
@ -30,9 +30,9 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & c
/// Extract from the input stream a set of `name` column values
template <typename T>
std::multiset<T> extractSingleValueFromBlock(const Block & block, const String & name)
auto extractSingleValueFromBlock(const Block & block, const String & name)
{
std::multiset<T> res;
std::unordered_set<T> res;
const ColumnWithTypeAndName & data = block.getByName(name);
size_t rows = block.rows();
for (size_t i = 0; i < rows; ++i)

View File

@ -237,7 +237,7 @@
"with_coverage": false
}
},
"Functional stateful tests (release, DatabaseAtomic)": {
"Functional stateful tests (release, DatabaseOrdinary)": {
"required_build_properties": {
"compiler": "gcc-10",
"package_type": "deb",
@ -345,7 +345,7 @@
"with_coverage": false
}
},
"Functional stateless tests (release, DatabaseAtomic)": {
"Functional stateless tests (release, DatabaseOrdinary)": {
"required_build_properties": {
"compiler": "gcc-10",
"package_type": "deb",
@ -441,6 +441,18 @@
"with_coverage": false
}
},
"Integration tests flaky check (asan)": {
"required_build_properties": {
"compiler": "clang-11",
"package_type": "deb",
"build_type": "relwithdebuginfo",
"sanitizer": "address",
"bundled": "bundled",
"splitted": "unsplitted",
"clang-tidy": "disable",
"with_coverage": false
}
},
"Compatibility check": {
"required_build_properties": {
"compiler": "gcc-10",
@ -560,6 +572,18 @@
"clang-tidy": "disable",
"with_coverage": false
}
},
"Release": {
"required_build_properties": {
"compiler": "gcc-10",
"package_type": "deb",
"build_type": "relwithdebuginfo",
"sanitizer": "none",
"bundled": "bundled",
"splitted": "unsplitted",
"clang-tidy": "disable",
"with_coverage": false
}
}
}
}

View File

@ -303,6 +303,12 @@ def run_tests_array(all_tests_with_params):
clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE)
clickhouse_proc.communicate("SELECT 'Running test {suite}/{case} from pid={pid}';".format(pid = os.getpid(), case = case, suite = suite))
if clickhouse_proc.returncode != 0:
failures += 1
print("Server does not respond to health check")
SERVER_DIED = True
break
reference_file = os.path.join(suite_dir, name) + '.reference'
stdout_file = os.path.join(suite_tmp_dir, name) + '.stdout'
stderr_file = os.path.join(suite_tmp_dir, name) + '.stderr'

View File

@ -3,5 +3,7 @@
<test>Hello, world!</test>
<shard>s1</shard>
<replica>r1</replica>
<default_path_test>/clickhouse/tables/{database}/{shard}/</default_path_test>
<default_name_test>table_{table}</default_name_test>
</macros>
</yandex>

View File

@ -330,10 +330,12 @@ def test_replicated_without_arguments(test_cluster):
assert "are supported only for ON CLUSTER queries with Atomic database engine" in \
instance.query_and_get_error("CREATE TABLE test_atomic.rmt (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n")
test_cluster.ddl_check_query(instance,
"CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n")
"CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree() ORDER BY n")
test_cluster.ddl_check_query(instance, "DROP TABLE test_atomic.rmt ON CLUSTER cluster")
test_cluster.ddl_check_query(instance,
"CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n")
"CREATE TABLE test_atomic.rmt UUID '12345678-0000-4000-8000-000000000001' ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n")
assert instance.query("SHOW CREATE test_atomic.rmt FORMAT TSVRaw") == \
"CREATE TABLE test_atomic.rmt\n(\n `n` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree('/clickhouse/tables/12345678-0000-4000-8000-000000000001/{shard}', '{replica}')\nORDER BY n\nSETTINGS index_granularity = 8192\n"
test_cluster.ddl_check_query(instance, "RENAME TABLE test_atomic.rmt TO test_atomic.rmt_renamed ON CLUSTER cluster")
test_cluster.ddl_check_query(instance,
"CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{uuid}/{shard}', '{replica}') ORDER BY n")
@ -349,6 +351,8 @@ def test_replicated_without_arguments(test_cluster):
assert "are supported only for ON CLUSTER queries with Atomic database engine" in \
instance.query_and_get_error("CREATE TABLE test_ordinary.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree('/{shard}/{uuid}/', '{replica}') ORDER BY n")
test_cluster.ddl_check_query(instance, "CREATE TABLE test_ordinary.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree('/{shard}/{table}/', '{replica}') ORDER BY n")
assert instance.query("SHOW CREATE test_ordinary.rmt FORMAT TSVRaw") == \
"CREATE TABLE test_ordinary.rmt\n(\n `n` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree('/{shard}/rmt/', '{replica}')\nORDER BY n\nSETTINGS index_granularity = 8192\n"
test_cluster.ddl_check_query(instance, "DROP DATABASE test_ordinary ON CLUSTER cluster")
test_cluster.pm_random_drops.push_rules(rules)

View File

@ -238,7 +238,9 @@ def test_mysql_federated(mysql_server, server_address):
node.query('''INSERT INTO mysql_federated.test VALUES (0), (1), (5)''', settings={"password": "123"})
def check_retryable_error_in_stderr(stderr):
return "Can't connect to local MySQL server through socket" in stderr or "MySQL server has gone away" in stderr
return ("Can't connect to local MySQL server through socket" in stderr
or "MySQL server has gone away" in stderr
or "Server shutdown in progress" in stderr)
code, (stdout, stderr) = mysql_server.exec_run('''
mysql

View File

@ -1,5 +1,6 @@
import logging
import os
import time
import pytest
from helpers.cluster import ClickHouseCluster
@ -37,10 +38,15 @@ def cluster():
def check_proxy_logs(cluster, proxy_instance, http_methods={"POST", "PUT", "GET", "DELETE"}):
logs = cluster.get_container_logs(proxy_instance)
# Check that all possible interactions with Minio are present
for http_method in http_methods:
assert logs.find(http_method + " http://minio1") >= 0
for i in range(10):
logs = cluster.get_container_logs(proxy_instance)
# Check with retry that all possible interactions with Minio are present
for http_method in http_methods:
if logs.find(http_method + " http://minio1") >= 0:
return
time.sleep(1)
else:
assert False, "http method not found in logs"
@pytest.mark.parametrize(

View File

@ -33,6 +33,25 @@ def started_cluster():
cluster.shutdown()
def test_many_connections(started_cluster):
table_name = 'test_many_connections'
conn = get_mysql_conn()
create_mysql_table(conn, table_name)
node1.query('''
CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse');
'''.format(table_name, table_name))
node1.query("INSERT INTO {} (id, name) SELECT number, concat('name_', toString(number)) from numbers(10) ".format(table_name))
query = "SELECT count() FROM ("
for i in range (24):
query += "SELECT id FROM {t} UNION ALL "
query += "SELECT id FROM {t})"
assert node1.query(query.format(t=table_name)) == '250\n'
conn.close()
def test_insert_select(started_cluster):
table_name = 'test_insert_select'
conn = get_mysql_conn()

View File

@ -12,7 +12,7 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE mutations_r1(d Date, x UInt32, s Stri
${CLICKHOUSE_CLIENT} --query="CREATE TABLE mutations_r2(d Date, x UInt32, s String, m MATERIALIZED x + 2) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00652/mutations', 'r2', d, intDiv(x, 10), 8192)"
# Test a mutation on empty table
${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_r1 DELETE WHERE x = 1"
${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_r1 DELETE WHERE x = 1 SETTINGS mutations_sync = 2"
# Insert some data
${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_r1(d, x, s) VALUES \
@ -34,6 +34,8 @@ ${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_r1 DELETE WHERE m = 3 SETTIN
${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_r1(d, x, s) VALUES \
('2000-01-01', 5, 'e'), ('2000-02-01', 5, 'e')"
${CLICKHOUSE_CLIENT} --query "SYSTEM SYNC REPLICA mutations_r2"
# Check that the table contains only the data that should not be deleted.
${CLICKHOUSE_CLIENT} --query="SELECT d, x, s, m FROM mutations_r2 ORDER BY d, x"
# Check the contents of the system.mutations table.

View File

@ -0,0 +1,4 @@
CREATE TABLE default.rmt\n(\n `n` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01148/{shard}/default/rmt\', \'{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192
CREATE TABLE default.rmt1\n(\n `n` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01148/{shard}/default/rmt\', \'{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192
CREATE TABLE default.rmt\n(\n `n` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'{default_path_test}test_01148\', \'{default_name_test}\')\nORDER BY n\nSETTINGS index_granularity = 8192
CREATE TABLE default.rmt\n(\n `n` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'{default_path_test}test_01148\', \'{default_name_test}\')\nORDER BY n\nSETTINGS index_granularity = 8192

View File

@ -0,0 +1,19 @@
DROP TABLE IF EXISTS rmt;
CREATE TABLE rmt (n UInt64, s String) ENGINE = ReplicatedMergeTree('/clickhouse/test_01148/{shard}/{database}/{table}', '{replica}') ORDER BY n;
SHOW CREATE TABLE rmt;
RENAME TABLE rmt TO rmt1;
DETACH TABLE rmt1;
ATTACH TABLE rmt1;
SHOW CREATE TABLE rmt1;
CREATE TABLE rmt (n UInt64, s String) ENGINE = ReplicatedMergeTree('{default_path_test}{uuid}', '{default_name_test}') ORDER BY n; -- { serverError 62 }
CREATE TABLE rmt (n UInt64, s String) ENGINE = ReplicatedMergeTree('{default_path_test}test_01148', '{default_name_test}') ORDER BY n;
SHOW CREATE TABLE rmt;
RENAME TABLE rmt TO rmt2; -- { serverError 48 }
DETACH TABLE rmt;
ATTACH TABLE rmt;
SHOW CREATE TABLE rmt;
DROP TABLE rmt;
DROP TABLE rmt1;

View File

@ -6,3 +6,5 @@
Finally implement this new feature.
42 ClickHouse
42 ClickHouse is a `fast` #open-source# (OLAP) database "management" :system:
1000000
1000

View File

@ -29,3 +29,15 @@ echo 'ClickHouse is a `fast` #open-source# (OLAP) 'database' "management" :syste
$CLICKHOUSE_CLIENT --query="SELECT * FROM line_as_string2 order by c";
$CLICKHOUSE_CLIENT --query="DROP TABLE line_as_string2"
$CLICKHOUSE_CLIENT --query="select repeat('aaa',50) from numbers(1000000)" > "${CLICKHOUSE_TMP}"/data1
$CLICKHOUSE_CLIENT --query="CREATE TABLE line_as_string3(field String) ENGINE = Memory";
$CLICKHOUSE_CLIENT --query="INSERT INTO line_as_string3 FORMAT LineAsString" < "${CLICKHOUSE_TMP}"/data1
$CLICKHOUSE_CLIENT --query="SELECT count(*) FROM line_as_string3";
$CLICKHOUSE_CLIENT --query="DROP TABLE line_as_string3"
$CLICKHOUSE_CLIENT --query="select randomString(50000) FROM numbers(1000)" > "${CLICKHOUSE_TMP}"/data2
$CLICKHOUSE_CLIENT --query="CREATE TABLE line_as_string4(field String) ENGINE = Memory";
$CLICKHOUSE_CLIENT --query="INSERT INTO line_as_string4 FORMAT LineAsString" < "${CLICKHOUSE_TMP}"/data2
$CLICKHOUSE_CLIENT --query="SELECT count(*) FROM line_as_string4";
$CLICKHOUSE_CLIENT --query="DROP TABLE line_as_string4"

View File

@ -1,2 +1,10 @@
01473_metric_log_table_event_start_time_microseconds_test
ok
01473_trace_log_table_event_start_time_microseconds_test
ok
01473_query_log_table_event_start_time_microseconds_test
ok
01473_query_thread_log_table_event_start_time_microseconds_test
ok
01473_text_log_table_event_start_time_microseconds_test
ok

View File

@ -3,22 +3,55 @@
-- an integration test as those metrics take 60s by default to be updated.
-- Refer: tests/integration/test_asynchronous_metric_log_table.
set log_queries = 1;
SET log_queries = 1;
SET query_profiler_real_time_period_ns = 100000000;
-- a long enough query to trigger the query profiler and to record trace log
SELECT sleep(2) FORMAT Null;
SET query_profiler_real_time_period_ns = 1000000000;
SYSTEM FLUSH LOGS;
select '01473_metric_log_table_event_start_time_microseconds_test';
system flush logs;
SELECT '01473_metric_log_table_event_start_time_microseconds_test';
-- query assumes that the event_time field is accurate.
WITH (
(
SELECT event_time_microseconds
SELECT event_time_microseconds, event_time
FROM system.metric_log
ORDER BY event_time DESC
LIMIT 1
) AS time_with_microseconds,
(
SELECT event_time
FROM system.metric_log
) AS time
SELECT if(dateDiff('second', toDateTime(time.1), toDateTime(time.2)) = 0, 'ok', toString(time));
SELECT '01473_trace_log_table_event_start_time_microseconds_test';
WITH (
SELECT event_time_microseconds, event_time
FROM system.trace_log
ORDER BY event_time DESC
LIMIT 1
) AS time
SELECT if(dateDiff('second', toDateTime(time.1), toDateTime(time.2)) = 0, 'ok', toString(time));
SELECT '01473_query_log_table_event_start_time_microseconds_test';
WITH (
SELECT event_time_microseconds, event_time
FROM system.query_log
ORDER BY event_time DESC
LIMIT 1
) AS time)
SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail')
) AS time
SELECT if(dateDiff('second', toDateTime(time.1), toDateTime(time.2)) = 0, 'ok', toString(time));
SELECT '01473_query_thread_log_table_event_start_time_microseconds_test';
WITH (
SELECT event_time_microseconds, event_time
FROM system.query_thread_log
ORDER BY event_time DESC
LIMIT 1
) AS time
SELECT if(dateDiff('second', toDateTime(time.1), toDateTime(time.2)) = 0, 'ok', toString(time));
SELECT '01473_text_log_table_event_start_time_microseconds_test';
WITH (
SELECT event_time_microseconds, event_time
FROM system.query_thread_log
ORDER BY event_time DESC
LIMIT 1
) AS time
SELECT if(dateDiff('second', toDateTime(time.1), toDateTime(time.2)) = 0, 'ok', toString(time));

View File

@ -0,0 +1 @@
abc\\ Hello, world!

View File

@ -0,0 +1,17 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} -n --query "
DROP TABLE IF EXISTS t;
CREATE TABLE t (a String, b String) ENGINE = Memory;
"
${CLICKHOUSE_CLIENT} --format_regexp_escaping_rule 'Raw' --format_regexp '^(.+?) separator (.+?)$' --query '
INSERT INTO t FORMAT Regexp abc\ separator Hello, world!'
${CLICKHOUSE_CLIENT} -n --query "
SELECT * FROM t;
DROP TABLE t;
"

View File

@ -0,0 +1,124 @@
DROP TABLE IF EXISTS tMM;
DROP TABLE IF EXISTS tDD;
DROP TABLE IF EXISTS sDD;
DROP TABLE IF EXISTS xMM;
CREATE TABLE tMM(d DateTime,a Int64) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY tuple() SETTINGS index_granularity = 8192;
SYSTEM STOP MERGES tMM;
INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, number FROM numbers(5000);
INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, number FROM numbers(5000);
INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, number FROM numbers(5000);
INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, number FROM numbers(5000);
INSERT INTO tMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, number FROM numbers(5000);
INSERT INTO tMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, number FROM numbers(5000);
CREATE TABLE tDD(d DateTime,a Int) ENGINE = MergeTree PARTITION BY toYYYYMMDD(d) ORDER BY tuple() SETTINGS index_granularity = 8192;
SYSTEM STOP MERGES tDD;
insert into tDD select toDateTime(toDate('2020-09-23')), number from numbers(10000) UNION ALL select toDateTime(toDateTime('2020-09-23 11:00:00')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-09-24')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-09-25')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-08-15')), number from numbers(10000);
CREATE TABLE sDD(d UInt64,a Int) ENGINE = MergeTree PARTITION BY toYYYYMM(toDate(intDiv(d,1000))) ORDER BY tuple() SETTINGS index_granularity = 8192;
SYSTEM STOP MERGES sDD;
insert into sDD select (1597536000+number*60)*1000, number from numbers(5000);
insert into sDD select (1597536000+number*60)*1000, number from numbers(5000);
insert into sDD select (1598918400+number*60)*1000, number from numbers(5000);
insert into sDD select (1598918400+number*60)*1000, number from numbers(5000);
insert into sDD select (1601510400+number*60)*1000, number from numbers(5000);
insert into sDD select (1602720000+number*60)*1000, number from numbers(5000);
CREATE TABLE xMM(d DateTime,a Int64, f Int64) ENGINE = MergeTree PARTITION BY (toYYYYMM(d), a) ORDER BY tuple() SETTINGS index_granularity = 8192;
SYSTEM STOP MERGES xMM;
INSERT INTO xMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, 1, number FROM numbers(5000);
INSERT INTO xMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, 2, number FROM numbers(5000);
INSERT INTO xMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, 3, number FROM numbers(5000);
INSERT INTO xMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, 2, number FROM numbers(5000);
INSERT INTO xMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, 1, number FROM numbers(5000);
INSERT INTO xMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, 1, number FROM numbers(5000);
SELECT '--------- tMM ----------------------------';
select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-15');
select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-01');
select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-10-15');
select uniqExact(_part), count() from tMM where toDate(d)='2020-09-15';
select uniqExact(_part), count() from tMM where toYYYYMM(d)=202009;
select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20200816;
select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20201015;
select uniqExact(_part), count() from tMM where toDate(d)='2020-10-15';
select uniqExact(_part), count() from tMM where d >= '2020-09-01 00:00:00' and d<'2020-10-15 00:00:00';
select uniqExact(_part), count() from tMM where d >= '2020-01-16 00:00:00' and d < toDateTime('2021-08-17 00:00:00');
select uniqExact(_part), count() from tMM where d >= '2020-09-16 00:00:00' and d < toDateTime('2020-10-01 00:00:00');
select uniqExact(_part), count() from tMM where d >= '2020-09-12 00:00:00' and d < '2020-10-16 00:00:00';
select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-12 00:00:00';
select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-09-01 00:00:00';
select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-10-01 00:00:00';
select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-15 00:00:00' and d < '2020-10-16 00:00:00';
select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010;
select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202009;
select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 and toStartOfDay(d) = '2020-10-01 00:00:00';
select uniqExact(_part), count() from tMM where toYYYYMM(d) >= 202009 and toStartOfDay(d) < '2020-10-02 00:00:00';
select uniqExact(_part), count() from tMM where toYYYYMM(d) > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00';
select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00';
select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 and toStartOfDay(d) < '2020-10-02 00:00:00';
select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010;
select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010;
select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-15';
select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01';
select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' and toStartOfMonth(d) < '2020-10-01';
SYSTEM START MERGES tMM;
OPTIMIZE TABLE tMM FINAL;
select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010;
select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010;
select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010;
SELECT '--------- tDD ----------------------------';
SYSTEM START MERGES tDD;
OPTIMIZE TABLE tDD FINAL;
select uniqExact(_part), count() from tDD where toDate(d)=toDate('2020-09-24');
select uniqExact(_part), count() FROM tDD WHERE toDate(d) = toDate('2020-09-24');
select uniqExact(_part), count() FROM tDD WHERE toDate(d) = '2020-09-24';
select uniqExact(_part), count() FROM tDD WHERE toDate(d) >= '2020-09-23' and toDate(d) <= '2020-09-26';
select uniqExact(_part), count() FROM tDD WHERE toYYYYMMDD(d) >= 20200923 and toDate(d) <= '2020-09-26';
SELECT '--------- sDD ----------------------------';
select uniqExact(_part), count() from sDD;
select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1)+1 = 202010;
select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) = 202010;
select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) = 202110;
select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000),'UTC')) < toDateTime('2020-10-02 00:00:00','UTC');
select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toDateTime(intDiv(d,1000),'UTC') < toDateTime('2020-10-01 00:00:00','UTC');
select uniqExact(_part), count() from sDD where d >= 1598918400000;
select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) < 202010;
SELECT '--------- xMM ----------------------------';
select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00';
select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00';
select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00';
select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a=1;
select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3;
select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3;
select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-11-01 00:00:00' and a = 1;
select uniqExact(_part), count() from xMM where a = 1;
select uniqExact(_part), count() from xMM where a = 66;
select uniqExact(_part), count() from xMM where a <> 66;
select uniqExact(_part), count() from xMM where a = 2;
SYSTEM START MERGES xMM;
optimize table xMM final;
select uniqExact(_part), count() from xMM where a = 1;
select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00';
select uniqExact(_part), count() from xMM where a <> 66;
select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3;
select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3;
DROP TABLE tMM;
DROP TABLE tDD;
DROP TABLE sDD;
DROP TABLE xMM;

View File

@ -0,0 +1,244 @@
--------- tMM ----------------------------
select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-15');
0 0
Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges
select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-01');
2 2880
Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges
select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-10-15');
1 1440
Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges
select uniqExact(_part), count() from tMM where toDate(d)='2020-09-15';
0 0
Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges
select uniqExact(_part), count() from tMM where toYYYYMM(d)=202009;
2 10000
Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges
select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20200816;
2 2880
Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges
select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20201015;
1 1440
Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges
select uniqExact(_part), count() from tMM where toDate(d)='2020-10-15';
1 1440
Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges
select uniqExact(_part), count() from tMM where d >= '2020-09-01 00:00:00' and d<'2020-10-15 00:00:00';
3 15000
Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges
select uniqExact(_part), count() from tMM where d >= '2020-01-16 00:00:00' and d < toDateTime('2021-08-17 00:00:00');
6 30000
Selected 6 parts by partition key, 6 parts by primary key, 6 marks by primary key, 6 marks to read from 6 ranges
select uniqExact(_part), count() from tMM where d >= '2020-09-16 00:00:00' and d < toDateTime('2020-10-01 00:00:00');
0 0
Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges
select uniqExact(_part), count() from tMM where d >= '2020-09-12 00:00:00' and d < '2020-10-16 00:00:00';
2 6440
Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges
select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-12 00:00:00';
2 10000
Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges
select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-09-01 00:00:00';
2 2880
Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges
select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-10-01 00:00:00';
1 1440
Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges
select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-15 00:00:00' and d < '2020-10-16 00:00:00';
2 6440
Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges
select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010;
4 20000
Selected 4 parts by partition key, 4 parts by primary key, 4 marks by primary key, 4 marks to read from 4 ranges
select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202009;
2 10000
Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges
select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 and toStartOfDay(d) = '2020-10-01 00:00:00';
1 1440
Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges
select uniqExact(_part), count() from tMM where toYYYYMM(d) >= 202009 and toStartOfDay(d) < '2020-10-02 00:00:00';
3 11440
Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges
select uniqExact(_part), count() from tMM where toYYYYMM(d) > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00';
1 1440
Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges
select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00';
3 11440
Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges
select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 and toStartOfDay(d) < '2020-10-02 00:00:00';
1 1440
Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges
select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010;
2 10000
Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges
select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010;
3 9999
Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges
select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-15';
2 10000
Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges
select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01';
4 20000
Selected 4 parts by partition key, 4 parts by primary key, 4 marks by primary key, 4 marks to read from 4 ranges
select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' and toStartOfMonth(d) < '2020-10-01';
2 10000
Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges
select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010;
2 9999
Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges
select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010;
1 10000
Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges
select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010;
2 20000
Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges
--------- tDD ----------------------------
select uniqExact(_part), count() from tDD where toDate(d)=toDate('2020-09-24');
1 10000
Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges
select uniqExact(_part), count() FROM tDD WHERE toDate(d) = toDate('2020-09-24');
1 10000
Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges
select uniqExact(_part), count() FROM tDD WHERE toDate(d) = '2020-09-24';
1 10000
Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges
select uniqExact(_part), count() FROM tDD WHERE toDate(d) >= '2020-09-23' and toDate(d) <= '2020-09-26';
3 40000
Selected 3 parts by partition key, 3 parts by primary key, 4 marks by primary key, 4 marks to read from 3 ranges
select uniqExact(_part), count() FROM tDD WHERE toYYYYMMDD(d) >= 20200923 and toDate(d) <= '2020-09-26';
3 40000
Selected 3 parts by partition key, 3 parts by primary key, 4 marks by primary key, 4 marks to read from 3 ranges
--------- sDD ----------------------------
select uniqExact(_part), count() from sDD;
6 30000
Selected 6 parts by partition key, 6 parts by primary key, 6 marks by primary key, 6 marks to read from 6 ranges
select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1)+1 = 202010;
3 9999
Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges
select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) = 202010;
2 9999
Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges
select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) = 202110;
0 0
Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges
select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000),'UTC')) < toDateTime('2020-10-02 00:00:00','UTC');
3 11440
Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges
select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toDateTime(intDiv(d,1000),'UTC') < toDateTime('2020-10-01 00:00:00','UTC');
2 10000
Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges
select uniqExact(_part), count() from sDD where d >= 1598918400000;
4 20000
Selected 4 parts by partition key, 4 parts by primary key, 4 marks by primary key, 4 marks to read from 4 ranges
select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) < 202010;
3 10001
Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges
--------- xMM ----------------------------
select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00';
2 10000
Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges
select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00';
3 10001
Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges
select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00';
2 10000
Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges
select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a=1;
1 1
Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges
select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3;
2 5001
Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges
select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3;
1 5000
Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges
select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-11-01 00:00:00' and a = 1;
2 10000
Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges
select uniqExact(_part), count() from xMM where a = 1;
3 15000
Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges
select uniqExact(_part), count() from xMM where a = 66;
0 0
Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges
select uniqExact(_part), count() from xMM where a <> 66;
6 30000
Selected 6 parts by partition key, 6 parts by primary key, 6 marks by primary key, 6 marks to read from 6 ranges
select uniqExact(_part), count() from xMM where a = 2;
2 10000
Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges
select uniqExact(_part), count() from xMM where a = 1;
2 15000
Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges
select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00';
1 10000
Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges
select uniqExact(_part), count() from xMM where a <> 66;
5 30000
Selected 5 parts by partition key, 5 parts by primary key, 5 marks by primary key, 5 marks to read from 5 ranges
select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3;
2 5001
Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges
select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3;
1 5000
Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges

View File

@ -0,0 +1,37 @@
#!/usr/bin/env bash
#-------------------------------------------------------------------------------------------
# Description of test result:
# Test the correctness of the partition
# pruning
#
# Script executes queries from a file 01508_partition_pruning.queries (1 line = 1 query)
# Queries are started with 'select' (but NOT with 'SELECT') are executed with log_level=debug
#-------------------------------------------------------------------------------------------
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh
#export CLICKHOUSE_CLIENT="clickhouse-client --send_logs_level=none"
#export CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none
#export CURDIR=.
queries="${CURDIR}/01508_partition_pruning.queries"
while IFS= read -r sql
do
[ -z "$sql" ] && continue
if [[ "$sql" == select* ]] ;
then
echo "$sql"
${CLICKHOUSE_CLIENT} --query "$sql"
CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=debug/g')
${CLICKHOUSE_CLIENT} --query "$sql" 2>&1 | grep -oh "Selected .* parts by partition key, *. parts by primary key, .* marks by primary key, .* marks to read from .* ranges.*$"
CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/--send_logs_level=debug/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/g')
echo ""
else
${CLICKHOUSE_CLIENT} --query "$sql"
fi
done < "$queries"

View File

@ -0,0 +1,16 @@
SELECT 116, 'Qqfu://2020-02-10isqkc1203 sp 2000-05-27T18:38:01', 13e100, Residue_id_breakfastDevice, park(Innervation), avgIf(remote('128.0.0.1'))
SELECT shell_dust_tintype between crumb and shoat, case when peach >= 116 then bombing else null end
SELECT
ChimeID,
Testimonial.ID, Testimonial.SipCauseway,
TankfulTRUMPET,
HUMIDITY.TermiteName, HUMIDITY.TermiteSculptural, HUMIDITY.TermiteGuilt, HUMIDITY.TermiteIntensity, HUMIDITY.SipCauseway, HUMIDITY.Coat
FROM merge.tinkle_efficiency
WHERE
FaithSeller >= '2020-10-13' AND FaithSeller <= '2020-10-21'
AND MandolinID = 30750384
AND intHash32(GafferID) = 448362928 AND intHash64(GafferID) = 12572659331310383983
AND ChimeID IN (8195672321757027078, 7079643623150622129, 5057006826979676478, 7886875230160484653, 7494974311229040743)
AND Stot = 1

View File

@ -0,0 +1,22 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_FORMAT --seed Hello --obfuscate <<< "SELECT 123, 'Test://2020-01-01hello1234 at 2000-01-01T01:02:03', 12e100, Gibberish_id_testCool, hello(World), avgIf(remote('127.0.0.1'))"
$CLICKHOUSE_FORMAT --seed Hello --obfuscate <<< "SELECT cost_first_screen between a and b, case when x >= 123 then y else null end"
$CLICKHOUSE_FORMAT --seed Hello --obfuscate <<< "
SELECT
VisitID,
Goals.ID, Goals.EventTime,
WatchIDs,
EAction.ProductName, EAction.ProductPrice, EAction.ProductCurrency, EAction.ProductQuantity, EAction.EventTime, EAction.Type
FROM merge.visits_v2
WHERE
StartDate >= '2020-09-17' AND StartDate <= '2020-09-25'
AND CounterID = 24226447
AND intHash32(UserID) = 416638616 AND intHash64(UserID) = 13269091395366875299
AND VisitID IN (5653048135597886819, 5556254872710352304, 5516214175671455313, 5476714937521999313, 5464051549483503043)
AND Sign = 1
"

View File

@ -0,0 +1,27 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS table_for_renames0"
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS table_for_renames50"
$CLICKHOUSE_CLIENT --query "CREATE TABLE table_for_renames0 (value UInt64, data String) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_01508/concurrent_rename', '1') ORDER BY tuple() SETTINGS cleanup_delay_period = 1, cleanup_delay_period_random_add = 0, min_rows_for_compact_part = 100000, min_rows_for_compact_part = 10000000, write_ahead_log_max_bytes = 1"
$CLICKHOUSE_CLIENT --query "INSERT INTO table_for_renames0 SELECT number, toString(number) FROM numbers(1000)"
$CLICKHOUSE_CLIENT --query "INSERT INTO table_for_renames0 SELECT number, toString(number) FROM numbers(1000, 1000)"
$CLICKHOUSE_CLIENT --query "INSERT INTO table_for_renames0 SELECT number, toString(number) FROM numbers(2000, 1000)"
for i in $(seq 1 50); do
prev_i=$((i - 1))
$CLICKHOUSE_CLIENT --query "RENAME TABLE table_for_renames$prev_i TO table_for_renames$i"
done
$CLICKHOUSE_CLIENT --query "SELECT COUNT() from table_for_renames50"
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS table_for_renames0"
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS table_for_renames50"

View File

@ -0,0 +1 @@
abc\\ Hello, world!

View File

@ -0,0 +1,17 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} -n --query "
DROP TABLE IF EXISTS t;
CREATE TABLE t (a String, b LowCardinality(Nullable(String))) ENGINE = Memory;
"
${CLICKHOUSE_CLIENT} --format_regexp_escaping_rule 'Raw' --format_regexp '^(.+?) separator (.+?)$' --query '
INSERT INTO t FORMAT Regexp abc\ separator Hello, world!'
${CLICKHOUSE_CLIENT} -n --query "
SELECT * FROM t;
DROP TABLE t;
"

View File

@ -17,6 +17,7 @@
"00151_replace_partition_with_different_granularity",
"00157_cache_dictionary",
"01193_metadata_loading",
"01473_event_time_microseconds",
"01474_executable_dictionary" /// informational stderr from sanitizer at start
],
"address-sanitizer": [
@ -25,6 +26,7 @@
"memory_profiler",
"odbc_roundtrip",
"01103_check_cpu_instructions_at_startup",
"01473_event_time_microseconds",
"01193_metadata_loading"
],
"ub-sanitizer": [
@ -33,6 +35,7 @@
"memory_profiler",
"01103_check_cpu_instructions_at_startup",
"00900_orc_load",
"01473_event_time_microseconds",
"01193_metadata_loading"
],
"memory-sanitizer": [
@ -43,6 +46,7 @@
"01086_odbc_roundtrip", /// can't pass because odbc libraries are not instrumented
"00877_memory_limit_for_new_delete", /// memory limits don't work correctly under msan because it replaces malloc/free
"01114_mysql_database_engine_segfault", /// it fails in _nss_files_parse_servent while using NSS from GLibc to authenticate (need to get rid of it)
"01473_event_time_microseconds",
"01193_metadata_loading"
],
"debug-build": [
@ -57,6 +61,7 @@
"01037_polygon_dicts_",
"hyperscan",
"01193_metadata_loading",
"01473_event_time_microseconds",
"01396_inactive_replica_cleanup_nodes"
],
"unbundled-build": [

View File

@ -1 +1,2 @@
# Fortunately, we have no suppressions!
# looks like a bug in clang-11 thread sanitizer, detects normal data race with random FD in this method
race:DB::LazyPipeFDs::close

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