mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Merge branch 'master' of github.com:yandex/ClickHouse
This commit is contained in:
commit
47934d1cce
@ -2,10 +2,14 @@
|
||||
|
||||
ClickHouse is an open-source column-oriented database management system that allows generating analytical data reports in real time.
|
||||
|
||||
## Useful links
|
||||
## Useful Links
|
||||
|
||||
* [Official website](https://clickhouse.yandex/) has quick high-level overview of ClickHouse on main page.
|
||||
* [Tutorial](https://clickhouse.yandex/tutorial.html) shows how to set up and query small ClickHouse cluster.
|
||||
* [Documentation](https://clickhouse.yandex/docs/en/) provides more in-depth information.
|
||||
* [Contacts](https://clickhouse.yandex/#contacts) can help to get your questions answered if there are any.
|
||||
|
||||
## Upcoming Meetups
|
||||
|
||||
* [Paris on October 2](http://bit.ly/clickhouse-paris-2-october-2018)
|
||||
* [Beijing on October 28](http://www.clickhouse.com.cn/topic/5ba0e3f99d28dfde2ddc62a1)
|
||||
|
@ -1,11 +1,11 @@
|
||||
# This strings autochanged from release_lib.sh:
|
||||
set(VERSION_REVISION 54407 CACHE STRING "")
|
||||
set(VERSION_REVISION 54408 CACHE STRING "")
|
||||
set(VERSION_MAJOR 18 CACHE STRING "")
|
||||
set(VERSION_MINOR 12 CACHE STRING "")
|
||||
set(VERSION_PATCH 17 CACHE STRING "")
|
||||
set(VERSION_GITHASH 13406d876ce264183c162a46974a39d8897cbf7b CACHE STRING "")
|
||||
set(VERSION_DESCRIBE v18.12.17-testing CACHE STRING "")
|
||||
set(VERSION_STRING 18.12.17 CACHE STRING "")
|
||||
set(VERSION_MINOR 13 CACHE STRING "")
|
||||
set(VERSION_PATCH 0 CACHE STRING "")
|
||||
set(VERSION_GITHASH 9613b980735b68ad7732cd0ce94ed3b115b76683 CACHE STRING "")
|
||||
set(VERSION_DESCRIBE v18.13.0-testing CACHE STRING "")
|
||||
set(VERSION_STRING 18.13.0 CACHE STRING "")
|
||||
# end of autochange
|
||||
|
||||
set(VERSION_EXTRA "" CACHE STRING "")
|
||||
|
@ -13,12 +13,28 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
struct AggregateFunctionAvgData
|
||||
{
|
||||
T sum = 0;
|
||||
UInt64 count = 0;
|
||||
|
||||
template <typename ResultT>
|
||||
ResultT result() const
|
||||
{
|
||||
if constexpr (std::is_floating_point_v<ResultT>)
|
||||
if constexpr (std::numeric_limits<ResultT>::is_iec559)
|
||||
return static_cast<ResultT>(sum) / count; /// allow division by zero
|
||||
|
||||
if (!count)
|
||||
throw Exception("AggregateFunctionAvg with zero values", ErrorCodes::LOGICAL_ERROR);
|
||||
return static_cast<ResultT>(sum) / count;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
@ -80,7 +96,7 @@ public:
|
||||
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
|
||||
{
|
||||
auto & column = static_cast<ColVecResult &>(to);
|
||||
column.getData().push_back(static_cast<ResultType>(this->data(place).sum) / this->data(place).count);
|
||||
column.getData().push_back(this->data(place).template result<ResultType>());
|
||||
}
|
||||
|
||||
const char * getHeaderFilePath() const override { return __FILE__; }
|
||||
|
@ -135,7 +135,7 @@ ASTPtr createASTIdentifierForColumnInTable(const String & column, const CollectT
|
||||
{
|
||||
ASTPtr database_name_identifier_node;
|
||||
if (!table.database_name.empty())
|
||||
database_name_identifier_node = std::make_shared<ASTIdentifier>(table.database_name, ASTIdentifier::Column);
|
||||
database_name_identifier_node = std::make_shared<ASTIdentifier>(table.database_name);
|
||||
|
||||
ASTPtr table_name_identifier_node;
|
||||
String table_name_or_alias;
|
||||
@ -146,9 +146,9 @@ ASTPtr createASTIdentifierForColumnInTable(const String & column, const CollectT
|
||||
table_name_or_alias = table.alias;
|
||||
|
||||
if (!table_name_or_alias.empty())
|
||||
table_name_identifier_node = std::make_shared<ASTIdentifier>(table_name_or_alias, ASTIdentifier::Column);
|
||||
table_name_identifier_node = std::make_shared<ASTIdentifier>(table_name_or_alias);
|
||||
|
||||
ASTPtr column_identifier_node = std::make_shared<ASTIdentifier>(column, ASTIdentifier::Column);
|
||||
ASTPtr column_identifier_node = std::make_shared<ASTIdentifier>(column);
|
||||
|
||||
String compound_name;
|
||||
if (database_name_identifier_node)
|
||||
@ -157,7 +157,7 @@ ASTPtr createASTIdentifierForColumnInTable(const String & column, const CollectT
|
||||
compound_name += table_name_or_alias + ".";
|
||||
compound_name += column;
|
||||
|
||||
auto elem = std::make_shared<ASTIdentifier>(compound_name, ASTIdentifier::Column);
|
||||
auto elem = std::make_shared<ASTIdentifier>(compound_name);
|
||||
|
||||
if (database_name_identifier_node)
|
||||
elem->children.emplace_back(std::move(database_name_identifier_node));
|
||||
|
@ -2,7 +2,6 @@
|
||||
|
||||
#include <Core/Field.h>
|
||||
#include <Core/AccurateComparison.h>
|
||||
#include <common/DateLUT.h>
|
||||
#include <common/demangle.h>
|
||||
|
||||
|
||||
|
@ -91,6 +91,7 @@
|
||||
M(CompileFunction, "Number of times a compilation of generated LLVM code (to create fused function for complex expressions) was initiated.") \
|
||||
M(CompiledFunctionExecute, "Number of times a compiled function was executed.") \
|
||||
M(CompileExpressionsMicroseconds, "Total time spent for compilation of expressions to LLVM code.") \
|
||||
M(CompileExpressionsBytes, "Number of bytes used for expressions compilation.") \
|
||||
\
|
||||
M(ExternalSortWritePart, "") \
|
||||
M(ExternalSortMerge, "") \
|
||||
|
@ -74,6 +74,9 @@ inline bool isAlphaASCII(char c)
|
||||
|
||||
inline bool isNumericASCII(char c)
|
||||
{
|
||||
/// This is faster than
|
||||
/// return UInt8(UInt8(c) - UInt8('0')) < UInt8(10);
|
||||
/// on Intel CPUs when compiled by gcc 8.
|
||||
return (c >= '0' && c <= '9');
|
||||
}
|
||||
|
||||
|
@ -908,20 +908,33 @@ void ZooKeeper::connect(
|
||||
in.emplace(socket);
|
||||
out.emplace(socket);
|
||||
|
||||
sendHandshake();
|
||||
receiveHandshake();
|
||||
try
|
||||
{
|
||||
sendHandshake();
|
||||
}
|
||||
catch (DB::Exception & e)
|
||||
{
|
||||
e.addMessage("while sending handshake to ZooKeeper");
|
||||
throw;
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
receiveHandshake();
|
||||
}
|
||||
catch (DB::Exception & e)
|
||||
{
|
||||
e.addMessage("while receiving handshake from ZooKeeper");
|
||||
throw;
|
||||
}
|
||||
|
||||
connected = true;
|
||||
break;
|
||||
}
|
||||
catch (const Poco::Net::NetException &)
|
||||
catch (...)
|
||||
{
|
||||
fail_reasons << "\n" << getCurrentExceptionMessage(false) << ", " << address.toString();
|
||||
}
|
||||
catch (const Poco::TimeoutException &)
|
||||
{
|
||||
fail_reasons << "\n" << getCurrentExceptionMessage(false);
|
||||
}
|
||||
}
|
||||
|
||||
if (connected)
|
||||
|
@ -13,18 +13,27 @@ namespace DB
|
||||
|
||||
void AddingDefaultBlockOutputStream::write(const Block & block)
|
||||
{
|
||||
Block res = block;
|
||||
Block res;
|
||||
/// We take given columns from input block
|
||||
/// and missed columns without default value (default and meterialized will be computed later)
|
||||
for (const auto & column : output_block)
|
||||
{
|
||||
if (block.has(column.name))
|
||||
res.insert(block.getByName(column.name));
|
||||
else if (!column_defaults.count(column.name))
|
||||
res.insert(column);
|
||||
}
|
||||
|
||||
/// Adds not specified default values.
|
||||
size_t rows = res.rows();
|
||||
size_t rows = block.rows();
|
||||
|
||||
/// For missing columns of nested structure, you need to create not a column of empty arrays, but a column of arrays of correct lengths.
|
||||
/// First, remember the offset columns for all arrays in the block.
|
||||
std::map<String, ColumnPtr> offset_columns;
|
||||
|
||||
for (size_t i = 0, size = res.columns(); i < size; ++i)
|
||||
for (size_t i = 0, size = block.columns(); i < size; ++i)
|
||||
{
|
||||
const auto & elem = res.getByPosition(i);
|
||||
const auto & elem = block.getByPosition(i);
|
||||
|
||||
if (const ColumnArray * array = typeid_cast<const ColumnArray *>(&*elem.column))
|
||||
{
|
||||
@ -37,38 +46,33 @@ void AddingDefaultBlockOutputStream::write(const Block & block)
|
||||
}
|
||||
}
|
||||
|
||||
for (const auto & requested_column : required_columns)
|
||||
/// In this loop we fill missed columns
|
||||
for (auto & column : res)
|
||||
{
|
||||
if (res.has(requested_column.name) || column_defaults.count(requested_column.name))
|
||||
if (block.has(column.name))
|
||||
continue;
|
||||
|
||||
ColumnWithTypeAndName column_to_add;
|
||||
column_to_add.name = requested_column.name;
|
||||
column_to_add.type = requested_column.type;
|
||||
|
||||
String offsets_name = Nested::extractTableName(column_to_add.name);
|
||||
String offsets_name = Nested::extractTableName(column.name);
|
||||
if (offset_columns.count(offsets_name))
|
||||
{
|
||||
ColumnPtr offsets_column = offset_columns[offsets_name];
|
||||
DataTypePtr nested_type = typeid_cast<const DataTypeArray &>(*column_to_add.type).getNestedType();
|
||||
DataTypePtr nested_type = typeid_cast<const DataTypeArray &>(*column.type).getNestedType();
|
||||
UInt64 nested_rows = rows ? get<UInt64>((*offsets_column)[rows - 1]) : 0;
|
||||
|
||||
ColumnPtr nested_column = nested_type->createColumnConstWithDefaultValue(nested_rows)->convertToFullColumnIfConst();
|
||||
column_to_add.column = ColumnArray::create(nested_column, offsets_column);
|
||||
column.column = ColumnArray::create(nested_column, offsets_column);
|
||||
}
|
||||
else
|
||||
{
|
||||
/** It is necessary to turn a constant column into a full column, since in part of blocks (from other parts),
|
||||
* it can be full (or the interpreter may decide that it is constant everywhere).
|
||||
*/
|
||||
column_to_add.column = column_to_add.type->createColumnConstWithDefaultValue(rows)->convertToFullColumnIfConst();
|
||||
column.column = column.type->createColumnConstWithDefaultValue(rows)->convertToFullColumnIfConst();
|
||||
}
|
||||
|
||||
res.insert(std::move(column_to_add));
|
||||
}
|
||||
|
||||
/// Computes explicitly specified values (in column_defaults) by default.
|
||||
evaluateMissingDefaults(res, required_columns, column_defaults, context);
|
||||
/// Computes explicitly specified values (in column_defaults) by default and materialized columns.
|
||||
evaluateMissingDefaults(res, output_block.getNamesAndTypesList(), column_defaults, context);
|
||||
|
||||
output->write(res);
|
||||
}
|
||||
|
@ -10,8 +10,11 @@ namespace DB
|
||||
{
|
||||
|
||||
|
||||
/** Adds missing columns to the block with default values.
|
||||
* These columns are materialized (not constants).
|
||||
/** This stream adds three types of columns into block
|
||||
* 1. Columns, that are missed inside request, but present in table without defaults (missed columns)
|
||||
* 2. Columns, that are missed inside request, but present in table with defaults (columns with default values)
|
||||
* 3. Columns that materialized from other columns (materialized columns)
|
||||
* All three types of columns are materialized (not constants).
|
||||
*/
|
||||
class AddingDefaultBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
@ -19,10 +22,10 @@ public:
|
||||
AddingDefaultBlockOutputStream(
|
||||
const BlockOutputStreamPtr & output_,
|
||||
const Block & header_,
|
||||
NamesAndTypesList required_columns_,
|
||||
const Block & output_block_,
|
||||
const ColumnDefaults & column_defaults_,
|
||||
const Context & context_)
|
||||
: output(output_), header(header_), required_columns(required_columns_),
|
||||
: output(output_), header(header_), output_block(output_block_),
|
||||
column_defaults(column_defaults_), context(context_)
|
||||
{
|
||||
}
|
||||
@ -37,8 +40,9 @@ public:
|
||||
|
||||
private:
|
||||
BlockOutputStreamPtr output;
|
||||
Block header;
|
||||
NamesAndTypesList required_columns;
|
||||
const Block header;
|
||||
/// Blocks after this stream should have this structure
|
||||
const Block output_block;
|
||||
const ColumnDefaults column_defaults;
|
||||
const Context & context;
|
||||
};
|
||||
|
78
dbms/src/DataStreams/CubeBlockInputStream.cpp
Normal file
78
dbms/src/DataStreams/CubeBlockInputStream.cpp
Normal file
@ -0,0 +1,78 @@
|
||||
#include <DataStreams/CubeBlockInputStream.h>
|
||||
#include <DataStreams/finalizeBlock.h>
|
||||
#include <DataTypes/DataTypeAggregateFunction.h>
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
#include <Columns/FilterDescription.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int TOO_MANY_COLUMNS;
|
||||
}
|
||||
|
||||
CubeBlockInputStream::CubeBlockInputStream(
|
||||
const BlockInputStreamPtr & input_, const Aggregator::Params & params_) : aggregator(params_),
|
||||
keys(params_.keys)
|
||||
{
|
||||
if (keys.size() > 30)
|
||||
throw Exception("Too many columns for cube", ErrorCodes::TOO_MANY_COLUMNS);
|
||||
|
||||
children.push_back(input_);
|
||||
Aggregator::CancellationHook hook = [this]() { return this->isCancelled(); };
|
||||
aggregator.setCancellationHook(hook);
|
||||
}
|
||||
|
||||
|
||||
Block CubeBlockInputStream::getHeader() const
|
||||
{
|
||||
Block res = children.at(0)->getHeader();
|
||||
finalizeBlock(res);
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
Block CubeBlockInputStream::readImpl()
|
||||
{
|
||||
/** After reading a block from input stream,
|
||||
* we will calculate all subsets of columns on next iterations of readImpl
|
||||
* by zeroing columns at positions, where bits are zero in current bitmask.
|
||||
*/
|
||||
if (mask)
|
||||
{
|
||||
--mask;
|
||||
Block cube_block = source_block;
|
||||
for (size_t i = 0; i < keys.size(); ++i)
|
||||
{
|
||||
if (!((mask >> i) & 1))
|
||||
{
|
||||
size_t pos = keys.size() - i - 1;
|
||||
auto & current = cube_block.getByPosition(keys[pos]);
|
||||
current.column = zero_block.getByPosition(pos).column;
|
||||
}
|
||||
}
|
||||
|
||||
BlocksList cube_blocks = { cube_block };
|
||||
Block finalized = aggregator.mergeBlocks(cube_blocks, true);
|
||||
return finalized;
|
||||
}
|
||||
|
||||
source_block = children[0]->read();
|
||||
if (!source_block)
|
||||
return source_block;
|
||||
|
||||
zero_block = source_block.cloneEmpty();
|
||||
for (auto key : keys)
|
||||
{
|
||||
auto & current = zero_block.getByPosition(key);
|
||||
current.column = current.column->cloneResized(source_block.rows());
|
||||
}
|
||||
Block finalized = source_block;
|
||||
finalizeBlock(finalized);
|
||||
mask = (1 << keys.size()) - 1;
|
||||
|
||||
return finalized;
|
||||
}
|
||||
}
|
42
dbms/src/DataStreams/CubeBlockInputStream.h
Normal file
42
dbms/src/DataStreams/CubeBlockInputStream.h
Normal file
@ -0,0 +1,42 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataStreams/IProfilingBlockInputStream.h>
|
||||
#include <Common/Arena.h>
|
||||
#include <Interpreters/Aggregator.h>
|
||||
#include <Core/ColumnNumbers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ExpressionActions;
|
||||
|
||||
|
||||
/** Takes blocks after grouping, with non-finalized aggregate functions.
|
||||
* Calculates all subsets of columns and aggreagetes over them.
|
||||
*/
|
||||
class CubeBlockInputStream : public IProfilingBlockInputStream
|
||||
{
|
||||
private:
|
||||
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
||||
using AggregateColumns = std::vector<ColumnRawPtrs>;
|
||||
public:
|
||||
CubeBlockInputStream(
|
||||
const BlockInputStreamPtr & input_, const Aggregator::Params & params_);
|
||||
|
||||
String getName() const override { return "Cube"; }
|
||||
|
||||
Block getHeader() const override;
|
||||
|
||||
protected:
|
||||
Block readImpl() override;
|
||||
|
||||
private:
|
||||
Aggregator aggregator;
|
||||
ColumnNumbers keys;
|
||||
UInt32 mask = 0;
|
||||
Block source_block;
|
||||
Block zero_block;
|
||||
};
|
||||
|
||||
}
|
@ -9,9 +9,9 @@ namespace DB
|
||||
{
|
||||
|
||||
PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
|
||||
const String & database, const String & table, const StoragePtr & storage,
|
||||
const String & database, const String & table, const StoragePtr & storage_,
|
||||
const Context & context_, const ASTPtr & query_ptr_, bool no_destination)
|
||||
: context(context_), query_ptr(query_ptr_)
|
||||
: storage(storage_), context(context_), query_ptr(query_ptr_)
|
||||
{
|
||||
/** TODO This is a very important line. At any insertion into the table one of streams should own lock.
|
||||
* Although now any insertion into the table is done via PushingToViewsBlockOutputStream,
|
||||
|
@ -19,7 +19,7 @@ class PushingToViewsBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
PushingToViewsBlockOutputStream(
|
||||
const String & database, const String & table, const StoragePtr & storage,
|
||||
const String & database, const String & table, const StoragePtr & storage_,
|
||||
const Context & context_, const ASTPtr & query_ptr_, bool no_destination = false);
|
||||
|
||||
Block getHeader() const override { return storage->getSampleBlock(); }
|
||||
|
@ -4,8 +4,8 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
SquashingBlockOutputStream::SquashingBlockOutputStream(BlockOutputStreamPtr & dst, size_t min_block_size_rows, size_t min_block_size_bytes)
|
||||
: output(dst), header(output->getHeader()), transform(min_block_size_rows, min_block_size_bytes)
|
||||
SquashingBlockOutputStream::SquashingBlockOutputStream(BlockOutputStreamPtr & dst, const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes)
|
||||
: output(dst), header(header), transform(min_block_size_rows, min_block_size_bytes)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -12,7 +12,7 @@ namespace DB
|
||||
class SquashingBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
SquashingBlockOutputStream(BlockOutputStreamPtr & dst, size_t min_block_size_rows, size_t min_block_size_bytes);
|
||||
SquashingBlockOutputStream(BlockOutputStreamPtr & dst, const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes);
|
||||
|
||||
Block getHeader() const override { return header; }
|
||||
void write(const Block & block) override;
|
||||
|
@ -360,31 +360,6 @@ void DataTypeArray::deserializeText(IColumn & column, ReadBuffer & istr, const F
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
void DataTypeArray::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
|
||||
void DataTypeArray::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
deserializeText(column, istr, settings);
|
||||
}
|
||||
|
||||
|
||||
void DataTypeArray::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
|
||||
void DataTypeArray::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
deserializeText(column, istr, settings);
|
||||
}
|
||||
|
||||
|
||||
void DataTypeArray::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
const ColumnArray & column_array = static_cast<const ColumnArray &>(column);
|
||||
|
@ -1,13 +1,13 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <DataTypes/DataTypeWithSimpleSerialization.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
class DataTypeArray final : public IDataType
|
||||
class DataTypeArray final : public DataTypeWithSimpleSerialization
|
||||
{
|
||||
private:
|
||||
/// The type of array elements.
|
||||
@ -41,13 +41,7 @@ public:
|
||||
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override;
|
||||
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
|
||||
|
||||
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
|
||||
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
|
||||
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
|
@ -14,7 +14,7 @@ void DataTypeDate::serializeText(const IColumn & column, size_t row_num, WriteBu
|
||||
writeDateText(DayNum(static_cast<const ColumnUInt16 &>(column).getData()[row_num]), ostr);
|
||||
}
|
||||
|
||||
static void deserializeText(IColumn & column, ReadBuffer & istr)
|
||||
void DataTypeDate::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
DayNum x;
|
||||
readDateText(x, istr);
|
||||
@ -26,11 +26,6 @@ void DataTypeDate::serializeTextEscaped(const IColumn & column, size_t row_num,
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void DataTypeDate::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
deserializeText(column, istr);
|
||||
}
|
||||
|
||||
void DataTypeDate::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
writeChar('\'', ostr);
|
||||
|
@ -19,14 +19,7 @@ void DataTypeNumberBase<T>::serializeText(const IColumn & column, size_t row_num
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void DataTypeNumberBase<T>::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
|
||||
template <typename T>
|
||||
static void deserializeText(IColumn & column, ReadBuffer & istr)
|
||||
void DataTypeNumberBase<T>::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
T x;
|
||||
|
||||
@ -38,26 +31,6 @@ static void deserializeText(IColumn & column, ReadBuffer & istr)
|
||||
static_cast<ColumnVector<T> &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
|
||||
template <typename T>
|
||||
void DataTypeNumberBase<T>::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
deserializeText<T>(column, istr);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void DataTypeNumberBase<T>::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void DataTypeNumberBase<T>::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
deserializeText<T>(column, istr);
|
||||
}
|
||||
|
||||
|
||||
template <typename T>
|
||||
static inline void writeDenormalNumber(T x, WriteBuffer & ostr)
|
||||
{
|
||||
@ -160,12 +133,6 @@ void DataTypeNumberBase<T>::deserializeTextJSON(IColumn & column, ReadBuffer & i
|
||||
static_cast<ColumnVector<T> &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void DataTypeNumberBase<T>::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void DataTypeNumberBase<T>::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <DataTypes/DataTypeWithSimpleSerialization.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -9,7 +10,7 @@ namespace DB
|
||||
/** Implements part of the IDataType interface, common to all numbers and for Date and DateTime.
|
||||
*/
|
||||
template <typename T>
|
||||
class DataTypeNumberBase : public IDataType
|
||||
class DataTypeNumberBase : public DataTypeWithSimpleSerialization
|
||||
{
|
||||
static_assert(IsNumber<T>);
|
||||
|
||||
@ -21,13 +22,9 @@ public:
|
||||
TypeIndex getTypeId() const override { return TypeId<T>::value; }
|
||||
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
Field getDefault() const override;
|
||||
|
||||
|
@ -188,26 +188,6 @@ void DataTypeTuple::deserializeText(IColumn & column, ReadBuffer & istr, const F
|
||||
assertChar(')', istr);
|
||||
}
|
||||
|
||||
void DataTypeTuple::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void DataTypeTuple::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
deserializeText(column, istr, settings);
|
||||
}
|
||||
|
||||
void DataTypeTuple::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void DataTypeTuple::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
deserializeText(column, istr, settings);
|
||||
}
|
||||
|
||||
void DataTypeTuple::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
writeChar('[', ostr);
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <DataTypes/DataTypeWithSimpleSerialization.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -16,7 +16,7 @@ namespace DB
|
||||
*
|
||||
* All tuples with same size and types of elements are equivalent for expressions, regardless to names of elements.
|
||||
*/
|
||||
class DataTypeTuple final : public IDataType
|
||||
class DataTypeTuple final : public DataTypeWithSimpleSerialization
|
||||
{
|
||||
private:
|
||||
DataTypes elems;
|
||||
@ -39,11 +39,7 @@ public:
|
||||
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;
|
||||
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override;
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
|
||||
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
|
@ -13,7 +13,7 @@ void DataTypeUUID::serializeText(const IColumn & column, size_t row_num, WriteBu
|
||||
writeText(UUID(static_cast<const ColumnUInt128 &>(column).getData()[row_num]), ostr);
|
||||
}
|
||||
|
||||
static void deserializeText(IColumn & column, ReadBuffer & istr)
|
||||
void DataTypeUUID::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
UUID x;
|
||||
readText(x, istr);
|
||||
@ -25,11 +25,6 @@ void DataTypeUUID::serializeTextEscaped(const IColumn & column, size_t row_num,
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void DataTypeUUID::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
deserializeText(column, istr);
|
||||
}
|
||||
|
||||
void DataTypeUUID::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
writeChar('\'', ostr);
|
||||
|
58
dbms/src/DataTypes/DataTypeWithSimpleSerialization.h
Normal file
58
dbms/src/DataTypes/DataTypeWithSimpleSerialization.h
Normal file
@ -0,0 +1,58 @@
|
||||
#pragma once
|
||||
#include <DataTypes/IDataType.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Helper class to define same IDataType text (de)serialization for all the variants (escaped, quoted, JSON, CSV).
|
||||
/// You need to define serializeText() and deserializeText() in derived class.
|
||||
class DataTypeWithSimpleSerialization : public IDataType
|
||||
{
|
||||
protected:
|
||||
DataTypeWithSimpleSerialization()
|
||||
{}
|
||||
|
||||
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||
{
|
||||
deserializeText(column, istr, settings);
|
||||
}
|
||||
|
||||
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||
{
|
||||
deserializeText(column, istr, settings);
|
||||
}
|
||||
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||
{
|
||||
deserializeText(column, istr, settings);
|
||||
}
|
||||
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||
{
|
||||
deserializeText(column, istr, settings);
|
||||
}
|
||||
|
||||
virtual void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
|
||||
};
|
||||
|
||||
}
|
@ -4,6 +4,7 @@
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeWithSimpleSerialization.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -16,53 +17,6 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_CONVERT_TYPE;
|
||||
}
|
||||
|
||||
///
|
||||
class DataTypeSimpleSerialization : public IDataType
|
||||
{
|
||||
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||
{
|
||||
deserializeText(column, istr, settings);
|
||||
}
|
||||
|
||||
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||
{
|
||||
deserializeText(column, istr, settings);
|
||||
}
|
||||
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||
{
|
||||
deserializeText(column, istr, settings);
|
||||
}
|
||||
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||
{
|
||||
deserializeText(column, istr, settings);
|
||||
}
|
||||
|
||||
virtual void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
|
||||
};
|
||||
|
||||
|
||||
class Context;
|
||||
bool decimalCheckComparisonOverflow(const Context & context);
|
||||
bool decimalCheckArithmeticOverflow(const Context & context);
|
||||
@ -89,7 +43,7 @@ DataTypePtr createDecimal(UInt64 precision, UInt64 scale);
|
||||
///
|
||||
/// NOTE: It's possible to set scale as a template parameter then most of functions become static.
|
||||
template <typename T>
|
||||
class DataTypeDecimal final : public DataTypeSimpleSerialization
|
||||
class DataTypeDecimal final : public DataTypeWithSimpleSerialization
|
||||
{
|
||||
static_assert(IsDecimalNumber<T>);
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <DataTypes/DataTypeWithSimpleSerialization.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -11,7 +11,7 @@ namespace DB
|
||||
*
|
||||
* That is, this class is used just to distinguish the corresponding data type from the others.
|
||||
*/
|
||||
class IDataTypeDummy : public IDataType
|
||||
class IDataTypeDummy : public DataTypeWithSimpleSerialization
|
||||
{
|
||||
private:
|
||||
void throwNoSerialization() const
|
||||
@ -26,15 +26,8 @@ public:
|
||||
void deserializeBinary(IColumn &, ReadBuffer &) const override { throwNoSerialization(); }
|
||||
void serializeBinaryBulk(const IColumn &, WriteBuffer &, size_t, size_t) const override { throwNoSerialization(); }
|
||||
void deserializeBinaryBulk(IColumn &, ReadBuffer &, size_t, double) const override { throwNoSerialization(); }
|
||||
void serializeText(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
|
||||
void serializeTextEscaped(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
|
||||
void deserializeTextEscaped(IColumn &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
|
||||
void serializeTextQuoted(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
|
||||
void deserializeTextQuoted(IColumn &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
|
||||
void serializeTextJSON(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
|
||||
void deserializeTextJSON(IColumn &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
|
||||
void serializeTextCSV(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
|
||||
void deserializeTextCSV(IColumn &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
|
||||
void serializeText(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
|
||||
void deserializeText(IColumn &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
|
||||
|
||||
MutableColumnPtr createColumn() const override
|
||||
{
|
||||
|
@ -18,7 +18,7 @@ namespace DB
|
||||
* BlockInputStream implementation for external dictionaries
|
||||
* read() returns single block consisting of the in-memory contents of the dictionaries
|
||||
*/
|
||||
template <typename DictionaryType, typename Key>
|
||||
template <typename DictionaryType, typename RangeType, typename Key>
|
||||
class RangeDictionaryBlockInputStream : public DictionaryBlockInputStreamBase
|
||||
{
|
||||
public:
|
||||
@ -26,7 +26,7 @@ public:
|
||||
|
||||
RangeDictionaryBlockInputStream(
|
||||
DictionaryPtr dictionary, size_t max_block_size, const Names & column_names, PaddedPODArray<Key> && ids_to_fill,
|
||||
PaddedPODArray<Int64> && start_dates, PaddedPODArray<Int64> && end_dates);
|
||||
PaddedPODArray<RangeType> && start_dates, PaddedPODArray<RangeType> && end_dates);
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
@ -57,35 +57,35 @@ private:
|
||||
const PaddedPODArray<T> & values, ColumnsWithTypeAndName & columns) const;
|
||||
|
||||
Block fillBlock(const PaddedPODArray<Key> & ids_to_fill,
|
||||
const PaddedPODArray<Int64> & block_start_dates, const PaddedPODArray<Int64> & block_end_dates) const;
|
||||
const PaddedPODArray<RangeType> & block_start_dates, const PaddedPODArray<RangeType> & block_end_dates) const;
|
||||
|
||||
PaddedPODArray<Int64> makeDateKey(
|
||||
const PaddedPODArray<Int64> & block_start_dates, const PaddedPODArray<Int64> & block_end_dates) const;
|
||||
const PaddedPODArray<RangeType> & block_start_dates, const PaddedPODArray<RangeType> & block_end_dates) const;
|
||||
|
||||
DictionaryPtr dictionary;
|
||||
Names column_names;
|
||||
PaddedPODArray<Key> ids;
|
||||
PaddedPODArray<Int64> start_dates;
|
||||
PaddedPODArray<Int64> end_dates;
|
||||
PaddedPODArray<RangeType> start_dates;
|
||||
PaddedPODArray<RangeType> end_dates;
|
||||
};
|
||||
|
||||
|
||||
template <typename DictionaryType, typename Key>
|
||||
RangeDictionaryBlockInputStream<DictionaryType, Key>::RangeDictionaryBlockInputStream(
|
||||
template <typename DictionaryType, typename RangeType, typename Key>
|
||||
RangeDictionaryBlockInputStream<DictionaryType, RangeType, Key>::RangeDictionaryBlockInputStream(
|
||||
DictionaryPtr dictionary, size_t max_column_size, const Names & column_names, PaddedPODArray<Key> && ids,
|
||||
PaddedPODArray<Int64> && block_start_dates, PaddedPODArray<Int64> && block_end_dates)
|
||||
PaddedPODArray<RangeType> && block_start_dates, PaddedPODArray<RangeType> && block_end_dates)
|
||||
: DictionaryBlockInputStreamBase(ids.size(), max_column_size),
|
||||
dictionary(dictionary), column_names(column_names),
|
||||
ids(std::move(ids)), start_dates(std::move(block_start_dates)), end_dates(std::move(block_end_dates))
|
||||
{
|
||||
}
|
||||
|
||||
template <typename DictionaryType, typename Key>
|
||||
Block RangeDictionaryBlockInputStream<DictionaryType, Key>::getBlock(size_t start, size_t length) const
|
||||
template <typename DictionaryType, typename RangeType, typename Key>
|
||||
Block RangeDictionaryBlockInputStream<DictionaryType, RangeType, Key>::getBlock(size_t start, size_t length) const
|
||||
{
|
||||
PaddedPODArray<Key> block_ids;
|
||||
PaddedPODArray<Int64> block_start_dates;
|
||||
PaddedPODArray<Int64> block_end_dates;
|
||||
PaddedPODArray<RangeType> block_start_dates;
|
||||
PaddedPODArray<RangeType> block_end_dates;
|
||||
block_ids.reserve(length);
|
||||
block_start_dates.reserve(length);
|
||||
block_end_dates.reserve(length);
|
||||
@ -100,9 +100,9 @@ Block RangeDictionaryBlockInputStream<DictionaryType, Key>::getBlock(size_t star
|
||||
return fillBlock(block_ids, block_start_dates, block_end_dates);
|
||||
}
|
||||
|
||||
template <typename DictionaryType, typename Key>
|
||||
template <typename DictionaryType, typename RangeType, typename Key>
|
||||
template <typename AttributeType>
|
||||
ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAttribute(
|
||||
ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, RangeType, Key>::getColumnFromAttribute(
|
||||
DictionaryGetter<AttributeType> getter, const PaddedPODArray<Key> & ids_to_fill,
|
||||
const PaddedPODArray<Int64> & dates, const DictionaryAttribute & attribute, const DictionaryType & concrete_dictionary) const
|
||||
{
|
||||
@ -111,8 +111,8 @@ ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAtt
|
||||
return column_vector;
|
||||
}
|
||||
|
||||
template <typename DictionaryType, typename Key>
|
||||
ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAttributeString(
|
||||
template <typename DictionaryType, typename RangeType, typename Key>
|
||||
ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, RangeType, Key>::getColumnFromAttributeString(
|
||||
const PaddedPODArray<Key> & ids_to_fill, const PaddedPODArray<Int64> & dates,
|
||||
const DictionaryAttribute & attribute, const DictionaryType & concrete_dictionary) const
|
||||
{
|
||||
@ -121,9 +121,9 @@ ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAtt
|
||||
return column_string;
|
||||
}
|
||||
|
||||
template <typename DictionaryType, typename Key>
|
||||
template <typename DictionaryType, typename RangeType, typename Key>
|
||||
template <typename T>
|
||||
ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromPODArray(const PaddedPODArray<T> & array) const
|
||||
ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, RangeType, Key>::getColumnFromPODArray(const PaddedPODArray<T> & array) const
|
||||
{
|
||||
auto column_vector = ColumnVector<T>::create();
|
||||
column_vector->getData().reserve(array.size());
|
||||
@ -133,9 +133,9 @@ ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromPOD
|
||||
}
|
||||
|
||||
|
||||
template <typename DictionaryType, typename Key>
|
||||
template <typename DictionaryType, typename RangeType, typename Key>
|
||||
template <typename DictionarySpecialAttributeType, typename T>
|
||||
void RangeDictionaryBlockInputStream<DictionaryType, Key>::addSpecialColumn(
|
||||
void RangeDictionaryBlockInputStream<DictionaryType, RangeType, Key>::addSpecialColumn(
|
||||
const std::optional<DictionarySpecialAttributeType> & attribute, DataTypePtr type,
|
||||
const std::string & default_name, const std::unordered_set<std::string> & column_names_set,
|
||||
const PaddedPODArray<T> & values, ColumnsWithTypeAndName & columns) const
|
||||
@ -148,9 +148,9 @@ void RangeDictionaryBlockInputStream<DictionaryType, Key>::addSpecialColumn(
|
||||
columns.emplace_back(getColumnFromPODArray(values), type, name);
|
||||
}
|
||||
|
||||
template <typename DictionaryType, typename Key>
|
||||
PaddedPODArray<Int64> RangeDictionaryBlockInputStream<DictionaryType, Key>::makeDateKey(
|
||||
const PaddedPODArray<Int64> & block_start_dates, const PaddedPODArray<Int64> & block_end_dates) const
|
||||
template <typename DictionaryType, typename RangeType, typename Key>
|
||||
PaddedPODArray<Int64> RangeDictionaryBlockInputStream<DictionaryType, RangeType, Key>::makeDateKey(
|
||||
const PaddedPODArray<RangeType> & block_start_dates, const PaddedPODArray<RangeType> & block_end_dates) const
|
||||
{
|
||||
PaddedPODArray<Int64> key(block_start_dates.size());
|
||||
for (size_t i = 0; i < key.size(); ++i)
|
||||
@ -165,10 +165,10 @@ PaddedPODArray<Int64> RangeDictionaryBlockInputStream<DictionaryType, Key>::make
|
||||
}
|
||||
|
||||
|
||||
template <typename DictionaryType, typename Key>
|
||||
Block RangeDictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
|
||||
template <typename DictionaryType, typename RangeType, typename Key>
|
||||
Block RangeDictionaryBlockInputStream<DictionaryType, RangeType, Key>::fillBlock(
|
||||
const PaddedPODArray<Key> & ids_to_fill,
|
||||
const PaddedPODArray<Int64> & block_start_dates, const PaddedPODArray<Int64> & block_end_dates) const
|
||||
const PaddedPODArray<RangeType> & block_start_dates, const PaddedPODArray<RangeType> & block_end_dates) const
|
||||
{
|
||||
ColumnsWithTypeAndName columns;
|
||||
const DictionaryStructure & structure = dictionary->getStructure();
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Dictionaries/RangeDictionaryBlockInputStream.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Common/TypeList.h>
|
||||
#include <ext/range.h>
|
||||
|
||||
|
||||
@ -14,7 +15,7 @@ using RangeStorageType = DB::RangeHashedDictionary::RangeStorageType;
|
||||
// To simplify comparison, null value of min bound should be bigger than any other value,
|
||||
// and null value of maxbound - less than any value.
|
||||
const RangeStorageType RANGE_MIN_NULL_VALUE = std::numeric_limits<RangeStorageType>::max();
|
||||
const RangeStorageType RANGE_MAX_NULL_VALUE = std::numeric_limits<RangeStorageType>::min();
|
||||
const RangeStorageType RANGE_MAX_NULL_VALUE = std::numeric_limits<RangeStorageType>::lowest();
|
||||
|
||||
// Handle both kinds of null values: explicit nulls of NullableColumn and 'implicit' nulls of Date type.
|
||||
RangeStorageType getColumnIntValueOrDefault(const DB::IColumn & column, size_t index, bool isDate, const RangeStorageType & default_value)
|
||||
@ -170,7 +171,7 @@ void RangeHashedDictionary::loadData()
|
||||
const auto & id_column = *block.safeGetByPosition(0).column;
|
||||
|
||||
// Support old behaviour, where invalid date means 'open range'.
|
||||
const bool is_date = isDate(block.safeGetByPosition(1).type);
|
||||
const bool is_date = isDate(dict_struct.range_min->type);
|
||||
|
||||
const auto & min_range_column = unwrapNullableColumn(*block.safeGetByPosition(1).column);
|
||||
const auto & max_range_column = unwrapNullableColumn(*block.safeGetByPosition(2).column);
|
||||
@ -184,11 +185,22 @@ void RangeHashedDictionary::loadData()
|
||||
|
||||
for (const auto row_idx : ext::range(0, id_column.size()))
|
||||
{
|
||||
const auto min = getColumnIntValueOrDefault(min_range_column, row_idx, is_date, RANGE_MIN_NULL_VALUE);
|
||||
const auto max = getColumnIntValueOrDefault(max_range_column, row_idx, is_date, RANGE_MAX_NULL_VALUE);
|
||||
RangeStorageType lower_bound;
|
||||
RangeStorageType upper_bound;
|
||||
|
||||
if (is_date)
|
||||
{
|
||||
lower_bound = getColumnIntValueOrDefault(min_range_column, row_idx, is_date, 0);
|
||||
upper_bound = getColumnIntValueOrDefault(max_range_column, row_idx, is_date, DATE_LUT_MAX_DAY_NUM + 1);
|
||||
}
|
||||
else
|
||||
{
|
||||
lower_bound = getColumnIntValueOrDefault(min_range_column, row_idx, is_date, RANGE_MIN_NULL_VALUE);
|
||||
upper_bound = getColumnIntValueOrDefault(max_range_column, row_idx, is_date, RANGE_MAX_NULL_VALUE);
|
||||
}
|
||||
|
||||
setAttributeValue(attribute, id_column.getUInt(row_idx),
|
||||
Range{min, max},
|
||||
Range{lower_bound, upper_bound},
|
||||
attribute_column[row_idx]);
|
||||
}
|
||||
}
|
||||
@ -416,9 +428,10 @@ const RangeHashedDictionary::Attribute & RangeHashedDictionary::getAttributeWith
|
||||
return attribute;
|
||||
}
|
||||
|
||||
template <typename RangeType>
|
||||
void RangeHashedDictionary::getIdsAndDates(PaddedPODArray<Key> & ids,
|
||||
PaddedPODArray<RangeStorageType> & start_dates,
|
||||
PaddedPODArray<RangeStorageType> & end_dates) const
|
||||
PaddedPODArray<RangeType> & start_dates,
|
||||
PaddedPODArray<RangeType> & end_dates) const
|
||||
{
|
||||
const auto & attribute = attributes.front();
|
||||
|
||||
@ -439,10 +452,10 @@ void RangeHashedDictionary::getIdsAndDates(PaddedPODArray<Key> & ids,
|
||||
}
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
template <typename T, typename RangeType>
|
||||
void RangeHashedDictionary::getIdsAndDates(const Attribute & attribute, PaddedPODArray<Key> & ids,
|
||||
PaddedPODArray<RangeStorageType> & start_dates,
|
||||
PaddedPODArray<RangeStorageType> & end_dates) const
|
||||
PaddedPODArray<RangeType> & start_dates,
|
||||
PaddedPODArray<RangeType> & end_dates) const
|
||||
{
|
||||
const HashMap<UInt64, Values<T>> & attr = *std::get<Ptr<T>>(attribute.maps);
|
||||
|
||||
@ -450,6 +463,8 @@ void RangeHashedDictionary::getIdsAndDates(const Attribute & attribute, PaddedPO
|
||||
start_dates.reserve(attr.size());
|
||||
end_dates.reserve(attr.size());
|
||||
|
||||
const bool is_date = isDate(dict_struct.range_min->type);
|
||||
|
||||
for (const auto & key : attr)
|
||||
{
|
||||
for (const auto & value : key.second)
|
||||
@ -457,22 +472,60 @@ void RangeHashedDictionary::getIdsAndDates(const Attribute & attribute, PaddedPO
|
||||
ids.push_back(key.first);
|
||||
start_dates.push_back(value.range.left);
|
||||
end_dates.push_back(value.range.right);
|
||||
|
||||
if (is_date && static_cast<UInt64>(end_dates.back()) > DATE_LUT_MAX_DAY_NUM)
|
||||
end_dates.back() = 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
BlockInputStreamPtr RangeHashedDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const
|
||||
|
||||
template <typename RangeType>
|
||||
BlockInputStreamPtr RangeHashedDictionary::getBlockInputStreamImpl(const Names & column_names, size_t max_block_size) const
|
||||
{
|
||||
PaddedPODArray<Key> ids;
|
||||
PaddedPODArray<RangeStorageType> start_dates;
|
||||
PaddedPODArray<RangeStorageType> end_dates;
|
||||
PaddedPODArray<RangeType> start_dates;
|
||||
PaddedPODArray<RangeType> end_dates;
|
||||
getIdsAndDates(ids, start_dates, end_dates);
|
||||
|
||||
using BlockInputStreamType = RangeDictionaryBlockInputStream<RangeHashedDictionary, Key>;
|
||||
using BlockInputStreamType = RangeDictionaryBlockInputStream<RangeHashedDictionary, RangeType, Key>;
|
||||
auto dict_ptr = std::static_pointer_cast<const RangeHashedDictionary>(shared_from_this());
|
||||
return std::make_shared<BlockInputStreamType>(
|
||||
dict_ptr, max_block_size, column_names, std::move(ids), std::move(start_dates), std::move(end_dates));
|
||||
}
|
||||
|
||||
struct RangeHashedDIctionaryCallGetBlockInputStreamImpl
|
||||
{
|
||||
BlockInputStreamPtr stream;
|
||||
const RangeHashedDictionary * dict;
|
||||
const Names * column_names;
|
||||
size_t max_block_size;
|
||||
|
||||
template <typename RangeType, size_t>
|
||||
void operator()()
|
||||
{
|
||||
auto & type = dict->dict_struct.range_min->type;
|
||||
if (!stream && dynamic_cast<const DataTypeNumberBase<RangeType> *>(type.get()))
|
||||
stream = dict->getBlockInputStreamImpl<RangeType>(*column_names, max_block_size);
|
||||
}
|
||||
};
|
||||
|
||||
BlockInputStreamPtr RangeHashedDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const
|
||||
{
|
||||
using ListType = TypeList<UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64>;
|
||||
|
||||
RangeHashedDIctionaryCallGetBlockInputStreamImpl callable;
|
||||
callable.dict = this;
|
||||
callable.column_names = &column_names;
|
||||
callable.max_block_size = max_block_size;
|
||||
|
||||
ListType::forEach(callable);
|
||||
|
||||
if (!callable.stream)
|
||||
throw Exception("Unexpected range type for RangeHashed dictionary: " + dict_struct.range_min->type->getName(),
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return callable.stream;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -162,12 +162,18 @@ private:
|
||||
|
||||
const Attribute & getAttributeWithType(const std::string & name, const AttributeUnderlyingType type) const;
|
||||
|
||||
template <typename RangeType>
|
||||
void getIdsAndDates(PaddedPODArray<Key> & ids,
|
||||
PaddedPODArray<RangeStorageType> & start_dates, PaddedPODArray<RangeStorageType> & end_dates) const;
|
||||
PaddedPODArray<RangeType> & start_dates, PaddedPODArray<RangeType> & end_dates) const;
|
||||
|
||||
template <typename T>
|
||||
template <typename T, typename RangeType>
|
||||
void getIdsAndDates(const Attribute & attribute, PaddedPODArray<Key> & ids,
|
||||
PaddedPODArray<RangeStorageType> & start_dates, PaddedPODArray<RangeStorageType> & end_dates) const;
|
||||
PaddedPODArray<RangeType> & start_dates, PaddedPODArray<RangeType> & end_dates) const;
|
||||
|
||||
template <typename RangeType>
|
||||
BlockInputStreamPtr getBlockInputStreamImpl(const Names & column_names, size_t max_block_size) const;
|
||||
|
||||
friend struct RangeHashedDIctionaryCallGetBlockInputStreamImpl;
|
||||
|
||||
const std::string dictionary_name;
|
||||
const DictionaryStructure dict_struct;
|
||||
|
@ -41,6 +41,7 @@ BlockInputStreamPtr FormatFactory::getInput(const String & name, ReadBuffer & bu
|
||||
format_settings.csv.allow_double_quotes = settings.format_csv_allow_double_quotes;
|
||||
format_settings.values.interpret_expressions = settings.input_format_values_interpret_expressions;
|
||||
format_settings.skip_unknown_fields = settings.input_format_skip_unknown_fields;
|
||||
format_settings.import_nested_json = settings.input_format_import_nested_json;
|
||||
format_settings.date_time_input_format = settings.date_time_input_format;
|
||||
format_settings.input_allow_errors_num = settings.input_format_allow_errors_num;
|
||||
format_settings.input_allow_errors_ratio = settings.input_format_allow_errors_ratio;
|
||||
|
@ -49,6 +49,7 @@ struct FormatSettings
|
||||
|
||||
bool skip_unknown_fields = false;
|
||||
bool write_statistics = true;
|
||||
bool import_nested_json = false;
|
||||
|
||||
enum class DateTimeInputFormat
|
||||
{
|
||||
|
@ -3,7 +3,7 @@
|
||||
#include <Formats/JSONEachRowRowInputStream.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <Formats/BlockInputStreamFromRowInputStream.h>
|
||||
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -14,6 +14,17 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_READ_ALL_DATA;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
enum
|
||||
{
|
||||
UNKNOWN_FIELD = size_t(-1),
|
||||
NESTED_FIELD = size_t(-2)
|
||||
};
|
||||
|
||||
} // unnamed namespace
|
||||
|
||||
|
||||
JSONEachRowRowInputStream::JSONEachRowRowInputStream(ReadBuffer & istr_, const Block & header_, const FormatSettings & format_settings)
|
||||
: istr(istr_), header(header_), format_settings(format_settings), name_map(header.columns())
|
||||
@ -23,17 +34,42 @@ JSONEachRowRowInputStream::JSONEachRowRowInputStream(ReadBuffer & istr_, const B
|
||||
|
||||
size_t num_columns = header.columns();
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
name_map[header.safeGetByPosition(i).name] = i; /// NOTE You could place names more cache-locally.
|
||||
{
|
||||
const String& colname = columnName(i);
|
||||
name_map[colname] = i; /// NOTE You could place names more cache-locally.
|
||||
if (format_settings.import_nested_json)
|
||||
{
|
||||
const auto splitted = Nested::splitName(colname);
|
||||
if (!splitted.second.empty())
|
||||
{
|
||||
const StringRef table_name(colname.data(), splitted.first.size());
|
||||
name_map[table_name] = NESTED_FIELD;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/** Read the field name in JSON format.
|
||||
* A reference to the field name will be written to ref.
|
||||
* You can also use temporary `tmp` buffer to copy field name there.
|
||||
*/
|
||||
static StringRef readName(ReadBuffer & buf, String & tmp)
|
||||
const String& JSONEachRowRowInputStream::columnName(size_t i) const
|
||||
{
|
||||
if (buf.position() + 1 < buf.buffer().end())
|
||||
return header.safeGetByPosition(i).name;
|
||||
}
|
||||
|
||||
size_t JSONEachRowRowInputStream::columnIndex(const StringRef& name) const
|
||||
{
|
||||
/// NOTE Optimization is possible by caching the order of fields (which is almost always the same)
|
||||
/// and a quick check to match the next expected field, instead of searching the hash table.
|
||||
|
||||
const auto it = name_map.find(name);
|
||||
return name_map.end() == it ? UNKNOWN_FIELD : it->second;
|
||||
}
|
||||
|
||||
/** Read the field name and convert it to column name
|
||||
* (taking into account the current nested name prefix)
|
||||
*/
|
||||
StringRef JSONEachRowRowInputStream::readColumnName(ReadBuffer & buf)
|
||||
{
|
||||
// This is just an optimization: try to avoid copying the name into current_column_name
|
||||
if (nested_prefix_length == 0 && buf.position() + 1 < buf.buffer().end())
|
||||
{
|
||||
const char * next_pos = find_first_symbols<'\\', '"'>(buf.position() + 1, buf.buffer().end());
|
||||
|
||||
@ -48,8 +84,9 @@ static StringRef readName(ReadBuffer & buf, String & tmp)
|
||||
}
|
||||
}
|
||||
|
||||
readJSONString(tmp, buf);
|
||||
return tmp;
|
||||
current_column_name.resize(nested_prefix_length);
|
||||
readJSONStringInto(current_column_name, buf);
|
||||
return current_column_name;
|
||||
}
|
||||
|
||||
|
||||
@ -60,6 +97,80 @@ static void skipColonDelimeter(ReadBuffer & istr)
|
||||
skipWhitespaceIfAny(istr);
|
||||
}
|
||||
|
||||
void JSONEachRowRowInputStream::skipUnknownField(const StringRef & name_ref)
|
||||
{
|
||||
if (!format_settings.skip_unknown_fields)
|
||||
throw Exception("Unknown field found while parsing JSONEachRow format: " + name_ref.toString(), ErrorCodes::INCORRECT_DATA);
|
||||
|
||||
skipJSONField(istr, name_ref);
|
||||
}
|
||||
|
||||
void JSONEachRowRowInputStream::readField(size_t index, MutableColumns & columns)
|
||||
{
|
||||
if (read_columns[index])
|
||||
throw Exception("Duplicate field found while parsing JSONEachRow format: " + columnName(index), ErrorCodes::INCORRECT_DATA);
|
||||
|
||||
try
|
||||
{
|
||||
header.getByPosition(index).type->deserializeTextJSON(*columns[index], istr, format_settings);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage("(while read the value of key " + columnName(index) + ")");
|
||||
throw;
|
||||
}
|
||||
|
||||
read_columns[index] = true;
|
||||
}
|
||||
|
||||
bool JSONEachRowRowInputStream::advanceToNextKey(size_t key_index)
|
||||
{
|
||||
skipWhitespaceIfAny(istr);
|
||||
|
||||
if (istr.eof())
|
||||
throw Exception("Unexpected end of stream while parsing JSONEachRow format", ErrorCodes::CANNOT_READ_ALL_DATA);
|
||||
else if (*istr.position() == '}')
|
||||
{
|
||||
++istr.position();
|
||||
return false;
|
||||
}
|
||||
|
||||
if (key_index > 0)
|
||||
{
|
||||
assertChar(',', istr);
|
||||
skipWhitespaceIfAny(istr);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
void JSONEachRowRowInputStream::readJSONObject(MutableColumns & columns)
|
||||
{
|
||||
assertChar('{', istr);
|
||||
|
||||
for (size_t key_index = 0; advanceToNextKey(key_index); ++key_index)
|
||||
{
|
||||
StringRef name_ref = readColumnName(istr);
|
||||
|
||||
skipColonDelimeter(istr);
|
||||
|
||||
const size_t column_index = columnIndex(name_ref);
|
||||
if (column_index == UNKNOWN_FIELD)
|
||||
skipUnknownField(name_ref);
|
||||
else if (column_index == NESTED_FIELD)
|
||||
readNestedData(name_ref.toString(), columns);
|
||||
else
|
||||
readField(column_index, columns);
|
||||
}
|
||||
}
|
||||
|
||||
void JSONEachRowRowInputStream::readNestedData(const String & name, MutableColumns & columns)
|
||||
{
|
||||
current_column_name = name;
|
||||
current_column_name.push_back('.');
|
||||
nested_prefix_length = current_column_name.size();
|
||||
readJSONObject(columns);
|
||||
nested_prefix_length = 0;
|
||||
}
|
||||
|
||||
bool JSONEachRowRowInputStream::read(MutableColumns & columns)
|
||||
{
|
||||
@ -78,71 +189,14 @@ bool JSONEachRowRowInputStream::read(MutableColumns & columns)
|
||||
if (istr.eof())
|
||||
return false;
|
||||
|
||||
assertChar('{', istr);
|
||||
|
||||
size_t num_columns = columns.size();
|
||||
|
||||
/// Set of columns for which the values were read. The rest will be filled with default values.
|
||||
/// TODO Ability to provide your DEFAULTs.
|
||||
bool read_columns[num_columns];
|
||||
memset(read_columns, 0, num_columns);
|
||||
read_columns.assign(num_columns, false);
|
||||
|
||||
bool first = true;
|
||||
while (true)
|
||||
{
|
||||
skipWhitespaceIfAny(istr);
|
||||
|
||||
if (istr.eof())
|
||||
throw Exception("Unexpected end of stream while parsing JSONEachRow format", ErrorCodes::CANNOT_READ_ALL_DATA);
|
||||
else if (*istr.position() == '}')
|
||||
{
|
||||
++istr.position();
|
||||
break;
|
||||
}
|
||||
|
||||
if (first)
|
||||
first = false;
|
||||
else
|
||||
{
|
||||
assertChar(',', istr);
|
||||
skipWhitespaceIfAny(istr);
|
||||
}
|
||||
|
||||
StringRef name_ref = readName(istr, name_buf);
|
||||
|
||||
/// NOTE Optimization is possible by caching the order of fields (which is almost always the same)
|
||||
/// and a quick check to match the next expected field, instead of searching the hash table.
|
||||
|
||||
auto it = name_map.find(name_ref);
|
||||
if (name_map.end() == it)
|
||||
{
|
||||
if (!format_settings.skip_unknown_fields)
|
||||
throw Exception("Unknown field found while parsing JSONEachRow format: " + name_ref.toString(), ErrorCodes::INCORRECT_DATA);
|
||||
|
||||
skipColonDelimeter(istr);
|
||||
skipJSONField(istr, name_ref);
|
||||
continue;
|
||||
}
|
||||
|
||||
size_t index = it->second;
|
||||
|
||||
if (read_columns[index])
|
||||
throw Exception("Duplicate field found while parsing JSONEachRow format: " + name_ref.toString(), ErrorCodes::INCORRECT_DATA);
|
||||
|
||||
skipColonDelimeter(istr);
|
||||
|
||||
read_columns[index] = true;
|
||||
|
||||
try
|
||||
{
|
||||
header.getByPosition(index).type->deserializeTextJSON(*columns[index], istr, format_settings);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage("(while read the value of key " + name_ref.toString() + ")");
|
||||
throw;
|
||||
}
|
||||
}
|
||||
nested_prefix_length = 0;
|
||||
readJSONObject(columns);
|
||||
|
||||
/// Fill non-visited columns with the default values.
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
|
@ -26,6 +26,16 @@ public:
|
||||
bool allowSyncAfterError() const override { return true; }
|
||||
void syncAfterError() override;
|
||||
|
||||
private:
|
||||
const String & columnName(size_t i) const;
|
||||
size_t columnIndex(const StringRef & name) const;
|
||||
bool advanceToNextKey(size_t key_index);
|
||||
void skipUnknownField(const StringRef & name_ref);
|
||||
StringRef readColumnName(ReadBuffer & buf);
|
||||
void readField(size_t index, MutableColumns & columns);
|
||||
void readJSONObject(MutableColumns & columns);
|
||||
void readNestedData(const String & name, MutableColumns & columns);
|
||||
|
||||
private:
|
||||
ReadBuffer & istr;
|
||||
Block header;
|
||||
@ -33,7 +43,19 @@ private:
|
||||
const FormatSettings format_settings;
|
||||
|
||||
/// Buffer for the read from the stream field name. Used when you have to copy it.
|
||||
String name_buf;
|
||||
/// Also, if processing of Nested data is in progress, it holds the common prefix
|
||||
/// of the nested column names (so that appending the field name to it produces
|
||||
/// the full column name)
|
||||
String current_column_name;
|
||||
|
||||
/// If processing Nested data, holds the length of the common prefix
|
||||
/// of the names of related nested columns. For example, for a table
|
||||
/// created as follows
|
||||
/// CREATE TABLE t (n Nested (i Int32, s String))
|
||||
/// the nested column names are 'n.i' and 'n.s' and the nested prefix is 'n.'
|
||||
size_t nested_prefix_length = 0;
|
||||
|
||||
std::vector<UInt8> read_columns;
|
||||
|
||||
/// Hash table match `field name -> position in the block`. NOTE You can use perfect hash map.
|
||||
using NameMap = HashMap<StringRef, size_t, StringRefHash>;
|
||||
|
@ -152,7 +152,7 @@ void registerOutputFormatPrettyCompact(FormatFactory & factory)
|
||||
const FormatSettings & format_settings)
|
||||
{
|
||||
BlockOutputStreamPtr impl = std::make_shared<PrettyCompactBlockOutputStream>(buf, sample, format_settings);
|
||||
auto res = std::make_shared<SquashingBlockOutputStream>(impl, format_settings.pretty.max_rows, 0);
|
||||
auto res = std::make_shared<SquashingBlockOutputStream>(impl, impl->getHeader(), format_settings.pretty.max_rows, 0);
|
||||
res->disableFlush();
|
||||
return res;
|
||||
});
|
||||
|
@ -62,11 +62,14 @@ void registerFunctionsDateTime(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionToMonth>();
|
||||
factory.registerFunction<FunctionToDayOfMonth>();
|
||||
factory.registerFunction<FunctionToDayOfWeek>();
|
||||
factory.registerFunction<FunctionToDayOfYear>();
|
||||
factory.registerFunction<FunctionToHour>();
|
||||
factory.registerFunction<FunctionToMinute>();
|
||||
factory.registerFunction<FunctionToSecond>();
|
||||
factory.registerFunction<FunctionToStartOfDay>();
|
||||
factory.registerFunction<FunctionToMonday>();
|
||||
factory.registerFunction<FunctionToISOWeek>();
|
||||
factory.registerFunction<FunctionToISOYear>();
|
||||
factory.registerFunction<FunctionToStartOfMonth>();
|
||||
factory.registerFunction<FunctionToStartOfQuarter>();
|
||||
factory.registerFunction<FunctionToStartOfYear>();
|
||||
@ -74,6 +77,7 @@ void registerFunctionsDateTime(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionToStartOfFiveMinute>();
|
||||
factory.registerFunction<FunctionToStartOfFifteenMinutes>();
|
||||
factory.registerFunction<FunctionToStartOfHour>();
|
||||
factory.registerFunction<FunctionToStartOfISOYear>();
|
||||
factory.registerFunction<FunctionToRelativeYearNum>();
|
||||
factory.registerFunction<FunctionToRelativeQuarterNum>();
|
||||
factory.registerFunction<FunctionToRelativeMonthNum>();
|
||||
@ -111,6 +115,9 @@ void registerFunctionsDateTime(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionDateDiff>(FunctionFactory::CaseInsensitive);
|
||||
|
||||
factory.registerFunction<FunctionToTimeZone>();
|
||||
|
||||
|
||||
factory.registerFunction<FunctionFormatDateTime>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -20,6 +20,7 @@
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
|
||||
#include <common/DateLUT.h>
|
||||
#include <common/find_first_symbols.h>
|
||||
|
||||
#include <Poco/String.h>
|
||||
|
||||
@ -32,6 +33,9 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
/** Functions for working with date and time.
|
||||
@ -351,6 +355,22 @@ struct ToDayOfWeekImpl
|
||||
using FactorTransform = ToMondayImpl;
|
||||
};
|
||||
|
||||
struct ToDayOfYearImpl
|
||||
{
|
||||
static constexpr auto name = "toDayOfYear";
|
||||
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDayOfYear(t);
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDayOfYear(DayNum(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ToStartOfYearImpl;
|
||||
};
|
||||
|
||||
struct ToHourImpl
|
||||
{
|
||||
static constexpr auto name = "toHour";
|
||||
@ -400,6 +420,54 @@ struct ToSecondImpl
|
||||
using FactorTransform = ToStartOfMinuteImpl;
|
||||
};
|
||||
|
||||
struct ToISOYearImpl
|
||||
{
|
||||
static constexpr auto name = "toISOYear";
|
||||
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toISOYear(time_zone.toDayNum(t));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toISOYear(DayNum(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
|
||||
struct ToStartOfISOYearImpl
|
||||
{
|
||||
static constexpr auto name = "toStartOfISOYear";
|
||||
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfISOYear(time_zone.toDayNum(t));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfISOYear(DayNum(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
|
||||
struct ToISOWeekImpl
|
||||
{
|
||||
static constexpr auto name = "toISOWeek";
|
||||
|
||||
static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toISOWeek(time_zone.toDayNum(t));
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toISOWeek(DayNum(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ToISOYearImpl;
|
||||
};
|
||||
|
||||
struct ToRelativeYearNumImpl
|
||||
{
|
||||
static constexpr auto name = "toRelativeYearNum";
|
||||
@ -1507,16 +1575,555 @@ public:
|
||||
};
|
||||
|
||||
|
||||
/** formatDateTime(time, 'pattern')
|
||||
* Performs formatting of time, according to provided pattern.
|
||||
*
|
||||
* This function is optimized with an assumption, that the resulting strings are fixed width.
|
||||
* (This assumption is fulfilled for currently supported formatting options).
|
||||
*
|
||||
* It is implemented in two steps.
|
||||
* At first step, it creates a pattern of zeros, literal characters, whitespaces, etc.
|
||||
* and quickly fills resulting charater array (string column) with this pattern.
|
||||
* At second step, it walks across the resulting character array and modifies/replaces specific charaters,
|
||||
* by calling some functions by pointers and shifting cursor by specified amount.
|
||||
*
|
||||
* Advantages:
|
||||
* - memcpy is mostly unrolled;
|
||||
* - low number of arithmetic ops due to pre-filled pattern;
|
||||
* - for somewhat reason, function by pointer call is faster than switch/case.
|
||||
*
|
||||
* Possible further optimization options:
|
||||
* - slightly interleave first and second step for better cache locality
|
||||
* (but it has no sense when character array fits in L1d cache);
|
||||
* - avoid indirect function calls and inline functions with JIT compilation.
|
||||
*
|
||||
* Performance on Intel(R) Core(TM) i7-6700 CPU @ 3.40GHz:
|
||||
*
|
||||
* WITH formatDateTime(now() + number, '%H:%M:%S') AS x SELECT count() FROM system.numbers WHERE NOT ignore(x);
|
||||
* - 97 million rows per second per core;
|
||||
*
|
||||
* WITH formatDateTime(toDateTime('2018-01-01 00:00:00') + number, '%F %T') AS x SELECT count() FROM system.numbers WHERE NOT ignore(x)
|
||||
* - 71 million rows per second per core;
|
||||
*
|
||||
* select count() from (select formatDateTime(t, '%m/%d/%Y %H:%M:%S') from (select toDateTime('2018-01-01 00:00:00')+number as t from numbers(100000000)));
|
||||
* - 53 million rows per second per core;
|
||||
*
|
||||
* select count() from (select formatDateTime(t, 'Hello %Y World') from (select toDateTime('2018-01-01 00:00:00')+number as t from numbers(100000000)));
|
||||
* - 138 million rows per second per core;
|
||||
*
|
||||
* PS. We can make this function to return FixedString. Currently it returns String.
|
||||
*/
|
||||
class FunctionFormatDateTime : public IFunction
|
||||
{
|
||||
private:
|
||||
/// Time is either UInt32 for DateTime or UInt16 for Date.
|
||||
template <typename Time>
|
||||
class Action
|
||||
{
|
||||
public:
|
||||
using Func = void (*)(char *, Time, const DateLUTImpl &);
|
||||
|
||||
Func func;
|
||||
size_t shift;
|
||||
|
||||
Action(Func func, size_t shift = 0) : func(func), shift(shift) {}
|
||||
|
||||
void perform(char *& target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
func(target, source, timezone);
|
||||
target += shift;
|
||||
}
|
||||
|
||||
private:
|
||||
template <typename T>
|
||||
static inline void writeNumber2(char * p, T v)
|
||||
{
|
||||
static const char digits[201] =
|
||||
"00010203040506070809"
|
||||
"10111213141516171819"
|
||||
"20212223242526272829"
|
||||
"30313233343536373839"
|
||||
"40414243444546474849"
|
||||
"50515253545556575859"
|
||||
"60616263646566676869"
|
||||
"70717273747576777879"
|
||||
"80818283848586878889"
|
||||
"90919293949596979899";
|
||||
|
||||
memcpy(p, &digits[v * 2], 2);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
static inline void writeNumber3(char * p, T v)
|
||||
{
|
||||
writeNumber2(p, v / 10);
|
||||
p[2] += v % 10;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
static inline void writeNumber4(char * p, T v)
|
||||
{
|
||||
writeNumber2(p, v / 100);
|
||||
writeNumber2(p + 2, v % 100);
|
||||
}
|
||||
|
||||
public:
|
||||
static void noop(char *, Time, const DateLUTImpl &)
|
||||
{
|
||||
}
|
||||
|
||||
static void century(char * target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
auto year = ToYearImpl::execute(source, timezone);
|
||||
auto century = year / 100;
|
||||
writeNumber2(target, century);
|
||||
}
|
||||
|
||||
static void dayOfMonth(char * target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeNumber2(target, ToDayOfMonthImpl::execute(source, timezone));
|
||||
}
|
||||
|
||||
static void americanDate(char * target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeNumber2(target, ToMonthImpl::execute(source, timezone));
|
||||
writeNumber2(target + 3, ToDayOfMonthImpl::execute(source, timezone));
|
||||
writeNumber2(target + 6, ToYearImpl::execute(source, timezone) % 100);
|
||||
}
|
||||
|
||||
static void dayOfMonthSpacePadded(char * target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
auto day = ToDayOfMonthImpl::execute(source, timezone);
|
||||
if (day < 10)
|
||||
target[1] += day;
|
||||
else
|
||||
writeNumber2(target, day);
|
||||
}
|
||||
|
||||
static void ISO8601Date(char * target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeNumber4(target, ToYearImpl::execute(source, timezone));
|
||||
writeNumber2(target + 5, ToMonthImpl::execute(source, timezone));
|
||||
writeNumber2(target + 8, ToDayOfMonthImpl::execute(source, timezone));
|
||||
}
|
||||
|
||||
static void dayOfYear(char * target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeNumber3(target, ToDayOfYearImpl::execute(source, timezone));
|
||||
}
|
||||
|
||||
static void month(char * target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeNumber2(target, ToMonthImpl::execute(source, timezone));
|
||||
}
|
||||
|
||||
static void dayOfWeek(char * target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
*target += ToDayOfWeekImpl::execute(source, timezone);
|
||||
}
|
||||
|
||||
static void dayOfWeek0To6(char * target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
auto day = ToDayOfWeekImpl::execute(source, timezone);
|
||||
*target += (day == 7 ? 0 : day);
|
||||
}
|
||||
|
||||
static void ISO8601Week(char * target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeNumber2(target, ToISOWeekImpl::execute(source, timezone));
|
||||
}
|
||||
|
||||
static void year2(char * target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeNumber2(target, ToYearImpl::execute(source, timezone) % 100);
|
||||
}
|
||||
|
||||
static void year4(char * target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeNumber4(target, ToYearImpl::execute(source, timezone));
|
||||
}
|
||||
|
||||
static void hour24(char * target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeNumber2(target, ToHourImpl::execute(source, timezone));
|
||||
}
|
||||
|
||||
static void hour12(char * target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
auto x = ToHourImpl::execute(source, timezone);
|
||||
writeNumber2(target, x == 0 ? 12 : (x > 12 ? x - 12 : x));
|
||||
}
|
||||
|
||||
static void minute(char * target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeNumber2(target, ToMinuteImpl::execute(source, timezone));
|
||||
}
|
||||
|
||||
static void AMPM(char * target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
auto hour = ToHourImpl::execute(source, timezone);
|
||||
if (hour >= 12)
|
||||
*target = 'P';
|
||||
}
|
||||
|
||||
static void hhmm24(char * target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeNumber2(target, ToHourImpl::execute(source, timezone));
|
||||
writeNumber2(target + 3, ToMinuteImpl::execute(source, timezone));
|
||||
}
|
||||
|
||||
static void second(char * target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeNumber2(target, ToSecondImpl::execute(source, timezone));
|
||||
}
|
||||
|
||||
static void ISO8601Time(char * target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeNumber2(target, ToHourImpl::execute(source, timezone));
|
||||
writeNumber2(target + 3, ToMinuteImpl::execute(source, timezone));
|
||||
writeNumber2(target + 6, ToSecondImpl::execute(source, timezone));
|
||||
}
|
||||
};
|
||||
|
||||
public:
|
||||
static constexpr auto name = "formatDateTime";
|
||||
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionFormatDateTime>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; }
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
if (arguments.size() != 2 && arguments.size() != 3)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 2 or 3",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (!WhichDataType(arguments[0].type).isDateOrDateTime())
|
||||
throw Exception("Illegal type " + arguments[0].type->getName() + " of 1 argument of function " + getName() +
|
||||
". Should be a date or a date with time", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!WhichDataType(arguments[1].type).isString())
|
||||
throw Exception("Illegal type " + arguments[1].type->getName() + " of 2 argument of function " + getName() + ". Must be String.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (arguments.size() == 3)
|
||||
{
|
||||
if (!WhichDataType(arguments[2].type).isString())
|
||||
throw Exception("Illegal type " + arguments[2].type->getName() + " of 3 argument of function " + getName() + ". Must be String.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||
{
|
||||
if (!executeType<UInt32>(block, arguments, result)
|
||||
&& !executeType<UInt16>(block, arguments, result))
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of function " + getName() + ", must be Date or DateTime",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool executeType(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (auto * times = checkAndGetColumn<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
const ColumnConst * pattern_column = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
|
||||
|
||||
if (!pattern_column)
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||
+ " of second ('format') argument of function " + getName()
|
||||
+ ". Must be constant string.",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
String pattern = pattern_column->getValue<String>();
|
||||
|
||||
std::vector<Action<T>> instructions;
|
||||
String pattern_to_fill = parsePattern(pattern, instructions);
|
||||
size_t result_size = pattern_to_fill.size();
|
||||
|
||||
const DateLUTImpl * time_zone_tmp = nullptr;
|
||||
if (arguments.size() == 3)
|
||||
time_zone_tmp = &extractTimeZoneFromFunctionArguments(block, arguments, 2, 0);
|
||||
else
|
||||
time_zone_tmp = &DateLUT::instance();
|
||||
|
||||
const DateLUTImpl & time_zone = *time_zone_tmp;
|
||||
|
||||
const typename ColumnVector<T>::Container & vec = times->getData();
|
||||
|
||||
auto col_res = ColumnString::create();
|
||||
auto & dst_data = col_res->getChars();
|
||||
auto & dst_offsets = col_res->getOffsets();
|
||||
dst_data.resize(vec.size() * (result_size + 1));
|
||||
dst_offsets.resize(vec.size());
|
||||
|
||||
/// Fill result with literals.
|
||||
{
|
||||
UInt8 * begin = dst_data.data();
|
||||
UInt8 * end = begin + dst_data.size();
|
||||
UInt8 * pos = begin;
|
||||
|
||||
if (pos < end)
|
||||
{
|
||||
memcpy(pos, pattern_to_fill.data(), result_size + 1); /// With zero terminator.
|
||||
pos += result_size + 1;
|
||||
}
|
||||
|
||||
/// Fill by copying exponential growing ranges.
|
||||
while (pos < end)
|
||||
{
|
||||
size_t bytes_to_copy = std::min(pos - begin, end - pos);
|
||||
memcpy(pos, begin, bytes_to_copy);
|
||||
pos += bytes_to_copy;
|
||||
}
|
||||
}
|
||||
|
||||
auto begin = reinterpret_cast<char *>(dst_data.data());
|
||||
auto pos = begin;
|
||||
|
||||
for (size_t i = 0; i < vec.size(); ++i)
|
||||
{
|
||||
for(auto & instruction : instructions)
|
||||
instruction.perform(pos, vec[i], time_zone);
|
||||
|
||||
dst_offsets[i] = pos - begin;
|
||||
}
|
||||
|
||||
dst_data.resize(pos - begin);
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
String parsePattern(const String & pattern, std::vector<Action<T>> & instructions) const
|
||||
{
|
||||
String result;
|
||||
|
||||
const char * pos = pattern.data();
|
||||
const char * end = pos + pattern.size();
|
||||
|
||||
/// Add shift to previous action; or if there were none, add noop action with shift.
|
||||
auto addShift = [&](size_t amount)
|
||||
{
|
||||
if (instructions.empty())
|
||||
instructions.emplace_back(&Action<T>::noop);
|
||||
instructions.back().shift += amount;
|
||||
};
|
||||
|
||||
/// If the argument was DateTime, add instruction for printing. If it was date, just shift (the buffer is pre-filled with default values).
|
||||
auto addInstructionOrShift = [&](typename Action<T>::Func func [[maybe_unused]], size_t shift)
|
||||
{
|
||||
if constexpr (std::is_same_v<T, UInt32>)
|
||||
instructions.emplace_back(func, shift);
|
||||
else
|
||||
addShift(shift);
|
||||
};
|
||||
|
||||
while (true)
|
||||
{
|
||||
const char * percent_pos = find_first_symbols<'%'>(pos, end);
|
||||
|
||||
if (percent_pos < end)
|
||||
{
|
||||
if (pos < percent_pos)
|
||||
{
|
||||
result.append(pos, percent_pos);
|
||||
addShift(percent_pos - pos);
|
||||
}
|
||||
|
||||
pos = percent_pos + 1;
|
||||
|
||||
if (pos >= end)
|
||||
throw Exception("Sign '%' is the last in pattern, if you need it, use '%%'", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
switch (*pos)
|
||||
{
|
||||
// Year, divided by 100, zero-padded
|
||||
case 'C':
|
||||
instructions.emplace_back(&Action<T>::century, 2);
|
||||
result.append("00");
|
||||
break;
|
||||
|
||||
// Day of month, zero-padded (01-31)
|
||||
case 'd':
|
||||
instructions.emplace_back(&Action<T>::dayOfMonth, 2);
|
||||
result.append("00");
|
||||
break;
|
||||
|
||||
// Short MM/DD/YY date, equivalent to %m/%d/%y
|
||||
case 'D':
|
||||
instructions.emplace_back(&Action<T>::americanDate, 8);
|
||||
result.append("00/00/00");
|
||||
break;
|
||||
|
||||
// Day of month, space-padded ( 1-31) 23
|
||||
case 'e':
|
||||
instructions.emplace_back(&Action<T>::dayOfMonthSpacePadded, 2);
|
||||
result.append(" 0");
|
||||
break;
|
||||
|
||||
// Short YYYY-MM-DD date, equivalent to %Y-%m-%d 2001-08-23
|
||||
case 'F':
|
||||
instructions.emplace_back(&Action<T>::ISO8601Date, 10);
|
||||
result.append("0000-00-00");
|
||||
break;
|
||||
|
||||
// Day of the year (001-366) 235
|
||||
case 'j':
|
||||
instructions.emplace_back(&Action<T>::dayOfYear, 3);
|
||||
result.append("000");
|
||||
break;
|
||||
|
||||
// Month as a decimal number (01-12)
|
||||
case 'm':
|
||||
instructions.emplace_back(&Action<T>::month, 2);
|
||||
result.append("00");
|
||||
break;
|
||||
|
||||
// ISO 8601 weekday as number with Monday as 1 (1-7)
|
||||
case 'u':
|
||||
instructions.emplace_back(&Action<T>::dayOfWeek, 1);
|
||||
result.append("0");
|
||||
break;
|
||||
|
||||
// ISO 8601 week number (01-53)
|
||||
case 'V':
|
||||
instructions.emplace_back(&Action<T>::ISO8601Week, 2);
|
||||
result.append("00");
|
||||
break;
|
||||
|
||||
// Weekday as a decimal number with Sunday as 0 (0-6) 4
|
||||
case 'w':
|
||||
instructions.emplace_back(&Action<T>::dayOfWeek0To6, 1);
|
||||
result.append("0");
|
||||
break;
|
||||
|
||||
// Two digits year
|
||||
case 'y':
|
||||
instructions.emplace_back(&Action<T>::year2, 2);
|
||||
result.append("00");
|
||||
break;
|
||||
|
||||
// Four digits year
|
||||
case 'Y':
|
||||
instructions.emplace_back(&Action<T>::year4, 4);
|
||||
result.append("0000");
|
||||
break;
|
||||
|
||||
/// Time components. If the argument is Date, not a DateTime, then this components will have default value.
|
||||
|
||||
// Minute (00-59)
|
||||
case 'M':
|
||||
addInstructionOrShift(&Action<T>::minute, 2);
|
||||
result.append("00");
|
||||
break;
|
||||
|
||||
// AM or PM
|
||||
case 'p':
|
||||
addInstructionOrShift(&Action<T>::AMPM, 2);
|
||||
result.append("AM");
|
||||
break;
|
||||
|
||||
// 24-hour HH:MM time, equivalent to %H:%M 14:55
|
||||
case 'R':
|
||||
addInstructionOrShift(&Action<T>::hhmm24, 5);
|
||||
result.append("00:00");
|
||||
break;
|
||||
|
||||
// Seconds
|
||||
case 'S':
|
||||
addInstructionOrShift(&Action<T>::second, 2);
|
||||
result.append("00");
|
||||
break;
|
||||
|
||||
// ISO 8601 time format (HH:MM:SS), equivalent to %H:%M:%S 14:55:02
|
||||
case 'T':
|
||||
addInstructionOrShift(&Action<T>::ISO8601Time, 8);
|
||||
result.append("00:00:00");
|
||||
break;
|
||||
|
||||
// Hour in 24h format (00-23)
|
||||
case 'H':
|
||||
addInstructionOrShift(&Action<T>::hour24, 2);
|
||||
result.append("00");
|
||||
break;
|
||||
|
||||
// Hour in 12h format (01-12)
|
||||
case 'I':
|
||||
addInstructionOrShift(&Action<T>::hour12, 2);
|
||||
result.append("12");
|
||||
break;
|
||||
|
||||
/// Escaped literal characters.
|
||||
case '%':
|
||||
result += '%';
|
||||
addShift(1);
|
||||
break;
|
||||
case 't':
|
||||
result += '\t';
|
||||
addShift(1);
|
||||
break;
|
||||
case 'n':
|
||||
result += '\n';
|
||||
addShift(1);
|
||||
break;
|
||||
|
||||
// Unimplemented
|
||||
case 'U': [[fallthrough]];
|
||||
case 'W':
|
||||
throw Exception("Wrong pattern '" + pattern + "', symbol '" + *pos + " is not implemented ' for function " + getName(),
|
||||
ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
||||
default:
|
||||
throw Exception(
|
||||
"Wrong pattern '" + pattern + "', unexpected symbol '" + *pos + "' for function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
++pos;
|
||||
}
|
||||
else
|
||||
{
|
||||
result.append(pos, end);
|
||||
addShift(end + 1 - pos); /// including zero terminator
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
using FunctionToYear = FunctionDateOrDateTimeToSomething<DataTypeUInt16, ToYearImpl>;
|
||||
using FunctionToQuarter = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToQuarterImpl>;
|
||||
using FunctionToMonth = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToMonthImpl>;
|
||||
using FunctionToDayOfMonth = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToDayOfMonthImpl>;
|
||||
using FunctionToDayOfWeek = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToDayOfWeekImpl>;
|
||||
using FunctionToDayOfYear = FunctionDateOrDateTimeToSomething<DataTypeUInt16, ToDayOfYearImpl>;
|
||||
using FunctionToHour = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToHourImpl>;
|
||||
using FunctionToMinute = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToMinuteImpl>;
|
||||
using FunctionToSecond = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToSecondImpl>;
|
||||
using FunctionToStartOfDay = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfDayImpl>;
|
||||
using FunctionToMonday = FunctionDateOrDateTimeToSomething<DataTypeDate, ToMondayImpl>;
|
||||
using FunctionToISOWeek = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToISOWeekImpl>;
|
||||
using FunctionToISOYear = FunctionDateOrDateTimeToSomething<DataTypeUInt16, ToISOYearImpl>;
|
||||
using FunctionToStartOfMonth = FunctionDateOrDateTimeToSomething<DataTypeDate, ToStartOfMonthImpl>;
|
||||
using FunctionToStartOfQuarter = FunctionDateOrDateTimeToSomething<DataTypeDate, ToStartOfQuarterImpl>;
|
||||
using FunctionToStartOfYear = FunctionDateOrDateTimeToSomething<DataTypeDate, ToStartOfYearImpl>;
|
||||
@ -1524,6 +2131,7 @@ using FunctionToStartOfMinute = FunctionDateOrDateTimeToSomething<DataTypeDateTi
|
||||
using FunctionToStartOfFiveMinute = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfFiveMinuteImpl>;
|
||||
using FunctionToStartOfFifteenMinutes = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfFifteenMinutesImpl>;
|
||||
using FunctionToStartOfHour = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfHourImpl>;
|
||||
using FunctionToStartOfISOYear = FunctionDateOrDateTimeToSomething<DataTypeDate, ToStartOfISOYearImpl>;
|
||||
using FunctionToTime = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToTimeImpl>;
|
||||
|
||||
using FunctionToRelativeYearNum = FunctionDateOrDateTimeToSomething<DataTypeUInt16, ToRelativeYearNumImpl>;
|
||||
|
@ -203,7 +203,7 @@ static bool isDictGetFunctionInjective(const ExternalDictionaries & dictionaries
|
||||
* or convert it to T type, stores the result in backup_storage and returns a reference to it.
|
||||
*/
|
||||
template <typename T>
|
||||
const PaddedPODArray<T> & getColumnDataAsPaddedPODArray(const IColumn & column, PaddedPODArray<T> & backup_storage);
|
||||
static const PaddedPODArray<T> & getColumnDataAsPaddedPODArray(const IColumn & column, PaddedPODArray<T> & backup_storage);
|
||||
|
||||
|
||||
class FunctionDictGetString final : public IFunction
|
||||
@ -1459,7 +1459,7 @@ private:
|
||||
|
||||
|
||||
template <typename T>
|
||||
const PaddedPODArray<T> & getColumnDataAsPaddedPODArray(const IColumn & column, PaddedPODArray<T> & backup_storage)
|
||||
static const PaddedPODArray<T> & getColumnDataAsPaddedPODArray(const IColumn & column, PaddedPODArray<T> & backup_storage)
|
||||
{
|
||||
if (const auto vector_col = checkAndGetColumn<ColumnVector<T>>(&column))
|
||||
{
|
||||
|
@ -55,13 +55,15 @@ struct ArrayCumSumNonNegativeImpl
|
||||
// skip empty arrays
|
||||
if (pos < offsets[i])
|
||||
{
|
||||
accum_sum = data[pos];
|
||||
res_values[pos] = accum_sum > 0 ? accum_sum : 0;
|
||||
accum_sum = data[pos] > 0 ? data[pos] : 0;
|
||||
res_values[pos] = accum_sum;
|
||||
for (++pos; pos < offsets[i]; ++pos)
|
||||
{
|
||||
accum_sum = accum_sum + data[pos];
|
||||
if (accum_sum < 0)
|
||||
accum_sum = 0;
|
||||
|
||||
res_values[pos] = accum_sum > 0 ? accum_sum : 0;
|
||||
res_values[pos] = accum_sum;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -667,6 +667,7 @@ void readJSONString(String & s, ReadBuffer & buf)
|
||||
template void readJSONStringInto<PaddedPODArray<UInt8>, void>(PaddedPODArray<UInt8> & s, ReadBuffer & buf);
|
||||
template bool readJSONStringInto<PaddedPODArray<UInt8>, bool>(PaddedPODArray<UInt8> & s, ReadBuffer & buf);
|
||||
template void readJSONStringInto<NullSink>(NullSink & s, ReadBuffer & buf);
|
||||
template void readJSONStringInto<String>(String & s, ReadBuffer & buf);
|
||||
|
||||
|
||||
template <typename ReturnType>
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Core/Defines.h>
|
||||
#include <common/shift10.h>
|
||||
#include <common/likely.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <double-conversion/double-conversion.h>
|
||||
|
||||
|
||||
@ -274,7 +275,7 @@ static inline void readUIntTextUpToNSignificantDigits(T & x, ReadBuffer & buf)
|
||||
{
|
||||
for (size_t i = 0; i < N; ++i)
|
||||
{
|
||||
if ((*buf.position() & 0xF0) == 0x30)
|
||||
if (isNumericASCII(*buf.position()))
|
||||
{
|
||||
x *= 10;
|
||||
x += *buf.position() & 0x0F;
|
||||
@ -284,14 +285,14 @@ static inline void readUIntTextUpToNSignificantDigits(T & x, ReadBuffer & buf)
|
||||
return;
|
||||
}
|
||||
|
||||
while (!buf.eof() && (*buf.position() & 0xF0) == 0x30)
|
||||
while (!buf.eof() && isNumericASCII(*buf.position()))
|
||||
++buf.position();
|
||||
}
|
||||
else
|
||||
{
|
||||
for (size_t i = 0; i < N; ++i)
|
||||
{
|
||||
if (!buf.eof() && (*buf.position() & 0xF0) == 0x30)
|
||||
if (!buf.eof() && isNumericASCII(*buf.position()))
|
||||
{
|
||||
x *= 10;
|
||||
x += *buf.position() & 0x0F;
|
||||
@ -301,7 +302,7 @@ static inline void readUIntTextUpToNSignificantDigits(T & x, ReadBuffer & buf)
|
||||
return;
|
||||
}
|
||||
|
||||
while (!buf.eof() && (*buf.position() & 0xF0) == 0x30)
|
||||
while (!buf.eof() && isNumericASCII(*buf.position()))
|
||||
++buf.position();
|
||||
}
|
||||
}
|
||||
@ -554,12 +555,15 @@ ReturnType readFloatTextSimpleImpl(T & x, ReadBuffer & buf)
|
||||
|
||||
|
||||
template <typename T>
|
||||
inline void readDecimalText(ReadBuffer & buf, T & x, unsigned int precision, unsigned int & scale, bool digits_only = false)
|
||||
inline void readDigits(ReadBuffer & buf, T & x, unsigned int & digits, int & exponent, bool digits_only = false)
|
||||
{
|
||||
x = 0;
|
||||
exponent = 0;
|
||||
unsigned int max_digits = digits;
|
||||
digits = 0;
|
||||
unsigned int places = 0;
|
||||
typename T::NativeType sign = 1;
|
||||
bool leading_zeores = true;
|
||||
bool trailing_zeores = false;
|
||||
bool leading_zeroes = true;
|
||||
bool after_point = false;
|
||||
|
||||
if (buf.eof())
|
||||
@ -578,16 +582,28 @@ inline void readDecimalText(ReadBuffer & buf, T & x, unsigned int precision, uns
|
||||
}
|
||||
}
|
||||
|
||||
while (!buf.eof())
|
||||
bool stop = false;
|
||||
while (!buf.eof() && !stop)
|
||||
{
|
||||
const char & byte = *buf.position();
|
||||
switch (byte)
|
||||
{
|
||||
case '.':
|
||||
after_point = true;
|
||||
if (scale == 0)
|
||||
trailing_zeores = true;
|
||||
leading_zeroes = false;
|
||||
break;
|
||||
case '0':
|
||||
{
|
||||
if (leading_zeroes)
|
||||
break;
|
||||
|
||||
if (after_point)
|
||||
{
|
||||
++places; /// Count trailing zeroes. They would be used only if there's some other digit after them.
|
||||
break;
|
||||
}
|
||||
[[fallthrough]];
|
||||
}
|
||||
case '1': [[fallthrough]];
|
||||
case '2': [[fallthrough]];
|
||||
case '3': [[fallthrough]];
|
||||
@ -597,40 +613,61 @@ inline void readDecimalText(ReadBuffer & buf, T & x, unsigned int precision, uns
|
||||
case '7': [[fallthrough]];
|
||||
case '8': [[fallthrough]];
|
||||
case '9':
|
||||
leading_zeores = false;
|
||||
if (trailing_zeores || precision == 0)
|
||||
throw Exception("Cannot read decimal value", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
[[fallthrough]];
|
||||
case '0':
|
||||
{
|
||||
/// ignore leading and trailing zeroes
|
||||
if (likely(!leading_zeores && !trailing_zeores))
|
||||
{
|
||||
if (precision == 0 || precision < scale || ((precision == scale) && !after_point))
|
||||
throw Exception("Cannot read decimal value", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
--precision;
|
||||
x = x * 10 + (byte - '0');
|
||||
}
|
||||
if (after_point && scale)
|
||||
{
|
||||
--scale;
|
||||
if (!scale)
|
||||
trailing_zeores = true;
|
||||
}
|
||||
leading_zeroes = false;
|
||||
|
||||
++places; // num zeroes before + current digit
|
||||
if (digits + places > max_digits)
|
||||
throw Exception("Too many digits in decimal value", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
|
||||
digits += places;
|
||||
if (after_point)
|
||||
exponent -= places;
|
||||
|
||||
// TODO: accurate shift10 for big integers
|
||||
for (; places; --places)
|
||||
x *= 10;
|
||||
x += (byte - '0');
|
||||
break;
|
||||
}
|
||||
case 'e': [[fallthrough]];
|
||||
case 'E':
|
||||
{
|
||||
++buf.position();
|
||||
Int32 addition_exp = 0;
|
||||
readIntText(addition_exp, buf);
|
||||
exponent += addition_exp;
|
||||
stop = true;
|
||||
continue;
|
||||
}
|
||||
|
||||
default:
|
||||
if (digits_only)
|
||||
throw Exception("Unexpected symbol while reading decimal", ErrorCodes::CANNOT_PARSE_NUMBER);
|
||||
x *= sign;
|
||||
return;
|
||||
stop = true;
|
||||
continue;
|
||||
}
|
||||
++buf.position();
|
||||
}
|
||||
|
||||
x *= sign;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
inline void readDecimalText(ReadBuffer & buf, T & x, unsigned int precision, unsigned int & scale, bool digits_only = false)
|
||||
{
|
||||
unsigned int digits = precision;
|
||||
int exponent;
|
||||
readDigits(buf, x, digits, exponent, digits_only);
|
||||
|
||||
if (static_cast<int>(digits) + exponent > static_cast<int>(precision - scale))
|
||||
throw Exception("Decimal value is too big", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
if (static_cast<int>(scale) + exponent < 0)
|
||||
throw Exception("Decimal value is too small", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
|
||||
scale += exponent;
|
||||
}
|
||||
|
||||
|
||||
template <typename T> void readFloatTextPrecise(T & x, ReadBuffer & in) { readFloatTextPreciseImpl<T, void>(x, in); }
|
||||
template <typename T> bool tryReadFloatTextPrecise(T & x, ReadBuffer & in) { return readFloatTextPreciseImpl<T, bool>(x, in); }
|
||||
|
@ -32,7 +32,7 @@ $ clickhouse-local --query="SELECT toString(rand64(1)) || toString(rand64(2)) ||
|
||||
|
||||
# Run test
|
||||
|
||||
$ for i in {1..10}; do echo $i; time ./read_float_perf < numbers$i.tsv; done
|
||||
$ for i in {1..10}; do echo $i; time ./read_float_perf 2 < numbers$i.tsv; done
|
||||
|
||||
*/
|
||||
|
||||
|
@ -205,6 +205,7 @@ static bool isSupportedAlterType(int type)
|
||||
ASTAlterCommand::MODIFY_PRIMARY_KEY,
|
||||
ASTAlterCommand::DROP_PARTITION,
|
||||
ASTAlterCommand::DELETE,
|
||||
ASTAlterCommand::UPDATE,
|
||||
};
|
||||
|
||||
return supported_alter_types.count(type) != 0;
|
||||
|
@ -169,6 +169,9 @@ public:
|
||||
input_columns.emplace_back(input_elem.name, input_elem.type);
|
||||
sample_block.insert(input_elem);
|
||||
}
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
compilation_cache = context_.getCompiledExpressionCache();
|
||||
#endif
|
||||
}
|
||||
|
||||
/// Add the input column.
|
||||
|
@ -308,7 +308,7 @@ void ExpressionAnalyzer::translateQualifiedNamesImpl(ASTPtr & ast, const std::ve
|
||||
{
|
||||
if (auto * identifier = typeid_cast<ASTIdentifier *>(ast.get()))
|
||||
{
|
||||
if (identifier->kind == ASTIdentifier::Column)
|
||||
if (identifier->general())
|
||||
{
|
||||
/// Select first table name with max number of qualifiers which can be stripped.
|
||||
size_t max_num_qualifiers_to_strip = 0;
|
||||
@ -646,7 +646,7 @@ void ExpressionAnalyzer::findExternalTables(ASTPtr & ast)
|
||||
StoragePtr external_storage;
|
||||
|
||||
if (ASTIdentifier * node = typeid_cast<ASTIdentifier *>(ast.get()))
|
||||
if (node->kind == ASTIdentifier::Table)
|
||||
if (node->special())
|
||||
if ((external_storage = context.tryGetExternalTable(node->name)))
|
||||
external_tables[node->name] = external_storage;
|
||||
}
|
||||
@ -830,7 +830,7 @@ void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name_or_t
|
||||
* instead of doing a subquery, you just need to read it.
|
||||
*/
|
||||
|
||||
auto database_and_table_name = std::make_shared<ASTIdentifier>(external_table_name, ASTIdentifier::Table);
|
||||
auto database_and_table_name = ASTIdentifier::createSpecial(external_table_name);
|
||||
|
||||
if (auto ast_table_expr = typeid_cast<ASTTableExpression *>(subquery_or_table_name_or_table_expression.get()))
|
||||
{
|
||||
@ -1659,7 +1659,7 @@ void ExpressionAnalyzer::getArrayJoinedColumnsImpl(const ASTPtr & ast)
|
||||
|
||||
if (ASTIdentifier * node = typeid_cast<ASTIdentifier *>(ast.get()))
|
||||
{
|
||||
if (node->kind == ASTIdentifier::Column)
|
||||
if (node->general())
|
||||
{
|
||||
auto splitted = Nested::splitName(node->name); /// ParsedParams, Key1
|
||||
|
||||
@ -2884,7 +2884,7 @@ void ExpressionAnalyzer::collectJoinedColumnsFromJoinOnExpr()
|
||||
auto * identifier = typeid_cast<const ASTIdentifier *>(ast.get());
|
||||
if (identifier)
|
||||
{
|
||||
if (identifier->kind == ASTIdentifier::Column)
|
||||
if (identifier->general())
|
||||
{
|
||||
auto left_num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, left_source_names);
|
||||
auto right_num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, right_source_names);
|
||||
@ -2923,7 +2923,7 @@ void ExpressionAnalyzer::collectJoinedColumnsFromJoinOnExpr()
|
||||
auto * identifier = typeid_cast<const ASTIdentifier *>(ast.get());
|
||||
if (identifier)
|
||||
{
|
||||
if (identifier->kind == ASTIdentifier::Column)
|
||||
if (identifier->general())
|
||||
{
|
||||
auto num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, source_names);
|
||||
stripIdentifier(ast, num_components);
|
||||
@ -3091,7 +3091,7 @@ void ExpressionAnalyzer::getRequiredSourceColumnsImpl(const ASTPtr & ast,
|
||||
|
||||
if (ASTIdentifier * node = typeid_cast<ASTIdentifier *>(ast.get()))
|
||||
{
|
||||
if (node->kind == ASTIdentifier::Column
|
||||
if (node->general()
|
||||
&& !ignored_names.count(node->name)
|
||||
&& !ignored_names.count(Nested::extractTableName(node->name)))
|
||||
{
|
||||
|
@ -60,6 +60,7 @@ namespace ProfileEvents
|
||||
{
|
||||
extern const Event CompileFunction;
|
||||
extern const Event CompileExpressionsMicroseconds;
|
||||
extern const Event CompileExpressionsBytes;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
@ -252,10 +253,11 @@ struct LLVMContext
|
||||
module->setTargetTriple(machine->getTargetTriple().getTriple());
|
||||
}
|
||||
|
||||
void finalize()
|
||||
/// returns used memory
|
||||
size_t compileAllFunctionsToNativeCode()
|
||||
{
|
||||
if (!module->size())
|
||||
return;
|
||||
return 0;
|
||||
llvm::PassManagerBuilder builder;
|
||||
llvm::legacy::PassManager mpm;
|
||||
llvm::legacy::FunctionPassManager fpm(module.get());
|
||||
@ -304,6 +306,11 @@ struct LLVMContext
|
||||
throw Exception("Function " + name + " failed to link", ErrorCodes::CANNOT_COMPILE_CODE);
|
||||
symbols[name] = reinterpret_cast<void *>(*address);
|
||||
}
|
||||
#if LLVM_VERSION_MAJOR >= 6
|
||||
return memory_mapper->memory_tracker.get();
|
||||
#else
|
||||
return 0;
|
||||
#endif
|
||||
}
|
||||
};
|
||||
|
||||
@ -344,7 +351,7 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
static void compileFunction(std::shared_ptr<LLVMContext> & context, const IFunctionBase & f)
|
||||
static void compileFunctionToLLVMByteCode(std::shared_ptr<LLVMContext> & context, const IFunctionBase & f)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::CompileFunction);
|
||||
|
||||
@ -428,7 +435,7 @@ static void compileFunction(std::shared_ptr<LLVMContext> & context, const IFunct
|
||||
|
||||
static llvm::Constant * getNativeValue(llvm::Type * type, const IColumn & column, size_t i)
|
||||
{
|
||||
if (!type)
|
||||
if (!type || column.size() <= i)
|
||||
return nullptr;
|
||||
if (auto * constant = typeid_cast<const ColumnConst *>(&column))
|
||||
return getNativeValue(type, constant->getDataColumn(), 0);
|
||||
@ -500,7 +507,7 @@ LLVMFunction::LLVMFunction(const ExpressionActions::Actions & actions, std::shar
|
||||
subexpressions[action.result_name] = subexpression(*action.function, std::move(args));
|
||||
originals.push_back(action.function);
|
||||
}
|
||||
compileFunction(context, *this);
|
||||
compileFunctionToLLVMByteCode(context, *this);
|
||||
}
|
||||
|
||||
PreparedFunctionPtr LLVMFunction::prepare(const Block &) const { return std::make_shared<LLVMPreparedFunction>(name, context); }
|
||||
@ -683,7 +690,6 @@ void compileFunctions(ExpressionActions::Actions & actions, const Names & output
|
||||
|
||||
static LLVMTargetInitializer initializer;
|
||||
|
||||
|
||||
auto dependents = getActionsDependents(actions, output_columns);
|
||||
/// Initialize context as late as possible and only if needed
|
||||
std::shared_ptr<LLVMContext> context;
|
||||
@ -700,16 +706,19 @@ void compileFunctions(ExpressionActions::Actions & actions, const Names & output
|
||||
if (fused[i].size() == 1)
|
||||
continue;
|
||||
|
||||
|
||||
auto hash_key = ExpressionActions::ActionsHash{}(fused[i]);
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
if (counter[hash_key]++ < min_count_to_compile)
|
||||
continue;
|
||||
}
|
||||
|
||||
std::shared_ptr<LLVMFunction> fn;
|
||||
if (compilation_cache)
|
||||
{
|
||||
/// Lock here, to be sure, that all functions will be compiled
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
/// Don't use getOrSet here, because sometimes we need to initialize context
|
||||
fn = compilation_cache->get(hash_key);
|
||||
if (!fn)
|
||||
{
|
||||
@ -743,7 +752,12 @@ void compileFunctions(ExpressionActions::Actions & actions, const Names & output
|
||||
}
|
||||
|
||||
if (context)
|
||||
context->finalize();
|
||||
{
|
||||
/// Lock here, because other threads can get uncompilted functions from cache
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
size_t used_memory = context->compileAllFunctionsToNativeCode();
|
||||
ProfileEvents::increment(ProfileEvents::CompileExpressionsBytes, used_memory);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -102,18 +102,18 @@ StoragePtr tryGetTable(const ASTPtr & database_and_table, const Context & contex
|
||||
|
||||
void replaceDatabaseAndTable(ASTPtr & database_and_table, const String & database_name, const String & table_name)
|
||||
{
|
||||
ASTPtr table = std::make_shared<ASTIdentifier>(table_name, ASTIdentifier::Table);
|
||||
ASTPtr table = ASTIdentifier::createSpecial(table_name);
|
||||
|
||||
if (!database_name.empty())
|
||||
{
|
||||
ASTPtr database = std::make_shared<ASTIdentifier>(database_name, ASTIdentifier::Database);
|
||||
ASTPtr database = ASTIdentifier::createSpecial(database_name);
|
||||
|
||||
database_and_table = std::make_shared<ASTIdentifier>(database_name + "." + table_name, ASTIdentifier::Table);
|
||||
database_and_table = ASTIdentifier::createSpecial(database_name + "." + table_name);
|
||||
database_and_table->children = {database, table};
|
||||
}
|
||||
else
|
||||
{
|
||||
database_and_table = std::make_shared<ASTIdentifier>(table_name, ASTIdentifier::Table);
|
||||
database_and_table = ASTIdentifier::createSpecial(table_name);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -54,14 +54,20 @@ StoragePtr InterpreterInsertQuery::getTable(const ASTInsertQuery & query)
|
||||
|
||||
Block InterpreterInsertQuery::getSampleBlock(const ASTInsertQuery & query, const StoragePtr & table)
|
||||
{
|
||||
Block table_sample_non_materialized = table->getSampleBlockNonMaterialized();
|
||||
|
||||
|
||||
Block table_sample_non_materialized = table->getSampleBlockNonMaterialized();
|
||||
/// If the query does not include information about columns
|
||||
if (!query.columns)
|
||||
return table_sample_non_materialized;
|
||||
{
|
||||
/// Format Native ignores header and write blocks as is.
|
||||
if (query.format == "Native")
|
||||
return {};
|
||||
else
|
||||
return table_sample_non_materialized;
|
||||
}
|
||||
|
||||
Block table_sample = table->getSampleBlock();
|
||||
|
||||
/// Form the block based on the column names from the query
|
||||
Block res;
|
||||
for (const auto & identifier : query.columns->children)
|
||||
@ -89,24 +95,25 @@ BlockIO InterpreterInsertQuery::execute()
|
||||
|
||||
auto table_lock = table->lockStructure(true, __PRETTY_FUNCTION__);
|
||||
|
||||
NamesAndTypesList required_columns = table->getColumns().getAllPhysical();
|
||||
|
||||
/// We create a pipeline of several streams, into which we will write data.
|
||||
BlockOutputStreamPtr out;
|
||||
|
||||
out = std::make_shared<PushingToViewsBlockOutputStream>(query.database, query.table, table, context, query_ptr, query.no_destination);
|
||||
|
||||
out = std::make_shared<AddingDefaultBlockOutputStream>(
|
||||
out, getSampleBlock(query, table), required_columns, table->getColumns().defaults, context);
|
||||
|
||||
/// Do not squash blocks if it is a sync INSERT into Distributed, since it lead to double bufferization on client and server side.
|
||||
/// Client-side bufferization might cause excessive timeouts (especially in case of big blocks).
|
||||
if (!(context.getSettingsRef().insert_distributed_sync && table->isRemote()))
|
||||
{
|
||||
out = std::make_shared<SquashingBlockOutputStream>(
|
||||
out, context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes);
|
||||
out, table->getSampleBlock(), context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes);
|
||||
}
|
||||
|
||||
/// Actually we don't know structure of input blocks from query/table,
|
||||
/// because some clients break insertion protocol (columns != header)
|
||||
out = std::make_shared<AddingDefaultBlockOutputStream>(
|
||||
out, getSampleBlock(query, table), table->getSampleBlock(), table->getColumns().defaults, context);
|
||||
|
||||
auto out_wrapper = std::make_shared<CountingBlockOutputStream>(out);
|
||||
out_wrapper->setProcessListElement(context.getProcessListElement());
|
||||
out = std::move(out_wrapper);
|
||||
|
@ -19,6 +19,7 @@
|
||||
#include <DataStreams/MaterializingBlockInputStream.h>
|
||||
#include <DataStreams/ConcatBlockInputStream.h>
|
||||
#include <DataStreams/RollupBlockInputStream.h>
|
||||
#include <DataStreams/CubeBlockInputStream.h>
|
||||
#include <DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h>
|
||||
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
@ -499,7 +500,7 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt
|
||||
bool aggregate_final =
|
||||
expressions.need_aggregate &&
|
||||
to_stage > QueryProcessingStage::WithMergeableState &&
|
||||
!query.group_by_with_totals && !query.group_by_with_rollup;
|
||||
!query.group_by_with_totals && !query.group_by_with_rollup && !query.group_by_with_cube;
|
||||
|
||||
if (expressions.first_stage)
|
||||
{
|
||||
@ -557,10 +558,15 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt
|
||||
if (!aggregate_final)
|
||||
{
|
||||
if (query.group_by_with_totals)
|
||||
executeTotalsAndHaving(pipeline, expressions.has_having, expressions.before_having, aggregate_overflow_row, !query.group_by_with_rollup);
|
||||
{
|
||||
bool final = !query.group_by_with_rollup && !query.group_by_with_cube;
|
||||
executeTotalsAndHaving(pipeline, expressions.has_having, expressions.before_having, aggregate_overflow_row, final);
|
||||
}
|
||||
|
||||
if (query.group_by_with_rollup)
|
||||
executeRollup(pipeline);
|
||||
if (query.group_by_with_rollup)
|
||||
executeRollupOrCube(pipeline, Modificator::ROLLUP);
|
||||
else if(query.group_by_with_cube)
|
||||
executeRollupOrCube(pipeline, Modificator::CUBE);
|
||||
}
|
||||
else if (expressions.has_having)
|
||||
executeHaving(pipeline, expressions.before_having);
|
||||
@ -575,10 +581,15 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt
|
||||
need_second_distinct_pass = query.distinct && pipeline.hasMoreThanOneStream();
|
||||
|
||||
if (query.group_by_with_totals && !aggregate_final)
|
||||
executeTotalsAndHaving(pipeline, false, nullptr, aggregate_overflow_row, !query.group_by_with_rollup);
|
||||
{
|
||||
bool final = !query.group_by_with_rollup && !query.group_by_with_cube;
|
||||
executeTotalsAndHaving(pipeline, expressions.has_having, expressions.before_having, aggregate_overflow_row, final);
|
||||
}
|
||||
|
||||
if (query.group_by_with_rollup && !aggregate_final)
|
||||
executeRollup(pipeline);
|
||||
executeRollupOrCube(pipeline, Modificator::ROLLUP);
|
||||
else if (query.group_by_with_cube && !aggregate_final)
|
||||
executeRollupOrCube(pipeline, Modificator::CUBE);
|
||||
}
|
||||
|
||||
if (expressions.has_order_by)
|
||||
@ -1087,7 +1098,7 @@ void InterpreterSelectQuery::executeTotalsAndHaving(Pipeline & pipeline, bool ha
|
||||
has_having ? query.having_expression->getColumnName() : "", settings.totals_mode, settings.totals_auto_threshold, final);
|
||||
}
|
||||
|
||||
void InterpreterSelectQuery::executeRollup(Pipeline & pipeline)
|
||||
void InterpreterSelectQuery::executeRollupOrCube(Pipeline & pipeline, Modificator modificator)
|
||||
{
|
||||
executeUnion(pipeline);
|
||||
|
||||
@ -1111,7 +1122,10 @@ void InterpreterSelectQuery::executeRollup(Pipeline & pipeline)
|
||||
settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set,
|
||||
context.getTemporaryPath());
|
||||
|
||||
pipeline.firstStream() = std::make_shared<RollupBlockInputStream>(pipeline.firstStream(), params);
|
||||
if (modificator == Modificator::ROLLUP)
|
||||
pipeline.firstStream() = std::make_shared<RollupBlockInputStream>(pipeline.firstStream(), params);
|
||||
else
|
||||
pipeline.firstStream() = std::make_shared<CubeBlockInputStream>(pipeline.firstStream(), params);
|
||||
}
|
||||
|
||||
|
||||
|
@ -190,7 +190,14 @@ private:
|
||||
void executeDistinct(Pipeline & pipeline, bool before_order, Names columns);
|
||||
void executeExtremes(Pipeline & pipeline);
|
||||
void executeSubqueriesInSetsAndJoins(Pipeline & pipeline, std::unordered_map<String, SubqueryForSet> & subqueries_for_sets);
|
||||
void executeRollup(Pipeline & pipeline);
|
||||
|
||||
enum class Modificator
|
||||
{
|
||||
ROLLUP = 0,
|
||||
CUBE = 1
|
||||
};
|
||||
|
||||
void executeRollupOrCube(Pipeline & pipeline, Modificator modificator);
|
||||
|
||||
/** If there is a SETTINGS section in the SELECT query, then apply settings from it.
|
||||
*
|
||||
|
@ -85,7 +85,7 @@ void QueryNormalizer::performImpl(ASTPtr & ast, MapOfASTs & finished_asts, SetOf
|
||||
if (functionIsInOrGlobalInOperator(func_node->name))
|
||||
if (ASTIdentifier * right = typeid_cast<ASTIdentifier *>(func_node->arguments->children.at(1).get()))
|
||||
if (!aliases.count(right->name))
|
||||
right->kind = ASTIdentifier::Table;
|
||||
right->setSpecial();
|
||||
|
||||
/// Special cases for count function.
|
||||
String func_name_lowercase = Poco::toLower(func_node->name);
|
||||
@ -108,7 +108,7 @@ void QueryNormalizer::performImpl(ASTPtr & ast, MapOfASTs & finished_asts, SetOf
|
||||
}
|
||||
else if ((identifier_node = typeid_cast<ASTIdentifier *>(ast.get())))
|
||||
{
|
||||
if (identifier_node->kind == ASTIdentifier::Column)
|
||||
if (identifier_node->general())
|
||||
{
|
||||
/// If it is an alias, but not a parent alias (for constructs like "SELECT column + 1 AS column").
|
||||
auto it_alias = aliases.find(identifier_node->name);
|
||||
@ -122,7 +122,7 @@ void QueryNormalizer::performImpl(ASTPtr & ast, MapOfASTs & finished_asts, SetOf
|
||||
{
|
||||
/// Avoid infinite recursion here
|
||||
auto replace_to_identifier = typeid_cast<ASTIdentifier *>(it_alias->second.get());
|
||||
bool is_cycle = replace_to_identifier && replace_to_identifier->kind == ASTIdentifier::Column
|
||||
bool is_cycle = replace_to_identifier && replace_to_identifier->general()
|
||||
&& replace_to_identifier->name == identifier_node->name;
|
||||
|
||||
if (!is_cycle)
|
||||
@ -164,9 +164,7 @@ void QueryNormalizer::performImpl(ASTPtr & ast, MapOfASTs & finished_asts, SetOf
|
||||
if (database_and_table_name)
|
||||
{
|
||||
if (ASTIdentifier * right = typeid_cast<ASTIdentifier *>(database_and_table_name.get()))
|
||||
{
|
||||
right->kind = ASTIdentifier::Table;
|
||||
}
|
||||
right->setSpecial();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -74,7 +74,7 @@ struct Settings
|
||||
M(SettingFloat, totals_auto_threshold, 0.5, "The threshold for totals_mode = 'auto'.") \
|
||||
\
|
||||
M(SettingBool, compile, false, "Whether query compilation is enabled.") \
|
||||
M(SettingBool, compile_expressions, false, "Compile some scalar functions and operators to native code.") \
|
||||
M(SettingBool, compile_expressions, true, "Compile some scalar functions and operators to native code.") \
|
||||
M(SettingUInt64, min_count_to_compile, 3, "The number of structurally identical queries before they are compiled.") \
|
||||
M(SettingUInt64, group_by_two_level_threshold, 100000, "From what number of keys, a two-level aggregation starts. 0 - the threshold is not set.") \
|
||||
M(SettingUInt64, group_by_two_level_threshold_bytes, 100000000, "From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set. Two-level aggregation is used when at least one of the thresholds is triggered.") \
|
||||
@ -148,6 +148,7 @@ struct Settings
|
||||
M(SettingBool, add_http_cors_header, false, "Write add http CORS header.") \
|
||||
\
|
||||
M(SettingBool, input_format_skip_unknown_fields, false, "Skip columns with unknown names from input data (it works for JSONEachRow and TSKV formats).") \
|
||||
M(SettingBool, input_format_import_nested_json, false, "Map nested JSON data to nested tables (it works for JSONEachRow format).") \
|
||||
\
|
||||
M(SettingBool, input_format_values_interpret_expressions, true, "For Values format: if field could not be parsed by streaming parser, run SQL parser and try to interpret it as SQL expression.") \
|
||||
\
|
||||
|
@ -50,12 +50,17 @@ void evaluateMissingDefaults(Block & block,
|
||||
ExpressionAnalyzer{default_expr_list, context, {}, available_columns}.getActions(true)->execute(copy_block);
|
||||
|
||||
/// move evaluated columns to the original block, materializing them at the same time
|
||||
for (auto & column_name_type : copy_block)
|
||||
size_t pos = 0;
|
||||
for (auto col = required_columns.begin(); col != required_columns.end(); ++col, ++pos)
|
||||
{
|
||||
if (ColumnPtr converted = column_name_type.column->convertToFullColumnIfConst())
|
||||
column_name_type.column = converted;
|
||||
if (copy_block.has(col->name))
|
||||
{
|
||||
auto evaluated_col = copy_block.getByName(col->name);
|
||||
if (ColumnPtr converted = evaluated_col.column->convertToFullColumnIfConst())
|
||||
evaluated_col.column = converted;
|
||||
|
||||
block.insert(std::move(column_name_type));
|
||||
block.insert(pos, std::move(evaluated_col));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -10,22 +10,17 @@ namespace DB
|
||||
*/
|
||||
class ASTIdentifier : public ASTWithAlias
|
||||
{
|
||||
public:
|
||||
enum Kind /// TODO This is semantic, not syntax. Remove it.
|
||||
{
|
||||
Column,
|
||||
Database,
|
||||
Table,
|
||||
Format,
|
||||
General,
|
||||
Special, // Database, Table, Format
|
||||
};
|
||||
|
||||
public:
|
||||
/// name. The composite identifier here will have a concatenated name (of the form a.b.c), and individual components will be available inside the children.
|
||||
String name;
|
||||
|
||||
/// what this identifier identifies
|
||||
Kind kind;
|
||||
|
||||
ASTIdentifier(const String & name_, const Kind kind_ = Column)
|
||||
ASTIdentifier(const String & name_, const Kind kind_ = General)
|
||||
: name(name_), kind(kind_) { range = StringRange(name.data(), name.data() + name.size()); }
|
||||
|
||||
/** Get the text that identifies this element. */
|
||||
@ -38,9 +33,21 @@ public:
|
||||
set.insert(name);
|
||||
}
|
||||
|
||||
void setSpecial() { kind = Special; }
|
||||
bool general() const { return kind == General; }
|
||||
bool special() const { return kind == Special; }
|
||||
|
||||
static std::shared_ptr<ASTIdentifier> createSpecial(const String & name_)
|
||||
{
|
||||
return std::make_shared<ASTIdentifier>(name_, ASTIdentifier::Special);
|
||||
}
|
||||
|
||||
protected:
|
||||
void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
||||
void appendColumnNameImpl(WriteBuffer & ostr) const override;
|
||||
|
||||
private:
|
||||
Kind kind;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -106,6 +106,9 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F
|
||||
if (group_by_with_rollup)
|
||||
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << (s.one_line ? "" : " ") << "WITH ROLLUP" << (s.hilite ? hilite_none : "");
|
||||
|
||||
if (group_by_with_cube)
|
||||
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << (s.one_line ? "" : " ") << "WITH CUBE" << (s.hilite ? hilite_none : "");
|
||||
|
||||
if (group_by_with_totals)
|
||||
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << (s.one_line ? "" : " ") << "WITH TOTALS" << (s.hilite ? hilite_none : "");
|
||||
|
||||
@ -345,11 +348,11 @@ void ASTSelectQuery::setDatabaseIfNeeded(const String & database_name)
|
||||
|
||||
if (table_expression->database_and_table_name->children.empty())
|
||||
{
|
||||
ASTPtr database = std::make_shared<ASTIdentifier>(database_name, ASTIdentifier::Database);
|
||||
ASTPtr database = ASTIdentifier::createSpecial(database_name);
|
||||
ASTPtr table = table_expression->database_and_table_name;
|
||||
|
||||
const String & old_name = static_cast<ASTIdentifier &>(*table_expression->database_and_table_name).name;
|
||||
table_expression->database_and_table_name = std::make_shared<ASTIdentifier>(database_name + "." + old_name, ASTIdentifier::Table);
|
||||
table_expression->database_and_table_name = ASTIdentifier::createSpecial(database_name + "." + old_name);
|
||||
table_expression->database_and_table_name->children = {database, table};
|
||||
}
|
||||
else if (table_expression->database_and_table_name->children.size() != 2)
|
||||
@ -376,18 +379,18 @@ void ASTSelectQuery::replaceDatabaseAndTable(const String & database_name, const
|
||||
table_expression = table_expr.get();
|
||||
}
|
||||
|
||||
ASTPtr table = std::make_shared<ASTIdentifier>(table_name, ASTIdentifier::Table);
|
||||
ASTPtr table = ASTIdentifier::createSpecial(table_name);
|
||||
|
||||
if (!database_name.empty())
|
||||
{
|
||||
ASTPtr database = std::make_shared<ASTIdentifier>(database_name, ASTIdentifier::Database);
|
||||
ASTPtr database = ASTIdentifier::createSpecial(database_name);
|
||||
|
||||
table_expression->database_and_table_name = std::make_shared<ASTIdentifier>(database_name + "." + table_name, ASTIdentifier::Table);
|
||||
table_expression->database_and_table_name = ASTIdentifier::createSpecial(database_name + "." + table_name);
|
||||
table_expression->database_and_table_name->children = {database, table};
|
||||
}
|
||||
else
|
||||
{
|
||||
table_expression->database_and_table_name = std::make_shared<ASTIdentifier>(table_name, ASTIdentifier::Table);
|
||||
table_expression->database_and_table_name = ASTIdentifier::createSpecial(table_name);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -29,6 +29,7 @@ public:
|
||||
ASTPtr group_expression_list;
|
||||
bool group_by_with_totals = false;
|
||||
bool group_by_with_rollup = false;
|
||||
bool group_by_with_cube = false;
|
||||
ASTPtr having_expression;
|
||||
ASTPtr order_expression_list;
|
||||
ASTPtr limit_by_value;
|
||||
|
@ -69,7 +69,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
|
||||
if (!format_p.parse(pos, query_with_output.format, expected))
|
||||
return false;
|
||||
typeid_cast<ASTIdentifier &>(*(query_with_output.format)).kind = ASTIdentifier::Format;
|
||||
typeid_cast<ASTIdentifier &>(*(query_with_output.format)).setSpecial();
|
||||
|
||||
query_with_output.children.push_back(query_with_output.format);
|
||||
}
|
||||
|
@ -40,6 +40,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
ParserKeyword s_settings("SETTINGS");
|
||||
ParserKeyword s_by("BY");
|
||||
ParserKeyword s_rollup("ROLLUP");
|
||||
ParserKeyword s_cube("CUBE");
|
||||
ParserKeyword s_top("TOP");
|
||||
ParserKeyword s_offset("OFFSET");
|
||||
|
||||
@ -116,24 +117,27 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
if (s_group_by.ignore(pos, expected))
|
||||
{
|
||||
if (s_rollup.ignore(pos, expected))
|
||||
{
|
||||
select_query->group_by_with_rollup = true;
|
||||
if (!open_bracket.ignore(pos, expected))
|
||||
return false;
|
||||
}
|
||||
else if (s_cube.ignore(pos, expected))
|
||||
select_query->group_by_with_cube = true;
|
||||
|
||||
if ((select_query->group_by_with_rollup || select_query->group_by_with_cube) && !open_bracket.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (!exp_list.parse(pos, select_query->group_expression_list, expected))
|
||||
return false;
|
||||
|
||||
if (select_query->group_by_with_rollup && !close_bracket.ignore(pos, expected))
|
||||
if ((select_query->group_by_with_rollup || select_query->group_by_with_cube) && !close_bracket.ignore(pos, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
/// WITH ROLLUP
|
||||
/// WITH ROLLUP, CUBE or TOTALS
|
||||
if (s_with.ignore(pos, expected))
|
||||
{
|
||||
if (s_rollup.ignore(pos, expected))
|
||||
select_query->group_by_with_rollup = true;
|
||||
else if (s_cube.ignore(pos, expected))
|
||||
select_query->group_by_with_cube = true;
|
||||
else if (s_totals.ignore(pos, expected))
|
||||
select_query->group_by_with_totals = true;
|
||||
else
|
||||
|
@ -151,8 +151,8 @@ bool ParserTablesInSelectQueryElement::parseImpl(Pos & pos, ASTPtr & node, Expec
|
||||
table_join->kind = ASTTableJoin::Kind::Cross;
|
||||
else
|
||||
{
|
||||
/// Maybe need use INNER by default as in another DBMS.
|
||||
return false;
|
||||
/// Use INNER by default as in another DBMS.
|
||||
table_join->kind = ASTTableJoin::Kind::Inner;
|
||||
}
|
||||
|
||||
if (table_join->strictness != ASTTableJoin::Strictness::Unspecified
|
||||
|
@ -47,7 +47,7 @@ Block ITableDeclaration::getSampleBlock() const
|
||||
{
|
||||
Block res;
|
||||
|
||||
for (const auto & col : boost::join(getColumns().ordinary, getColumns().materialized))
|
||||
for (const auto & col : getColumns().getAllPhysical())
|
||||
res.insert({ col.type->createColumn(), col.type, col.name });
|
||||
|
||||
return res;
|
||||
|
@ -727,7 +727,7 @@ void registerStorageKafka(StorageFactory & factory)
|
||||
String schema;
|
||||
if (args_count >= 6)
|
||||
{
|
||||
engine_args[5] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[4], args.local_context);
|
||||
engine_args[5] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[5], args.local_context);
|
||||
|
||||
auto ast = typeid_cast<const ASTLiteral *>(engine_args[5].get());
|
||||
if (ast && ast->value.getType() == Field::Types::String)
|
||||
|
@ -1,8 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Storages/MergeTree/MergeTreePartInfo.h>
|
||||
#include <mutex>
|
||||
#include <common/DateLUT.h>
|
||||
#include <Core/Types.h>
|
||||
#include <map>
|
||||
|
||||
|
@ -910,7 +910,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
|
||||
auto in = mutations_interpreter.execute();
|
||||
NamesAndTypesList all_columns = data.getColumns().getAllPhysical();
|
||||
|
||||
if (in->getHeader().columns() == all_columns.size())
|
||||
Block in_header = in->getHeader();
|
||||
|
||||
if (in_header.columns() == all_columns.size())
|
||||
{
|
||||
/// All columns are modified, proceed to write a new part from scratch.
|
||||
|
||||
@ -944,7 +946,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
|
||||
/// TODO: check that we modify only non-key columns in this case.
|
||||
|
||||
NameSet files_to_skip = {"checksums.txt", "columns.txt"};
|
||||
for (const auto & entry : in->getHeader())
|
||||
for (const auto & entry : in_header)
|
||||
{
|
||||
IDataType::StreamCallback callback = [&](const IDataType::SubstreamPath & substream_path)
|
||||
{
|
||||
@ -969,7 +971,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
|
||||
createHardLink(dir_it.path().toString(), destination.toString());
|
||||
}
|
||||
|
||||
MergedColumnOnlyOutputStream out(data, in->getHeader(), new_part_tmp_path, /* sync = */ false, compression_settings, /* skip_offsets = */ false);
|
||||
MergedColumnOnlyOutputStream out(data, in_header, new_part_tmp_path, /* sync = */ false, compression_settings, /* skip_offsets = */ false);
|
||||
|
||||
in->readPrefix();
|
||||
out.writePrefix();
|
||||
@ -989,11 +991,16 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
|
||||
new_data_part->checksums.write(out);
|
||||
}
|
||||
|
||||
new_data_part->columns = all_columns;
|
||||
new_data_part->columns = source_part->columns;
|
||||
for (const auto & new_column : in_header.getColumnsWithTypeAndName())
|
||||
{
|
||||
if (!new_data_part->columns.contains(new_column.name))
|
||||
new_data_part->columns.emplace_back(new_column.name, new_column.type);
|
||||
}
|
||||
{
|
||||
/// Write a file with a description of columns.
|
||||
WriteBufferFromFile out(new_part_tmp_path + "columns.txt", 4096);
|
||||
all_columns.writeText(out);
|
||||
new_data_part->columns.writeText(out);
|
||||
}
|
||||
|
||||
new_data_part->rows_count = source_part->rows_count;
|
||||
|
@ -645,10 +645,10 @@ void MergeTreeDataPart::checkConsistency(bool require_part_metadata)
|
||||
String mrk_file_name = file_name + ".mrk";
|
||||
String bin_file_name = file_name + ".bin";
|
||||
if (!checksums.files.count(mrk_file_name))
|
||||
throw Exception("No " + mrk_file_name + " file checksum for column " + name + " in part " + path,
|
||||
throw Exception("No " + mrk_file_name + " file checksum for column " + name_type.name + " in part " + path,
|
||||
ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
if (!checksums.files.count(bin_file_name))
|
||||
throw Exception("No " + bin_file_name + " file checksum for column " + name + " in part " + path,
|
||||
throw Exception("No " + bin_file_name + " file checksum for column " + name_type.name + " in part " + path,
|
||||
ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
}, stream_path);
|
||||
}
|
||||
|
@ -1,8 +1,9 @@
|
||||
#pragma once
|
||||
|
||||
#include <tuple>
|
||||
#include <Core/Types.h>
|
||||
#include <common/DayNum.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataFormatVersion.h>
|
||||
#include <common/DateLUT.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -384,7 +384,7 @@ bool MergeTreeWhereOptimizer::cannotBeMoved(const IAST * ptr) const
|
||||
else if (const auto identifier_ptr = typeid_cast<const ASTIdentifier *>(ptr))
|
||||
{
|
||||
/// disallow moving result of ARRAY JOIN to PREWHERE
|
||||
if (identifier_ptr->kind == ASTIdentifier::Column)
|
||||
if (identifier_ptr->general())
|
||||
if (array_joined_names.count(identifier_ptr->name) ||
|
||||
array_joined_names.count(Nested::extractTableName(identifier_ptr->name)))
|
||||
return true;
|
||||
|
@ -595,7 +595,7 @@ void StorageBuffer::writeBlockToDestination(const Block & block, StoragePtr tabl
|
||||
insert->columns = list_of_columns;
|
||||
list_of_columns->children.reserve(columns_intersection.size());
|
||||
for (const String & column : columns_intersection)
|
||||
list_of_columns->children.push_back(std::make_shared<ASTIdentifier>(column, ASTIdentifier::Column));
|
||||
list_of_columns->children.push_back(std::make_shared<ASTIdentifier>(column));
|
||||
|
||||
InterpreterInsertQuery interpreter{insert, context, allow_materialized};
|
||||
|
||||
|
@ -259,8 +259,19 @@ BlockInputStreams StorageMerge::read(
|
||||
switch (processed_stage)
|
||||
{
|
||||
case QueryProcessingStage::FetchColumns:
|
||||
{
|
||||
header = getSampleBlockForColumns(column_names);
|
||||
|
||||
if (query_info.prewhere_info)
|
||||
{
|
||||
query_info.prewhere_info->prewhere_actions->execute(header);
|
||||
header = materializeBlock(header);
|
||||
if (query_info.prewhere_info->remove_prewhere_column)
|
||||
header.erase(query_info.prewhere_info->prewhere_column_name);
|
||||
}
|
||||
|
||||
break;
|
||||
}
|
||||
case QueryProcessingStage::WithMergeableState:
|
||||
case QueryProcessingStage::Complete:
|
||||
header = materializeBlock(InterpreterSelectQuery(
|
||||
|
@ -47,7 +47,7 @@ BlockInputStreams StorageView::read(
|
||||
{
|
||||
BlockInputStreams res;
|
||||
|
||||
ASTPtr & current_inner_query = inner_query;
|
||||
ASTPtr current_inner_query = inner_query;
|
||||
|
||||
if (context.getSettings().enable_optimize_predicate_expression)
|
||||
{
|
||||
|
@ -97,7 +97,7 @@ static bool isValidFunction(const ASTPtr & expression, const NameSet & columns)
|
||||
|
||||
if (const ASTIdentifier * identifier = typeid_cast<const ASTIdentifier *>(&*expression))
|
||||
{
|
||||
if (identifier->kind == ASTIdentifier::Kind::Column)
|
||||
if (identifier->general())
|
||||
return columns.count(identifier->name);
|
||||
}
|
||||
return true;
|
||||
|
@ -292,7 +292,7 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C
|
||||
/// We need to mark them as the name of the database or table, because the default value is column.
|
||||
for (auto & arg : args)
|
||||
if (ASTIdentifier * id = typeid_cast<ASTIdentifier *>(arg.get()))
|
||||
id->kind = ASTIdentifier::Table;
|
||||
id->setSpecial();
|
||||
|
||||
ClusterPtr cluster;
|
||||
if (!cluster_name.empty())
|
||||
|
@ -66,7 +66,7 @@ StoragePtr TableFunctionShardByHash::executeImpl(const ASTPtr & ast_function, co
|
||||
/// Similar to other TableFunctions.
|
||||
for (auto & arg : args)
|
||||
if (ASTIdentifier * id = typeid_cast<ASTIdentifier *>(arg.get()))
|
||||
id->kind = ASTIdentifier::Table;
|
||||
id->setSpecial();
|
||||
|
||||
auto cluster = context.getCluster(cluster_name);
|
||||
size_t shard_index = sipHash64(key) % cluster->getShardCount();
|
||||
|
@ -30,9 +30,12 @@
|
||||
<value>toHour</value>
|
||||
<value>toDayOfWeek</value>
|
||||
<value>toDayOfMonth</value>
|
||||
<value>toDayOfYear</value>
|
||||
<value>toMonth</value>
|
||||
<!--value>toQuarter</value-->
|
||||
<value>toQuarter</value>
|
||||
<value>toYear</value>
|
||||
<value>toISOWeek</value>
|
||||
<value>toISOYear</value>
|
||||
|
||||
<value>toStartOfMinute</value>
|
||||
<value>toStartOfFiveMinute</value>
|
||||
@ -43,7 +46,8 @@
|
||||
<value>toMonday</value>
|
||||
<value>toStartOfMonth</value>
|
||||
<value>toStartOfQuarter</value>
|
||||
<value>toYear</value>
|
||||
<value>toStartOfYear</value>
|
||||
<value>toStartOfISOYear</value>
|
||||
|
||||
<value>toTime</value>
|
||||
|
||||
@ -65,16 +69,20 @@
|
||||
<values>
|
||||
<value>toDayOfWeek</value>
|
||||
<value>toDayOfMonth</value>
|
||||
<value>toDayOfYear</value>
|
||||
<value>toMonth</value>
|
||||
<!--value>toQuarter</value-->
|
||||
<value>toQuarter</value>
|
||||
<value>toYear</value>
|
||||
<value>toISOWeek</value>
|
||||
<value>toISOYear</value>
|
||||
|
||||
<value>toStartOfDay</value>
|
||||
<value>toDate</value>
|
||||
<value>toMonday</value>
|
||||
<value>toStartOfMonth</value>
|
||||
<value>toStartOfQuarter</value>
|
||||
<value>toYear</value>
|
||||
<value>toStartOfYear</value>
|
||||
<value>toStartOfISOYear</value>
|
||||
|
||||
<value>toRelativeYearNum</value>
|
||||
<value>toRelativeMonthNum</value>
|
||||
|
38
dbms/tests/performance/date_time/format_date_time.xml
Normal file
38
dbms/tests/performance/date_time/format_date_time.xml
Normal file
@ -0,0 +1,38 @@
|
||||
<test>
|
||||
<name>Function formatDateTime</name>
|
||||
|
||||
<type>once</type>
|
||||
|
||||
<tags>
|
||||
</tags>
|
||||
|
||||
<stop_conditions>
|
||||
<any_of>
|
||||
<average_speed_not_changing_for_ms>10000</average_speed_not_changing_for_ms>
|
||||
<total_time_ms>1000</total_time_ms>
|
||||
</any_of>
|
||||
</stop_conditions>
|
||||
|
||||
<metrics>
|
||||
<max_rows_per_second />
|
||||
<max_bytes_per_second />
|
||||
<avg_rows_per_second />
|
||||
<avg_bytes_per_second />
|
||||
</metrics>
|
||||
|
||||
<substitutions>
|
||||
<substitution>
|
||||
<name>format</name>
|
||||
<values>
|
||||
<value>%F %T</value>
|
||||
<value>%H:%M:%S</value>
|
||||
<value>%Y-%m-%d %H:%M:%S</value>
|
||||
<value>%m/%d/%Y %H:%M:%S</value>
|
||||
<value>Hello</value>
|
||||
<value>The current time is: %I:%M %p</value>
|
||||
</values>
|
||||
</substitution>
|
||||
</substitutions>
|
||||
|
||||
<query>SELECT count() FROM system.numbers WHERE NOT ignore(toDateTime('2017-01-01 00:00:00') + number % 100000000 + rand() % 100000 AS t, formatDateTime(t, '{format}'))</query>
|
||||
</test>
|
@ -25,6 +25,7 @@
|
||||
2000-01-01 1 test string 1 1
|
||||
1 2000-01-01 2000-01-01 1 test string 1 1
|
||||
2000-01-01 1 test string 1 1
|
||||
2000-01-01 2 test string 2 2
|
||||
1
|
||||
1
|
||||
-------Push to having expression, need check.-------
|
||||
|
@ -57,6 +57,7 @@ SELECT * FROM (SELECT 1 AS id, toDate('2000-01-01') AS date FROM system.numbers
|
||||
|
||||
-- Optimize predicate expression with view
|
||||
SELECT * FROM test.test_view WHERE id = 1;
|
||||
SELECT * FROM test.test_view WHERE id = 2;
|
||||
SELECT id FROM test.test_view WHERE id = 1;
|
||||
SELECT s.id FROM test.test_view AS s WHERE id = 1;
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
-999999999 -999999999999999999 0 -0.999999999 0.000000000000000000 0.00000000000000000000000000000000000000 -9999.99999 0.000000000 0.000000000000000000 0
|
||||
-900000000 -900000000000000000 -90000000000000000000000000000000000000 -0.000000009 -0.000000000000000009 -0.00000000000000000000000000000000000009 0.00000 0.000000000 0.000000000000000000 0
|
||||
-1 -1 -1 -0.000000001 0.000000000000000000 0.00000000000000000000000000000000000000 -0.00001 -0.000000001 0.000000000000000000 -1
|
||||
0 0 -99999999999999999999999999999999999999 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
|
||||
0 0 0 0.000000000 -0.999999999999999999 0.00000000000000000000000000000000000000 0.00000 -999999999.999999999 0.000000000000000000 0
|
||||
@ -18,4 +19,5 @@
|
||||
0 0 99999999999999999999999999999999999999 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
|
||||
1 1 1 0.000000001 0.000000000000000000 0.00000000000000000000000000000000000000 0.00001 0.000000001 0.000000000000000000 1
|
||||
42 42 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.99999 0.000000000 0.000000000000000000 0
|
||||
900000000 900000000000000000 90000000000000000000000000000000000000 0.000000009 0.000000000000000009 0.00000000000000000000000000000000000009 0.00000 0.000000000 0.000000000000000000 0
|
||||
999999999 999999999999999999 0 0.999999999 0.000000000000000000 0.00000000000000000000000000000000000000 9999.99999 0.000000000 0.000000000000000000 0
|
||||
|
@ -90,7 +90,12 @@ INSERT INTO test.decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-0.0, -0.0, -0.0
|
||||
INSERT INTO test.decimal (a, b, g) VALUES ('42.00000', 42.0000000000000000000000000000000, '0.999990');
|
||||
INSERT INTO test.decimal (a) VALUES ('-9x'); -- { clientError 72 }
|
||||
INSERT INTO test.decimal (a) VALUES ('0x1'); -- { clientError 72 }
|
||||
INSERT INTO test.decimal (a) VALUES ('1e2'); -- { clientError 72 }
|
||||
|
||||
INSERT INTO test.decimal (a, b, c, d, e, f) VALUES ('0.9e9', '0.9e18', '0.9e38', '9e-9', '9e-18', '9e-38');
|
||||
INSERT INTO test.decimal (a, b, c, d, e, f) VALUES ('-0.9e9', '-0.9e18', '-0.9e38', '-9e-9', '-9e-18', '-9e-38');
|
||||
|
||||
INSERT INTO test.decimal (a, b, c, d, e, f) VALUES ('1e9', '1e18', '1e38', '1e-10', '1e-19', '1e-39'); -- { clientError 69 }
|
||||
INSERT INTO test.decimal (a, b, c, d, e, f) VALUES ('-1e9', '-1e18', '-1e38', '-1e-10', '-1e-19', '-1e-39'); -- { clientError 69 }
|
||||
|
||||
SELECT * FROM test.decimal ORDER BY a, b, c, d, e, f, g, h, i, j;
|
||||
DROP TABLE IF EXISTS test.decimal;
|
||||
|
@ -5,6 +5,58 @@ SELECT toDecimal32('1.1', 1), toDecimal32('1.1', 2), toDecimal32('1.1', 8);
|
||||
SELECT toDecimal32('1.1', 0); -- { serverError 69 }
|
||||
SELECT toDecimal32(1.1, 0), toDecimal32(1.1, 1), toDecimal32(1.1, 2), toDecimal32(1.1, 8);
|
||||
|
||||
SELECT '1000000000' AS x, toDecimal32(x, 0); -- { serverError 69 }
|
||||
SELECT '-1000000000' AS x, toDecimal32(x, 0); -- { serverError 69 }
|
||||
SELECT '1000000000000000000' AS x, toDecimal64(x, 0); -- { serverError 69 }
|
||||
SELECT '-1000000000000000000' AS x, toDecimal64(x, 0); -- { serverError 69 }
|
||||
SELECT '100000000000000000000000000000000000000' AS x, toDecimal128(x, 0); -- { serverError 69 }
|
||||
SELECT '-100000000000000000000000000000000000000' AS x, toDecimal128(x, 0); -- { serverError 69 }
|
||||
SELECT '1' AS x, toDecimal32(x, 9); -- { serverError 69 }
|
||||
SELECT '-1' AS x, toDecimal32(x, 9); -- { serverError 69 }
|
||||
SELECT '1' AS x, toDecimal64(x, 18); -- { serverError 69 }
|
||||
SELECT '-1' AS x, toDecimal64(x, 18); -- { serverError 69 }
|
||||
SELECT '1' AS x, toDecimal128(x, 38); -- { serverError 69 }
|
||||
SELECT '-1' AS x, toDecimal128(x, 38); -- { serverError 69 }
|
||||
|
||||
SELECT '0.1' AS x, toDecimal32(x, 0); -- { serverError 69 }
|
||||
SELECT '-0.1' AS x, toDecimal32(x, 0); -- { serverError 69 }
|
||||
SELECT '0.1' AS x, toDecimal64(x, 0); -- { serverError 69 }
|
||||
SELECT '-0.1' AS x, toDecimal64(x, 0); -- { serverError 69 }
|
||||
SELECT '0.1' AS x, toDecimal128(x, 0); -- { serverError 69 }
|
||||
SELECT '-0.1' AS x, toDecimal128(x, 0); -- { serverError 69 }
|
||||
SELECT '0.0000000001' AS x, toDecimal32(x, 9); -- { serverError 69 }
|
||||
SELECT '-0.0000000001' AS x, toDecimal32(x, 9); -- { serverError 69 }
|
||||
SELECT '0.0000000000000000001' AS x, toDecimal64(x, 18); -- { serverError 69 }
|
||||
SELECT '-0.0000000000000000001' AS x, toDecimal64(x, 18); -- { serverError 69 }
|
||||
SELECT '0.000000000000000000000000000000000000001' AS x, toDecimal128(x, 38); -- { serverError 69 }
|
||||
SELECT '-0.000000000000000000000000000000000000001' AS x, toDecimal128(x, 38); -- { serverError 69 }
|
||||
|
||||
SELECT '1e9' AS x, toDecimal32(x, 0); -- { serverError 69 }
|
||||
SELECT '-1E9' AS x, toDecimal32(x, 0); -- { serverError 69 }
|
||||
SELECT '1E18' AS x, toDecimal64(x, 0); -- { serverError 69 }
|
||||
SELECT '-1e18' AS x, toDecimal64(x, 0); -- { serverError 69 }
|
||||
SELECT '1e38' AS x, toDecimal128(x, 0); -- { serverError 69 }
|
||||
SELECT '-1E38' AS x, toDecimal128(x, 0); -- { serverError 69 }
|
||||
SELECT '1e0' AS x, toDecimal32(x, 9); -- { serverError 69 }
|
||||
SELECT '-1e-0' AS x, toDecimal32(x, 9); -- { serverError 69 }
|
||||
SELECT '1e0' AS x, toDecimal64(x, 18); -- { serverError 69 }
|
||||
SELECT '-1e-0' AS x, toDecimal64(x, 18); -- { serverError 69 }
|
||||
SELECT '1e-0' AS x, toDecimal128(x, 38); -- { serverError 69 }
|
||||
SELECT '-1e0' AS x, toDecimal128(x, 38); -- { serverError 69 }
|
||||
|
||||
SELECT '1e-1' AS x, toDecimal32(x, 0); -- { serverError 69 }
|
||||
SELECT '-1e-1' AS x, toDecimal32(x, 0); -- { serverError 69 }
|
||||
SELECT '1e-1' AS x, toDecimal64(x, 0); -- { serverError 69 }
|
||||
SELECT '-1e-1' AS x, toDecimal64(x, 0); -- { serverError 69 }
|
||||
SELECT '1e-1' AS x, toDecimal128(x, 0); -- { serverError 69 }
|
||||
SELECT '-1e-1' AS x, toDecimal128(x, 0); -- { serverError 69 }
|
||||
SELECT '1e-10' AS x, toDecimal32(x, 9); -- { serverError 69 }
|
||||
SELECT '-1e-10' AS x, toDecimal32(x, 9); -- { serverError 69 }
|
||||
SELECT '1e-19' AS x, toDecimal64(x, 18); -- { serverError 69 }
|
||||
SELECT '-1e-19' AS x, toDecimal64(x, 18); -- { serverError 69 }
|
||||
SELECT '1e-39' AS x, toDecimal128(x, 38); -- { serverError 69 }
|
||||
SELECT '-1e-39' AS x, toDecimal128(x, 38); -- { serverError 69 }
|
||||
|
||||
SELECT toFloat32(9999999) as x, toDecimal32(x, 0), toDecimal32(-x, 0), toDecimal64(x, 0), toDecimal64(-x, 0);
|
||||
SELECT toFloat32(999999.9) as x, toDecimal32(x, 1), toDecimal32(-x, 1), toDecimal64(x, 1), toDecimal64(-x, 1);
|
||||
SELECT toFloat32(99999.99) as x, toDecimal32(x, 2), toDecimal32(-x, 2), toDecimal64(x, 2), toDecimal64(-x, 2);
|
||||
|
@ -0,0 +1,14 @@
|
||||
a Decimal(9, 4) DEFAULT CAST(0, \'Decimal(9, 4)\')
|
||||
b Decimal(18, 4) DEFAULT CAST(a / 2, \'Decimal(18, 4)\')
|
||||
c Decimal(38, 4) DEFAULT CAST(b / 3, \'Decimal(38, 4)\')
|
||||
d Decimal(9, 4) MATERIALIZED a + toDecimal32(\'0.2\', 1)
|
||||
e Decimal(18, 4) ALIAS b * 2
|
||||
f Decimal(38, 4) ALIAS c * 6
|
||||
0.0000 0.0000 0.0000
|
||||
1.0000 0.5000 0.1666
|
||||
2.0000 1.0000 0.3333
|
||||
3.0000 1.5000 0.5000
|
||||
0.0000 0.0000 0.0000 0.2000 0.0000 0.0000
|
||||
1.0000 0.5000 0.1666 1.2000 1.0000 0.9996
|
||||
2.0000 1.0000 0.3333 2.2000 2.0000 1.9998
|
||||
3.0000 1.5000 0.5000 3.2000 3.0000 3.0000
|
23
dbms/tests/queries/0_stateless/00700_decimal_defaults.sql
Normal file
23
dbms/tests/queries/0_stateless/00700_decimal_defaults.sql
Normal file
@ -0,0 +1,23 @@
|
||||
SET allow_experimental_decimal_type = 1;
|
||||
SET send_logs_level = 'none';
|
||||
|
||||
CREATE DATABASE IF NOT EXISTS test;
|
||||
DROP TABLE IF EXISTS test.decimal;
|
||||
|
||||
CREATE TABLE IF NOT EXISTS test.decimal
|
||||
(
|
||||
a DECIMAL(9,4) DEFAULT 0,
|
||||
b DECIMAL(18,4) DEFAULT a / 2,
|
||||
c DECIMAL(38,4) DEFAULT b / 3,
|
||||
d MATERIALIZED a + toDecimal32('0.2', 1),
|
||||
e ALIAS b * 2,
|
||||
f ALIAS c * 6
|
||||
) ENGINE = Memory;
|
||||
|
||||
DESC TABLE test.decimal;
|
||||
|
||||
INSERT INTO test.decimal (a) VALUES (0), (1), (2), (3);
|
||||
SELECT * FROM test.decimal;
|
||||
SELECT a, b, c, d, e, f FROM test.decimal;
|
||||
|
||||
DROP TABLE IF EXISTS test.decimal;
|
@ -1,3 +1,4 @@
|
||||
{"a":0.055,"b":-0.000000005,"c":0.000000000000000005}
|
||||
{"a":0.100,"b":-0.100000000,"c":0.100000000000000000}
|
||||
{"a":0.200,"b":-0.200000000,"c":0.200000000000000000}
|
||||
{"a":0.300,"b":-0.300000000,"c":0.300000000000000000}
|
||||
@ -10,6 +11,8 @@
|
||||
{"a":3.300,"b":-3.300000000,"c":3.300000000000000000}
|
||||
{"a":42.000,"b":-42.000000000,"c":42.000000000000000000}
|
||||
{"a":42.420,"b":-42.420000000,"c":42.420000000000000000}
|
||||
{"a":440000.000,"b":-400000000.000000000,"c":40000000000000000000.000000000000000000}
|
||||
0.055,-0.000000005,0.000000000000000005
|
||||
0.100,-0.100000000,0.100000000000000000
|
||||
0.200,-0.200000000,0.200000000000000000
|
||||
0.300,-0.300000000,0.300000000000000000
|
||||
@ -22,6 +25,8 @@
|
||||
3.300,-3.300000000,3.300000000000000000
|
||||
42.000,-42.000000000,42.000000000000000000
|
||||
42.420,-42.420000000,42.420000000000000000
|
||||
440000.000,-400000000.000000000,40000000000000000000.000000000000000000
|
||||
0.055 -0.000000005 0.000000000000000005
|
||||
0.100 -0.100000000 0.100000000000000000
|
||||
0.200 -0.200000000 0.200000000000000000
|
||||
0.300 -0.300000000 0.300000000000000000
|
||||
@ -34,3 +39,4 @@
|
||||
3.300 -3.300000000 3.300000000000000000
|
||||
42.000 -42.000000000 42.000000000000000000
|
||||
42.420 -42.420000000 42.420000000000000000
|
||||
440000.000 -400000000.000000000 40000000000000000000.000000000000000000
|
||||
|
@ -11,9 +11,9 @@ CREATE TABLE IF NOT EXISTS test.decimal
|
||||
|
||||
INSERT INTO test.decimal (a, b, c) VALUES (42.0, -42.0, 42) (0.42, -0.42, .42) (42.42, -42.42, 42.42);
|
||||
INSERT INTO test.decimal (a, b, c) FORMAT JSONEachRow {"a":1.1, "b":-1.1, "c":1.1} {"a":1.0, "b":-1.0, "c":1} {"a":0.1, "b":-0.1, "c":.1};
|
||||
INSERT INTO test.decimal (a, b, c) FORMAT CSV 2.0, -2.0, 2
|
||||
INSERT INTO test.decimal (a, b, c) FORMAT CSV 2.0,-2.0,2
|
||||
;
|
||||
INSERT INTO test.decimal (a, b, c) FORMAT CSV 0.2, -0.2, .2
|
||||
INSERT INTO test.decimal (a, b, c) FORMAT CSV 0.2 ,-0.2 ,.2
|
||||
;
|
||||
INSERT INTO test.decimal (a, b, c) FORMAT CSV 2.2 , -2.2 , 2.2
|
||||
;
|
||||
@ -23,6 +23,10 @@ INSERT INTO test.decimal (a, b, c) FORMAT TabSeparated 3.0 -3.0 3
|
||||
;
|
||||
INSERT INTO test.decimal (a, b, c) FORMAT TabSeparated 0.3 -0.3 .3
|
||||
;
|
||||
INSERT INTO test.decimal (a, b, c) FORMAT CSV 4.4E+5,-4E+8,.4E+20
|
||||
;
|
||||
INSERT INTO test.decimal (a, b, c) FORMAT CSV 5.5e-2, -5e-9 ,.5e-17
|
||||
;
|
||||
|
||||
SELECT * FROM test.decimal ORDER BY a FORMAT JSONEachRow;
|
||||
SELECT * FROM test.decimal ORDER BY b DESC FORMAT CSV;
|
||||
|
@ -1,8 +1,8 @@
|
||||
[1,3,6,10]
|
||||
[1,0,1,0]
|
||||
[1,0,5,3]
|
||||
[0,1,1,1]
|
||||
[0,6,93,-95]
|
||||
[1,0,0,0]
|
||||
[1,0,0,1]
|
||||
[1,1.4,1.2999999999999998]
|
||||
[1,4,5]
|
||||
[0,-4,3,1]
|
||||
|
@ -0,0 +1,26 @@
|
||||
256
|
||||
2018-09-17 2018-09-17 00:00:00 38 38 2018 2018 2018-01-01 2018-01-01
|
||||
2018-12-25 2018-12-25 00:00:00 52 52 2018 2018 2018-01-01 2018-01-01 359 359
|
||||
2018-12-26 2018-12-26 00:00:00 52 52 2018 2018 2018-01-01 2018-01-01 360 360
|
||||
2018-12-27 2018-12-27 00:00:00 52 52 2018 2018 2018-01-01 2018-01-01 361 361
|
||||
2018-12-28 2018-12-28 00:00:00 52 52 2018 2018 2018-01-01 2018-01-01 362 362
|
||||
2018-12-29 2018-12-29 00:00:00 52 52 2018 2018 2018-01-01 2018-01-01 363 363
|
||||
2018-12-30 2018-12-30 00:00:00 52 52 2018 2018 2018-01-01 2018-01-01 364 364
|
||||
2018-12-31 2018-12-31 00:00:00 1 1 2019 2019 2018-12-31 2018-12-31 365 365
|
||||
2019-01-01 2019-01-01 00:00:00 1 1 2019 2019 2018-12-31 2018-12-31 1 1
|
||||
2019-01-02 2019-01-02 00:00:00 1 1 2019 2019 2018-12-31 2018-12-31 2 2
|
||||
2019-01-03 2019-01-03 00:00:00 1 1 2019 2019 2018-12-31 2018-12-31 3 3
|
||||
2016-12-25 2016-12-25 00:00:00 51 51 2016 2016 2016-01-04 2016-01-04 360 360
|
||||
2016-12-26 2016-12-26 00:00:00 52 52 2016 2016 2016-01-04 2016-01-04 361 361
|
||||
2016-12-27 2016-12-27 00:00:00 52 52 2016 2016 2016-01-04 2016-01-04 362 362
|
||||
2016-12-28 2016-12-28 00:00:00 52 52 2016 2016 2016-01-04 2016-01-04 363 363
|
||||
2016-12-29 2016-12-29 00:00:00 52 52 2016 2016 2016-01-04 2016-01-04 364 364
|
||||
2016-12-30 2016-12-30 00:00:00 52 52 2016 2016 2016-01-04 2016-01-04 365 365
|
||||
2016-12-31 2016-12-31 00:00:00 52 52 2016 2016 2016-01-04 2016-01-04 366 366
|
||||
2017-01-01 2017-01-01 00:00:00 52 52 2016 2016 2016-01-04 2016-01-04 1 1
|
||||
2017-01-02 2017-01-02 00:00:00 1 1 2017 2017 2017-01-02 2017-01-02 2 2
|
||||
2017-01-03 2017-01-03 00:00:00 1 1 2017 2017 2017-01-02 2017-01-02 3 3
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
@ -0,0 +1,17 @@
|
||||
-- International Programmers' Day
|
||||
SELECT toDayOfYear(toDate('2018-09-13'));
|
||||
|
||||
SELECT toDate('2018-09-17') AS x, toDateTime(x) AS x_t, toISOWeek(x), toISOWeek(x_t), toISOYear(x), toISOYear(x_t), toStartOfISOYear(x), toStartOfISOYear(x_t);
|
||||
|
||||
SELECT toDate('2018-12-25') + number AS x, toDateTime(x) AS x_t, toISOWeek(x) AS w, toISOWeek(x_t) AS wt, toISOYear(x) AS y, toISOYear(x_t) AS yt, toStartOfISOYear(x) AS ys, toStartOfISOYear(x_t) AS yst, toDayOfYear(x) AS dy, toDayOfYear(x_t) AS dyt FROM system.numbers LIMIT 10;
|
||||
SELECT toDate('2016-12-25') + number AS x, toDateTime(x) AS x_t, toISOWeek(x) AS w, toISOWeek(x_t) AS wt, toISOYear(x) AS y, toISOYear(x_t) AS yt, toStartOfISOYear(x) AS ys, toStartOfISOYear(x_t) AS yst, toDayOfYear(x) AS dy, toDayOfYear(x_t) AS dyt FROM system.numbers LIMIT 10;
|
||||
|
||||
-- ISO year always begins at monday.
|
||||
SELECT DISTINCT toDayOfWeek(toStartOfISOYear(toDateTime(1000000000 + rand64() % 1000000000))) FROM numbers(10000);
|
||||
SELECT DISTINCT toDayOfWeek(toStartOfISOYear(toDate(10000 + rand64() % 20000))) FROM numbers(10000);
|
||||
|
||||
-- Year and ISO year don't differ by more than one.
|
||||
WITH toDateTime(1000000000 + rand64() % 1000000000) AS time SELECT max(abs(toYear(time) - toISOYear(time))) <= 1 FROM numbers(10000);
|
||||
|
||||
-- ISO week is between 1 and 53
|
||||
WITH toDateTime(1000000000 + rand64() % 1000000000) AS time SELECT DISTINCT toISOWeek(time) BETWEEN 1 AND 53 FROM numbers(1000000);
|
@ -0,0 +1,4 @@
|
||||
123.456 789.012 345678 Hello
|
||||
123.456 789.012 345678 Hello
|
||||
123.456 789.012 345678 Hello
|
||||
123.456 789.012 345678 Hello
|
14
dbms/tests/queries/0_stateless/00707_float_csv_delimiter.sql
Normal file
14
dbms/tests/queries/0_stateless/00707_float_csv_delimiter.sql
Normal file
@ -0,0 +1,14 @@
|
||||
CREATE TEMPORARY TABLE test (x Float32, y Float64, z UInt64, s String);
|
||||
|
||||
INSERT INTO test FORMAT CSV 123.456,789.012,345678,Hello
|
||||
|
||||
SET format_csv_delimiter = ';';
|
||||
INSERT INTO test FORMAT CSV 123.456;789.012;345678;Hello
|
||||
|
||||
SET format_csv_delimiter = ':';
|
||||
INSERT INTO test FORMAT CSV 123.456:789.012:345678:Hello
|
||||
|
||||
SET format_csv_delimiter = '|';
|
||||
INSERT INTO test FORMAT CSV 123.456|789.012|345678|Hello
|
||||
|
||||
SELECT * FROM test;
|
@ -0,0 +1,15 @@
|
||||
1 ok ['abc','def'] [1,23]
|
||||
0 [] []
|
||||
0 ['x','y','z'] [45,67,8]
|
||||
1 ok ['dog','cat','pig'] [3,3,3]
|
||||
1 ok ['zero','negative one'] [0,-1]
|
||||
1 ok [] []
|
||||
0 [] []
|
||||
0 [] []
|
||||
1 ok [] []
|
||||
1 ok [] []
|
||||
1 ok ['abc','def'] [1,23]
|
||||
0 [] []
|
||||
0 ['x','y','z'] [45,67,8]
|
||||
1 ok ['dog','cat','pig'] [3,3,3]
|
||||
1 ok ['zero','negative one'] [0,-1]
|
38
dbms/tests/queries/0_stateless/00715_json_each_row_input_nested.sh
Executable file
38
dbms/tests/queries/0_stateless/00715_json_each_row_input_nested.sh
Executable file
@ -0,0 +1,38 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
set -e
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test.json_each_row_nested"
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE test.json_each_row_nested (d1 UInt8, d2 String, n Nested (s String, i Int32) ) ENGINE = Memory"
|
||||
|
||||
echo '{"d1" : 1, "d2" : "ok", "n.s" : ["abc", "def"], "n.i" : [1, 23]}
|
||||
{ }
|
||||
{"t1" : 0, "n.t2":true,"n.i":[45, 67, 8], "n.s":["x", "y", "z"],"t5":[],"t6":"trash" }
|
||||
{"d2":"ok","n.s":["dog", "cat", "pig"], "n.x":[["1","2"]], "d1":"1", "n.i":[3, 3, 3]}
|
||||
{"t0" : -0.1, "n.s" : ["zero","negative one"], "a.b" : 0, "n.i" : [0, -1], "d2" : "ok", "d1" : 1}' \
|
||||
| $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 -q "INSERT INTO test.json_each_row_nested FORMAT JSONEachRow"
|
||||
|
||||
$CLICKHOUSE_CLIENT --max_threads=1 -q "SELECT * FROM test.json_each_row_nested"
|
||||
|
||||
test_nested_json()
|
||||
{
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test.json_each_row_nested"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE test.json_each_row_nested (d1 UInt8, d2 String, n Nested (s String, i Int32) ) ENGINE = Memory"
|
||||
|
||||
echo '{"d1" : 1, "d2" : "ok", "n" : { "s" : ["abc", "def"], "i" : [1, 23]} }
|
||||
{ }
|
||||
{"t1" : 0, "n.t2":true,"n" : {"i":[45, 67, 8], "s":["x", "y", "z"]}, "t5":[],"t6":"trash" }
|
||||
{"d2":"ok","n" : {"s":["dog", "cat", "pig"], "x":[["1","2"]], "i":[3, 3, 3]}, "d1":"1", "n.j":[4, 4, 4]}
|
||||
{"t0" : -0.1, "n": {"s" : ["zero","negative one"], "i" : [0, -1]}, "d2" : "ok", "d1" : 1}' \
|
||||
| $CLICKHOUSE_CLIENT "$@" --input_format_skip_unknown_fields=1 -q "INSERT INTO test.json_each_row_nested FORMAT JSONEachRow"
|
||||
|
||||
$CLICKHOUSE_CLIENT --max_threads=1 -q "SELECT * FROM test.json_each_row_nested"
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test.json_each_row_nested"
|
||||
}
|
||||
|
||||
test_nested_json
|
||||
test_nested_json --input_format_import_nested_json=1
|
@ -0,0 +1 @@
|
||||
3 3
|
10
dbms/tests/queries/0_stateless/00717_default_join_type.sql
Normal file
10
dbms/tests/queries/0_stateless/00717_default_join_type.sql
Normal file
@ -0,0 +1,10 @@
|
||||
DROP TABLE IF EXISTS test.default_join1;
|
||||
DROP TABLE IF EXISTS test.default_join2;
|
||||
|
||||
CREATE TABLE test.default_join1(a Int64, b Int64) ENGINE=Memory;
|
||||
CREATE TABLE test.default_join2(a Int64, b Int64) ENGINE=Memory;
|
||||
|
||||
INSERT INTO test.default_join1 VALUES(1, 1), (2, 2), (3, 3);
|
||||
INSERT INTO test.default_join2 VALUES(3, 3), (4, 4);
|
||||
|
||||
SELECT a, b FROM test.default_join1 JOIN (SELECT a, b FROM test.default_join2) USING a ORDER BY b SETTINGS join_default_strictness='ANY';
|
@ -0,0 +1,31 @@
|
||||
20
|
||||
02
|
||||
01/02/18
|
||||
2
|
||||
2018-01-02
|
||||
22
|
||||
02
|
||||
10
|
||||
11
|
||||
12
|
||||
001
|
||||
366
|
||||
01
|
||||
33
|
||||
\n
|
||||
AM
|
||||
AM
|
||||
PM
|
||||
22:33
|
||||
44
|
||||
\t
|
||||
22:33:44
|
||||
1 7
|
||||
01 01 53 52
|
||||
1 0
|
||||
18
|
||||
2018
|
||||
%
|
||||
no formatting pattern
|
||||
2018-01-01 00:00:00
|
||||
2018-01-01 01:00:00 2018-01-01 04:00:00
|
45
dbms/tests/queries/0_stateless/00718_format_datetime.sql
Normal file
45
dbms/tests/queries/0_stateless/00718_format_datetime.sql
Normal file
@ -0,0 +1,45 @@
|
||||
SET send_logs_level = 'none';
|
||||
|
||||
SELECT formatDateTime(); -- { serverError 42 }
|
||||
SELECT formatDateTime('not a datetime', 'IGNORED'); -- { serverError 43 }
|
||||
SELECT formatDateTime(now(), now()); -- { serverError 43 }
|
||||
SELECT formatDateTime(now(), 'good format pattern', now()); -- { serverError 43 }
|
||||
SELECT formatDateTime(now(), 'unescaped %'); -- { serverError 36 }
|
||||
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%U'); -- { serverError 48 }
|
||||
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%W'); -- { serverError 48 }
|
||||
|
||||
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%C');
|
||||
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%d');
|
||||
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%D');
|
||||
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%e');
|
||||
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%F');
|
||||
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%H');
|
||||
SELECT formatDateTime(toDateTime('2018-01-02 02:33:44'), '%H');
|
||||
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%I');
|
||||
SELECT formatDateTime(toDateTime('2018-01-02 11:33:44'), '%I');
|
||||
SELECT formatDateTime(toDateTime('2018-01-02 00:33:44'), '%I');
|
||||
SELECT formatDateTime(toDateTime('2018-01-01 00:33:44'), '%j');
|
||||
SELECT formatDateTime(toDateTime('2000-12-31 00:33:44'), '%j');
|
||||
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%m');
|
||||
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%M');
|
||||
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%n');
|
||||
SELECT formatDateTime(toDateTime('2018-01-02 00:33:44'), '%p');
|
||||
SELECT formatDateTime(toDateTime('2018-01-02 11:33:44'), '%p');
|
||||
SELECT formatDateTime(toDateTime('2018-01-02 12:33:44'), '%p');
|
||||
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%R');
|
||||
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%S');
|
||||
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%t');
|
||||
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%T');
|
||||
SELECT formatDateTime(toDateTime('2018-01-01 22:33:44'), '%u'), formatDateTime(toDateTime('2018-01-07 22:33:44'), '%u');
|
||||
SELECT formatDateTime(toDateTime('1996-01-01 22:33:44'), '%V'), formatDateTime(toDateTime('1996-12-31 22:33:44'), '%V'),
|
||||
formatDateTime(toDateTime('1999-01-01 22:33:44'), '%V'), formatDateTime(toDateTime('1999-12-31 22:33:44'), '%V');
|
||||
SELECT formatDateTime(toDateTime('2018-01-01 22:33:44'), '%w'), formatDateTime(toDateTime('2018-01-07 22:33:44'), '%w');
|
||||
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%y');
|
||||
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%Y');
|
||||
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%%');
|
||||
SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), 'no formatting pattern');
|
||||
|
||||
SELECT formatDateTime(toDate('2018-01-01'), '%F %T');
|
||||
SELECT
|
||||
formatDateTime(toDateTime('2018-01-01 01:00:00', 'UTC'), '%F %T', 'UTC'),
|
||||
formatDateTime(toDateTime('2018-01-01 01:00:00', 'UTC'), '%F %T', 'Europe/Moscow')
|
@ -0,0 +1,6 @@
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
@ -0,0 +1,7 @@
|
||||
WITH toDateTime(rand()) AS t SELECT count() FROM numbers(1000000) WHERE formatDateTime(t, '%F %T') != toString(t);
|
||||
WITH toDateTime(rand()) AS t SELECT count() FROM numbers(1000000) WHERE formatDateTime(t, '%Y-%m-%d %H:%M:%S') != toString(t);
|
||||
WITH toDateTime(rand()) AS t SELECT count() FROM numbers(1000000) WHERE formatDateTime(t, '%Y-%m-%d %R:%S') != toString(t);
|
||||
WITH toDateTime(rand()) AS t SELECT count() FROM numbers(1000000) WHERE formatDateTime(t, '%F %R:%S') != toString(t);
|
||||
|
||||
WITH toDate(today() + rand() % 4096) AS t SELECT count() FROM numbers(1000000) WHERE formatDateTime(t, '%F') != toString(t);
|
||||
WITH toDate(today() + rand() % 4096) AS t SELECT count() FROM numbers(1000000) WHERE formatDateTime(t, '%F %T') != toString(toDateTime(t));
|
@ -0,0 +1 @@
|
||||
Still alive
|
21
dbms/tests/queries/0_stateless/00719_insert_block_without_column.sh
Executable file
21
dbms/tests/queries/0_stateless/00719_insert_block_without_column.sh
Executable file
@ -0,0 +1,21 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
set -e
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
|
||||
[ -e ${CLICKHOUSE_TMP}/test_squashing_block_without_column.out ] && rm ${CLICKHOUSE_TMP}/test_squashing_block_without_column.out
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "select number as SomeID, number+1 as OtherID from system.numbers limit 1000 into outfile '${CLICKHOUSE_TMP}/test_squashing_block_without_column.out' format Native"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "drop table if exists test.squashed_numbers"
|
||||
${CLICKHOUSE_CLIENT} --query "create table test.squashed_numbers (SomeID UInt64, DifferentID UInt64, OtherID UInt64) engine Memory"
|
||||
|
||||
address=${CLICKHOUSE_HOST}
|
||||
port=${CLICKHOUSE_PORT_HTTP}
|
||||
url="${CLICKHOUSE_PORT_HTTP_PROTO}://$address:$port/"
|
||||
|
||||
${CLICKHOUSE_CURL} -sS --data-binary "@${CLICKHOUSE_TMP}/test_squashing_block_without_column.out" "${url}?query=insert%20into%20test.squashed_numbers%20format%20Native"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "select 'Still alive'"
|
40
dbms/tests/queries/0_stateless/00720_with_cube.reference
Normal file
40
dbms/tests/queries/0_stateless/00720_with_cube.reference
Normal file
@ -0,0 +1,40 @@
|
||||
0 120 8
|
||||
1 40 4
|
||||
2 80 4
|
||||
a 0 70 4
|
||||
a 1 25 2
|
||||
a 2 45 2
|
||||
b 0 50 4
|
||||
b 1 15 2
|
||||
b 2 35 2
|
||||
0 120 8
|
||||
1 40 4
|
||||
2 80 4
|
||||
a 0 70 4
|
||||
a 1 25 2
|
||||
a 2 45 2
|
||||
b 0 50 4
|
||||
b 1 15 2
|
||||
b 2 35 2
|
||||
|
||||
0 120 8
|
||||
1 40 4
|
||||
0 120 8
|
||||
2 80 4
|
||||
a 0 70 4
|
||||
a 1 25 2
|
||||
a 2 45 2
|
||||
b 0 50 4
|
||||
b 1 15 2
|
||||
b 2 35 2
|
||||
1 40 4
|
||||
0 120 8
|
||||
2 80 4
|
||||
a 0 70 4
|
||||
a 1 25 2
|
||||
a 2 45 2
|
||||
b 0 50 4
|
||||
b 1 15 2
|
||||
b 2 35 2
|
||||
|
||||
0 120 8
|
19
dbms/tests/queries/0_stateless/00720_with_cube.sql
Normal file
19
dbms/tests/queries/0_stateless/00720_with_cube.sql
Normal file
@ -0,0 +1,19 @@
|
||||
DROP TABLE IF EXISTS test.rollup;
|
||||
CREATE TABLE test.rollup(a String, b Int32, s Int32) ENGINE = Memory;
|
||||
|
||||
INSERT INTO test.rollup VALUES('a', 1, 10);
|
||||
INSERT INTO test.rollup VALUES('a', 1, 15);
|
||||
INSERT INTO test.rollup VALUES('a', 2, 20);
|
||||
INSERT INTO test.rollup VALUES('a', 2, 25);
|
||||
INSERT INTO test.rollup VALUES('b', 1, 10);
|
||||
INSERT INTO test.rollup VALUES('b', 1, 5);
|
||||
INSERT INTO test.rollup VALUES('b', 2, 20);
|
||||
INSERT INTO test.rollup VALUES('b', 2, 15);
|
||||
|
||||
SELECT a, b, sum(s), count() from test.rollup GROUP BY CUBE(a, b) ORDER BY a, b;
|
||||
|
||||
SELECT a, b, sum(s), count() from test.rollup GROUP BY CUBE(a, b) WITH TOTALS ORDER BY a, b;
|
||||
|
||||
SELECT a, b, sum(s), count() from test.rollup GROUP BY a, b WITH CUBE ORDER BY a;
|
||||
|
||||
SELECT a, b, sum(s), count() from test.rollup GROUP BY a, b WITH CUBE WITH TOTALS ORDER BY a;
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user