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

This commit is contained in:
proller 2019-08-05 17:55:22 +03:00
commit 78c5d570ea
8 changed files with 34 additions and 8 deletions

View File

@ -127,7 +127,7 @@ namespace ErrorCodes
extern const int INCORRECT_DATA = 117;
extern const int ENGINE_REQUIRED = 119;
extern const int CANNOT_INSERT_VALUE_OF_DIFFERENT_SIZE_INTO_TUPLE = 120;
extern const int UNKNOWN_SET_DATA_VARIANT = 121;
extern const int UNSUPPORTED_JOIN_KEYS = 121;
extern const int INCOMPATIBLE_COLUMNS = 122;
extern const int UNKNOWN_TYPE_OF_AST_NODE = 123;
extern const int INCORRECT_ELEMENT_OF_SET = 124;

View File

@ -1,4 +1,5 @@
#include <DataStreams/AddingDefaultBlockOutputStream.h>
#include <DataStreams/ConvertingBlockInputStream.h>
#include <DataStreams/PushingToViewsBlockOutputStream.h>
#include <DataStreams/SquashingBlockInputStream.h>
#include <DataTypes/NestedUtils.h>
@ -192,6 +193,7 @@ void PushingToViewsBlockOutputStream::process(const Block & block, size_t view_n
/// and two-level aggregation is triggered).
in = std::make_shared<SquashingBlockInputStream>(
in, context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes);
in = std::make_shared<ConvertingBlockInputStream>(context, in, view.out->getHeader(), ConvertingBlockInputStream::MatchColumnsMode::Position);
in->readPrefix();

View File

@ -26,7 +26,7 @@ namespace DB
namespace ErrorCodes
{
extern const int UNKNOWN_SET_DATA_VARIANT;
extern const int UNSUPPORTED_JOIN_KEYS;
extern const int LOGICAL_ERROR;
extern const int SET_SIZE_LIMIT_EXCEEDED;
extern const int TYPE_MISMATCH;
@ -770,7 +770,7 @@ IColumn::Filter switchJoinRightColumns(
#undef M
default:
throw Exception("Unknown JOIN keys variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT);
throw Exception("Unsupported JOIN keys. Type: " + toString(static_cast<UInt32>(type)), ErrorCodes::UNSUPPORTED_JOIN_KEYS);
}
}
@ -1350,7 +1350,8 @@ private:
APPLY_FOR_JOIN_VARIANTS(M)
#undef M
default:
throw Exception("Unknown JOIN keys variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT);
throw Exception("Unsupported JOIN keys. Type: " + toString(static_cast<UInt32>(parent.type)),
ErrorCodes::UNSUPPORTED_JOIN_KEYS);
}
__builtin_unreachable();

View File

@ -8,7 +8,6 @@ namespace DB
namespace ErrorCodes
{
extern const int UNKNOWN_SET_DATA_VARIANT;
extern const int LOGICAL_ERROR;
}

View File

@ -18,7 +18,7 @@ namespace DB
namespace ErrorCodes
{
extern const int UNKNOWN_SET_DATA_VARIANT;
extern const int UNSUPPORTED_JOIN_KEYS;
extern const int NO_SUCH_COLUMN_IN_TABLE;
extern const int INCOMPATIBLE_TYPE_OF_JOIN;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
@ -285,7 +285,8 @@ private:
#undef M
default:
throw Exception("Unknown JOIN keys variant for limited use", ErrorCodes::UNKNOWN_SET_DATA_VARIANT);
throw Exception("Unsupported JOIN keys in StorageJoin. Type: " + toString(static_cast<UInt32>(parent.type)),
ErrorCodes::UNSUPPORTED_JOIN_KEYS);
}
if (!rows_added)

View File

@ -137,7 +137,7 @@ create table summing_merge_tree_null (
) engine=Null;
create materialized view summing_merge_tree_aggregate_function (
d materialized today(),
d Date,
k UInt64,
c UInt64,
u AggregateFunction(uniq, UInt64)

View File

@ -0,0 +1,4 @@
2
3
3
4

View File

@ -0,0 +1,19 @@
DROP TABLE IF EXISTS test.src;
DROP TABLE IF EXISTS test.dst1;
DROP TABLE IF EXISTS test.dst2;
USE test;
CREATE TABLE src (x UInt8) ENGINE Memory;
CREATE TABLE dst1 (x UInt8) ENGINE Memory;
CREATE MATERIALIZED VIEW src_to_dst1 TO dst1 AS SELECT x + 1 as x FROM src;
CREATE MATERIALIZED VIEW dst2 ENGINE Memory AS SELECT x + 1 as x FROM dst1;
INSERT INTO src VALUES (1), (2);
SELECT * FROM dst1 ORDER BY x;
SELECT * FROM dst2 ORDER BY x;
DROP TABLE src;
DROP TABLE src_to_dst1;
DROP TABLE dst1;
DROP TABLE dst2;