mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Clarification and cleanup
This commit is contained in:
parent
0f98ee163e
commit
5a9acac12f
@ -1,5 +1,6 @@
|
||||
#include <Interpreters/FillingRow.h>
|
||||
#include <Common/FieldVisitorsAccurateComparison.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -107,4 +108,22 @@ void FillingRow::initFromDefaults(size_t from_pos)
|
||||
row[i] = getFillDescription(i).fill_from;
|
||||
}
|
||||
|
||||
String FillingRow::dump() const
|
||||
{
|
||||
WriteBufferFromOwnString out;
|
||||
for (size_t i = 0; i < row.size(); ++i)
|
||||
{
|
||||
if (i != 0)
|
||||
out << ", ";
|
||||
out << row[i].dump();
|
||||
}
|
||||
return out.str();
|
||||
}
|
||||
|
||||
WriteBuffer & operator<<(WriteBuffer & out, const FillingRow & row)
|
||||
{
|
||||
out << row.dump();
|
||||
return out;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,7 +1,5 @@
|
||||
#pragma once
|
||||
#include <Core/SortDescription.h>
|
||||
#include <Core/InterpolateDescription.h>
|
||||
#include <Columns/IColumn.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -34,9 +32,13 @@ public:
|
||||
int getDirection(size_t index) const { return sort_description[index].direction; }
|
||||
FillColumnDescription & getFillDescription(size_t index) { return sort_description[index].fill_description; }
|
||||
|
||||
String dump() const;
|
||||
|
||||
private:
|
||||
Row row;
|
||||
SortDescription sort_description;
|
||||
};
|
||||
|
||||
WriteBuffer & operator<<(WriteBuffer & out, const FillingRow & row);
|
||||
|
||||
}
|
||||
|
@ -8,11 +8,29 @@
|
||||
#include <Functions/FunctionDateOrDateTimeAddInterval.h>
|
||||
#include <Common/FieldVisitorSum.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
constexpr bool debug_logging_enabled = true;
|
||||
|
||||
template <typename T>
|
||||
void logDebug(String key, const T & value, const char * separator = " : ")
|
||||
{
|
||||
if constexpr (debug_logging_enabled)
|
||||
{
|
||||
WriteBufferFromOwnString ss;
|
||||
if constexpr (std::is_pointer_v<T>)
|
||||
ss << *value;
|
||||
else
|
||||
ss << value;
|
||||
|
||||
LOG_DEBUG(&Poco::Logger::get("FillingTransform"), "{}{}{}", key, separator, ss.str());
|
||||
}
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INVALID_WITH_FILL_EXPRESSION;
|
||||
@ -233,6 +251,9 @@ FillingTransform::FillingTransform(
|
||||
interpolate_column_positions.push_back(header_.getPositionByName(name));
|
||||
}
|
||||
|
||||
/// prepair() is overrididen to cover cases when we need to generate rows for no input (so chunk in transform() will have no rows)
|
||||
/// (1) when all data are processed and WITH FILL .. TO is provided, we may need to generate suffix
|
||||
/// (2) for empty result set when WITH FILL FROM .. TO is provided (see PR #30888) (first and generate_suffix are both true)
|
||||
IProcessor::Status FillingTransform::prepare()
|
||||
{
|
||||
if (input.isFinished() && !output.isFinished() && !has_input && !generate_suffix)
|
||||
@ -244,8 +265,8 @@ IProcessor::Status FillingTransform::prepare()
|
||||
|
||||
if (first || filling_row < next_row)
|
||||
{
|
||||
/// Output if has data.
|
||||
if (has_output)
|
||||
/// push output data to output port if we can
|
||||
if (has_output && output.canPush())
|
||||
{
|
||||
output.pushData(std::move(output_data));
|
||||
has_output = false;
|
||||
@ -370,9 +391,6 @@ void FillingTransform::initColumns(
|
||||
|
||||
void FillingTransform::transform(Chunk & chunk)
|
||||
{
|
||||
if (!chunk.hasRows() && !generate_suffix)
|
||||
return;
|
||||
|
||||
Columns old_fill_columns;
|
||||
Columns old_interpolate_columns;
|
||||
Columns old_other_columns;
|
||||
@ -385,6 +403,8 @@ void FillingTransform::transform(Chunk & chunk)
|
||||
|
||||
if (generate_suffix)
|
||||
{
|
||||
chassert(!chunk.hasRows());
|
||||
|
||||
const auto & empty_columns = input.getHeader().getColumns();
|
||||
initColumns(
|
||||
empty_columns,
|
||||
@ -405,11 +425,10 @@ void FillingTransform::transform(Chunk & chunk)
|
||||
insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block);
|
||||
}
|
||||
|
||||
interpolate(result_columns, interpolate_block);
|
||||
while (filling_row.next(next_row))
|
||||
{
|
||||
insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block);
|
||||
interpolate(result_columns, interpolate_block);
|
||||
insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block);
|
||||
}
|
||||
|
||||
size_t num_output_rows = result_columns[0]->size();
|
||||
@ -453,7 +472,12 @@ void FillingTransform::transform(Chunk & chunk)
|
||||
|
||||
for (size_t row_ind = 0; row_ind < num_rows; ++row_ind)
|
||||
{
|
||||
logDebug("row", row_ind);
|
||||
logDebug("filling_row", filling_row);
|
||||
logDebug("next_row", next_row);
|
||||
|
||||
should_insert_first = next_row < filling_row;
|
||||
logDebug("should_insert_first", true);
|
||||
|
||||
for (size_t i = 0, size = filling_row.size(); i < size; ++i)
|
||||
{
|
||||
@ -465,6 +489,7 @@ void FillingTransform::transform(Chunk & chunk)
|
||||
else
|
||||
next_row[i] = fill_to;
|
||||
}
|
||||
logDebug("next_row updated", next_row);
|
||||
|
||||
/// A case, when at previous step row was initialized from defaults 'fill_from' values
|
||||
/// and probably we need to insert it to block.
|
||||
@ -474,11 +499,10 @@ void FillingTransform::transform(Chunk & chunk)
|
||||
insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block);
|
||||
}
|
||||
|
||||
interpolate(result_columns, interpolate_block);
|
||||
while (filling_row.next(next_row))
|
||||
{
|
||||
insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block);
|
||||
interpolate(result_columns, interpolate_block);
|
||||
insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block);
|
||||
}
|
||||
|
||||
copyRowFromColumns(res_fill_columns, old_fill_columns, row_ind);
|
||||
|
@ -1,9 +1,9 @@
|
||||
#pragma once
|
||||
|
||||
#include <Processors/ISimpleTransform.h>
|
||||
#include <Core/SortDescription.h>
|
||||
#include <Core/InterpolateDescription.h>
|
||||
#include <Core/SortDescription.h>
|
||||
#include <Interpreters/FillingRow.h>
|
||||
#include <Processors/ISimpleTransform.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -29,7 +29,7 @@ protected:
|
||||
|
||||
private:
|
||||
void saveLastRow(const MutableColumns & result_columns);
|
||||
void interpolate(const MutableColumns& result_columns, Block & interpolate_block);
|
||||
void interpolate(const MutableColumns & result_columns, Block & interpolate_block);
|
||||
|
||||
using MutableColumnRawPtrs = std::vector<IColumn *>;
|
||||
void initColumns(
|
||||
@ -54,8 +54,8 @@ private:
|
||||
Positions other_column_positions;
|
||||
std::vector<std::pair<size_t, NameAndTypePair>> input_positions; /// positions in result columns required for actions
|
||||
ExpressionActionsPtr interpolate_actions;
|
||||
bool first = true;
|
||||
bool generate_suffix = false;
|
||||
bool first = true; /// flag to determine if transform is/will be called for the first time
|
||||
bool generate_suffix = false; /// flag to determine if we need to generate filling rows after latest chunk is processed (only for WITH FILL ... TO)
|
||||
|
||||
Columns last_row;
|
||||
|
||||
|
@ -1,4 +1,9 @@
|
||||
*** table without fill to compare ***
|
||||
--{ echoOn }
|
||||
DROP TABLE IF EXISTS fill;
|
||||
CREATE TABLE fill (date Date, val Int, str String) ENGINE = Memory;
|
||||
INSERT INTO fill VALUES (toDate('2019-05-24'), 13, 'sd0')(toDate('2019-05-10'), 16, 'vp7')(toDate('2019-05-25'), 17, '0ei')(toDate('2019-05-30'), 18, '3kd')(toDate('2019-05-15'), 27, 'enb')(toDate('2019-06-04'), 5, '6az')(toDate('2019-05-23'), 15, '01v')(toDate('2019-05-08'), 28, 'otf')(toDate('2019-05-19'), 20, 'yfh')(toDate('2019-05-07'), 26, '2ke')(toDate('2019-05-07'), 18, 'prh')(toDate('2019-05-09'), 25, '798')(toDate('2019-05-10'), 1, 'myj')(toDate('2019-05-11'), 18, '3s2')(toDate('2019-05-23'), 29, '72y');
|
||||
-- *** table without fill to compare ***
|
||||
SELECT * FROM fill ORDER BY date, val;
|
||||
2019-05-07 18 prh
|
||||
2019-05-07 26 2ke
|
||||
2019-05-08 28 otf
|
||||
@ -14,7 +19,9 @@
|
||||
2019-05-25 17 0ei
|
||||
2019-05-30 18 3kd
|
||||
2019-06-04 5 6az
|
||||
*** date WITH FILL, val ***
|
||||
-- Some useful cases
|
||||
|
||||
SELECT * FROM fill ORDER BY date WITH FILL, val;
|
||||
2019-05-07 18 prh
|
||||
2019-05-07 26 2ke
|
||||
2019-05-08 28 otf
|
||||
@ -47,7 +54,7 @@
|
||||
2019-06-02 0
|
||||
2019-06-03 0
|
||||
2019-06-04 5 6az
|
||||
*** date WITH FILL FROM 2019-05-01 TO 2019-05-31, val WITH FILL ***
|
||||
SELECT * FROM fill ORDER BY date WITH FILL FROM toDate('2019-05-01') TO toDate('2019-05-31'), val WITH FILL;
|
||||
2019-05-01 0
|
||||
2019-05-02 0
|
||||
2019-05-03 0
|
||||
@ -116,7 +123,7 @@
|
||||
2019-05-29 0
|
||||
2019-05-30 18 3kd
|
||||
2019-06-04 5 6az
|
||||
*** date DESC WITH FILL, val WITH FILL FROM 1 TO 6 ***
|
||||
SELECT * FROM fill ORDER BY date DESC WITH FILL, val WITH FILL FROM 1 TO 6;
|
||||
2019-06-04 1
|
||||
2019-06-04 2
|
||||
2019-06-04 3
|
||||
@ -275,7 +282,9 @@
|
||||
2019-05-07 5
|
||||
2019-05-07 18 prh
|
||||
2019-05-07 26 2ke
|
||||
*** date DESC WITH FILL TO 2019-05-01 STEP -2, val DESC WITH FILL FROM 10 TO -5 STEP -3 ***
|
||||
-- Some weird cases
|
||||
|
||||
SELECT * FROM fill ORDER BY date DESC WITH FILL TO toDate('2019-05-01') STEP -2, val DESC WITH FILL FROM 10 TO -5 STEP -3;
|
||||
2019-06-04 10
|
||||
2019-06-04 7
|
||||
2019-06-04 5 6az
|
||||
@ -376,7 +385,7 @@
|
||||
2019-05-03 4
|
||||
2019-05-03 1
|
||||
2019-05-03 -2
|
||||
*** date WITH FILL TO 2019-06-23 STEP 3, val WITH FILL FROM -10 STEP 2
|
||||
SELECT * FROM fill ORDER BY date WITH FILL TO toDate('2019-06-23') STEP 3, val WITH FILL FROM -10 STEP 2;
|
||||
2019-05-07 -10
|
||||
2019-05-07 -8
|
||||
2019-05-07 -6
|
||||
@ -463,14 +472,18 @@
|
||||
2019-06-15 -10
|
||||
2019-06-18 -10
|
||||
2019-06-21 -10
|
||||
*** table without fill to compare ***
|
||||
DROP TABLE fill;
|
||||
CREATE TABLE fill (a UInt32, b Int32) ENGINE = Memory;
|
||||
INSERT INTO fill VALUES (1, -2), (1, 3), (3, 2), (5, -1), (6, 5), (8, 0);
|
||||
-- *** table without fill to compare ***
|
||||
SELECT * FROM fill ORDER BY a, b;
|
||||
1 -2
|
||||
1 3
|
||||
3 2
|
||||
5 -1
|
||||
6 5
|
||||
8 0
|
||||
*** a WITH FILL, b WITH fill ***
|
||||
SELECT * FROM fill ORDER BY a WITH FILL, b WITH fill;
|
||||
1 -2
|
||||
1 -1
|
||||
1 0
|
||||
@ -484,7 +497,7 @@
|
||||
6 5
|
||||
7 0
|
||||
8 0
|
||||
*** a WITH FILL, b WITH fill TO 6 STEP 2 ***
|
||||
SELECT * FROM fill ORDER BY a WITH FILL, b WITH fill TO 6 STEP 2;
|
||||
1 -2
|
||||
1 0
|
||||
1 2
|
||||
@ -503,3 +516,8 @@
|
||||
8 0
|
||||
8 2
|
||||
8 4
|
||||
SELECT * FROM fill ORDER BY a WITH FILL STEP -1; -- { serverError 475 }
|
||||
SELECT * FROM fill ORDER BY a WITH FILL FROM 10 TO 1; -- { serverError 475 }
|
||||
SELECT * FROM fill ORDER BY a DESC WITH FILL FROM 1 TO 10; -- { serverError 475 }
|
||||
SELECT * FROM fill ORDER BY a WITH FILL FROM -10 to 10; -- { serverError 475 }
|
||||
DROP TABLE fill;
|
||||
|
@ -1,40 +1,34 @@
|
||||
--{ echoOn }
|
||||
DROP TABLE IF EXISTS fill;
|
||||
CREATE TABLE fill (date Date, val Int, str String) ENGINE = Memory;
|
||||
INSERT INTO fill VALUES (toDate('2019-05-24'), 13, 'sd0')(toDate('2019-05-10'), 16, 'vp7')(toDate('2019-05-25'), 17, '0ei')(toDate('2019-05-30'), 18, '3kd')(toDate('2019-05-15'), 27, 'enb')(toDate('2019-06-04'), 5, '6az')(toDate('2019-05-23'), 15, '01v')(toDate('2019-05-08'), 28, 'otf')(toDate('2019-05-19'), 20, 'yfh')(toDate('2019-05-07'), 26, '2ke')(toDate('2019-05-07'), 18, 'prh')(toDate('2019-05-09'), 25, '798')(toDate('2019-05-10'), 1, 'myj')(toDate('2019-05-11'), 18, '3s2')(toDate('2019-05-23'), 29, '72y');
|
||||
|
||||
SELECT '*** table without fill to compare ***';
|
||||
-- *** table without fill to compare ***
|
||||
SELECT * FROM fill ORDER BY date, val;
|
||||
|
||||
-- Some useful cases
|
||||
|
||||
SELECT '*** date WITH FILL, val ***';
|
||||
SELECT * FROM fill ORDER BY date WITH FILL, val;
|
||||
|
||||
SELECT '*** date WITH FILL FROM 2019-05-01 TO 2019-05-31, val WITH FILL ***';
|
||||
SELECT * FROM fill ORDER BY date WITH FILL FROM toDate('2019-05-01') TO toDate('2019-05-31'), val WITH FILL;
|
||||
|
||||
SELECT '*** date DESC WITH FILL, val WITH FILL FROM 1 TO 6 ***';
|
||||
SELECT * FROM fill ORDER BY date DESC WITH FILL, val WITH FILL FROM 1 TO 6;
|
||||
|
||||
-- Some weird cases
|
||||
|
||||
SELECT '*** date DESC WITH FILL TO 2019-05-01 STEP -2, val DESC WITH FILL FROM 10 TO -5 STEP -3 ***';
|
||||
SELECT * FROM fill ORDER BY date DESC WITH FILL TO toDate('2019-05-01') STEP -2, val DESC WITH FILL FROM 10 TO -5 STEP -3;
|
||||
|
||||
SELECT '*** date WITH FILL TO 2019-06-23 STEP 3, val WITH FILL FROM -10 STEP 2';
|
||||
SELECT * FROM fill ORDER BY date WITH FILL TO toDate('2019-06-23') STEP 3, val WITH FILL FROM -10 STEP 2;
|
||||
|
||||
DROP TABLE fill;
|
||||
CREATE TABLE fill (a UInt32, b Int32) ENGINE = Memory;
|
||||
INSERT INTO fill VALUES (1, -2), (1, 3), (3, 2), (5, -1), (6, 5), (8, 0);
|
||||
|
||||
SELECT '*** table without fill to compare ***';
|
||||
-- *** table without fill to compare ***
|
||||
SELECT * FROM fill ORDER BY a, b;
|
||||
|
||||
SELECT '*** a WITH FILL, b WITH fill ***';
|
||||
SELECT * FROM fill ORDER BY a WITH FILL, b WITH fill;
|
||||
|
||||
SELECT '*** a WITH FILL, b WITH fill TO 6 STEP 2 ***';
|
||||
SELECT * FROM fill ORDER BY a WITH FILL, b WITH fill TO 6 STEP 2;
|
||||
|
||||
SELECT * FROM fill ORDER BY a WITH FILL STEP -1; -- { serverError 475 }
|
||||
|
Loading…
Reference in New Issue
Block a user