Merge remote-tracking branch 'origin/master' into pr-local-plan

This commit is contained in:
Igor Nikonov 2024-07-16 08:57:57 +00:00
commit 0056db42be
8 changed files with 41 additions and 79 deletions

View File

@ -6,7 +6,20 @@ sidebar_label: MySQL Interface
# MySQL Interface
ClickHouse supports the MySQL wire protocol. This allow tools that are MySQL-compatible to interact with ClickHouse seamlessly (e.g. [Looker Studio](../integrations/data-visualization/looker-studio-and-clickhouse.md)).
ClickHouse supports the MySQL wire protocol. This allows certain clients that do not have native ClickHouse connectors leverage the MySQL protocol instead, and it has been validated with the following BI tools:
- [Looker Studio](../integrations/data-visualization/looker-studio-and-clickhouse.md)
- [Tableau Online](../integrations/tableau-online)
- [QuickSight](../integrations/quicksight)
If you are trying other untested clients or integrations, keep in mind that there could be the following limitations:
- SSL implementation might not be fully compatible; there could be potential [TLS SNI](https://www.cloudflare.com/learning/ssl/what-is-sni/) issues.
- A particular tool might require dialect features (e.g., MySQL-specific functions or settings) that are not implemented yet.
If there is a native driver available (e.g., [DBeaver](../integrations/dbeaver)), it is always preferred to use it instead of the MySQL interface. Additionally, while most of the MySQL language clients should work fine, MySQL interface is not guaranteed to be a drop-in replacement for a codebase with existing MySQL queries.
If your use case involves a particular tool that does not have a native ClickHouse driver, and you would like to use it via the MySQL interface and you found certain incompatibilities - please [create an issue](https://github.com/ClickHouse/ClickHouse/issues) in the ClickHouse repository.
## Enabling the MySQL Interface On ClickHouse Cloud

View File

@ -103,7 +103,7 @@ public:
Entry get(const ConnectionTimeouts & timeouts, /// NOLINT
const Settings & settings,
bool force_connected = true) override;
bool force_connected) override;
std::string getDescription() const
{

View File

@ -49,6 +49,7 @@ void logAboutProgress(LoggerPtr log, size_t processed, size_t total, AtomicStopw
AsyncLoader::Pool::Pool(const AsyncLoader::PoolInitializer & init)
: name(init.name)
, priority(init.priority)
, max_threads(init.max_threads > 0 ? init.max_threads : getNumberOfPhysicalCPUCores())
, thread_pool(std::make_unique<ThreadPool>(
init.metric_threads,
init.metric_active_threads,
@ -56,17 +57,16 @@ AsyncLoader::Pool::Pool(const AsyncLoader::PoolInitializer & init)
/* max_threads = */ std::numeric_limits<size_t>::max(), // Unlimited number of threads, we do worker management ourselves
/* max_free_threads = */ 0, // We do not require free threads
/* queue_size = */0)) // Unlimited queue to avoid blocking during worker spawning
, max_threads(init.max_threads > 0 ? init.max_threads : getNumberOfPhysicalCPUCores())
{}
AsyncLoader::Pool::Pool(Pool&& o) noexcept
: name(o.name)
, priority(o.priority)
, thread_pool(std::move(o.thread_pool))
, ready_queue(std::move(o.ready_queue))
, max_threads(o.max_threads)
, workers(o.workers)
, suspended_workers(o.suspended_workers.load()) // All these constructors are needed because std::atomic is neither copy-constructible, nor move-constructible. We never move pools after init, so it is safe.
, thread_pool(std::move(o.thread_pool))
{}
void cancelOnDependencyFailure(const LoadJobPtr & self, const LoadJobPtr & dependency, std::exception_ptr & cancel)

View File

@ -365,11 +365,11 @@ private:
{
const String name;
const Priority priority;
std::unique_ptr<ThreadPool> thread_pool; // NOTE: we avoid using a `ThreadPool` queue to be able to move jobs between pools.
std::map<UInt64, LoadJobPtr> ready_queue; // FIFO queue of jobs to be executed in this pool. Map is used for faster erasing. Key is `ready_seqno`
size_t max_threads; // Max number of workers to be spawn
size_t workers = 0; // Number of currently executing workers
std::atomic<size_t> suspended_workers{0}; // Number of workers that are blocked by `wait()` call on a job executing in the same pool (for deadlock resolution)
std::unique_ptr<ThreadPool> thread_pool; // NOTE: we avoid using a `ThreadPool` queue to be able to move jobs between pools.
explicit Pool(const PoolInitializer & init);
Pool(Pool&& o) noexcept;

View File

@ -1,102 +1,42 @@
flat
-- { echoOn }
INSERT INTO table_for_update_field_dictionary VALUES (1, 'First', now());
SELECT key, value FROM dict_flat ORDER BY key ASC;
1 First
INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now());
SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null;
SELECT key, value FROM dict_flat ORDER BY key ASC;
1 First
2 Second
INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now());
INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now());
SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null;
SELECT key, value FROM dict_flat ORDER BY key ASC;
1 First
2 SecondUpdated
3 Third
flat/custom
-- { echoOn }
INSERT INTO table_for_update_field_dictionary VALUES (1, 'First', now());
SELECT key, value FROM dict_flat_custom ORDER BY key ASC;
1 First
INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now());
SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null;
SELECT key, value FROM dict_flat_custom ORDER BY key ASC;
1 First
2 Second
INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now());
INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now());
SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null;
SELECT key, value FROM dict_flat_custom ORDER BY key ASC;
1 First
2 SecondUpdated
3 Third
hashed
-- { echoOn }
INSERT INTO table_for_update_field_dictionary VALUES (1, 'First', now());
SELECT key, value FROM dict_hashed ORDER BY key ASC;
1 First
INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now());
SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null;
SELECT key, value FROM dict_hashed ORDER BY key ASC;
1 First
2 Second
INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now());
INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now());
SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null;
SELECT key, value FROM dict_hashed ORDER BY key ASC;
1 First
2 SecondUpdated
3 Third
hashed/custom
-- { echoOn }
INSERT INTO table_for_update_field_dictionary VALUES (1, 'First', now());
SELECT key, value FROM dict_hashed_custom ORDER BY key ASC;
1 First
INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now());
SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null;
SELECT key, value FROM dict_hashed_custom ORDER BY key ASC;
1 First
2 Second
INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now());
INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now());
SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null;
SELECT key, value FROM dict_hashed_custom ORDER BY key ASC;
1 First
2 SecondUpdated
3 Third
complex_key_hashed
-- { echoOn }
INSERT INTO table_for_update_field_dictionary VALUES (1, 'First', now());
SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC;
1 First
INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now());
SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null;
SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC;
1 First
2 Second
INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now());
INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now());
SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null;
SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC;
1 First
2 SecondUpdated
3 Third
complex_key_hashed/custom
-- { echoOn }
INSERT INTO table_for_update_field_dictionary VALUES (1, 'First', now());
SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC;
1 First
INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now());
SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null;
SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC;
1 First
2 Second
INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now());
INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now());
SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null;
SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC;
1 First
2 SecondUpdated
3 Third

View File

@ -35,7 +35,7 @@ for layout in "${layouts[@]}"; do
echo "$layout"
fi
$CLICKHOUSE_CLIENT -nm -q "
$CLICKHOUSE_CLIENT --multiquery "
TRUNCATE TABLE table_for_update_field_dictionary;
CREATE DICTIONARY $dictionary_name
@ -49,24 +49,31 @@ for layout in "${layouts[@]}"; do
LAYOUT($layout())
LIFETIME(1);
-- { echoOn }
INSERT INTO table_for_update_field_dictionary VALUES (1, 'First', now());
SELECT key, value FROM $dictionary_name ORDER BY key ASC;
INSERT INTO table_for_update_field_dictionary VALUES (1, 'First', now());"
INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now());
SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null;
while true
do
$CLICKHOUSE_CLIENT --query "SELECT key, value FROM $dictionary_name ORDER BY key ASC" | grep -A10 -B10 'First' && break;
sleep .1;
done
SELECT key, value FROM $dictionary_name ORDER BY key ASC;
$CLICKHOUSE_CLIENT --query "INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now());"
INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now());
INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now());
SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null;
while true
do
$CLICKHOUSE_CLIENT --query "SELECT key, value FROM $dictionary_name ORDER BY key ASC" | grep -A10 -B10 'Second' && break;
sleep .1;
done
SELECT key, value FROM $dictionary_name ORDER BY key ASC;
-- { echoOff }
$CLICKHOUSE_CLIENT --query "INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()), (3, 'Third', now())"
DROP DICTIONARY $dictionary_name;
"
while true
do
$CLICKHOUSE_CLIENT --query "SELECT key, value FROM $dictionary_name ORDER BY key ASC" | grep -A10 -B10 'SecondUpdated' && break;
sleep .1;
done
$CLICKHOUSE_CLIENT --query "DROP DICTIONARY $dictionary_name"
done
done

View File

@ -761,6 +761,7 @@ QueryCacheMisses
QueryPreempted
QueryThread
QuickAssist
QuickSight
QuoteMeta
RBAC
RClickHouse

View File

@ -9,6 +9,7 @@
#include <Core/ColumnWithTypeAndName.h>
#include <Core/ColumnsWithTypeAndName.h>
#include <Disks/DiskLocal.h>
#include <Core/Settings.h>
#include <Formats/ReadSchemaUtils.h>
#include <Formats/registerFormats.h>
#include <IO/ReadBuffer.h>