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

This commit is contained in:
Alexey Milovidov 2018-03-07 00:00:48 +03:00
commit dc3364f33f
12 changed files with 39 additions and 33 deletions

View File

@ -1,6 +1,6 @@
# This strings autochanged from release_lib.sh:
set(VERSION_DESCRIBE v1.1.54356-testing)
set(VERSION_REVISION 54356)
set(VERSION_DESCRIBE v1.1.54357-testing)
set(VERSION_REVISION 54357)
# end of autochange
set (VERSION_MAJOR 1)

View File

@ -33,19 +33,18 @@ FilterBlockInputStream::FilterBlockInputStream(const BlockInputStreamPtr & input
expression->execute(header);
filter_column = header.getPositionByName(filter_column_name);
auto & column_elem = header.safeGetByPosition(filter_column);
/// Isn't the filter already constant?
ColumnPtr column = header.safeGetByPosition(filter_column).column;
if (column)
constant_filter_description = ConstantFilterDescription(*column);
if (column_elem.column)
constant_filter_description = ConstantFilterDescription(*column_elem.column);
if (!constant_filter_description.always_false
&& !constant_filter_description.always_true)
{
/// Replace the filter column to a constant with value 1.
auto & header_filter_elem = header.getByPosition(filter_column);
header_filter_elem.column = header_filter_elem.type->createColumnConst(header.rows(), UInt64(1));
FilterDescription filter_description_check(*column_elem.column);
column_elem.column = column_elem.type->createColumnConst(header.rows(), UInt64(1));
}
}

View File

@ -182,10 +182,12 @@ void GraphiteRollupSortedBlockInputStream::merge(MutableColumns & merged_columns
/// Within all rows with same key, we should leave only one row with maximum version;
/// and for rows with same maximum version - only last row.
UInt64 next_version = next_cursor->all_columns[version_column_num]->get64(next_cursor->pos);
if (is_new_key || next_version >= current_subgroup_max_version)
if (is_new_key
|| next_cursor->all_columns[version_column_num]->compareAt(
next_cursor->pos, current_subgroup_newest_row.row_num,
*(*current_subgroup_newest_row.columns)[version_column_num],
/* nan_direction_hint = */ 1) >= 0)
{
current_subgroup_max_version = next_version;
setRowRef(current_subgroup_newest_row, next_cursor);
/// Small hack: group and subgroups have the same path, so we can set current_group_path here instead of startNextGroup
@ -244,7 +246,8 @@ void GraphiteRollupSortedBlockInputStream::finishCurrentGroup(MutableColumns & m
{
/// Insert calculated values of the columns `time`, `value`, `version`.
merged_columns[time_column_num]->insert(UInt64(current_time_rounded));
merged_columns[version_column_num]->insert(current_subgroup_max_version);
merged_columns[version_column_num]->insertFrom(
*(*current_subgroup_newest_row.columns)[version_column_num], current_subgroup_newest_row.row_num);
if (aggregate_state_created)
{

View File

@ -184,7 +184,6 @@ private:
/// Last row with maximum version for current primary key (time bucket).
RowRef current_subgroup_newest_row;
UInt64 current_subgroup_max_version = 0;
/// Time of last read row
time_t current_time = 0;

View File

@ -105,6 +105,12 @@ protected:
return !(*this == other);
}
void reset()
{
RowRef empty;
swap(empty);
}
bool empty() const { return columns == nullptr; }
size_t size() const { return empty() ? 0 : columns->size(); }
};

View File

@ -73,10 +73,6 @@ void ReplacingSortedBlockInputStream::merge(MutableColumns & merged_columns, std
if (current_key.empty())
setPrimaryKeyRef(current_key, current);
UInt64 version = version_column_number != -1
? current->all_columns[version_column_number]->get64(current->pos)
: 0;
setPrimaryKeyRef(next_key, current);
bool key_differs = next_key != current_key;
@ -89,9 +85,9 @@ void ReplacingSortedBlockInputStream::merge(MutableColumns & merged_columns, std
if (key_differs)
{
max_version = 0;
/// Write the data for the previous primary key.
insertRow(merged_columns, merged_rows);
selected_row.reset();
current_key.swap(next_key);
}
@ -101,9 +97,13 @@ void ReplacingSortedBlockInputStream::merge(MutableColumns & merged_columns, std
current_row_sources.emplace_back(current.impl->order, true);
/// A non-strict comparison, since we select the last row for the same version values.
if (version >= max_version)
if (version_column_number == -1
|| selected_row.empty()
|| current->all_columns[version_column_number]->compareAt(
current->pos, selected_row.row_num,
*(*selected_row.columns)[version_column_number],
/* nan_direction_hint = */ 1) >= 0)
{
max_version = version;
max_pos = current_pos;
setRowRef(selected_row, current);
}

View File

@ -43,8 +43,6 @@ private:
RowRef next_key;
/// Last row with maximum version for current primary key.
RowRef selected_row;
/// Max version for current primary key.
UInt64 max_version = 0;
/// The position (into current_row_sources) of the row with the highest version.
size_t max_pos = 0;

View File

@ -10,12 +10,12 @@ String ASTKillQueryQuery::getID() const
void ASTKillQueryQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << "KILL QUERY WHERE ";
settings.ostr << (settings.hilite ? hilite_keyword : "") << "KILL QUERY WHERE " << (settings.hilite ? hilite_none : "");
if (where_expression)
where_expression->formatImpl(settings, state, frame);
settings.ostr << " " << (test ? "TEST" : (sync ? "SYNC" : "ASYNC"));
settings.ostr << " " << (settings.hilite ? hilite_keyword : "") << (test ? "TEST" : (sync ? "SYNC" : "ASYNC")) << (settings.hilite ? hilite_none : "");
}
}

View File

@ -111,7 +111,8 @@ MergeTreeData::MergeTreeData(
data_parts_by_info(data_parts_indexes.get<TagByInfo>()),
data_parts_by_state_and_info(data_parts_indexes.get<TagByStateAndInfo>())
{
merging_params.check(columns);
/// NOTE: using the same columns list as is read when performing actual merges.
merging_params.check(getColumnsList());
if (!primary_expr_ast)
throw Exception("Primary key cannot be empty", ErrorCodes::BAD_ARGUMENTS);
@ -355,10 +356,11 @@ void MergeTreeData::MergingParams::check(const NamesAndTypesList & columns) cons
{
if (column.name == version_column)
{
if (!column.type->isUnsignedInteger() && !column.type->isDateOrDateTime())
throw Exception("Version column (" + version_column + ")"
" for storage " + storage + " must have type of UInt family or Date or DateTime."
" Provided column of type " + column.type->getName() + ".", ErrorCodes::BAD_TYPE_OF_FIELD);
if (!column.type->canBeUsedAsVersion())
throw Exception("The column " + version_column +
" cannot be used as a version column for storage " + storage +
" because it is of type " + column.type->getName() +
" (must be of an integer type or of type Date or DateTime)", ErrorCodes::BAD_TYPE_OF_FIELD);
miss_column = false;
break;
}

View File

@ -3,7 +3,6 @@
<!-- Default profile settings. -->
<default>
<log_queries>1</log_queries>
<distributed_ddl_allow_replicated_alter>1</distributed_ddl_allow_replicated_alter>
</default>
</profiles>
</yandex>

4
debian/changelog vendored
View File

@ -1,5 +1,5 @@
clickhouse (1.1.54356) unstable; urgency=low
clickhouse (1.1.54357) unstable; urgency=low
* Modified source code
-- <robot-metrika-test@yandex-team.ru> Tue, 06 Mar 2018 00:07:31 +0300
-- <robot-metrika-test@yandex-team.ru> Tue, 06 Mar 2018 19:20:07 +0300

2
debian/compat vendored
View File

@ -1 +1 @@
9
8