Merge remote-tracking branch 'upstream/master' into fix4

This commit is contained in:
proller 2017-11-01 14:43:48 +03:00
commit 9bc029ba17
13 changed files with 83 additions and 69 deletions

View File

@ -5,5 +5,5 @@ add_library(cityhash
include/city.h
src/config.h)
target_include_directories (cityhash PUBLIC include)
target_include_directories (cityhash BEFORE PUBLIC include)
target_include_directories (cityhash PRIVATE src)

View File

@ -1,6 +1,6 @@
# This strings autochanged from release_lib.sh:
set(VERSION_DESCRIBE v1.1.54307-testing)
set(VERSION_REVISION 54307)
set(VERSION_DESCRIBE v1.1.54310-testing)
set(VERSION_REVISION 54310)
# end of autochange
set (VERSION_MAJOR 1)

View File

@ -1,6 +1,6 @@
#include <DataStreams/MaterializingBlockInputStream.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataStreams/materializeBlock.h>
namespace DB
{
@ -24,31 +24,7 @@ String MaterializingBlockInputStream::getID() const
Block MaterializingBlockInputStream::readImpl()
{
Block res = children.back()->read();
if (!res)
return res;
size_t columns = res.columns();
for (size_t i = 0; i < columns; ++i)
{
auto & element = res.safeGetByPosition(i);
auto & src = element.column;
ColumnPtr converted = src->convertToFullColumnIfConst();
if (converted)
{
src = converted;
auto & type = element.type;
if (type->isNull())
{
/// A ColumnNull that is converted to a full column
/// has the type DataTypeNullable(DataTypeUInt8).
type = std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt8>());
}
}
}
return res;
return materializeBlock(children.back()->read());
}
}

View File

@ -1,6 +1,6 @@
#pragma once
#include <Core/Block.h>
#include <DataStreams/materializeBlock.h>
#include <DataStreams/IBlockOutputStream.h>
@ -15,24 +15,18 @@ public:
MaterializingBlockOutputStream(const BlockOutputStreamPtr & output)
: output{output} {}
void write(const Block & block) override
{
output->write(materialize(block));
}
void flush() override { output->flush(); }
void writePrefix() override { output->writePrefix(); }
void writeSuffix() override { output->writeSuffix(); }
void setRowsBeforeLimit(size_t rows_before_limit) override { output->setRowsBeforeLimit(rows_before_limit); }
void setTotals(const Block & totals) override { output->setTotals(materialize(totals)); }
void setExtremes(const Block & extremes) override { output->setExtremes(materialize(extremes)); }
void onProgress(const Progress & progress) override { output->onProgress(progress); }
String getContentType() const override { return output->getContentType(); }
void write(const Block & block) override { output->write(materializeBlock(block)); }
void flush() override { output->flush(); }
void writePrefix() override { output->writePrefix(); }
void writeSuffix() override { output->writeSuffix(); }
void setRowsBeforeLimit(size_t rows_before_limit) override { output->setRowsBeforeLimit(rows_before_limit); }
void setTotals(const Block & totals) override { output->setTotals(materializeBlock(totals)); }
void setExtremes(const Block & extremes) override { output->setExtremes(materializeBlock(extremes)); }
void onProgress(const Progress & progress) override { output->onProgress(progress); }
String getContentType() const override { return output->getContentType(); }
private:
BlockOutputStreamPtr output;
static Block materialize(const Block & original_block);
};
}

View File

@ -1,25 +1,27 @@
#include <DataStreams/MaterializingBlockOutputStream.h>
#include <DataStreams/materializeBlock.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h>
#include <ext/range.h>
namespace DB
{
Block MaterializingBlockOutputStream::materialize(const Block & original_block)
Block materializeBlock(const Block & block)
{
/// copy block to get rid of const
auto block = original_block;
if (!block)
return block;
for (const auto i : ext::range(0, block.columns()))
Block res = block;
size_t columns = res.columns();
for (size_t i = 0; i < columns; ++i)
{
auto & element = block.safeGetByPosition(i);
auto & element = res.getByPosition(i);
auto & src = element.column;
ColumnPtr converted = src->convertToFullColumnIfConst();
if (converted)
{
src = converted;
auto & type = element.type;
if (type->isNull())
{
@ -30,7 +32,7 @@ Block MaterializingBlockOutputStream::materialize(const Block & original_block)
}
}
return block;
return res;
}
}

View File

@ -0,0 +1,13 @@
#pragma once
#include <Core/Block.h>
namespace DB
{
/** Converts columns-constants to full columns ("materializes" them).
*/
Block materializeBlock(const Block & block);
}

View File

@ -1465,11 +1465,26 @@ public:
}
};
template <bool is_first_line_zero>
struct FunctionRunningDifferenceName;
template <>
struct FunctionRunningDifferenceName<true>
{
static constexpr auto name = "runningDifference";
};
template <>
struct FunctionRunningDifferenceName<false>
{
static constexpr auto name = "runningIncome";
};
/** Calculate difference of consecutive values in block.
* So, result of function depends on partition of data to blocks and on order of data in block.
*/
class FunctionRunningDifference : public IFunction
template <bool is_first_line_zero>
class FunctionRunningDifferenceImpl : public IFunction
{
private:
/// It is possible to track value from previous block, to calculate continuously across all blocks. Not implemented.
@ -1485,7 +1500,7 @@ private:
/// It is possible to SIMD optimize this loop. By no need for that in practice.
dst[0] = 0;
dst[0] = is_first_line_zero ? 0 : src[0];
Src prev = src[0];
for (size_t i = 1; i < size; ++i)
{
@ -1532,10 +1547,11 @@ private:
}
public:
static constexpr auto name = "runningDifference";
static constexpr auto name = FunctionRunningDifferenceName<is_first_line_zero>::name;
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionRunningDifference>();
return std::make_shared<FunctionRunningDifferenceImpl<is_first_line_zero>>();
}
String getName() const override
@ -1585,6 +1601,9 @@ public:
}
};
using FunctionRunningDifference = FunctionRunningDifferenceImpl<true>;
using FunctionRunningIncome = FunctionRunningDifferenceImpl<false>;
/** Takes state of aggregate function. Returns result of aggregation (finalized state).
*/
@ -1813,6 +1832,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
factory.registerFunction<FunctionRunningAccumulate>();
factory.registerFunction<FunctionRunningDifference>();
factory.registerFunction<FunctionRunningIncome>();
factory.registerFunction<FunctionFinalizeAggregation>();
}
}

View File

@ -7,7 +7,7 @@ namespace DB
namespace
{
const ExternalLoaderUpdateSettings externalDictionariesUpdateSettings;
const ExternalLoaderUpdateSettings externalDictionariesUpdateSettings { };
const ExternalLoaderConfigSettings & getExternalDictionariesConfigSettings()
{

View File

@ -12,7 +12,7 @@ namespace ErrorCodes
namespace
{
const ExternalLoaderUpdateSettings externalModelsUpdateSettings;
const ExternalLoaderUpdateSettings externalModelsUpdateSettings { };
const ExternalLoaderConfigSettings & getExternalModelsConfigSettings()
{

View File

@ -117,7 +117,8 @@ MergeTreeData::MergeTreeData(
initPrimaryKey();
if (sampling_expression && (!primary_expr_ast || !primary_key_sample.has(sampling_expression->getColumnName())))
if (sampling_expression && (!primary_expr_ast || !primary_key_sample.has(sampling_expression->getColumnName()))
&& !attach && !settings.compatibility_allow_sampling_expression_not_in_primary_key) /// This is for backward compatibility.
throw Exception("Sampling expression must be present in the primary key", ErrorCodes::BAD_ARGUMENTS);
MergeTreeDataFormatVersion min_format_version(0);
@ -125,7 +126,7 @@ MergeTreeData::MergeTreeData(
{
try
{
String partition_expr_str = "toYYYYMM(" + date_column_name + ")";
String partition_expr_str = "toYYYYMM(" + backQuoteIfNeed(date_column_name) + ")";
ParserNotEmptyExpressionList parser(/* allow_alias_without_as_keyword = */ false);
partition_expr_ast = parseQuery(
parser, partition_expr_str.data(), partition_expr_str.data() + partition_expr_str.length(), "partition expression");

View File

@ -122,7 +122,15 @@ struct MergeTreeSettings
M(SettingUInt64, vertical_merge_algorithm_min_rows_to_activate, 16 * DEFAULT_MERGE_BLOCK_SIZE) \
\
/** Minimal amount of non-PK columns to activate Vertical merge algorithm */ \
M(SettingUInt64, vertical_merge_algorithm_min_columns_to_activate, 11)
M(SettingUInt64, vertical_merge_algorithm_min_columns_to_activate, 11) \
\
/** Compatibility settings */ \
\
/** Allow to create a table with sampling expression not in primary key. \
* This is needed only to temporarily allow to run the server with wrong tables \
* for backward compatibility. \
*/ \
M(SettingBool, compatibility_allow_sampling_expression_not_in_primary_key, false) \
/// Settings that should not change after the creation of a table.
#define APPLY_FOR_IMMUTABLE_MERGE_TREE_SETTINGS(M) \

View File

@ -10,11 +10,11 @@ USE test;
CREATE MATERIALIZED VIEW mv TO dst AS SELECT * FROM src;
INSERT INTO src VALUES (1), (2);
SELECT * FROM mv;
SELECT * FROM mv ORDER BY x;
-- Detach MV and see if the data is still readable
DETACH TABLE mv;
SELECT * FROM dst;
SELECT * FROM dst ORDER BY x;
USE default;
@ -23,11 +23,11 @@ ATTACH TABLE test.mv;
INSERT INTO test.src VALUES (3);
SELECT * FROM test.mv;
SELECT * FROM test.mv ORDER BY x;
-- Drop the MV and see if the data is still readable
DROP TABLE test.mv;
SELECT * FROM test.dst;
SELECT * FROM test.dst ORDER BY x;
DROP TABLE test.src;
DROP TABLE test.dst;

4
debian/changelog vendored
View File

@ -1,5 +1,5 @@
clickhouse (1.1.54307) unstable; urgency=low
clickhouse (1.1.54310) unstable; urgency=low
* Modified source code
-- <robot-metrika-test@yandex-team.ru> Mon, 30 Oct 2017 22:35:37 +0300
-- <robot-metrika-test@yandex-team.ru> Wed, 01 Nov 2017 08:03:17 +0300