mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Remove some methods
This commit is contained in:
parent
6924a931c6
commit
fd235919aa
@ -519,7 +519,7 @@ private:
|
||||
template <typename FieldType>
|
||||
bool compareImpl(FieldType & x) const
|
||||
{
|
||||
auto val = get<FieldType>(rhs);
|
||||
auto val = rhs.get<FieldType>();
|
||||
if (val > x)
|
||||
{
|
||||
x = val;
|
||||
@ -554,7 +554,7 @@ private:
|
||||
template <typename FieldType>
|
||||
bool compareImpl(FieldType & x) const
|
||||
{
|
||||
auto val = get<FieldType>(rhs);
|
||||
auto val = rhs.get<FieldType>();
|
||||
if (val < x)
|
||||
{
|
||||
x = val;
|
||||
|
@ -141,7 +141,7 @@ void ColumnArray::get(size_t n, Field & res) const
|
||||
size, max_array_size_as_field);
|
||||
|
||||
res = Array();
|
||||
Array & res_arr = DB::get<Array &>(res);
|
||||
Array & res_arr = res.get<Array &>();
|
||||
res_arr.reserve(size);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
@ -296,7 +296,7 @@ void ColumnArray::updateHashFast(SipHash & hash) const
|
||||
|
||||
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();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
getData().insert(array[i]);
|
||||
|
@ -63,7 +63,7 @@ public:
|
||||
{
|
||||
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 popBack(size_t n) override
|
||||
|
@ -59,7 +59,7 @@ bool ColumnFixedString::isDefaultAt(size_t index) const
|
||||
|
||||
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)
|
||||
throw Exception("Too large string '" + s + "' for FixedString column", ErrorCodes::TOO_LARGE_STRING_SIZE);
|
||||
|
@ -74,7 +74,7 @@ void ColumnMap::get(size_t n, Field & res) const
|
||||
size_t size = offsets[n] - offsets[n - 1];
|
||||
|
||||
res = Map();
|
||||
auto & map = DB::get<Map &>(res);
|
||||
auto & map = res.get<Map &>();
|
||||
map.reserve(size);
|
||||
|
||||
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)
|
||||
{
|
||||
const auto & map = DB::get<const Map &>(x);
|
||||
const auto & map = x.get<const Map &>();
|
||||
nested->insert(Array(map.begin(), map.end()));
|
||||
}
|
||||
|
||||
|
@ -128,7 +128,7 @@ public:
|
||||
|
||||
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 size_to_append = s.size() + 1;
|
||||
const size_t new_size = old_size + size_to_append;
|
||||
|
@ -109,7 +109,7 @@ void ColumnTuple::get(size_t n, Field & res) const
|
||||
const size_t tuple_size = columns.size();
|
||||
|
||||
res = Tuple();
|
||||
Tuple & res_tuple = DB::get<Tuple &>(res);
|
||||
Tuple & res_tuple = res.get<Tuple &>();
|
||||
res_tuple.reserve(tuple_size);
|
||||
|
||||
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)
|
||||
{
|
||||
const auto & tuple = DB::get<const Tuple &>(x);
|
||||
const auto & tuple = x.get<const Tuple &>();
|
||||
|
||||
const size_t tuple_size = columns.size();
|
||||
if (tuple.size() != tuple_size)
|
||||
|
@ -918,7 +918,7 @@ ColumnPtr ColumnVector<T>::createWithOffsets(const IColumn::Offsets & offsets, c
|
||||
auto res = this->create();
|
||||
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);
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
res_data[offsets[i]] = data[i + shift];
|
||||
|
@ -301,7 +301,7 @@ public:
|
||||
|
||||
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;
|
||||
|
@ -19,7 +19,7 @@ bool FieldVisitorSum::operator() (UInt64 & x) const
|
||||
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() (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); }
|
||||
|
||||
}
|
||||
|
||||
|
@ -33,7 +33,7 @@ public:
|
||||
template <typename T>
|
||||
bool operator() (DecimalField<T> & x) const
|
||||
{
|
||||
x += get<DecimalField<T>>(rhs);
|
||||
x += rhs.get<DecimalField<T>>();
|
||||
return x.getValue() != T(0);
|
||||
}
|
||||
|
||||
@ -47,4 +47,3 @@ public:
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
@ -882,30 +882,6 @@ inline char & Field::reinterpret<char>()
|
||||
return *reinterpret_cast<char *>(&storage);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
T get(const Field & field)
|
||||
{
|
||||
return field.template get<T>();
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
T get(Field & field)
|
||||
{
|
||||
return field.template get<T>();
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
T safeGet(const Field & field)
|
||||
{
|
||||
return field.template safeGet<T>();
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
T safeGet(Field & field)
|
||||
{
|
||||
return field.template safeGet<T>();
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
Field::Field(T && rhs, enable_if_not_field_or_bool_or_stringlike_t<T>) //-V730
|
||||
{
|
||||
@ -1036,4 +1012,3 @@ struct fmt::formatter<DB::Field>
|
||||
return format_to(ctx.out(), "{}", toString(x));
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -131,7 +131,7 @@ void JSONDataParser<ParserImpl>::traverseArrayElement(const Element & element, P
|
||||
auto nested_key = getNameOfNested(paths[i], values[i]);
|
||||
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];
|
||||
|
||||
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]);
|
||||
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];
|
||||
|
||||
if (current_nested_sizes.empty())
|
||||
|
@ -19,14 +19,14 @@ namespace DB
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
void SerializationAggregateFunction::deserializeBinary(Field & field, ReadBuffer & istr) const
|
||||
{
|
||||
field = AggregateFunctionStateData();
|
||||
AggregateFunctionStateData & s = get<AggregateFunctionStateData &>(field);
|
||||
AggregateFunctionStateData & s = field.get<AggregateFunctionStateData &>();
|
||||
readBinary(s.data, istr);
|
||||
s.name = type_name;
|
||||
}
|
||||
|
@ -24,7 +24,7 @@ namespace ErrorCodes
|
||||
|
||||
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);
|
||||
for (size_t i = 0; i < a.size(); ++i)
|
||||
{
|
||||
@ -38,7 +38,7 @@ void SerializationArray::deserializeBinary(Field & field, ReadBuffer & istr) con
|
||||
size_t size;
|
||||
readVarUInt(size, istr);
|
||||
field = Array();
|
||||
Array & arr = get<Array &>(field);
|
||||
Array & arr = field.get<Array &>();
|
||||
arr.reserve(size);
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
nested->deserializeBinary(arr.emplace_back(), istr);
|
||||
|
@ -14,7 +14,7 @@ namespace DB
|
||||
template <typename T>
|
||||
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);
|
||||
}
|
||||
|
||||
|
@ -26,7 +26,7 @@ namespace ErrorCodes
|
||||
|
||||
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));
|
||||
if (s.size() < n)
|
||||
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
|
||||
{
|
||||
field = String();
|
||||
String & s = get<String &>(field);
|
||||
String & s = field.get<String &>();
|
||||
s.resize(n);
|
||||
istr.readStrict(s.data(), n);
|
||||
}
|
||||
|
@ -38,7 +38,7 @@ static IColumn & extractNestedColumn(IColumn & column)
|
||||
|
||||
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);
|
||||
for (const auto & elem : map)
|
||||
{
|
||||
|
@ -105,7 +105,7 @@ template <typename T>
|
||||
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
|
||||
typename ColumnVector<T>::ValueType x = get<FieldType>(field);
|
||||
typename ColumnVector<T>::ValueType x = field.get<FieldType>();
|
||||
writeBinary(x, ostr);
|
||||
}
|
||||
|
||||
|
@ -22,7 +22,7 @@ namespace DB
|
||||
|
||||
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);
|
||||
writeString(s, ostr);
|
||||
}
|
||||
@ -33,7 +33,7 @@ void SerializationString::deserializeBinary(Field & field, ReadBuffer & istr) co
|
||||
UInt64 size;
|
||||
readVarUInt(size, istr);
|
||||
field = String();
|
||||
String & s = get<String &>(field);
|
||||
String & s = field.get<String &>();
|
||||
s.resize(size);
|
||||
istr.readStrict(s.data(), size);
|
||||
}
|
||||
|
@ -31,7 +31,7 @@ static inline const IColumn & extractElementColumn(const IColumn & column, size_
|
||||
|
||||
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)
|
||||
{
|
||||
const auto & serialization = elems[element_index];
|
||||
@ -44,7 +44,7 @@ void SerializationTuple::deserializeBinary(Field & field, ReadBuffer & istr) con
|
||||
const size_t size = elems.size();
|
||||
|
||||
field = Tuple();
|
||||
Tuple & tuple = get<Tuple &>(field);
|
||||
Tuple & tuple = field.get<Tuple &>();
|
||||
tuple.reserve(size);
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
elems[i]->deserializeBinary(tuple.emplace_back(), istr);
|
||||
|
@ -84,7 +84,7 @@ void SerializationUUID::deserializeTextCSV(IColumn & column, ReadBuffer & istr,
|
||||
|
||||
void SerializationUUID::serializeBinary(const Field & field, WriteBuffer & ostr) const
|
||||
{
|
||||
UUID x = get<UUID>(field);
|
||||
UUID x = field.get<UUID>();
|
||||
writeBinary(x, ostr);
|
||||
}
|
||||
|
||||
|
@ -633,7 +633,7 @@ static void writeFieldsToColumn(
|
||||
{
|
||||
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];
|
||||
|
||||
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)
|
||||
{
|
||||
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];
|
||||
|
||||
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)
|
||||
{
|
||||
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];
|
||||
|
||||
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)
|
||||
{
|
||||
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];
|
||||
|
||||
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] = 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)
|
||||
|
@ -232,7 +232,7 @@ QueryPipeline MongoDBDictionarySource::loadKeys(const Columns & key_columns, con
|
||||
}
|
||||
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
|
||||
if (key_attribute.is_object_id)
|
||||
{
|
||||
@ -259,7 +259,7 @@ QueryPipeline MongoDBDictionarySource::loadKeys(const Columns & key_columns, con
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -220,7 +220,7 @@ namespace DB
|
||||
if (isInteger(type))
|
||||
key << DB::toString(key_columns[i]->get64(row));
|
||||
else if (isString(type))
|
||||
key << get<const String &>((*key_columns[i])[row]);
|
||||
key << (*key_columns[i])[row].get<const String &>();
|
||||
else
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected type of key in Redis dictionary");
|
||||
}
|
||||
|
@ -476,10 +476,10 @@ ColumnPtr FunctionArrayElement::executeNumberConst(
|
||||
auto col_res = ColumnVector<DataType>::create();
|
||||
|
||||
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>(
|
||||
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)
|
||||
{
|
||||
@ -493,7 +493,7 @@ ColumnPtr FunctionArrayElement::executeNumberConst(
|
||||
/// arr[-2] is the element at offset 1 from the last and so on.
|
||||
|
||||
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
|
||||
throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -539,12 +539,12 @@ FunctionArrayElement::executeStringConst(const ColumnsWithTypeAndName & argument
|
||||
auto col_res = ColumnString::create();
|
||||
|
||||
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>(
|
||||
col_nested->getChars(),
|
||||
col_array->getOffsets(),
|
||||
col_nested->getOffsets(),
|
||||
get<UInt64>(index) - 1,
|
||||
index.get<UInt64>() - 1,
|
||||
col_res->getChars(),
|
||||
col_res->getOffsets(),
|
||||
builder);
|
||||
@ -553,7 +553,7 @@ FunctionArrayElement::executeStringConst(const ColumnsWithTypeAndName & argument
|
||||
col_nested->getChars(),
|
||||
col_array->getOffsets(),
|
||||
col_nested->getOffsets(),
|
||||
-(UInt64(get<Int64>(index)) + 1),
|
||||
-(UInt64(index.get<Int64>()) + 1),
|
||||
col_res->getChars(),
|
||||
col_res->getOffsets(),
|
||||
builder);
|
||||
@ -603,12 +603,12 @@ ColumnPtr FunctionArrayElement::executeGenericConst(
|
||||
auto col_res = col_nested.cloneEmpty();
|
||||
|
||||
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>(
|
||||
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)
|
||||
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
|
||||
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)>;
|
||||
if (index.getType() != Field::Types::String)
|
||||
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);
|
||||
return true;
|
||||
});
|
||||
|
@ -114,7 +114,7 @@ static Block createBlockFromCollection(const Collection & collection, const Data
|
||||
throw Exception("Invalid type in set. Expected tuple, got "
|
||||
+ 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();
|
||||
|
||||
if (tuple_size != columns_num)
|
||||
@ -306,9 +306,9 @@ Block createBlockForSet(
|
||||
{
|
||||
auto type_index = right_arg_type->getTypeId();
|
||||
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)
|
||||
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
|
||||
throw_unsupported_type(right_arg_type);
|
||||
}
|
||||
|
@ -327,7 +327,7 @@ BlockIO InterpreterKillQueryQuery::execute()
|
||||
for (size_t i = 0; i < moves_block.rows(); ++i)
|
||||
{
|
||||
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;
|
||||
|
||||
|
@ -111,11 +111,11 @@ void RewriteFunctionToSubcolumnData::visit(ASTFunction & function, ASTPtr & ast)
|
||||
if (value_type == Field::Types::UInt64)
|
||||
{
|
||||
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);
|
||||
}
|
||||
else if (value_type == Field::Types::String)
|
||||
subcolumn_name = get<const String &>(literal->value);
|
||||
subcolumn_name = literal->value.get<const String &>();
|
||||
else
|
||||
return;
|
||||
|
||||
|
@ -29,7 +29,7 @@ TEST(ComparisonGraph, Bounds)
|
||||
|
||||
const auto & [lower, strict] = *res;
|
||||
|
||||
ASSERT_EQ(get<UInt64>(lower), 3);
|
||||
ASSERT_EQ(lower.get<UInt64>(), 3);
|
||||
ASSERT_TRUE(strict);
|
||||
}
|
||||
|
||||
@ -39,7 +39,7 @@ TEST(ComparisonGraph, Bounds)
|
||||
|
||||
const auto & [upper, strict] = *res;
|
||||
|
||||
ASSERT_EQ(get<UInt64>(upper), 7);
|
||||
ASSERT_EQ(upper.get<UInt64>(), 7);
|
||||
ASSERT_TRUE(strict);
|
||||
}
|
||||
|
||||
|
@ -177,12 +177,11 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
tryGetIdentifierNameInto(database, query->from);
|
||||
|
||||
if (like)
|
||||
query->like = safeGet<const String &>(like->as<ASTLiteral &>().value);
|
||||
query->like = like->as<ASTLiteral &>().value.safeGet<const String &>();
|
||||
|
||||
node = query;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -40,7 +40,7 @@ static FillColumnDescription::StepFunction getStepFunction(
|
||||
{
|
||||
#define DECLARE_CASE(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)
|
||||
#undef DECLARE_CASE
|
||||
@ -106,21 +106,21 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr &
|
||||
{
|
||||
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)
|
||||
throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION,
|
||||
"Value of step is to low ({} seconds). Must be >= 1 day", std::abs(avg_seconds));
|
||||
}
|
||||
|
||||
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())
|
||||
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()))
|
||||
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()))
|
||||
{
|
||||
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());
|
||||
|
||||
switch (*descr.step_kind)
|
||||
@ -129,7 +129,7 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr &
|
||||
case IntervalKind::NAME: \
|
||||
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()); \
|
||||
field = DecimalField(res, field_decimal.getScale()); \
|
||||
}; \
|
||||
|
@ -582,7 +582,7 @@ Pipe StorageLiveView::watch(
|
||||
if (query.limit_length)
|
||||
{
|
||||
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)
|
||||
|
@ -613,7 +613,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTEquals(
|
||||
|
||||
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 ASTs & arguments = typeid_cast<const ASTExpressionList &>(*function->arguments).children;
|
||||
|
||||
|
@ -333,7 +333,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
/// Get path and name from engine arguments
|
||||
ast_zk_path = engine_args[arg_num]->as<ASTLiteral>();
|
||||
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
|
||||
throw Exception(
|
||||
"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>();
|
||||
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
|
||||
throw Exception(
|
||||
"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>();
|
||||
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
|
||||
throw Exception(
|
||||
"Index granularity must be a positive integer" + getMergeTreeVerboseHelp(is_extended_storage_def),
|
||||
|
@ -1127,7 +1127,7 @@ Pipe StorageWindowView::watch(
|
||||
if (query.limit_length)
|
||||
{
|
||||
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>(
|
||||
|
@ -196,8 +196,8 @@ ColumnsDescriptionByShardNum getExtendedObjectsOfRemoteTables(
|
||||
size_t size = name_col.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
auto name = get<const String &>(name_col[i]);
|
||||
auto type_name = get<const String &>(type_col[i]);
|
||||
auto name = name_col[i].get<const String &>();
|
||||
auto type_name = type_col[i].get<const String &>();
|
||||
|
||||
auto storage_column = storage_columns.tryGetPhysical(name);
|
||||
if (storage_column && isObject(storage_column->type))
|
||||
|
@ -107,7 +107,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr
|
||||
if (lit->value.getType() != Field::Types::String)
|
||||
return false;
|
||||
|
||||
res = safeGet<const String &>(lit->value);
|
||||
res = lit->value.safeGet<const String &>();
|
||||
return true;
|
||||
};
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user