mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
Merge branch 'master' into mvcc_prototype
This commit is contained in:
commit
141fbc6980
12
.github/workflows/backport_branches.yml
vendored
12
.github/workflows/backport_branches.yml
vendored
@ -9,6 +9,18 @@ on: # yamllint disable-line rule:truthy
|
||||
branches:
|
||||
- 'backport/**'
|
||||
jobs:
|
||||
PythonUnitTests:
|
||||
runs-on: [self-hosted, style-checker]
|
||||
steps:
|
||||
- name: Clear repository
|
||||
run: |
|
||||
sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE"
|
||||
- name: Check out repository code
|
||||
uses: actions/checkout@v2
|
||||
- name: Python unit tests
|
||||
run: |
|
||||
cd "$GITHUB_WORKSPACE/tests/ci"
|
||||
python3 -m unittest discover -s . -p '*_test.py'
|
||||
DockerHubPushAarch64:
|
||||
runs-on: [self-hosted, style-checker-aarch64]
|
||||
steps:
|
||||
|
4
.github/workflows/release.yml
vendored
4
.github/workflows/release.yml
vendored
@ -52,8 +52,8 @@ jobs:
|
||||
- name: Check docker clickhouse/clickhouse-server building
|
||||
run: |
|
||||
cd "$GITHUB_WORKSPACE/tests/ci"
|
||||
python3 docker_server.py --release-type auto
|
||||
python3 docker_server.py --release-type auto --no-ubuntu \
|
||||
python3 docker_server.py --release-type auto --version "${{ github.ref }}"
|
||||
python3 docker_server.py --release-type auto --version "${{ github.ref }}" --no-ubuntu \
|
||||
--image-repo clickhouse/clickhouse-keeper --image-path docker/keeper
|
||||
- name: Cleanup
|
||||
if: always()
|
||||
|
@ -158,6 +158,7 @@ toc_title: Adopters
|
||||
| <a href="https://www.staffcop.ru/" class="favicon">Staffcop</a> | Information Security | Main Product | — | — | [Official website, Documentation](https://www.staffcop.ru/sce43) |
|
||||
| <a href="https://www.suning.com/" class="favicon">Suning</a> | E-Commerce | User behaviour analytics | — | — | [Blog article](https://www.sohu.com/a/434152235_411876) |
|
||||
| <a href="https://superwall.me/" class="favicon">Superwall</a> | Monetization Tooling | Main product | — | — | [Word of mouth, Jan 2022](https://github.com/ClickHouse/ClickHouse/pull/33573) |
|
||||
| <a href="https://swetrix.com" class="favicon">Swetrix</a> | Analytics | Main Product | — | — | [Source code](https://github.com/swetrix/swetrix-api) |
|
||||
| <a href="https://www.teralytics.net/" class="favicon">Teralytics</a> | Mobility | Analytics | — | — | [Tech blog](https://www.teralytics.net/knowledge-hub/visualizing-mobility-data-the-scalability-challenge) |
|
||||
| <a href="https://www.tencent.com" class="favicon">Tencent</a> | Big Data | Data processing | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/5.%20ClickHouse大数据集群应用_李俊飞腾讯网媒事业部.pdf) |
|
||||
| <a href="https://www.tencent.com" class="favicon">Tencent</a> | Messaging | Logging | — | — | [Talk in Chinese, November 2019](https://youtu.be/T-iVQRuw-QY?t=5050) |
|
||||
|
@ -519,6 +519,33 @@ Possible values:
|
||||
|
||||
Default value: `1`.
|
||||
|
||||
## allow_settings_after_format_in_insert {#allow_settings_after_format_in_insert}
|
||||
|
||||
Control whether `SETTINGS` after `FORMAT` in `INSERT` queries is allowed or not. It is not recommended to use this, since this may interpret part of `SETTINGS` as values.
|
||||
|
||||
Example:
|
||||
|
||||
```sql
|
||||
INSERT INTO FUNCTION null('foo String') SETTINGS max_threads=1 VALUES ('bar');
|
||||
```
|
||||
|
||||
But the following query will work only with `allow_settings_after_format_in_insert`:
|
||||
|
||||
```sql
|
||||
SET allow_settings_after_format_in_insert=1;
|
||||
INSERT INTO FUNCTION null('foo String') VALUES ('bar') SETTINGS max_threads=1;
|
||||
```
|
||||
|
||||
Possible values:
|
||||
|
||||
- 0 — Disallow.
|
||||
- 1 — Allow.
|
||||
|
||||
Default value: `0`.
|
||||
|
||||
!!! note "Warning"
|
||||
Use this setting only for backward compatibility if your use cases depend on old syntax.
|
||||
|
||||
## input_format_skip_unknown_fields {#settings-input-format-skip-unknown-fields}
|
||||
|
||||
Enables or disables skipping insertion of extra data.
|
||||
|
@ -54,6 +54,7 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
|
||||
("multiquery,n", "allow multiple queries in the same file")
|
||||
("obfuscate", "obfuscate instead of formatting")
|
||||
("backslash", "add a backslash at the end of each line of the formatted query")
|
||||
("allow_settings_after_format_in_insert", "Allow SETTINGS after FORMAT, but note, that this is not always safe")
|
||||
("seed", po::value<std::string>(), "seed (arbitrary string) that determines the result of obfuscation")
|
||||
;
|
||||
|
||||
@ -83,6 +84,7 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
|
||||
bool multiple = options.count("multiquery");
|
||||
bool obfuscate = options.count("obfuscate");
|
||||
bool backslash = options.count("backslash");
|
||||
bool allow_settings_after_format_in_insert = options.count("allow_settings_after_format_in_insert");
|
||||
|
||||
if (quiet && (hilite || oneline || obfuscate))
|
||||
{
|
||||
@ -154,7 +156,7 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
|
||||
const char * pos = query.data();
|
||||
const char * end = pos + query.size();
|
||||
|
||||
ParserQuery parser(end);
|
||||
ParserQuery parser(end, allow_settings_after_format_in_insert);
|
||||
do
|
||||
{
|
||||
ASTPtr res = parseQueryAndMovePosition(
|
||||
|
@ -276,7 +276,7 @@ void ClientBase::setupSignalHandler()
|
||||
|
||||
ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, bool allow_multi_statements) const
|
||||
{
|
||||
ParserQuery parser(end);
|
||||
ParserQuery parser(end, global_context->getSettings().allow_settings_after_format_in_insert);
|
||||
ASTPtr res;
|
||||
|
||||
const auto & settings = global_context->getSettingsRef();
|
||||
|
@ -465,6 +465,7 @@ class IColumn;
|
||||
M(Bool, use_compact_format_in_distributed_parts_names, true, "Changes format of directories names for distributed table insert parts.", 0) \
|
||||
M(Bool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \
|
||||
M(UInt64, max_parser_depth, DBMS_DEFAULT_MAX_PARSER_DEPTH, "Maximum parser depth (recursion depth of recursive descend parser).", 0) \
|
||||
M(Bool, allow_settings_after_format_in_insert, false, "Allow SETTINGS after FORMAT, but note, that this is not always safe (note: this is a compatibility setting).", 0) \
|
||||
M(Seconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.", 0) \
|
||||
M(Seconds, periodic_live_view_refresh, DEFAULT_PERIODIC_LIVE_VIEW_REFRESH_SEC, "Interval after which periodically refreshed live view is forced to refresh.", 0) \
|
||||
M(Bool, transform_null_in, false, "If enabled, NULL values will be matched with 'IN' operator as if they are considered equal.", 0) \
|
||||
|
@ -165,7 +165,7 @@ DECLARE_SETTING_ENUM(DistributedDDLOutputMode)
|
||||
|
||||
enum class HandleKafkaErrorMode
|
||||
{
|
||||
DEFAULT = 0, // Ignore errors whit threshold.
|
||||
DEFAULT = 0, // Ignore errors with threshold.
|
||||
STREAM, // Put errors to stream in the virtual column named ``_error.
|
||||
/*FIXED_SYSTEM_TABLE, Put errors to in a fixed system table likey system.kafka_errors. This is not implemented now. */
|
||||
/*CUSTOM_SYSTEM_TABLE, Put errors to in a custom system table. This is not implemented now. */
|
||||
|
@ -142,10 +142,11 @@ void DDLTaskBase::parseQueryFromEntry(ContextPtr context)
|
||||
{
|
||||
const char * begin = entry.query.data();
|
||||
const char * end = begin + entry.query.size();
|
||||
const auto & settings = context->getSettingsRef();
|
||||
|
||||
ParserQuery parser_query(end);
|
||||
ParserQuery parser_query(end, settings.allow_settings_after_format_in_insert);
|
||||
String description;
|
||||
query = parseQuery(parser_query, begin, end, description, 0, context->getSettingsRef().max_parser_depth);
|
||||
query = parseQuery(parser_query, begin, end, description, 0, settings.max_parser_depth);
|
||||
}
|
||||
|
||||
ContextMutablePtr DDLTaskBase::makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & /*zookeeper*/)
|
||||
|
@ -435,7 +435,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
String query_table;
|
||||
try
|
||||
{
|
||||
ParserQuery parser(end);
|
||||
ParserQuery parser(end, settings.allow_settings_after_format_in_insert);
|
||||
|
||||
/// TODO: parser should fail early when max_query_size limit is reached.
|
||||
ast = parseQuery(parser, begin, end, "", max_query_size, settings.max_parser_depth);
|
||||
|
@ -94,33 +94,48 @@ void ASTInsertQuery::formatImpl(const FormatSettings & settings, FormatState & s
|
||||
<< quoteString(compression->as<ASTLiteral &>().value.safeGet<std::string>());
|
||||
}
|
||||
|
||||
if (select)
|
||||
{
|
||||
settings.ostr << " ";
|
||||
select->formatImpl(settings, state, frame);
|
||||
}
|
||||
else if (watch)
|
||||
{
|
||||
settings.ostr << " ";
|
||||
watch->formatImpl(settings, state, frame);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!format.empty())
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FORMAT " << (settings.hilite ? hilite_none : "") << format;
|
||||
}
|
||||
else if (!infile)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " VALUES" << (settings.hilite ? hilite_none : "");
|
||||
}
|
||||
}
|
||||
|
||||
if (settings_ast)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << "SETTINGS " << (settings.hilite ? hilite_none : "");
|
||||
settings_ast->formatImpl(settings, state, frame);
|
||||
}
|
||||
|
||||
/// Compatibility for INSERT w/o SETTINGS to format in oneline, i.e.:
|
||||
///
|
||||
/// INSERT INTO foo VALUES
|
||||
///
|
||||
/// But
|
||||
///
|
||||
/// INSERT INTO foo
|
||||
/// SETTINGS max_threads=1
|
||||
/// VALUES
|
||||
///
|
||||
char delim = settings_ast ? settings.nl_or_ws : ' ';
|
||||
|
||||
if (select)
|
||||
{
|
||||
settings.ostr << delim;
|
||||
select->formatImpl(settings, state, frame);
|
||||
}
|
||||
else if (watch)
|
||||
{
|
||||
settings.ostr << delim;
|
||||
watch->formatImpl(settings, state, frame);
|
||||
}
|
||||
|
||||
if (!select && !watch)
|
||||
{
|
||||
if (!format.empty())
|
||||
{
|
||||
settings.ostr << delim
|
||||
<< (settings.hilite ? hilite_keyword : "") << "FORMAT " << (settings.hilite ? hilite_none : "") << format;
|
||||
}
|
||||
else if (!infile)
|
||||
{
|
||||
settings.ostr << delim
|
||||
<< (settings.hilite ? hilite_keyword : "") << "VALUES" << (settings.hilite ? hilite_none : "");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void ASTInsertQuery::updateTreeHashImpl(SipHash & hash_state) const
|
||||
|
@ -61,11 +61,11 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
|
||||
ParserCreateTableQuery create_p;
|
||||
ParserSelectWithUnionQuery select_p;
|
||||
ParserInsertQuery insert_p(end);
|
||||
ParserInsertQuery insert_p(end, allow_settings_after_format_in_insert);
|
||||
ASTPtr query;
|
||||
if (kind == ASTExplainQuery::ExplainKind::ParsedAST)
|
||||
{
|
||||
ParserQuery p(end);
|
||||
ParserQuery p(end, allow_settings_after_format_in_insert);
|
||||
if (p.parse(pos, query, expected))
|
||||
explain_query->setExplainedQuery(std::move(query));
|
||||
else
|
||||
|
@ -10,11 +10,15 @@ class ParserExplainQuery : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * end;
|
||||
bool allow_settings_after_format_in_insert;
|
||||
|
||||
const char * getName() const override { return "EXPLAIN"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
public:
|
||||
explicit ParserExplainQuery(const char* end_) : end(end_) {}
|
||||
explicit ParserExplainQuery(const char* end_, bool allow_settings_after_format_in_insert_)
|
||||
: end(end_)
|
||||
, allow_settings_after_format_in_insert(allow_settings_after_format_in_insert_)
|
||||
{}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -130,8 +130,17 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
}
|
||||
}
|
||||
|
||||
Pos before_values = pos;
|
||||
/// Read SETTINGS if they are defined
|
||||
if (s_settings.ignore(pos, expected))
|
||||
{
|
||||
/// Settings are written like SET query, so parse them with ParserSetQuery
|
||||
ParserSetQuery parser_settings(true);
|
||||
if (!parser_settings.parse(pos, settings_ast, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
String format_str;
|
||||
Pos before_values = pos;
|
||||
|
||||
/// VALUES or FORMAT or SELECT or WITH or WATCH.
|
||||
/// After FROM INFILE we expect FORMAT, SELECT, WITH or nothing.
|
||||
@ -177,9 +186,21 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
return false;
|
||||
}
|
||||
|
||||
/// Read SETTINGS if they are defined
|
||||
if (s_settings.ignore(pos, expected))
|
||||
/// Read SETTINGS after FORMAT.
|
||||
///
|
||||
/// Note, that part of SETTINGS can be interpreted as values,
|
||||
/// hence it is done only under option.
|
||||
///
|
||||
/// Refs: https://github.com/ClickHouse/ClickHouse/issues/35100
|
||||
if (allow_settings_after_format_in_insert && s_settings.ignore(pos, expected))
|
||||
{
|
||||
if (settings_ast)
|
||||
throw Exception("You have SETTINGS before and after FORMAT, "
|
||||
"this is not allowed. "
|
||||
"Consider switching to SETTINGS before FORMAT "
|
||||
"and disable allow_settings_after_format_in_insert.",
|
||||
ErrorCodes::SYNTAX_ERROR);
|
||||
|
||||
/// Settings are written like SET query, so parse them with ParserSetQuery
|
||||
ParserSetQuery parser_settings(true);
|
||||
if (!parser_settings.parse(pos, settings_ast, expected))
|
||||
|
@ -26,11 +26,15 @@ class ParserInsertQuery : public IParserBase
|
||||
{
|
||||
private:
|
||||
const char * end;
|
||||
bool allow_settings_after_format_in_insert;
|
||||
|
||||
const char * getName() const override { return "INSERT query"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
public:
|
||||
explicit ParserInsertQuery(const char * end_) : end(end_) {}
|
||||
explicit ParserInsertQuery(const char * end_, bool allow_settings_after_format_in_insert_)
|
||||
: end(end_)
|
||||
, allow_settings_after_format_in_insert(allow_settings_after_format_in_insert_)
|
||||
{}
|
||||
};
|
||||
|
||||
/** Insert accepts an identifier and an asterisk with variants.
|
||||
|
@ -31,8 +31,8 @@ namespace DB
|
||||
|
||||
bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ParserQueryWithOutput query_with_output_p(end);
|
||||
ParserInsertQuery insert_p(end);
|
||||
ParserQueryWithOutput query_with_output_p(end, allow_settings_after_format_in_insert);
|
||||
ParserInsertQuery insert_p(end, allow_settings_after_format_in_insert);
|
||||
ParserUseQuery use_p;
|
||||
ParserSetQuery set_p;
|
||||
ParserSystemQuery system_p;
|
||||
|
@ -10,12 +10,16 @@ class ParserQuery : public IParserBase
|
||||
{
|
||||
private:
|
||||
const char * end;
|
||||
bool allow_settings_after_format_in_insert;
|
||||
|
||||
const char * getName() const override { return "Query"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
|
||||
public:
|
||||
explicit ParserQuery(const char * end_) : end(end_) {}
|
||||
explicit ParserQuery(const char * end_, bool allow_settings_after_format_in_insert_ = false)
|
||||
: end(end_)
|
||||
, allow_settings_after_format_in_insert(allow_settings_after_format_in_insert_)
|
||||
{}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -49,7 +49,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
ParserShowCreateAccessEntityQuery show_create_access_entity_p;
|
||||
ParserShowGrantsQuery show_grants_p;
|
||||
ParserShowPrivilegesQuery show_privileges_p;
|
||||
ParserExplainQuery explain_p(end);
|
||||
ParserExplainQuery explain_p(end, allow_settings_after_format_in_insert);
|
||||
|
||||
ASTPtr query;
|
||||
|
||||
|
@ -12,10 +12,16 @@ class ParserQueryWithOutput : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * end;
|
||||
bool allow_settings_after_format_in_insert;
|
||||
|
||||
const char * getName() const override { return "Query with output"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
|
||||
public:
|
||||
explicit ParserQueryWithOutput(const char * end_) : end(end_) {}
|
||||
explicit ParserQueryWithOutput(const char * end_, bool allow_settings_after_format_in_insert_ = false)
|
||||
: end(end_)
|
||||
, allow_settings_after_format_in_insert(allow_settings_after_format_in_insert_)
|
||||
{}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -386,7 +386,8 @@ std::pair<const char *, bool> splitMultipartQuery(
|
||||
const std::string & queries,
|
||||
std::vector<std::string> & queries_list,
|
||||
size_t max_query_size,
|
||||
size_t max_parser_depth)
|
||||
size_t max_parser_depth,
|
||||
bool allow_settings_after_format_in_insert)
|
||||
{
|
||||
ASTPtr ast;
|
||||
|
||||
@ -394,7 +395,7 @@ std::pair<const char *, bool> splitMultipartQuery(
|
||||
const char * pos = begin; /// parser moves pos from begin to the end of current query
|
||||
const char * end = begin + queries.size();
|
||||
|
||||
ParserQuery parser(end);
|
||||
ParserQuery parser(end, allow_settings_after_format_in_insert);
|
||||
|
||||
queries_list.clear();
|
||||
|
||||
|
@ -61,6 +61,7 @@ std::pair<const char *, bool> splitMultipartQuery(
|
||||
const std::string & queries,
|
||||
std::vector<std::string> & queries_list,
|
||||
size_t max_query_size,
|
||||
size_t max_parser_depth);
|
||||
size_t max_parser_depth,
|
||||
bool allow_settings_after_format_in_insert);
|
||||
|
||||
}
|
||||
|
@ -868,7 +868,7 @@ namespace
|
||||
query_text = std::move(*(query_info.mutable_query()));
|
||||
const char * begin = query_text.data();
|
||||
const char * end = begin + query_text.size();
|
||||
ParserQuery parser(end);
|
||||
ParserQuery parser(end, settings.allow_settings_after_format_in_insert);
|
||||
ast = parseQuery(parser, begin, end, "", settings.max_query_size, settings.max_parser_depth);
|
||||
|
||||
/// Choose input format.
|
||||
|
@ -275,7 +275,10 @@ void PostgreSQLHandler::processQuery()
|
||||
|
||||
const auto & settings = session->sessionContext()->getSettingsRef();
|
||||
std::vector<String> queries;
|
||||
auto parse_res = splitMultipartQuery(query->query, queries, settings.max_query_size, settings.max_parser_depth);
|
||||
auto parse_res = splitMultipartQuery(query->query, queries,
|
||||
settings.max_query_size,
|
||||
settings.max_parser_depth,
|
||||
settings.allow_settings_after_format_in_insert);
|
||||
if (!parse_res.second)
|
||||
throw Exception("Cannot parse and execute the following part of query: " + String(parse_res.first), ErrorCodes::SYNTAX_ERROR);
|
||||
|
||||
|
@ -5462,7 +5462,8 @@ std::optional<ProjectionCandidate> MergeTreeData::getQueryProcessingStageWithAgg
|
||||
max_added_blocks = std::make_shared<PartitionIdToMaxBlock>(replicated->getMaxAddedBlocks());
|
||||
}
|
||||
|
||||
auto parts = getVisibleDataPartsVector(query_context);
|
||||
const auto & snapshot_data = assert_cast<const MergeTreeData::SnapshotData &>(*storage_snapshot->data);
|
||||
const auto & parts = snapshot_data.parts;
|
||||
|
||||
// If minmax_count_projection is a valid candidate, check its completeness.
|
||||
if (minmax_count_projection_candidate)
|
||||
|
@ -71,13 +71,18 @@ static String clusterNameFromDDLQuery(ContextPtr context, const DDLTask & task)
|
||||
{
|
||||
const char * begin = task.entry.query.data();
|
||||
const char * end = begin + task.entry.query.size();
|
||||
String cluster_name;
|
||||
ParserQuery parser_query(end);
|
||||
const auto & settings = context->getSettingsRef();
|
||||
|
||||
String description = fmt::format("from {}", task.entry_path);
|
||||
ParserQuery parser_query(end, settings.allow_settings_after_format_in_insert);
|
||||
ASTPtr query = parseQuery(parser_query, begin, end, description,
|
||||
context->getSettingsRef().max_query_size, context->getSettingsRef().max_parser_depth);
|
||||
settings.max_query_size,
|
||||
settings.max_parser_depth);
|
||||
|
||||
String cluster_name;
|
||||
if (const auto * query_on_cluster = dynamic_cast<const ASTQueryWithOnCluster *>(query.get()))
|
||||
cluster_name = query_on_cluster->cluster;
|
||||
|
||||
return cluster_name;
|
||||
}
|
||||
|
||||
|
@ -1,6 +1,7 @@
|
||||
#!/usr/bin/env python3
|
||||
|
||||
from collections import namedtuple
|
||||
from typing import Any, Dict
|
||||
import json
|
||||
import time
|
||||
|
||||
@ -21,6 +22,8 @@ API_URL = "https://api.github.com/repos/ClickHouse/ClickHouse"
|
||||
|
||||
MAX_RETRY = 5
|
||||
|
||||
DEBUG_INFO = {} # type: Dict[str, Any]
|
||||
|
||||
|
||||
def get_installation_id(jwt_token):
|
||||
headers = {
|
||||
@ -110,6 +113,10 @@ def get_workflows_description_for_pull_request(pull_request_event):
|
||||
|
||||
workflow_descriptions = []
|
||||
for workflow in workflows_data:
|
||||
DEBUG_INFO["workflow"] = workflow
|
||||
# Some time workflow["head_repository"]["full_name"] is None
|
||||
if workflow["head_repository"] is None:
|
||||
continue
|
||||
# unfortunately we cannot filter workflows from forks in request to API
|
||||
# so doing it manually
|
||||
if (
|
||||
@ -162,7 +169,8 @@ def exec_workflow_url(urls_to_cancel, token):
|
||||
|
||||
def main(event):
|
||||
token = get_token_from_aws()
|
||||
event_data = json.loads(event["body"])
|
||||
DEBUG_INFO["event_body"] = event["body"]
|
||||
event_data = event["body"]
|
||||
|
||||
print("Got event for PR", event_data["number"])
|
||||
action = event_data["action"]
|
||||
@ -210,4 +218,9 @@ def main(event):
|
||||
|
||||
|
||||
def handler(event, _):
|
||||
try:
|
||||
main(event)
|
||||
except Exception:
|
||||
for name, value in DEBUG_INFO.items():
|
||||
print(f"Value of {name}: ", value)
|
||||
raise
|
||||
|
@ -24,7 +24,7 @@ from version_helper import (
|
||||
ClickHouseVersion,
|
||||
get_tagged_versions,
|
||||
get_version_from_repo,
|
||||
get_version_from_string,
|
||||
version_arg,
|
||||
)
|
||||
|
||||
TEMP_PATH = p.join(RUNNER_TEMP, "docker_images_check")
|
||||
@ -49,7 +49,8 @@ def parse_args() -> argparse.Namespace:
|
||||
"--version",
|
||||
type=version_arg,
|
||||
default=get_version_from_repo().string,
|
||||
help="a version to build",
|
||||
help="a version to build, automaticaly got from version_helper, accepts either "
|
||||
"tag ('refs/tags/' is removed automatically) or a normal 22.2.2.2 format",
|
||||
)
|
||||
parser.add_argument(
|
||||
"--release-type",
|
||||
@ -111,13 +112,6 @@ def parse_args() -> argparse.Namespace:
|
||||
return parser.parse_args()
|
||||
|
||||
|
||||
def version_arg(version: str) -> ClickHouseVersion:
|
||||
try:
|
||||
return get_version_from_string(version)
|
||||
except ValueError as e:
|
||||
raise argparse.ArgumentTypeError(e)
|
||||
|
||||
|
||||
def auto_release_type(version: ClickHouseVersion, release_type: str) -> str:
|
||||
if release_type != "auto":
|
||||
return release_type
|
||||
@ -125,7 +119,7 @@ def auto_release_type(version: ClickHouseVersion, release_type: str) -> str:
|
||||
git_versions = get_tagged_versions()
|
||||
reference_version = git_versions[0]
|
||||
for i in reversed(range(len(git_versions))):
|
||||
if git_versions[i] < version:
|
||||
if git_versions[i] <= version:
|
||||
if i == len(git_versions) - 1:
|
||||
return "latest"
|
||||
reference_version = git_versions[i + 1]
|
||||
@ -209,7 +203,7 @@ def build_and_push_image(
|
||||
result = []
|
||||
if os != "ubuntu":
|
||||
tag += f"-{os}"
|
||||
init_args = ["docker", "buildx", "build"]
|
||||
init_args = ["docker", "buildx", "build", "--build-arg BUILDKIT_INLINE_CACHE=1"]
|
||||
if push:
|
||||
init_args.append("--push")
|
||||
init_args.append("--output=type=image,push-by-digest=true")
|
||||
|
@ -9,7 +9,7 @@ from pr_info import PRInfo
|
||||
import docker_images_check as di
|
||||
|
||||
with patch("git_helper.Git"):
|
||||
from version_helper import get_version_from_string, get_tagged_versions
|
||||
from version_helper import get_version_from_string
|
||||
import docker_server as ds
|
||||
|
||||
# di.logging.basicConfig(level=di.logging.INFO)
|
||||
@ -254,7 +254,8 @@ class TestDockerServer(unittest.TestCase):
|
||||
get_version_from_string("2.2.1.1"),
|
||||
get_version_from_string("2.2.2.1"),
|
||||
]
|
||||
cases = (
|
||||
|
||||
cases_less = (
|
||||
(get_version_from_string("1.0.1.1"), "minor"),
|
||||
(get_version_from_string("1.1.2.1"), "minor"),
|
||||
(get_version_from_string("1.3.1.1"), "major"),
|
||||
@ -263,8 +264,18 @@ class TestDockerServer(unittest.TestCase):
|
||||
(get_version_from_string("2.2.3.1"), "latest"),
|
||||
(get_version_from_string("2.3.1.1"), "latest"),
|
||||
)
|
||||
_ = get_tagged_versions()
|
||||
for case in cases:
|
||||
for case in cases_less:
|
||||
release = ds.auto_release_type(case[0], "auto")
|
||||
self.assertEqual(case[1], release)
|
||||
|
||||
cases_equal = (
|
||||
(get_version_from_string("1.1.1.1"), "minor"),
|
||||
(get_version_from_string("1.2.1.1"), "major"),
|
||||
(get_version_from_string("2.1.1.1"), "minor"),
|
||||
(get_version_from_string("2.2.1.1"), "patch"),
|
||||
(get_version_from_string("2.2.2.1"), "latest"),
|
||||
)
|
||||
for case in cases_equal:
|
||||
release = ds.auto_release_type(case[0], "auto")
|
||||
self.assertEqual(case[1], release)
|
||||
|
||||
|
@ -93,7 +93,7 @@ class Git:
|
||||
if value == "":
|
||||
return
|
||||
if not self._tag_pattern.match(value):
|
||||
raise Exception(f"last tag {value} doesn't match the pattern")
|
||||
raise ValueError(f"last tag {value} doesn't match the pattern")
|
||||
|
||||
@property
|
||||
def latest_tag(self) -> str:
|
||||
|
@ -40,13 +40,12 @@ class Packages:
|
||||
"_".join((name, version, arch + ".deb")) for name, arch in self.packages
|
||||
)
|
||||
|
||||
rev = "2"
|
||||
self.rpm = tuple(
|
||||
"-".join((name, version, rev + "." + self.rpm_arch[arch] + ".rpm"))
|
||||
"-".join((name, version + "." + self.rpm_arch[arch] + ".rpm"))
|
||||
for name, arch in self.packages
|
||||
)
|
||||
|
||||
self.tgz = tuple(f"{name}-{version}.tgz" for name, _ in self.packages)
|
||||
self.tgz = tuple(f"{name}-{version}-amd64.tgz" for name, _ in self.packages)
|
||||
|
||||
def arch(self, deb_pkg: str) -> str:
|
||||
if deb_pkg not in self.deb:
|
||||
|
@ -255,7 +255,9 @@ if __name__ == "__main__":
|
||||
elif SUBMODULE_CHANGED_LABEL in pr_info.labels:
|
||||
pr_labels_to_remove.append(SUBMODULE_CHANGED_LABEL)
|
||||
|
||||
print("change labels: add {}, remove {}".format(pr_labels_to_add, pr_labels_to_remove))
|
||||
print(
|
||||
"change labels: add {}, remove {}".format(pr_labels_to_add, pr_labels_to_remove)
|
||||
)
|
||||
if pr_labels_to_add:
|
||||
post_labels(gh, pr_info, pr_labels_to_add)
|
||||
|
||||
|
@ -1,7 +1,7 @@
|
||||
#!/usr/bin/env python3
|
||||
import logging
|
||||
import os.path as p
|
||||
from argparse import ArgumentParser, ArgumentDefaultsHelpFormatter
|
||||
from argparse import ArgumentParser, ArgumentDefaultsHelpFormatter, ArgumentTypeError
|
||||
from typing import Dict, List, Tuple, Union
|
||||
|
||||
from git_helper import Git, removeprefix
|
||||
@ -150,6 +150,9 @@ class ClickHouseVersion:
|
||||
|
||||
return False
|
||||
|
||||
def __le__(self, other: "ClickHouseVersion") -> bool:
|
||||
return self == other or self < other
|
||||
|
||||
|
||||
class VersionType:
|
||||
LTS = "lts"
|
||||
@ -217,6 +220,20 @@ def get_version_from_tag(tag: str) -> ClickHouseVersion:
|
||||
return get_version_from_string(tag)
|
||||
|
||||
|
||||
def version_arg(version: str) -> ClickHouseVersion:
|
||||
version = removeprefix(version, "refs/tags/")
|
||||
try:
|
||||
return get_version_from_string(version)
|
||||
except ValueError:
|
||||
pass
|
||||
try:
|
||||
return get_version_from_tag(version)
|
||||
except ValueError:
|
||||
pass
|
||||
|
||||
raise ArgumentTypeError(f"version {version} does not match tag of plain version")
|
||||
|
||||
|
||||
def get_tagged_versions() -> List[ClickHouseVersion]:
|
||||
versions = []
|
||||
for tag in git.get_tags():
|
||||
|
33
tests/ci/version_test.py
Normal file
33
tests/ci/version_test.py
Normal file
@ -0,0 +1,33 @@
|
||||
#!/usr/bin/env python
|
||||
|
||||
import unittest
|
||||
from argparse import ArgumentTypeError
|
||||
|
||||
import version_helper as vh
|
||||
|
||||
|
||||
class TestFunctions(unittest.TestCase):
|
||||
def test_version_arg(self):
|
||||
cases = (
|
||||
("0.0.0.0", vh.get_version_from_string("0.0.0.0")),
|
||||
("1.1.1.2", vh.get_version_from_string("1.1.1.2")),
|
||||
("v1.1.1.2-lts", vh.get_version_from_string("1.1.1.2")),
|
||||
("v1.1.1.2-prestable", vh.get_version_from_string("1.1.1.2")),
|
||||
("v1.1.1.2-stable", vh.get_version_from_string("1.1.1.2")),
|
||||
("v1.1.1.2-testing", vh.get_version_from_string("1.1.1.2")),
|
||||
("refs/tags/v1.1.1.2-testing", vh.get_version_from_string("1.1.1.2")),
|
||||
)
|
||||
for case in cases:
|
||||
version = vh.version_arg(case[0])
|
||||
self.assertEqual(case[1], version)
|
||||
error_cases = (
|
||||
"0.0.0",
|
||||
"1.1.1.a",
|
||||
"1.1.1.1.1",
|
||||
"1.1.1.2-testing",
|
||||
"v1.1.1.2-testin",
|
||||
"refs/tags/v1.1.1.2-testin",
|
||||
)
|
||||
for case in error_cases:
|
||||
with self.assertRaises(ArgumentTypeError):
|
||||
version = vh.version_arg(case[0])
|
@ -29,7 +29,7 @@ def create_simple_table():
|
||||
def test_protobuf_format_input(started_cluster):
|
||||
create_simple_table()
|
||||
instance.http_query(
|
||||
"INSERT INTO test.simple FORMAT Protobuf SETTINGS format_schema='simple:KeyValuePair'",
|
||||
"INSERT INTO test.simple SETTINGS format_schema='simple:KeyValuePair' FORMAT Protobuf",
|
||||
"\x07\x08\x01\x12\x03abc\x07\x08\x02\x12\x03def",
|
||||
)
|
||||
assert instance.query("SELECT * from test.simple") == "1\tabc\n2\tdef\n"
|
||||
|
@ -162,7 +162,7 @@ def test_put(started_cluster, maybe_auth, positive, compression):
|
||||
values_csv = "1,2,3\n3,2,1\n78,43,45\n"
|
||||
filename = "test.csv"
|
||||
put_query = f"""insert into table function s3('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/{filename}',
|
||||
{maybe_auth}'CSV', '{table_format}', '{compression}') values settings s3_truncate_on_insert=1 {values}"""
|
||||
{maybe_auth}'CSV', '{table_format}', '{compression}') settings s3_truncate_on_insert=1 values {values}"""
|
||||
|
||||
try:
|
||||
run_query(instance, put_query)
|
||||
@ -362,7 +362,7 @@ def test_put_csv(started_cluster, maybe_auth, positive):
|
||||
instance = started_cluster.instances["dummy"] # type: ClickHouseInstance
|
||||
table_format = "column1 UInt32, column2 UInt32, column3 UInt32"
|
||||
filename = "test.csv"
|
||||
put_query = "insert into table function s3('http://{}:{}/{}/{}', {}'CSV', '{}') format CSV settings s3_truncate_on_insert=1".format(
|
||||
put_query = "insert into table function s3('http://{}:{}/{}/{}', {}'CSV', '{}') settings s3_truncate_on_insert=1 format CSV".format(
|
||||
started_cluster.minio_ip,
|
||||
MINIO_INTERNAL_PORT,
|
||||
bucket,
|
||||
@ -392,7 +392,7 @@ def test_put_get_with_redirect(started_cluster):
|
||||
values = "(1, 1, 1), (1, 1, 1), (11, 11, 11)"
|
||||
values_csv = "1,1,1\n1,1,1\n11,11,11\n"
|
||||
filename = "test.csv"
|
||||
query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values settings s3_truncate_on_insert=1 {}".format(
|
||||
query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') settings s3_truncate_on_insert=1 values {}".format(
|
||||
started_cluster.minio_redirect_host,
|
||||
started_cluster.minio_redirect_port,
|
||||
bucket,
|
||||
@ -431,7 +431,7 @@ def test_put_with_zero_redirect(started_cluster):
|
||||
filename = "test.csv"
|
||||
|
||||
# Should work without redirect
|
||||
query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values settings s3_truncate_on_insert=1 {}".format(
|
||||
query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') settings s3_truncate_on_insert=1 values {}".format(
|
||||
started_cluster.minio_ip,
|
||||
MINIO_INTERNAL_PORT,
|
||||
bucket,
|
||||
@ -442,7 +442,7 @@ def test_put_with_zero_redirect(started_cluster):
|
||||
run_query(instance, query)
|
||||
|
||||
# Should not work with redirect
|
||||
query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values settings s3_truncate_on_insert=1 {}".format(
|
||||
query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') settings s3_truncate_on_insert=1 values {}".format(
|
||||
started_cluster.minio_redirect_host,
|
||||
started_cluster.minio_redirect_port,
|
||||
bucket,
|
||||
|
@ -31,7 +31,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format
|
||||
|
||||
# Check the input in the protobuf format (now the table contains the same data twice).
|
||||
echo
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO array_3dim_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_array_3dim:ABC'" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO array_3dim_protobuf_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_array_3dim:ABC' FORMAT Protobuf" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM array_3dim_protobuf_00825"
|
||||
|
||||
rm "$BINARY_FILE_PATH"
|
||||
|
@ -36,7 +36,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format
|
||||
|
||||
# Check the input in the protobuf format (now the table contains the same data twice).
|
||||
echo
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO array_of_arrays_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_array_of_arrays:AA'" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO array_of_arrays_protobuf_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_array_of_arrays:AA' FORMAT Protobuf" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM array_of_arrays_protobuf_00825"
|
||||
|
||||
rm "$BINARY_FILE_PATH"
|
||||
|
@ -33,7 +33,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format
|
||||
|
||||
# Check the input in the protobuf format (now the table contains the same data twice).
|
||||
echo
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO enum_mapping_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_enum_mapping:EnumMessage'" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO enum_mapping_protobuf_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_enum_mapping:EnumMessage' FORMAT Protobuf" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM enum_mapping_protobuf_00825"
|
||||
|
||||
rm "$BINARY_FILE_PATH"
|
||||
|
@ -34,7 +34,7 @@ hexdump -C $BINARY_FILE_PATH
|
||||
|
||||
# Check the input in the protobuf format (now the table contains the same data twice).
|
||||
echo
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO map_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_map:Message'" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO map_protobuf_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_map:Message' FORMAT Protobuf" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM map_protobuf_00825"
|
||||
|
||||
rm "$BINARY_FILE_PATH"
|
||||
|
@ -30,7 +30,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format
|
||||
|
||||
# Check the input in the protobuf format (now the table contains the same data twice).
|
||||
echo
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO nested_in_nested_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_nested_in_nested:MessageType'" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO nested_in_nested_protobuf_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_nested_in_nested:MessageType' FORMAT Protobuf" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM nested_in_nested_protobuf_00825"
|
||||
|
||||
rm "$BINARY_FILE_PATH"
|
||||
|
@ -37,7 +37,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format
|
||||
|
||||
# Check the input in the protobuf format (now the table contains the same data twice).
|
||||
echo
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO nested_optional_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_nested_optional:Message'" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO nested_optional_protobuf_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_nested_optional:Message' FORMAT Protobuf" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM nested_optional_protobuf_00825"
|
||||
|
||||
rm "$BINARY_FILE_PATH"
|
||||
|
@ -38,7 +38,7 @@ echo
|
||||
echo
|
||||
echo "Roundtrip:"
|
||||
$CLICKHOUSE_CLIENT --query "CREATE TABLE roundtrip_no_length_delimiter_protobuf_00825 AS no_length_delimiter_protobuf_00825"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO roundtrip_no_length_delimiter_protobuf_00825 FORMAT ProtobufSingle SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_no_length_delimiter:Message'" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO roundtrip_no_length_delimiter_protobuf_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_no_length_delimiter:Message' FORMAT ProtobufSingle" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM roundtrip_no_length_delimiter_protobuf_00825"
|
||||
rm "$BINARY_FILE_PATH"
|
||||
|
||||
|
@ -68,7 +68,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format
|
||||
echo
|
||||
echo "Roundtrip:"
|
||||
$CLICKHOUSE_CLIENT --query "CREATE TABLE roundtrip_persons_00825 AS persons_00825"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO roundtrip_persons_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_persons:Person'" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO roundtrip_persons_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_persons:Person' FORMAT Protobuf" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM roundtrip_persons_00825 ORDER BY name"
|
||||
rm "$BINARY_FILE_PATH"
|
||||
|
||||
@ -82,7 +82,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format
|
||||
echo
|
||||
echo "Roundtrip:"
|
||||
$CLICKHOUSE_CLIENT --query "CREATE TABLE alt_persons_00825 AS persons_00825"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO alt_persons_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_persons:AltPerson'" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO alt_persons_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_persons:AltPerson' FORMAT Protobuf" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM alt_persons_00825 ORDER BY name"
|
||||
rm "$BINARY_FILE_PATH"
|
||||
|
||||
@ -96,7 +96,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format
|
||||
echo
|
||||
echo "Roundtrip:"
|
||||
$CLICKHOUSE_CLIENT --query "CREATE TABLE str_persons_00825 AS persons_00825"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO str_persons_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_persons:StrPerson'" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO str_persons_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_persons:StrPerson' FORMAT Protobuf" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM str_persons_00825 ORDER BY name"
|
||||
rm "$BINARY_FILE_PATH"
|
||||
|
||||
@ -110,7 +110,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format
|
||||
echo
|
||||
echo "Roundtrip:"
|
||||
$CLICKHOUSE_CLIENT --query "CREATE TABLE syntax2_persons_00825 AS persons_00825"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO syntax2_persons_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_persons_syntax2:Syntax2Person'" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO syntax2_persons_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_persons_syntax2:Syntax2Person' FORMAT Protobuf" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM syntax2_persons_00825 ORDER BY name"
|
||||
rm "$BINARY_FILE_PATH"
|
||||
|
||||
|
@ -48,7 +48,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format
|
||||
|
||||
# Check the input in the protobuf format (now the table contains the same data twice).
|
||||
echo
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO table_skipped_column_in_nested_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_skipped_column_in_nested:UpdateMessage'" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO table_skipped_column_in_nested_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_skipped_column_in_nested:UpdateMessage' FORMAT Protobuf" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM table_skipped_column_in_nested_00825 ORDER BY unused1"
|
||||
|
||||
rm "$BINARY_FILE_PATH"
|
||||
|
@ -67,7 +67,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format
|
||||
|
||||
# Check the input in the protobuf format (now the table contains the same data twice).
|
||||
echo
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO splitted_nested_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_splitted_nested:Some'" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO splitted_nested_protobuf_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_splitted_nested:Some' FORMAT Protobuf" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM splitted_nested_protobuf_00825"
|
||||
|
||||
rm "$BINARY_FILE_PATH"
|
||||
|
@ -28,7 +28,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format
|
||||
|
||||
# Check the input in the protobuf format (now the table contains the same data twice).
|
||||
echo
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO squares_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_squares:NumberAndSquare'" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO squares_protobuf_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_squares:NumberAndSquare' FORMAT Protobuf" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM squares_protobuf_00825"
|
||||
|
||||
rm "$BINARY_FILE_PATH"
|
||||
|
@ -34,7 +34,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format
|
||||
|
||||
# Check the input in the protobuf format (now the table contains the same data twice).
|
||||
echo
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO table_default_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_table_default:Message'" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO table_default_protobuf_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_table_default:Message' FORMAT Protobuf" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM table_default_protobuf_00825 ORDER BY x,y,z"
|
||||
|
||||
rm "$BINARY_FILE_PATH"
|
||||
|
@ -8,10 +8,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS defaults"
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE defaults (x UInt32, y UInt32, a DEFAULT x + y, b Float32 DEFAULT round(log(1 + x + y), 5), c UInt32 DEFAULT 42, e MATERIALIZED x + y, f ALIAS x + y) ENGINE = Memory"
|
||||
|
||||
echo -ne '{"x":1, "y":1}\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=INSERT%20INTO%20defaults%20FORMAT%20JSONEachRow%20SETTINGS%20input_format_defaults_for_omitted_fields=1" --data-binary @-
|
||||
echo -ne '{"x":2, "y":2, "c":2}\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=INSERT+INTO+defaults+FORMAT+JSONEachRow+SETTINGS+input_format_defaults_for_omitted_fields=1" --data-binary @-
|
||||
echo -ne '{"x":3, "y":3, "a":3, "b":3, "c":3}\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&database=${CLICKHOUSE_DATABASE}&query=INSERT+INTO+defaults+FORMAT+JSONEachRow+SETTINGS+input_format_defaults_for_omitted_fields=1" --data-binary @-
|
||||
echo -ne '{"x":4} {"y":5, "c":5} {"a":6, "b":6, "c":6}\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&database=${CLICKHOUSE_DATABASE}&query=INSERT+INTO+defaults+FORMAT+JSONEachRow+SETTINGS+input_format_defaults_for_omitted_fields=1" --data-binary @-
|
||||
echo -ne '{"x":1, "y":1}\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=INSERT%20INTO%20defaults%20SETTINGS%20input_format_defaults_for_omitted_fields=1%20FORMAT%20JSONEachRow" --data-binary @-
|
||||
echo -ne '{"x":2, "y":2, "c":2}\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=INSERT+INTO+defaults+SETTINGS+input_format_defaults_for_omitted_fields=1+FORMAT+JSONEachRow" --data-binary @-
|
||||
echo -ne '{"x":3, "y":3, "a":3, "b":3, "c":3}\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&database=${CLICKHOUSE_DATABASE}&query=INSERT+INTO+defaults+SETTINGS+input_format_defaults_for_omitted_fields=1+FORMAT+JSONEachRow" --data-binary @-
|
||||
echo -ne '{"x":4} {"y":5, "c":5} {"a":6, "b":6, "c":6}\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&database=${CLICKHOUSE_DATABASE}&query=INSERT+INTO+defaults+SETTINGS+input_format_defaults_for_omitted_fields=1+FORMAT+JSONEachRow" --data-binary @-
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM defaults ORDER BY x, y FORMAT JSONEachRow"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE defaults"
|
||||
|
@ -20,17 +20,15 @@ for ((i = 0; i < 3; i++)) do
|
||||
echo ${formats[i]}
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE nested_table"
|
||||
cat $CUR_DIR/data_orc_arrow_parquet_nested/nested_table.${format_files[i]} | ${CLICKHOUSE_CLIENT} -q "INSERT INTO nested_table FORMAT ${formats[i]} SETTINGS input_format_${format_files[i]}_import_nested = 1"
|
||||
cat $CUR_DIR/data_orc_arrow_parquet_nested/nested_table.${format_files[i]} | ${CLICKHOUSE_CLIENT} -q "INSERT INTO nested_table SETTINGS input_format_${format_files[i]}_import_nested = 1 FORMAT ${formats[i]}"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM nested_table"
|
||||
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE nested_nested_table"
|
||||
cat $CUR_DIR/data_orc_arrow_parquet_nested/nested_nested_table.${format_files[i]} | ${CLICKHOUSE_CLIENT} -q "INSERT INTO nested_nested_table FORMAT ${formats[i]} SETTINGS input_format_${format_files[i]}_import_nested = 1"
|
||||
cat $CUR_DIR/data_orc_arrow_parquet_nested/nested_nested_table.${format_files[i]} | ${CLICKHOUSE_CLIENT} -q "INSERT INTO nested_nested_table SETTINGS input_format_${format_files[i]}_import_nested = 1 FORMAT ${formats[i]}"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM nested_nested_table"
|
||||
|
||||
|
||||
done
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE nested_table"
|
||||
|
@ -22,10 +22,11 @@ cv bn m\", d: 2016-01-01 ;
|
||||
n: 456, s1: as\"df\\'gh , s2: '', s3: \"zx\\ncv\\tbn m\", s4: \"qwe,rty\", d: 2016-01-02 ;
|
||||
n: 9876543210, s1: , s2: 'zx\\ncv\\tbn m', s3: \"qwe,rty\", s4: \"as\"\"df'gh\", d: 2016-01-03 ;
|
||||
n: 789, s1: zx\\ncv\\tbn m , s2: 'qwe,rty', s3: \"as\\\"df'gh\", s4: \"\", d: 2016-01-04"$'\t'"
|
||||
$ suffix $" | $CLICKHOUSE_CLIENT --query="INSERT INTO template1 FORMAT Template SETTINGS \
|
||||
$ suffix $" | $CLICKHOUSE_CLIENT --query="INSERT INTO template1 SETTINGS \
|
||||
format_template_resultset = '$CURDIR/00938_template_input_format_resultset.tmp', \
|
||||
format_template_row = '$CURDIR/00938_template_input_format_row.tmp', \
|
||||
format_template_rows_between_delimiter = ';\n'";
|
||||
format_template_rows_between_delimiter = ';\n' \
|
||||
FORMAT Template";
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM template1 ORDER BY n FORMAT CSV";
|
||||
|
||||
@ -33,10 +34,11 @@ echo "==== parse json (sophisticated template) ===="
|
||||
echo -ne '{${:}"meta"${:}:${:}[${:}{${:}"name"${:}:${:}"s1"${:},${:}"type"${:}:${:}"String"${:}}${:},${:}{${:}"name"${:}:${:}"s2"${:},${:}"type"${:}:${:}"String"${:}}${:},${:}{${:}"name"${:}:${:}"s3"${:},${:}"type"${:}:${:}"String"${:}}${:},${:}{${:}"name"${:}:${:}"s4"${:},${:}"type"${:}:${:}"String"${:}}${:},${:}{${:}"name"${:}:${:}"n"${:},${:}"type"${:}:${:}"UInt64"${:}}${:},${:}{${:}"name"${:}:${:}"d"${:},${:}"type"${:}:${:}"Date"${:}}${:}]${:},${:}"data"${:}:${:}[${data}]${:},${:}"rows"${:}:${:}${:CSV}${:},${:}"statistics"${:}:${:}{${:}"elapsed"${:}:${:}${:CSV}${:},${:}"rows_read"${:}:${:}${:CSV}${:},${:}"bytes_read"${:}:${:}${:CSV}${:}}${:}}' > "$CURDIR"/00938_template_input_format_resultset.tmp
|
||||
echo -ne '{${:}"s1"${:}:${:}${s1:JSON}${:},${:}"s2"${:}:${:}${s2:JSON}${:},${:}"s3"${:}:${:}${s3:JSON}${:},${:}"s4"${:}:${:}${s4:JSON}${:},${:}"n"${:}:${:}${n:JSON}${:},${:}"d"${:}:${:}${d:JSON}${:}${:}}' > "$CURDIR"/00938_template_input_format_row.tmp
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM template1 ORDER BY n FORMAT JSON" | $CLICKHOUSE_CLIENT --query="INSERT INTO template2 FORMAT TemplateIgnoreSpaces SETTINGS \
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM template1 ORDER BY n FORMAT JSON" | $CLICKHOUSE_CLIENT --query="INSERT INTO template2 SETTINGS \
|
||||
format_template_resultset = '$CURDIR/00938_template_input_format_resultset.tmp', \
|
||||
format_template_row = '$CURDIR/00938_template_input_format_row.tmp', \
|
||||
format_template_rows_between_delimiter = ','";
|
||||
format_template_rows_between_delimiter = ',' \
|
||||
FORMAT TemplateIgnoreSpaces";
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM template2 ORDER BY n FORMAT CSV";
|
||||
$CLICKHOUSE_CLIENT --query="TRUNCATE TABLE template2";
|
||||
@ -45,10 +47,11 @@ echo "==== parse json ===="
|
||||
|
||||
echo -ne '{${:}"meta"${:}:${:JSON},${:}"data"${:}:${:}[${data}]${:},${:}"rows"${:}:${:JSON},${:}"statistics"${:}:${:JSON}${:}}' > "$CURDIR"/00938_template_input_format_resultset.tmp
|
||||
echo -ne '{${:}"s1"${:}:${:}${s3:JSON}${:},${:}"s2"${:}:${:}${:JSON}${:},${:}"s3"${:}:${:}${s1:JSON}${:},${:}"s4"${:}:${:}${:JSON}${:},${:}"n"${:}:${:}${n:JSON}${:},${:}"d"${:}:${:}${d:JSON}${:}${:}}' > "$CURDIR"/00938_template_input_format_row.tmp
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM template1 ORDER BY n FORMAT JSON" | $CLICKHOUSE_CLIENT --query="INSERT INTO template2 FORMAT TemplateIgnoreSpaces SETTINGS \
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM template1 ORDER BY n FORMAT JSON" | $CLICKHOUSE_CLIENT --query="INSERT INTO template2 SETTINGS \
|
||||
format_template_resultset = '$CURDIR/00938_template_input_format_resultset.tmp', \
|
||||
format_template_row = '$CURDIR/00938_template_input_format_row.tmp', \
|
||||
format_template_rows_between_delimiter = ','";
|
||||
format_template_rows_between_delimiter = ',' \
|
||||
FORMAT TemplateIgnoreSpaces";
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM template2 ORDER BY n FORMAT CSV";
|
||||
|
||||
@ -66,10 +69,11 @@ cv bn m\", d: 2016-01-01 ;
|
||||
n: 456, s1: as\"df\\'gh , s2: '', s3: \"zx\\ncv\\tbn m\", s4: \"qwe,rty\", d: 2016-01-02 ;
|
||||
n: 9876543210, s1: , s2: 'zx\\ncv\\tbn m', s3: \"qwe,rty\", s4: \"as\"\"df'gh\", d: 2016-01-03 ;
|
||||
n: 789, s1: zx\cv\bn m , s2: 'qwe,rty', s3: \"as\\\"df'gh\", s4: \"\", d: 2016-01-04"$'\t'"
|
||||
$ suffix $" | $CLICKHOUSE_CLIENT --query="INSERT INTO template1 FORMAT Template SETTINGS \
|
||||
$ suffix $" | $CLICKHOUSE_CLIENT --query="INSERT INTO template1 SETTINGS \
|
||||
format_template_resultset = '$CURDIR/00938_template_input_format_resultset.tmp', \
|
||||
format_template_row = '$CURDIR/00938_template_input_format_row.tmp', \
|
||||
format_template_rows_between_delimiter = ';\n'";
|
||||
format_template_rows_between_delimiter = ';\n' \
|
||||
FORMAT Template";
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM template1 ORDER BY n FORMAT CSV";
|
||||
|
||||
|
@ -23,12 +23,13 @@ echo '0, "2019-09-24", "hello"
|
||||
1, 2019-09-25, "world"
|
||||
2, "2019-09-26", custom
|
||||
3, 2019-09-27, separated
|
||||
end' | $CLICKHOUSE_CLIENT --query="INSERT INTO custom_separated FORMAT CustomSeparated SETTINGS \
|
||||
end' | $CLICKHOUSE_CLIENT --query="INSERT INTO custom_separated SETTINGS \
|
||||
format_custom_escaping_rule = 'CSV', \
|
||||
format_custom_field_delimiter = ', ', \
|
||||
format_custom_row_after_delimiter = '\n', \
|
||||
format_custom_row_between_delimiter = '', \
|
||||
format_custom_result_after_delimiter = 'end\n'"
|
||||
format_custom_result_after_delimiter = 'end\n'
|
||||
FORMAT CustomSeparated"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM custom_separated ORDER BY n FORMAT CSV"
|
||||
|
||||
|
@ -9,19 +9,19 @@ $CLICKHOUSE_CLIENT --query="CREATE TABLE regexp (id UInt32, array Array(UInt32),
|
||||
|
||||
echo 'id: 1 array: [1,2,3] string: str1 date: 2020-01-01
|
||||
id: 2 array: [1,2,3] string: str2 date: 2020-01-02
|
||||
id: 3 array: [1,2,3] string: str3 date: 2020-01-03' | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp FORMAT Regexp SETTINGS format_regexp='id: (.+?) array: (.+?) string: (.+?) date: (.+?)', format_regexp_escaping_rule='Escaped'";
|
||||
id: 3 array: [1,2,3] string: str3 date: 2020-01-03' | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp SETTINGS format_regexp='id: (.+?) array: (.+?) string: (.+?) date: (.+?)', format_regexp_escaping_rule='Escaped' FORMAT Regexp ";
|
||||
|
||||
echo 'id: 4 array: "[1,2,3]" string: "str4" date: "2020-01-04"
|
||||
id: 5 array: "[1,2,3]" string: "str5" date: "2020-01-05"
|
||||
id: 6 array: "[1,2,3]" string: "str6" date: "2020-01-06"' | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp FORMAT Regexp SETTINGS format_regexp='id: (.+?) array: (.+?) string: (.+?) date: (.+?)', format_regexp_escaping_rule='CSV'";
|
||||
id: 6 array: "[1,2,3]" string: "str6" date: "2020-01-06"' | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp SETTINGS format_regexp='id: (.+?) array: (.+?) string: (.+?) date: (.+?)', format_regexp_escaping_rule='CSV' FORMAT Regexp";
|
||||
|
||||
echo "id: 7 array: [1,2,3] string: 'str7' date: '2020-01-07'
|
||||
id: 8 array: [1,2,3] string: 'str8' date: '2020-01-08'
|
||||
id: 9 array: [1,2,3] string: 'str9' date: '2020-01-09'" | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp FORMAT Regexp SETTINGS format_regexp='id: (.+?) array: (.+?) string: (.+?) date: (.+?)', format_regexp_escaping_rule='Quoted'";
|
||||
id: 9 array: [1,2,3] string: 'str9' date: '2020-01-09'" | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp SETTINGS format_regexp='id: (.+?) array: (.+?) string: (.+?) date: (.+?)', format_regexp_escaping_rule='Quoted' FORMAT Regexp";
|
||||
|
||||
echo 'id: 10 array: [1,2,3] string: "str10" date: "2020-01-10"
|
||||
id: 11 array: [1,2,3] string: "str11" date: "2020-01-11"
|
||||
id: 12 array: [1,2,3] string: "str12" date: "2020-01-12"' | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp FORMAT Regexp SETTINGS format_regexp='id: (.+?) array: (.+?) string: (.+?) date: (.+?)', format_regexp_escaping_rule='JSON'";
|
||||
id: 12 array: [1,2,3] string: "str12" date: "2020-01-12"' | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp SETTINGS format_regexp='id: (.+?) array: (.+?) string: (.+?) date: (.+?)', format_regexp_escaping_rule='JSON' FORMAT Regexp";
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM regexp ORDER BY id";
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE regexp";
|
||||
|
@ -10,7 +10,7 @@ $CLICKHOUSE_CLIENT --query="CREATE TABLE regexp (id UInt32, string String) ENGIN
|
||||
echo 'id: 1 string: str1
|
||||
id: 2 string: str2
|
||||
id=3, string=str3
|
||||
id: 4 string: str4' | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp FORMAT Regexp SETTINGS format_regexp='id: (.+?) string: (.+?)', format_regexp_escaping_rule='Escaped', format_regexp_skip_unmatched=1";
|
||||
id: 4 string: str4' | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp SETTINGS format_regexp='id: (.+?) string: (.+?)', format_regexp_escaping_rule='Escaped', format_regexp_skip_unmatched=1 FORMAT Regexp";
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM regexp";
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE regexp";
|
||||
|
@ -6,9 +6,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
|
||||
echo 'DROP TABLE IF EXISTS mydb' | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}" -d @-
|
||||
echo 'CREATE TABLE mydb (datetime String, d1 String, d2 String ) ENGINE=Memory' | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}" -d @-
|
||||
echo "2021-Jan^d1^d2" | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&query=INSERT%20INTO%20mydb%20FORMAT%20CustomSeparated%20SETTINGS%20format_custom_escaping_rule%3D%27CSV%27%2C%20format_custom_field_delimiter%20%3D%20%27%5E%27" --data-binary @-
|
||||
echo -n "" | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&query=INSERT%20INTO%20mydb%20FORMAT%20CustomSeparated%20SETTINGS%20format_custom_escaping_rule%3D%27CSV%27%2C%20format_custom_field_delimiter%20%3D%20%27%5E%27" --data-binary @-
|
||||
echo "2021-Jan^d1^d2" | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&query=INSERT%20INTO%20mydb%20SETTINGS%20format_custom_escaping_rule%3D%27CSV%27%2C%20format_custom_field_delimiter%20%3D%20%27%5E%27%20FORMAT%20CustomSeparated" --data-binary @-
|
||||
echo -n "" | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&query=INSERT%20INTO%20mydb%20SETTINGS%20format_custom_escaping_rule%3D%27CSV%27%2C%20format_custom_field_delimiter%20%3D%20%27%5E%27%20FORMAT%20CustomSeparated" --data-binary @-
|
||||
echo 'SELECT * FROM mydb' | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}" -d @-
|
||||
printf "2021-Jan^d1^d2\n%.0s" {1..999999} | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&query=INSERT%20INTO%20mydb%20FORMAT%20CustomSeparated%20SETTINGS%20format_custom_escaping_rule%3D%27CSV%27%2C%20format_custom_field_delimiter%20%3D%20%27%5E%27" --data-binary @-
|
||||
printf "2021-Jan^d1^d2\n%.0s" {1..999999} | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&query=INSERT%20INTO%20mydb%20SETTINGS%20format_custom_escaping_rule%3D%27CSV%27%2C%20format_custom_field_delimiter%20%3D%20%27%5E%27%20FORMAT%20CustomSeparated" --data-binary @-
|
||||
echo 'SELECT count(*), countDistinct(datetime, d1, d2) FROM mydb' | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}" -d @-
|
||||
echo 'DROP TABLE mydb' | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}" -d @-
|
||||
|
@ -2,9 +2,8 @@ drop table if exists data_01593;
|
||||
create table data_01593 (key Int) engine=MergeTree() order by key partition by key;
|
||||
|
||||
insert into data_01593 select * from numbers_mt(10);
|
||||
-- TOO_MANY_PARTS error
|
||||
insert into data_01593 select * from numbers_mt(10) settings max_partitions_per_insert_block=1; -- { serverError 252 }
|
||||
insert into data_01593 select * from numbers_mt(10) settings max_partitions_per_insert_block=1; -- { serverError TOO_MANY_PARTS }
|
||||
-- settings for INSERT is prefered
|
||||
insert into data_01593 select * from numbers_mt(10) settings max_partitions_per_insert_block=1 settings max_partitions_per_insert_block=100;
|
||||
insert into data_01593 settings max_partitions_per_insert_block=100 select * from numbers_mt(10) settings max_partitions_per_insert_block=1;
|
||||
|
||||
drop table data_01593;
|
||||
|
@ -143,7 +143,7 @@ def sendQuery(s, query):
|
||||
writeStringBinary('', ba) # No interserver secret
|
||||
writeVarUInt(2, ba) # Stage - Complete
|
||||
ba.append(0) # No compression
|
||||
writeStringBinary(query + ' settings input_format_defaults_for_omitted_fields=0', ba) # query, finally
|
||||
writeStringBinary(query, ba) # query, finally
|
||||
s.sendall(ba)
|
||||
|
||||
|
||||
@ -205,7 +205,7 @@ def insertValidLowCardinalityRow():
|
||||
s.connect((CLICKHOUSE_HOST, CLICKHOUSE_PORT))
|
||||
sendHello(s)
|
||||
receiveHello(s)
|
||||
sendQuery(s, 'insert into {}.tab format TSV'.format(CLICKHOUSE_DATABASE))
|
||||
sendQuery(s, 'insert into {}.tab settings input_format_defaults_for_omitted_fields=0 format TSV'.format(CLICKHOUSE_DATABASE))
|
||||
|
||||
# external tables
|
||||
sendEmptyBlock(s)
|
||||
@ -241,7 +241,7 @@ def insertLowCardinalityRowWithIndexOverflow():
|
||||
s.connect((CLICKHOUSE_HOST, CLICKHOUSE_PORT))
|
||||
sendHello(s)
|
||||
receiveHello(s)
|
||||
sendQuery(s, 'insert into {}.tab format TSV'.format(CLICKHOUSE_DATABASE))
|
||||
sendQuery(s, 'insert into {}.tab settings input_format_defaults_for_omitted_fields=0 format TSV'.format(CLICKHOUSE_DATABASE))
|
||||
|
||||
# external tables
|
||||
sendEmptyBlock(s)
|
||||
@ -275,7 +275,7 @@ def insertLowCardinalityRowWithIncorrectDictType():
|
||||
s.connect((CLICKHOUSE_HOST, CLICKHOUSE_PORT))
|
||||
sendHello(s)
|
||||
receiveHello(s)
|
||||
sendQuery(s, 'insert into {}.tab format TSV'.format(CLICKHOUSE_DATABASE))
|
||||
sendQuery(s, 'insert into {}.tab settings input_format_defaults_for_omitted_fields=0 format TSV'.format(CLICKHOUSE_DATABASE))
|
||||
|
||||
# external tables
|
||||
sendEmptyBlock(s)
|
||||
@ -308,7 +308,7 @@ def insertLowCardinalityRowWithIncorrectAdditionalKeys():
|
||||
s.connect((CLICKHOUSE_HOST, CLICKHOUSE_PORT))
|
||||
sendHello(s)
|
||||
receiveHello(s)
|
||||
sendQuery(s, 'insert into {}.tab format TSV'.format(CLICKHOUSE_DATABASE))
|
||||
sendQuery(s, 'insert into {}.tab settings input_format_defaults_for_omitted_fields=0 format TSV'.format(CLICKHOUSE_DATABASE))
|
||||
|
||||
# external tables
|
||||
sendEmptyBlock(s)
|
||||
|
@ -19,7 +19,8 @@ cp -r $CLIENT_SCHEMADIR/02030_* $SCHEMADIR/$SERVER_SCHEMADIR/
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS capnp_simple_types";
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE capnp_simple_types (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixed FixedString(5), data String, date Date, datetime DateTime, datetime64 DateTime64(3)) ENGINE=Memory"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO capnp_simple_types values (-1, 1, -1000, 1000, -10000000, 1000000, -1000000000, 1000000000, 123.123, 123123123.123123123, 'Some string', 'fixed', 'Some data', '2000-01-06', '2000-06-01 19:42:42', '2000-04-01 11:21:33.123')"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_simple_types FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_simple_types:Message'" | $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_simple_types FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_simple_types:Message'"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_simple_types FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_simple_types:Message'" | \
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO capnp_simple_types SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_simple_types:Message' FORMAT CapnProto"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_simple_types"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE capnp_simple_types"
|
||||
|
||||
@ -27,7 +28,8 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE capnp_simple_types"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS capnp_tuples"
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE capnp_tuples (value UInt64, tuple1 Tuple(one UInt64, two Tuple(three UInt64, four UInt64)), tuple2 Tuple(nested1 Tuple(nested2 Tuple(x UInt64)))) ENGINE=Memory";
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO capnp_tuples VALUES (1, (2, (3, 4)), (((5))))"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_tuples FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_tuples:Message'" | $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_tuples FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_tuples:Message'"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_tuples FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_tuples:Message'" | \
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO capnp_tuples SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_tuples:Message' FORMAT CapnProto"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_tuples"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE capnp_tuples"
|
||||
|
||||
@ -35,7 +37,8 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE capnp_tuples"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS capnp_lists"
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE capnp_lists (value UInt64, list1 Array(UInt64), list2 Array(Array(Array(UInt64)))) ENGINE=Memory";
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO capnp_lists VALUES (1, [1, 2, 3], [[[1, 2, 3], [4, 5, 6]], [[7, 8, 9], []], []])"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_lists FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_lists:Message'" | $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_lists FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_lists:Message'"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_lists FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_lists:Message'" | \
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO capnp_lists SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_lists:Message' FORMAT CapnProto"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_lists"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE capnp_lists"
|
||||
|
||||
@ -43,7 +46,8 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE capnp_lists"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS capnp_nested_lists_and_tuples"
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE capnp_nested_lists_and_tuples (value UInt64, nested Tuple(a Tuple(b UInt64, c Array(Array(UInt64))), d Array(Tuple(e Array(Array(Tuple(f UInt64, g UInt64))), h Array(Tuple(k Array(UInt64))))))) ENGINE=Memory";
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO capnp_nested_lists_and_tuples VALUES (1, ((2, [[3, 4], [5, 6], []]), [([[(7, 8), (9, 10)], [(11, 12), (13, 14)], []], [([15, 16, 17]), ([])])]))"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_nested_lists_and_tuples FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nested_lists_and_tuples:Message'" | $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_nested_lists_and_tuples FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nested_lists_and_tuples:Message'"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_nested_lists_and_tuples FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nested_lists_and_tuples:Message'" | \
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO capnp_nested_lists_and_tuples SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nested_lists_and_tuples:Message' FORMAT CapnProto"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_nested_lists_and_tuples"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE capnp_nested_lists_and_tuples"
|
||||
|
||||
@ -51,7 +55,8 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE capnp_nested_lists_and_tuples"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS capnp_nested_table"
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE capnp_nested_table (nested Nested(value UInt64, array Array(UInt64), tuple Tuple(one UInt64, two UInt64))) ENGINE=Memory";
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO capnp_nested_table VALUES ([1, 2, 3], [[4, 5, 6], [], [7, 8]], [(9, 10), (11, 12), (13, 14)])"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_nested_table FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nested_table:Message'" | $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_nested_table FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nested_table:Message'"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_nested_table FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nested_table:Message'" | \
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO capnp_nested_table SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nested_table:Message' FORMAT CapnProto"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_nested_table"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE capnp_nested_table"
|
||||
|
||||
@ -59,7 +64,8 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE capnp_nested_table"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS capnp_nullable"
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE capnp_nullable (nullable Nullable(UInt64), array Array(Nullable(UInt64)), tuple Tuple(nullable Nullable(UInt64))) ENGINE=Memory";
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO capnp_nullable VALUES (1, [1, Null, 2], (1)), (Null, [Null, Null, 42], (Null))"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_nullable FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nullable:Message'" | $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_nullable FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nullable:Message'"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_nullable FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nullable:Message'" | \
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO capnp_nullable SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nullable:Message' FORMAT CapnProto"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_nullable"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE capnp_nullable"
|
||||
|
||||
@ -78,7 +84,8 @@ $CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'value
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS capnp_low_cardinality"
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE capnp_low_cardinality (lc1 LowCardinality(String), lc2 LowCardinality(Nullable(String)), lc3 Array(LowCardinality(Nullable(String)))) ENGINE=Memory"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO capnp_low_cardinality VALUES ('one', 'two', ['one', Null, 'two', Null]), ('two', Null, [Null])"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_low_cardinality FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_low_cardinality:Message'" | $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_low_cardinality FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_low_cardinality:Message'"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_low_cardinality FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_low_cardinality:Message'" | \
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO capnp_low_cardinality SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_low_cardinality:Message' FORMAT CapnProto"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_low_cardinality"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE capnp_low_cardinality"
|
||||
|
||||
|
@ -14,7 +14,8 @@ for format in CustomSeparated CustomSeparatedWithNames CustomSeparatedWithNamesA
|
||||
do
|
||||
echo $format
|
||||
$CLICKHOUSE_CLIENT -q "SELECT number AS x, number + 1 AS y, 'hello' AS s FROM numbers(5) FORMAT $format $CUSTOM_SETTINGS"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT number AS x, number + 1 AS y, 'hello' AS s FROM numbers(5) FORMAT $format $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT -q "INSERT INTO test_02117 FORMAT $format $CUSTOM_SETTINGS"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT number AS x, number + 1 AS y, 'hello' AS s FROM numbers(5) FORMAT $format $CUSTOM_SETTINGS" | \
|
||||
$CLICKHOUSE_CLIENT -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT $format"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117"
|
||||
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117"
|
||||
done
|
||||
@ -23,66 +24,80 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE test_02117"
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE test_02117 (x UInt32, y String DEFAULT 'default', z Date) engine=Memory()"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | \
|
||||
$CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNames"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117"
|
||||
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | \
|
||||
$CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNamesAndTypes"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117"
|
||||
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=0 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | \
|
||||
$CLICKHOUSE_CLIENT --input_format_with_names_use_header=0 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNames"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117"
|
||||
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=0 --input_format_with_types_use_header=0 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | \
|
||||
$CLICKHOUSE_CLIENT --input_format_with_names_use_header=0 --input_format_with_types_use_header=0 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNamesAndTypes"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117"
|
||||
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT 'text' AS y, toDate('2020-01-01') AS z, toUInt32(1) AS x FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT 'text' AS y, toDate('2020-01-01') AS z, toUInt32(1) AS x FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | \
|
||||
$CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNames"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117"
|
||||
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT 'text' AS y, toDate('2020-01-01') AS z, toUInt32(1) AS x FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT 'text' AS y, toDate('2020-01-01') AS z, toUInt32(1) AS x FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | \
|
||||
$CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNamesAndTypes"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117"
|
||||
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | \
|
||||
$CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNames"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117"
|
||||
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | \
|
||||
$CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNamesAndTypes"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117"
|
||||
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=0 --input_format_with_names_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | \
|
||||
$CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=0 --input_format_with_names_use_header=1 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNames"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117"
|
||||
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=0 --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | \
|
||||
$CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=0 --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNamesAndTypes"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117"
|
||||
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, [[1, 2, 3], [4, 5], []] as a FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_with_names_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, [[1, 2, 3], [4, 5], []] as a FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | \
|
||||
$CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_with_names_use_header=1 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNames"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117"
|
||||
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, [[1, 2, 3], [4, 5], []] as a FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, [[1, 2, 3], [4, 5], []] as a FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | \
|
||||
$CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNamesAndTypes"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117"
|
||||
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117"
|
||||
|
||||
TMP_FILE=$CURDIR/test_02117
|
||||
$CLICKHOUSE_CLIENT -q "SELECT 'text' AS x, toDate('2020-01-01') AS y, toUInt32(1) AS z FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" > $TMP_FILE
|
||||
cat $TMP_FILE | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL'
|
||||
cat $TMP_FILE | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNamesAndTypes" 2>&1 | \
|
||||
grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL'
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' as z, toDate('2020-01-01') AS y FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" > $TMP_FILE
|
||||
cat $TMP_FILE | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL'
|
||||
cat $TMP_FILE | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNamesAndTypes" 2>&1 | \
|
||||
grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL'
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE test_02117"
|
||||
rm $TMP_FILE
|
||||
|
@ -11,7 +11,7 @@ $CLICKHOUSE_CLIENT -q "create table test_02127 (x Float32, y Float64) engine=Mem
|
||||
|
||||
for escaping_rule in Quoted JSON Escaped CSV Raw
|
||||
do
|
||||
echo -e "+42.42\t+42.42" | $CLICKHOUSE_CLIENT -q "insert into test_02127 format CustomSeparated settings format_custom_escaping_rule='$escaping_rule'"
|
||||
echo -e "+42.42\t+42.42" | $CLICKHOUSE_CLIENT -q "insert into test_02127 settings format_custom_escaping_rule='$escaping_rule' format CustomSeparated"
|
||||
done
|
||||
|
||||
|
||||
|
@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
$CLICKHOUSE_CLIENT -q "drop table if exists test_02129"
|
||||
$CLICKHOUSE_CLIENT -q "create table test_02129 (x UInt64, y UInt64) engine=Memory()"
|
||||
|
||||
QUERY="insert into test_02129 format CustomSeparatedWithNames settings input_format_skip_unknown_fields=1, format_custom_escaping_rule='Quoted'"
|
||||
QUERY="insert into test_02129 settings input_format_skip_unknown_fields=1, format_custom_escaping_rule='Quoted' format CustomSeparatedWithNames"
|
||||
|
||||
# Skip string
|
||||
echo -e "'x'\t'trash'\t'y'\n1\t'Some string'\t42" | $CLICKHOUSE_CLIENT -q "$QUERY"
|
||||
|
@ -9,23 +9,23 @@ url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=1"
|
||||
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS async_inserts"
|
||||
${CLICKHOUSE_CLIENT} -q "CREATE TABLE async_inserts (id UInt32, s String) ENGINE = MergeTree ORDER BY id"
|
||||
|
||||
${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO async_inserts FORMAT CustomSeparated settings format_custom_escaping_rule='CSV', format_custom_field_delimiter=','
|
||||
${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO async_inserts settings format_custom_escaping_rule='CSV', format_custom_field_delimiter=',' FORMAT CustomSeparated
|
||||
1,\"a\"
|
||||
2,\"b\"
|
||||
" &
|
||||
|
||||
${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO async_inserts FORMAT CustomSeparated settings format_custom_escaping_rule='CSV', format_custom_field_delimiter=','
|
||||
${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO async_inserts settings format_custom_escaping_rule='CSV', format_custom_field_delimiter=',' FORMAT CustomSeparated
|
||||
3,\"a\"
|
||||
4,\"b\"
|
||||
" &
|
||||
|
||||
${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO async_inserts FORMAT CustomSeparatedWithNames settings format_custom_escaping_rule='CSV', format_custom_field_delimiter=','
|
||||
${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO async_inserts settings format_custom_escaping_rule='CSV', format_custom_field_delimiter=',' FORMAT CustomSeparatedWithNames
|
||||
\"id\",\"s\"
|
||||
5,\"a\"
|
||||
6,\"b\"
|
||||
" &
|
||||
|
||||
${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO async_inserts FORMAT CustomSeparatedWithNames settings format_custom_escaping_rule='CSV', format_custom_field_delimiter=','
|
||||
${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO async_inserts settings format_custom_escaping_rule='CSV', format_custom_field_delimiter=',' FORMAT CustomSeparatedWithNames
|
||||
\"id\",\"s\"
|
||||
7,\"a\"
|
||||
8,\"b\"
|
||||
|
@ -26,6 +26,6 @@ GZDATA="H4sIAHTzuWEAA9VTuw3CMBB9+RCsyIULhFIwAC0SJQWZACkNi1CAxCCMwCCMQMEIKdkgPJ8P
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS t1"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE t1 ( x Int64, y Int64, z Int64 ) ENGINE = Memory"
|
||||
|
||||
echo ${GZDATA} | base64 --decode | gunzip | ${CLICKHOUSE_CLIENT} -q "INSERT INTO t1 FORMAT Arrow settings input_format_arrow_allow_missing_columns = true" 2>&1 | grep -qF "DUPLICATE_COLUMN" && echo 'OK' || echo 'FAIL' ||:
|
||||
echo ${GZDATA} | base64 --decode | gunzip | ${CLICKHOUSE_CLIENT} -q "INSERT INTO t1 settings input_format_arrow_allow_missing_columns = true FORMAT Arrow" 2>&1 | grep -qF "DUPLICATE_COLUMN" && echo 'OK' || echo 'FAIL' ||:
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t1"
|
||||
|
@ -10,13 +10,13 @@ ${CLICKHOUSE_CLIENT} --query="create table test_02155_csv (A Int64, S String, D
|
||||
|
||||
|
||||
echo "input_format_null_as_default = 1"
|
||||
cat $CUR_DIR/data_csv/csv_with_slash.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_02155_csv FORMAT CSV SETTINGS input_format_null_as_default = 1"
|
||||
cat $CUR_DIR/data_csv/csv_with_slash.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_02155_csv SETTINGS input_format_null_as_default = 1 FORMAT CSV"
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM test_02155_csv"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE test_02155_csv"
|
||||
|
||||
echo "input_format_null_as_default = 0"
|
||||
cat $CUR_DIR/data_csv/csv_with_slash.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_02155_csv FORMAT CSV SETTINGS input_format_null_as_default = 0"
|
||||
cat $CUR_DIR/data_csv/csv_with_slash.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_02155_csv SETTINGS input_format_null_as_default = 0 FORMAT CSV"
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM test_02155_csv"
|
||||
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
drop table if exists t;
|
||||
create table t (x Bool) engine=Memory();
|
||||
insert into t values settings bool_true_representation='да' ('да');
|
||||
insert into t settings bool_true_representation='да' values ('да');
|
||||
drop table t;
|
||||
|
@ -1,12 +1,14 @@
|
||||
SET log_queries = 1;
|
||||
|
||||
DROP TABLE IF EXISTS t_async_insert_02193_1;
|
||||
|
||||
CREATE TABLE t_async_insert_02193_1 (id UInt32, s String) ENGINE = Memory;
|
||||
|
||||
INSERT INTO t_async_insert_02193_1 FORMAT CSV SETTINGS async_insert = 1
|
||||
INSERT INTO t_async_insert_02193_1 SETTINGS async_insert = 1 FORMAT CSV
|
||||
1,aaa
|
||||
;
|
||||
|
||||
INSERT INTO t_async_insert_02193_1 FORMAT Values SETTINGS async_insert = 1 (2, 'bbb');
|
||||
INSERT INTO t_async_insert_02193_1 SETTINGS async_insert = 1 FORMAT Values (2, 'bbb');
|
||||
|
||||
SET async_insert = 1;
|
||||
|
||||
|
@ -9,8 +9,8 @@ ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_async_insert_02193_2"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_async_insert_02193_2 (id UInt32, s String) ENGINE = Memory"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "INSERT INTO t_async_insert_02193_2 FORMAT CSV SETTINGS async_insert = 1 1,aaa"
|
||||
${CLICKHOUSE_CLIENT} -q "INSERT INTO t_async_insert_02193_2 FORMAT Values SETTINGS async_insert = 1 (2, 'bbb')"
|
||||
${CLICKHOUSE_CLIENT} -q "INSERT INTO t_async_insert_02193_2 SETTINGS async_insert = 1 FORMAT CSV 1,aaa"
|
||||
${CLICKHOUSE_CLIENT} -q "INSERT INTO t_async_insert_02193_2 SETTINGS async_insert = 1 FORMAT Values (2, 'bbb')"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "INSERT INTO t_async_insert_02193_2 VALUES (3, 'ccc')" --async_insert=1
|
||||
${CLICKHOUSE_CLIENT} -q 'INSERT INTO t_async_insert_02193_2 FORMAT JSONEachRow {"id": 4, "s": "ddd"}' --async_insert=1
|
||||
|
@ -72,7 +72,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format
|
||||
echo
|
||||
echo "Roundtrip:"
|
||||
$CLICKHOUSE_CLIENT --query "CREATE TABLE roundtrip_persons_02240 AS persons_02240"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO roundtrip_persons_02240 FORMAT ProtobufList SETTINGS format_schema='$SCHEMADIR/02240_protobuflist1_format_persons:Person'" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO roundtrip_persons_02240 SETTINGS format_schema='$SCHEMADIR/02240_protobuflist1_format_persons:Person' FORMAT ProtobufList" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM roundtrip_persons_02240 ORDER BY name"
|
||||
rm "$BINARY_FILE_PATH"
|
||||
|
||||
@ -86,7 +86,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format
|
||||
echo
|
||||
echo "Roundtrip:"
|
||||
$CLICKHOUSE_CLIENT --query "CREATE TABLE alt_persons_02240 AS persons_02240"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO alt_persons_02240 FORMAT ProtobufList SETTINGS format_schema='$SCHEMADIR/02240_protobuflist2_format_persons:AltPerson'" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO alt_persons_02240 SETTINGS format_schema='$SCHEMADIR/02240_protobuflist2_format_persons:AltPerson' FORMAT ProtobufList" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM alt_persons_02240 ORDER BY name"
|
||||
rm "$BINARY_FILE_PATH"
|
||||
|
||||
@ -100,7 +100,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format
|
||||
# echo
|
||||
echo "Roundtrip:"
|
||||
$CLICKHOUSE_CLIENT --query "CREATE TABLE str_persons_02240 AS persons_02240"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO str_persons_02240 FORMAT ProtobufList SETTINGS format_schema='$SCHEMADIR/02240_protobuflist3_format_persons:StrPerson'" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO str_persons_02240 SETTINGS format_schema='$SCHEMADIR/02240_protobuflist3_format_persons:StrPerson' FORMAT ProtobufList" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM str_persons_02240 ORDER BY name"
|
||||
rm "$BINARY_FILE_PATH"
|
||||
|
||||
@ -114,7 +114,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format
|
||||
echo
|
||||
echo "Roundtrip:"
|
||||
$CLICKHOUSE_CLIENT --query "CREATE TABLE syntax2_persons_02240 AS persons_02240"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO syntax2_persons_02240 FORMAT ProtobufList SETTINGS format_schema='$SCHEMADIR/02240_protobuflist_format_persons_syntax2:Syntax2Person'" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO syntax2_persons_02240 SETTINGS format_schema='$SCHEMADIR/02240_protobuflist_format_persons_syntax2:Syntax2Person' FORMAT ProtobufList" < "$BINARY_FILE_PATH"
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM syntax2_persons_02240 ORDER BY name"
|
||||
rm "$BINARY_FILE_PATH"
|
||||
|
||||
|
@ -22,7 +22,7 @@ for case_insensitive in "true" "false"; do
|
||||
original_width Nullable(UInt32),
|
||||
original_height Nullable(UInt32)) engine=Memory"
|
||||
|
||||
cat $CUR_DIR/data_parquet_bad_column/metadata_0.parquet | $CLICKHOUSE_CLIENT -q "insert into test_02241 format Parquet SETTINGS input_format_parquet_case_insensitive_column_matching=$case_insensitive"
|
||||
cat $CUR_DIR/data_parquet_bad_column/metadata_0.parquet | $CLICKHOUSE_CLIENT -q "insert into test_02241 SETTINGS input_format_parquet_case_insensitive_column_matching=$case_insensitive format Parquet"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "select count() from test_02241"
|
||||
$CLICKHOUSE_CLIENT -q "drop table test_02241"
|
||||
|
@ -9,7 +9,7 @@ echo "Parquet"
|
||||
DATA_FILE=$CUR_DIR/data_parquet/case_insensitive_column_matching.parquet
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_load"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_load (iD String, scOre Int32) ENGINE = Memory"
|
||||
cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "INSERT INTO parquet_load FORMAT Parquet SETTINGS input_format_parquet_case_insensitive_column_matching=true"
|
||||
cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "INSERT INTO parquet_load SETTINGS input_format_parquet_case_insensitive_column_matching=true FORMAT Parquet"
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_load"
|
||||
${CLICKHOUSE_CLIENT} --query="drop table parquet_load"
|
||||
|
||||
@ -17,7 +17,7 @@ echo "ORC"
|
||||
DATA_FILE=$CUR_DIR/data_orc/case_insensitive_column_matching.orc
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_load"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_load (iD String, sCorE Int32) ENGINE = Memory"
|
||||
cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "INSERT INTO orc_load FORMAT ORC SETTINGS input_format_orc_case_insensitive_column_matching=true"
|
||||
cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "INSERT INTO orc_load SETTINGS input_format_orc_case_insensitive_column_matching=true FORMAT ORC"
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM orc_load"
|
||||
${CLICKHOUSE_CLIENT} --query="drop table orc_load"
|
||||
|
||||
@ -25,6 +25,6 @@ echo "Arrow"
|
||||
DATA_FILE=$CUR_DIR/data_arrow/case_insensitive_column_matching.arrow
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_load"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_load (iD String, sCorE Int32) ENGINE = Memory"
|
||||
cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "INSERT INTO arrow_load FORMAT Arrow SETTINGS input_format_arrow_case_insensitive_column_matching=true"
|
||||
cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "INSERT INTO arrow_load SETTINGS input_format_arrow_case_insensitive_column_matching=true FORMAT Arrow"
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_load"
|
||||
${CLICKHOUSE_CLIENT} --query="drop table arrow_load"
|
||||
|
@ -17,7 +17,7 @@ for ((i = 0; i < 3; i++)) do
|
||||
echo ${formats[i]}
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE nested_table"
|
||||
cat $CUR_DIR/data_orc_arrow_parquet_nested/nested_table.${format_files[i]} | ${CLICKHOUSE_CLIENT} -q "INSERT INTO nested_table FORMAT ${formats[i]} SETTINGS input_format_${format_files[i]}_import_nested = 1, input_format_${format_files[i]}_case_insensitive_column_matching = true"
|
||||
cat $CUR_DIR/data_orc_arrow_parquet_nested/nested_table.${format_files[i]} | ${CLICKHOUSE_CLIENT} -q "INSERT INTO nested_table SETTINGS input_format_${format_files[i]}_import_nested = 1, input_format_${format_files[i]}_case_insensitive_column_matching = true FORMAT ${formats[i]}"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM nested_table"
|
||||
|
||||
|
@ -0,0 +1,69 @@
|
||||
insert into foo settings max_threads=1
|
||||
Syntax error (query): failed at position 40 (end of query):
|
||||
insert into foo format tsv settings max_threads=1
|
||||
Can't format ASTInsertQuery with data, since data will be lost.
|
||||
[multi] insert into foo format tsv settings max_threads=1
|
||||
INSERT INTO foo
|
||||
SETTINGS max_threads = 1
|
||||
FORMAT tsv
|
||||
[oneline] insert into foo format tsv settings max_threads=1
|
||||
INSERT INTO foo SETTINGS max_threads = 1 FORMAT tsv
|
||||
insert into foo settings max_threads=1 format tsv settings max_threads=1
|
||||
You have SETTINGS before and after FORMAT
|
||||
Cannot parse input: expected '\n' before: 'settings max_threads=1 1'
|
||||
1
|
||||
You have SETTINGS before and after FORMAT
|
||||
[multi] insert into foo values
|
||||
INSERT INTO foo FORMAT Values
|
||||
[oneline] insert into foo values
|
||||
INSERT INTO foo FORMAT Values
|
||||
[multi] insert into foo select 1
|
||||
INSERT INTO foo SELECT 1
|
||||
[oneline] insert into foo select 1
|
||||
INSERT INTO foo SELECT 1
|
||||
[multi] insert into foo watch bar
|
||||
INSERT INTO foo WATCH bar
|
||||
[oneline] insert into foo watch bar
|
||||
INSERT INTO foo WATCH bar
|
||||
[multi] insert into foo format tsv
|
||||
INSERT INTO foo FORMAT tsv
|
||||
[oneline] insert into foo format tsv
|
||||
INSERT INTO foo FORMAT tsv
|
||||
[multi] insert into foo settings max_threads=1 values
|
||||
INSERT INTO foo
|
||||
SETTINGS max_threads = 1
|
||||
FORMAT Values
|
||||
[oneline] insert into foo settings max_threads=1 values
|
||||
INSERT INTO foo SETTINGS max_threads = 1 FORMAT Values
|
||||
[multi] insert into foo settings max_threads=1 select 1
|
||||
INSERT INTO foo
|
||||
SETTINGS max_threads = 1
|
||||
SELECT 1
|
||||
[oneline] insert into foo settings max_threads=1 select 1
|
||||
INSERT INTO foo SETTINGS max_threads = 1 SELECT 1
|
||||
[multi] insert into foo settings max_threads=1 watch bar
|
||||
INSERT INTO foo
|
||||
SETTINGS max_threads = 1
|
||||
WATCH bar
|
||||
[oneline] insert into foo settings max_threads=1 watch bar
|
||||
INSERT INTO foo SETTINGS max_threads = 1 WATCH bar
|
||||
[multi] insert into foo settings max_threads=1 format tsv
|
||||
INSERT INTO foo
|
||||
SETTINGS max_threads = 1
|
||||
FORMAT tsv
|
||||
[oneline] insert into foo settings max_threads=1 format tsv
|
||||
INSERT INTO foo SETTINGS max_threads = 1 FORMAT tsv
|
||||
[multi] insert into foo select 1 settings max_threads=1
|
||||
INSERT INTO foo
|
||||
SETTINGS max_threads = 1
|
||||
SELECT 1
|
||||
SETTINGS max_threads = 1
|
||||
[oneline] insert into foo select 1 settings max_threads=1
|
||||
INSERT INTO foo SETTINGS max_threads = 1 SELECT 1 SETTINGS max_threads = 1
|
||||
[multi] insert into foo settings max_threads=1 select 1 settings max_threads=1
|
||||
INSERT INTO foo
|
||||
SETTINGS max_threads = 1
|
||||
SELECT 1
|
||||
SETTINGS max_threads = 1
|
||||
[oneline] insert into foo settings max_threads=1 select 1 settings max_threads=1
|
||||
INSERT INTO foo SETTINGS max_threads = 1 SELECT 1 SETTINGS max_threads = 1
|
49
tests/queries/0_stateless/02263_format_insert_settings.sh
Executable file
49
tests/queries/0_stateless/02263_format_insert_settings.sh
Executable file
@ -0,0 +1,49 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
function run_format()
|
||||
{
|
||||
local q="$1" && shift
|
||||
|
||||
echo "$q"
|
||||
$CLICKHOUSE_FORMAT "$@" <<<"$q"
|
||||
}
|
||||
function run_format_both()
|
||||
{
|
||||
local q="$1" && shift
|
||||
|
||||
echo "[multi] $q"
|
||||
$CLICKHOUSE_FORMAT "$@" <<<"$q"
|
||||
echo "[oneline] $q"
|
||||
$CLICKHOUSE_FORMAT --oneline "$@" <<<"$q"
|
||||
}
|
||||
|
||||
# NOTE: that those queries may work slow, due to stack trace obtaining
|
||||
run_format 'insert into foo settings max_threads=1' 2> >(grep -m1 -o "Syntax error (query): failed at position .* (end of query):")
|
||||
# compatibility
|
||||
run_format 'insert into foo format tsv settings max_threads=1' 2> >(grep -m1 -F -o "Can't format ASTInsertQuery with data, since data will be lost.")
|
||||
run_format_both 'insert into foo format tsv settings max_threads=1' --allow_settings_after_format_in_insert
|
||||
run_format 'insert into foo settings max_threads=1 format tsv settings max_threads=1' --allow_settings_after_format_in_insert 2> >(grep -m1 -F -o "You have SETTINGS before and after FORMAT")
|
||||
# and via server (since this is a separate code path)
|
||||
$CLICKHOUSE_CLIENT -q 'drop table if exists data_02263'
|
||||
$CLICKHOUSE_CLIENT -q 'create table data_02263 (key Int) engine=Memory()'
|
||||
$CLICKHOUSE_CLIENT -q 'insert into data_02263 format TSV settings max_threads=1 1' 2> >(grep -m1 -F -o "Cannot parse input: expected '\n' before: 'settings max_threads=1 1'")
|
||||
$CLICKHOUSE_CLIENT --allow_settings_after_format_in_insert=1 -q 'insert into data_02263 format TSV settings max_threads=1 1'
|
||||
$CLICKHOUSE_CLIENT -q 'select * from data_02263'
|
||||
$CLICKHOUSE_CLIENT --allow_settings_after_format_in_insert=1 -q 'insert into data_02263 settings max_threads=1 format tsv settings max_threads=1' 2> >(grep -m1 -F -o "You have SETTINGS before and after FORMAT")
|
||||
$CLICKHOUSE_CLIENT -q 'drop table data_02263'
|
||||
|
||||
run_format_both 'insert into foo values'
|
||||
run_format_both 'insert into foo select 1'
|
||||
run_format_both 'insert into foo watch bar'
|
||||
run_format_both 'insert into foo format tsv'
|
||||
|
||||
run_format_both 'insert into foo settings max_threads=1 values'
|
||||
run_format_both 'insert into foo settings max_threads=1 select 1'
|
||||
run_format_both 'insert into foo settings max_threads=1 watch bar'
|
||||
run_format_both 'insert into foo settings max_threads=1 format tsv'
|
||||
run_format_both 'insert into foo select 1 settings max_threads=1'
|
||||
run_format_both 'insert into foo settings max_threads=1 select 1 settings max_threads=1'
|
@ -15,7 +15,7 @@ do
|
||||
echo "$format, false";
|
||||
$CLICKHOUSE_CLIENT --output_format_parallel_formatting=false -q \
|
||||
"SELECT URLRegions as d, toTimeZone(ClientEventTime, 'Asia/Dubai') as a, MobilePhoneModel as b, ParamPrice as e, ClientIP6 as c FROM test.hits LIMIT 5000 Format $format" | \
|
||||
$CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=false -q "INSERT INTO parsing_with_names FORMAT $format SETTINGS input_format_null_as_default=0"
|
||||
$CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=false -q "INSERT INTO parsing_with_names SETTINGS input_format_null_as_default=0 FORMAT $format"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT * FROM parsing_with_names;" | md5sum
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parsing_with_names"
|
||||
@ -25,7 +25,7 @@ do
|
||||
echo "$format, true";
|
||||
$CLICKHOUSE_CLIENT --output_format_parallel_formatting=false -q \
|
||||
"SELECT URLRegions as d, toTimeZone(ClientEventTime, 'Asia/Dubai') as a, MobilePhoneModel as b, ParamPrice as e, ClientIP6 as c FROM test.hits LIMIT 5000 Format $format" | \
|
||||
$CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=true -q "INSERT INTO parsing_with_names FORMAT $format SETTINGS input_format_null_as_default=0"
|
||||
$CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=true -q "INSERT INTO parsing_with_names SETTINGS input_format_null_as_default=0 FORMAT $format"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT * FROM parsing_with_names;" | md5sum
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parsing_with_names"
|
||||
|
Loading…
Reference in New Issue
Block a user