Merge branch 'master' of github.com:yandex/ClickHouse

This commit is contained in:
Alexey Milovidov 2018-09-20 23:21:37 +03:00
commit 47934d1cce
120 changed files with 2126 additions and 503 deletions

View File

@ -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)

View File

@ -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 "")

View File

@ -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__; }

View 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));

View File

@ -2,7 +2,6 @@
#include <Core/Field.h>
#include <Core/AccurateComparison.h>
#include <common/DateLUT.h>
#include <common/demangle.h>

View File

@ -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, "") \

View File

@ -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');
}

View File

@ -908,20 +908,33 @@ void ZooKeeper::connect(
in.emplace(socket);
out.emplace(socket);
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)

View File

@ -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);
}

View File

@ -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;
};

View 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;
}
}

View 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;
};
}

View File

@ -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,

View File

@ -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(); }

View File

@ -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)
{
}

View File

@ -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;

View File

@ -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);

View File

@ -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;

View File

@ -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);

View File

@ -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
{

View File

@ -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;

View File

@ -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);

View File

@ -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;

View File

@ -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);

View 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;
};
}

View File

@ -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>);

View File

@ -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
@ -27,14 +27,7 @@ public:
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 deserializeText(IColumn &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
MutableColumnPtr createColumn() const override
{

View File

@ -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();

View File

@ -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;
}
}

View File

@ -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;

View File

@ -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;

View File

@ -49,6 +49,7 @@ struct FormatSettings
bool skip_unknown_fields = false;
bool write_statistics = true;
bool import_nested_json = false;
enum class DateTimeInputFormat
{

View File

@ -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)

View File

@ -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>;

View File

@ -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;
});

View File

@ -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>();
}
}

View File

@ -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>;

View File

@ -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))
{

View File

@ -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;
}
}
}

View File

@ -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>

View File

@ -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); }

View File

@ -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
*/

View File

@ -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;

View File

@ -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.

View File

@ -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)))
{

View File

@ -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);
}
}
}

View File

@ -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);
}
}

View File

@ -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)
{
/// 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);

View File

@ -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);
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());
if (modificator == Modificator::ROLLUP)
pipeline.firstStream() = std::make_shared<RollupBlockInputStream>(pipeline.firstStream(), params);
else
pipeline.firstStream() = std::make_shared<CubeBlockInputStream>(pipeline.firstStream(), params);
}

View File

@ -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.
*

View File

@ -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();
}
}
}

View File

@ -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.") \
\

View File

@ -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));
}
}
}

View File

@ -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;
};
}

View File

@ -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);
}
}

View File

@ -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;

View File

@ -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);
}

View File

@ -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))
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

View File

@ -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

View File

@ -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;

View File

@ -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)

View File

@ -1,8 +1,6 @@
#pragma once
#include <Storages/MergeTree/MergeTreePartInfo.h>
#include <mutex>
#include <common/DateLUT.h>
#include <Core/Types.h>
#include <map>

View File

@ -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;

View File

@ -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);
}

View File

@ -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

View File

@ -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;

View File

@ -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};

View File

@ -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(

View File

@ -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)
{

View File

@ -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;

View File

@ -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())

View File

@ -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();

View File

@ -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>

View 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>

View File

@ -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.-------

View File

@ -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;

View File

@ -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

View File

@ -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;

View File

@ -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);

View File

@ -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

View 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;

View File

@ -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

View File

@ -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;

View File

@ -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]

View File

@ -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

View File

@ -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);

View File

@ -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

View 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;

View File

@ -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]

View 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

View File

@ -0,0 +1 @@
3 3

View 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';

View File

@ -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

View 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')

View File

@ -0,0 +1,6 @@
0
0
0
0
0
0

View File

@ -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));

View File

@ -0,0 +1 @@
Still alive

View 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'"

View 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

View 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