Merge branch 'master' into fix-half-of-trash

This commit is contained in:
Alexey Milovidov 2022-09-11 06:20:47 +02:00
commit 7f1e7b5967
39 changed files with 76 additions and 103 deletions

View File

@ -519,7 +519,7 @@ private:
template <typename FieldType> template <typename FieldType>
bool compareImpl(FieldType & x) const bool compareImpl(FieldType & x) const
{ {
auto val = get<FieldType>(rhs); auto val = rhs.get<FieldType>();
if (val > x) if (val > x)
{ {
x = val; x = val;
@ -554,7 +554,7 @@ private:
template <typename FieldType> template <typename FieldType>
bool compareImpl(FieldType & x) const bool compareImpl(FieldType & x) const
{ {
auto val = get<FieldType>(rhs); auto val = rhs.get<FieldType>();
if (val < x) if (val < x)
{ {
x = val; x = val;

View File

@ -141,7 +141,7 @@ void ColumnArray::get(size_t n, Field & res) const
size, max_array_size_as_field); size, max_array_size_as_field);
res = Array(); res = Array();
Array & res_arr = DB::get<Array &>(res); Array & res_arr = res.get<Array &>();
res_arr.reserve(size); res_arr.reserve(size);
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
@ -296,7 +296,7 @@ void ColumnArray::updateHashFast(SipHash & hash) const
void ColumnArray::insert(const Field & x) void ColumnArray::insert(const Field & x)
{ {
const Array & array = DB::get<const Array &>(x); const Array & array = x.get<const Array &>();
size_t size = array.size(); size_t size = array.size();
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
getData().insert(array[i]); getData().insert(array[i]);

View File

@ -63,7 +63,7 @@ public:
{ {
data.resize_fill(data.size() + length); data.resize_fill(data.size() + length);
} }
void insert(const Field & x) override { data.push_back(DB::get<T>(x)); } void insert(const Field & x) override { data.push_back(x.get<T>()); }
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;
void popBack(size_t n) override void popBack(size_t n) override

View File

@ -59,7 +59,7 @@ bool ColumnFixedString::isDefaultAt(size_t index) const
void ColumnFixedString::insert(const Field & x) void ColumnFixedString::insert(const Field & x)
{ {
const String & s = DB::get<const String &>(x); const String & s = x.get<const String &>();
if (s.size() > n) if (s.size() > n)
throw Exception("Too large string '" + s + "' for FixedString column", ErrorCodes::TOO_LARGE_STRING_SIZE); throw Exception("Too large string '" + s + "' for FixedString column", ErrorCodes::TOO_LARGE_STRING_SIZE);

View File

@ -74,7 +74,7 @@ void ColumnMap::get(size_t n, Field & res) const
size_t size = offsets[n] - offsets[n - 1]; size_t size = offsets[n] - offsets[n - 1];
res = Map(); res = Map();
auto & map = DB::get<Map &>(res); auto & map = res.get<Map &>();
map.reserve(size); map.reserve(size);
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
@ -98,7 +98,7 @@ void ColumnMap::insertData(const char *, size_t)
void ColumnMap::insert(const Field & x) void ColumnMap::insert(const Field & x)
{ {
const auto & map = DB::get<const Map &>(x); const auto & map = x.get<const Map &>();
nested->insert(Array(map.begin(), map.end())); nested->insert(Array(map.begin(), map.end()));
} }

View File

@ -128,7 +128,7 @@ public:
void insert(const Field & x) override void insert(const Field & x) override
{ {
const String & s = DB::get<const String &>(x); const String & s = x.get<const String &>();
const size_t old_size = chars.size(); const size_t old_size = chars.size();
const size_t size_to_append = s.size() + 1; const size_t size_to_append = s.size() + 1;
const size_t new_size = old_size + size_to_append; const size_t new_size = old_size + size_to_append;

View File

@ -109,7 +109,7 @@ void ColumnTuple::get(size_t n, Field & res) const
const size_t tuple_size = columns.size(); const size_t tuple_size = columns.size();
res = Tuple(); res = Tuple();
Tuple & res_tuple = DB::get<Tuple &>(res); Tuple & res_tuple = res.get<Tuple &>();
res_tuple.reserve(tuple_size); res_tuple.reserve(tuple_size);
for (size_t i = 0; i < tuple_size; ++i) for (size_t i = 0; i < tuple_size; ++i)
@ -137,7 +137,7 @@ void ColumnTuple::insertData(const char *, size_t)
void ColumnTuple::insert(const Field & x) void ColumnTuple::insert(const Field & x)
{ {
const auto & tuple = DB::get<const Tuple &>(x); const auto & tuple = x.get<const Tuple &>();
const size_t tuple_size = columns.size(); const size_t tuple_size = columns.size();
if (tuple.size() != tuple_size) if (tuple.size() != tuple_size)

View File

@ -918,7 +918,7 @@ ColumnPtr ColumnVector<T>::createWithOffsets(const IColumn::Offsets & offsets, c
auto res = this->create(); auto res = this->create();
auto & res_data = res->getData(); auto & res_data = res->getData();
T default_value = safeGet<T>(default_field); T default_value = default_field.safeGet<T>();
res_data.resize_fill(total_rows, default_value); res_data.resize_fill(total_rows, default_value);
for (size_t i = 0; i < offsets.size(); ++i) for (size_t i = 0; i < offsets.size(); ++i)
res_data[offsets[i]] = data[i + shift]; res_data[offsets[i]] = data[i + shift];

View File

@ -301,7 +301,7 @@ public:
void insert(const Field & x) override void insert(const Field & x) override
{ {
data.push_back(DB::get<T>(x)); data.push_back(x.get<T>());
} }
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;

View File

@ -19,7 +19,7 @@ bool FieldVisitorSum::operator() (UInt64 & x) const
return x != 0; return x != 0;
} }
bool FieldVisitorSum::operator() (Float64 & x) const { x += get<Float64>(rhs); return x != 0; } bool FieldVisitorSum::operator() (Float64 & x) const { x += rhs.get<Float64>(); return x != 0; }
bool FieldVisitorSum::operator() (Null &) const { throw Exception("Cannot sum Nulls", ErrorCodes::LOGICAL_ERROR); } bool FieldVisitorSum::operator() (Null &) const { throw Exception("Cannot sum Nulls", ErrorCodes::LOGICAL_ERROR); }
bool FieldVisitorSum::operator() (String &) const { throw Exception("Cannot sum Strings", ErrorCodes::LOGICAL_ERROR); } bool FieldVisitorSum::operator() (String &) const { throw Exception("Cannot sum Strings", ErrorCodes::LOGICAL_ERROR); }
@ -37,4 +37,3 @@ bool FieldVisitorSum::operator() (AggregateFunctionStateData &) const
bool FieldVisitorSum::operator() (bool &) const { throw Exception("Cannot sum Bools", ErrorCodes::LOGICAL_ERROR); } bool FieldVisitorSum::operator() (bool &) const { throw Exception("Cannot sum Bools", ErrorCodes::LOGICAL_ERROR); }
} }

View File

@ -33,7 +33,7 @@ public:
template <typename T> template <typename T>
bool operator() (DecimalField<T> & x) const bool operator() (DecimalField<T> & x) const
{ {
x += get<DecimalField<T>>(rhs); x += rhs.get<DecimalField<T>>();
return x.getValue() != T(0); return x.getValue() != T(0);
} }
@ -47,4 +47,3 @@ public:
}; };
} }

View File

@ -882,30 +882,6 @@ inline char & Field::reinterpret<char>()
return *reinterpret_cast<char *>(&storage); return *reinterpret_cast<char *>(&storage);
} }
template <typename T>
T get(const Field & field)
{
return static_cast<T>(field.template get<T>());
}
template <typename T>
T get(Field & field)
{
return static_cast<T>(field.template get<T>());
}
template <typename T>
T safeGet(const Field & field)
{
return static_cast<T>(field.template safeGet<T>());
}
template <typename T>
T safeGet(Field & field)
{
return static_cast<T>(field.template safeGet<T>());
}
template <typename T> template <typename T>
Field::Field(T && rhs, enable_if_not_field_or_bool_or_stringlike_t<T>) //-V730 Field::Field(T && rhs, enable_if_not_field_or_bool_or_stringlike_t<T>) //-V730
{ {

View File

@ -37,7 +37,7 @@ int main(int argc, char ** argv)
std::cerr << applyVisitor(to_string, field) << std::endl; std::cerr << applyVisitor(to_string, field) << std::endl;
} }
get<Array &>(field).push_back(field); field.get<Array &>().push_back(field);
std::cerr << applyVisitor(to_string, field) << std::endl; std::cerr << applyVisitor(to_string, field) << std::endl;
std::cerr << (field < field2) << std::endl; std::cerr << (field < field2) << std::endl;
@ -71,7 +71,7 @@ int main(int argc, char ** argv)
size_t sum = 0; size_t sum = 0;
for (size_t i = 0; i < n; ++i) for (size_t i = 0; i < n; ++i)
sum += safeGet<const String &>(array[i]).size(); sum += array[i].safeGet<const String &>().size();
watch.stop(); watch.stop();
std::cerr << std::fixed << std::setprecision(2) std::cerr << std::fixed << std::setprecision(2)

View File

@ -131,7 +131,7 @@ void JSONDataParser<ParserImpl>::traverseArrayElement(const Element & element, P
auto nested_key = getNameOfNested(paths[i], values[i]); auto nested_key = getNameOfNested(paths[i], values[i]);
if (!nested_key.empty()) if (!nested_key.empty())
{ {
size_t array_size = get<const Array &>(values[i]).size(); size_t array_size = values[i].template get<const Array &>().size();
auto & current_nested_sizes = ctx.nested_sizes_by_key[nested_key]; auto & current_nested_sizes = ctx.nested_sizes_by_key[nested_key];
if (current_nested_sizes.size() == ctx.current_size) if (current_nested_sizes.size() == ctx.current_size)
@ -154,7 +154,7 @@ void JSONDataParser<ParserImpl>::traverseArrayElement(const Element & element, P
auto nested_key = getNameOfNested(paths[i], values[i]); auto nested_key = getNameOfNested(paths[i], values[i]);
if (!nested_key.empty()) if (!nested_key.empty())
{ {
size_t array_size = get<const Array &>(values[i]).size(); size_t array_size = values[i].template get<const Array &>().size();
auto & current_nested_sizes = ctx.nested_sizes_by_key[nested_key]; auto & current_nested_sizes = ctx.nested_sizes_by_key[nested_key];
if (current_nested_sizes.empty()) if (current_nested_sizes.empty())

View File

@ -19,14 +19,14 @@ namespace DB
void SerializationAggregateFunction::serializeBinary(const Field & field, WriteBuffer & ostr) const void SerializationAggregateFunction::serializeBinary(const Field & field, WriteBuffer & ostr) const
{ {
const AggregateFunctionStateData & state = get<const AggregateFunctionStateData &>(field); const AggregateFunctionStateData & state = field.get<const AggregateFunctionStateData &>();
writeBinary(state.data, ostr); writeBinary(state.data, ostr);
} }
void SerializationAggregateFunction::deserializeBinary(Field & field, ReadBuffer & istr) const void SerializationAggregateFunction::deserializeBinary(Field & field, ReadBuffer & istr) const
{ {
field = AggregateFunctionStateData(); field = AggregateFunctionStateData();
AggregateFunctionStateData & s = get<AggregateFunctionStateData &>(field); AggregateFunctionStateData & s = field.get<AggregateFunctionStateData &>();
readBinary(s.data, istr); readBinary(s.data, istr);
s.name = type_name; s.name = type_name;
} }

View File

@ -24,7 +24,7 @@ namespace ErrorCodes
void SerializationArray::serializeBinary(const Field & field, WriteBuffer & ostr) const void SerializationArray::serializeBinary(const Field & field, WriteBuffer & ostr) const
{ {
const Array & a = get<const Array &>(field); const Array & a = field.get<const Array &>();
writeVarUInt(a.size(), ostr); writeVarUInt(a.size(), ostr);
for (size_t i = 0; i < a.size(); ++i) for (size_t i = 0; i < a.size(); ++i)
{ {
@ -38,7 +38,7 @@ void SerializationArray::deserializeBinary(Field & field, ReadBuffer & istr) con
size_t size; size_t size;
readVarUInt(size, istr); readVarUInt(size, istr);
field = Array(); field = Array();
Array & arr = get<Array &>(field); Array & arr = field.get<Array &>();
arr.reserve(size); arr.reserve(size);
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
nested->deserializeBinary(arr.emplace_back(), istr); nested->deserializeBinary(arr.emplace_back(), istr);

View File

@ -14,7 +14,7 @@ namespace DB
template <typename T> template <typename T>
void SerializationDecimalBase<T>::serializeBinary(const Field & field, WriteBuffer & ostr) const void SerializationDecimalBase<T>::serializeBinary(const Field & field, WriteBuffer & ostr) const
{ {
FieldType x = get<DecimalField<T>>(field); FieldType x = field.get<DecimalField<T>>();
writeBinary(x, ostr); writeBinary(x, ostr);
} }

View File

@ -26,7 +26,7 @@ namespace ErrorCodes
void SerializationFixedString::serializeBinary(const Field & field, WriteBuffer & ostr) const void SerializationFixedString::serializeBinary(const Field & field, WriteBuffer & ostr) const
{ {
const String & s = get<const String &>(field); const String & s = field.get<const String &>();
ostr.write(s.data(), std::min(s.size(), n)); ostr.write(s.data(), std::min(s.size(), n));
if (s.size() < n) if (s.size() < n)
for (size_t i = s.size(); i < n; ++i) for (size_t i = s.size(); i < n; ++i)
@ -37,7 +37,7 @@ void SerializationFixedString::serializeBinary(const Field & field, WriteBuffer
void SerializationFixedString::deserializeBinary(Field & field, ReadBuffer & istr) const void SerializationFixedString::deserializeBinary(Field & field, ReadBuffer & istr) const
{ {
field = String(); field = String();
String & s = get<String &>(field); String & s = field.get<String &>();
s.resize(n); s.resize(n);
istr.readStrict(s.data(), n); istr.readStrict(s.data(), n);
} }

View File

@ -38,7 +38,7 @@ static IColumn & extractNestedColumn(IColumn & column)
void SerializationMap::serializeBinary(const Field & field, WriteBuffer & ostr) const void SerializationMap::serializeBinary(const Field & field, WriteBuffer & ostr) const
{ {
const auto & map = get<const Map &>(field); const auto & map = field.get<const Map &>();
writeVarUInt(map.size(), ostr); writeVarUInt(map.size(), ostr);
for (const auto & elem : map) for (const auto & elem : map)
{ {

View File

@ -105,7 +105,7 @@ template <typename T>
void SerializationNumber<T>::serializeBinary(const Field & field, WriteBuffer & ostr) const void SerializationNumber<T>::serializeBinary(const Field & field, WriteBuffer & ostr) const
{ {
/// ColumnVector<T>::ValueType is a narrower type. For example, UInt8, when the Field type is UInt64 /// ColumnVector<T>::ValueType is a narrower type. For example, UInt8, when the Field type is UInt64
typename ColumnVector<T>::ValueType x = get<FieldType>(field); typename ColumnVector<T>::ValueType x = field.get<FieldType>();
writeBinary(x, ostr); writeBinary(x, ostr);
} }

View File

@ -22,7 +22,7 @@ namespace DB
void SerializationString::serializeBinary(const Field & field, WriteBuffer & ostr) const void SerializationString::serializeBinary(const Field & field, WriteBuffer & ostr) const
{ {
const String & s = get<const String &>(field); const String & s = field.get<const String &>();
writeVarUInt(s.size(), ostr); writeVarUInt(s.size(), ostr);
writeString(s, ostr); writeString(s, ostr);
} }
@ -33,7 +33,7 @@ void SerializationString::deserializeBinary(Field & field, ReadBuffer & istr) co
UInt64 size; UInt64 size;
readVarUInt(size, istr); readVarUInt(size, istr);
field = String(); field = String();
String & s = get<String &>(field); String & s = field.get<String &>();
s.resize(size); s.resize(size);
istr.readStrict(s.data(), size); istr.readStrict(s.data(), size);
} }

View File

@ -31,7 +31,7 @@ static inline const IColumn & extractElementColumn(const IColumn & column, size_
void SerializationTuple::serializeBinary(const Field & field, WriteBuffer & ostr) const void SerializationTuple::serializeBinary(const Field & field, WriteBuffer & ostr) const
{ {
const auto & tuple = get<const Tuple &>(field); const auto & tuple = field.get<const Tuple &>();
for (size_t element_index = 0; element_index < elems.size(); ++element_index) for (size_t element_index = 0; element_index < elems.size(); ++element_index)
{ {
const auto & serialization = elems[element_index]; const auto & serialization = elems[element_index];
@ -44,7 +44,7 @@ void SerializationTuple::deserializeBinary(Field & field, ReadBuffer & istr) con
const size_t size = elems.size(); const size_t size = elems.size();
field = Tuple(); field = Tuple();
Tuple & tuple = get<Tuple &>(field); Tuple & tuple = field.get<Tuple &>();
tuple.reserve(size); tuple.reserve(size);
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
elems[i]->deserializeBinary(tuple.emplace_back(), istr); elems[i]->deserializeBinary(tuple.emplace_back(), istr);

View File

@ -84,7 +84,7 @@ void SerializationUUID::deserializeTextCSV(IColumn & column, ReadBuffer & istr,
void SerializationUUID::serializeBinary(const Field & field, WriteBuffer & ostr) const void SerializationUUID::serializeBinary(const Field & field, WriteBuffer & ostr) const
{ {
UUID x = get<UUID>(field); UUID x = field.get<UUID>();
writeBinary(x, ostr); writeBinary(x, ostr);
} }

View File

@ -633,7 +633,7 @@ static void writeFieldsToColumn(
{ {
for (size_t index = 0; index < rows_data.size(); ++index) for (size_t index = 0; index < rows_data.size(); ++index)
{ {
const Tuple & row_data = DB::get<const Tuple &>(rows_data[index]); const Tuple & row_data = rows_data[index].get<const Tuple &>();
const Field & value = row_data[column_index]; const Field & value = row_data[column_index];
if (write_data_to_null_map(value, index)) if (write_data_to_null_map(value, index))
@ -673,7 +673,7 @@ static void writeFieldsToColumn(
{ {
for (size_t index = 0; index < rows_data.size(); ++index) for (size_t index = 0; index < rows_data.size(); ++index)
{ {
const Tuple & row_data = DB::get<const Tuple &>(rows_data[index]); const Tuple & row_data = rows_data[index].get<const Tuple &>();
const Field & value = row_data[column_index]; const Field & value = row_data[column_index];
if (write_data_to_null_map(value, index)) if (write_data_to_null_map(value, index))
@ -695,7 +695,7 @@ static void writeFieldsToColumn(
{ {
for (size_t index = 0; index < rows_data.size(); ++index) for (size_t index = 0; index < rows_data.size(); ++index)
{ {
const Tuple & row_data = DB::get<const Tuple &>(rows_data[index]); const Tuple & row_data = rows_data[index].get<const Tuple &>();
const Field & value = row_data[column_index]; const Field & value = row_data[column_index];
if (write_data_to_null_map(value, index)) if (write_data_to_null_map(value, index))
@ -709,7 +709,7 @@ static void writeFieldsToColumn(
{ {
for (size_t index = 0; index < rows_data.size(); ++index) for (size_t index = 0; index < rows_data.size(); ++index)
{ {
const Tuple & row_data = DB::get<const Tuple &>(rows_data[index]); const Tuple & row_data = rows_data[index].get<const Tuple &>();
const Field & value = row_data[column_index]; const Field & value = row_data[column_index];
if (write_data_to_null_map(value, index)) if (write_data_to_null_map(value, index))
@ -761,7 +761,7 @@ static inline size_t onUpdateData(const Row & rows_data, Block & buffer, size_t
{ {
writeable_rows_mask[index + 1] = true; writeable_rows_mask[index + 1] = true;
writeable_rows_mask[index] = differenceSortingKeys( writeable_rows_mask[index] = differenceSortingKeys(
DB::get<const Tuple &>(rows_data[index]), DB::get<const Tuple &>(rows_data[index + 1]), sorting_columns_index); rows_data[index].get<const Tuple &>(), rows_data[index + 1].get<const Tuple &>(), sorting_columns_index);
} }
for (size_t column = 0; column < buffer.columns() - 2; ++column) for (size_t column = 0; column < buffer.columns() - 2; ++column)

View File

@ -232,7 +232,7 @@ QueryPipeline MongoDBDictionarySource::loadKeys(const Columns & key_columns, con
} }
case AttributeUnderlyingType::String: case AttributeUnderlyingType::String:
{ {
String loaded_str(get<String>((*key_columns[attribute_index])[row_idx])); String loaded_str((*key_columns[attribute_index])[row_idx].get<String>());
/// Convert string to ObjectID /// Convert string to ObjectID
if (key_attribute.is_object_id) if (key_attribute.is_object_id)
{ {
@ -259,7 +259,7 @@ QueryPipeline MongoDBDictionarySource::loadKeys(const Columns & key_columns, con
std::string MongoDBDictionarySource::toString() const std::string MongoDBDictionarySource::toString() const
{ {
return "MongoDB: " + db + '.' + collection + ',' + (user.empty() ? " " : " " + user + '@') + host + ':' + DB::toString(port); return fmt::format("MongoDB: {}.{},{}{}:{}", db, collection, (user.empty() ? " " : " " + user + '@'), host, port);
} }
} }

View File

@ -220,7 +220,7 @@ namespace DB
if (isInteger(type)) if (isInteger(type))
key << DB::toString(key_columns[i]->get64(row)); key << DB::toString(key_columns[i]->get64(row));
else if (isString(type)) else if (isString(type))
key << get<const String &>((*key_columns[i])[row]); key << (*key_columns[i])[row].get<const String &>();
else else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected type of key in Redis dictionary"); throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected type of key in Redis dictionary");
} }

View File

@ -476,10 +476,10 @@ ColumnPtr FunctionArrayElement::executeNumberConst(
auto col_res = ColumnVector<DataType>::create(); auto col_res = ColumnVector<DataType>::create();
if (index.getType() == Field::Types::UInt64 if (index.getType() == Field::Types::UInt64
|| (index.getType() == Field::Types::Int64 && get<Int64>(index) >= 0)) || (index.getType() == Field::Types::Int64 && index.get<Int64>() >= 0))
{ {
ArrayElementNumImpl<DataType>::template vectorConst<false>( ArrayElementNumImpl<DataType>::template vectorConst<false>(
col_nested->getData(), col_array->getOffsets(), get<UInt64>(index) - 1, col_res->getData(), builder); col_nested->getData(), col_array->getOffsets(), index.get<UInt64>() - 1, col_res->getData(), builder);
} }
else if (index.getType() == Field::Types::Int64) else if (index.getType() == Field::Types::Int64)
{ {
@ -493,7 +493,7 @@ ColumnPtr FunctionArrayElement::executeNumberConst(
/// arr[-2] is the element at offset 1 from the last and so on. /// arr[-2] is the element at offset 1 from the last and so on.
ArrayElementNumImpl<DataType>::template vectorConst<true>( ArrayElementNumImpl<DataType>::template vectorConst<true>(
col_nested->getData(), col_array->getOffsets(), -(static_cast<UInt64>(safeGet<Int64>(index)) + 1), col_res->getData(), builder); col_nested->getData(), col_array->getOffsets(), -(static_cast<UInt64>(index.safeGet<Int64>()) + 1), col_res->getData(), builder);
} }
else else
throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR); throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR);
@ -539,12 +539,12 @@ FunctionArrayElement::executeStringConst(const ColumnsWithTypeAndName & argument
auto col_res = ColumnString::create(); auto col_res = ColumnString::create();
if (index.getType() == Field::Types::UInt64 if (index.getType() == Field::Types::UInt64
|| (index.getType() == Field::Types::Int64 && get<Int64>(index) >= 0)) || (index.getType() == Field::Types::Int64 && index.get<Int64>() >= 0))
ArrayElementStringImpl::vectorConst<false>( ArrayElementStringImpl::vectorConst<false>(
col_nested->getChars(), col_nested->getChars(),
col_array->getOffsets(), col_array->getOffsets(),
col_nested->getOffsets(), col_nested->getOffsets(),
get<UInt64>(index) - 1, index.get<UInt64>() - 1,
col_res->getChars(), col_res->getChars(),
col_res->getOffsets(), col_res->getOffsets(),
builder); builder);
@ -553,7 +553,7 @@ FunctionArrayElement::executeStringConst(const ColumnsWithTypeAndName & argument
col_nested->getChars(), col_nested->getChars(),
col_array->getOffsets(), col_array->getOffsets(),
col_nested->getOffsets(), col_nested->getOffsets(),
-(UInt64(get<Int64>(index)) + 1), -(UInt64(index.get<Int64>()) + 1),
col_res->getChars(), col_res->getChars(),
col_res->getOffsets(), col_res->getOffsets(),
builder); builder);
@ -603,12 +603,12 @@ ColumnPtr FunctionArrayElement::executeGenericConst(
auto col_res = col_nested.cloneEmpty(); auto col_res = col_nested.cloneEmpty();
if (index.getType() == Field::Types::UInt64 if (index.getType() == Field::Types::UInt64
|| (index.getType() == Field::Types::Int64 && get<Int64>(index) >= 0)) || (index.getType() == Field::Types::Int64 && index.get<Int64>() >= 0))
ArrayElementGenericImpl::vectorConst<false>( ArrayElementGenericImpl::vectorConst<false>(
col_nested, col_array->getOffsets(), get<UInt64>(index) - 1, *col_res, builder); col_nested, col_array->getOffsets(), index.get<UInt64>() - 1, *col_res, builder);
else if (index.getType() == Field::Types::Int64) else if (index.getType() == Field::Types::Int64)
ArrayElementGenericImpl::vectorConst<true>( ArrayElementGenericImpl::vectorConst<true>(
col_nested, col_array->getOffsets(), -(static_cast<UInt64>(get<Int64>(index) + 1)), *col_res, builder); col_nested, col_array->getOffsets(), -(static_cast<UInt64>(index.get<Int64>() + 1)), *col_res, builder);
else else
throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR); throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR);
@ -877,7 +877,7 @@ bool FunctionArrayElement::matchKeyToIndexStringConst(
using DataColumn = std::decay_t<decltype(data_column)>; using DataColumn = std::decay_t<decltype(data_column)>;
if (index.getType() != Field::Types::String) if (index.getType() != Field::Types::String)
return false; return false;
MatcherStringConst<DataColumn> matcher{data_column, get<const String &>(index)}; MatcherStringConst<DataColumn> matcher{data_column, index.get<const String &>()};
executeMatchKeyToIndex(offsets, matched_idxs, matcher); executeMatchKeyToIndex(offsets, matched_idxs, matcher);
return true; return true;
}); });

View File

@ -114,7 +114,7 @@ static Block createBlockFromCollection(const Collection & collection, const Data
throw Exception("Invalid type in set. Expected tuple, got " throw Exception("Invalid type in set. Expected tuple, got "
+ String(value.getTypeName()), ErrorCodes::INCORRECT_ELEMENT_OF_SET); + String(value.getTypeName()), ErrorCodes::INCORRECT_ELEMENT_OF_SET);
const auto & tuple = DB::get<const Tuple &>(value); const auto & tuple = value.template get<const Tuple &>();
size_t tuple_size = tuple.size(); size_t tuple_size = tuple.size();
if (tuple_size != columns_num) if (tuple_size != columns_num)
@ -306,9 +306,9 @@ Block createBlockForSet(
{ {
auto type_index = right_arg_type->getTypeId(); auto type_index = right_arg_type->getTypeId();
if (type_index == TypeIndex::Tuple) if (type_index == TypeIndex::Tuple)
block = createBlockFromCollection(DB::get<const Tuple &>(right_arg_value), set_element_types, tranform_null_in); block = createBlockFromCollection(right_arg_value.get<const Tuple &>(), set_element_types, tranform_null_in);
else if (type_index == TypeIndex::Array) else if (type_index == TypeIndex::Array)
block = createBlockFromCollection(DB::get<const Array &>(right_arg_value), set_element_types, tranform_null_in); block = createBlockFromCollection(right_arg_value.get<const Array &>(), set_element_types, tranform_null_in);
else else
throw_unsupported_type(right_arg_type); throw_unsupported_type(right_arg_type);
} }

View File

@ -327,7 +327,7 @@ BlockIO InterpreterKillQueryQuery::execute()
for (size_t i = 0; i < moves_block.rows(); ++i) for (size_t i = 0; i < moves_block.rows(); ++i)
{ {
table_id = StorageID{database_col.getDataAt(i).toString(), table_col.getDataAt(i).toString()}; table_id = StorageID{database_col.getDataAt(i).toString(), table_col.getDataAt(i).toString()};
auto task_uuid = get<UUID>(task_uuid_col[i]); auto task_uuid = task_uuid_col[i].get<UUID>();
CancellationCode code = CancellationCode::Unknown; CancellationCode code = CancellationCode::Unknown;

View File

@ -111,11 +111,11 @@ void RewriteFunctionToSubcolumnData::visit(ASTFunction & function, ASTPtr & ast)
if (value_type == Field::Types::UInt64) if (value_type == Field::Types::UInt64)
{ {
const auto & type_tuple = assert_cast<const DataTypeTuple &>(*column_type); const auto & type_tuple = assert_cast<const DataTypeTuple &>(*column_type);
auto index = get<UInt64>(literal->value); auto index = literal->value.get<UInt64>();
subcolumn_name = type_tuple.getNameByPosition(index); subcolumn_name = type_tuple.getNameByPosition(index);
} }
else if (value_type == Field::Types::String) else if (value_type == Field::Types::String)
subcolumn_name = get<const String &>(literal->value); subcolumn_name = literal->value.get<const String &>();
else else
return; return;

View File

@ -29,7 +29,7 @@ TEST(ComparisonGraph, Bounds)
const auto & [lower, strict] = *res; const auto & [lower, strict] = *res;
ASSERT_EQ(get<UInt64>(lower), 3); ASSERT_EQ(lower.get<UInt64>(), 3);
ASSERT_TRUE(strict); ASSERT_TRUE(strict);
} }
@ -39,7 +39,7 @@ TEST(ComparisonGraph, Bounds)
const auto & [upper, strict] = *res; const auto & [upper, strict] = *res;
ASSERT_EQ(get<UInt64>(upper), 7); ASSERT_EQ(upper.get<UInt64>(), 7);
ASSERT_TRUE(strict); ASSERT_TRUE(strict);
} }

View File

@ -177,12 +177,11 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
tryGetIdentifierNameInto(database, query->from); tryGetIdentifierNameInto(database, query->from);
if (like) if (like)
query->like = safeGet<const String &>(like->as<ASTLiteral &>().value); query->like = like->as<ASTLiteral &>().value.safeGet<const String &>();
node = query; node = query;
return true; return true;
} }
} }

View File

@ -40,7 +40,7 @@ static FillColumnDescription::StepFunction getStepFunction(
{ {
#define DECLARE_CASE(NAME) \ #define DECLARE_CASE(NAME) \
case IntervalKind::NAME: \ case IntervalKind::NAME: \
return [step, scale, &date_lut](Field & field) { field = Add##NAME##sImpl::execute(get<T>(field), step, date_lut, scale); }; return [step, scale, &date_lut](Field & field) { field = Add##NAME##sImpl::execute(static_cast<T>(field.get<T>()), step, date_lut, scale); };
FOR_EACH_INTERVAL_KIND(DECLARE_CASE) FOR_EACH_INTERVAL_KIND(DECLARE_CASE)
#undef DECLARE_CASE #undef DECLARE_CASE
@ -106,21 +106,21 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr &
{ {
if (which.isDate() || which.isDate32()) if (which.isDate() || which.isDate32())
{ {
Int64 avg_seconds = get<Int64>(descr.fill_step) * descr.step_kind->toAvgSeconds(); Int64 avg_seconds = descr.fill_step.get<Int64>() * descr.step_kind->toAvgSeconds();
if (std::abs(avg_seconds) < 86400) if (std::abs(avg_seconds) < 86400)
throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION,
"Value of step is to low ({} seconds). Must be >= 1 day", std::abs(avg_seconds)); "Value of step is to low ({} seconds). Must be >= 1 day", std::abs(avg_seconds));
} }
if (which.isDate()) if (which.isDate())
descr.step_func = getStepFunction<UInt16>(*descr.step_kind, get<Int64>(descr.fill_step), DateLUT::instance()); descr.step_func = getStepFunction<UInt16>(*descr.step_kind, descr.fill_step.get<Int64>(), DateLUT::instance());
else if (which.isDate32()) else if (which.isDate32())
descr.step_func = getStepFunction<Int32>(*descr.step_kind, get<Int64>(descr.fill_step), DateLUT::instance()); descr.step_func = getStepFunction<Int32>(*descr.step_kind, descr.fill_step.get<Int64>(), DateLUT::instance());
else if (const auto * date_time = checkAndGetDataType<DataTypeDateTime>(type.get())) else if (const auto * date_time = checkAndGetDataType<DataTypeDateTime>(type.get()))
descr.step_func = getStepFunction<UInt32>(*descr.step_kind, get<Int64>(descr.fill_step), date_time->getTimeZone()); descr.step_func = getStepFunction<UInt32>(*descr.step_kind, descr.fill_step.get<Int64>(), date_time->getTimeZone());
else if (const auto * date_time64 = checkAndGetDataType<DataTypeDateTime64>(type.get())) else if (const auto * date_time64 = checkAndGetDataType<DataTypeDateTime64>(type.get()))
{ {
const auto & step_dec = get<const DecimalField<Decimal64> &>(descr.fill_step); const auto & step_dec = descr.fill_step.get<const DecimalField<Decimal64> &>();
Int64 step = DecimalUtils::convertTo<Int64>(step_dec.getValue(), step_dec.getScale()); Int64 step = DecimalUtils::convertTo<Int64>(step_dec.getValue(), step_dec.getScale());
switch (*descr.step_kind) switch (*descr.step_kind)
@ -129,7 +129,7 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr &
case IntervalKind::NAME: \ case IntervalKind::NAME: \
descr.step_func = [step, &time_zone = date_time64->getTimeZone()](Field & field) \ descr.step_func = [step, &time_zone = date_time64->getTimeZone()](Field & field) \
{ \ { \
auto field_decimal = get<DecimalField<DateTime64>>(field); \ auto field_decimal = field.get<DecimalField<DateTime64>>(); \
auto res = Add##NAME##sImpl::execute(field_decimal.getValue(), step, time_zone, field_decimal.getScale()); \ auto res = Add##NAME##sImpl::execute(field_decimal.getValue(), step, time_zone, field_decimal.getScale()); \
field = DecimalField(res, field_decimal.getScale()); \ field = DecimalField(res, field_decimal.getScale()); \
}; \ }; \

View File

@ -582,7 +582,7 @@ Pipe StorageLiveView::watch(
if (query.limit_length) if (query.limit_length)
{ {
has_limit = true; has_limit = true;
limit = safeGet<UInt64>(typeid_cast<ASTLiteral &>(*query.limit_length).value); limit = typeid_cast<ASTLiteral &>(*query.limit_length).value.safeGet<UInt64>();
} }
if (query.is_watch_events) if (query.is_watch_events)

View File

@ -613,7 +613,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTEquals(
if (which.isTuple() && function->name == "tuple") if (which.isTuple() && function->name == "tuple")
{ {
const Tuple & tuple = get<const Tuple &>(value_field); const Tuple & tuple = value_field.get<const Tuple &>();
const auto * value_tuple_data_type = typeid_cast<const DataTypeTuple *>(value_type.get()); const auto * value_tuple_data_type = typeid_cast<const DataTypeTuple *>(value_type.get());
const ASTs & arguments = typeid_cast<const ASTExpressionList &>(*function->arguments).children; const ASTs & arguments = typeid_cast<const ASTExpressionList &>(*function->arguments).children;

View File

@ -333,7 +333,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
/// Get path and name from engine arguments /// Get path and name from engine arguments
ast_zk_path = engine_args[arg_num]->as<ASTLiteral>(); ast_zk_path = engine_args[arg_num]->as<ASTLiteral>();
if (ast_zk_path && ast_zk_path->value.getType() == Field::Types::String) if (ast_zk_path && ast_zk_path->value.getType() == Field::Types::String)
zookeeper_path = safeGet<String>(ast_zk_path->value); zookeeper_path = ast_zk_path->value.safeGet<String>();
else else
throw Exception( throw Exception(
"Path in ZooKeeper must be a string literal" + getMergeTreeVerboseHelp(is_extended_storage_def), "Path in ZooKeeper must be a string literal" + getMergeTreeVerboseHelp(is_extended_storage_def),
@ -342,7 +342,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
ast_replica_name = engine_args[arg_num]->as<ASTLiteral>(); ast_replica_name = engine_args[arg_num]->as<ASTLiteral>();
if (ast_replica_name && ast_replica_name->value.getType() == Field::Types::String) if (ast_replica_name && ast_replica_name->value.getType() == Field::Types::String)
replica_name = safeGet<String>(ast_replica_name->value); replica_name = ast_replica_name->value.safeGet<String>();
else else
throw Exception( throw Exception(
"Replica name must be a string literal" + getMergeTreeVerboseHelp(is_extended_storage_def), ErrorCodes::BAD_ARGUMENTS); "Replica name must be a string literal" + getMergeTreeVerboseHelp(is_extended_storage_def), ErrorCodes::BAD_ARGUMENTS);
@ -654,7 +654,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
const auto * ast = engine_args[arg_num]->as<ASTLiteral>(); const auto * ast = engine_args[arg_num]->as<ASTLiteral>();
if (ast && ast->value.getType() == Field::Types::UInt64) if (ast && ast->value.getType() == Field::Types::UInt64)
storage_settings->index_granularity = safeGet<UInt64>(ast->value); storage_settings->index_granularity = ast->value.safeGet<UInt64>();
else else
throw Exception( throw Exception(
"Index granularity must be a positive integer" + getMergeTreeVerboseHelp(is_extended_storage_def), "Index granularity must be a positive integer" + getMergeTreeVerboseHelp(is_extended_storage_def),

View File

@ -1127,7 +1127,7 @@ Pipe StorageWindowView::watch(
if (query.limit_length) if (query.limit_length)
{ {
has_limit = true; has_limit = true;
limit = safeGet<UInt64>(typeid_cast<ASTLiteral &>(*query.limit_length).value); limit = typeid_cast<ASTLiteral &>(*query.limit_length).value.safeGet<UInt64>();
} }
auto reader = std::make_shared<WindowViewSource>( auto reader = std::make_shared<WindowViewSource>(

View File

@ -196,8 +196,8 @@ ColumnsDescriptionByShardNum getExtendedObjectsOfRemoteTables(
size_t size = name_col.size(); size_t size = name_col.size();
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
{ {
auto name = get<const String &>(name_col[i]); auto name = name_col[i].get<const String &>();
auto type_name = get<const String &>(type_col[i]); auto type_name = type_col[i].get<const String &>();
auto storage_column = storage_columns.tryGetPhysical(name); auto storage_column = storage_columns.tryGetPhysical(name);
if (storage_column && isObject(storage_column->type)) if (storage_column && isObject(storage_column->type))

View File

@ -107,7 +107,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr
if (lit->value.getType() != Field::Types::String) if (lit->value.getType() != Field::Types::String)
return false; return false;
res = safeGet<const String &>(lit->value); res = lit->value.safeGet<const String &>();
return true; return true;
}; };