Merge branch 'ClickHouse:master' into time_buckets_impl

This commit is contained in:
Yarik Briukhovetskyi 2024-09-05 11:36:28 +02:00 committed by GitHub
commit ab378a0a46
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
17 changed files with 257 additions and 191 deletions

View File

@ -66,13 +66,11 @@ TRAP(gethostbyname)
TRAP(gethostbyname2)
TRAP(gethostent)
TRAP(getlogin)
TRAP(getmntent)
TRAP(getnetbyaddr)
TRAP(getnetbyname)
TRAP(getnetent)
TRAP(getnetgrent)
TRAP(getnetgrent_r)
TRAP(getopt)
TRAP(getopt_long)
TRAP(getopt_long_only)
TRAP(getpass)
@ -133,7 +131,6 @@ TRAP(nrand48)
TRAP(__ppc_get_timebase_freq)
TRAP(ptsname)
TRAP(putchar_unlocked)
TRAP(putenv)
TRAP(pututline)
TRAP(pututxline)
TRAP(putwchar_unlocked)
@ -148,7 +145,6 @@ TRAP(sethostent)
TRAP(sethostid)
TRAP(setkey)
//TRAP(setlocale) // Used by replxx at startup
TRAP(setlogmask)
TRAP(setnetent)
TRAP(setnetgrent)
TRAP(setprotoent)
@ -203,7 +199,6 @@ TRAP(lgammal)
TRAP(nftw)
TRAP(nl_langinfo)
TRAP(putc_unlocked)
TRAP(rand)
/** In the current POSIX.1 specification (POSIX.1-2008), readdir() is not required to be thread-safe. However, in modern
* implementations (including the glibc implementation), concurrent calls to readdir() that specify different directory streams
* are thread-safe. In cases where multiple threads must read from the same directory stream, using readdir() with external
@ -288,4 +283,14 @@ TRAP(tss_get)
TRAP(tss_set)
TRAP(tss_delete)
#ifndef USE_MUSL
/// These produce duplicate symbol errors when statically linking with musl.
/// Maybe we can remove them from the musl fork.
TRAP(getopt)
TRAP(putenv)
TRAP(setlogmask)
TRAP(rand)
TRAP(getmntent)
#endif
#endif

View File

@ -48,25 +48,17 @@ std::string PathImpl::currentImpl()
std::string PathImpl::homeImpl()
{
std::string path;
#if defined(_POSIX_C_SOURCE) || defined(_BSD_SOURCE) || defined(_POSIX_C_SOURCE)
size_t buf_size = 1024; // Same as glibc use for getpwuid
std::vector<char> buf(buf_size);
struct passwd res;
struct passwd* pwd = nullptr;
getpwuid_r(getuid(), &res, buf.data(), buf_size, &pwd);
#else
struct passwd* pwd = getpwuid(getuid());
#endif
if (pwd)
path = pwd->pw_dir;
else
{
#if defined(_POSIX_C_SOURCE) || defined(_BSD_SOURCE) || defined(_POSIX_C_SOURCE)
getpwuid_r(getuid(), &res, buf.data(), buf_size, &pwd);
#else
pwd = getpwuid(geteuid());
#endif
if (pwd)
path = pwd->pw_dir;
else

View File

@ -140,6 +140,12 @@ if (CMAKE_CROSSCOMPILING)
message (STATUS "CROSS COMPILING SET LLVM HOST TRIPLE ${LLVM_HOST_TRIPLE}")
endif()
# llvm-project/llvm/cmake/config-ix.cmake does a weird thing: it defines _LARGEFILE64_SOURCE,
# then checks if lseek64() function exists, then undefines _LARGEFILE64_SOURCE.
# Then the actual code that uses this function *doesn't* define _LARGEFILE64_SOURCE, so lseek64()
# may not exist and compilation fails. This happens with musl.
add_compile_definitions("_LARGEFILE64_SOURCE")
add_subdirectory ("${LLVM_SOURCE_DIR}" "${LLVM_BINARY_DIR}")
set_directory_properties (PROPERTIES

2
contrib/sysroot vendored

@ -1 +1 @@
Subproject commit cc385041b226d1fc28ead14dbab5d40a5f821dd8
Subproject commit 5be834147d5b5dd77ca2b821f356982029320513

View File

@ -2035,6 +2035,7 @@ Query:
SELECT arrayZip(['a', 'b', 'c'], [5, 2, 1]);
```
Result:
``` text
@ -2043,6 +2044,43 @@ Result:
└──────────────────────────────────────┘
```
## arrayZipUnaligned
Combines multiple arrays into a single array, allowing for unaligned arrays. The resulting array contains the corresponding elements of the source arrays grouped into tuples in the listed order of arguments.
**Syntax**
``` sql
arrayZipUnaligned(arr1, arr2, ..., arrN)
```
**Arguments**
- `arrN` — [Array](../data-types/array.md).
The function can take any number of arrays of different types.
**Returned value**
- Array with elements from the source arrays grouped into [tuples](../data-types/tuple.md). Data types in the tuple are the same as types of the input arrays and in the same order as arrays are passed. [Array](../data-types/array.md). If the arrays have different sizes, the shorter arrays will be padded with `null` values.
**Example**
Query:
``` sql
SELECT arrayZipUnaligned(['a'], [1, 2, 3]);
```
Result:
``` text
┌─arrayZipUnaligned(['a'], [1, 2, 3])─┐
│ [('a',1),(NULL,2),(NULL,3)] │
└─────────────────────────────────────┘
```
## arrayAUC
Calculate AUC (Area Under the Curve, which is a concept in machine learning, see more details: <https://en.wikipedia.org/wiki/Receiver_operating_characteristic#Area_under_the_curve>).

View File

@ -1,7 +1,8 @@
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnTuple.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <IO/WriteHelpers.h>
@ -19,16 +20,15 @@ namespace ErrorCodes
}
/// arrayZip(['a', 'b', 'c'], ['d', 'e', 'f']) = [('a', 'd'), ('b', 'e'), ('c', 'f')]
/// arrayZipUnaligned(['a', 'b', 'c'], ['d', 'e']) = [('a', 'd'), ('b', 'e'), ('c', null)]
template <bool allow_unaligned>
class FunctionArrayZip : public IFunction
{
public:
static constexpr auto name = "arrayZip";
static constexpr auto name = allow_unaligned ? "arrayZipUnaligned" : "arrayZip";
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionArrayZip>(); }
String getName() const override
{
return name;
}
String getName() const override { return name; }
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
@ -39,8 +39,11 @@ public:
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (arguments.empty())
throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION,
"Function {} needs at least one argument; passed {}." , getName(), arguments.size());
throw Exception(
ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION,
"Function {} needs at least one argument; passed {}.",
getName(),
arguments.size());
DataTypes arguments_types;
for (size_t index = 0; index < arguments.size(); ++index)
@ -48,56 +51,142 @@ public:
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[index].type.get());
if (!array_type)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Argument {} of function {} must be array. Found {} instead.",
toString(index + 1), getName(), arguments[0].type->getName());
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Argument {} of function {} must be array. Found {} instead.",
toString(index + 1),
getName(),
arguments[0].type->getName());
arguments_types.emplace_back(array_type->getNestedType());
auto nested_type = array_type->getNestedType();
if constexpr (allow_unaligned)
nested_type = makeNullable(nested_type);
arguments_types.emplace_back(nested_type);
}
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeTuple>(arguments_types));
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
ColumnPtr
executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
{
size_t num_arguments = arguments.size();
ColumnPtr first_array_column;
Columns holders(num_arguments);
Columns tuple_columns(num_arguments);
bool has_unaligned = false;
size_t unaligned_index = 0;
for (size_t i = 0; i < num_arguments; ++i)
{
/// Constant columns cannot be inside tuple. It's only possible to have constant tuple as a whole.
ColumnPtr holder = arguments[i].column->convertToFullColumnIfConst();
holders[i] = holder;
const ColumnArray * column_array = checkAndGetColumn<ColumnArray>(holder.get());
if (!column_array)
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Argument {} of function {} must be array. Found column {} instead.",
i + 1, getName(), holder->getName());
if (i == 0)
{
first_array_column = holder;
}
else if (!column_array->hasEqualOffsets(static_cast<const ColumnArray &>(*first_array_column)))
{
throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH,
"The argument 1 and argument {} of function {} have different array sizes",
i + 1, getName());
}
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Argument {} of function {} must be array. Found column {} instead.",
i + 1,
getName(),
holder->getName());
tuple_columns[i] = column_array->getDataPtr();
if (i && !column_array->hasEqualOffsets(static_cast<const ColumnArray &>(*holders[0])))
{
has_unaligned = true;
unaligned_index = i;
}
}
if constexpr (!allow_unaligned)
{
if (has_unaligned)
throw Exception(
ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH,
"The argument 1 and argument {} of function {} have different array sizes",
unaligned_index + 1,
getName());
else
return ColumnArray::create(
ColumnTuple::create(tuple_columns), static_cast<const ColumnArray &>(*first_array_column).getOffsetsPtr());
ColumnTuple::create(std::move(tuple_columns)), static_cast<const ColumnArray &>(*holders[0]).getOffsetsPtr());
}
else
return executeUnaligned(holders, tuple_columns, input_rows_count, has_unaligned);
}
private:
ColumnPtr executeUnaligned(const Columns & holders, Columns & tuple_columns, size_t input_rows_count, bool has_unaligned) const
{
std::vector<const ColumnArray *> array_columns(holders.size());
for (size_t i = 0; i < holders.size(); ++i)
array_columns[i] = checkAndGetColumn<ColumnArray>(holders[i].get());
for (auto & tuple_column : tuple_columns)
tuple_column = makeNullable(tuple_column);
if (!has_unaligned)
return ColumnArray::create(ColumnTuple::create(std::move(tuple_columns)), array_columns[0]->getOffsetsPtr());
MutableColumns res_tuple_columns(tuple_columns.size());
for (size_t i = 0; i < tuple_columns.size(); ++i)
{
res_tuple_columns[i] = tuple_columns[i]->cloneEmpty();
res_tuple_columns[i]->reserve(tuple_columns[i]->size());
}
auto res_offsets_column = ColumnArray::ColumnOffsets::create(input_rows_count);
auto & res_offsets = assert_cast<ColumnArray::ColumnOffsets &>(*res_offsets_column).getData();
size_t curr_offset = 0;
for (size_t row_i = 0; row_i < input_rows_count; ++row_i)
{
size_t max_size = 0;
for (size_t arg_i = 0; arg_i < holders.size(); ++arg_i)
{
const auto * array_column = array_columns[arg_i];
const auto & offsets = array_column->getOffsets();
size_t array_offset = offsets[row_i - 1];
size_t array_size = offsets[row_i] - array_offset;
res_tuple_columns[arg_i]->insertRangeFrom(*tuple_columns[arg_i], array_offset, array_size);
max_size = std::max(max_size, array_size);
}
for (size_t arg_i = 0; arg_i < holders.size(); ++arg_i)
{
const auto * array_column = array_columns[arg_i];
const auto & offsets = array_column->getOffsets();
size_t array_offset = offsets[row_i - 1];
size_t array_size = offsets[row_i] - array_offset;
res_tuple_columns[arg_i]->insertManyDefaults(max_size - array_size);
}
curr_offset += max_size;
res_offsets[row_i] = curr_offset;
}
return ColumnArray::create(ColumnTuple::create(std::move(res_tuple_columns)), std::move(res_offsets_column));
}
};
REGISTER_FUNCTION(ArrayZip)
{
factory.registerFunction<FunctionArrayZip>();
factory.registerFunction<FunctionArrayZip<false>>(
{.description = R"(
Combines multiple arrays into a single array. The resulting array contains the corresponding elements of the source arrays grouped into tuples in the listed order of arguments.
)",
.categories{"String"}});
factory.registerFunction<FunctionArrayZip<true>>(
{.description = R"(
Combines multiple arrays into a single array, allowing for unaligned arrays. The resulting array contains the corresponding elements of the source arrays grouped into tuples in the listed order of arguments.
If the arrays have different sizes, the shorter arrays will be padded with `null` values.
)",
.categories{"String"}}
);
}
}

View File

@ -315,7 +315,18 @@ void AsynchronousInsertQueue::preprocessInsertQuery(const ASTPtr & query, const
auto sample_block = InterpreterInsertQuery::getSampleBlock(insert_query, table, table->getInMemoryMetadataPtr(), query_context);
if (!FormatFactory::instance().isInputFormat(insert_query.format))
{
if (insert_query.format.empty() && insert_query.infile)
{
const auto & in_file_node = insert_query.infile->as<ASTLiteral &>();
const auto in_file = in_file_node.value.safeGet<std::string>();
const auto in_file_format = FormatFactory::instance().getFormatFromFileName(in_file);
if (!FormatFactory::instance().isInputFormat(in_file_format))
throw Exception(ErrorCodes::UNKNOWN_FORMAT, "Unknown input INFILE format {}", in_file_format);
}
else
throw Exception(ErrorCodes::UNKNOWN_FORMAT, "Unknown input format {}", insert_query.format);
}
/// For table functions we check access while executing
/// InterpreterInsertQuery::getTable() -> ITableFunction::execute().

View File

@ -821,19 +821,6 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti
{
properties.indices = as_storage_metadata->getSecondaryIndices();
properties.projections = as_storage_metadata->getProjections().clone();
/// CREATE TABLE AS should copy PRIMARY KEY, ORDER BY, and similar clauses.
if (!create.storage->primary_key && as_storage_metadata->isPrimaryKeyDefined() && as_storage_metadata->hasPrimaryKey())
create.storage->set(create.storage->primary_key, as_storage_metadata->getPrimaryKeyAST()->clone());
if (!create.storage->partition_by && as_storage_metadata->isPartitionKeyDefined() && as_storage_metadata->hasPartitionKey())
create.storage->set(create.storage->partition_by, as_storage_metadata->getPartitionKeyAST()->clone());
if (!create.storage->order_by && as_storage_metadata->isSortingKeyDefined() && as_storage_metadata->hasSortingKey())
create.storage->set(create.storage->order_by, as_storage_metadata->getSortingKeyAST()->clone());
if (!create.storage->sample_by && as_storage_metadata->isSamplingKeyDefined() && as_storage_metadata->hasSamplingKey())
create.storage->set(create.storage->sample_by, as_storage_metadata->getSamplingKeyAST()->clone());
}
else
{

View File

@ -33,25 +33,19 @@ public:
}
};
/// Duplicate of code from StringUtils.h. Copied here for less dependencies.
static bool startsWith(const std::string & s, const char * prefix)
{
return s.size() >= strlen(prefix) && 0 == memcmp(s.data(), prefix, strlen(prefix));
}
Lemmatizers::Lemmatizers(const Poco::Util::AbstractConfiguration & config)
{
String prefix = "lemmatizers";
Poco::Util::AbstractConfiguration::Keys keys;
const String prefix = "lemmatizers";
if (!config.has(prefix))
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "No lemmatizers specified in server config on prefix '{}'", prefix);
return;
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(prefix, keys);
for (const auto & key : keys)
{
if (startsWith(key, "lemmatizer"))
if (key.starts_with("lemmatizer"))
{
const auto & lemm_name = config.getString(prefix + "." + key + ".lang", "");
const auto & lemm_path = config.getString(prefix + "." + key + ".path", "");
@ -81,13 +75,13 @@ Lemmatizers::LemmPtr Lemmatizers::getLemmatizer(const String & name)
if (paths.find(name) != paths.end())
{
if (!std::filesystem::exists(paths[name]))
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Incorrect path to lemmatizer: {}", paths[name]);
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Path to lemmatizer does not exist: {}", paths[name]);
lemmatizers[name] = std::make_shared<Lemmatizer>(paths[name]);
return lemmatizers[name];
}
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Lemmatizer named: '{}' is not found", name);
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Lemmatizer with the name '{}' was not found in the configuration", name);
}
}

View File

@ -1,70 +0,0 @@
-------------- Test copy sorting clauses from source table --------------
CREATE TABLE default.x
(
`CounterID` UInt32,
`EventDate` Date,
`UserID` UInt64
)
ENGINE = MergeTree
PARTITION BY toYYYYMM(EventDate)
ORDER BY (CounterID, EventDate, intHash32(UserID))
SAMPLE BY intHash32(UserID)
SETTINGS index_granularity = 8192
-------------------------------------------------------------------------
CREATE TABLE default.x_as
(
`CounterID` UInt32,
`EventDate` Date,
`UserID` UInt64
)
ENGINE = MergeTree
PARTITION BY toYYYYMM(EventDate)
ORDER BY (CounterID, EventDate, intHash32(UserID))
SAMPLE BY intHash32(UserID)
SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1, index_granularity = 8192
-------------- Test copy sorting clauses from destination table (source table without the same type clauses) --------------
CREATE TABLE default.x
(
`CounterID` UInt32,
`EventDate` Date,
`UserID` UInt64
)
ENGINE = MergeTree
PRIMARY KEY (CounterID, EventDate, intHash32(UserID))
ORDER BY (CounterID, EventDate, intHash32(UserID))
SETTINGS index_granularity = 8192
-------------------------------------------------------------------------
CREATE TABLE default.x_as
(
`CounterID` UInt32,
`EventDate` Date,
`UserID` UInt64
)
ENGINE = MergeTree
PARTITION BY toYYYYMM(EventDate)
PRIMARY KEY (CounterID, EventDate, intHash32(UserID))
ORDER BY (CounterID, EventDate, intHash32(UserID))
SAMPLE BY intHash32(UserID)
SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1, index_granularity = 8192
-------------- Test copy sorting clauses from destination table (source table with the same type clauses) --------------
CREATE TABLE default.x
(
`CounterID` UInt32,
`EventDate` Date,
`UserID` UInt64
)
ENGINE = MergeTree
ORDER BY CounterID
SETTINGS index_granularity = 8192
-------------------------------------------------------------------------
CREATE TABLE default.x_as
(
`CounterID` UInt32,
`EventDate` Date,
`UserID` UInt64
)
ENGINE = MergeTree
PARTITION BY toYYYYMM(EventDate)
ORDER BY (CounterID, EventDate, intHash32(UserID))
SAMPLE BY intHash32(UserID)
SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1, index_granularity = 8192

View File

@ -1,37 +0,0 @@
DROP TABLE IF EXISTS x;
DROP TABLE IF EXISTS x_as;
SELECT '-------------- Test copy sorting clauses from source table --------------';
CREATE TABLE x (`CounterID` UInt32, `EventDate` Date, `UserID` UInt64) ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID);
CREATE TABLE x_as AS x ENGINE = MergeTree SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1;
SHOW CREATE TABLE x FORMAT TSVRaw;
SELECT '-------------------------------------------------------------------------';
SHOW CREATE TABLE x_as FORMAT TSVRaw;
DROP TABLE x;
DROP TABLE x_as;
SELECT '-------------- Test copy sorting clauses from destination table (source table without the same type clauses) --------------';
CREATE TABLE x (`CounterID` UInt32, `EventDate` Date, `UserID` UInt64) ENGINE = MergeTree PRIMARY KEY (CounterID, EventDate, intHash32(UserID));
CREATE TABLE x_as AS x ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1;
SHOW CREATE TABLE x FORMAT TSVRaw;
SELECT '-------------------------------------------------------------------------';
SHOW CREATE TABLE x_as FORMAT TSVRaw;
DROP TABLE x;
DROP TABLE x_as;
SELECT '-------------- Test copy sorting clauses from destination table (source table with the same type clauses) --------------';
CREATE TABLE x (`CounterID` UInt32, `EventDate` Date, `UserID` UInt64) ENGINE = MergeTree ORDER BY (CounterID);
CREATE TABLE x_as AS x ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1;
SHOW CREATE TABLE x FORMAT TSVRaw;
SELECT '-------------------------------------------------------------------------';
SHOW CREATE TABLE x_as FORMAT TSVRaw;
DROP TABLE x;
DROP TABLE x_as;

View File

@ -143,7 +143,6 @@ arrayStringConcat
arraySum
arrayUniq
arrayWithConstant
arrayZip
asinh
assumeNotNull
atan

View File

@ -0,0 +1,8 @@
[('a','d'),('b','e'),('c','f')] Array(Tuple(Nullable(String), Nullable(String)))
[('a','d','g'),('b','e','h'),('c','f','i')]
[('a','d'),('b','e'),('c','f'),(NULL,'g')]
[('a',1),(NULL,2),(NULL,3)]
[('a',1,1.1),('b',2,2.2),('c',NULL,3.3),(NULL,NULL,4.4)]
[('g'),('h'),('i')]
[('g'),('h'),('i')]
[('g'),('h'),('i')]

View File

@ -0,0 +1,15 @@
SELECT arrayZipUnaligned(['a', 'b', 'c'], ['d', 'e', 'f']) as x, toTypeName(x);
SELECT arrayZipUnaligned(['a', 'b', 'c'], ['d', 'e', 'f'], ['g', 'h', 'i']);
SELECT arrayZipUnaligned(); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION }
SELECT arrayZipUnaligned('a', 'b', 'c'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT arrayZipUnaligned(['a', 'b', 'c'], ['d', 'e', 'f', 'g']);
SELECT arrayZipUnaligned(['a'], [1, 2, 3]);
SELECT arrayZipUnaligned(['a', 'b', 'c'], [1, 2], [1.1, 2.2, 3.3, 4.4]);
SELECT arrayZipUnaligned(materialize(['g', 'h', 'i'])) from numbers(3);

View File

@ -0,0 +1,3 @@
1 ClickHouse
2 HelloWorld
OK

View File

@ -0,0 +1,25 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
function cleanup()
{
[ -e "${CLICKHOUSE_TMP}"/test_infile.csv ] && rm "${CLICKHOUSE_TMP}"/test_infile.csv
}
trap cleanup EXIT
cleanup
echo -e "id,\"word\"\n1,\"ClickHouse\"\n2,\"HelloWorld\"" > "${CLICKHOUSE_TMP}"/test_infile.csv
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS async_insert_infile_data;"
${CLICKHOUSE_CLIENT} --query "CREATE TABLE async_insert_infile_data (id UInt32, word String) ENGINE=Memory();"
${CLICKHOUSE_CLIENT} --query "INSERT INTO async_insert_infile_data FROM INFILE '${CLICKHOUSE_TMP}/test_infile.csv' SETTINGS async_insert=1;"
${CLICKHOUSE_CLIENT} --query "SELECT * FROM async_insert_infile_data ORDER BY id;"
${CLICKHOUSE_CLIENT} --query "INSERT INTO async_insert_infile_data FROM INFILE '${CLICKHOUSE_TMP}/test_infile.csv' SETTINGS async_insert=1 FORMAT NotExists;" 2>&1 | grep -q "UNKNOWN_FORMAT" && echo OK || echo FAIL
${CLICKHOUSE_CLIENT} --query "DROP TABLE async_insert_infile_data SYNC;"

View File

@ -1210,6 +1210,7 @@ arraySum
arrayUniq
arrayWithConstant
arrayZip
arrayZipUnaligned
ascii
asin
asinh