diff --git a/README.md b/README.md index a39ebef04a3..b555abdcc1b 100644 --- a/README.md +++ b/README.md @@ -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) diff --git a/dbms/cmake/version.cmake b/dbms/cmake/version.cmake index f1bd9fb2e39..3a796bf235f 100644 --- a/dbms/cmake/version.cmake +++ b/dbms/cmake/version.cmake @@ -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 "") diff --git a/dbms/src/AggregateFunctions/AggregateFunctionAvg.h b/dbms/src/AggregateFunctions/AggregateFunctionAvg.h index 1c60f5f68d9..9cc2d8e26ed 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionAvg.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionAvg.h @@ -13,12 +13,28 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} template struct AggregateFunctionAvgData { T sum = 0; UInt64 count = 0; + + template + ResultT result() const + { + if constexpr (std::is_floating_point_v) + if constexpr (std::numeric_limits::is_iec559) + return static_cast(sum) / count; /// allow division by zero + + if (!count) + throw Exception("AggregateFunctionAvg with zero values", ErrorCodes::LOGICAL_ERROR); + return static_cast(sum) / count; + } }; @@ -80,7 +96,7 @@ public: void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { auto & column = static_cast(to); - column.getData().push_back(static_cast(this->data(place).sum) / this->data(place).count); + column.getData().push_back(this->data(place).template result()); } const char * getHeaderFilePath() const override { return __FILE__; } diff --git a/dbms/src/Analyzers/AnalyzeColumns.cpp b/dbms/src/Analyzers/AnalyzeColumns.cpp index 73760c4ff01..93b43157664 100644 --- a/dbms/src/Analyzers/AnalyzeColumns.cpp +++ b/dbms/src/Analyzers/AnalyzeColumns.cpp @@ -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(table.database_name, ASTIdentifier::Column); + database_name_identifier_node = std::make_shared(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(table_name_or_alias, ASTIdentifier::Column); + table_name_identifier_node = std::make_shared(table_name_or_alias); - ASTPtr column_identifier_node = std::make_shared(column, ASTIdentifier::Column); + ASTPtr column_identifier_node = std::make_shared(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(compound_name, ASTIdentifier::Column); + auto elem = std::make_shared(compound_name); if (database_name_identifier_node) elem->children.emplace_back(std::move(database_name_identifier_node)); diff --git a/dbms/src/Common/FieldVisitors.h b/dbms/src/Common/FieldVisitors.h index 06d561b6a4a..6b5c0730f93 100644 --- a/dbms/src/Common/FieldVisitors.h +++ b/dbms/src/Common/FieldVisitors.h @@ -2,7 +2,6 @@ #include #include -#include #include diff --git a/dbms/src/Common/ProfileEvents.cpp b/dbms/src/Common/ProfileEvents.cpp index 8708dd34dcc..de50c625f15 100644 --- a/dbms/src/Common/ProfileEvents.cpp +++ b/dbms/src/Common/ProfileEvents.cpp @@ -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, "") \ diff --git a/dbms/src/Common/StringUtils/StringUtils.h b/dbms/src/Common/StringUtils/StringUtils.h index ffb4c0d85c1..db3d5102f76 100644 --- a/dbms/src/Common/StringUtils/StringUtils.h +++ b/dbms/src/Common/StringUtils/StringUtils.h @@ -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'); } diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 6f218b87bd0..b47452cedb6 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -908,20 +908,33 @@ void ZooKeeper::connect( in.emplace(socket); out.emplace(socket); - sendHandshake(); - receiveHandshake(); + try + { + sendHandshake(); + } + catch (DB::Exception & e) + { + e.addMessage("while sending handshake to ZooKeeper"); + throw; + } + + try + { + receiveHandshake(); + } + catch (DB::Exception & e) + { + e.addMessage("while receiving handshake from ZooKeeper"); + throw; + } connected = true; break; } - catch (const Poco::Net::NetException &) + catch (...) { fail_reasons << "\n" << getCurrentExceptionMessage(false) << ", " << address.toString(); } - catch (const Poco::TimeoutException &) - { - fail_reasons << "\n" << getCurrentExceptionMessage(false); - } } if (connected) diff --git a/dbms/src/DataStreams/AddingDefaultBlockOutputStream.cpp b/dbms/src/DataStreams/AddingDefaultBlockOutputStream.cpp index fe773b40776..4b3f216e182 100644 --- a/dbms/src/DataStreams/AddingDefaultBlockOutputStream.cpp +++ b/dbms/src/DataStreams/AddingDefaultBlockOutputStream.cpp @@ -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 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(&*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(*column_to_add.type).getNestedType(); + DataTypePtr nested_type = typeid_cast(*column.type).getNestedType(); UInt64 nested_rows = rows ? get((*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); } diff --git a/dbms/src/DataStreams/AddingDefaultBlockOutputStream.h b/dbms/src/DataStreams/AddingDefaultBlockOutputStream.h index b36aaee501f..e04fdc0faeb 100644 --- a/dbms/src/DataStreams/AddingDefaultBlockOutputStream.h +++ b/dbms/src/DataStreams/AddingDefaultBlockOutputStream.h @@ -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; }; diff --git a/dbms/src/DataStreams/CubeBlockInputStream.cpp b/dbms/src/DataStreams/CubeBlockInputStream.cpp new file mode 100644 index 00000000000..ecada4534d1 --- /dev/null +++ b/dbms/src/DataStreams/CubeBlockInputStream.cpp @@ -0,0 +1,78 @@ +#include +#include +#include +#include +#include +#include + +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; +} +} diff --git a/dbms/src/DataStreams/CubeBlockInputStream.h b/dbms/src/DataStreams/CubeBlockInputStream.h new file mode 100644 index 00000000000..46f1a5bb432 --- /dev/null +++ b/dbms/src/DataStreams/CubeBlockInputStream.h @@ -0,0 +1,42 @@ +#pragma once + +#include +#include +#include +#include + + +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; + using AggregateColumns = std::vector; +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; +}; + +} diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 7034aaf66fa..8216d53d197 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -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, diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h index 2166ee4339b..0de1d1e2ee4 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h @@ -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(); } diff --git a/dbms/src/DataStreams/SquashingBlockOutputStream.cpp b/dbms/src/DataStreams/SquashingBlockOutputStream.cpp index b811dc3e5d1..5698ee99f64 100644 --- a/dbms/src/DataStreams/SquashingBlockOutputStream.cpp +++ b/dbms/src/DataStreams/SquashingBlockOutputStream.cpp @@ -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) { } diff --git a/dbms/src/DataStreams/SquashingBlockOutputStream.h b/dbms/src/DataStreams/SquashingBlockOutputStream.h index bd19c5e2cdc..153f671a600 100644 --- a/dbms/src/DataStreams/SquashingBlockOutputStream.h +++ b/dbms/src/DataStreams/SquashingBlockOutputStream.h @@ -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; diff --git a/dbms/src/DataTypes/DataTypeArray.cpp b/dbms/src/DataTypes/DataTypeArray.cpp index 9a533424198..de0b0d91276 100644 --- a/dbms/src/DataTypes/DataTypeArray.cpp +++ b/dbms/src/DataTypes/DataTypeArray.cpp @@ -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(column); diff --git a/dbms/src/DataTypes/DataTypeArray.h b/dbms/src/DataTypes/DataTypeArray.h index c13ebb9cd98..5a89a5d8e5a 100644 --- a/dbms/src/DataTypes/DataTypeArray.h +++ b/dbms/src/DataTypes/DataTypeArray.h @@ -1,13 +1,13 @@ #pragma once -#include +#include 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; diff --git a/dbms/src/DataTypes/DataTypeDate.cpp b/dbms/src/DataTypes/DataTypeDate.cpp index 810a16f0c5d..688e3c91db3 100644 --- a/dbms/src/DataTypes/DataTypeDate.cpp +++ b/dbms/src/DataTypes/DataTypeDate.cpp @@ -14,7 +14,7 @@ void DataTypeDate::serializeText(const IColumn & column, size_t row_num, WriteBu writeDateText(DayNum(static_cast(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); diff --git a/dbms/src/DataTypes/DataTypeNumberBase.cpp b/dbms/src/DataTypes/DataTypeNumberBase.cpp index 2db7713c545..5f598a09920 100644 --- a/dbms/src/DataTypes/DataTypeNumberBase.cpp +++ b/dbms/src/DataTypes/DataTypeNumberBase.cpp @@ -19,14 +19,7 @@ void DataTypeNumberBase::serializeText(const IColumn & column, size_t row_num } template -void DataTypeNumberBase::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const -{ - serializeText(column, row_num, ostr, settings); -} - - -template -static void deserializeText(IColumn & column, ReadBuffer & istr) +void DataTypeNumberBase::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { T x; @@ -38,26 +31,6 @@ static void deserializeText(IColumn & column, ReadBuffer & istr) static_cast &>(column).getData().push_back(x); } - -template -void DataTypeNumberBase::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const -{ - deserializeText(column, istr); -} - -template -void DataTypeNumberBase::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const -{ - serializeText(column, row_num, ostr, settings); -} - -template -void DataTypeNumberBase::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const -{ - deserializeText(column, istr); -} - - template static inline void writeDenormalNumber(T x, WriteBuffer & ostr) { @@ -160,12 +133,6 @@ void DataTypeNumberBase::deserializeTextJSON(IColumn & column, ReadBuffer & i static_cast &>(column).getData().push_back(x); } -template -void DataTypeNumberBase::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const -{ - serializeText(column, row_num, ostr, settings); -} - template void DataTypeNumberBase::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { diff --git a/dbms/src/DataTypes/DataTypeNumberBase.h b/dbms/src/DataTypes/DataTypeNumberBase.h index 1b1b2c39851..2aac709090c 100644 --- a/dbms/src/DataTypes/DataTypeNumberBase.h +++ b/dbms/src/DataTypes/DataTypeNumberBase.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB @@ -9,7 +10,7 @@ namespace DB /** Implements part of the IDataType interface, common to all numbers and for Date and DateTime. */ template -class DataTypeNumberBase : public IDataType +class DataTypeNumberBase : public DataTypeWithSimpleSerialization { static_assert(IsNumber); @@ -21,13 +22,9 @@ public: TypeIndex getTypeId() const override { return TypeId::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; diff --git a/dbms/src/DataTypes/DataTypeTuple.cpp b/dbms/src/DataTypes/DataTypeTuple.cpp index c74b7c9191f..35f7ea3365f 100644 --- a/dbms/src/DataTypes/DataTypeTuple.cpp +++ b/dbms/src/DataTypes/DataTypeTuple.cpp @@ -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); diff --git a/dbms/src/DataTypes/DataTypeTuple.h b/dbms/src/DataTypes/DataTypeTuple.h index 02fdda3cd17..cded65246ac 100644 --- a/dbms/src/DataTypes/DataTypeTuple.h +++ b/dbms/src/DataTypes/DataTypeTuple.h @@ -1,6 +1,6 @@ #pragma once -#include +#include 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; diff --git a/dbms/src/DataTypes/DataTypeUUID.cpp b/dbms/src/DataTypes/DataTypeUUID.cpp index 664f5f279c6..8e022aaa6a0 100644 --- a/dbms/src/DataTypes/DataTypeUUID.cpp +++ b/dbms/src/DataTypes/DataTypeUUID.cpp @@ -13,7 +13,7 @@ void DataTypeUUID::serializeText(const IColumn & column, size_t row_num, WriteBu writeText(UUID(static_cast(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); diff --git a/dbms/src/DataTypes/DataTypeWithSimpleSerialization.h b/dbms/src/DataTypes/DataTypeWithSimpleSerialization.h new file mode 100644 index 00000000000..8f897153fd0 --- /dev/null +++ b/dbms/src/DataTypes/DataTypeWithSimpleSerialization.h @@ -0,0 +1,58 @@ +#pragma once +#include + +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; +}; + +} diff --git a/dbms/src/DataTypes/DataTypesDecimal.h b/dbms/src/DataTypes/DataTypesDecimal.h index f90add26eae..8b65fbfe388 100644 --- a/dbms/src/DataTypes/DataTypesDecimal.h +++ b/dbms/src/DataTypes/DataTypesDecimal.h @@ -4,6 +4,7 @@ #include #include #include +#include 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 -class DataTypeDecimal final : public DataTypeSimpleSerialization +class DataTypeDecimal final : public DataTypeWithSimpleSerialization { static_assert(IsDecimalNumber); diff --git a/dbms/src/DataTypes/IDataTypeDummy.h b/dbms/src/DataTypes/IDataTypeDummy.h index 20016b57674..8f204b0e67c 100644 --- a/dbms/src/DataTypes/IDataTypeDummy.h +++ b/dbms/src/DataTypes/IDataTypeDummy.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace DB @@ -11,7 +11,7 @@ namespace DB * * That is, this class is used just to distinguish the corresponding data type from the others. */ -class IDataTypeDummy : public IDataType +class IDataTypeDummy : public DataTypeWithSimpleSerialization { private: void throwNoSerialization() const @@ -26,15 +26,8 @@ public: void deserializeBinary(IColumn &, ReadBuffer &) const override { throwNoSerialization(); } void serializeBinaryBulk(const IColumn &, WriteBuffer &, size_t, size_t) const override { throwNoSerialization(); } void deserializeBinaryBulk(IColumn &, ReadBuffer &, size_t, double) const override { throwNoSerialization(); } - void serializeText(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); } - void serializeTextEscaped(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); } - void deserializeTextEscaped(IColumn &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); } - void serializeTextQuoted(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); } - void deserializeTextQuoted(IColumn &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); } - void serializeTextJSON(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); } - void deserializeTextJSON(IColumn &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); } - void serializeTextCSV(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); } - void deserializeTextCSV(IColumn &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); } + void serializeText(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); } + void deserializeText(IColumn &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); } MutableColumnPtr createColumn() const override { diff --git a/dbms/src/Dictionaries/RangeDictionaryBlockInputStream.h b/dbms/src/Dictionaries/RangeDictionaryBlockInputStream.h index beef86cc9a5..93c937e7140 100644 --- a/dbms/src/Dictionaries/RangeDictionaryBlockInputStream.h +++ b/dbms/src/Dictionaries/RangeDictionaryBlockInputStream.h @@ -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 +template class RangeDictionaryBlockInputStream : public DictionaryBlockInputStreamBase { public: @@ -26,7 +26,7 @@ public: RangeDictionaryBlockInputStream( DictionaryPtr dictionary, size_t max_block_size, const Names & column_names, PaddedPODArray && ids_to_fill, - PaddedPODArray && start_dates, PaddedPODArray && end_dates); + PaddedPODArray && start_dates, PaddedPODArray && end_dates); String getName() const override { @@ -57,35 +57,35 @@ private: const PaddedPODArray & values, ColumnsWithTypeAndName & columns) const; Block fillBlock(const PaddedPODArray & ids_to_fill, - const PaddedPODArray & block_start_dates, const PaddedPODArray & block_end_dates) const; + const PaddedPODArray & block_start_dates, const PaddedPODArray & block_end_dates) const; PaddedPODArray makeDateKey( - const PaddedPODArray & block_start_dates, const PaddedPODArray & block_end_dates) const; + const PaddedPODArray & block_start_dates, const PaddedPODArray & block_end_dates) const; DictionaryPtr dictionary; Names column_names; PaddedPODArray ids; - PaddedPODArray start_dates; - PaddedPODArray end_dates; + PaddedPODArray start_dates; + PaddedPODArray end_dates; }; -template -RangeDictionaryBlockInputStream::RangeDictionaryBlockInputStream( +template +RangeDictionaryBlockInputStream::RangeDictionaryBlockInputStream( DictionaryPtr dictionary, size_t max_column_size, const Names & column_names, PaddedPODArray && ids, - PaddedPODArray && block_start_dates, PaddedPODArray && block_end_dates) + PaddedPODArray && block_start_dates, PaddedPODArray && 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 -Block RangeDictionaryBlockInputStream::getBlock(size_t start, size_t length) const +template +Block RangeDictionaryBlockInputStream::getBlock(size_t start, size_t length) const { PaddedPODArray block_ids; - PaddedPODArray block_start_dates; - PaddedPODArray block_end_dates; + PaddedPODArray block_start_dates; + PaddedPODArray block_end_dates; block_ids.reserve(length); block_start_dates.reserve(length); block_end_dates.reserve(length); @@ -100,9 +100,9 @@ Block RangeDictionaryBlockInputStream::getBlock(size_t star return fillBlock(block_ids, block_start_dates, block_end_dates); } -template +template template -ColumnPtr RangeDictionaryBlockInputStream::getColumnFromAttribute( +ColumnPtr RangeDictionaryBlockInputStream::getColumnFromAttribute( DictionaryGetter getter, const PaddedPODArray & ids_to_fill, const PaddedPODArray & dates, const DictionaryAttribute & attribute, const DictionaryType & concrete_dictionary) const { @@ -111,8 +111,8 @@ ColumnPtr RangeDictionaryBlockInputStream::getColumnFromAtt return column_vector; } -template -ColumnPtr RangeDictionaryBlockInputStream::getColumnFromAttributeString( +template +ColumnPtr RangeDictionaryBlockInputStream::getColumnFromAttributeString( const PaddedPODArray & ids_to_fill, const PaddedPODArray & dates, const DictionaryAttribute & attribute, const DictionaryType & concrete_dictionary) const { @@ -121,9 +121,9 @@ ColumnPtr RangeDictionaryBlockInputStream::getColumnFromAtt return column_string; } -template +template template -ColumnPtr RangeDictionaryBlockInputStream::getColumnFromPODArray(const PaddedPODArray & array) const +ColumnPtr RangeDictionaryBlockInputStream::getColumnFromPODArray(const PaddedPODArray & array) const { auto column_vector = ColumnVector::create(); column_vector->getData().reserve(array.size()); @@ -133,9 +133,9 @@ ColumnPtr RangeDictionaryBlockInputStream::getColumnFromPOD } -template +template template -void RangeDictionaryBlockInputStream::addSpecialColumn( +void RangeDictionaryBlockInputStream::addSpecialColumn( const std::optional & attribute, DataTypePtr type, const std::string & default_name, const std::unordered_set & column_names_set, const PaddedPODArray & values, ColumnsWithTypeAndName & columns) const @@ -148,9 +148,9 @@ void RangeDictionaryBlockInputStream::addSpecialColumn( columns.emplace_back(getColumnFromPODArray(values), type, name); } -template -PaddedPODArray RangeDictionaryBlockInputStream::makeDateKey( - const PaddedPODArray & block_start_dates, const PaddedPODArray & block_end_dates) const +template +PaddedPODArray RangeDictionaryBlockInputStream::makeDateKey( + const PaddedPODArray & block_start_dates, const PaddedPODArray & block_end_dates) const { PaddedPODArray key(block_start_dates.size()); for (size_t i = 0; i < key.size(); ++i) @@ -165,10 +165,10 @@ PaddedPODArray RangeDictionaryBlockInputStream::make } -template -Block RangeDictionaryBlockInputStream::fillBlock( +template +Block RangeDictionaryBlockInputStream::fillBlock( const PaddedPODArray & ids_to_fill, - const PaddedPODArray & block_start_dates, const PaddedPODArray & block_end_dates) const + const PaddedPODArray & block_start_dates, const PaddedPODArray & block_end_dates) const { ColumnsWithTypeAndName columns; const DictionaryStructure & structure = dictionary->getStructure(); diff --git a/dbms/src/Dictionaries/RangeHashedDictionary.cpp b/dbms/src/Dictionaries/RangeHashedDictionary.cpp index e96e83fc078..57a948f1037 100644 --- a/dbms/src/Dictionaries/RangeHashedDictionary.cpp +++ b/dbms/src/Dictionaries/RangeHashedDictionary.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include @@ -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::max(); -const RangeStorageType RANGE_MAX_NULL_VALUE = std::numeric_limits::min(); +const RangeStorageType RANGE_MAX_NULL_VALUE = std::numeric_limits::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 void RangeHashedDictionary::getIdsAndDates(PaddedPODArray & ids, - PaddedPODArray & start_dates, - PaddedPODArray & end_dates) const + PaddedPODArray & start_dates, + PaddedPODArray & end_dates) const { const auto & attribute = attributes.front(); @@ -439,10 +452,10 @@ void RangeHashedDictionary::getIdsAndDates(PaddedPODArray & ids, } } -template +template void RangeHashedDictionary::getIdsAndDates(const Attribute & attribute, PaddedPODArray & ids, - PaddedPODArray & start_dates, - PaddedPODArray & end_dates) const + PaddedPODArray & start_dates, + PaddedPODArray & end_dates) const { const HashMap> & attr = *std::get>(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(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 +BlockInputStreamPtr RangeHashedDictionary::getBlockInputStreamImpl(const Names & column_names, size_t max_block_size) const { PaddedPODArray ids; - PaddedPODArray start_dates; - PaddedPODArray end_dates; + PaddedPODArray start_dates; + PaddedPODArray end_dates; getIdsAndDates(ids, start_dates, end_dates); - using BlockInputStreamType = RangeDictionaryBlockInputStream; + using BlockInputStreamType = RangeDictionaryBlockInputStream; auto dict_ptr = std::static_pointer_cast(shared_from_this()); return std::make_shared( 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 + void operator()() + { + auto & type = dict->dict_struct.range_min->type; + if (!stream && dynamic_cast *>(type.get())) + stream = dict->getBlockInputStreamImpl(*column_names, max_block_size); + } +}; + +BlockInputStreamPtr RangeHashedDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const +{ + using ListType = TypeList; + + 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; +} } diff --git a/dbms/src/Dictionaries/RangeHashedDictionary.h b/dbms/src/Dictionaries/RangeHashedDictionary.h index 73cb7ff4610..23153afd12c 100644 --- a/dbms/src/Dictionaries/RangeHashedDictionary.h +++ b/dbms/src/Dictionaries/RangeHashedDictionary.h @@ -162,12 +162,18 @@ private: const Attribute & getAttributeWithType(const std::string & name, const AttributeUnderlyingType type) const; + template void getIdsAndDates(PaddedPODArray & ids, - PaddedPODArray & start_dates, PaddedPODArray & end_dates) const; + PaddedPODArray & start_dates, PaddedPODArray & end_dates) const; - template + template void getIdsAndDates(const Attribute & attribute, PaddedPODArray & ids, - PaddedPODArray & start_dates, PaddedPODArray & end_dates) const; + PaddedPODArray & start_dates, PaddedPODArray & end_dates) const; + + template + BlockInputStreamPtr getBlockInputStreamImpl(const Names & column_names, size_t max_block_size) const; + + friend struct RangeHashedDIctionaryCallGetBlockInputStreamImpl; const std::string dictionary_name; const DictionaryStructure dict_struct; diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index 45a7a78d828..60672817db5 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -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; diff --git a/dbms/src/Formats/FormatSettings.h b/dbms/src/Formats/FormatSettings.h index 2f47914b052..55996c3c7a0 100644 --- a/dbms/src/Formats/FormatSettings.h +++ b/dbms/src/Formats/FormatSettings.h @@ -49,6 +49,7 @@ struct FormatSettings bool skip_unknown_fields = false; bool write_statistics = true; + bool import_nested_json = false; enum class DateTimeInputFormat { diff --git a/dbms/src/Formats/JSONEachRowRowInputStream.cpp b/dbms/src/Formats/JSONEachRowRowInputStream.cpp index 01ddfa96df9..8da319be00b 100644 --- a/dbms/src/Formats/JSONEachRowRowInputStream.cpp +++ b/dbms/src/Formats/JSONEachRowRowInputStream.cpp @@ -3,7 +3,7 @@ #include #include #include - +#include 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) diff --git a/dbms/src/Formats/JSONEachRowRowInputStream.h b/dbms/src/Formats/JSONEachRowRowInputStream.h index 737811be51c..33b745082f8 100644 --- a/dbms/src/Formats/JSONEachRowRowInputStream.h +++ b/dbms/src/Formats/JSONEachRowRowInputStream.h @@ -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 read_columns; /// Hash table match `field name -> position in the block`. NOTE You can use perfect hash map. using NameMap = HashMap; diff --git a/dbms/src/Formats/PrettyCompactBlockOutputStream.cpp b/dbms/src/Formats/PrettyCompactBlockOutputStream.cpp index 01805a12df7..b4085fb256b 100644 --- a/dbms/src/Formats/PrettyCompactBlockOutputStream.cpp +++ b/dbms/src/Formats/PrettyCompactBlockOutputStream.cpp @@ -152,7 +152,7 @@ void registerOutputFormatPrettyCompact(FormatFactory & factory) const FormatSettings & format_settings) { BlockOutputStreamPtr impl = std::make_shared(buf, sample, format_settings); - auto res = std::make_shared(impl, format_settings.pretty.max_rows, 0); + auto res = std::make_shared(impl, impl->getHeader(), format_settings.pretty.max_rows, 0); res->disableFlush(); return res; }); diff --git a/dbms/src/Functions/FunctionsDateTime.cpp b/dbms/src/Functions/FunctionsDateTime.cpp index 6196d8da2e2..f89d82b241e 100644 --- a/dbms/src/Functions/FunctionsDateTime.cpp +++ b/dbms/src/Functions/FunctionsDateTime.cpp @@ -62,11 +62,14 @@ void registerFunctionsDateTime(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); @@ -74,6 +77,7 @@ void registerFunctionsDateTime(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); @@ -111,6 +115,9 @@ void registerFunctionsDateTime(FunctionFactory & factory) factory.registerFunction(FunctionFactory::CaseInsensitive); factory.registerFunction(); + + + factory.registerFunction(); } } diff --git a/dbms/src/Functions/FunctionsDateTime.h b/dbms/src/Functions/FunctionsDateTime.h index e83dcb60929..ffeaa237388 100644 --- a/dbms/src/Functions/FunctionsDateTime.h +++ b/dbms/src/Functions/FunctionsDateTime.h @@ -20,6 +20,7 @@ #include #include +#include #include @@ -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 + 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 + 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 + static inline void writeNumber3(char * p, T v) + { + writeNumber2(p, v / 10); + p[2] += v % 10; + } + + template + 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(); } + + 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(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + if (!executeType(block, arguments, result) + && !executeType(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 + bool executeType(Block & block, const ColumnNumbers & arguments, size_t result) + { + if (auto * times = checkAndGetColumn>(block.getByPosition(arguments[0]).column.get())) + { + const ColumnConst * pattern_column = checkAndGetColumnConst(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(); + + std::vector> 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::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(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 + String parsePattern(const String & pattern, std::vector> & 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::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::Func func [[maybe_unused]], size_t shift) + { + if constexpr (std::is_same_v) + 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::century, 2); + result.append("00"); + break; + + // Day of month, zero-padded (01-31) + case 'd': + instructions.emplace_back(&Action::dayOfMonth, 2); + result.append("00"); + break; + + // Short MM/DD/YY date, equivalent to %m/%d/%y + case 'D': + instructions.emplace_back(&Action::americanDate, 8); + result.append("00/00/00"); + break; + + // Day of month, space-padded ( 1-31) 23 + case 'e': + instructions.emplace_back(&Action::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::ISO8601Date, 10); + result.append("0000-00-00"); + break; + + // Day of the year (001-366) 235 + case 'j': + instructions.emplace_back(&Action::dayOfYear, 3); + result.append("000"); + break; + + // Month as a decimal number (01-12) + case 'm': + instructions.emplace_back(&Action::month, 2); + result.append("00"); + break; + + // ISO 8601 weekday as number with Monday as 1 (1-7) + case 'u': + instructions.emplace_back(&Action::dayOfWeek, 1); + result.append("0"); + break; + + // ISO 8601 week number (01-53) + case 'V': + instructions.emplace_back(&Action::ISO8601Week, 2); + result.append("00"); + break; + + // Weekday as a decimal number with Sunday as 0 (0-6) 4 + case 'w': + instructions.emplace_back(&Action::dayOfWeek0To6, 1); + result.append("0"); + break; + + // Two digits year + case 'y': + instructions.emplace_back(&Action::year2, 2); + result.append("00"); + break; + + // Four digits year + case 'Y': + instructions.emplace_back(&Action::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::minute, 2); + result.append("00"); + break; + + // AM or PM + case 'p': + addInstructionOrShift(&Action::AMPM, 2); + result.append("AM"); + break; + + // 24-hour HH:MM time, equivalent to %H:%M 14:55 + case 'R': + addInstructionOrShift(&Action::hhmm24, 5); + result.append("00:00"); + break; + + // Seconds + case 'S': + addInstructionOrShift(&Action::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::ISO8601Time, 8); + result.append("00:00:00"); + break; + + // Hour in 24h format (00-23) + case 'H': + addInstructionOrShift(&Action::hour24, 2); + result.append("00"); + break; + + // Hour in 12h format (01-12) + case 'I': + addInstructionOrShift(&Action::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; using FunctionToQuarter = FunctionDateOrDateTimeToSomething; using FunctionToMonth = FunctionDateOrDateTimeToSomething; using FunctionToDayOfMonth = FunctionDateOrDateTimeToSomething; using FunctionToDayOfWeek = FunctionDateOrDateTimeToSomething; +using FunctionToDayOfYear = FunctionDateOrDateTimeToSomething; using FunctionToHour = FunctionDateOrDateTimeToSomething; using FunctionToMinute = FunctionDateOrDateTimeToSomething; using FunctionToSecond = FunctionDateOrDateTimeToSomething; using FunctionToStartOfDay = FunctionDateOrDateTimeToSomething; using FunctionToMonday = FunctionDateOrDateTimeToSomething; +using FunctionToISOWeek = FunctionDateOrDateTimeToSomething; +using FunctionToISOYear = FunctionDateOrDateTimeToSomething; using FunctionToStartOfMonth = FunctionDateOrDateTimeToSomething; using FunctionToStartOfQuarter = FunctionDateOrDateTimeToSomething; using FunctionToStartOfYear = FunctionDateOrDateTimeToSomething; @@ -1524,6 +2131,7 @@ using FunctionToStartOfMinute = FunctionDateOrDateTimeToSomething; using FunctionToStartOfFifteenMinutes = FunctionDateOrDateTimeToSomething; using FunctionToStartOfHour = FunctionDateOrDateTimeToSomething; +using FunctionToStartOfISOYear = FunctionDateOrDateTimeToSomething; using FunctionToTime = FunctionDateOrDateTimeToSomething; using FunctionToRelativeYearNum = FunctionDateOrDateTimeToSomething; diff --git a/dbms/src/Functions/FunctionsExternalDictionaries.h b/dbms/src/Functions/FunctionsExternalDictionaries.h index d70d1de0318..fceb68be0ee 100644 --- a/dbms/src/Functions/FunctionsExternalDictionaries.h +++ b/dbms/src/Functions/FunctionsExternalDictionaries.h @@ -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 -const PaddedPODArray & getColumnDataAsPaddedPODArray(const IColumn & column, PaddedPODArray & backup_storage); +static const PaddedPODArray & getColumnDataAsPaddedPODArray(const IColumn & column, PaddedPODArray & backup_storage); class FunctionDictGetString final : public IFunction @@ -1459,7 +1459,7 @@ private: template -const PaddedPODArray & getColumnDataAsPaddedPODArray(const IColumn & column, PaddedPODArray & backup_storage) +static const PaddedPODArray & getColumnDataAsPaddedPODArray(const IColumn & column, PaddedPODArray & backup_storage) { if (const auto vector_col = checkAndGetColumn>(&column)) { diff --git a/dbms/src/Functions/arrayCumSumNonNegative.cpp b/dbms/src/Functions/arrayCumSumNonNegative.cpp index d5b7f3b8e3d..4dc4c9d0ef1 100644 --- a/dbms/src/Functions/arrayCumSumNonNegative.cpp +++ b/dbms/src/Functions/arrayCumSumNonNegative.cpp @@ -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; } } } diff --git a/dbms/src/IO/ReadHelpers.cpp b/dbms/src/IO/ReadHelpers.cpp index 9b8b28f5def..fb5b89babe2 100644 --- a/dbms/src/IO/ReadHelpers.cpp +++ b/dbms/src/IO/ReadHelpers.cpp @@ -667,6 +667,7 @@ void readJSONString(String & s, ReadBuffer & buf) template void readJSONStringInto, void>(PaddedPODArray & s, ReadBuffer & buf); template bool readJSONStringInto, bool>(PaddedPODArray & s, ReadBuffer & buf); template void readJSONStringInto(NullSink & s, ReadBuffer & buf); +template void readJSONStringInto(String & s, ReadBuffer & buf); template diff --git a/dbms/src/IO/readFloatText.h b/dbms/src/IO/readFloatText.h index d91a250ac77..9bcf49906ca 100644 --- a/dbms/src/IO/readFloatText.h +++ b/dbms/src/IO/readFloatText.h @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -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 -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 +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(digits) + exponent > static_cast(precision - scale)) + throw Exception("Decimal value is too big", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + if (static_cast(scale) + exponent < 0) + throw Exception("Decimal value is too small", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + scale += exponent; +} + template void readFloatTextPrecise(T & x, ReadBuffer & in) { readFloatTextPreciseImpl(x, in); } template bool tryReadFloatTextPrecise(T & x, ReadBuffer & in) { return readFloatTextPreciseImpl(x, in); } diff --git a/dbms/src/IO/tests/read_float_perf.cpp b/dbms/src/IO/tests/read_float_perf.cpp index 80d584c6dac..cb4c5e90a06 100644 --- a/dbms/src/IO/tests/read_float_perf.cpp +++ b/dbms/src/IO/tests/read_float_perf.cpp @@ -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 */ diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index b8c58dcf978..f14dfcaddeb 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -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; diff --git a/dbms/src/Interpreters/ExpressionActions.h b/dbms/src/Interpreters/ExpressionActions.h index 8d0fbe17cfe..8ca9582b0b0 100644 --- a/dbms/src/Interpreters/ExpressionActions.h +++ b/dbms/src/Interpreters/ExpressionActions.h @@ -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. diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 743e0835096..7d59b50798a 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -308,7 +308,7 @@ void ExpressionAnalyzer::translateQualifiedNamesImpl(ASTPtr & ast, const std::ve { if (auto * identifier = typeid_cast(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(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(external_table_name, ASTIdentifier::Table); + auto database_and_table_name = ASTIdentifier::createSpecial(external_table_name); if (auto ast_table_expr = typeid_cast(subquery_or_table_name_or_table_expression.get())) { @@ -1659,7 +1659,7 @@ void ExpressionAnalyzer::getArrayJoinedColumnsImpl(const ASTPtr & ast) if (ASTIdentifier * node = typeid_cast(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(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(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(ast.get())) { - if (node->kind == ASTIdentifier::Column + if (node->general() && !ignored_names.count(node->name) && !ignored_names.count(Nested::extractTableName(node->name))) { diff --git a/dbms/src/Interpreters/ExpressionJIT.cpp b/dbms/src/Interpreters/ExpressionJIT.cpp index 139df9b2944..470fea14b71 100644 --- a/dbms/src/Interpreters/ExpressionJIT.cpp +++ b/dbms/src/Interpreters/ExpressionJIT.cpp @@ -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(*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 & context, const IFunctionBase & f) +static void compileFunctionToLLVMByteCode(std::shared_ptr & context, const IFunctionBase & f) { ProfileEvents::increment(ProfileEvents::CompileFunction); @@ -428,7 +435,7 @@ static void compileFunction(std::shared_ptr & 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(&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(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 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 lock(mutex); if (counter[hash_key]++ < min_count_to_compile) continue; } + std::shared_ptr fn; if (compilation_cache) { + /// Lock here, to be sure, that all functions will be compiled + std::lock_guard 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 lock(mutex); + size_t used_memory = context->compileAllFunctionsToNativeCode(); + ProfileEvents::increment(ProfileEvents::CompileExpressionsBytes, used_memory); + } } } diff --git a/dbms/src/Interpreters/InJoinSubqueriesPreprocessor.cpp b/dbms/src/Interpreters/InJoinSubqueriesPreprocessor.cpp index 9df80de4c2f..45105594c4f 100644 --- a/dbms/src/Interpreters/InJoinSubqueriesPreprocessor.cpp +++ b/dbms/src/Interpreters/InJoinSubqueriesPreprocessor.cpp @@ -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(table_name, ASTIdentifier::Table); + ASTPtr table = ASTIdentifier::createSpecial(table_name); if (!database_name.empty()) { - ASTPtr database = std::make_shared(database_name, ASTIdentifier::Database); + ASTPtr database = ASTIdentifier::createSpecial(database_name); - database_and_table = std::make_shared(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(table_name, ASTIdentifier::Table); + database_and_table = ASTIdentifier::createSpecial(table_name); } } diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.cpp b/dbms/src/Interpreters/InterpreterInsertQuery.cpp index ea7913eb2c5..d303154c3a3 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.cpp +++ b/dbms/src/Interpreters/InterpreterInsertQuery.cpp @@ -54,14 +54,20 @@ StoragePtr InterpreterInsertQuery::getTable(const ASTInsertQuery & query) Block InterpreterInsertQuery::getSampleBlock(const ASTInsertQuery & query, const StoragePtr & table) { - Block table_sample_non_materialized = table->getSampleBlockNonMaterialized(); + + Block table_sample_non_materialized = table->getSampleBlockNonMaterialized(); /// If the query does not include information about columns if (!query.columns) - return table_sample_non_materialized; + { + /// Format Native ignores header and write blocks as is. + if (query.format == "Native") + return {}; + else + return table_sample_non_materialized; + } Block table_sample = table->getSampleBlock(); - /// Form the block based on the column names from the query Block res; for (const auto & identifier : query.columns->children) @@ -89,24 +95,25 @@ BlockIO InterpreterInsertQuery::execute() auto table_lock = table->lockStructure(true, __PRETTY_FUNCTION__); - NamesAndTypesList required_columns = table->getColumns().getAllPhysical(); - /// We create a pipeline of several streams, into which we will write data. BlockOutputStreamPtr out; out = std::make_shared(query.database, query.table, table, context, query_ptr, query.no_destination); - out = std::make_shared( - 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( - 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( + out, getSampleBlock(query, table), table->getSampleBlock(), table->getColumns().defaults, context); + auto out_wrapper = std::make_shared(out); out_wrapper->setProcessListElement(context.getProcessListElement()); out = std::move(out_wrapper); diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 6eecbca144f..a9b928c1cb3 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include @@ -499,7 +500,7 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt bool aggregate_final = expressions.need_aggregate && to_stage > QueryProcessingStage::WithMergeableState && - !query.group_by_with_totals && !query.group_by_with_rollup; + !query.group_by_with_totals && !query.group_by_with_rollup && !query.group_by_with_cube; if (expressions.first_stage) { @@ -557,10 +558,15 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt if (!aggregate_final) { if (query.group_by_with_totals) - executeTotalsAndHaving(pipeline, expressions.has_having, expressions.before_having, aggregate_overflow_row, !query.group_by_with_rollup); + { + bool final = !query.group_by_with_rollup && !query.group_by_with_cube; + executeTotalsAndHaving(pipeline, expressions.has_having, expressions.before_having, aggregate_overflow_row, final); + } - if (query.group_by_with_rollup) - executeRollup(pipeline); + if (query.group_by_with_rollup) + executeRollupOrCube(pipeline, Modificator::ROLLUP); + else if(query.group_by_with_cube) + executeRollupOrCube(pipeline, Modificator::CUBE); } else if (expressions.has_having) executeHaving(pipeline, expressions.before_having); @@ -575,10 +581,15 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt need_second_distinct_pass = query.distinct && pipeline.hasMoreThanOneStream(); if (query.group_by_with_totals && !aggregate_final) - executeTotalsAndHaving(pipeline, false, nullptr, aggregate_overflow_row, !query.group_by_with_rollup); + { + bool final = !query.group_by_with_rollup && !query.group_by_with_cube; + executeTotalsAndHaving(pipeline, expressions.has_having, expressions.before_having, aggregate_overflow_row, final); + } if (query.group_by_with_rollup && !aggregate_final) - executeRollup(pipeline); + executeRollupOrCube(pipeline, Modificator::ROLLUP); + else if (query.group_by_with_cube && !aggregate_final) + executeRollupOrCube(pipeline, Modificator::CUBE); } if (expressions.has_order_by) @@ -1087,7 +1098,7 @@ void InterpreterSelectQuery::executeTotalsAndHaving(Pipeline & pipeline, bool ha has_having ? query.having_expression->getColumnName() : "", settings.totals_mode, settings.totals_auto_threshold, final); } -void InterpreterSelectQuery::executeRollup(Pipeline & pipeline) +void InterpreterSelectQuery::executeRollupOrCube(Pipeline & pipeline, Modificator modificator) { executeUnion(pipeline); @@ -1111,7 +1122,10 @@ void InterpreterSelectQuery::executeRollup(Pipeline & pipeline) settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, context.getTemporaryPath()); - pipeline.firstStream() = std::make_shared(pipeline.firstStream(), params); + if (modificator == Modificator::ROLLUP) + pipeline.firstStream() = std::make_shared(pipeline.firstStream(), params); + else + pipeline.firstStream() = std::make_shared(pipeline.firstStream(), params); } diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index 9c534c2846a..8d433ab1754 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -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 & 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. * diff --git a/dbms/src/Interpreters/QueryNormalizer.cpp b/dbms/src/Interpreters/QueryNormalizer.cpp index e7785600114..701f140b306 100644 --- a/dbms/src/Interpreters/QueryNormalizer.cpp +++ b/dbms/src/Interpreters/QueryNormalizer.cpp @@ -85,7 +85,7 @@ void QueryNormalizer::performImpl(ASTPtr & ast, MapOfASTs & finished_asts, SetOf if (functionIsInOrGlobalInOperator(func_node->name)) if (ASTIdentifier * right = typeid_cast(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(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(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(database_and_table_name.get())) - { - right->kind = ASTIdentifier::Table; - } + right->setSpecial(); } } } diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index f2fa1dd30d7..1928d9d0828 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -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.") \ \ diff --git a/dbms/src/Interpreters/evaluateMissingDefaults.cpp b/dbms/src/Interpreters/evaluateMissingDefaults.cpp index 0b9bcb5417f..c9758ab0cd0 100644 --- a/dbms/src/Interpreters/evaluateMissingDefaults.cpp +++ b/dbms/src/Interpreters/evaluateMissingDefaults.cpp @@ -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)); + } } } diff --git a/dbms/src/Parsers/ASTIdentifier.h b/dbms/src/Parsers/ASTIdentifier.h index 1ef06711505..0ada2b60852 100644 --- a/dbms/src/Parsers/ASTIdentifier.h +++ b/dbms/src/Parsers/ASTIdentifier.h @@ -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 createSpecial(const String & name_) + { + return std::make_shared(name_, ASTIdentifier::Special); + } + protected: void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void appendColumnNameImpl(WriteBuffer & ostr) const override; + +private: + Kind kind; }; } diff --git a/dbms/src/Parsers/ASTSelectQuery.cpp b/dbms/src/Parsers/ASTSelectQuery.cpp index 0e48ac585b7..5f7ea7dc91a 100644 --- a/dbms/src/Parsers/ASTSelectQuery.cpp +++ b/dbms/src/Parsers/ASTSelectQuery.cpp @@ -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(database_name, ASTIdentifier::Database); + ASTPtr database = ASTIdentifier::createSpecial(database_name); ASTPtr table = table_expression->database_and_table_name; const String & old_name = static_cast(*table_expression->database_and_table_name).name; - table_expression->database_and_table_name = std::make_shared(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(table_name, ASTIdentifier::Table); + ASTPtr table = ASTIdentifier::createSpecial(table_name); if (!database_name.empty()) { - ASTPtr database = std::make_shared(database_name, ASTIdentifier::Database); + ASTPtr database = ASTIdentifier::createSpecial(database_name); - table_expression->database_and_table_name = std::make_shared(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(table_name, ASTIdentifier::Table); + table_expression->database_and_table_name = ASTIdentifier::createSpecial(table_name); } } diff --git a/dbms/src/Parsers/ASTSelectQuery.h b/dbms/src/Parsers/ASTSelectQuery.h index 7422c70212c..0ffdb44395e 100644 --- a/dbms/src/Parsers/ASTSelectQuery.h +++ b/dbms/src/Parsers/ASTSelectQuery.h @@ -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; diff --git a/dbms/src/Parsers/ParserQueryWithOutput.cpp b/dbms/src/Parsers/ParserQueryWithOutput.cpp index 3ec71de5f0c..7c2b6afc442 100644 --- a/dbms/src/Parsers/ParserQueryWithOutput.cpp +++ b/dbms/src/Parsers/ParserQueryWithOutput.cpp @@ -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(*(query_with_output.format)).kind = ASTIdentifier::Format; + typeid_cast(*(query_with_output.format)).setSpecial(); query_with_output.children.push_back(query_with_output.format); } diff --git a/dbms/src/Parsers/ParserSelectQuery.cpp b/dbms/src/Parsers/ParserSelectQuery.cpp index 480cb32b8bd..ffd9273dd8a 100644 --- a/dbms/src/Parsers/ParserSelectQuery.cpp +++ b/dbms/src/Parsers/ParserSelectQuery.cpp @@ -40,6 +40,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_settings("SETTINGS"); ParserKeyword s_by("BY"); ParserKeyword s_rollup("ROLLUP"); + ParserKeyword s_cube("CUBE"); ParserKeyword s_top("TOP"); ParserKeyword s_offset("OFFSET"); @@ -116,24 +117,27 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (s_group_by.ignore(pos, expected)) { if (s_rollup.ignore(pos, expected)) - { select_query->group_by_with_rollup = true; - if (!open_bracket.ignore(pos, expected)) - return false; - } + else if (s_cube.ignore(pos, expected)) + select_query->group_by_with_cube = true; + + if ((select_query->group_by_with_rollup || select_query->group_by_with_cube) && !open_bracket.ignore(pos, expected)) + return false; if (!exp_list.parse(pos, select_query->group_expression_list, expected)) return false; - if (select_query->group_by_with_rollup && !close_bracket.ignore(pos, expected)) + if ((select_query->group_by_with_rollup || select_query->group_by_with_cube) && !close_bracket.ignore(pos, expected)) return false; } - /// WITH ROLLUP + /// WITH ROLLUP, CUBE or TOTALS if (s_with.ignore(pos, expected)) { if (s_rollup.ignore(pos, expected)) select_query->group_by_with_rollup = true; + else if (s_cube.ignore(pos, expected)) + select_query->group_by_with_cube = true; else if (s_totals.ignore(pos, expected)) select_query->group_by_with_totals = true; else diff --git a/dbms/src/Parsers/ParserTablesInSelectQuery.cpp b/dbms/src/Parsers/ParserTablesInSelectQuery.cpp index 5eb00858e7c..088cd567fba 100644 --- a/dbms/src/Parsers/ParserTablesInSelectQuery.cpp +++ b/dbms/src/Parsers/ParserTablesInSelectQuery.cpp @@ -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 diff --git a/dbms/src/Storages/ITableDeclaration.cpp b/dbms/src/Storages/ITableDeclaration.cpp index d40d576cc5b..c05f56c7939 100644 --- a/dbms/src/Storages/ITableDeclaration.cpp +++ b/dbms/src/Storages/ITableDeclaration.cpp @@ -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; diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index 73d4320616a..ed18e03df7f 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -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(engine_args[5].get()); if (ast && ast->value.getType() == Field::Types::String) diff --git a/dbms/src/Storages/MergeTree/ActiveDataPartSet.h b/dbms/src/Storages/MergeTree/ActiveDataPartSet.h index 0ddd6beb31d..17b30205bd5 100644 --- a/dbms/src/Storages/MergeTree/ActiveDataPartSet.h +++ b/dbms/src/Storages/MergeTree/ActiveDataPartSet.h @@ -1,8 +1,6 @@ #pragma once #include -#include -#include #include #include diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index a5d247d82e3..54ffad594d1 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -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; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp index af37d4867b0..1c3d21d4653 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp @@ -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); } diff --git a/dbms/src/Storages/MergeTree/MergeTreePartInfo.h b/dbms/src/Storages/MergeTree/MergeTreePartInfo.h index f67d6fcb460..87623c1b790 100644 --- a/dbms/src/Storages/MergeTree/MergeTreePartInfo.h +++ b/dbms/src/Storages/MergeTree/MergeTreePartInfo.h @@ -1,8 +1,9 @@ #pragma once +#include #include +#include #include -#include namespace DB diff --git a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 36dba0719f4..998084381cb 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -384,7 +384,7 @@ bool MergeTreeWhereOptimizer::cannotBeMoved(const IAST * ptr) const else if (const auto identifier_ptr = typeid_cast(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; diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index c641cbc915b..4ec32883e6b 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -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(column, ASTIdentifier::Column)); + list_of_columns->children.push_back(std::make_shared(column)); InterpreterInsertQuery interpreter{insert, context, allow_materialized}; diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 68b53f359c0..5b8dbc80d21 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -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( diff --git a/dbms/src/Storages/StorageView.cpp b/dbms/src/Storages/StorageView.cpp index ea1e9828ec9..bb31bd81e53 100644 --- a/dbms/src/Storages/StorageView.cpp +++ b/dbms/src/Storages/StorageView.cpp @@ -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) { diff --git a/dbms/src/Storages/VirtualColumnUtils.cpp b/dbms/src/Storages/VirtualColumnUtils.cpp index 04b5ee405bf..71e26c323a4 100644 --- a/dbms/src/Storages/VirtualColumnUtils.cpp +++ b/dbms/src/Storages/VirtualColumnUtils.cpp @@ -97,7 +97,7 @@ static bool isValidFunction(const ASTPtr & expression, const NameSet & columns) if (const ASTIdentifier * identifier = typeid_cast(&*expression)) { - if (identifier->kind == ASTIdentifier::Kind::Column) + if (identifier->general()) return columns.count(identifier->name); } return true; diff --git a/dbms/src/TableFunctions/TableFunctionRemote.cpp b/dbms/src/TableFunctions/TableFunctionRemote.cpp index 4f7d1c11ac2..acc094408f8 100644 --- a/dbms/src/TableFunctions/TableFunctionRemote.cpp +++ b/dbms/src/TableFunctions/TableFunctionRemote.cpp @@ -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(arg.get())) - id->kind = ASTIdentifier::Table; + id->setSpecial(); ClusterPtr cluster; if (!cluster_name.empty()) diff --git a/dbms/src/TableFunctions/TableFunctionShardByHash.cpp b/dbms/src/TableFunctions/TableFunctionShardByHash.cpp index 616b6d117d4..4588796edeb 100644 --- a/dbms/src/TableFunctions/TableFunctionShardByHash.cpp +++ b/dbms/src/TableFunctions/TableFunctionShardByHash.cpp @@ -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(arg.get())) - id->kind = ASTIdentifier::Table; + id->setSpecial(); auto cluster = context.getCluster(cluster_name); size_t shard_index = sipHash64(key) % cluster->getShardCount(); diff --git a/dbms/tests/performance/date_time/date_time.xml b/dbms/tests/performance/date_time/date_time.xml index 12694107511..b783e5be808 100644 --- a/dbms/tests/performance/date_time/date_time.xml +++ b/dbms/tests/performance/date_time/date_time.xml @@ -30,9 +30,12 @@ toHour toDayOfWeek toDayOfMonth + toDayOfYear toMonth - + toQuarter toYear + toISOWeek + toISOYear toStartOfMinute toStartOfFiveMinute @@ -43,7 +46,8 @@ toMonday toStartOfMonth toStartOfQuarter - toYear + toStartOfYear + toStartOfISOYear toTime @@ -65,16 +69,20 @@ toDayOfWeek toDayOfMonth + toDayOfYear toMonth - + toQuarter toYear + toISOWeek + toISOYear toStartOfDay toDate toMonday toStartOfMonth toStartOfQuarter - toYear + toStartOfYear + toStartOfISOYear toRelativeYearNum toRelativeMonthNum diff --git a/dbms/tests/performance/date_time/format_date_time.xml b/dbms/tests/performance/date_time/format_date_time.xml new file mode 100644 index 00000000000..d99dafa4be2 --- /dev/null +++ b/dbms/tests/performance/date_time/format_date_time.xml @@ -0,0 +1,38 @@ + + Function formatDateTime + + once + + + + + + + 10000 + 1000 + + + + + + + + + + + + + format + + %F %T + %H:%M:%S + %Y-%m-%d %H:%M:%S + %m/%d/%Y %H:%M:%S + Hello + The current time is: %I:%M %p + + + + + SELECT count() FROM system.numbers WHERE NOT ignore(toDateTime('2017-01-01 00:00:00') + number % 100000000 + rand() % 100000 AS t, formatDateTime(t, '{format}')) + diff --git a/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference b/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference index aa266c87fcd..22142c4748f 100644 --- a/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference +++ b/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference @@ -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.------- diff --git a/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql b/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql index 532b5b9cc72..ede8b008867 100644 --- a/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql +++ b/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql @@ -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; diff --git a/dbms/tests/queries/0_stateless/00700_decimal_bounds.reference b/dbms/tests/queries/0_stateless/00700_decimal_bounds.reference index 710df36ebdb..e589b6c1dfa 100644 --- a/dbms/tests/queries/0_stateless/00700_decimal_bounds.reference +++ b/dbms/tests/queries/0_stateless/00700_decimal_bounds.reference @@ -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 diff --git a/dbms/tests/queries/0_stateless/00700_decimal_bounds.sql b/dbms/tests/queries/0_stateless/00700_decimal_bounds.sql index e4ea6eb9608..c2cceb27774 100644 --- a/dbms/tests/queries/0_stateless/00700_decimal_bounds.sql +++ b/dbms/tests/queries/0_stateless/00700_decimal_bounds.sql @@ -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; diff --git a/dbms/tests/queries/0_stateless/00700_decimal_casts.sql b/dbms/tests/queries/0_stateless/00700_decimal_casts.sql index f2d0d63ffc2..111dc5fb1cb 100644 --- a/dbms/tests/queries/0_stateless/00700_decimal_casts.sql +++ b/dbms/tests/queries/0_stateless/00700_decimal_casts.sql @@ -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); diff --git a/dbms/tests/queries/0_stateless/00700_decimal_defaults.reference b/dbms/tests/queries/0_stateless/00700_decimal_defaults.reference new file mode 100644 index 00000000000..30500d7c53f --- /dev/null +++ b/dbms/tests/queries/0_stateless/00700_decimal_defaults.reference @@ -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 diff --git a/dbms/tests/queries/0_stateless/00700_decimal_defaults.sql b/dbms/tests/queries/0_stateless/00700_decimal_defaults.sql new file mode 100644 index 00000000000..cddd766177a --- /dev/null +++ b/dbms/tests/queries/0_stateless/00700_decimal_defaults.sql @@ -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; diff --git a/dbms/tests/queries/0_stateless/00700_decimal_formats.reference b/dbms/tests/queries/0_stateless/00700_decimal_formats.reference index fe36e7af689..0bea4ba27be 100644 --- a/dbms/tests/queries/0_stateless/00700_decimal_formats.reference +++ b/dbms/tests/queries/0_stateless/00700_decimal_formats.reference @@ -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 diff --git a/dbms/tests/queries/0_stateless/00700_decimal_formats.sql b/dbms/tests/queries/0_stateless/00700_decimal_formats.sql index e2979b84cfc..ba7161a8249 100644 --- a/dbms/tests/queries/0_stateless/00700_decimal_formats.sql +++ b/dbms/tests/queries/0_stateless/00700_decimal_formats.sql @@ -11,9 +11,9 @@ CREATE TABLE IF NOT EXISTS test.decimal INSERT INTO test.decimal (a, b, c) VALUES (42.0, -42.0, 42) (0.42, -0.42, .42) (42.42, -42.42, 42.42); INSERT INTO test.decimal (a, b, c) FORMAT JSONEachRow {"a":1.1, "b":-1.1, "c":1.1} {"a":1.0, "b":-1.0, "c":1} {"a":0.1, "b":-0.1, "c":.1}; -INSERT INTO test.decimal (a, b, c) FORMAT CSV 2.0, -2.0, 2 +INSERT INTO test.decimal (a, b, c) FORMAT CSV 2.0,-2.0,2 ; -INSERT INTO test.decimal (a, b, c) FORMAT CSV 0.2, -0.2, .2 +INSERT INTO test.decimal (a, b, c) FORMAT CSV 0.2 ,-0.2 ,.2 ; INSERT INTO test.decimal (a, b, c) FORMAT CSV 2.2 , -2.2 , 2.2 ; @@ -23,6 +23,10 @@ INSERT INTO test.decimal (a, b, c) FORMAT TabSeparated 3.0 -3.0 3 ; INSERT INTO test.decimal (a, b, c) FORMAT TabSeparated 0.3 -0.3 .3 ; +INSERT INTO test.decimal (a, b, c) FORMAT CSV 4.4E+5,-4E+8,.4E+20 +; +INSERT INTO test.decimal (a, b, c) FORMAT CSV 5.5e-2, -5e-9 ,.5e-17 +; SELECT * FROM test.decimal ORDER BY a FORMAT JSONEachRow; SELECT * FROM test.decimal ORDER BY b DESC FORMAT CSV; diff --git a/dbms/tests/queries/0_stateless/00704_arrayCumSumLimited_arrayDifference.reference b/dbms/tests/queries/0_stateless/00704_arrayCumSumLimited_arrayDifference.reference index 6355a1c30ab..11a07746516 100644 --- a/dbms/tests/queries/0_stateless/00704_arrayCumSumLimited_arrayDifference.reference +++ b/dbms/tests/queries/0_stateless/00704_arrayCumSumLimited_arrayDifference.reference @@ -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] diff --git a/dbms/tests/queries/0_stateless/00706_iso_week_and_day_of_year.reference b/dbms/tests/queries/0_stateless/00706_iso_week_and_day_of_year.reference new file mode 100644 index 00000000000..300d9e22aa1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00706_iso_week_and_day_of_year.reference @@ -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 diff --git a/dbms/tests/queries/0_stateless/00706_iso_week_and_day_of_year.sql b/dbms/tests/queries/0_stateless/00706_iso_week_and_day_of_year.sql new file mode 100644 index 00000000000..64bfc3b1a91 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00706_iso_week_and_day_of_year.sql @@ -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); diff --git a/dbms/tests/queries/0_stateless/00707_float_csv_delimiter.reference b/dbms/tests/queries/0_stateless/00707_float_csv_delimiter.reference new file mode 100644 index 00000000000..669fcacbd97 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00707_float_csv_delimiter.reference @@ -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 diff --git a/dbms/tests/queries/0_stateless/00707_float_csv_delimiter.sql b/dbms/tests/queries/0_stateless/00707_float_csv_delimiter.sql new file mode 100644 index 00000000000..ac9ef192116 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00707_float_csv_delimiter.sql @@ -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; diff --git a/dbms/tests/queries/0_stateless/00715_json_each_row_input_nested.reference b/dbms/tests/queries/0_stateless/00715_json_each_row_input_nested.reference new file mode 100644 index 00000000000..e700b217714 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00715_json_each_row_input_nested.reference @@ -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] diff --git a/dbms/tests/queries/0_stateless/00715_json_each_row_input_nested.sh b/dbms/tests/queries/0_stateless/00715_json_each_row_input_nested.sh new file mode 100755 index 00000000000..ac7dab7febe --- /dev/null +++ b/dbms/tests/queries/0_stateless/00715_json_each_row_input_nested.sh @@ -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 diff --git a/dbms/tests/queries/0_stateless/00717_default_join_type.reference b/dbms/tests/queries/0_stateless/00717_default_join_type.reference new file mode 100644 index 00000000000..7f6efad4038 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00717_default_join_type.reference @@ -0,0 +1 @@ +3 3 diff --git a/dbms/tests/queries/0_stateless/00717_default_join_type.sql b/dbms/tests/queries/0_stateless/00717_default_join_type.sql new file mode 100644 index 00000000000..4249790747c --- /dev/null +++ b/dbms/tests/queries/0_stateless/00717_default_join_type.sql @@ -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'; diff --git a/dbms/tests/queries/0_stateless/00718_format_datetime.reference b/dbms/tests/queries/0_stateless/00718_format_datetime.reference new file mode 100644 index 00000000000..b774017f835 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00718_format_datetime.reference @@ -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 diff --git a/dbms/tests/queries/0_stateless/00718_format_datetime.sql b/dbms/tests/queries/0_stateless/00718_format_datetime.sql new file mode 100644 index 00000000000..21dc504985d --- /dev/null +++ b/dbms/tests/queries/0_stateless/00718_format_datetime.sql @@ -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') \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/00719_format_datetime_rand.reference b/dbms/tests/queries/0_stateless/00719_format_datetime_rand.reference new file mode 100644 index 00000000000..f7eb44d66e0 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00719_format_datetime_rand.reference @@ -0,0 +1,6 @@ +0 +0 +0 +0 +0 +0 diff --git a/dbms/tests/queries/0_stateless/00719_format_datetime_rand.sql b/dbms/tests/queries/0_stateless/00719_format_datetime_rand.sql new file mode 100644 index 00000000000..3b4022bf235 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00719_format_datetime_rand.sql @@ -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)); diff --git a/dbms/tests/queries/0_stateless/00719_insert_block_without_column.reference b/dbms/tests/queries/0_stateless/00719_insert_block_without_column.reference new file mode 100644 index 00000000000..7193c3d3f3d --- /dev/null +++ b/dbms/tests/queries/0_stateless/00719_insert_block_without_column.reference @@ -0,0 +1 @@ +Still alive diff --git a/dbms/tests/queries/0_stateless/00719_insert_block_without_column.sh b/dbms/tests/queries/0_stateless/00719_insert_block_without_column.sh new file mode 100755 index 00000000000..8724eb7f09a --- /dev/null +++ b/dbms/tests/queries/0_stateless/00719_insert_block_without_column.sh @@ -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'" diff --git a/dbms/tests/queries/0_stateless/00720_with_cube.reference b/dbms/tests/queries/0_stateless/00720_with_cube.reference new file mode 100644 index 00000000000..a0b951978f9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00720_with_cube.reference @@ -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 diff --git a/dbms/tests/queries/0_stateless/00720_with_cube.sql b/dbms/tests/queries/0_stateless/00720_with_cube.sql new file mode 100644 index 00000000000..032c83f17b5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00720_with_cube.sql @@ -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; diff --git a/debian/.pbuilderrc b/debian/.pbuilderrc index 7d198625f24..8b787718d76 100644 --- a/debian/.pbuilderrc +++ b/debian/.pbuilderrc @@ -160,7 +160,7 @@ case "$DIST" in esac if [ "$ARCH" != arm64 ]; then - case "$DIST" in + case "$DIST" in "cosmic" | "bionic" | "experimental" | "unstable" | "testing") EXTRAPACKAGES+=" liblld-6.0-dev libclang-6.0-dev liblld-6.0 " export CMAKE_FLAGS="-DLLVM_VERSION_POSTFIX=-6.0 $CMAKE_FLAGS" @@ -168,7 +168,7 @@ if [ "$ARCH" != arm64 ]; then "artful" ) EXTRAPACKAGES+=" liblld-5.0-dev libclang-5.0-dev liblld-5.0 " ;; - esac + esac else export CMAKE_FLAGS="-DENABLE_EMBEDDED_COMPILER=0 $CMAKE_FLAGS" fi diff --git a/debian/changelog b/debian/changelog index f5554b358da..1b95c94274f 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (18.12.17) unstable; urgency=low +clickhouse (18.13.0) unstable; urgency=low * Modified source code - -- Sun, 16 Sep 2018 05:24:57 +0300 + -- Mon, 17 Sep 2018 09:05:31 +0300 diff --git a/debian/rules b/debian/rules index 33967073091..9cf58799610 100755 --- a/debian/rules +++ b/debian/rules @@ -47,7 +47,12 @@ else endif endif -CMAKE_FLAGS += -DCMAKE_CXX_COMPILER=`which $(CXX)` -DCMAKE_C_COMPILER=`which $(CC)` +ifdef CXX + CMAKE_FLAGS += -DCMAKE_CXX_COMPILER=`which $(CXX)` +endif +ifdef CC + CMAKE_FLAGS += -DCMAKE_C_COMPILER=`which $(CC)` +endif ifndef DISABLE_NINJA NINJA=$(shell which ninja) diff --git a/docker/client/Dockerfile b/docker/client/Dockerfile index 006d88e77dc..bd24b0dd41f 100644 --- a/docker/client/Dockerfile +++ b/docker/client/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=18.12.17 +ARG version=18.13.0 RUN apt-get update && \ apt-get install -y apt-transport-https dirmngr && \ diff --git a/docker/server/Dockerfile b/docker/server/Dockerfile index fdd802216e5..ce9b244b658 100644 --- a/docker/server/Dockerfile +++ b/docker/server/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=18.12.17 +ARG version=18.13.0 RUN apt-get update && \ apt-get install -y apt-transport-https dirmngr && \ diff --git a/docker/test/Dockerfile b/docker/test/Dockerfile index 87b7c6c1011..bb342742eb1 100644 --- a/docker/test/Dockerfile +++ b/docker/test/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=18.12.17 +ARG version=18.13.0 RUN apt-get update && \ apt-get install -y apt-transport-https dirmngr && \ diff --git a/docs/en/operations/table_engines/distributed.md b/docs/en/operations/table_engines/distributed.md index fac2320af64..0bb55bc2603 100644 --- a/docs/en/operations/table_engines/distributed.md +++ b/docs/en/operations/table_engines/distributed.md @@ -47,7 +47,8 @@ Clusters are set like this: example01-02-2 - 9000 + 1 + 9440 @@ -58,12 +59,16 @@ Here a cluster is defined with the name 'logs' that consists of two shards, each Shards refer to the servers that contain different parts of the data (in order to read all the data, you must access all the shards). Replicas are duplicating servers (in order to read all the data, you can access the data on any one of the replicas). -The parameters `host`, `port`, and optionally `user` and `password` are specified for each server: +Cluster names must not contain dots. + +The parameters `host`, `port`, and optionally `user`, `password`, `secure`, `compression` are specified for each server: : - `host` – The address of the remote server. You can use either the domain or the IPv4 or IPv6 address. If you specify the domain, the server makes a DNS request when it starts, and the result is stored as long as the server is running. If the DNS request fails, the server doesn't start. If you change the DNS record, restart the server. - `port`– The TCP port for messenger activity ('tcp_port' in the config, usually set to 9000). Do not confuse it with http_port. - `user`– Name of the user for connecting to a remote server. Default value: default. This user must have access to connect to the specified server. Access is configured in the users.xml file. For more information, see the section "Access rights". -- `password` – The password for connecting to a remote server (not masked). Default value: empty string. +- `password` – The password for connecting to a remote server (not masked). Default value: empty string. +- `secure` - Use ssl for connection, usually you also should define `port` = 9440. Server should listen on 9440 and have correct certificates. +- `compression` - Use data compression. Default value: true. When specifying replicas, one of the available replicas will be selected for each of the shards when reading. You can configure the algorithm for load balancing (the preference for which replica to access) – see the 'load_balancing' setting. If the connection with the server is not established, there will be an attempt to connect with a short timeout. If the connection failed, the next replica will be selected, and so on for all the replicas. If the connection attempt failed for all the replicas, the attempt will be repeated the same way, several times. diff --git a/docs/en/operations/table_engines/kafka.md b/docs/en/operations/table_engines/kafka.md index 6fd1a90a503..eb79599e214 100644 --- a/docs/en/operations/table_engines/kafka.md +++ b/docs/en/operations/table_engines/kafka.md @@ -119,7 +119,7 @@ If you want to change the target table by using `ALTER`, we recommend disabling ## Configuration -Similar to GraphiteMergeTree, the Kafka engine supports extended configuration using the ClickHouse config file. There are two configuration keys that you can use: global (`kafka`) and topic-level (`kafka_topic_*`). The global configuration is applied first, and then the topic-level configuration is applied (if it exists). +Similar to GraphiteMergeTree, the Kafka engine supports extended configuration using the ClickHouse config file. There are two configuration keys that you can use: global (`kafka`) and topic-level (`kafka_*`). The global configuration is applied first, and then the topic-level configuration is applied (if it exists). ```xml @@ -129,10 +129,10 @@ Similar to GraphiteMergeTree, the Kafka engine supports extended configuration u - + 250 100000 - + ``` For a list of possible configuration options, see the [librdkafka configuration reference](https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md). Use the underscore (`_`) instead of a dot in the ClickHouse configuration. For example, `check.crcs=true` will be `true`. diff --git a/docs/en/query_language/functions/date_time_functions.md b/docs/en/query_language/functions/date_time_functions.md index 1299baa6c5a..8be3c14ffba 100644 --- a/docs/en/query_language/functions/date_time_functions.md +++ b/docs/en/query_language/functions/date_time_functions.md @@ -151,3 +151,34 @@ For a time interval starting at 'StartTime' and continuing for 'Duration' second For example, `timeSlots(toDateTime('2012-01-01 12:20:00'), 600) = [toDateTime('2012-01-01 12:00:00'), toDateTime('2012-01-01 12:30:00')]`. This is necessary for searching for pageviews in the corresponding session. +## formatDateTime(Time, Format\[, Timezone\]) + +Function formats a Time according given Format string. N.B.: Format is a constant expression, e.g. you can not have multiple formats for single result column. + +Supported modifiers for Format: +("Example" column shows formatting result for time `2018-01-02 22:33:44`) + +| Modifier | Description | Example | +| ----------- | -------- | --------------- | +|%C|year divided by 100 and truncated to integer (00-99)|20 +|%d|day of the month, zero-padded (01-31)|02 +|%D|Short MM/DD/YY date, equivalent to %m/%d/%y|01/02/2018| +|%e|day of the month, space-padded ( 1-31)| 2| +|%F|short YYYY-MM-DD date, equivalent to %Y-%m-%d|2018-01-02 +|%H|hour in 24h format (00-23)|22| +|%I|hour in 12h format (01-12)|10| +|%j|day of the year (001-366)|002| +|%m|month as a decimal number (01-12)|01| +|%M|minute (00-59)|33| +|%n|new-line character ('\n')|| +|%p|AM or PM designation|PM| +|%R|24-hour HH:MM time, equivalent to %H:%M|22:33| +|%S|second (00-59)|44| +|%t|horizontal-tab character ('\t')|| +|%T|ISO 8601 time format (HH:MM:SS), equivalent to %H:%M:%S|22:33:44| +|%u|ISO 8601 weekday as number with Monday as 1 (1-7)|2| +|%V|ISO 8601 week number (01-53)|01| +|%w|weekday as a decimal number with Sunday as 0 (0-6)|2| +|%y|Year, last two digits (00-99)|18| +|%Y|Year|2018| +|%%|a % sign|%| diff --git a/docs/ru/data_types/decimal.md b/docs/ru/data_types/decimal.md new file mode 100644 index 00000000000..f1718b33ca6 --- /dev/null +++ b/docs/ru/data_types/decimal.md @@ -0,0 +1,94 @@ + + +# Decimal(P, S), Decimal32(S), Decimal64(S), Decimal128(S) + +Знаковые дробные числа с сохранением точности операций сложения, умножения и вычитания. Для деления осуществляется отбрасывание (не округление) знаков, не попадающих в младший десятичный разряд. + +## Параметры + +- P - precision. Значение из диапазона [ 1 : 38 ]. Определяет, сколько десятичных знаков (с учетом дробной части) может содержать чило. +- S - scale. Значение из диапазона [ 0 : P ]. Определяет, сколько десятичных знаков содержится в дробной части числа. + +В зависимости от параметра P Decimal(P, S) является синонимом: +- P из [ 1 : 9 ] - для Decimal32(S) +- P из [ 10 : 18 ] - для Decimal64(S) +- P из [ 19 : 38 ] - для Decimal128(S) + +## Диапазоны Decimal + +- Decimal32(S) - ( -1 * 10^(9 - S), 1 * 10^(9 - S) ) +- Decimal64(S) - ( -1 * 10^(18 - S), 1 * 10^(18 - S) ) +- Decimal128(S) - ( -1 * 10^(38 - S), 1 * 10^(38 - S) ) + +Например, Decimal32(4) содержит числа от -99999.9999 до 99999.9999 c шагом 0.0001. + +## Внутреннее представление + +Внутри данные представляются как знаковые целые числа, соответсвующей разрядности. Реальные диапазоны, хранящиеся в ячейках памяти несколько больше заявленных. Заявленные диапазоны Decimal проверяются только при вводе числа из строкового представления. +Поскольку современные CPU не поддежривают 128-битные числа, операции над Decimal128 эмулируются программно. Decimal128 работает в разы медленней чем Decimal32/Decimal64. + +## Операции и типы результата + +Результат операции между двумя Decimal расширяется до большего типа (независимо от порядка аргументов). + +- Decimal64(S1) Decimal32(S2) -> Decimal64(S) +- Decimal128(S1) Decimal32(S2) -> Decimal128(S) +- Decimal128(S1) Decimal64(S2) -> Decimal128(S) + +Для размера дробной части (scale) результата действуют следующие правила: +- сложение, вычитание: S = max(S1, S2). +- умножение: S = S1 + S2. +- деление: S = S1. + +При операциях между Decimal и целыми числами результатом является Decimal, аналогичный аргументу. + +Операции между Decimal и Float32/64 не определены. Для осуществления таких операций нужно явно привести один из агруметнов функциями: toDecimal32, toDecimal64, toDecimal128, или toFloat32, toFloat64. Это сделано из двух соображений. Во-первых, результат операции будет с потерей точности. Во-вторых, преобразование типа - дорогая операция, из-за ее наличия пользовательский запрос может работать в несколько раз дольше. + +Часть функций над Decimal возвращают Float64 (например, var, stddev). Для некоторых из них промежуточные операции проходят в Decimal. +Для таких функций результат над одинаковыми данными во Float64 и Decimal может отличаться, несмотря на одинаковый тип результата. + +## Проверка переполнений + +При выполнении операций над типом Decimal могут происходить целочисленные переполнения. Лишняя дробная часть отбрасывается (не округляется). Лишняя целочисленная часть приводит к исключению. +``` +SELECT toDecimal32(2, 4) AS x, x / 3 +``` +``` +┌──────x─┬─divide(toDecimal32(2, 4), 3)─┐ +│ 2.0000 │ 0.6666 │ +└────────┴──────────────────────────────┘ +``` + +``` +SELECT toDecimal32(4.2, 8) AS x, x * x +``` +``` +DB::Exception: Scale is out of bounds. +``` + +``` +SELECT toDecimal32(4.2, 8) AS x, 6 * x +``` +``` +DB::Exception: Decimal math overflow. +``` + +Проверка переполнения приводит к замедлению операций. При уверенности, что типа результата хватит для его записи проверку переполнения можно отключить настройкой decimal_check_overflow. В этом случае при переполнении вернется неверное значение: +``` +SET decimal_check_overflow = 0; +SELECT toDecimal32(4.2, 8) AS x, 6 * x +``` +``` +┌──────────x─┬─multiply(6, toDecimal32(4.2, 8))─┐ +│ 4.20000000 │ -17.74967296 │ +└────────────┴──────────────────────────────────┘ +``` + +Переполнения происходят не только на арифметических операциях, но и на операциях сравнения. Отключать проверку стоит только при полной уверенности в корректности результата: + +``` +SELECT toDecimal32(1, 8) < 100 +``` +``` +DB::Exception: Can't compare. +``` diff --git a/docs/ru/operations/table_engines/distributed.md b/docs/ru/operations/table_engines/distributed.md index 11d16606478..a797677690e 100644 --- a/docs/ru/operations/table_engines/distributed.md +++ b/docs/ru/operations/table_engines/distributed.md @@ -58,12 +58,17 @@ logs - имя кластера в конфигурационном файле с Шардами называются серверы, содержащие разные части данных (чтобы прочитать все данные, нужно идти на все шарды). Репликами называются дублирующие серверы (чтобы прочитать данные, можно идти за данными на любую из реплик). -В качестве параметров для каждого сервера указываются `host`, `port` и, не обязательно, `user`, `password`: +Имя кластера не должно содержать точки. + +В качестве параметров для каждого сервера указываются `host`, `port` и, не обязательно, `user`, `password`, `secure`, `compression`: : - `host` - адрес удалённого сервера. Может быть указан домен, или IPv4 или IPv6 адрес. В случае указания домена, при старте сервера делается DNS запрос, и результат запоминается на всё время работы сервера. Если DNS запрос неуспешен, то сервер не запускается. Если вы изменяете DNS-запись, перезапустите сервер. - `port` - TCP-порт для межсерверного взаимодействия (в конфиге - tcp_port, обычно 9000). Не перепутайте с http_port. - `user` - имя пользователя для соединения с удалённым сервером. по умолчанию - default. Этот пользователь должен иметь доступ для соединения с указанным сервером. Доступы настраиваются в файле users.xml, подробнее смотрите в разделе "Права доступа". - `password` - пароль для соединения с удалённым сервером, в открытом виде. по умолчанию - пустая строка. + - `secure` - Использовать шифрованое соединение ssl, Обычно используется с портом `port` = 9440. Сервер должен слушать порт 9440 с корректными настройками сертификатов. + - `compression` - Использовать сжатие данных. По умолчанию: true. + При указании реплик, для каждого из шардов, при чтении, будет выбрана одна из доступных реплик. Можно настроить алгоритм балансировки нагрузки (то есть, предпочтения, на какую из реплик идти) - см. настройку load_balancing. Если соединение с сервером не установлено, то будет произведена попытка соединения с небольшим таймаутом. Если соединиться не удалось, то будет выбрана следующая реплика, и так для всех реплик. Если попытка соединения для всех реплик не удалась, то будут снова произведены попытки соединения по кругу, и так несколько раз. diff --git a/docs/ru/operations/table_engines/kafka.md b/docs/ru/operations/table_engines/kafka.md index a0f370df795..282140d4350 100644 --- a/docs/ru/operations/table_engines/kafka.md +++ b/docs/ru/operations/table_engines/kafka.md @@ -119,7 +119,7 @@ Kafka SETTINGS ## Конфигурация -Аналогично GraphiteMergeTree, движок Kafka поддерживает расширенную конфигурацию с помощью конфигурационного файла ClickHouse. Существует два конфигурационных ключа, которые можно использовать - глобальный (`kafka`) и по топикам (`kafka_topic_*`). Сначала применяется глобальная конфигурация, затем конфигурация по топикам (если она существует). +Аналогично GraphiteMergeTree, движок Kafka поддерживает расширенную конфигурацию с помощью конфигурационного файла ClickHouse. Существует два конфигурационных ключа, которые можно использовать - глобальный (`kafka`) и по топикам (`kafka_*`). Сначала применяется глобальная конфигурация, затем конфигурация по топикам (если она существует). ```xml @@ -129,10 +129,10 @@ Kafka SETTINGS - + 250 100000 - + ``` В документе [librdkafka configuration reference](https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md) можно увидеть список возможных опций конфигурации. Используйте подчёркивания (`_`) вместо точек в конфигурации ClickHouse, например, `check.crcs=true` будет соответствовать `true`. diff --git a/docs/ru/query_language/functions/date_time_functions.md b/docs/ru/query_language/functions/date_time_functions.md index c9e0c3eff28..40f12cea07a 100644 --- a/docs/ru/query_language/functions/date_time_functions.md +++ b/docs/ru/query_language/functions/date_time_functions.md @@ -122,3 +122,34 @@ SELECT Для интервала времени, начинающегося в StartTime и продолжающегося Duration секунд, возвращает массив моментов времени, состоящий из округлений вниз до получаса точек из этого интервала. Например, `timeSlots(toDateTime('2012-01-01 12:20:00'), toUInt32(600)) = [toDateTime('2012-01-01 12:00:00'), toDateTime('2012-01-01 12:30:00')]`. Это нужно для поиска хитов, входящих в соответствующий визит. + +## formatDateTime(Time, Format\[, Timezone\]) +Функция преобразования даты-с-временем в String согласно заданному шаблону. Важно - шаблон является константным выражением, т.е. невозможно использование разных шаблонов в одной колонке. + +Поддерживаемые модификаторы в шаблоне Format: +(колонка "Пример" показана для времени `2018-01-02 22:33:44`) + +| Модификатор | Описание | Пример | +| ----------- | -------- | --------------- | +|%C|номер года, поделённый на 100 (00-99)|20 +|%d|день месяца, с ведущим нулём (01-31)|02 +|%D|короткая запись %m/%d/%y|01/02/2018| +|%e|день месяца, с ведущим пробелом ( 1-31)| 2| +|%F|короткая запись %Y-%m-%d|2018-01-02 +|%H|час в 24-часовом формате (00-23)|22| +|%I|час в 12-часовом формате (01-12)|10| +|%j|номер дня в году, с ведущими нулями (001-366)|002| +|%m|месяц, с ведущим нулём (01-12)|01| +|%M|минуты, с ведущим нулём (00-59)|33| +|%n|символ переноса строки ('\n')|| +|%p|обозначения AM или PM|PM| +|%R|короткая запись %H:%M|22:33| +|%S|секунды, с ведущими нулями (00-59)|44| +|%t|символ табуляции ('\t')|| +|%T|формат времени ISO 8601, одинаковый с %H:%M:%S|22:33:44| +|%u|номер дня недели согласно ISO 8601, понедельник - 1, воскресенье - 7|2| +|%V|номер недели согласно ISO 8601 (01-53)|01| +|%w|номер дня недели, начиная с воскресенья (0-6)|2| +|%y|год, последние 2 цифры (00-99)|18| +|%Y|год, 4 цифры|2018| +|%%|символ %|%| diff --git a/docs/ru/query_language/functions/type_conversion_functions.md b/docs/ru/query_language/functions/type_conversion_functions.md index 21c8556f255..ea39c4f6160 100644 --- a/docs/ru/query_language/functions/type_conversion_functions.md +++ b/docs/ru/query_language/functions/type_conversion_functions.md @@ -12,6 +12,10 @@ ## toDate, toDateTime +## toDecimal32(value, S), toDecimal64(value, S), toDecimal128(value, S) +Приводит строку или число value к типу [Decimal](../../data_types/decimal.md#data_type-decimal) указанной точности. +Параметр S (scale) определяет число десятичных знаков после запятой. + ## toString Функции преобразования между числами, строками (но не фиксированными строками), датами и датами-с-временем. Все эти функции принимают один аргумент. diff --git a/libs/libcommon/CMakeLists.txt b/libs/libcommon/CMakeLists.txt index d232abfd328..6d341bee4c2 100644 --- a/libs/libcommon/CMakeLists.txt +++ b/libs/libcommon/CMakeLists.txt @@ -23,6 +23,7 @@ add_library (common ${LINK_MODE} src/SetTerminalEcho.cpp include/common/Types.h + include/common/DayNum.h include/common/DateLUT.h include/common/DateLUTImpl.h include/common/LocalDate.h diff --git a/libs/libcommon/include/common/DateLUTImpl.h b/libs/libcommon/include/common/DateLUTImpl.h index ab04b7cbcbf..505ee968f6b 100644 --- a/libs/libcommon/include/common/DateLUTImpl.h +++ b/libs/libcommon/include/common/DateLUTImpl.h @@ -1,8 +1,8 @@ #pragma once #include +#include #include -#include #include #include @@ -15,9 +15,6 @@ #define DATE_LUT_YEARS (1 + DATE_LUT_MAX_YEAR - DATE_LUT_MIN_YEAR) /// Number of years in lookup table -STRONG_TYPEDEF(UInt16, DayNum) - - /** Lookup table to conversion of time to date, and to month / year / day of week / day of month and so on. * First time was implemented for OLAPServer, that needed to do billions of such transformations. */ @@ -309,6 +306,9 @@ public: inline unsigned toYear(DayNum d) const { return lut[d].year; } inline unsigned toDayOfWeek(DayNum d) const { return lut[d].day_of_week; } inline unsigned toDayOfMonth(DayNum d) const { return lut[d].day_of_month; } + inline unsigned toDayOfYear(DayNum d) const { return d + 1 - toFirstDayNumOfYear(d); } + + inline unsigned toDayOfYear(time_t t) const { return toDayOfYear(toDayNum(t)); } /// Number of week from some fixed moment in the past. Week begins at monday. /// (round down to monday and divide DayNum by 7; we made an assumption, @@ -316,7 +316,7 @@ public: inline unsigned toRelativeWeekNum(DayNum d) const { /// We add 8 to avoid underflow at beginning of unix epoch. - return (d + 8 - lut[d].day_of_week) / 7; + return (d + 8 - toDayOfWeek(d)) / 7; } inline unsigned toRelativeWeekNum(time_t t) const @@ -324,6 +324,55 @@ public: return toRelativeWeekNum(toDayNum(t)); } + /// Get year that contains most of the current week. Week begins at monday. + inline unsigned toISOYear(DayNum d) const + { + /// That's effectively the year of thursday of current week. + return toYear(DayNum(d + 4 - toDayOfWeek(d))); + } + + inline unsigned toISOYear(time_t t) const + { + return toISOYear(toDayNum(t)); + } + + /// ISO year begins with a monday of the week that is contained more than by half in the corresponding calendar year. + /// Example: ISO year 2019 begins at 2018-12-31. And ISO year 2017 begins at 2017-01-02. + /// https://en.wikipedia.org/wiki/ISO_week_date + inline DayNum toFirstDayNumOfISOYear(DayNum d) const + { + auto iso_year = toISOYear(d); + + DayNum first_day_of_year = years_lut[iso_year - DATE_LUT_MIN_YEAR]; + auto first_day_of_week_of_year = lut[first_day_of_year].day_of_week; + + return DayNum(first_day_of_week_of_year <= 4 + ? first_day_of_year + 1 - first_day_of_week_of_year + : first_day_of_year + 8 - first_day_of_week_of_year); + } + + inline DayNum toFirstDayNumOfISOYear(time_t t) const + { + return toFirstDayNumOfISOYear(toDayNum(t)); + } + + inline time_t toFirstDayOfISOYear(time_t t) const + { + return fromDayNum(toFirstDayNumOfISOYear(t)); + } + + /// ISO 8601 week number. Week begins at monday. + /// The week number 1 is the first week in year that contains 4 or more days (that's more than half). + inline unsigned toISOWeek(DayNum d) const + { + return 1 + (toFirstDayNumOfWeek(d) - toFirstDayNumOfISOYear(d)) / 7; + } + + inline unsigned toISOWeek(time_t t) const + { + return toISOWeek(toDayNum(t)); + } + /// Number of month from some fixed moment in the past (year * 12 + month) inline unsigned toRelativeMonthNum(DayNum d) const { diff --git a/libs/libcommon/include/common/DayNum.h b/libs/libcommon/include/common/DayNum.h new file mode 100644 index 00000000000..904a9281d64 --- /dev/null +++ b/libs/libcommon/include/common/DayNum.h @@ -0,0 +1,9 @@ +#pragma once + +#include +#include + +/** Represents number of days since 1970-01-01. + * See DateLUTImpl for usage examples. + */ +STRONG_TYPEDEF(UInt16, DayNum) diff --git a/libs/libmysqlxx/include/mysqlxx/Value.h b/libs/libmysqlxx/include/mysqlxx/Value.h index 04944088208..889a97ea05b 100644 --- a/libs/libmysqlxx/include/mysqlxx/Value.h +++ b/libs/libmysqlxx/include/mysqlxx/Value.h @@ -16,10 +16,6 @@ #include -/// Обрезать длинный запрос до указанной длины для текста исключения. -#define MYSQLXX_QUERY_PREVIEW_LENGTH 1000 - - namespace mysqlxx { diff --git a/libs/libmysqlxx/src/Value.cpp b/libs/libmysqlxx/src/Value.cpp index 87cc0d15f0f..8e448604e5a 100644 --- a/libs/libmysqlxx/src/Value.cpp +++ b/libs/libmysqlxx/src/Value.cpp @@ -156,6 +156,8 @@ double Value::readFloatText(const char * buf, size_t length) const void Value::throwException(const char * text) const { + static constexpr size_t MYSQLXX_QUERY_PREVIEW_LENGTH = 1000; + std::stringstream info; info << text; diff --git a/release b/release index dcf8a0f2b29..23bfd6f2dd6 100755 --- a/release +++ b/release @@ -34,8 +34,6 @@ cd $CURDIR source "./release_lib.sh" -DEB_CC=${DEB_CC:=gcc-7} -DEB_CXX=${DEB_CXX:=g++-7} PBUILDER_AUTOUPDATE=${PBUILDER_AUTOUPDATE=4320} DEBUILD_NOSIGN_OPTIONS="-us -uc" @@ -110,6 +108,8 @@ echo -e "\nCurrent version is $VERSION_STRING" gen_changelog "$VERSION_STRING" "" "$AUTHOR" "" if [ -z "$USE_PBUILDER" ] ; then + DEB_CC=${DEB_CC:=`which gcc-7 gcc-8 gcc | head -n1`} + DEB_CXX=${DEB_CXX:=`which g++-7 g++-8 g++ | head -n1`} # Build (only binary packages). debuild -e PATH -e SSH_AUTH_SOCK \ -e DEB_CC=$DEB_CC -e DEB_CXX=$DEB_CXX -e CMAKE_FLAGS="$CMAKE_FLAGS" \