Merge remote-tracking branch 'upstream/master'

This commit is contained in:
BayoNet 2018-04-28 11:13:06 +03:00
commit 1b0d7361cc
27 changed files with 281 additions and 67 deletions

View File

@ -1,3 +1,23 @@
# ClickHouse release 1.1.54380, 2018-04-21
## New features:
* Added table function `file(path, format, structure)`. An example reading bytes from `/dev/urandom`: `ln -s /dev/urandom /var/lib/clickhouse/user_files/random` `clickhouse-client -q "SELECT * FROM file('random', 'RowBinary', 'd UInt8') LIMIT 10"`.
## Improvements:
* Subqueries could be wrapped by `()` braces (to enhance queries readability). For example, `(SELECT 1) UNION ALL (SELECT 1)`.
* Simple `SELECT` queries from table `system.processes` are not counted in `max_concurrent_queries` limit.
## Bug fixes:
* Fixed incorrect behaviour of `IN` operator when select from `MATERIALIZED VIEW`.
* Fixed incorrect filtering by partition index in expressions like `WHERE partition_key_column IN (...)`
* Fixed inability to execute `OPTIMIZE` query on non-leader replica if the table was `REANAME`d.
* Fixed authorization error when execute `OPTIMIZE` or `ALTER` queries on a non-leader replica.
* Fixed freezing of `KILL QUERY` queries.
* Fixed an error in ZooKeeper client library which led to watches loses, freezing of distributed DDL queue and slowing replication queue if non-empty `chroot` prefix is used in ZooKeeper configuration.
## Backward incompatible changes:
* Removed support of expressions like `(a, b) IN (SELECT (a, b))` (instead of them you can use their equivalent `(a, b) IN (SELECT a, b)`). In previous releases, these expressions led to undermined data filtering or caused errors.
# ClickHouse release 1.1.54378, 2018-04-16
## New features:

View File

@ -1,3 +1,24 @@
# ClickHouse release 1.1.54380, 2018-04-21
## Новые возможности:
* Добавлена табличная функция `file(path, format, structure)`. Пример, читающий байты из `/dev/urandom`: `ln -s /dev/urandom /var/lib/clickhouse/user_files/random` `clickhouse-client -q "SELECT * FROM file('random', 'RowBinary', 'd UInt8') LIMIT 10"`.
## Улучшения:
* Добавлена возможность оборачивать подзапросы скобками `()` для повышения читаемости запросов. Например: `(SELECT 1) UNION ALL (SELECT 1)`.
* Простые запросы `SELECT` из таблицы `system.processes` не учитываются в ограничении `max_concurrent_queries`.
## Исправление ошибок:
* Исправлена неправильная работа оператора `IN` в `MATERIALIZED VIEW`.
* Исправлена неправильная работа индекса по ключу партиционирования в выражениях типа `partition_key_column IN (...)`.
* Исправлена невозможность выполнить `OPTIMIZE` запрос на лидирующей реплике после выполнения `RENAME` таблицы.
* Исправлены ошибки авторизации возникающие при выполнении запросов `OPTIMIZE` и `ALTER` на нелидирующей реплике.
* Исправлены зависания запросов `KILL QUERY`.
* Исправлена ошибка в клиентской библиотеке ZooKeeper, которая при использовании непустого префикса `chroot` в конфигурации приводила к потере watch'ей, остановке очереди distributed DDL запросов и замедлению репликации.
## Обратно несовместимые изменения:
* Убрана поддержка выражений типа `(a, b) IN (SELECT (a, b))` (можно использовать эквивалентные выражение `(a, b) IN (SELECT a, b)`). Раньше такие запросы могли приводить к недетерминированной фильтрации в `WHERE`.
# ClickHouse release 1.1.54378, 2018-04-16
## Новые возможности:

View File

@ -1,4 +1,4 @@
option (ENABLE_EMBEDDED_COMPILER "Set to TRUE to enable support for 'compile' option for query execution" FALSE)
option (ENABLE_EMBEDDED_COMPILER "Set to TRUE to enable support for 'compile' option for query execution" 1)
if (ENABLE_EMBEDDED_COMPILER)
# Based on source code of YT.
@ -32,7 +32,7 @@ if (ENABLE_EMBEDDED_COMPILER)
mark_as_advanced(LLVM_CONFIG_EXECUTABLE)
if(NOT LLVM_CONFIG_EXECUTABLE)
message(FATAL_ERROR "Cannot find LLVM (looking for `llvm-config${LLVM_VERSION_POSTFIX}`, `llvm-config`, `llvm-config-devel`). Please, provide LLVM_ROOT environment variable.")
message(WARNING "Cannot find LLVM (looking for `llvm-config${LLVM_VERSION_POSTFIX}`, `llvm-config`, `llvm-config-devel`). Please, provide LLVM_ROOT environment variable.")
else()
set(LLVM_FOUND TRUE)
@ -102,6 +102,6 @@ if (ENABLE_EMBEDDED_COMPILER)
endif()
if (LLVM_FOUND AND LLVM_INCLUDE_DIRS AND LLVM_LIBRARY_DIRS)
set(USE_EMBEDDED_COMPILER TRUE)
set (USE_EMBEDDED_COMPILER 1)
endif()
endif()

2
contrib/librdkafka vendored

@ -1 +1 @@
Subproject commit c3d50eb613704fb9c8ab3bce95a88275cb5875b7
Subproject commit 7478b5ef16aadd6543fe38bc6a2deb895c70da98

View File

@ -739,6 +739,22 @@ std::future<ZooKeeperImpl::ZooKeeper::MultiResponse> ZooKeeper::asyncMulti(const
return future;
}
int32_t ZooKeeper::tryMultiNoThrow(const Requests & requests, Responses & responses)
{
try
{
return multiImpl(requests, responses);
}
catch (ZooKeeperImpl::Exception & e)
{
return e.code;
}
catch (...)
{
throw;
}
}
size_t KeeperMultiException::getFailedOpIndex(int32_t code, const Responses & responses) const
{

View File

@ -148,11 +148,8 @@ public:
/// Throws only if some operation has returned an "unexpected" error
/// - an error that would cause the corresponding try- method to throw.
int32_t tryMulti(const Requests & requests, Responses & responses);
/// Throws nothing, just alias of multiImpl
int32_t tryMultiNoThrow(const Requests & requests, Responses & responses)
{
return multiImpl(requests, responses);
}
/// Throws nothing (even session expired errors)
int32_t tryMultiNoThrow(const Requests & requests, Responses & responses);
Int64 getClientID();

View File

@ -7,8 +7,8 @@ namespace DB
{
CSVRowOutputStream::CSVRowOutputStream(WriteBuffer & ostr_, const Block & sample_, bool with_names_, bool with_types_)
: ostr(ostr_), sample(sample_), with_names(with_names_), with_types(with_types_)
CSVRowOutputStream::CSVRowOutputStream(WriteBuffer & ostr_, const Block & sample_, const char delimiter_, bool with_names_, bool with_types_)
: ostr(ostr_), sample(sample_), delimiter(delimiter_), with_names(with_names_), with_types(with_types_)
{
size_t columns = sample.columns();
data_types.resize(columns);
@ -32,7 +32,7 @@ void CSVRowOutputStream::writePrefix()
for (size_t i = 0; i < columns; ++i)
{
writeCSVString(sample.safeGetByPosition(i).name, ostr);
writeChar(i == columns - 1 ? '\n' : ',', ostr);
writeChar(i == columns - 1 ? '\n' : delimiter, ostr);
}
}
@ -41,7 +41,7 @@ void CSVRowOutputStream::writePrefix()
for (size_t i = 0; i < columns; ++i)
{
writeCSVString(sample.safeGetByPosition(i).type->getName(), ostr);
writeChar(i == columns - 1 ? '\n' : ',', ostr);
writeChar(i == columns - 1 ? '\n' : delimiter, ostr);
}
}
}
@ -55,7 +55,7 @@ void CSVRowOutputStream::writeField(const IColumn & column, const IDataType & ty
void CSVRowOutputStream::writeFieldDelimiter()
{
writeChar(',', ostr);
writeChar(delimiter, ostr);
}

View File

@ -19,7 +19,7 @@ public:
/** with_names - output in the first line a header with column names
* with_types - output in the next line header with the names of the types
*/
CSVRowOutputStream(WriteBuffer & ostr_, const Block & sample_, bool with_names_ = false, bool with_types_ = false);
CSVRowOutputStream(WriteBuffer & ostr_, const Block & sample_, const char delimiter_, bool with_names_ = false, bool with_types_ = false);
void writeField(const IColumn & column, const IDataType & type, size_t row_num) override;
void writeFieldDelimiter() override;
@ -44,6 +44,7 @@ protected:
WriteBuffer & ostr;
const Block sample;
const char delimiter;
bool with_names;
bool with_types;
DataTypes data_types;

View File

@ -81,13 +81,12 @@ BlockInputStreamPtr FormatFactory::getInput(const String & name, ReadBuffer & bu
{
return wrap_row_stream(std::make_shared<ValuesRowInputStream>(buf, sample, context, settings.input_format_values_interpret_expressions));
}
else if (name == "CSV")
else if (name == "CSV" || name == "CSVWithNames")
{
return wrap_row_stream(std::make_shared<CSVRowInputStream>(buf, sample, ','));
}
else if (name == "CSVWithNames")
{
return wrap_row_stream(std::make_shared<CSVRowInputStream>(buf, sample, ',', true));
char csv_delimiter = settings.format_csv_delimiter;
bool with_names = name == "CSVWithNames";
return wrap_row_stream(std::make_shared<CSVRowInputStream>(buf, sample, csv_delimiter, with_names));
}
else if (name == "TSKV")
{
@ -152,10 +151,13 @@ static BlockOutputStreamPtr getOutputImpl(const String & name, WriteBuffer & buf
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<TabSeparatedRowOutputStream>(buf, sample, true, true), sample);
else if (name == "TabSeparatedRaw" || name == "TSVRaw")
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<TabSeparatedRawRowOutputStream>(buf, sample), sample);
else if (name == "CSV")
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<CSVRowOutputStream>(buf, sample), sample);
else if (name == "CSVWithNames")
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<CSVRowOutputStream>(buf, sample, true), sample);
else if (name == "CSV" || name == "CSVWithNames")
{
char csv_delimiter = settings.format_csv_delimiter;
bool with_names = name == "CSVWithNames";
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<CSVRowOutputStream>(buf, sample, csv_delimiter, with_names), sample);
}
else if (name == "Pretty")
return std::make_shared<PrettyBlockOutputStream>(buf, sample, false, settings.output_format_pretty_max_rows, context);
else if (name == "PrettyCompact")

View File

@ -194,9 +194,9 @@ void DataTypeFixedString::serializeTextCSV(const IColumn & column, size_t row_nu
}
void DataTypeFixedString::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const char /*delimiter*/) const
void DataTypeFixedString::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const char delimiter) const
{
read(*this, column, [&istr](ColumnFixedString::Chars_t & data) { readCSVStringInto(data, istr); });
read(*this, column, [&istr, delimiter](ColumnFixedString::Chars_t & data) { readCSVStringInto(data, istr, delimiter); });
}

View File

@ -285,9 +285,9 @@ void DataTypeString::serializeTextCSV(const IColumn & column, size_t row_num, Wr
}
void DataTypeString::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const char /*delimiter*/) const
void DataTypeString::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const char delimiter) const
{
read(column, [&](ColumnString::Chars_t & data) { readCSVStringInto(data, istr); });
read(column, [&](ColumnString::Chars_t & data) { readCSVStringInto(data, istr, delimiter); });
}

View File

@ -484,18 +484,17 @@ using FunctionExp = FunctionMathUnaryFloat64<UnaryFunctionVectorized<ExpName, ex
using FunctionLog = FunctionMathUnaryFloat64<UnaryFunctionVectorized<LogName, log>>;
using FunctionExp2 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Exp2Name, exp2>>;
using FunctionLog2 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Log2Name, log2>>;
using FunctionExp10 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Exp10Name, preciseExp10>>;
using FunctionLog10 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Log10Name, log10>>;
using FunctionSqrt = FunctionMathUnaryFloat64<UnaryFunctionVectorized<SqrtName, sqrt>>;
using FunctionCbrt = FunctionMathUnaryFloat64<UnaryFunctionVectorized<CbrtName,
using FunctionExp10 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Exp10Name,
#if USE_VECTORCLASS
Power_rational<1, 3>::pow
exp10
#else
cbrt
preciseExp10
#endif
>>;
using FunctionLog10 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Log10Name, log10>>;
using FunctionSqrt = FunctionMathUnaryFloat64<UnaryFunctionVectorized<SqrtName, sqrt>>;
using FunctionCbrt = FunctionMathUnaryFloat64<UnaryFunctionVectorized<CbrtName, cbrt>>;
using FunctionSin = FunctionMathUnaryFloat64<UnaryFunctionVectorized<SinName, sin>>;
using FunctionCos = FunctionMathUnaryFloat64<UnaryFunctionVectorized<CosName, cos>>;
using FunctionTan = FunctionMathUnaryFloat64<UnaryFunctionVectorized<TanName, tan>>;

View File

@ -247,7 +247,8 @@ struct Settings
M(SettingUInt64, max_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for a query. Zero means unlimited.") \
M(SettingUInt64, max_network_bytes, 0, "The maximum number of bytes (compressed) to receive or transmit over the network for execution of the query.") \
M(SettingUInt64, max_network_bandwidth_for_user, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running user queries. Zero means unlimited.")\
M(SettingUInt64, max_network_bandwidth_for_all_users, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running queries. Zero means unlimited.")
M(SettingUInt64, max_network_bandwidth_for_all_users, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running queries. Zero means unlimited.") \
M(SettingChar, format_csv_delimiter, ',', "The character to be considered as a delimiter in CSV data. If setting with a string, a string has to have a length of 1.") \
#define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \
TYPE NAME {DEFAULT};

View File

@ -26,6 +26,7 @@ namespace ErrorCodes
extern const int UNKNOWN_COMPRESSION_METHOD;
extern const int UNKNOWN_DISTRIBUTED_PRODUCT_MODE;
extern const int UNKNOWN_GLOBAL_SUBQUERIES_METHOD;
extern const int SIZE_OF_FIXED_STRING_DOESNT_MATCH;
}
@ -706,4 +707,58 @@ struct SettingString
}
};
struct SettingChar
{
private:
void checkStringIsACharacter(const String & x) const
{
if (x.size() != 1)
throw Exception("A setting's value string has to be an exactly one character long", ErrorCodes::SIZE_OF_FIXED_STRING_DOESNT_MATCH);
}
public:
char value;
bool changed = false;
SettingChar(char x = '\0') : value(x) {}
operator char() const { return value; }
SettingChar & operator= (char x) { set(x); return *this; }
String toString() const
{
return String(1, value);
}
void set(char x) {
value = x;
changed = true;
}
void set(const String & x)
{
checkStringIsACharacter(x);
value = x[0];
changed = true;
}
void set(const Field & x)
{
const String & s = safeGet<const String &>(x);
set(s);
}
void set(ReadBuffer & buf)
{
String x;
readBinary(x, buf);
checkStringIsACharacter(x);
set(x);
}
void write(WriteBuffer & buf) const
{
writeBinary(toString(), buf);
}
};
}

View File

@ -3638,7 +3638,7 @@ void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK()
}
catch (...)
{
LOG_ERROR(log, "There is a problem with deleting parts from ZooKeeper: " << getCurrentExceptionMessage(false));
LOG_ERROR(log, "There is a problem with deleting parts from ZooKeeper: " << getCurrentExceptionMessage(true));
}
/// Part names that were reliably deleted from ZooKeeper should be deleted from filesystem

View File

@ -39,9 +39,9 @@ DataTypePtr getDataType(SQLSMALLINT type)
switch (type)
{
case SQL_INTEGER:
return factory.get("UInt32");
return factory.get("Int32");
case SQL_SMALLINT:
return factory.get("UInt16");
return factory.get("Int16");
case SQL_FLOAT:
return factory.get("Float32");
case SQL_REAL:

View File

@ -76,6 +76,7 @@ def main(args):
# Keep same default values as in queries/0_stateless/00000_sh_lib.sh
os.environ.setdefault("CLICKHOUSE_BINARY", args.binary)
os.environ.setdefault("CLICKHOUSE_CLIENT", args.client)
os.environ.setdefault("CLICKHOUSE_CONFIG", args.configserver)
os.environ.setdefault("CLICKHOUSE_TMP", tmp_dir)
# TODO ! use clickhouse-extract-from-config here:

View File

@ -1,20 +1,21 @@
DROP TABLE IF EXISTS test.clear_column1;
DROP TABLE IF EXISTS test.clear_column2;
CREATE TABLE test.clear_column1 (p Int64, i Int64, v UInt64) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/tables/test/clear_column', '1', v) PARTITION BY p ORDER BY i;
CREATE TABLE test.clear_column2 (p Int64, i Int64, v UInt64) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/tables/test/clear_column', '2', v) PARTITION BY p ORDER BY i;
DROP TABLE IF EXISTS test.rename1;
DROP TABLE IF EXISTS test.rename2;
CREATE TABLE test.rename1 (p Int64, i Int64, v UInt64) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/test/tables/rename', '1', v) PARTITION BY p ORDER BY i;
CREATE TABLE test.rename2 (p Int64, i Int64, v UInt64) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/test/tables/rename', '2', v) PARTITION BY p ORDER BY i;
INSERT INTO test.clear_column1 VALUES (0, 1, 0);
INSERT INTO test.clear_column1 VALUES (0, 1, 1);
INSERT INTO test.rename1 VALUES (0, 1, 0);
INSERT INTO test.rename1 VALUES (0, 1, 1);
OPTIMIZE TABLE test.clear_column1;
OPTIMIZE TABLE test.clear_column2;
SELECT * FROM test.clear_column1;
OPTIMIZE TABLE test.rename1;
OPTIMIZE TABLE test.rename2;
SELECT * FROM test.rename1;
RENAME TABLE test.clear_column2 TO test.clear_column3;
RENAME TABLE test.rename2 TO test.rename3;
INSERT INTO test.clear_column1 VALUES (0, 1, 2);
OPTIMIZE TABLE test.clear_column3;
SELECT * FROM test.clear_column1;
INSERT INTO test.rename1 VALUES (0, 1, 2);
OPTIMIZE TABLE test.rename3;
SELECT * FROM test.rename1;
DROP TABLE IF EXISTS test.clear_column1;
DROP TABLE IF EXISTS test.clear_column2;
DROP TABLE IF EXISTS test.rename1;
DROP TABLE IF EXISTS test.rename2;
DROP TABLE IF EXISTS test.rename3;

View File

@ -0,0 +1,30 @@
Hello, world 123 2016-01-01
Hello, "world" 456 2016-01-02
Hello "world" 789 2016-01-03
Hello\n world 100 2016-01-04
Hello, world 123 2016-01-01
Hello, "world" 456 2016-01-02
Hello "world" 789 2016-01-03
Hello\n world 100 2016-01-04
"Hello, world";123;"2016-01-01"
"Hello, ""world""";456;"2016-01-02"
"Hello ""world""";789;"2016-01-03"
"Hello
world";100;"2016-01-04"
"Hello, world"/123/"2016-01-01"
"Hello, ""world"""/456/"2016-01-02"
"Hello ""world"""/789/"2016-01-03"
"Hello
world"/100/"2016-01-04"
abc,def hello
hello world
hello "world" abc,def
"abc,def";"hello"
"hello";"world"
"hello ""world""";"abc,def"
"abc,def","hello"
"hello","world"
"hello ""world""","abc,def"
"abc,def"/"hello"
"hello"/"world"
"hello ""world"""/"abc,def"

View File

@ -0,0 +1,52 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.csv";
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.csv (s String, n UInt64, d Date) ENGINE = Memory";
echo '"Hello, world"| 123| "2016-01-01"
"Hello, ""world"""| "456"| 2016-01-02|
Hello "world"| 789 |2016-01-03
"Hello
world"| 100| 2016-01-04|' | $CLICKHOUSE_CLIENT --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMAT CSV";
$CLICKHOUSE_CLIENT --query="SELECT * FROM test.csv ORDER BY d";
$CLICKHOUSE_CLIENT --query="DROP TABLE test.csv";
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.csv (s String, n UInt64, d Date) ENGINE = Memory";
echo '"Hello, world"; 123; "2016-01-01"
"Hello, ""world"""; "456"; 2016-01-02;
Hello "world"; 789 ;2016-01-03
"Hello
world"; 100; 2016-01-04;' | $CLICKHOUSE_CLIENT --multiquery --query="SET format_csv_delimiter=';'; INSERT INTO test.csv FORMAT CSV";
$CLICKHOUSE_CLIENT --query="SELECT * FROM test.csv ORDER BY d";
$CLICKHOUSE_CLIENT --format_csv_delimiter=";" --query="SELECT * FROM test.csv ORDER BY d FORMAT CSV";
$CLICKHOUSE_CLIENT --format_csv_delimiter="/" --query="SELECT * FROM test.csv ORDER BY d FORMAT CSV";
$CLICKHOUSE_CLIENT --query="DROP TABLE test.csv";
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.csv (s1 String, s2 String) ENGINE = Memory";
echo 'abc,def;hello;
hello; world;
"hello ""world""";abc,def;' | $CLICKHOUSE_CLIENT --multiquery --query="SET format_csv_delimiter=';'; INSERT INTO test.csv FORMAT CSV";
$CLICKHOUSE_CLIENT --query="SELECT * FROM test.csv";
$CLICKHOUSE_CLIENT --query="DROP TABLE test.csv";
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.csv (s1 String, s2 String) ENGINE = Memory";
echo '"s1";"s2"
abc,def;hello;
hello; world;
"hello ""world""";abc,def;' | $CLICKHOUSE_CLIENT --multiquery --query="SET format_csv_delimiter=';'; INSERT INTO test.csv FORMAT CSVWithNames";
$CLICKHOUSE_CLIENT --format_csv_delimiter=";" --query="SELECT * FROM test.csv FORMAT CSV";
$CLICKHOUSE_CLIENT --format_csv_delimiter="," --query="SELECT * FROM test.csv FORMAT CSV";
$CLICKHOUSE_CLIENT --format_csv_delimiter="/" --query="SELECT * FROM test.csv FORMAT CSV";
$CLICKHOUSE_CLIENT --query="DROP TABLE test.csv";

6
debian/.pbuilderrc vendored
View File

@ -167,8 +167,10 @@ case "$DIST" in
export CMAKE_FLAGS="-DENABLE_EMBEDDED_COMPILER=1 -DLLVM_VERSION_POSTFIX=-6.0 $CMAKE_FLAGS"
;;
"artful" | "experimental" | "unstable" | "testing" )
EXTRAPACKAGES+=" liblld-5.0-dev libclang-5.0-dev liblld-5.0 "
export CMAKE_FLAGS="-DENABLE_EMBEDDED_COMPILER=1 $CMAKE_FLAGS"
if [ "$ARCH" != arm64 ]; then
EXTRAPACKAGES+=" liblld-5.0-dev libclang-5.0-dev liblld-5.0 "
export CMAKE_FLAGS="-DENABLE_EMBEDDED_COMPILER=1 $CMAKE_FLAGS"
fi
;;
esac

10
debian/control vendored
View File

@ -4,7 +4,7 @@ Priority: optional
Maintainer: Alexey Milovidov <milovidov@yandex-team.ru>
Build-Depends: debhelper (>= 9),
cmake3 | cmake,
ninja-build,
ninja-build [amd64 i386],
gcc-7, g++-7,
libc6-dev,
libmariadbclient-dev | default-libmysqlclient-dev | libmysqlclient-dev,
@ -16,7 +16,7 @@ Build-Depends: debhelper (>= 9),
Standards-Version: 3.9.8
Package: clickhouse-client
Architecture: any
Architecture: all
Depends: ${shlibs:Depends}, ${misc:Depends}, clickhouse-common-static (= ${binary:Version}) | clickhouse-server-base (= ${binary:Version})
Replaces: clickhouse-compressor
Conflicts: clickhouse-compressor
@ -38,7 +38,7 @@ Description: Common files for clickhouse
This package provides common files for both clickhouse server and client
Package: clickhouse-server
Architecture: any
Architecture: all
Depends: ${shlibs:Depends}, ${misc:Depends}, clickhouse-common-static (= ${binary:Version}), adduser
Replaces: clickhouse-server-common, clickhouse-server-base
Provides: clickhouse-server-common
@ -60,7 +60,7 @@ Description: debugging symbols for clickhouse-common-static
Package: clickhouse-test
Priority: optional
Architecture: any
Architecture: all
Depends: ${shlibs:Depends}, ${misc:Depends}, clickhouse-client, bash, expect, python, python-lxml, python-termcolor, python-requests, curl, perl, sudo, openssl
Description: Clickhouse tests
@ -75,7 +75,7 @@ Depends: ${shlibs:Depends}, ${misc:Depends}, adduser, tzdata
Description: DEPRECATED PACKAGE (use clickhouse-common-static): Server binary for clickhouse
Package: clickhouse-server-common
Architecture: any
Architecture: all
Priority: optional
Depends: ${shlibs:Depends}, ${misc:Depends}, clickhouse-server-base (= ${binary:Version})
Description: DEPRECATED PACKAGE (use clickhouse-server): Common configuration files for clickhouse-server-base package

7
debian/rules vendored
View File

@ -53,8 +53,13 @@ ifndef DH_VERBOSE
CMAKE_FLAGS += -DCMAKE_VERBOSE_MAKEFILE=0
endif
# Useful for bulding on low memory systems
ifndef DISABLE_PARALLEL
DH_FLAGS += --parallel
endif
%:
dh $@ --parallel --buildsystem=cmake --builddirectory=$(BUILDDIR)
dh $@ $(DH_FLAGS) --buildsystem=cmake --builddirectory=$(BUILDDIR)
override_dh_auto_configure:
dh_auto_configure -- $(CMAKE_FLAGS)

View File

@ -2,9 +2,11 @@
Comma Separated Values format ([RFC](https://tools.ietf.org/html/rfc4180)).
When formatting, rows are enclosed in double quotes. A double quote inside a string is output as two double quotes in a row. There are no other rules for escaping characters. Date and date-time are enclosed in double quotes. Numbers are output without quotes. Values are separated by commas. Rows are separated using the Unix line feed (LF). Arrays are serialized in CSV as follows: first the array is serialized to a string as in TabSeparated format, and then the resulting string is output to CSV in double quotes. Tuples in CSV format are serialized as separate columns (that is, their nesting in the tuple is lost).
When formatting, rows are enclosed in double quotes. A double quote inside a string is output as two double quotes in a row. There are no other rules for escaping characters. Date and date-time are enclosed in double quotes. Numbers are output without quotes. Values are separated by a delimiter&ast;. Rows are separated using the Unix line feed (LF). Arrays are serialized in CSV as follows: first the array is serialized to a string as in TabSeparated format, and then the resulting string is output to CSV in double quotes. Tuples in CSV format are serialized as separate columns (that is, their nesting in the tuple is lost).
When parsing, all values can be parsed either with or without quotes. Both double and single quotes are supported. Rows can also be arranged without quotes. In this case, they are parsed up to a comma or line feed (CR or LF). In violation of the RFC, when parsing rows without quotes, the leading and trailing spaces and tabs are ignored. For the line feed, Unix (LF), Windows (CR LF) and Mac OS Classic (CR LF) are all supported.
&ast;By default — `,`. See a [format_csv_delimiter](/docs/en/operations/settings/settings/#format_csv_delimiter) setting for additional info.
When parsing, all values can be parsed either with or without quotes. Both double and single quotes are supported. Rows can also be arranged without quotes. In this case, they are parsed up to a delimiter or line feed (CR or LF). In violation of the RFC, when parsing rows without quotes, the leading and trailing spaces and tabs are ignored. For the line feed, Unix (LF), Windows (CR LF) and Mac OS Classic (CR LF) are all supported.
The CSV format supports the output of totals and extremes the same way as `TabSeparated`.

View File

@ -339,3 +339,6 @@ It works for JSONEachRow and TSKV formats.
If the value is true, integers appear in quotes when using JSON\* Int64 and UInt64 formats (for compatibility with most JavaScript implementations); otherwise, integers are output without the quotes.
## format_csv_delimiter
The character to be considered as a delimiter in CSV data. By default, `,`.

View File

@ -2,8 +2,10 @@
Формат comma separated values ([RFC](https://tools.ietf.org/html/rfc4180)).
При форматировании, строки выводятся в двойных кавычках. Двойная кавычка внутри строки выводится как две двойные кавычки подряд. Других правил экранирования нет. Даты и даты-с-временем выводятся в двойных кавычках. Числа выводятся без кавычек. Значения разделяются запятыми. Строки разделяются unix переводом строки (LF). Массивы сериализуются в CSV следующим образом: сначала массив сериализуется в строку, как в формате TabSeparated, а затем полученная строка выводится в CSV в двойных кавычках. Кортежи в формате CSV сериализуются, как отдельные столбцы (то есть, теряется их вложенность в кортеж).
При форматировании, строки выводятся в двойных кавычках. Двойная кавычка внутри строки выводится как две двойные кавычки подряд. Других правил экранирования нет. Даты и даты-с-временем выводятся в двойных кавычках. Числа выводятся без кавычек. Значения разделяются символом-разделителем&ast;. Строки разделяются unix переводом строки (LF). Массивы сериализуются в CSV следующим образом: сначала массив сериализуется в строку, как в формате TabSeparated, а затем полученная строка выводится в CSV в двойных кавычках. Кортежи в формате CSV сериализуются, как отдельные столбцы (то есть, теряется их вложенность в кортеж).
При парсинге, все значения могут парситься как в кавычках, так и без кавычек. Поддерживаются как двойные, так и одинарные кавычки. В том числе, строки могут быть расположены без кавычек - тогда они парсятся до запятой или перевода строки (CR или LF). В нарушение RFC, в случае парсинга строк не в кавычках, начальные и конечные пробелы и табы игнорируются. В качестве перевода строки, поддерживаются как Unix (LF), так и Windows (CR LF) и Mac OS Classic (LF CR) варианты.
&ast;По умолчанию — `,`. См. настройку [format_csv_delimiter](/docs/ru/operations/settings/settings/#format_csv_delimiter) для дополнительной информации.
При парсинге, все значения могут парситься как в кавычках, так и без кавычек. Поддерживаются как двойные, так и одинарные кавычки. В том числе, строки могут быть расположены без кавычек - тогда они парсятся до символа-разделителя или перевода строки (CR или LF). В нарушение RFC, в случае парсинга строк не в кавычках, начальные и конечные пробелы и табы игнорируются. В качестве перевода строки, поддерживаются как Unix (LF), так и Windows (CR LF) и Mac OS Classic (LF CR) варианты.
Формат CSV поддерживает вывод totals и extremes аналогично `TabSeparated`.

View File

@ -335,3 +335,7 @@ ClickHouse применяет настройку в том случае, ког
## output_format_json_quote_64bit_integers
Если значение истинно, то при использовании JSON\* форматов UInt64 и Int64 числа выводятся в кавычках (из соображений совместимости с большинством реализаций JavaScript), иначе - без кавычек.
## format_csv_delimiter
Символ, интерпретируемый как разделитель в данных формата CSV. По умолчанию — `,`.