Merge branch 'master' of github.com:yandex/ClickHouse into errorcodes-style

This commit is contained in:
Alexey Milovidov 2020-02-26 17:16:22 +03:00
commit 425ed714eb
68 changed files with 774 additions and 146 deletions

2
.gitmodules vendored
View File

@ -140,7 +140,7 @@
url = https://github.com/ClickHouse-Extras/libc-headers.git
[submodule "contrib/replxx"]
path = contrib/replxx
url = https://github.com/AmokHuginnsson/replxx.git
url = https://github.com/ClickHouse-Extras/replxx.git
[submodule "contrib/ryu"]
path = contrib/ryu
url = https://github.com/ClickHouse-Extras/ryu.git

View File

@ -88,8 +88,7 @@ endif()
include (cmake/sanitize.cmake)
if (CMAKE_GENERATOR STREQUAL "Ninja")
if (CMAKE_GENERATOR STREQUAL "Ninja" AND NOT DISABLE_COLORED_BUILD)
# Turn on colored output. https://github.com/ninja-build/ninja/wiki/FAQ
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fdiagnostics-color=always")
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fdiagnostics-color=always")

View File

@ -24,6 +24,11 @@ if (ENABLE_REPLXX)
ReplxxLineReader.cpp
ReplxxLineReader.h
)
elseif (ENABLE_READLINE)
set (SRCS ${SRCS}
ReadlineLineReader.cpp
ReadlineLineReader.h
)
endif ()
if (USE_DEBUG_HELPERS)
@ -57,6 +62,28 @@ endif()
target_link_libraries(common PUBLIC replxx)
# allow explicitly fallback to readline
if (NOT ENABLE_REPLXX AND ENABLE_READLINE)
message (STATUS "Attempt to fallback to readline explicitly")
set (READLINE_PATHS "/usr/local/opt/readline/lib")
# First try find custom lib for macos users (default lib without history support)
find_library (READLINE_LIB NAMES readline PATHS ${READLINE_PATHS} NO_DEFAULT_PATH)
if (NOT READLINE_LIB)
find_library (READLINE_LIB NAMES readline PATHS ${READLINE_PATHS})
endif ()
set(READLINE_INCLUDE_PATHS "/usr/local/opt/readline/include")
find_path (READLINE_INCLUDE_DIR NAMES readline/readline.h PATHS ${READLINE_INCLUDE_PATHS} NO_DEFAULT_PATH)
if (NOT READLINE_INCLUDE_DIR)
find_path (READLINE_INCLUDE_DIR NAMES readline/readline.h PATHS ${READLINE_INCLUDE_PATHS})
endif ()
if (READLINE_INCLUDE_DIR AND READLINE_LIB)
target_link_libraries(common PUBLIC ${READLINE_LIB})
target_compile_definitions(common PUBLIC USE_READLINE=1)
message (STATUS "Using readline: ${READLINE_INCLUDE_DIR} : ${READLINE_LIB}")
endif ()
endif ()
target_link_libraries (common
PUBLIC
${Poco_Util_LIBRARY}

View File

@ -53,11 +53,18 @@ LineReader::Suggest::WordsRange LineReader::Suggest::getCompletions(const String
/// last_word can be empty.
return std::equal_range(
words.begin(), words.end(), last_word, [prefix_length](std::string_view s, std::string_view prefix_searched)
{
return strncmp(s.data(), prefix_searched.data(), prefix_length) < 0;
});
if (case_insensitive)
return std::equal_range(
words.begin(), words.end(), last_word, [prefix_length](std::string_view s, std::string_view prefix_searched)
{
return strncasecmp(s.data(), prefix_searched.data(), prefix_length) < 0;
});
else
return std::equal_range(
words.begin(), words.end(), last_word, [prefix_length](std::string_view s, std::string_view prefix_searched)
{
return strncmp(s.data(), prefix_searched.data(), prefix_length) < 0;
});
}
LineReader::LineReader(const String & history_file_path_, char extender_, char delimiter_)

View File

@ -8,18 +8,19 @@
class LineReader
{
public:
class Suggest
struct Suggest
{
protected:
using Words = std::vector<std::string>;
using WordsRange = std::pair<Words::const_iterator, Words::const_iterator>;
Words words;
std::atomic<bool> ready{false};
public:
/// Get iterators for the matched range of words if any.
WordsRange getCompletions(const String & prefix, size_t prefix_length) const;
/// case sensitive suggestion
bool case_insensitive = false;
};
LineReader(const String & history_file_path, char extender, char delimiter = 0); /// if delimiter != 0, then it's multiline mode
@ -31,6 +32,13 @@ public:
/// Typical delimiter is ';' (semicolon) and typical extender is '\' (backslash).
String readLine(const String & first_prompt, const String & second_prompt);
/// When bracketed paste mode is set, pasted text is bracketed with control sequences so
/// that the program can differentiate pasted text from typed-in text. This helps
/// clickhouse-client so that without -m flag, one can still paste multiline queries, and
/// possibly get better pasting performance. See https://cirw.in/blog/bracketed-paste for
/// more details.
virtual void enableBracketedPaste() {}
protected:
enum InputStatus
{

View File

@ -0,0 +1,173 @@
#include <common/ReadlineLineReader.h>
#include <ext/scope_guard.h>
#include <errno.h>
#include <signal.h>
#include <string.h>
#include <unistd.h>
namespace
{
/// Trim ending whitespace inplace
void trim(String & s)
{
s.erase(std::find_if(s.rbegin(), s.rend(), [](int ch) { return !std::isspace(ch); }).base(), s.end());
}
}
static const LineReader::Suggest * suggest;
/// Points to current word to suggest.
static LineReader::Suggest::Words::const_iterator pos;
/// Points after the last possible match.
static LineReader::Suggest::Words::const_iterator end;
/// Set iterators to the matched range of words if any.
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);
}
/// Iterates through matched range.
static char * nextMatch()
{
if (pos >= end)
return nullptr;
/// readline will free memory by itself.
char * word = strdup(pos->c_str());
++pos;
return word;
}
static char * generate(const char * text, int state)
{
if (!suggest->ready)
return nullptr;
if (state == 0)
findRange(text, strlen(text));
/// Do not append whitespace after word. For unknown reason, rl_completion_append_character = '\0' does not work.
rl_completion_suppress_append = 1;
return nextMatch();
};
ReadlineLineReader::ReadlineLineReader(const Suggest & suggest_, const String & history_file_path_, char extender_, char delimiter_)
: LineReader(history_file_path_, extender_, delimiter_)
{
suggest = &suggest_;
if (!history_file_path.empty())
{
int res = read_history(history_file_path.c_str());
if (res)
std::cerr << "Cannot read history from file " + history_file_path + ": "+ strerror(errno) << std::endl;
}
/// Added '.' to the default list. Because it is used to separate database and table.
rl_basic_word_break_characters = word_break_characters;
/// Not append whitespace after single suggestion. Because whitespace after function name is meaningless.
rl_completion_append_character = '\0';
rl_completion_entry_function = generate;
/// Install Ctrl+C signal handler that will be used in interactive mode.
if (rl_initialize())
throw std::runtime_error("Cannot initialize readline");
auto clear_prompt_or_exit = [](int)
{
/// This is signal safe.
ssize_t res = write(STDOUT_FILENO, "\n", 1);
/// Allow to quit client while query is in progress by pressing Ctrl+C twice.
/// (First press to Ctrl+C will try to cancel query by InterruptListener).
if (res == 1 && rl_line_buffer[0] && !RL_ISSTATE(RL_STATE_DONE))
{
rl_replace_line("", 0);
if (rl_forced_update_display())
_exit(0);
}
else
{
/// A little dirty, but we struggle to find better way to correctly
/// force readline to exit after returning from the signal handler.
_exit(0);
}
};
if (signal(SIGINT, clear_prompt_or_exit) == SIG_ERR)
throw std::runtime_error(std::string("Cannot set signal handler for readline: ") + strerror(errno));
}
ReadlineLineReader::~ReadlineLineReader()
{
}
LineReader::InputStatus ReadlineLineReader::readOneLine(const String & prompt)
{
input.clear();
const char* cinput = readline(prompt.c_str());
if (cinput == nullptr)
return (errno != EAGAIN) ? ABORT : RESET_LINE;
input = cinput;
trim(input);
return INPUT_LINE;
}
void ReadlineLineReader::addToHistory(const String & line)
{
add_history(line.c_str());
}
#if RL_VERSION_MAJOR >= 7
#define BRACK_PASTE_PREF "\033[200~"
#define BRACK_PASTE_SUFF "\033[201~"
#define BRACK_PASTE_LAST '~'
#define BRACK_PASTE_SLEN 6
/// This handler bypasses some unused macro/event checkings and remove trailing newlines before insertion.
static int clickhouse_rl_bracketed_paste_begin(int /* count */, int /* key */)
{
std::string buf;
buf.reserve(128);
RL_SETSTATE(RL_STATE_MOREINPUT);
SCOPE_EXIT(RL_UNSETSTATE(RL_STATE_MOREINPUT));
int c;
while ((c = rl_read_key()) >= 0)
{
if (c == '\r')
c = '\n';
buf.push_back(c);
if (buf.size() >= BRACK_PASTE_SLEN && c == BRACK_PASTE_LAST && buf.substr(buf.size() - BRACK_PASTE_SLEN) == BRACK_PASTE_SUFF)
{
buf.resize(buf.size() - BRACK_PASTE_SLEN);
break;
}
}
trim(buf);
return static_cast<size_t>(rl_insert_text(buf.c_str())) == buf.size() ? 0 : 1;
}
#endif
void ReadlineLineReader::enableBracketedPaste()
{
#if RL_VERSION_MAJOR >= 7
rl_variable_bind("enable-bracketed-paste", "on");
/// Use our bracketed paste handler to get better user experience. See comments above.
rl_bind_keyseq(BRACK_PASTE_PREF, clickhouse_rl_bracketed_paste_begin);
#endif
};

View File

@ -0,0 +1,19 @@
#pragma once
#include "LineReader.h"
#include <readline/readline.h>
#include <readline/history.h>
class ReadlineLineReader : public LineReader
{
public:
ReadlineLineReader(const Suggest & suggest, const String & history_file_path, char extender, char delimiter = 0);
~ReadlineLineReader() override;
void enableBracketedPaste() override;
private:
InputStatus readOneLine(const String & prompt) override;
void addToHistory(const String & line) override;
};

View File

@ -55,3 +55,8 @@ void ReplxxLineReader::addToHistory(const String & line)
{
rx.history_add(line);
}
void ReplxxLineReader::enableBracketedPaste()
{
rx.enable_bracketed_paste();
};

View File

@ -10,6 +10,8 @@ public:
ReplxxLineReader(const Suggest & suggest, const String & history_file_path, char extender, char delimiter = 0);
~ReplxxLineReader() override;
void enableBracketedPaste() override;
private:
InputStatus readOneLine(const String & prompt) override;
void addToHistory(const String & line) override;

2
contrib/replxx vendored

@ -1 +1 @@
Subproject commit 37582f0bb8c52513c6c6b76797c02d852d701dad
Subproject commit 07cbfbec550133b88c91c4073fa5af2ae2ae6a9a

View File

@ -4,6 +4,8 @@
#if USE_REPLXX
# include <common/ReplxxLineReader.h>
#elif USE_READLINE
# include <common/ReadlineLineReader.h>
#else
# include <common/LineReader.h>
#endif
@ -480,8 +482,12 @@ private:
throw Exception("time option could be specified only in non-interactive mode", ErrorCodes::BAD_ARGUMENTS);
if (server_revision >= Suggest::MIN_SERVER_REVISION && !config().getBool("disable_suggestion", false))
{
if (config().has("case_insensitive_suggestion"))
Suggest::instance().setCaseInsensitive();
/// Load suggestion data from the server.
Suggest::instance().load(connection_parameters, config().getInt("suggestion_limit"));
}
/// Load command history if present.
if (config().has("history_file"))
@ -500,10 +506,18 @@ private:
#if USE_REPLXX
ReplxxLineReader lr(Suggest::instance(), history_file, '\\', config().has("multiline") ? ';' : 0);
#elif USE_READLINE
ReadlineLineReader lr(Suggest::instance(), history_file, '\\', config().has("multiline") ? ';' : 0);
#else
LineReader lr(history_file, '\\', config().has("multiline") ? ';' : 0);
#endif
/// Enable bracketed-paste-mode only when multiquery is enabled and multiline is
/// disabled, so that we are able to paste and execute multiline queries in a whole
/// instead of erroring out, while be less intrusive.
if (config().has("multiquery") && !config().has("multiline"))
lr.enableBracketedPaste();
do
{
auto input = lr.readLine(prompt(), ":-] ");
@ -1674,6 +1688,7 @@ public:
("always_load_suggestion_data", "Load suggestion data even if clickhouse-client is run in non-interactive mode. Used for testing.")
("suggestion_limit", po::value<int>()->default_value(10000),
"Suggestion limit for how many databases, tables and columns to fetch.")
("case_insensitive_suggestion", "Case sensitive suggestions.")
("multiline,m", "multiline")
("multiquery,n", "multiquery")
("format,f", po::value<std::string>(), "default output format")

View File

@ -36,7 +36,17 @@ void Suggest::load(const ConnectionParameters & connection_parameters, size_t su
/// Note that keyword suggestions are available even if we cannot load data from server.
std::sort(words.begin(), words.end());
if (case_insensitive)
std::sort(words.begin(), words.end(), [](const std::string & str1, const std::string & str2)
{
return std::lexicographical_compare(begin(str1), end(str1), begin(str2), end(str2), [](const char char1, const char char2)
{
return std::tolower(char1) < std::tolower(char2);
});
});
else
std::sort(words.begin(), words.end());
ready = true;
});
}

View File

@ -23,6 +23,9 @@ public:
return instance;
}
/// Need to set before load
void setCaseInsensitive() { case_insensitive = true; }
void load(const ConnectionParameters & connection_parameters, size_t suggestion_limit);
/// Older server versions cannot execute the query above.

View File

@ -253,7 +253,7 @@ struct Settings : public SettingsCollection<Settings>
M(SettingSeconds, http_receive_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP receive timeout", 0) \
M(SettingBool, optimize_throw_if_noop, false, "If setting is enabled and OPTIMIZE query didn't actually assign a merge then an explanatory exception is thrown", 0) \
M(SettingBool, use_index_for_in_with_subqueries, true, "Try using an index if there is a subquery or a table expression on the right side of the IN operator.", 0) \
M(SettingBool, joined_subquery_requires_alias, false, "Force joined subqueries to have aliases for correct name qualification.", 0) \
M(SettingBool, joined_subquery_requires_alias, true, "Force joined subqueries and table functions to have aliases for correct name qualification.", 0) \
M(SettingBool, empty_result_for_aggregation_by_empty_set, false, "Return empty result when aggregating without keys on empty set.", 0) \
M(SettingBool, allow_distributed_ddl, true, "If it is set to true, then a user is allowed to executed distributed DDL queries.", 0) \
M(SettingUInt64, odbc_max_field_size, 1024, "Max size of filed can be read from ODBC dictionary. Long strings are truncated.", 0) \

View File

@ -18,8 +18,8 @@ struct BlockIO
BlockIO(const BlockIO &) = default;
~BlockIO() = default;
/** process_list_entry should be destroyed after in and after out,
* since in and out contain pointer to objects inside process_list_entry (query-level MemoryTracker for example),
/** process_list_entry should be destroyed after in, after out and after pipeline,
* since in, out and pipeline contain pointer to objects inside process_list_entry (query-level MemoryTracker for example),
* which could be used before destroying of in and out.
*/
std::shared_ptr<ProcessListEntry> process_list_entry;
@ -56,6 +56,7 @@ struct BlockIO
out.reset();
in.reset();
pipeline = QueryPipeline();
process_list_entry.reset();
process_list_entry = rhs.process_list_entry;

View File

@ -12,14 +12,16 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_ENOUGH_SPACE;
extern const int LOGICAL_ERROR;
extern const int UNKNOWN_POLICY;
extern const int UNKNOWN_DISK;
extern const int EXCESSIVE_ELEMENT_IN_CONFIG;
extern const int UNKNOWN_DISK;
extern const int UNKNOWN_POLICY;
extern const int LOGICAL_ERROR;
}
DiskSelector::DiskSelector(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const Context & context)
{
Poco::Util::AbstractConfiguration::Keys keys;
@ -261,10 +263,10 @@ DiskPtr StoragePolicy::getAnyDisk() const
/// StoragePolicy must contain at least one Volume
/// Volume must contain at least one Disk
if (volumes.empty())
throw Exception("StoragePolicy has no volumes. It's a bug.", ErrorCodes::NOT_ENOUGH_SPACE);
throw Exception("StoragePolicy has no volumes. It's a bug.", ErrorCodes::LOGICAL_ERROR);
if (volumes[0]->disks.empty())
throw Exception("Volume '" + volumes[0]->getName() + "' has no disks. It's a bug.", ErrorCodes::NOT_ENOUGH_SPACE);
throw Exception("Volume '" + volumes[0]->getName() + "' has no disks. It's a bug.", ErrorCodes::LOGICAL_ERROR);
return volumes[0]->disks[0];
}

View File

@ -16,9 +16,6 @@
namespace DB
{
namespace ErrorCodes
{
}
/// Parse .xml configuration and store information about disks
/// Mostly used for introspection.

View File

@ -27,7 +27,6 @@ namespace DB
#if USE_EMBEDDED_COMPILER
static constexpr bool compilable = false;
#endif
};

View File

@ -110,7 +110,7 @@ std::vector<TableWithColumnNames> getTablesWithColumns(const std::vector<const A
{
for (auto & pr : tables_with_columns)
if (pr.table.table.empty() && pr.table.alias.empty())
throw Exception("Not unique subquery in FROM requires an alias (or joined_subquery_requires_alias=0 to disable restriction).",
throw Exception("No alias for subquery or table function in JOIN (set joined_subquery_requires_alias=0 to disable restriction).",
ErrorCodes::ALIAS_REQUIRED);
}

View File

@ -115,6 +115,8 @@ namespace ErrorCodes
extern const int ABORTED;
extern const int UNKNOWN_PART_TYPE;
extern const int UNEXPECTED_AST_STRUCTURE;
extern const int UNKNOWN_DISK;
extern const int NOT_ENOUGH_SPACE;
}

View File

@ -19,7 +19,7 @@ namespace ErrorCodes
}
/// 0b11 -- can be true and false at the same time
const Field UNKNOWN_FIELD(3u);
static const Field UNKNOWN_FIELD(3u);
MergeTreeIndexGranuleSet::MergeTreeIndexGranuleSet(const MergeTreeIndexSet & index_)
@ -237,8 +237,6 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet(
expression_ast = select.where()->clone();
else if (select.prewhere())
expression_ast = select.prewhere()->clone();
else
expression_ast = std::make_shared<ASTLiteral>(UNKNOWN_FIELD);
useless = checkASTUseless(expression_ast);
/// Do not proceed if index is useless for this query.
@ -261,6 +259,9 @@ bool MergeTreeIndexConditionSet::alwaysUnknownOrTrue() const
bool MergeTreeIndexConditionSet::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const
{
if (useless)
return true;
auto granule = std::dynamic_pointer_cast<MergeTreeIndexGranuleSet>(idx_granule);
if (!granule)
throw Exception(
@ -406,8 +407,11 @@ bool MergeTreeIndexConditionSet::operatorFromAST(ASTPtr & node) const
return true;
}
bool MergeTreeIndexConditionSet::checkASTUseless(const ASTPtr &node, bool atomic) const
bool MergeTreeIndexConditionSet::checkASTUseless(const ASTPtr & node, bool atomic) const
{
if (!node)
return true;
if (const auto * func = node->as<ASTFunction>())
{
if (key_columns.count(func->getColumnName()))
@ -423,7 +427,7 @@ bool MergeTreeIndexConditionSet::checkASTUseless(const ASTPtr &node, bool atomic
return checkASTUseless(args[0], atomic);
else
return std::any_of(args.begin(), args.end(),
[this](const auto & arg) { return checkASTUseless(arg, true); });
[this](const auto & arg) { return checkASTUseless(arg, true); });
}
else if (const auto * literal = node->as<ASTLiteral>())
return !atomic && literal->value.get<bool>();

View File

@ -80,7 +80,7 @@ private:
bool atomFromAST(ASTPtr & node) const;
bool operatorFromAST(ASTPtr & node) const;
bool checkASTUseless(const ASTPtr &node, bool atomic = false) const;
bool checkASTUseless(const ASTPtr & node, bool atomic = false) const;
const MergeTreeIndexSet & index;

View File

@ -158,6 +158,7 @@ public:
}
reader->readSuffix();
reader.reset();
}
}

View File

@ -14,7 +14,7 @@ Don't use Docker from your system repository.
* [pip](https://pypi.python.org/pypi/pip) and `libpq-dev`. To install: `sudo apt-get install python-pip libpq-dev`
* [py.test](https://docs.pytest.org/) testing framework. To install: `sudo -H pip install pytest`
* [docker-compose](https://docs.docker.com/compose/) and additional python libraries. To install: `sudo -H pip install docker-compose docker dicttoxml kazoo PyMySQL psycopg2 pymongo tzlocal kafka-python protobuf pytest-timeout minio`
* [docker-compose](https://docs.docker.com/compose/) and additional python libraries. To install: `sudo -H pip install docker-compose docker dicttoxml kazoo PyMySQL psycopg2 pymongo tzlocal kafka-python protobuf pytest-timeout minio rpm-confluent-schemaregistry`
(highly not recommended) If you really want to use OS packages on modern debian/ubuntu instead of "pip": `sudo apt install -y docker docker-compose python-pytest python-dicttoxml python-docker python-pymysql python-pymongo python-tzlocal python-kazoo python-psycopg2 python-kafka python-pytest-timeout python-minio`

View File

@ -1,4 +1,4 @@
[pytest]
python_files = test*.py
norecursedirs = _instances
timeout = 600
timeout = 300

View File

@ -88,14 +88,14 @@ SELECT PN, PVq, PVt FROM
ARRAY JOIN ProfileEvents.Names AS PN, ProfileEvents.Values AS PV
WHERE event_date >= today()-1 AND query_id='$query_id'
GROUP BY PN
)
) js1
ANY INNER JOIN
(
SELECT PN, PV AS PVq
FROM system.query_log
ARRAY JOIN ProfileEvents.Names AS PN, ProfileEvents.Values AS PV
WHERE event_date >= today()-1 AND query_id='$query_id'
)
) js2
USING PN
WHERE
NOT PN IN ('ContextLock') AND

View File

@ -18,12 +18,12 @@ ${CLICKHOUSE_CLIENT} --query="SELECT * FROM input_function_table_2 FORMAT CSV"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS input_function_table_3"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE input_function_table_3 (a String, b Date, c Int32, d Int16) ENGINE=Memory()"
cat ${CLICKHOUSE_TMP}/data_for_input_function.csv | ${CLICKHOUSE_CLIENT} --query="INSERT INTO input_function_table_3 (a, b, c) SELECT * FROM (SELECT s, b, c*c FROM input('s String, b Int32, c Int32') JOIN input_function_table_1 ON s=input_function_table_1.a) FORMAT CSV"
cat ${CLICKHOUSE_TMP}/data_for_input_function.csv | ${CLICKHOUSE_CLIENT} --query="INSERT INTO input_function_table_3 (a, b, c) SELECT * FROM (SELECT s, b, c*c FROM input('s String, b Int32, c Int32') js1 JOIN input_function_table_1 ON s=input_function_table_1.a) FORMAT CSV"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM input_function_table_3 FORMAT CSV"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS input_function_table_4"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE input_function_table_4 (a String, b Date, c Int32, d Int16) ENGINE=Memory()"
cat ${CLICKHOUSE_TMP}/data_for_input_function.csv | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=INSERT%20INTO%20input_function_table_4%20%28a%2C%20b%2C%20c%29%20SELECT%20%2A%20FROM%20%28SELECT%20s%2C%20b%2C%20c%2Ac%20FROM%20input%28%27s%20String%2C%20b%20Int32%2C%20c%20Int32%27%29%20JOIN%20input_function_table_1%20ON%20s%3Dinput_function_table_1.a%29%20FORMAT%20CSV" --data-binary @-
cat ${CLICKHOUSE_TMP}/data_for_input_function.csv | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=INSERT%20INTO%20input_function_table_4%20%28a%2C%20b%2C%20c%29%20SELECT%20%2A%20FROM%20%28SELECT%20s%2C%20b%2C%20c%2Ac%20FROM%20input%28%27s%20String%2C%20b%20Int32%2C%20c%20Int32%27%29%20js1%20JOIN%20input_function_table_1%20ON%20s%3Dinput_function_table_1.a%29%20FORMAT%20CSV" --data-binary @-
${CLICKHOUSE_CLIENT} --query="SELECT * FROM input_function_table_4 FORMAT CSV"

View File

@ -7,8 +7,8 @@ insert into test_join values ('2019-01-01', 2, 'b');
insert into test_join values ('2019-01-01', 3, 'c');
insert into test_join values ('2019-01-01', 1, null);
SELECT id, date, name FROM (SELECT id, date, name FROM test_join GROUP BY id, name, date)
FULL OUTER JOIN (SELECT id, date, name FROM test_join GROUP BY id, name, date)
SELECT id, date, name FROM (SELECT id, date, name FROM test_join GROUP BY id, name, date) js1
FULL OUTER JOIN (SELECT id, date, name FROM test_join GROUP BY id, name, date) js2
USING (id, name, date)
ORDER BY id, name;

View File

@ -1,8 +1,8 @@
0 ['left'] 0 ['left'] \N
1 ['left'] 1 ['left'] 1
2 [] \N [] 2
['left'] 0 ['left'] \N
['left'] 1 ['left'] 1
[] \N [] 2
0 ['left'] 0 [] \N
1 ['left'] 1 ['right'] 1
2 [] \N ['right'] 2
['left'] 0 [] \N
['left'] 1 ['right'] 1
[] \N ['right'] 2
['left'] 42 \N
['right'] \N 42

View File

@ -3,32 +3,32 @@ SET join_use_nulls = 1;
SELECT * FROM
(
SELECT number, ['left'] as ar, number AS left_number FROM system.numbers LIMIT 2
)
) js1
FULL JOIN
(
SELECT number, ['right'] as ar, number AS right_number FROM system.numbers LIMIT 1, 2
)
) js2
USING (number)
ORDER BY number;
SELECT * FROM
(
SELECT ['left'] as ar, number AS left_number FROM system.numbers LIMIT 2
)
) js1
FULL JOIN
(
SELECT ['right'] as ar, number AS right_number FROM system.numbers LIMIT 1, 2
)
) js2
ON left_number = right_number
ORDER BY left_number;
SELECT * FROM
(
SELECT ['left'] as ar, 42 AS left_number
)
) js1
FULL JOIN
(
SELECT ['right'] as ar, 42 AS right_number
)
) js2
USING(ar)
ORDER BY left_number;

View File

@ -5,7 +5,7 @@ DROP TABLE IF EXISTS view_foo_bar;
create table foo (ddate Date, id Int64, n String) ENGINE = ReplacingMergeTree(ddate, (id), 8192);
create table bar (ddate Date, id Int64, n String, foo_id Int64) ENGINE = ReplacingMergeTree(ddate, (id), 8192);
insert into bar (id, n, foo_id) values (1, 'bar_n_1', 1);
create MATERIALIZED view view_foo_bar ENGINE = ReplacingMergeTree(ddate, (bar_id), 8192) as select ddate, bar_id, bar_n, foo_id, foo_n from (select ddate, id as bar_id, n as bar_n, foo_id from bar) any left join (select id as foo_id, n as foo_n from foo) using foo_id;
create MATERIALIZED view view_foo_bar ENGINE = ReplacingMergeTree(ddate, (bar_id), 8192) as select ddate, bar_id, bar_n, foo_id, foo_n from (select ddate, id as bar_id, n as bar_n, foo_id from bar) js1 any left join (select id as foo_id, n as foo_n from foo) js2 using foo_id;
insert into bar (id, n, foo_id) values (1, 'bar_n_1', 1);
SELECT * FROM view_foo_bar;

View File

@ -1,7 +1,9 @@
drop table if exists tab;
create table tab (x UInt64) engine = MergeTree order by tuple();
insert into tab select number as n from numbers(20) semi left join (select number * 10 as n from numbers(2)) using(n) settings max_block_size = 5;
insert into tab select number as n from numbers(20) nums
semi left join (select number * 10 as n from numbers(2)) js2 using(n)
settings max_block_size = 5;
select * from tab order by x;
drop table tab;

View File

@ -1,6 +1,6 @@
SET join_algorithm = 'hash';
SELECT number as n, j FROM numbers(4)
SELECT number as n, j FROM numbers(4) nums
ANY LEFT JOIN (
SELECT number * 2 AS n, number + 10 AS j
FROM numbers(4000)
@ -9,7 +9,7 @@ USING n;
SET max_rows_in_join = 1000;
SELECT number as n, j FROM numbers(4)
SELECT number as n, j FROM numbers(4) nums
ANY LEFT JOIN (
SELECT number * 2 AS n, number + 10 AS j
FROM numbers(4000)
@ -18,7 +18,7 @@ USING n; -- { serverError 191 }
SET join_algorithm = 'partial_merge';
SELECT number as n, j FROM numbers(4)
SELECT number as n, j FROM numbers(4) nums
ANY LEFT JOIN (
SELECT number * 2 AS n, number + 10 AS j
FROM numbers(4000)
@ -27,7 +27,7 @@ USING n;
SET partial_merge_join_optimizations = 1;
SELECT number as n, j FROM numbers(4)
SELECT number as n, j FROM numbers(4) nums
ANY LEFT JOIN (
SELECT number * 2 AS n, number + 10 AS j
FROM numbers(4000)
@ -36,7 +36,7 @@ USING n;
SET join_algorithm = 'auto';
SELECT number as n, j FROM numbers(4)
SELECT number as n, j FROM numbers(4) nums
ANY LEFT JOIN (
SELECT number * 2 AS n, number + 10 AS j
FROM numbers(4000)

View File

@ -1,6 +1,6 @@
SET max_memory_usage = 32000000;
SELECT number * 200000 as n, j FROM numbers(5)
SELECT number * 200000 as n, j FROM numbers(5) nums
ANY LEFT JOIN (
SELECT number * 2 AS n, number AS j
FROM numbers(1000000)
@ -10,14 +10,14 @@ USING n; -- { serverError 241 }
SET partial_merge_join = 1;
SET default_max_bytes_in_join = 0;
SELECT number * 200000 as n, j FROM numbers(5)
SELECT number * 200000 as n, j FROM numbers(5) nums
ANY LEFT JOIN (
SELECT number * 2 AS n, number AS j
FROM numbers(1000000)
) js2
USING n; -- { serverError 12 }
SELECT number * 200000 as n, j FROM numbers(5)
SELECT number * 200000 as n, j FROM numbers(5) nums
ANY LEFT JOIN (
SELECT number * 2 AS n, number AS j
FROM numbers(1000000)
@ -25,7 +25,7 @@ ANY LEFT JOIN (
USING n
SETTINGS max_bytes_in_join = 30000000; -- { serverError 241 }
SELECT number * 200000 as n, j FROM numbers(5)
SELECT number * 200000 as n, j FROM numbers(5) nums
ANY LEFT JOIN (
SELECT number * 2 AS n, number AS j
FROM numbers(1000000)
@ -36,7 +36,7 @@ SETTINGS max_bytes_in_join = 10000000;
SET partial_merge_join_optimizations = 1;
SELECT number * 200000 as n, j FROM numbers(5)
SELECT number * 200000 as n, j FROM numbers(5) nums
LEFT JOIN (
SELECT number * 2 AS n, number AS j
FROM numbers(1000000)
@ -47,7 +47,7 @@ SETTINGS max_rows_in_join = 100000;
SET default_max_bytes_in_join = 10000000;
SELECT number * 200000 as n, j FROM numbers(5)
SELECT number * 200000 as n, j FROM numbers(5) nums
JOIN (
SELECT number * 2 AS n, number AS j
FROM numbers(1000000)

View File

@ -1,3 +1,5 @@
SET joined_subquery_requires_alias = 0;
SELECT 'IN empty set',count() FROM system.numbers WHERE number IN (SELECT toUInt64(1) WHERE 0);
SELECT 'IN non-empty set',count() FROM (SELECT number FROM system.numbers LIMIT 10) t1 WHERE t1.number IN (SELECT toUInt64(1) WHERE 1);
SELECT 'NOT IN empty set',count() FROM (SELECT number FROM system.numbers LIMIT 10) WHERE number NOT IN (SELECT toUInt64(1) WHERE 0);

View File

@ -0,0 +1,5 @@
DROP TABLE IF EXISTS tab;
create table tab (A Int64) Engine=MergeTree order by tuple();
insert into tab select cityHash64(number) from numbers(1000);
select sum(sleep(0.1)) from tab settings max_block_size = 1, max_execution_time=1; -- { serverError 159 }
DROP TABLE IF EXISTS tab;

View File

@ -5,7 +5,7 @@ DROP TABLE IF EXISTS t_d;
DROP TABLE IF EXISTS t_v;
CREATE TABLE t (`A` Int64) ENGINE = MergeTree() ORDER BY tuple();
CREATE TABLE t_d AS t ENGINE = Distributed(test_shard_localhost, currentDatabase(), t);
CREATE MATERIALIZED VIEW t_v ENGINE = MergeTree() ORDER BY tuple() AS SELECT A FROM t LEFT JOIN ( SELECT toInt64(dummy) AS A FROM system.one ) USING (A);
CREATE MATERIALIZED VIEW t_v ENGINE = MergeTree() ORDER BY tuple() AS SELECT A FROM t LEFT JOIN ( SELECT toInt64(dummy) AS A FROM system.one ) js2 USING (A);
INSERT INTO t_d SELECT number FROM numbers(2);
SELECT * FROM t_v ORDER BY A;

View File

@ -7,13 +7,13 @@ CREATE TABLE Beta (foo LowCardinality(String), baz UInt64) ENGINE = Memory;
INSERT INTO Alpha VALUES ('a', 1);
INSERT INTO Beta VALUES ('a', 2), ('b', 3);
SELECT * FROM Alpha FULL JOIN (SELECT 'b' as foo) USING (foo) ORDER BY foo;
SELECT * FROM Alpha FULL JOIN (SELECT 'b' as foo) js2 USING (foo) ORDER BY foo;
SELECT * FROM Alpha FULL JOIN Beta USING (foo) ORDER BY foo;
SELECT * FROM Alpha FULL JOIN Beta ON Alpha.foo = Beta.foo ORDER BY foo;
SET join_use_nulls = 1;
SELECT * FROM Alpha FULL JOIN (SELECT 'b' as foo) USING (foo) ORDER BY foo;
SELECT * FROM Alpha FULL JOIN (SELECT 'b' as foo) js2 USING (foo) ORDER BY foo;
SELECT * FROM Alpha FULL JOIN Beta USING (foo) ORDER BY foo;
SELECT * FROM Alpha FULL JOIN Beta ON Alpha.foo = Beta.foo ORDER BY foo;

View File

@ -7,9 +7,9 @@ CREATE TABLE testJoinTable (number UInt64, data String) ENGINE = Join(ANY, INNER
INSERT INTO testJoinTable VALUES (1, '1'), (2, '2'), (3, '3');
SELECT * FROM (SELECT * FROM numbers(10)) INNER JOIN testJoinTable USING number; -- { serverError 264 }
SELECT * FROM (SELECT * FROM numbers(10)) INNER JOIN (SELECT * FROM testJoinTable) USING number;
SELECT * FROM (SELECT * FROM numbers(10)) ANY INNER JOIN testJoinTable USING number;
SELECT * FROM (SELECT * FROM numbers(10)) js1 INNER JOIN testJoinTable USING number; -- { serverError 264 }
SELECT * FROM (SELECT * FROM numbers(10)) js1 INNER JOIN (SELECT * FROM testJoinTable) js2 USING number;
SELECT * FROM (SELECT * FROM numbers(10)) js1 ANY INNER JOIN testJoinTable USING number;
SELECT * FROM testJoinTable;
DROP TABLE testJoinTable;

View File

@ -11,9 +11,9 @@ INSERT INTO a VALUES (1,1,1)(2,2,2)(3,3,3);
INSERT INTO id1 VALUES (1,1)(2,2)(3,3);
INSERT INTO id2 VALUES (1,1)(2,2)(3,3);
SELECT * from (SELECT * FROM a ANY LEFT OUTER JOIN id1 USING id1) ANY LEFT OUTER JOIN id2 USING id2;
SELECT * from (SELECT * FROM a ANY LEFT OUTER JOIN id1 USING id1) js1 ANY LEFT OUTER JOIN id2 USING id2;
create view b as (SELECT * from (SELECT * FROM a ANY LEFT OUTER JOIN id1 USING id1) ANY LEFT OUTER JOIN id2 USING id2);
create view b as (SELECT * from (SELECT * FROM a ANY LEFT OUTER JOIN id1 USING id1) js1 ANY LEFT OUTER JOIN id2 USING id2);
SELECT '-';
SELECT * FROM b;

View File

@ -1,5 +1,6 @@
SET enable_debug_queries = 1;
SET enable_optimize_predicate_expression = 1;
SET joined_subquery_requires_alias = 0;
-- https://github.com/ClickHouse/ClickHouse/issues/3885
-- https://github.com/ClickHouse/ClickHouse/issues/5485

View File

@ -4,17 +4,17 @@ SET partial_merge_join = 1;
SELECT 'defaults';
SELECT count(1) FROM (
SELECT materialize(1) as k, n FROM numbers(10)
SELECT materialize(1) as k, n FROM numbers(10) nums
JOIN (SELECT materialize(1) AS k, number n FROM numbers(1000000)) j
USING k);
SELECT count(1) FROM (
SELECT materialize(1) as k, n FROM numbers(1000)
SELECT materialize(1) as k, n FROM numbers(1000) nums
JOIN (SELECT materialize(1) AS k, number n FROM numbers(10000)) j
USING k);
SELECT count(1), uniqExact(n) FROM (
SELECT materialize(1) as k, n FROM numbers(1000000)
SELECT materialize(1) as k, n FROM numbers(1000000) nums
JOIN (SELECT materialize(1) AS k, number n FROM numbers(10)) j
USING k);
@ -22,12 +22,12 @@ SELECT count(1), uniqExact(n) FROM (
SET max_joined_block_size_rows = 0;
SELECT count(1) FROM (
SELECT materialize(1) as k, n FROM numbers(10)
SELECT materialize(1) as k, n FROM numbers(10) nums
JOIN (SELECT materialize(1) AS k, number n FROM numbers(1000000)) j
USING k); -- { serverError 241 }
SELECT count(1) FROM (
SELECT materialize(1) as k, n FROM numbers(1000)
SELECT materialize(1) as k, n FROM numbers(1000) nums
JOIN (SELECT materialize(1) AS k, number n FROM numbers(10000)) j
USING k); -- { serverError 241 }
@ -35,17 +35,17 @@ SELECT 'max_joined_block_size_rows = 2000';
SET max_joined_block_size_rows = 2000;
SELECT count(1) FROM (
SELECT materialize(1) as k, n FROM numbers(10)
SELECT materialize(1) as k, n FROM numbers(10) nums
JOIN (SELECT materialize(1) AS k, number n FROM numbers(1000000)) j
USING k);
SELECT count(1), uniqExact(n) FROM (
SELECT materialize(1) as k, n FROM numbers(1000)
SELECT materialize(1) as k, n FROM numbers(1000) nums
JOIN (SELECT materialize(1) AS k, number n FROM numbers(10000)) j
USING k);
SELECT count(1), uniqExact(n) FROM (
SELECT materialize(1) as k, n FROM numbers(1000000)
SELECT materialize(1) as k, n FROM numbers(1000000) nums
JOIN (SELECT materialize(1) AS k, number n FROM numbers(10)) j
USING k);
@ -53,16 +53,16 @@ SELECT 'max_rows_in_join = 1000';
SET max_rows_in_join = 1000;
SELECT count(1) FROM (
SELECT materialize(1) as k, n FROM numbers(10)
SELECT materialize(1) as k, n FROM numbers(10) nums
JOIN (SELECT materialize(1) AS k, number n FROM numbers(1000000)) j
USING k);
SELECT count(1), uniqExact(n) FROM (
SELECT materialize(1) as k, n FROM numbers(1000)
SELECT materialize(1) as k, n FROM numbers(1000) nums
JOIN (SELECT materialize(1) AS k, number n FROM numbers(10000)) j
USING k);
SELECT count(1), uniqExact(n) FROM (
SELECT materialize(1) as k, n FROM numbers(1000000)
SELECT materialize(1) as k, n FROM numbers(1000000) nums
JOIN (SELECT materialize(1) AS k, number n FROM numbers(10)) j
USING k);

View File

@ -6,7 +6,7 @@ FROM mt
ANY LEFT JOIN
(
SELECT 1 AS x
) USING (x)
) js2 USING (x)
PREWHERE x IN (1) WHERE y = today();
DROP TABLE mt;

View File

@ -1,3 +1,5 @@
SET joined_subquery_requires_alias = 0;
SYSTEM STOP MERGES;
-- incremental streaming usecase

View File

@ -1,87 +1,87 @@
SET partial_merge_join = 1;
SELECT count(1), uniqExact(1) FROM (
SELECT materialize(1) as k FROM numbers(1)
SELECT materialize(1) as k FROM numbers(1) nums
JOIN (SELECT materialize(1) AS k, number n FROM numbers(100000)) j
USING k);
SELECT count(1), uniqExact(1) FROM (
SELECT materialize(1) as k FROM numbers(1)
SELECT materialize(1) as k FROM numbers(1) nums
JOIN (SELECT 1 AS k, number n FROM numbers(100000)) j
USING k);
SELECT count(1), uniqExact(1) FROM (
SELECT 1 as k FROM numbers(1)
SELECT 1 as k FROM numbers(1) nums
JOIN (SELECT materialize(1) AS k, number n FROM numbers(100000)) j
USING k);
SELECT count(1), uniqExact(1) FROM (
SELECT 1 as k FROM numbers(1)
SELECT 1 as k FROM numbers(1) nums
JOIN (SELECT 1 AS k, number n FROM numbers(100000)) j
USING k);
SELECT 'first nullable';
SELECT count(1), uniqExact(1) FROM (
SELECT materialize(toNullable(1)) as k FROM numbers(1)
SELECT materialize(toNullable(1)) as k FROM numbers(1) nums
JOIN (SELECT materialize(1) AS k, number n FROM numbers(100000)) j
USING k);
SELECT count(1), uniqExact(1) FROM (
SELECT materialize(toNullable(1)) as k FROM numbers(1)
SELECT materialize(toNullable(1)) as k FROM numbers(1) nums
JOIN (SELECT 1 AS k, number n FROM numbers(100000)) j
USING k);
SELECT count(1), uniqExact(1) FROM (
SELECT toNullable(1) as k FROM numbers(1)
SELECT toNullable(1) as k FROM numbers(1) nums
JOIN (SELECT materialize(1) AS k, number n FROM numbers(100000)) j
USING k);
SELECT count(1), uniqExact(1) FROM (
SELECT toNullable(1) as k FROM numbers(1)
SELECT toNullable(1) as k FROM numbers(1) nums
JOIN (SELECT 1 AS k, number n FROM numbers(100000)) j
USING k);
SELECT 'second nullable';
SELECT count(1), uniqExact(1) FROM (
SELECT materialize(1) as k FROM numbers(1)
SELECT materialize(1) as k FROM numbers(1) nums
JOIN (SELECT materialize(toNullable(1)) AS k, number n FROM numbers(100000)) j
USING k);
SELECT count(1), uniqExact(1) FROM (
SELECT materialize(1) as k FROM numbers(1)
SELECT materialize(1) as k FROM numbers(1) nums
JOIN (SELECT toNullable(1) AS k, number n FROM numbers(100000)) j
USING k);
SELECT count(1), uniqExact(1) FROM (
SELECT 1 as k FROM numbers(1)
SELECT 1 as k FROM numbers(1) nums
JOIN (SELECT materialize(toNullable(1)) AS k, number n FROM numbers(100000)) j
USING k);
SELECT count(1), uniqExact(1) FROM (
SELECT 1 as k FROM numbers(1)
SELECT 1 as k FROM numbers(1) nums
JOIN (SELECT toNullable(1) AS k, number n FROM numbers(100000)) j
USING k);
SELECT 'both nullable';
SELECT count(1), uniqExact(1) FROM (
SELECT materialize(toNullable(1)) as k FROM numbers(1)
SELECT materialize(toNullable(1)) as k FROM numbers(1) nums
JOIN (SELECT materialize(toNullable(1)) AS k, number n FROM numbers(100000)) j
USING k);
SELECT count(1), uniqExact(1) FROM (
SELECT materialize(toNullable(1)) as k FROM numbers(1)
SELECT materialize(toNullable(1)) as k FROM numbers(1) nums
JOIN (SELECT toNullable(1) AS k, number n FROM numbers(100000)) j
USING k);
SELECT count(1), uniqExact(1) FROM (
SELECT toNullable(1) as k FROM numbers(1)
SELECT toNullable(1) as k FROM numbers(1) nums
JOIN (SELECT materialize(toNullable(1)) AS k, number n FROM numbers(100000)) j
USING k);
SELECT count(1), uniqExact(1) FROM (
SELECT toNullable(1) as k FROM numbers(1)
SELECT toNullable(1) as k FROM numbers(1) nums
JOIN (SELECT toNullable(1) AS k, number n FROM numbers(100000)) j
USING k);

View File

@ -6,7 +6,7 @@ FROM
SELECT NULL
UNION ALL
SELECT NULL
)
) js1
ALL FULL OUTER JOIN
(
SELECT 1 AS id
@ -14,7 +14,7 @@ ALL FULL OUTER JOIN
SELECT NULL
UNION ALL
SELECT NULL
) USING (id)
) js2 USING (id)
ORDER BY id;
SELECT '---';
@ -23,11 +23,11 @@ SELECT *
FROM
(
SELECT NULL AS x
)
) js1
INNER JOIN
(
SELECT NULL AS x
) USING (x);
) js2 USING (x);
SELECT '---';
@ -35,8 +35,8 @@ SELECT *
FROM
(
SELECT NULL AS x
)
) js1
FULL OUTER JOIN
(
SELECT NULL AS x
) USING (x);
) js2 USING (x);

View File

@ -19,7 +19,7 @@ for i in `seq $REPLICAS`; do
done
for i in `seq $REPLICAS`; do
$CLICKHOUSE_CLIENT --query "CREATE TABLE concurrent_mutate_mt_$i (key UInt64, value1 UInt64, value2 String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/concurrent_mutate_mt', '$i') ORDER BY key SETTINGS max_replicated_mutations_in_queue=1000, number_of_free_entries_in_pool_to_execute_mutation=0,max_replicated_merges_in_queue=1000"
$CLICKHOUSE_CLIENT --query "CREATE TABLE concurrent_mutate_mt_$i (key UInt64, value1 UInt64, value2 String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/concurrent_mutate_mt', '$i') ORDER BY key SETTINGS max_replicated_mutations_in_queue=1000, number_of_free_entries_in_pool_to_execute_mutation=0,max_replicated_merges_in_queue=1000,temporary_directories_lifetime=10"
done
$CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_mutate_mt_1 SELECT number, number + 10, toString(number) from numbers(10)"

View File

@ -2,7 +2,7 @@ drop table if exists test_01081;
create table test_01081 (key Int) engine=MergeTree() order by key;
insert into test_01081 select * from system.numbers limit 10;
select 1 from remote('127.{1,2}', currentDatabase(), test_01081) join system.one as rhs on rhs.dummy = 1 order by 1;
select 1 from remote('127.{1,2}', currentDatabase(), test_01081) lhs join system.one as rhs on rhs.dummy = 1 order by 1;
-- With multiple blocks triggers:
--
@ -13,6 +13,6 @@ select 1 from remote('127.{1,2}', currentDatabase(), test_01081) join system.one
-- With experimental_use_processors=1 (default at the time of writing).
insert into test_01081 select * from system.numbers limit 10;
select 1 from remote('127.{1,2}', currentDatabase(), test_01081) join system.one as rhs on rhs.dummy = 1 order by 1;
select 1 from remote('127.{1,2}', currentDatabase(), test_01081) lhs join system.one as rhs on rhs.dummy = 1 order by 1;
drop table if exists test_01081;

View File

@ -0,0 +1,4 @@
1 1
1 2
1 3
1 4

View File

@ -0,0 +1,5 @@
DROP TABLE IF EXISTS t;
create table t (i Int, a Int, s String, index ind_s (s) type set(1) granularity 1) engine = MergeTree order by i;
insert into t values (1, 1, 'a') (2, 1, 'a') (3, 1, 'a') (4, 1, 'a');
SELECT a, i from t ORDER BY a, i;
DROP TABLE t;

View File

@ -25,6 +25,7 @@ find . -name '*.so.*' -print -exec mv '{}' /output \;
if [ "performance" == "$COMBINED_OUTPUT" ]
then
cp -r ../dbms/tests/performance /output
cp -r ../docker/test/performance-comparison/config /output ||:
rm /output/unit_tests_dbms ||:
rm /output/clickhouse-odbc-bridge ||:
fi

View File

@ -53,12 +53,12 @@ function download
function configure
{
sed -i 's/<tcp_port>9000/<tcp_port>9001/g' left/config/config.xml
sed -i 's/<tcp_port>9000/<tcp_port>9002/g' right/config/config.xml
# Use the new config for both servers, so that we can change it in a PR.
mkdir right/config/users.d ||:
mkdir left/config/users.d ||:
mkdir right/config/config.d ||:
# FIXME delete these two configs when the performance.tgz with configs rolls out.
cat > right/config/config.d/zz-perf-test-tweaks-config.xml <<EOF
<yandex>
<logger>
@ -67,10 +67,7 @@ function configure
<text_log remove="remove">
<table remove="remove"/>
</text_log>
<metric_log remove="remove">
<table remove="remove"/>
</metric_log>
<use_uncompressed_cache>1</use_uncompressed_cache>
<use_uncompressed_cache>0</use_uncompressed_cache>
<!--1 GB-->
<uncompressed_cache_size>1000000000</uncompressed_cache_size>
</yandex>
@ -89,14 +86,13 @@ EOF
</yandex>
EOF
cp right/config/config.d/zz-perf-test-tweaks-config.xml left/config/config.d/zz-perf-test-tweaks-config.xml
cp right/config/users.d/zz-perf-test-tweaks-users.xml left/config/users.d/zz-perf-test-tweaks-users.xml
rm left/config/config.d/metric_log.xml ||:
rm left/config/config.d/text_log.xml ||:
rm right/config/config.d/metric_log.xml ||:
rm right/config/config.d/text_log.xml ||:
cp -rv right/config left ||:
sed -i 's/<tcp_port>9000/<tcp_port>9001/g' left/config/config.xml
sed -i 's/<tcp_port>9000/<tcp_port>9002/g' right/config/config.xml
# Start a temporary server to rename the tables
while killall clickhouse; do echo . ; sleep 1 ; done
echo all killed
@ -120,12 +116,6 @@ EOF
# servers with hardlink might cause unpredictable behavior.
rm db0/data/system/* -rf ||:
rm db0/metadata/system/* -rf ||:
}
function restart
{
while killall clickhouse; do echo . ; sleep 1 ; done
echo all killed
# Make copies of the original db for both servers. Use hardlinks instead
# of copying.
@ -133,15 +123,21 @@ function restart
rm -r right/db ||:
cp -al db0/ left/db/
cp -al db0/ right/db/
}
function restart
{
while killall clickhouse; do echo . ; sleep 1 ; done
echo all killed
set -m # Spawn servers in their own process groups
left/clickhouse server --config-file=left/config/config.xml -- --path left/db &> left-server-log.log &
left/clickhouse server --config-file=left/config/config.xml -- --path left/db &>> left-server-log.log &
left_pid=$!
kill -0 $left_pid
disown $left_pid
right/clickhouse server --config-file=right/config/config.xml -- --path right/db &> right-server-log.log &
right/clickhouse server --config-file=right/config/config.xml -- --path right/db &>> right-server-log.log &
right_pid=$!
kill -0 $right_pid
disown $right_pid
@ -180,7 +176,7 @@ function run_tests
# if some performance tests xmls were changed in a PR, run only these ones.
if [ "$PR_TO_TEST" != "0" ]
then
test_files_override=$(cd right/performance && readlink -e $changed_files ||:)
test_files_override=$(sed 's/dbms\/tests/right/' changed-tests.txt)
if [ "$test_files_override" != "" ]
then
test_files=$test_files_override
@ -193,9 +189,16 @@ function run_tests
test_files=$(ls right/performance/${CHPC_TEST_GLOB}.xml)
fi
# Run the tests
# Run the tests.
test_name="<none>"
for test in $test_files
do
# Check that both servers are alive, to fail faster if they die.
left/clickhouse client --port 9001 --query "select 1 format Null" \
|| { echo $test_name >> left-server-died.log ; restart ; continue ; }
right/clickhouse client --port 9002 --query "select 1 format Null" \
|| { echo $test_name >> right-server-died.log ; restart ; continue ; }
test_name=$(basename $test ".xml")
echo test $test_name
@ -208,9 +211,6 @@ function run_tests
# this may be slow, run it in background
right/clickhouse local --file "$test_name-queries.tsv" --structure 'query text, run int, version UInt32, time float' --query "$(cat $script_dir/eqmed.sql)" > "$test_name-report.tsv" &
# Check that both servers are alive, to fail faster if they die.
left/clickhouse client --port 9001 --query "select 1 format Null"
right/clickhouse client --port 9002 --query "select 1 format Null"
done
unset TIMEFORMAT
@ -229,10 +229,12 @@ function get_profiles
left/clickhouse client --port 9001 --query "select * from system.query_log where type = 2 format TSVWithNamesAndTypes" > left-query-log.tsv ||: &
left/clickhouse client --port 9001 --query "select * from system.trace_log format TSVWithNamesAndTypes" > left-trace-log.tsv ||: &
left/clickhouse client --port 9001 --query "select arrayJoin(trace) addr, concat(splitByChar('/', addressToLine(addr))[-1], '#', demangle(addressToSymbol(addr)) ) name from system.trace_log group by addr format TSVWithNamesAndTypes" > left-addresses.tsv ||: &
left/clickhouse client --port 9001 --query "select * from system.metric_log format TSVWithNamesAndTypes" > left-metric-log.tsv ||: &
right/clickhouse client --port 9002 --query "select * from system.query_log where type = 2 format TSVWithNamesAndTypes" > right-query-log.tsv ||: &
right/clickhouse client --port 9002 --query "select * from system.trace_log format TSVWithNamesAndTypes" > right-trace-log.tsv ||: &
right/clickhouse client --port 9002 --query "select arrayJoin(trace) addr, concat(splitByChar('/', addressToLine(addr))[-1], '#', demangle(addressToSymbol(addr)) ) name from system.trace_log group by addr format TSVWithNamesAndTypes" > right-addresses.tsv ||: &
right/clickhouse client --port 9002 --query "select * from system.metric_log format TSVWithNamesAndTypes" > right-metric-log.tsv ||: &
wait
}
@ -241,7 +243,7 @@ function get_profiles
function report
{
for x in {right,left}-{addresses,{query,trace}-log}.tsv
for x in {right,left}-{addresses,{query,trace,metric}-log}.tsv
do
# FIXME This loop builds column definitons from TSVWithNamesAndTypes in an
# absolutely atrocious way. This should be done by the file() function itself.
@ -422,7 +424,10 @@ case "$stage" in
time restart
;&
"run_tests")
time run_tests
# If the tests fail with OOM or something, still try to restart the servers
# to collect the logs.
time run_tests ||:
time restart
;&
"get_profiles")
time get_profiles

View File

@ -0,0 +1,20 @@
<yandex>
<logger>
<console>true</console>
</logger>
<text_log remove="remove">
<table remove="remove"/>
</text_log>
<metric_log replace="replace">
<database>system</database>
<table>metric_log</table>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
<collect_interval_milliseconds>1000</collect_interval_milliseconds>
</metric_log>
<use_uncompressed_cache>0</use_uncompressed_cache>
<!--64 GiB-->
<uncompressed_cache_size>1000000000</uncompressed_cache_size>
</yandex>

View File

@ -0,0 +1,10 @@
<yandex>
<profiles>
<default>
<query_profiler_real_time_period_ns>10000000</query_profiler_real_time_period_ns>
<query_profiler_cpu_time_period_ns>0</query_profiler_cpu_time_period_ns>
<allow_introspection_functions>1</allow_introspection_functions>
<log_queries>1</log_queries>
</default>
</profiles>
</yandex>

View File

@ -42,6 +42,8 @@ done
echo
) | tee right-commit.txt
(cd ch && git diff --name-only $SHA_TO_TEST $(git merge-base $SHA_TO_TEST master) -- dbms/tests/performance) | tee changed-tests.txt
# Set python output encoding so that we can print queries with Russian letters.
export PYTHONIOENCODING=utf-8
@ -57,7 +59,7 @@ set -m
time ../compare.sh 0 $ref_sha $PR_TO_TEST $SHA_TO_TEST 2>&1 | ts "$(printf '%%Y-%%m-%%d %%H:%%M:%%S\t')" | tee compare.log
set +m
dmesg > dmesg.log
dmesg -T > dmesg.log
7z a /output/output.7z *.log *.tsv *.html *.txt *.rep *.svg
7z a /output/output.7z *.log *.tsv *.html *.txt *.rep *.svg {right,left}/db/preprocessed_configs
cp compare.log /output

View File

@ -783,6 +783,10 @@ test: string with 'quotes' and with some special
This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table).
## VerticalRaw {#verticalraw}
Similar to [Vertical](#vertical), but with escaping disabled. This format is only suitable for outputting query results, not for parsing (receiving data and inserting it in the table).
## XML {#xml}
XML format is suitable only for output, not for parsing. Example:

View File

@ -28,6 +28,11 @@ Format | INSERT | SELECT
[PrettyCompactMonoBlock](formats.md#prettycompactmonoblock) | ✗ | ✔ |
[PrettyNoEscapes](formats.md#prettynoescapes) | ✗ | ✔ |
[PrettySpace](formats.md#prettyspace) | ✗ | ✔ |
[Protobuf](#protobuf) | ✔ | ✔ |
[Avro](#data-format-avro) | ✔ | ✔ |
[AvroConfluent](#data-format-avro-confluent) | ✔ | ✗ |
[Parquet](#data-format-parquet) | ✔ | ✔ |
[ORC](#data-format-orc) | ✔ | ✗ |
[RowBinary](formats.md#rowbinary) | ✔ | ✔ |
[Native](formats.md#native) | ✔ | ✔ |
[Null](formats.md#null) | ✗ | ✔ |
@ -750,4 +755,273 @@ struct Message {
</div>
## Protobuf {#protobuf}
Protobuf - is a [Protocol Buffers](https://developers.google.com/protocol-buffers/) format.
This format requires an external format schema. The schema is cached between queries.
ClickHouse supports both `proto2` and `proto3` syntaxes. Repeated/optional/required fields are supported.
Usage examples:
```sql
SELECT * FROM test.table FORMAT Protobuf SETTINGS format_schema = 'schemafile:MessageType'
```
```bash
cat protobuf_messages.bin | clickhouse-client --query "INSERT INTO test.table FORMAT Protobuf SETTINGS format_schema='schemafile:MessageType'"
```
where the file `schemafile.proto` looks like this:
```capnp
syntax = "proto3";
message MessageType {
string name = 1;
string surname = 2;
uint32 birthDate = 3;
repeated string phoneNumbers = 4;
};
```
To find the correspondence between table columns and fields of Protocol Buffers' message type ClickHouse compares their names.
This comparison is case-insensitive and the characters `_` (underscore) and `.` (dot) are considered as equal.
If types of a column and a field of Protocol Buffers' message are different the necessary conversion is applied.
Nested messages are supported. For example, for the field `z` in the following message type
```capnp
message MessageType {
message XType {
message YType {
int32 z;
};
repeated YType y;
};
XType x;
};
```
ClickHouse tries to find a column named `x.y.z` (or `x_y_z` or `X.y_Z` and so on).
Nested messages are suitable to input or output a [nested data structures](../data_types/nested_data_structures/nested.md).
Default values defined in a protobuf schema like this
```capnp
syntax = "proto2";
message MessageType {
optional int32 result_per_page = 3 [default = 10];
}
```
are not applied; the [table defaults](../query_language/create.md#create-default-values) are used instead of them.
ClickHouse inputs and outputs protobuf messages in the `length-delimited` format.
It means before every message should be written its length as a [varint](https://developers.google.com/protocol-buffers/docs/encoding#varints).
See also [how to read/write length-delimited protobuf messages in popular languages](https://cwiki.apache.org/confluence/display/GEODE/Delimiting+Protobuf+Messages).
## Avro {#data-format-avro}
[Apache Avro](http://avro.apache.org/) is a row-oriented data serialization framework developed within Apache's Hadoop project.
ClickHouse Avro format supports reading and writing [Avro data files](http://avro.apache.org/docs/current/spec.html#Object+Container+Files).
### Data Types Matching
The table below shows supported data types and how they match ClickHouse [data types](../data_types/index.md) in `INSERT` and `SELECT` queries.
| Avro data type `INSERT` | ClickHouse data type | Avro data type `SELECT` |
| -------------------- | -------------------- | ------------------ |
| `boolean`, `int`, `long`, `float`, `double` | [Int(8\|16\|32\)](../data_types/int_uint.md), [UInt(8\|16\|32)](../data_types/int_uint.md) | `int` |
| `boolean`, `int`, `long`, `float`, `double` | [Int64](../data_types/int_uint.md), [UInt64](../data_types/int_uint.md) | `long` |
| `boolean`, `int`, `long`, `float`, `double` | [Float32](../data_types/float.md) | `float` |
| `boolean`, `int`, `long`, `float`, `double` | [Float64](../data_types/float.md) | `double` |
| `bytes`, `string`, `fixed`, `enum` | [String](../data_types/string.md) | `bytes` |
| `bytes`, `string`, `fixed` | [FixedString(N)](../data_types/fixedstring.md) | `fixed(N)` |
| `enum` | [Enum(8\|16)](../data_types/enum.md) | `enum` |
| `array(T)` | [Array(T)](../data_types/array.md) | `array(T)` |
| `union(null, T)`, `union(T, null)` | [Nullable(T)](../data_types/date.md) | `union(null, T)`|
| `null` | [Nullable(Nothing)](../data_types/special_data_types/nothing.md) | `null` |
| `int (date)` * | [Date](../data_types/date.md) | `int (date)` * |
| `long (timestamp-millis)` * | [DateTime64(3)](../data_types/datetime.md) | `long (timestamp-millis)` * |
| `long (timestamp-micros)` * | [DateTime64(6)](../data_types/datetime.md) | `long (timestamp-micros)` * |
\* [Avro logical types](http://avro.apache.org/docs/current/spec.html#Logical+Types)
Unsupported Avro data types: `record` (non-root), `map`
Unsupported Avro logical data types: `uuid`, `time-millis`, `time-micros`, `duration`
### Inserting Data
To insert data from an Avro file into ClickHouse table:
```bash
$ cat file.avro | clickhouse-client --query="INSERT INTO {some_table} FORMAT Avro"
```
The root schema of input Avro file must be of `record` type.
To find the correspondence between table columns and fields of Avro schema ClickHouse compares their names. This comparison is case-sensitive.
Unused fields are skipped.
Data types of a ClickHouse table columns can differ from the corresponding fields of the Avro data inserted. When inserting data, ClickHouse interprets data types according to the table above and then [casts](../query_language/functions/type_conversion_functions/#type_conversion_function-cast) the data to corresponding column type.
### Selecting Data
To select data from ClickHouse table into an Avro file:
```bash
$ clickhouse-client --query="SELECT * FROM {some_table} FORMAT Avro" > file.avro
```
Column names must:
- start with `[A-Za-z_]`
- subsequently contain only `[A-Za-z0-9_]`
Output Avro file compression and sync interval can be configured with [output_format_avro_codec](../operations/settings/settings.md#settings-output_format_avro_codec) and [output_format_avro_sync_interval](../operations/settings/settings.md#settings-output_format_avro_sync_interval) respectively.
## AvroConfluent {#data-format-avro-confluent}
AvroConfluent supports decoding single-object Avro messages commonly used with [Kafka](https://kafka.apache.org/) and [Confluent Schema Registry](https://docs.confluent.io/current/schema-registry/index.html).
Each Avro message embeds a schema id that can be resolved to the actual schema with help of the Schema Registry.
Schemas are cached once resolved.
Schema Registry URL is configured with [format_avro_schema_registry_url](../operations/settings/settings.md#settings-format_avro_schema_registry_url)
### Data Types Matching
Same as [Avro](#data-format-avro)
### Usage
To quickly verify schema resolution you can use [kafkacat](https://github.com/edenhill/kafkacat) with [clickhouse-local](../operations/utils/clickhouse-local.md):
```bash
$ kafkacat -b kafka-broker -C -t topic1 -o beginning -f '%s' -c 3 | clickhouse-local --input-format AvroConfluent --format_avro_schema_registry_url 'http://schema-registry' -S "field1 Int64, field2 String" -q 'select * from table'
1 a
2 b
3 c
```
To use `AvroConfluent` with [Kafka](../operations/table_engines/kafka.md):
```sql
CREATE TABLE topic1_stream
(
field1 String,
field2 String
)
ENGINE = Kafka()
SETTINGS
kafka_broker_list = 'kafka-broker',
kafka_topic_list = 'topic1',
kafka_group_name = 'group1',
kafka_format = 'AvroConfluent';
SET format_avro_schema_registry_url = 'http://schema-registry';
SELECT * FROM topic1_stream;
```
!!! note "Warning"
Setting `format_avro_schema_registry_url` needs to be configured in `users.xml` to maintain it's value after a restart.
## Parquet {#data-format-parquet}
[Apache Parquet](http://parquet.apache.org/) is a columnar storage format widespread in the Hadoop ecosystem. ClickHouse supports read and write operations for this format.
### Data Types Matching
The table below shows supported data types and how they match ClickHouse [data types](../data_types/index.md) in `INSERT` and `SELECT` queries.
| Parquet data type (`INSERT`) | ClickHouse data type | Parquet data type (`SELECT`) |
| -------------------- | ------------------ | ---- |
| `UINT8`, `BOOL` | [UInt8](../data_types/int_uint.md) | `UINT8` |
| `INT8` | [Int8](../data_types/int_uint.md) | `INT8` |
| `UINT16` | [UInt16](../data_types/int_uint.md) | `UINT16` |
| `INT16` | [Int16](../data_types/int_uint.md) | `INT16` |
| `UINT32` | [UInt32](../data_types/int_uint.md) | `UINT32` |
| `INT32` | [Int32](../data_types/int_uint.md) | `INT32` |
| `UINT64` | [UInt64](../data_types/int_uint.md) | `UINT64` |
| `INT64` | [Int64](../data_types/int_uint.md) | `INT64` |
| `FLOAT`, `HALF_FLOAT` | [Float32](../data_types/float.md) | `FLOAT` |
| `DOUBLE` | [Float64](../data_types/float.md) | `DOUBLE` |
| `DATE32` | [Date](../data_types/date.md) | `UINT16` |
| `DATE64`, `TIMESTAMP` | [DateTime](../data_types/datetime.md) | `UINT32` |
| `STRING`, `BINARY` | [String](../data_types/string.md) | `STRING` |
| — | [FixedString](../data_types/fixedstring.md) | `STRING` |
| `DECIMAL` | [Decimal](../data_types/decimal.md) | `DECIMAL` |
ClickHouse supports configurable precision of `Decimal` type. The `INSERT` query treats the Parquet `DECIMAL` type as the ClickHouse `Decimal128` type.
Unsupported Parquet data types: `DATE32`, `TIME32`, `FIXED_SIZE_BINARY`, `JSON`, `UUID`, `ENUM`.
Data types of a ClickHouse table columns can differ from the corresponding fields of the Parquet data inserted. When inserting data, ClickHouse interprets data types according to the table above and then [cast](../query_language/functions/type_conversion_functions/#type_conversion_function-cast) the data to that data type which is set for the ClickHouse table column.
### Inserting and Selecting Data
You can insert Parquet data from a file into ClickHouse table by the following command:
```bash
$ cat {filename} | clickhouse-client --query="INSERT INTO {some_table} FORMAT Parquet"
```
You can select data from a ClickHouse table and save them into some file in the Parquet format by the following command:
```bash
$ clickhouse-client --query="SELECT * FROM {some_table} FORMAT Parquet" > {some_file.pq}
```
To exchange data with Hadoop, you can use [HDFS table engine](../operations/table_engines/hdfs.md).
## ORC {#data-format-orc}
[Apache ORC](https://orc.apache.org/) is a columnar storage format widespread in the Hadoop ecosystem. You can only insert data in this format to ClickHouse.
### Data Types Matching
The table below shows supported data types and how they match ClickHouse [data types](../data_types/index.md) in `INSERT` queries.
| ORC data type (`INSERT`) | ClickHouse data type |
| -------------------- | ------------------ |
| `UINT8`, `BOOL` | [UInt8](../data_types/int_uint.md) |
| `INT8` | [Int8](../data_types/int_uint.md) |
| `UINT16` | [UInt16](../data_types/int_uint.md) |
| `INT16` | [Int16](../data_types/int_uint.md) |
| `UINT32` | [UInt32](../data_types/int_uint.md) |
| `INT32` | [Int32](../data_types/int_uint.md) |
| `UINT64` | [UInt64](../data_types/int_uint.md) |
| `INT64` | [Int64](../data_types/int_uint.md) |
| `FLOAT`, `HALF_FLOAT` | [Float32](../data_types/float.md) |
| `DOUBLE` | [Float64](../data_types/float.md) |
| `DATE32` | [Date](../data_types/date.md) |
| `DATE64`, `TIMESTAMP` | [DateTime](../data_types/datetime.md) |
| `STRING`, `BINARY` | [String](../data_types/string.md) |
| `DECIMAL` | [Decimal](../data_types/decimal.md) |
ClickHouse supports configurable precision of the `Decimal` type. The `INSERT` query treats the ORC `DECIMAL` type as the ClickHouse `Decimal128` type.
Unsupported ORC data types: `DATE32`, `TIME32`, `FIXED_SIZE_BINARY`, `JSON`, `UUID`, `ENUM`.
The data types of ClickHouse table columns don't have to match the corresponding ORC data fields. When inserting data, ClickHouse interprets data types according to the table above and then [casts](../query_language/functions/type_conversion_functions/#type_conversion_function-cast) the data to the data type set for the ClickHouse table column.
### Inserting Data
You can insert ORC data from a file into ClickHouse table by the following command:
```bash
$ cat filename.orc | clickhouse-client --query="INSERT INTO some_table FORMAT ORC"
```
To exchange data with Hadoop, you can use [HDFS table engine](../operations/table_engines/hdfs.md).
[مقاله اصلی](https://clickhouse.tech/docs/fa/interfaces/formats/) <!--hide-->

View File

@ -1,6 +1,6 @@
# Начало работы
Если вы новичок в ClickHouse и хотите получить вживую оценить его производительность, прежде всего нужно пройти через [процесс установки](install.md).
Если вы новичок в ClickHouse и хотите вживую оценить его производительность, прежде всего нужно пройти через [процесс установки](install.md).
После этого можно выбрать один из следующих вариантов:

View File

@ -26,6 +26,7 @@ nav:
- ' کلاینت Command-line': 'interfaces/cli.md'
- 'Native interface (TCP)': 'interfaces/tcp.md'
- 'HTTP interface': 'interfaces/http.md'
- 'MySQL Interface': 'interfaces/mysql.md'
- ' فرمت های Input و Output': 'interfaces/formats.md'
- ' درایور JDBC': 'interfaces/jdbc.md'
- ' درایور ODBC': 'interfaces/odbc.md'
@ -183,6 +184,10 @@ nav:
- 'Operators': 'query_language/operators.md'
- 'General Syntax': 'query_language/syntax.md'
- 'Guides':
- 'Overview': 'guides/index.md'
- 'Applying CatBoost Models': 'guides/apply_catboost_model.md'
- 'Operations':
- 'Introduction': 'operations/index.md'
- 'Requirements': 'operations/requirements.md'
@ -225,6 +230,8 @@ nav:
- 'Browse ClickHouse Source Code': 'development/browse_code.md'
- 'How to Build ClickHouse on Linux': 'development/build.md'
- 'How to Build ClickHouse on Mac OS X': 'development/build_osx.md'
- 'How to Build ClickHouse on Linux for Mac OS X': 'development/build_cross_osx.md'
- 'How to Build ClickHouse on Linux for AARCH64 (ARM64)': 'development/build_cross_arm.md'
- 'How to Write C++ code': 'development/style.md'
- 'How to Run ClickHouse Tests': 'development/tests.md'
- 'Third-Party Libraries Used': 'development/contrib.md'

View File

@ -26,6 +26,7 @@ nav:
- 'Command-Line Client': 'interfaces/cli.md'
- 'Native Interface (TCP)': 'interfaces/tcp.md'
- 'HTTP Interface': 'interfaces/http.md'
- 'MySQL Interface': 'interfaces/mysql.md'
- 'Input and Output Formats': 'interfaces/formats.md'
- 'JDBC Driver': 'interfaces/jdbc.md'
- 'ODBC Driver': 'interfaces/odbc.md'

View File

@ -41,6 +41,7 @@ nav:
- 'Движки баз данных':
- 'Введение': 'database_engines/index.md'
- 'MySQL': 'database_engines/mysql.md'
- 'Lazy': 'database_engines/lazy.md'
- 'Движки таблиц':
- 'Введение': 'operations/table_engines/index.md'
@ -218,6 +219,7 @@ nav:
- 'Введение': 'operations/utils/index.md'
- 'clickhouse-copier': 'operations/utils/clickhouse-copier.md'
- 'clickhouse-local': 'operations/utils/clickhouse-local.md'
- 'clickhouse-benchmark': 'operations/utils/clickhouse-benchmark.md'
- 'Разработка':
- 'hidden': 'development/index.md'
@ -227,6 +229,7 @@ nav:
- 'Как собрать ClickHouse на Linux': 'development/build.md'
- 'Как собрать ClickHouse на Mac OS X': 'development/build_osx.md'
- 'Как собрать ClickHouse на Linux для Mac OS X': 'development/build_cross_osx.md'
- 'Как собрать ClickHouse на Linux для AARCH64 (ARM64)': 'development/build_cross_arm.md'
- 'Как писать код на C++': 'development/style.md'
- 'Как запустить тесты': 'development/tests.md'
- 'Сторонние библиотеки': 'development/contrib.md'

View File

@ -26,6 +26,7 @@ nav:
- '命令行客户端接口': 'interfaces/cli.md'
- '原生客户端接口 (TCP)': 'interfaces/tcp.md'
- 'HTTP 客户端接口': 'interfaces/http.md'
- 'MySQL 客户端接口': 'interfaces/mysql.md'
- '输入输出格式': 'interfaces/formats.md'
- 'JDBC 驱动': 'interfaces/jdbc.md'
- 'ODBC 驱动': 'interfaces/odbc.md'
@ -69,6 +70,7 @@ nav:
- '数据库引擎':
- '介绍': 'database_engines/index.md'
- 'MySQL': 'database_engines/mysql.md'
- 'Lazy': 'database_engines/lazy.md'
- '表引擎':
- '介绍': 'operations/table_engines/index.md'
@ -182,6 +184,10 @@ nav:
- '操作符': 'query_language/operators.md'
- '语法说明': 'query_language/syntax.md'
- 'Guides':
- 'Overview': 'guides/index.md'
- 'Applying CatBoost Models': 'guides/apply_catboost_model.md'
- '运维':
- '介绍': 'operations/index.md'
- '环境要求': 'operations/requirements.md'
@ -225,6 +231,7 @@ nav:
- '如何在Linux中编译ClickHouse': 'development/build.md'
- '如何在Mac OS X中编译ClickHouse': 'development/build_osx.md'
- '如何在Linux中编译Mac OS X ClickHouse': 'development/build_cross_osx.md'
- '如何在Linux中编译AARCH64 (ARM64) ClickHouse': 'development/build_cross_arm.md'
- '如何编写C++代码': 'development/style.md'
- '如何运行ClickHouse测试': 'development/tests.md'
- '使用的第三方库': 'development/contrib.md'

View File

@ -33,4 +33,3 @@ tornado==5.1
typing==3.7.4.1
Unidecode==1.1.1
urllib3==1.25.8
gitpython==2.1.14

View File

@ -29,7 +29,7 @@ ClickHouse 可以接受多种数据格式,可以在 (`INSERT`) 以及 (`SELECT
| [PrettySpace](#prettyspace) | ✗ | ✔ |
| [Protobuf](#protobuf) | ✔ | ✔ |
| [Avro](#data-format-avro) | ✔ | ✔ |
| AvroConfluent | ✔ | ✗ |
| [AvroConfluent](#data-format-avro-confluent) | ✔ | ✗ |
| [Parquet](#data-format-parquet) | ✔ | ✔ |
| [ORC](#data-format-orc) | ✔ | ✗ |
| [RowBinary](#rowbinary) | ✔ | ✔ |
@ -914,7 +914,7 @@ Column names must:
Output Avro file compression and sync interval can be configured with [output_format_avro_codec](../operations/settings/settings.md#settings-output_format_avro_codec) and [output_format_avro_sync_interval](../operations/settings/settings.md#settings-output_format_avro_sync_interval) respectively.
## AvroConfluent
## AvroConfluent {#data-format-avro-confluent}
AvroConfluent supports decoding single-object Avro messages commonly used with [Kafka](https://kafka.apache.org/) and [Confluent Schema Registry](https://docs.confluent.io/current/schema-registry/index.html).

View File

@ -652,7 +652,7 @@ SELECT replicate(1, ['a', 'b', 'c'])
使用指定的连接键从Join类型引擎的表中获取数据。
## modelEvaluate(model_name, ...)
## modelEvaluate(model_name, ...) {#function-modelevaluate}
使用外部模型计算。
接受模型的名称以及模型的参数。返回Float64类型的值。