Skip to content

Commit

Permalink
20.3 backports part 1 (#11018)
Browse files Browse the repository at this point in the history
* Merge pull request #10268 from ClickHouse/max-rows-to-sort

Added failing tests about "max_rows_to_sort" setting.

(cherry picked from commit f7b1263)

* Merge pull request #10560 from Enmk/DateTime64_fixes

Fixed comparing DateTime64 in WHERE against String value

(cherry picked from commit 67efc7f)

* Merge pull request #10563 from azat/SELECT-ALIAS-CAST

Fix SELECT of column ALIAS which default expression type different from column type

(cherry picked from commit 15e38c8)

* Merge pull request #10569 from zhang2014/fix/ISSUES-10551

ISSUES-10551 add backward compatibility for create bloom filter index

(cherry picked from commit db4c235)

* Merge pull request #10659 from ClickHouse/fix_multiple_simultaneous_alters

Fix alter-mutations assignment

(cherry picked from commit 5eebe8c)

* Merge pull request #10711 from ClickHouse/h3-range-check

Range check in function h3EdgeAngle

(cherry picked from commit d0b61f9)

* Merge pull request #10741 from hczhcz/patch-0422

Fix OrNull and OrDefault

(cherry picked from commit 699ef4f)

* Merge pull request #10757 from ClickHouse/fix-parallel-mv

Fix parallel MV

(cherry picked from commit ef1c7da)

* Merge pull request #10791 from oandrew/key-condition-source-type

Use src_type for conversion in KeyCondition

(cherry picked from commit a133389)

* Merge pull request #10798 from ClickHouse/fix-date-lut-msan-ubsan

Fix UBSan and MSan report in DateLUT

(cherry picked from commit a2f220f)

* Merge pull request #10821 from ClickHouse/fix-odbc-bridge-clickhouse

Fix the issue with ODBC bridge and identifier_quoting_style = None #7984

(cherry picked from commit 5115ac2)

* Merge pull request #10826 from azat/block-sort-fix

Fix columns order after Block::sortColumns()

(cherry picked from commit 75607db)

* Merge pull request #10849 from ClickHouse/fix_optimize_and_alter_hangs

Fix mutations and OPTIMIZE hangs when replica becomes inactive

(cherry picked from commit 623b2e5)

* Merge pull request #10870 from azat/fix-SIGSEGV-in-hash-table-for-string

Fix SIGSEGV in StringHashTable (if such key does not exist)

(cherry picked from commit 0433d9c)

* Merge pull request #10952 from ClickHouse/fix-distributed-queries-incompatibility-19.16-20.1

Fix incompatibility of two-level aggregation between 19.16 and 20.1

(cherry picked from commit c957154)

* Merge pull request #10964 from ObjatieGroba/bugfix_raw_column_size

Fix incorrect Column byte size

(cherry picked from commit b6b1947)

Co-authored-by: alexey-milovidov <milovidov@yandex-team.ru>
Co-authored-by: alesapin <alesapin@gmail.com>
Co-authored-by: Anton Popov <pad11rus@gmail.com>
Co-authored-by: Nikolai Kochetov <KochetovNicolai@users.noreply.github.com>
  • Loading branch information
5 people committed May 23, 2020
1 parent ab3adc4 commit d3f9095
Show file tree
Hide file tree
Showing 61 changed files with 882 additions and 115 deletions.
2 changes: 1 addition & 1 deletion base/common/DateLUTImpl.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -137,7 +137,7 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_)
/// Fill excessive part of lookup table. This is needed only to simplify handling of overflow cases.
while (i < DATE_LUT_SIZE)
{
lut[i] = lut[DATE_LUT_MAX_DAY_NUM];
lut[i] = lut[i - 1];
++i;
}

Expand Down
8 changes: 8 additions & 0 deletions src/AggregateFunctions/AggregateFunctionOrFill.h
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,8 @@
#include <Columns/ColumnNullable.h>
#include <Common/typeid_cast.h>
#include <DataTypes/DataTypeNullable.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>


namespace DB
Expand Down Expand Up @@ -102,13 +104,17 @@ class AggregateFunctionOrFill final : public IAggregateFunctionHelper<AggregateF
Arena * arena) const override
{
nested_function->merge(place, rhs, arena);

place[size_of_data] |= rhs[size_of_data];
}

void serialize(
ConstAggregateDataPtr place,
WriteBuffer & buf) const override
{
nested_function->serialize(place, buf);

writeChar(place[size_of_data], buf);
}

void deserialize(
Expand All @@ -117,6 +123,8 @@ class AggregateFunctionOrFill final : public IAggregateFunctionHelper<AggregateF
Arena * arena) const override
{
nested_function->deserialize(place, buf, arena);

readChar(place[size_of_data], buf);
}

DataTypePtr getReturnType() const override
Expand Down
2 changes: 1 addition & 1 deletion src/Columns/ColumnDecimal.h
Original file line number Diff line number Diff line change
Expand Up @@ -112,7 +112,7 @@ class ColumnDecimal final : public COWHelper<ColumnVectorHelper, ColumnDecimal<T

Field operator[](size_t n) const override { return DecimalField(data[n], scale); }

StringRef getRawData() const override { return StringRef(reinterpret_cast<const char*>(data.data()), data.size()); }
StringRef getRawData() const override { return StringRef(reinterpret_cast<const char*>(data.data()), byteSize()); }
StringRef getDataAt(size_t n) const override { return StringRef(reinterpret_cast<const char *>(&data[n]), sizeof(data[n])); }
void get(size_t n, Field & res) const override { res = (*this)[n]; }
bool getBool(size_t n) const override { return bool(data[n]); }
Expand Down
2 changes: 1 addition & 1 deletion src/Columns/ColumnVector.h
Original file line number Diff line number Diff line change
Expand Up @@ -261,7 +261,7 @@ class ColumnVector final : public COWHelper<ColumnVectorHelper, ColumnVector<T>>

bool isFixedAndContiguous() const override { return true; }
size_t sizeOfValueIfFixed() const override { return sizeof(T); }
StringRef getRawData() const override { return StringRef(reinterpret_cast<const char*>(data.data()), data.size()); }
StringRef getRawData() const override { return StringRef(reinterpret_cast<const char*>(data.data()), byteSize()); }


bool structureEquals(const IColumn & rhs) const override
Expand Down
6 changes: 5 additions & 1 deletion src/Common/HashTable/StringHashTable.h
Original file line number Diff line number Diff line change
Expand Up @@ -333,7 +333,11 @@ class StringHashTable : private boost::noncopyable
template <typename Submap, typename SubmapKey>
auto ALWAYS_INLINE operator()(Submap & map, const SubmapKey & key, size_t hash)
{
return &map.find(key, hash)->getMapped();
auto it = map.find(key, hash);
if (!it)
return decltype(&it->getMapped()){};
else
return &it->getMapped();
}
};

Expand Down
47 changes: 25 additions & 22 deletions src/Common/ZooKeeper/ZooKeeper.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -622,51 +622,54 @@ namespace
{
struct WaitForDisappearState
{
int32_t code = 0;
int32_t event_type = 0;
std::atomic_int32_t code = 0;
std::atomic_int32_t event_type = 0;
Poco::Event event;
};
using WaitForDisappearStatePtr = std::shared_ptr<WaitForDisappearState>;
}

void ZooKeeper::waitForDisappear(const std::string & path)
bool ZooKeeper::waitForDisappear(const std::string & path, const WaitCondition & condition)
{
WaitForDisappearStatePtr state = std::make_shared<WaitForDisappearState>();

while (true)
auto callback = [state](const Coordination::ExistsResponse & response)
{
auto callback = [state](const Coordination::ExistsResponse & response)
{
state->code = response.error;
if (state->code)
state->event.set();
};
state->code = response.error;
if (state->code)
state->event.set();
};

auto watch = [state](const Coordination::WatchResponse & response)
auto watch = [state](const Coordination::WatchResponse & response)
{
if (!state->code)
{
state->code = response.error;
if (!state->code)
{
state->code = response.error;
if (!state->code)
state->event_type = response.type;
state->event.set();
}
};
state->event_type = response.type;
state->event.set();
}
};

while (!condition || !condition())
{
/// NOTE: if the node doesn't exist, the watch will leak.

impl->exists(path, callback, watch);
state->event.wait();
if (!condition)
state->event.wait();
else if (!state->event.tryWait(1000))
continue;

if (state->code == Coordination::ZNONODE)
return;
return true;

if (state->code)
throw KeeperException(state->code, path);

if (state->event_type == Coordination::DELETED)
return;
return true;
}
return false;
}

ZooKeeperPtr ZooKeeper::startNewSession() const
Expand Down
5 changes: 4 additions & 1 deletion src/Common/ZooKeeper/ZooKeeper.h
Original file line number Diff line number Diff line change
Expand Up @@ -181,8 +181,11 @@ class ZooKeeper
/// Remove all children nodes (non recursive).
void removeChildren(const std::string & path);

using WaitCondition = std::function<bool()>;
/// Wait for the node to disappear or return immediately if it doesn't exist.
void waitForDisappear(const std::string & path);
/// If condition is speficied, it is used to return early (when condition returns false)
/// The function returns true if waited and false if waiting was interrupted by condition.
bool waitForDisappear(const std::string & path, const WaitCondition & condition = {});

/// Async interface (a small subset of operations is implemented).
///
Expand Down
16 changes: 14 additions & 2 deletions src/Core/Block.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -403,8 +403,20 @@ Block Block::sortColumns() const
{
Block sorted_block;

for (const auto & name : index_by_name)
sorted_block.insert(data[name.second]);
/// std::unordered_map (index_by_name) cannot be used to guarantee the sort order
std::vector<decltype(index_by_name.begin())> sorted_index_by_name(index_by_name.size());
{
size_t i = 0;
for (auto it = index_by_name.begin(); it != index_by_name.end(); ++it)
sorted_index_by_name[i++] = it;
}
std::sort(sorted_index_by_name.begin(), sorted_index_by_name.end(), [](const auto & lhs, const auto & rhs)
{
return lhs->first < rhs->first;
});

for (const auto & it : sorted_index_by_name)
sorted_block.insert(data[it->second]);

return sorted_block;
}
Expand Down
2 changes: 1 addition & 1 deletion src/Core/Defines.h
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@
/// Minimum revision with exactly the same set of aggregation methods and rules to select them.
/// Two-level (bucketed) aggregation is incompatible if servers are inconsistent in these rules
/// (keys will be placed in different buckets and result will not be fully aggregated).
#define DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD 54408
#define DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD 54431
#define DBMS_MIN_REVISION_WITH_COLUMN_DEFAULTS_METADATA 54410

#define DBMS_MIN_REVISION_WITH_LOW_CARDINALITY_TYPE 54405
Expand Down
26 changes: 24 additions & 2 deletions src/DataStreams/PushingToViewsBlockOutputStream.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
else
out = std::make_shared<PushingToViewsBlockOutputStream>(dependent_table, *views_context, ASTPtr());

views.emplace_back(ViewInfo{std::move(query), database_table, std::move(out)});
views.emplace_back(ViewInfo{std::move(query), database_table, std::move(out), nullptr});
}

/* Do not push to destination table if the flag is set */
Expand Down Expand Up @@ -159,7 +159,12 @@ void PushingToViewsBlockOutputStream::write(const Block & block)
{
// Process sequentially
for (size_t view_num = 0; view_num < views.size(); ++view_num)
{
process(block, view_num);

if (views[view_num].exception)
std::rethrow_exception(views[view_num].exception);
}
}
}

Expand Down Expand Up @@ -187,8 +192,18 @@ void PushingToViewsBlockOutputStream::writeSuffix()
if (output)
output->writeSuffix();

std::exception_ptr first_exception;

for (auto & view : views)
{
if (view.exception)
{
if (!first_exception)
first_exception = view.exception;

continue;
}

try
{
view.out->writeSuffix();
Expand All @@ -199,6 +214,9 @@ void PushingToViewsBlockOutputStream::writeSuffix()
throw;
}
}

if (first_exception)
std::rethrow_exception(first_exception);
}

void PushingToViewsBlockOutputStream::flush()
Expand Down Expand Up @@ -266,7 +284,11 @@ void PushingToViewsBlockOutputStream::process(const Block & block, size_t view_n
catch (Exception & ex)
{
ex.addMessage("while pushing to view " + view.table_id.getNameForLogs());
throw;
view.exception = std::current_exception();
}
catch (...)
{
view.exception = std::current_exception();
}
}

Expand Down
1 change: 1 addition & 0 deletions src/DataStreams/PushingToViewsBlockOutputStream.h
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@ class PushingToViewsBlockOutputStream : public IBlockOutputStream
ASTPtr query;
StorageID table_id;
BlockOutputStreamPtr out;
std::exception_ptr exception;
};

std::vector<ViewInfo> views;
Expand Down
69 changes: 63 additions & 6 deletions src/Functions/FunctionsComparison.h
Original file line number Diff line number Diff line change
Expand Up @@ -1133,12 +1133,10 @@ class FunctionComparison : public IFunction

if (!((both_represented_by_number && !has_date) /// Do not allow compare date and number.
|| (left.isStringOrFixedString() && right.isStringOrFixedString())
|| (left.isDate() && right.isDate())
|| (left.isDate() && right.isString()) /// You can compare the date, datetime and an enumeration with a constant string.
|| (left.isString() && right.isDate())
|| (left.isDateTime() && right.isDateTime())
|| (left.isDateTime() && right.isString())
|| (left.isString() && right.isDateTime())
/// You can compare the date, datetime, or datatime64 and an enumeration with a constant string.
|| (left.isString() && right.isDateOrDateTime())
|| (left.isDateOrDateTime() && right.isString())
|| (left.isDateOrDateTime() && right.isDateOrDateTime() && left.idx == right.idx) /// only date vs date, or datetime vs datetime
|| (left.isUUID() && right.isUUID())
|| (left.isUUID() && right.isString())
|| (left.isString() && right.isUUID())
Expand Down Expand Up @@ -1236,6 +1234,65 @@ class FunctionComparison : public IFunction
{
executeTuple(block, result, col_with_type_and_name_left, col_with_type_and_name_right, input_rows_count);
}
else if (which_left.idx != which_right.idx
&& (which_left.isDateTime64() || which_right.isDateTime64())
&& (which_left.isStringOrFixedString() || which_right.isStringOrFixedString()))
{
/** Special case of comparing DateTime64 against a string.
*
* Can't be moved to executeDateOrDateTimeOrEnumOrUUIDWithConstString()
* since DateTime64 is basically a Decimal, but we do similar things, except type inference.
* Outline:
* - Extract string content
* - Parse it as a ColumnDateTime64 value (same type as DateTime64, means same precision)
* - Fabricate a column with type and name
* - Compare left and right comlumns as DateTime64 columns.
*/

const size_t datetime64_col_index = which_left.isDateTime64() ? 0 : 1;
const size_t string_col_index = which_left.isStringOrFixedString() ? 0 : 1;

const auto & datetime64_col_with_type_and_name = block.getByPosition(arguments[datetime64_col_index]);
const auto & string_col_with_type_and_name = block.getByPosition(arguments[string_col_index]);

if (!isColumnConst(*string_col_with_type_and_name.column))
throw Exception(getName() + ", illegal column type of argument #" + std::to_string(string_col_index)
+ " '" + string_col_with_type_and_name.name + "'"
" expected const String or const FixedString,"
" got " + string_col_with_type_and_name.type->getName(),
ErrorCodes::ILLEGAL_COLUMN);

if (datetime64_col_with_type_and_name.column->size() == 0 || string_col_with_type_and_name.column->size() == 0)
{
// For some reason, when both left and right columns are empty (dry run while building a header block)
// executeDecimal() fills result column with bogus value.
block.getByPosition(result).column = ColumnUInt8::create();
return;
}

auto parsed_tmp_column_holder = datetime64_col_with_type_and_name.type->createColumn();

{
const StringRef string_value = string_col_with_type_and_name.column->getDataAt(0);
ReadBufferFromMemory in(string_value.data, string_value.size);
datetime64_col_with_type_and_name.type->deserializeAsWholeText(*parsed_tmp_column_holder, in, FormatSettings{});

if (!in.eof())
throw Exception(getName() + ": String is too long for " + datetime64_col_with_type_and_name.type->getName() + " : " + string_value.toString(), ErrorCodes::TOO_LARGE_STRING_SIZE);
}

// It is necessary to wrap tmp column in ColumnConst to avoid overflow when comparing.
// (non-const columns are expected to have same number of rows as every other column in block).
const ColumnWithTypeAndName parsed_tmp_col_with_type_and_name{
ColumnConst::create(std::move(parsed_tmp_column_holder), 1),
datetime64_col_with_type_and_name.type,
string_col_with_type_and_name.name};

executeDecimal(block, result,
which_left.isDateTime64() ? datetime64_col_with_type_and_name : parsed_tmp_col_with_type_and_name,
which_right.isDateTime64() ? datetime64_col_with_type_and_name : parsed_tmp_col_with_type_and_name);

}
else if (isColumnedAsDecimal(left_type) || isColumnedAsDecimal(right_type))
{
// compare
Expand Down
8 changes: 8 additions & 0 deletions src/Functions/h3EdgeAngle.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -5,12 +5,15 @@
# include <Functions/FunctionFactory.h>
# include <Functions/IFunction.h>
# include <Common/typeid_cast.h>
# include <IO/WriteHelpers.h>
# include <ext/range.h>

# if __has_include(<h3/h3api.h>)
# include <h3/h3api.h>
# include <h3/constants.h>
# else
# include <h3api.h>
# include <constants.h>
# endif


Expand All @@ -19,7 +22,9 @@ namespace DB
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ARGUMENT_OUT_OF_BOUND;
}

class FunctionH3EdgeAngle : public IFunction
{
public:
Expand Down Expand Up @@ -54,6 +59,9 @@ class FunctionH3EdgeAngle : public IFunction
for (const auto row : ext::range(0, input_rows_count))
{
const int resolution = col_hindex->getUInt(row);
if (resolution > MAX_H3_RES)
throw Exception("The argument 'resolution' (" + toString(resolution) + ") of function " + getName()
+ " is out of bounds because the maximum resolution in H3 library is " + toString(MAX_H3_RES), ErrorCodes::ARGUMENT_OUT_OF_BOUND);

// Numerical constant is 180 degrees / pi / Earth radius, Earth radius is from h3 sources
Float64 res = 8.99320592271288084e-6 * edgeLengthM(resolution);
Expand Down
Loading

0 comments on commit d3f9095

Please sign in to comment.