hide Field::get in private, only use Field::safeGet

This commit is contained in:
Yakov Olkhovskiy 2024-08-10 13:01:55 +00:00
parent f947f91b10
commit c1b5b908ba
192 changed files with 749 additions and 774 deletions

View File

@ -223,7 +223,7 @@ std::vector<String> Client::loadWarningMessages()
size_t rows = packet.block.rows();
for (size_t i = 0; i < rows; ++i)
messages.emplace_back(column[i].get<String>());
messages.emplace_back(column[i].safeGet<String>());
}
continue;

View File

@ -95,7 +95,7 @@ void SetCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) co
client->zookeeper->set(
client->getAbsolutePath(query->args[0].safeGet<String>()),
query->args[1].safeGet<String>(),
static_cast<Int32>(query->args[2].get<Int32>()));
static_cast<Int32>(query->args[2].safeGet<Int32>()));
}
bool CreateCommand::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const
@ -494,7 +494,7 @@ void RMCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) con
{
Int32 version{-1};
if (query->args.size() == 2)
version = static_cast<Int32>(query->args[1].get<Int32>());
version = static_cast<Int32>(query->args[1].safeGet<Int32>());
client->zookeeper->remove(client->getAbsolutePath(query->args[0].safeGet<String>()), version);
}
@ -549,7 +549,7 @@ void ReconfigCommand::execute(const DB::ASTKeeperQuery * query, DB::KeeperClient
String leaving;
String new_members;
auto operation = query->args[0].get<ReconfigCommand::Operation>();
auto operation = query->args[0].safeGet<ReconfigCommand::Operation>();
switch (operation)
{
case static_cast<UInt8>(ReconfigCommand::Operation::ADD):

View File

@ -780,12 +780,12 @@ AggregateFunctionPtr createAggregateFunctionGroupArray(
if (type != Field::Types::Int64 && type != Field::Types::UInt64)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name);
if ((type == Field::Types::Int64 && parameters[0].get<Int64>() < 0) ||
(type == Field::Types::UInt64 && parameters[0].get<UInt64>() == 0))
if ((type == Field::Types::Int64 && parameters[0].safeGet<Int64>() < 0) ||
(type == Field::Types::UInt64 && parameters[0].safeGet<UInt64>() == 0))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name);
has_limit = true;
max_elems = parameters[0].get<UInt64>();
max_elems = parameters[0].safeGet<UInt64>();
}
else
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
@ -816,11 +816,11 @@ AggregateFunctionPtr createAggregateFunctionGroupArraySample(
if (type != Field::Types::Int64 && type != Field::Types::UInt64)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name);
if ((type == Field::Types::Int64 && parameters[i].get<Int64>() < 0) ||
(type == Field::Types::UInt64 && parameters[i].get<UInt64>() == 0))
if ((type == Field::Types::Int64 && parameters[i].safeGet<Int64>() < 0) ||
(type == Field::Types::UInt64 && parameters[i].safeGet<UInt64>() == 0))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name);
return parameters[i].get<UInt64>();
return parameters[i].safeGet<UInt64>();
};
UInt64 max_elems = get_parameter(0);

View File

@ -83,16 +83,16 @@ public:
if (version == 1)
{
for (size_t i = 0; i < arr_size; ++i)
set.insert(static_cast<T>((*data_column)[offset + i].get<T>()));
set.insert(static_cast<T>((*data_column)[offset + i].safeGet<T>()));
}
else if (!set.empty())
{
typename State::Set new_set;
for (size_t i = 0; i < arr_size; ++i)
{
typename State::Set::LookupResult set_value = set.find(static_cast<T>((*data_column)[offset + i].get<T>()));
typename State::Set::LookupResult set_value = set.find(static_cast<T>((*data_column)[offset + i].safeGet<T>()));
if (set_value != nullptr)
new_set.insert(static_cast<T>((*data_column)[offset + i].get<T>()));
new_set.insert(static_cast<T>((*data_column)[offset + i].safeGet<T>()));
}
set = std::move(new_set);
}

View File

@ -269,12 +269,12 @@ AggregateFunctionPtr createAggregateFunctionMoving(
if (type != Field::Types::Int64 && type != Field::Types::UInt64)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive integer", name);
if ((type == Field::Types::Int64 && parameters[0].get<Int64>() <= 0) ||
(type == Field::Types::UInt64 && parameters[0].get<UInt64>() == 0))
if ((type == Field::Types::Int64 && parameters[0].safeGet<Int64>() <= 0) ||
(type == Field::Types::UInt64 && parameters[0].safeGet<UInt64>() == 0))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive integer", name);
limit_size = true;
max_elems = parameters[0].get<UInt64>();
max_elems = parameters[0].safeGet<UInt64>();
}
else
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,

View File

@ -397,11 +397,11 @@ AggregateFunctionPtr createAggregateFunctionGroupArray(
if (type != Field::Types::Int64 && type != Field::Types::UInt64)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name);
if ((type == Field::Types::Int64 && parameters[0].get<Int64>() < 0) ||
(type == Field::Types::UInt64 && parameters[0].get<UInt64>() == 0))
if ((type == Field::Types::Int64 && parameters[0].safeGet<Int64>() < 0) ||
(type == Field::Types::UInt64 && parameters[0].safeGet<UInt64>() == 0))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name);
max_elems = parameters[0].get<UInt64>();
max_elems = parameters[0].safeGet<UInt64>();
}
else
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,

View File

@ -247,7 +247,7 @@ AggregateFunctionPtr createAggregateFunctionGroupConcat(
if (type != Field::Types::String)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First parameter for aggregate function {} should be string", name);
delimiter = parameters[0].get<String>();
delimiter = parameters[0].safeGet<String>();
}
if (parameters.size() == 2)
{
@ -256,12 +256,12 @@ AggregateFunctionPtr createAggregateFunctionGroupConcat(
if (type != Field::Types::Int64 && type != Field::Types::UInt64)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second parameter for aggregate function {} should be a positive number", name);
if ((type == Field::Types::Int64 && parameters[1].get<Int64>() <= 0) ||
(type == Field::Types::UInt64 && parameters[1].get<UInt64>() == 0))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second parameter for aggregate function {} should be a positive number, got: {}", name, parameters[1].get<Int64>());
if ((type == Field::Types::Int64 && parameters[1].safeGet<Int64>() <= 0) ||
(type == Field::Types::UInt64 && parameters[1].safeGet<UInt64>() == 0))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second parameter for aggregate function {} should be a positive number, got: {}", name, parameters[1].safeGet<Int64>());
has_limit = true;
limit = parameters[1].get<UInt64>();
limit = parameters[1].safeGet<UInt64>();
}
if (has_limit)

View File

@ -323,12 +323,12 @@ AggregateFunctionPtr createAggregateFunctionGroupUniqArray(
if (type != Field::Types::Int64 && type != Field::Types::UInt64)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name);
if ((type == Field::Types::Int64 && parameters[0].get<Int64>() < 0) ||
(type == Field::Types::UInt64 && parameters[0].get<UInt64>() == 0))
if ((type == Field::Types::Int64 && parameters[0].safeGet<Int64>() < 0) ||
(type == Field::Types::UInt64 && parameters[0].safeGet<UInt64>() == 0))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name);
limit_size = true;
max_elems = parameters[0].get<UInt64>();
max_elems = parameters[0].safeGet<UInt64>();
}
else
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,

View File

@ -238,7 +238,7 @@ public:
if (params[0].getType() != Field::Types::String)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require first parameter to be a String", getName());
const auto & param = params[0].get<String>();
const auto & param = params[0].safeGet<String>();
if (param == "two-sided")
alternative = Alternative::TwoSided;
else if (param == "less")
@ -255,7 +255,7 @@ public:
if (params[1].getType() != Field::Types::String)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require second parameter to be a String", getName());
method = params[1].get<String>();
method = params[1].safeGet<String>();
if (method != "auto" && method != "exact" && method != "asymp" && method != "asymptotic")
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown method in aggregate function {}. "
"It must be one of: 'auto', 'exact', 'asymp' (or 'asymptotic')", getName());

View File

@ -181,7 +181,7 @@ public:
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require first parameter to be a UInt64", getName());
total_buckets = params[0].get<UInt64>();
total_buckets = params[0].safeGet<UInt64>();
this->x_type = WhichDataType(arguments[0]).idx;
this->y_type = WhichDataType(arguments[1]).idx;

View File

@ -152,7 +152,7 @@ public:
if (params[0].getType() != Field::Types::String)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require first parameter to be a String", getName());
const auto & param = params[0].get<String>();
const auto & param = params[0].safeGet<String>();
if (param == "two-sided")
alternative = Alternative::TwoSided;
else if (param == "less")
@ -169,7 +169,7 @@ public:
if (params[1].getType() != Field::Types::UInt64)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require second parameter to be a UInt64", getName());
continuity_correction = static_cast<bool>(params[1].get<UInt64>());
continuity_correction = static_cast<bool>(params[1].safeGet<UInt64>());
}
String getName() const override

View File

@ -117,7 +117,7 @@ public:
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} requires relative accuracy parameter with Float64 type", getName());
relative_accuracy = relative_accuracy_field.get<Float64>();
relative_accuracy = relative_accuracy_field.safeGet<Float64>();
if (relative_accuracy <= 0 || relative_accuracy >= 1 || isNaN(relative_accuracy))
throw Exception(
@ -147,9 +147,9 @@ public:
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} requires accuracy parameter with integer type", getName());
if (accuracy_field.getType() == Field::Types::Int64)
accuracy = accuracy_field.get<Int64>();
accuracy = accuracy_field.safeGet<Int64>();
else
accuracy = accuracy_field.get<UInt64>();
accuracy = accuracy_field.safeGet<UInt64>();
if (accuracy <= 0)
throw Exception(

View File

@ -300,12 +300,12 @@ public:
/// Compatibility with previous versions.
if (value.getType() == Field::Types::Decimal32)
{
auto source = value.get<DecimalField<Decimal32>>();
auto source = value.safeGet<DecimalField<Decimal32>>();
value = DecimalField<Decimal128>(source.getValue(), source.getScale());
}
else if (value.getType() == Field::Types::Decimal64)
{
auto source = value.get<DecimalField<Decimal64>>();
auto source = value.safeGet<DecimalField<Decimal64>>();
value = DecimalField<Decimal128>(source.getValue(), source.getScale());
}
@ -355,7 +355,7 @@ public:
/// Compatibility with previous versions.
if (value.getType() == Field::Types::Decimal128)
{
auto source = value.get<DecimalField<Decimal128>>();
auto source = value.safeGet<DecimalField<Decimal128>>();
WhichDataType value_type(values_types[col_idx]);
if (value_type.isDecimal32())
{
@ -560,7 +560,7 @@ private:
template <typename FieldType>
bool compareImpl(FieldType & x) const
{
auto val = rhs.get<FieldType>();
auto val = rhs.safeGet<FieldType>();
if (val > x)
{
x = val;
@ -600,7 +600,7 @@ private:
template <typename FieldType>
bool compareImpl(FieldType & x) const
{
auto val = rhs.get<FieldType>();
auto val = rhs.safeGet<FieldType>();
if (val < x)
{
x = val;

View File

@ -129,7 +129,7 @@ private:
if (constant_node_value.getType() != Field::Types::Which::Tuple)
return {};
const auto & constant_tuple = constant_node_value.get<const Tuple &>();
const auto & constant_tuple = constant_node_value.safeGet<const Tuple &>();
const auto & function_arguments_nodes = function_node_typed.getArguments().getNodes();
size_t function_arguments_nodes_size = function_arguments_nodes.size();

View File

@ -89,7 +89,7 @@ public:
if (!pattern || !isString(pattern->getResultType()))
continue;
auto regexp = likePatternToRegexp(pattern->getValue().get<String>());
auto regexp = likePatternToRegexp(pattern->getValue().safeGet<String>());
/// Case insensitive. Works with UTF-8 as well.
if (is_ilike)
regexp = "(?i)" + regexp;

View File

@ -68,10 +68,10 @@ void optimizeFunctionEmpty(QueryTreeNodePtr &, FunctionNode & function_node, Col
String getSubcolumnNameForElement(const Field & value, const DataTypeTuple & data_type_tuple)
{
if (value.getType() == Field::Types::String)
return value.get<const String &>();
return value.safeGet<const String &>();
if (value.getType() == Field::Types::UInt64)
return data_type_tuple.getNameByPosition(value.get<UInt64>());
return data_type_tuple.getNameByPosition(value.safeGet<UInt64>());
return "";
}
@ -79,7 +79,7 @@ String getSubcolumnNameForElement(const Field & value, const DataTypeTuple & dat
String getSubcolumnNameForElement(const Field & value, const DataTypeVariant &)
{
if (value.getType() == Field::Types::String)
return value.get<const String &>();
return value.safeGet<const String &>();
return "";
}

View File

@ -187,7 +187,7 @@ FunctionNodePtr createFusedQuantilesNode(std::vector<QueryTreeNodePtr *> & nodes
/// Sort nodes and parameters in ascending order of quantile level
std::vector<size_t> permutation(nodes.size());
iota(permutation.data(), permutation.size(), size_t(0));
std::sort(permutation.begin(), permutation.end(), [&](size_t i, size_t j) { return parameters[i].get<Float64>() < parameters[j].get<Float64>(); });
std::sort(permutation.begin(), permutation.end(), [&](size_t i, size_t j) { return parameters[i].safeGet<Float64>() < parameters[j].safeGet<Float64>(); });
std::vector<QueryTreeNodePtr *> new_nodes;
new_nodes.reserve(permutation.size());

View File

@ -134,8 +134,8 @@ public:
return;
std::set<std::string> string_values;
string_values.insert(first_literal->getValue().get<std::string>());
string_values.insert(second_literal->getValue().get<std::string>());
string_values.insert(first_literal->getValue().safeGet<std::string>());
string_values.insert(second_literal->getValue().safeGet<std::string>());
changeIfArguments(*function_if_node, string_values, context);
wrapIntoToString(*function_node, std::move(modified_if_node), context);
@ -163,7 +163,7 @@ public:
if (!isArray(literal_to->getResultType()) || !isString(literal_default->getResultType()))
return;
auto array_to = literal_to->getValue().get<Array>();
auto array_to = literal_to->getValue().safeGet<Array>();
if (array_to.empty())
return;
@ -178,9 +178,9 @@ public:
std::set<std::string> string_values;
for (const auto & value : array_to)
string_values.insert(value.get<std::string>());
string_values.insert(value.safeGet<std::string>());
string_values.insert(literal_default->getValue().get<std::string>());
string_values.insert(literal_default->getValue().safeGet<std::string>());
changeTransformArguments(*function_modified_transform_node, string_values, context);
wrapIntoToString(*function_node, std::move(modified_transform_node), context);

View File

@ -54,7 +54,7 @@ public:
}
else if (function_node->getFunctionName() == "sum" &&
first_argument_constant_literal.getType() == Field::Types::UInt64 &&
first_argument_constant_literal.get<UInt64>() == 1)
first_argument_constant_literal.safeGet<UInt64>() == 1)
{
function_node->getArguments().getNodes().clear();
resolveAggregateFunctionNodeByName(*function_node, "count");

View File

@ -143,13 +143,13 @@ private:
const auto & column_type = column_node_typed.getColumnType().get();
if (isDateOrDate32(column_type))
{
start_date_or_date_time = date_lut.dateToString(range.first.get<DateLUTImpl::Time>());
end_date_or_date_time = date_lut.dateToString(range.second.get<DateLUTImpl::Time>());
start_date_or_date_time = date_lut.dateToString(range.first.safeGet<DateLUTImpl::Time>());
end_date_or_date_time = date_lut.dateToString(range.second.safeGet<DateLUTImpl::Time>());
}
else if (isDateTime(column_type) || isDateTime64(column_type))
{
start_date_or_date_time = date_lut.timeToString(range.first.get<DateLUTImpl::Time>());
end_date_or_date_time = date_lut.timeToString(range.second.get<DateLUTImpl::Time>());
start_date_or_date_time = date_lut.timeToString(range.first.safeGet<DateLUTImpl::Time>());
end_date_or_date_time = date_lut.timeToString(range.second.safeGet<DateLUTImpl::Time>());
}
else [[unlikely]]
return {};

View File

@ -60,7 +60,7 @@ public:
{
const auto & second_const_value = second_const_node->getValue();
if (second_const_value.isNull()
|| (lower_name == "sum" && isInt64OrUInt64FieldType(second_const_value.getType()) && second_const_value.get<UInt64>() == 0
|| (lower_name == "sum" && isInt64OrUInt64FieldType(second_const_value.getType()) && second_const_value.safeGet<UInt64>() == 0
&& !if_node->getResultType()->isNullable()))
{
/// avg(if(cond, a, null)) -> avgIf(a::ResultTypeIf, cond)
@ -89,7 +89,7 @@ public:
{
const auto & first_const_value = first_const_node->getValue();
if (first_const_value.isNull()
|| (lower_name == "sum" && isInt64OrUInt64FieldType(first_const_value.getType()) && first_const_value.get<UInt64>() == 0
|| (lower_name == "sum" && isInt64OrUInt64FieldType(first_const_value.getType()) && first_const_value.safeGet<UInt64>() == 0
&& !if_node->getResultType()->isNullable()))
{
/// avg(if(cond, null, a) -> avgIf(a::ResultTypeIf, !cond))

View File

@ -66,7 +66,7 @@ public:
resolveAggregateFunctionNodeByName(*function_node, "countIf");
if (constant_value_literal.get<UInt64>() != 1)
if (constant_value_literal.safeGet<UInt64>() != 1)
{
/// Rewrite `sumIf(123, cond)` into `123 * countIf(cond)`
node = getMultiplyFunction(std::move(multiplier_node), node);
@ -105,8 +105,8 @@ public:
const auto & if_true_condition_constant_value_literal = if_true_condition_constant_node->getValue();
const auto & if_false_condition_constant_value_literal = if_false_condition_constant_node->getValue();
auto if_true_condition_value = if_true_condition_constant_value_literal.get<UInt64>();
auto if_false_condition_value = if_false_condition_constant_value_literal.get<UInt64>();
auto if_true_condition_value = if_true_condition_constant_value_literal.safeGet<UInt64>();
auto if_false_condition_value = if_false_condition_constant_value_literal.safeGet<UInt64>();
if (if_false_condition_value == 0)
{

View File

@ -471,7 +471,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildSortList(const ASTPtr & order_by_express
std::shared_ptr<Collator> collator;
if (order_by_element.getCollation())
collator = std::make_shared<Collator>(order_by_element.getCollation()->as<ASTLiteral &>().value.get<String &>());
collator = std::make_shared<Collator>(order_by_element.getCollation()->as<ASTLiteral &>().value.safeGet<String &>());
const auto & sort_expression_ast = order_by_element.children.at(0);
auto sort_expression = buildExpression(sort_expression_ast, context);

View File

@ -1273,7 +1273,7 @@ QueryTreeNodePtr IdentifierResolver::matchArrayJoinSubcolumns(
const auto & constant_node_value = constant_node.getValue();
if (constant_node_value.getType() == Field::Types::String)
{
array_join_subcolumn_prefix = constant_node_value.get<String>() + ".";
array_join_subcolumn_prefix = constant_node_value.safeGet<String>() + ".";
array_join_parent_column = argument_nodes.at(0).get();
}
}
@ -1287,7 +1287,7 @@ QueryTreeNodePtr IdentifierResolver::matchArrayJoinSubcolumns(
if (!second_argument || second_argument->getValue().getType() != Field::Types::String)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected constant string as second argument of getSubcolumn function {}", resolved_function->dumpTree());
const auto & resolved_subcolumn_path = second_argument->getValue().get<String &>();
const auto & resolved_subcolumn_path = second_argument->getValue().safeGet<String &>();
if (!startsWith(resolved_subcolumn_path, array_join_subcolumn_prefix))
return {};
@ -1331,7 +1331,7 @@ QueryTreeNodePtr IdentifierResolver::tryResolveExpressionFromArrayJoinExpression
size_t nested_function_arguments_size = nested_function_arguments.size();
const auto & nested_keys_names_constant_node = nested_function_arguments[0]->as<ConstantNode & >();
const auto & nested_keys_names = nested_keys_names_constant_node.getValue().get<Array &>();
const auto & nested_keys_names = nested_keys_names_constant_node.getValue().safeGet<Array &>();
size_t nested_keys_names_size = nested_keys_names.size();
if (nested_keys_names_size == nested_function_arguments_size - 1)
@ -1344,7 +1344,7 @@ QueryTreeNodePtr IdentifierResolver::tryResolveExpressionFromArrayJoinExpression
auto array_join_column = std::make_shared<ColumnNode>(array_join_column_expression_typed.getColumn(),
array_join_column_expression_typed.getColumnSource());
const auto & nested_key_name = nested_keys_names[i - 1].get<String &>();
const auto & nested_key_name = nested_keys_names[i - 1].safeGet<String &>();
Identifier nested_identifier = Identifier(nested_key_name);
array_join_resolved_expression = wrapExpressionNodeInTupleElement(array_join_column, nested_identifier, scope.context);
break;

View File

@ -746,11 +746,11 @@ void QueryAnalyzer::replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_
UInt64 pos;
if (constant_node->getValue().getType() == Field::Types::UInt64)
{
pos = constant_node->getValue().get<UInt64>();
pos = constant_node->getValue().safeGet<UInt64>();
}
else // Int64
{
auto value = constant_node->getValue().get<Int64>();
auto value = constant_node->getValue().safeGet<Int64>();
if (value > 0)
pos = value;
else

View File

@ -93,7 +93,7 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes&
"Invalid type in set. Expected tuple, got {}",
value.getTypeName());
const auto & tuple = value.template get<const Tuple &>();
const auto & tuple = value.template safeGet<const Tuple &>();
const DataTypePtr & value_type = value_types[collection_index];
const DataTypes & tuple_value_type = typeid_cast<const DataTypeTuple *>(value_type.get())->getElements();
@ -169,15 +169,15 @@ Block getSetElementsForConstantValue(const DataTypePtr & expression_type, const
if (rhs_which_type.isArray())
{
const DataTypeArray * value_array_type = assert_cast<const DataTypeArray *>(value_type.get());
size_t value_array_size = value.get<const Array &>().size();
size_t value_array_size = value.safeGet<const Array &>().size();
DataTypes value_types(value_array_size, value_array_type->getNestedType());
result_block = createBlockFromCollection(value.get<const Array &>(), value_types, set_element_types, transform_null_in);
result_block = createBlockFromCollection(value.safeGet<const Array &>(), value_types, set_element_types, transform_null_in);
}
else if (rhs_which_type.isTuple())
{
const DataTypeTuple * value_tuple_type = assert_cast<const DataTypeTuple *>(value_type.get());
const DataTypes & value_types = value_tuple_type->getElements();
result_block = createBlockFromCollection(value.get<const Tuple &>(), value_types, set_element_types, transform_null_in);
result_block = createBlockFromCollection(value.safeGet<const Tuple &>(), value_types, set_element_types, transform_null_in);
}
else
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,

View File

@ -125,7 +125,7 @@ std::vector<Strings> BackupSettings::Util::clusterHostIDsFromAST(const IAST & as
throw Exception(
ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS,
"Setting cluster_host_ids has wrong format, must be array of arrays of string literals");
const auto & replicas = array_of_replicas->value.get<const Array &>();
const auto & replicas = array_of_replicas->value.safeGet<const Array &>();
res[i].resize(replicas.size());
for (size_t j = 0; j != replicas.size(); ++j)
{
@ -134,7 +134,7 @@ std::vector<Strings> BackupSettings::Util::clusterHostIDsFromAST(const IAST & as
throw Exception(
ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS,
"Setting cluster_host_ids has wrong format, must be array of arrays of string literals");
res[i][j] = replica.get<const String &>();
res[i][j] = replica.safeGet<const String &>();
}
}
}

View File

@ -46,8 +46,8 @@ namespace
if (zookeeper_path_ast && (zookeeper_path_ast->value.getType() == Field::Types::String) &&
replica_name_ast && (replica_name_ast->value.getType() == Field::Types::String))
{
String & zookeeper_path_arg = zookeeper_path_ast->value.get<String>();
String & replica_name_arg = replica_name_ast->value.get<String>();
String & zookeeper_path_arg = zookeeper_path_ast->value.safeGet<String>();
String & replica_name_arg = replica_name_ast->value.safeGet<String>();
if (create.uuid != UUIDHelpers::Nil)
{
String table_uuid_str = toString(create.uuid);

View File

@ -31,7 +31,7 @@ namespace
{
if (field.getType() == Field::Types::String)
{
const String & str = field.get<const String &>();
const String & str = field.safeGet<const String &>();
if (str == "1" || boost::iequals(str, "true") || boost::iequals(str, "create"))
{
value = RestoreTableCreationMode::kCreate;
@ -54,7 +54,7 @@ namespace
if (field.getType() == Field::Types::UInt64)
{
UInt64 number = field.get<UInt64>();
UInt64 number = field.safeGet<UInt64>();
if (number == 1)
{
value = RestoreTableCreationMode::kCreate;
@ -95,7 +95,7 @@ namespace
{
if (field.getType() == Field::Types::String)
{
const String & str = field.get<const String &>();
const String & str = field.safeGet<const String &>();
if (str == "1" || boost::iequals(str, "true") || boost::iequals(str, "create"))
{
value = RestoreAccessCreationMode::kCreate;
@ -118,7 +118,7 @@ namespace
if (field.getType() == Field::Types::UInt64)
{
UInt64 number = field.get<UInt64>();
UInt64 number = field.safeGet<UInt64>();
if (number == 1)
{
value = RestoreAccessCreationMode::kCreate;

View File

@ -19,7 +19,7 @@ SettingFieldOptionalString::SettingFieldOptionalString(const Field & field)
if (field.getType() == Field::Types::String)
{
value = field.get<const String &>();
value = field.safeGet<const String &>();
return;
}

View File

@ -22,7 +22,7 @@ namespace ErrorCodes
if (field.getType() == Field::Types::String)
{
const String & str = field.get<const String &>();
const String & str = field.safeGet<const String &>();
if (str.empty())
{
value = std::nullopt;

View File

@ -133,7 +133,7 @@ Field QueryFuzzer::fuzzField(Field field)
if (type == Field::Types::String)
{
auto & str = field.get<std::string>();
auto & str = field.safeGet<std::string>();
UInt64 action = fuzz_rand() % 10;
switch (action)
{
@ -159,7 +159,7 @@ Field QueryFuzzer::fuzzField(Field field)
}
else if (type == Field::Types::Array)
{
auto & arr = field.get<Array>();
auto & arr = field.safeGet<Array>();
if (fuzz_rand() % 5 == 0 && !arr.empty())
{
@ -191,7 +191,7 @@ Field QueryFuzzer::fuzzField(Field field)
}
else if (type == Field::Types::Tuple)
{
auto & arr = field.get<Tuple>();
auto & arr = field.safeGet<Tuple>();
if (fuzz_rand() % 5 == 0 && !arr.empty())
{
@ -912,17 +912,17 @@ ASTPtr QueryFuzzer::fuzzLiteralUnderExpressionList(ASTPtr child)
auto type = l->value.getType();
if (type == Field::Types::Which::String && fuzz_rand() % 7 == 0)
{
String value = l->value.get<String>();
String value = l->value.safeGet<String>();
child = makeASTFunction(
"toFixedString", std::make_shared<ASTLiteral>(value), std::make_shared<ASTLiteral>(static_cast<UInt64>(value.size())));
}
else if (type == Field::Types::Which::UInt64 && fuzz_rand() % 7 == 0)
{
child = makeASTFunction(fuzz_rand() % 2 == 0 ? "toUInt128" : "toUInt256", std::make_shared<ASTLiteral>(l->value.get<UInt64>()));
child = makeASTFunction(fuzz_rand() % 2 == 0 ? "toUInt128" : "toUInt256", std::make_shared<ASTLiteral>(l->value.safeGet<UInt64>()));
}
else if (type == Field::Types::Which::Int64 && fuzz_rand() % 7 == 0)
{
child = makeASTFunction(fuzz_rand() % 2 == 0 ? "toInt128" : "toInt256", std::make_shared<ASTLiteral>(l->value.get<Int64>()));
child = makeASTFunction(fuzz_rand() % 2 == 0 ? "toInt128" : "toInt256", std::make_shared<ASTLiteral>(l->value.safeGet<Int64>()));
}
else if (type == Field::Types::Which::Float64 && fuzz_rand() % 7 == 0)
{
@ -930,22 +930,22 @@ ASTPtr QueryFuzzer::fuzzLiteralUnderExpressionList(ASTPtr child)
if (decimal == 0)
child = makeASTFunction(
"toDecimal32",
std::make_shared<ASTLiteral>(l->value.get<Float64>()),
std::make_shared<ASTLiteral>(l->value.safeGet<Float64>()),
std::make_shared<ASTLiteral>(static_cast<UInt64>(fuzz_rand() % 9)));
else if (decimal == 1)
child = makeASTFunction(
"toDecimal64",
std::make_shared<ASTLiteral>(l->value.get<Float64>()),
std::make_shared<ASTLiteral>(l->value.safeGet<Float64>()),
std::make_shared<ASTLiteral>(static_cast<UInt64>(fuzz_rand() % 18)));
else if (decimal == 2)
child = makeASTFunction(
"toDecimal128",
std::make_shared<ASTLiteral>(l->value.get<Float64>()),
std::make_shared<ASTLiteral>(l->value.safeGet<Float64>()),
std::make_shared<ASTLiteral>(static_cast<UInt64>(fuzz_rand() % 38)));
else
child = makeASTFunction(
"toDecimal256",
std::make_shared<ASTLiteral>(l->value.get<Float64>()),
std::make_shared<ASTLiteral>(l->value.safeGet<Float64>()),
std::make_shared<ASTLiteral>(static_cast<UInt64>(fuzz_rand() % 76)));
}

View File

@ -214,7 +214,7 @@ void Suggest::fillWordsFromBlock(const Block & block)
Words new_words;
new_words.reserve(rows);
for (size_t i = 0; i < rows; ++i)
new_words.emplace_back(column[i].get<String>());
new_words.emplace_back(column[i].safeGet<String>());
addWords(std::move(new_words));
}

View File

@ -426,9 +426,9 @@ MutableColumnPtr ColumnAggregateFunction::cloneEmpty() const
Field ColumnAggregateFunction::operator[](size_t n) const
{
Field field = AggregateFunctionStateData();
field.get<AggregateFunctionStateData &>().name = type_string;
field.safeGet<AggregateFunctionStateData &>().name = type_string;
{
WriteBufferFromString buffer(field.get<AggregateFunctionStateData &>().data);
WriteBufferFromString buffer(field.safeGet<AggregateFunctionStateData &>().data);
func->serialize(data[n], buffer, version);
}
return field;
@ -436,12 +436,7 @@ Field ColumnAggregateFunction::operator[](size_t n) const
void ColumnAggregateFunction::get(size_t n, Field & res) const
{
res = AggregateFunctionStateData();
res.get<AggregateFunctionStateData &>().name = type_string;
{
WriteBufferFromString buffer(res.get<AggregateFunctionStateData &>().data);
func->serialize(data[n], buffer, version);
}
res = operator[](n);
}
StringRef ColumnAggregateFunction::getDataAt(size_t n) const
@ -521,7 +516,7 @@ void ColumnAggregateFunction::insert(const Field & x)
"Inserting field of type {} into ColumnAggregateFunction. Expected {}",
x.getTypeName(), Field::Types::AggregateFunctionState);
const auto & field_name = x.get<const AggregateFunctionStateData &>().name;
const auto & field_name = x.safeGet<const AggregateFunctionStateData &>().name;
if (type_string != field_name)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Cannot insert filed with type {} into column with type {}",
field_name, type_string);
@ -529,7 +524,7 @@ void ColumnAggregateFunction::insert(const Field & x)
ensureOwnership();
Arena & arena = createOrGetArena();
pushBackAndCreateState(data, arena, func.get());
ReadBufferFromString read_buffer(x.get<const AggregateFunctionStateData &>().data);
ReadBufferFromString read_buffer(x.safeGet<const AggregateFunctionStateData &>().data);
func->deserialize(data.back(), read_buffer, version, &arena);
}
@ -538,14 +533,14 @@ bool ColumnAggregateFunction::tryInsert(const DB::Field & x)
if (x.getType() != Field::Types::AggregateFunctionState)
return false;
const auto & field_name = x.get<const AggregateFunctionStateData &>().name;
const auto & field_name = x.safeGet<const AggregateFunctionStateData &>().name;
if (type_string != field_name)
return false;
ensureOwnership();
Arena & arena = createOrGetArena();
pushBackAndCreateState(data, arena, func.get());
ReadBufferFromString read_buffer(x.get<const AggregateFunctionStateData &>().data);
ReadBufferFromString read_buffer(x.safeGet<const AggregateFunctionStateData &>().data);
func->deserialize(data.back(), read_buffer, version, &arena);
return true;
}

View File

@ -141,7 +141,7 @@ void ColumnArray::get(size_t n, Field & res) const
size, max_array_size_as_field);
res = Array();
Array & res_arr = res.get<Array &>();
Array & res_arr = res.safeGet<Array &>();
res_arr.reserve(size);
for (size_t i = 0; i < size; ++i)
@ -309,7 +309,7 @@ void ColumnArray::updateHashFast(SipHash & hash) const
void ColumnArray::insert(const Field & x)
{
const Array & array = x.get<const Array &>();
const Array & array = x.safeGet<const Array &>();
size_t size = array.size();
for (size_t i = 0; i < size; ++i)
getData().insert(array[i]);
@ -321,7 +321,7 @@ bool ColumnArray::tryInsert(const Field & x)
if (x.getType() != Field::Types::Which::Array)
return false;
const Array & array = x.get<const Array &>();
const Array & array = x.safeGet<const Array &>();
size_t size = array.size();
for (size_t i = 0; i < size; ++i)
{

View File

@ -74,7 +74,7 @@ public:
void insertData(const char * src, size_t /*length*/) override;
void insertDefault() override { data.push_back(T()); }
void insertManyDefaults(size_t length) override { data.resize_fill(data.size() + length); }
void insert(const Field & x) override { data.push_back(x.get<T>()); }
void insert(const Field & x) override { data.push_back(x.safeGet<T>()); }
bool tryInsert(const Field & x) override;
#if !defined(DEBUG_OR_SANITIZER_BUILD)
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;

View File

@ -59,7 +59,7 @@ bool ColumnFixedString::isDefaultAt(size_t index) const
void ColumnFixedString::insert(const Field & x)
{
const String & s = x.get<const String &>();
const String & s = x.safeGet<const String &>();
insertData(s.data(), s.size());
}
@ -67,7 +67,7 @@ bool ColumnFixedString::tryInsert(const Field & x)
{
if (x.getType() != Field::Types::Which::String)
return false;
const String & s = x.get<const String &>();
const String & s = x.safeGet<const String &>();
if (s.size() > n)
return false;
insertData(s.data(), s.size());

View File

@ -72,7 +72,7 @@ void ColumnMap::get(size_t n, Field & res) const
size_t size = offsets[n] - offsets[n - 1];
res = Map();
auto & map = res.get<Map &>();
auto & map = res.safeGet<Map &>();
map.reserve(size);
for (size_t i = 0; i < size; ++i)
@ -96,7 +96,7 @@ void ColumnMap::insertData(const char *, size_t)
void ColumnMap::insert(const Field & x)
{
const auto & map = x.get<const Map &>();
const auto & map = x.safeGet<const Map &>();
nested->insert(Array(map.begin(), map.end()));
}
@ -105,7 +105,7 @@ bool ColumnMap::tryInsert(const Field & x)
if (x.getType() != Field::Types::Which::Map)
return false;
const auto & map = x.get<const Map &>();
const auto & map = x.safeGet<const Map &>();
return nested->tryInsert(Array(map.begin(), map.end()));
}
@ -288,8 +288,8 @@ void ColumnMap::getExtremes(Field & min, Field & max) const
/// Convert result Array fields to Map fields because client expect min and max field to have type Map
Array nested_min_value = nested_min.get<Array>();
Array nested_max_value = nested_max.get<Array>();
Array nested_min_value = nested_min.safeGet<Array>();
Array nested_max_value = nested_max.safeGet<Array>();
Map map_min_value(nested_min_value.begin(), nested_min_value.end());
Map map_max_value(nested_max_value.begin(), nested_max_value.end());

View File

@ -698,7 +698,7 @@ void ColumnObject::forEachSubcolumnRecursively(RecursiveMutableColumnCallback ca
void ColumnObject::insert(const Field & field)
{
const auto & object = field.get<const Object &>();
const auto & object = field.safeGet<const Object &>();
HashSet<StringRef, StringRefHash> inserted_paths;
size_t old_size = size();
@ -754,7 +754,7 @@ void ColumnObject::get(size_t n, Field & res) const
{
assert(n < size());
res = Object();
auto & object = res.get<Object &>();
auto & object = res.safeGet<Object &>();
for (const auto & entry : subcolumns)
{

View File

@ -123,7 +123,7 @@ public:
void insert(const Field & x) override
{
const String & s = x.get<const String &>();
const String & s = x.safeGet<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;

View File

@ -141,7 +141,7 @@ void ColumnTuple::get(size_t n, Field & res) const
const size_t tuple_size = columns.size();
res = Tuple();
Tuple & res_tuple = res.get<Tuple &>();
Tuple & res_tuple = res.safeGet<Tuple &>();
res_tuple.reserve(tuple_size);
for (size_t i = 0; i < tuple_size; ++i)
@ -169,7 +169,7 @@ void ColumnTuple::insertData(const char *, size_t)
void ColumnTuple::insert(const Field & x)
{
const auto & tuple = x.get<const Tuple &>();
const auto & tuple = x.safeGet<const Tuple &>();
const size_t tuple_size = columns.size();
if (tuple.size() != tuple_size)
@ -185,7 +185,7 @@ bool ColumnTuple::tryInsert(const Field & x)
if (x.getType() != Field::Types::Which::Tuple)
return false;
const auto & tuple = x.get<const Tuple &>();
const auto & tuple = x.safeGet<const Tuple &>();
const size_t tuple_size = columns.size();
if (tuple.size() != tuple_size)

View File

@ -85,7 +85,7 @@ public:
void insertMany(const Field & field, size_t length) override
{
data.resize_fill(data.size() + length, static_cast<T>(field.get<T>()));
data.resize_fill(data.size() + length, static_cast<T>(field.safeGet<T>()));
}
void insertData(const char * pos, size_t) override
@ -235,7 +235,7 @@ public:
void insert(const Field & x) override
{
data.push_back(static_cast<T>(x.get<T>()));
data.push_back(static_cast<T>(x.safeGet<T>()));
}
bool tryInsert(const DB::Field & x) override;

View File

@ -108,10 +108,10 @@ void checkColumnVariant1(ColumnVariant * column)
ASSERT_EQ(offsets[1], 0);
ASSERT_EQ(offsets[3], 1);
ASSERT_TRUE(column->isDefaultAt(2) && column->isDefaultAt(4));
ASSERT_EQ((*column)[0].get<UInt32>(), 42);
ASSERT_EQ((*column)[1].get<String>(), "Hello");
ASSERT_EQ((*column)[0].safeGet<UInt32>(), 42);
ASSERT_EQ((*column)[1].safeGet<String>(), "Hello");
ASSERT_TRUE((*column)[2].isNull());
ASSERT_EQ((*column)[3].get<String>(), "World");
ASSERT_EQ((*column)[3].safeGet<String>(), "World");
ASSERT_TRUE((*column)[4].isNull());
}
@ -209,9 +209,9 @@ TEST(ColumnVariant, CreateFromDiscriminatorsAndOneFullColumnNoNulls)
ASSERT_EQ(offsets[0], 0);
ASSERT_EQ(offsets[1], 1);
ASSERT_EQ(offsets[2], 2);
ASSERT_EQ((*column)[0].get<UInt64>(), 0);
ASSERT_EQ((*column)[1].get<UInt64>(), 1);
ASSERT_EQ((*column)[2].get<UInt64>(), 2);
ASSERT_EQ((*column)[0].safeGet<UInt64>(), 0);
ASSERT_EQ((*column)[1].safeGet<UInt64>(), 1);
ASSERT_EQ((*column)[2].safeGet<UInt64>(), 2);
}
TEST(ColumnVariant, CreateFromDiscriminatorsAndOneFullColumnNoNullsWithLocalOrder)
@ -222,9 +222,9 @@ TEST(ColumnVariant, CreateFromDiscriminatorsAndOneFullColumnNoNullsWithLocalOrde
ASSERT_EQ(offsets[0], 0);
ASSERT_EQ(offsets[1], 1);
ASSERT_EQ(offsets[2], 2);
ASSERT_EQ((*column)[0].get<UInt64>(), 0);
ASSERT_EQ((*column)[1].get<UInt64>(), 1);
ASSERT_EQ((*column)[2].get<UInt64>(), 2);
ASSERT_EQ((*column)[0].safeGet<UInt64>(), 0);
ASSERT_EQ((*column)[1].safeGet<UInt64>(), 1);
ASSERT_EQ((*column)[2].safeGet<UInt64>(), 2);
ASSERT_EQ(column->localDiscriminatorAt(0), 2);
ASSERT_EQ(column->localDiscriminatorAt(1), 2);
ASSERT_EQ(column->localDiscriminatorAt(2), 2);
@ -331,9 +331,9 @@ TEST(ColumnVariant, CloneResizedGeneral1)
ASSERT_EQ(offsets[0], 0);
ASSERT_EQ(offsets[1], 0);
ASSERT_EQ(offsets[3], 1);
ASSERT_EQ((*resized_column_variant)[0].get<UInt64>(), 42);
ASSERT_EQ((*resized_column_variant)[1].get<String>(), "Hello");
ASSERT_EQ((*resized_column_variant)[3].get<UInt64>(), 43);
ASSERT_EQ((*resized_column_variant)[0].safeGet<UInt64>(), 42);
ASSERT_EQ((*resized_column_variant)[1].safeGet<String>(), "Hello");
ASSERT_EQ((*resized_column_variant)[3].safeGet<UInt64>(), 43);
}
TEST(ColumnVariant, CloneResizedGeneral2)
@ -367,7 +367,7 @@ TEST(ColumnVariant, CloneResizedGeneral2)
ASSERT_EQ(discriminators[2], ColumnVariant::NULL_DISCRIMINATOR);
const auto & offsets = resized_column_variant->getOffsets();
ASSERT_EQ(offsets[0], 0);
ASSERT_EQ((*resized_column_variant)[0].get<UInt64>(), 42);
ASSERT_EQ((*resized_column_variant)[0].safeGet<UInt64>(), 42);
}
TEST(ColumnVariant, CloneResizedGeneral3)
@ -405,10 +405,10 @@ TEST(ColumnVariant, CloneResizedGeneral3)
ASSERT_EQ(offsets[1], 0);
ASSERT_EQ(offsets[2], 1);
ASSERT_EQ(offsets[3], 1);
ASSERT_EQ((*resized_column_variant)[0].get<UInt64>(), 42);
ASSERT_EQ((*resized_column_variant)[1].get<String>(), "Hello");
ASSERT_EQ((*resized_column_variant)[2].get<String>(), "World");
ASSERT_EQ((*resized_column_variant)[3].get<UInt64>(), 43);
ASSERT_EQ((*resized_column_variant)[0].safeGet<UInt64>(), 42);
ASSERT_EQ((*resized_column_variant)[1].safeGet<String>(), "Hello");
ASSERT_EQ((*resized_column_variant)[2].safeGet<String>(), "World");
ASSERT_EQ((*resized_column_variant)[3].safeGet<UInt64>(), 43);
}
MutableColumnPtr createDiscriminators2()
@ -465,7 +465,7 @@ TEST(ColumnVariant, InsertFrom)
auto column_from = createVariantColumn2(change_order);
column_to->insertFrom(*column_from, 3);
ASSERT_EQ(column_to->globalDiscriminatorAt(5), 0);
ASSERT_EQ((*column_to)[5].get<UInt64>(), 43);
ASSERT_EQ((*column_to)[5].safeGet<UInt64>(), 43);
}
}
@ -478,8 +478,8 @@ TEST(ColumnVariant, InsertRangeFromOneColumnNoNulls)
column_to->insertRangeFrom(*column_from, 2, 2);
ASSERT_EQ(column_to->globalDiscriminatorAt(7), 0);
ASSERT_EQ(column_to->globalDiscriminatorAt(8), 0);
ASSERT_EQ((*column_to)[7].get<UInt64>(), 2);
ASSERT_EQ((*column_to)[8].get<UInt64>(), 3);
ASSERT_EQ((*column_to)[7].safeGet<UInt64>(), 2);
ASSERT_EQ((*column_to)[8].safeGet<UInt64>(), 3);
}
}
@ -494,9 +494,9 @@ TEST(ColumnVariant, InsertRangeFromGeneral)
ASSERT_EQ(column_to->globalDiscriminatorAt(6), ColumnVariant::NULL_DISCRIMINATOR);
ASSERT_EQ(column_to->globalDiscriminatorAt(7), 0);
ASSERT_EQ(column_to->globalDiscriminatorAt(8), 1);
ASSERT_EQ((*column_to)[5].get<String>(), "Hello");
ASSERT_EQ((*column_to)[7].get<UInt64>(), 43);
ASSERT_EQ((*column_to)[8].get<String>(), "World");
ASSERT_EQ((*column_to)[5].safeGet<String>(), "Hello");
ASSERT_EQ((*column_to)[7].safeGet<UInt64>(), 43);
ASSERT_EQ((*column_to)[8].safeGet<String>(), "World");
}
}
@ -509,8 +509,8 @@ TEST(ColumnVariant, InsertManyFrom)
column_to->insertManyFrom(*column_from, 3, 2);
ASSERT_EQ(column_to->globalDiscriminatorAt(5), 0);
ASSERT_EQ(column_to->globalDiscriminatorAt(6), 0);
ASSERT_EQ((*column_to)[5].get<UInt64>(), 43);
ASSERT_EQ((*column_to)[6].get<UInt64>(), 43);
ASSERT_EQ((*column_to)[5].safeGet<UInt64>(), 43);
ASSERT_EQ((*column_to)[6].safeGet<UInt64>(), 43);
}
}
@ -520,8 +520,8 @@ TEST(ColumnVariant, PopBackOneColumnNoNulls)
column->popBack(3);
ASSERT_EQ(column->size(), 2);
ASSERT_EQ(column->getVariantByLocalDiscriminator(0).size(), 2);
ASSERT_EQ((*column)[0].get<UInt64>(), 0);
ASSERT_EQ((*column)[1].get<UInt64>(), 1);
ASSERT_EQ((*column)[0].safeGet<UInt64>(), 0);
ASSERT_EQ((*column)[1].safeGet<UInt64>(), 1);
}
TEST(ColumnVariant, PopBackGeneral)
@ -531,8 +531,8 @@ TEST(ColumnVariant, PopBackGeneral)
ASSERT_EQ(column->size(), 3);
ASSERT_EQ(column->getVariantByLocalDiscriminator(0).size(), 1);
ASSERT_EQ(column->getVariantByLocalDiscriminator(1).size(), 1);
ASSERT_EQ((*column)[0].get<UInt64>(), 42);
ASSERT_EQ((*column)[1].get<String>(), "Hello");
ASSERT_EQ((*column)[0].safeGet<UInt64>(), 42);
ASSERT_EQ((*column)[1].safeGet<String>(), "Hello");
ASSERT_TRUE((*column)[2].isNull());
}
@ -545,8 +545,8 @@ TEST(ColumnVariant, FilterOneColumnNoNulls)
filter.push_back(1);
auto filtered_column = column->filter(filter, -1);
ASSERT_EQ(filtered_column->size(), 2);
ASSERT_EQ((*filtered_column)[0].get<UInt64>(), 0);
ASSERT_EQ((*filtered_column)[1].get<UInt64>(), 2);
ASSERT_EQ((*filtered_column)[0].safeGet<UInt64>(), 0);
ASSERT_EQ((*filtered_column)[1].safeGet<UInt64>(), 2);
}
TEST(ColumnVariant, FilterGeneral)
@ -562,7 +562,7 @@ TEST(ColumnVariant, FilterGeneral)
filter.push_back(0);
auto filtered_column = column->filter(filter, -1);
ASSERT_EQ(filtered_column->size(), 3);
ASSERT_EQ((*filtered_column)[0].get<String>(), "Hello");
ASSERT_EQ((*filtered_column)[0].safeGet<String>(), "Hello");
ASSERT_TRUE((*filtered_column)[1].isNull());
ASSERT_TRUE((*filtered_column)[2].isNull());
}
@ -577,9 +577,9 @@ TEST(ColumnVariant, PermuteAndIndexOneColumnNoNulls)
permutation.push_back(0);
auto permuted_column = column->permute(permutation, 3);
ASSERT_EQ(permuted_column->size(), 3);
ASSERT_EQ((*permuted_column)[0].get<UInt64>(), 1);
ASSERT_EQ((*permuted_column)[1].get<UInt64>(), 3);
ASSERT_EQ((*permuted_column)[2].get<UInt64>(), 2);
ASSERT_EQ((*permuted_column)[0].safeGet<UInt64>(), 1);
ASSERT_EQ((*permuted_column)[1].safeGet<UInt64>(), 3);
ASSERT_EQ((*permuted_column)[2].safeGet<UInt64>(), 2);
auto index = ColumnUInt64::create();
index->getData().push_back(1);
@ -588,9 +588,9 @@ TEST(ColumnVariant, PermuteAndIndexOneColumnNoNulls)
index->getData().push_back(0);
auto indexed_column = column->index(*index, 3);
ASSERT_EQ(indexed_column->size(), 3);
ASSERT_EQ((*indexed_column)[0].get<UInt64>(), 1);
ASSERT_EQ((*indexed_column)[1].get<UInt64>(), 3);
ASSERT_EQ((*indexed_column)[2].get<UInt64>(), 2);
ASSERT_EQ((*indexed_column)[0].safeGet<UInt64>(), 1);
ASSERT_EQ((*indexed_column)[1].safeGet<UInt64>(), 3);
ASSERT_EQ((*indexed_column)[2].safeGet<UInt64>(), 2);
}
TEST(ColumnVariant, PermuteGeneral)
@ -603,9 +603,9 @@ TEST(ColumnVariant, PermuteGeneral)
permutation.push_back(5);
auto permuted_column = column->permute(permutation, 4);
ASSERT_EQ(permuted_column->size(), 4);
ASSERT_EQ((*permuted_column)[0].get<UInt64>(), 43);
ASSERT_EQ((*permuted_column)[1].get<String>(), "World");
ASSERT_EQ((*permuted_column)[2].get<String>(), "Hello");
ASSERT_EQ((*permuted_column)[0].safeGet<UInt64>(), 43);
ASSERT_EQ((*permuted_column)[1].safeGet<String>(), "World");
ASSERT_EQ((*permuted_column)[2].safeGet<String>(), "Hello");
ASSERT_TRUE((*permuted_column)[3].isNull());
}
@ -618,12 +618,12 @@ TEST(ColumnVariant, ReplicateOneColumnNoNull)
offsets.push_back(6);
auto replicated_column = column->replicate(offsets);
ASSERT_EQ(replicated_column->size(), 6);
ASSERT_EQ((*replicated_column)[0].get<UInt64>(), 1);
ASSERT_EQ((*replicated_column)[1].get<UInt64>(), 1);
ASSERT_EQ((*replicated_column)[2].get<UInt64>(), 1);
ASSERT_EQ((*replicated_column)[3].get<UInt64>(), 2);
ASSERT_EQ((*replicated_column)[4].get<UInt64>(), 2);
ASSERT_EQ((*replicated_column)[5].get<UInt64>(), 2);
ASSERT_EQ((*replicated_column)[0].safeGet<UInt64>(), 1);
ASSERT_EQ((*replicated_column)[1].safeGet<UInt64>(), 1);
ASSERT_EQ((*replicated_column)[2].safeGet<UInt64>(), 1);
ASSERT_EQ((*replicated_column)[3].safeGet<UInt64>(), 2);
ASSERT_EQ((*replicated_column)[4].safeGet<UInt64>(), 2);
ASSERT_EQ((*replicated_column)[5].safeGet<UInt64>(), 2);
}
TEST(ColumnVariant, ReplicateGeneral)
@ -637,9 +637,9 @@ TEST(ColumnVariant, ReplicateGeneral)
offsets.push_back(7);
auto replicated_column = column->replicate(offsets);
ASSERT_EQ(replicated_column->size(), 7);
ASSERT_EQ((*replicated_column)[0].get<UInt64>(), 42);
ASSERT_EQ((*replicated_column)[1].get<String>(), "Hello");
ASSERT_EQ((*replicated_column)[2].get<String>(), "Hello");
ASSERT_EQ((*replicated_column)[0].safeGet<UInt64>(), 42);
ASSERT_EQ((*replicated_column)[1].safeGet<String>(), "Hello");
ASSERT_EQ((*replicated_column)[2].safeGet<String>(), "Hello");
ASSERT_TRUE((*replicated_column)[3].isNull());
ASSERT_TRUE((*replicated_column)[4].isNull());
ASSERT_TRUE((*replicated_column)[5].isNull());
@ -657,13 +657,13 @@ TEST(ColumnVariant, ScatterOneColumnNoNulls)
selector.push_back(1);
auto columns = column->scatter(3, selector);
ASSERT_EQ(columns[0]->size(), 2);
ASSERT_EQ((*columns[0])[0].get<UInt64>(), 0);
ASSERT_EQ((*columns[0])[1].get<UInt64>(), 3);
ASSERT_EQ((*columns[0])[0].safeGet<UInt64>(), 0);
ASSERT_EQ((*columns[0])[1].safeGet<UInt64>(), 3);
ASSERT_EQ(columns[1]->size(), 2);
ASSERT_EQ((*columns[1])[0].get<UInt64>(), 1);
ASSERT_EQ((*columns[1])[1].get<UInt64>(), 4);
ASSERT_EQ((*columns[1])[0].safeGet<UInt64>(), 1);
ASSERT_EQ((*columns[1])[1].safeGet<UInt64>(), 4);
ASSERT_EQ(columns[2]->size(), 1);
ASSERT_EQ((*columns[2])[0].get<UInt64>(), 2);
ASSERT_EQ((*columns[2])[0].safeGet<UInt64>(), 2);
}
TEST(ColumnVariant, ScatterGeneral)
@ -680,12 +680,12 @@ TEST(ColumnVariant, ScatterGeneral)
auto columns = column->scatter(3, selector);
ASSERT_EQ(columns[0]->size(), 3);
ASSERT_EQ((*columns[0])[0].get<UInt64>(), 42);
ASSERT_EQ((*columns[0])[1].get<String>(), "Hello");
ASSERT_EQ((*columns[0])[2].get<UInt64>(), 43);
ASSERT_EQ((*columns[0])[0].safeGet<UInt64>(), 42);
ASSERT_EQ((*columns[0])[1].safeGet<String>(), "Hello");
ASSERT_EQ((*columns[0])[2].safeGet<UInt64>(), 43);
ASSERT_EQ(columns[1]->size(), 2);
ASSERT_EQ((*columns[1])[0].get<String>(), "World");
ASSERT_EQ((*columns[1])[1].get<UInt64>(), 44);
ASSERT_EQ((*columns[1])[0].safeGet<String>(), "World");
ASSERT_EQ((*columns[1])[1].safeGet<UInt64>(), 44);
ASSERT_EQ(columns[2]->size(), 2);
ASSERT_TRUE((*columns[2])[0].isNull());
ASSERT_TRUE((*columns[2])[1].isNull());

View File

@ -20,13 +20,13 @@ void testLowCardinalityNumberInsert(const DataTypePtr & data_type)
Field value;
column->get(0, value);
ASSERT_EQ(value.get<T>(), 15);
ASSERT_EQ(value.safeGet<T>(), 15);
column->get(1, value);
ASSERT_EQ(value.get<T>(), 20);
ASSERT_EQ(value.safeGet<T>(), 20);
column->get(2, value);
ASSERT_EQ(value.get<T>(), 25);
ASSERT_EQ(value.safeGet<T>(), 25);
}
TEST(ColumnLowCardinality, Insert)

View File

@ -168,7 +168,7 @@ private:
records.emplace(it, type_idx, item);
}
Records::const_iterator getImpl(std::type_index type_idx) const
typename Records::const_iterator getImpl(std::type_index type_idx) const
{
auto it = std::lower_bound(records.cbegin(), records.cend(), type_idx);

View File

@ -208,7 +208,7 @@ void FieldVisitorEncodeBinary::operator() (const Map & x, WriteBuffer & buf) con
writeVarUInt(size, buf);
for (size_t i = 0; i < size; ++i)
{
const Tuple & key_and_value = x[i].get<Tuple>();
const Tuple & key_and_value = x[i].safeGet<Tuple>();
Field::dispatch([&buf] (const auto & value) { FieldVisitorEncodeBinary()(value, buf); }, key_and_value[0]);
Field::dispatch([&buf] (const auto & value) { FieldVisitorEncodeBinary()(value, buf); }, key_and_value[1]);
}

View File

@ -19,7 +19,7 @@ bool FieldVisitorSum::operator() (UInt64 & x) const
return x != 0;
}
bool FieldVisitorSum::operator() (Float64 & x) const { x += rhs.get<Float64>(); return x != 0; }
bool FieldVisitorSum::operator() (Float64 & x) const { x += rhs.safeGet<Float64>(); return x != 0; }
bool FieldVisitorSum::operator() (Null &) const
{

View File

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

View File

@ -172,7 +172,7 @@ String FieldVisitorToString::operator() (const Object & x) const
String convertFieldToString(const Field & field)
{
if (field.getType() == Field::Types::Which::String)
return field.get<String>();
return field.safeGet<String>();
return applyVisitor(FieldVisitorToString(), field);
}

View File

@ -297,7 +297,7 @@ public:
}
/// Only inserts the value if key isn't already present
void ALWAYS_INLINE insertIfNotPresent(const Key & x, const Cell::Mapped & value)
void ALWAYS_INLINE insertIfNotPresent(const Key & x, const typename Cell::Mapped & value)
{
LookupResult it;
bool inserted;

View File

@ -174,19 +174,19 @@ struct Dictionary
{
switch (attribute.type)
{
case AttributeUnderlyingTypeTest::UInt8: std::get<ContainerPtrType<UInt8>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingTypeTest::UInt16: std::get<ContainerPtrType<UInt16>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingTypeTest::UInt32: std::get<ContainerPtrType<UInt32>>(attribute.arrays)[idx] = static_cast<UInt32>(value.get<UInt64>()); break;
case AttributeUnderlyingTypeTest::UInt64: std::get<ContainerPtrType<UInt64>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingTypeTest::Int8: std::get<ContainerPtrType<Int8>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingTypeTest::Int16: std::get<ContainerPtrType<Int16>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingTypeTest::Int32: std::get<ContainerPtrType<Int32>>(attribute.arrays)[idx] = static_cast<Int32>(value.get<Int64>()); break;
case AttributeUnderlyingTypeTest::Int64: std::get<ContainerPtrType<Int64>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingTypeTest::Float32: std::get<ContainerPtrType<Float32>>(attribute.arrays)[idx] = static_cast<Float32>(value.get<Float64>()); break;
case AttributeUnderlyingTypeTest::Float64: std::get<ContainerPtrType<Float64>>(attribute.arrays)[idx] = value.get<Float64>(); break;
case AttributeUnderlyingTypeTest::UInt8: std::get<ContainerPtrType<UInt8>>(attribute.arrays)[idx] = value.safeGet<UInt64>(); break;
case AttributeUnderlyingTypeTest::UInt16: std::get<ContainerPtrType<UInt16>>(attribute.arrays)[idx] = value.safeGet<UInt64>(); break;
case AttributeUnderlyingTypeTest::UInt32: std::get<ContainerPtrType<UInt32>>(attribute.arrays)[idx] = static_cast<UInt32>(value.safeGet<UInt64>()); break;
case AttributeUnderlyingTypeTest::UInt64: std::get<ContainerPtrType<UInt64>>(attribute.arrays)[idx] = value.safeGet<UInt64>(); break;
case AttributeUnderlyingTypeTest::Int8: std::get<ContainerPtrType<Int8>>(attribute.arrays)[idx] = value.safeGet<Int64>(); break;
case AttributeUnderlyingTypeTest::Int16: std::get<ContainerPtrType<Int16>>(attribute.arrays)[idx] = value.safeGet<Int64>(); break;
case AttributeUnderlyingTypeTest::Int32: std::get<ContainerPtrType<Int32>>(attribute.arrays)[idx] = static_cast<Int32>(value.safeGet<Int64>()); break;
case AttributeUnderlyingTypeTest::Int64: std::get<ContainerPtrType<Int64>>(attribute.arrays)[idx] = value.safeGet<Int64>(); break;
case AttributeUnderlyingTypeTest::Float32: std::get<ContainerPtrType<Float32>>(attribute.arrays)[idx] = static_cast<Float32>(value.safeGet<Float64>()); break;
case AttributeUnderlyingTypeTest::Float64: std::get<ContainerPtrType<Float64>>(attribute.arrays)[idx] = value.safeGet<Float64>(); break;
case AttributeUnderlyingTypeTest::String:
{
const auto & string = value.get<String>();
const auto & string = value.safeGet<String>();
auto & string_ref = std::get<ContainerPtrType<StringRef>>(attribute.arrays)[idx];
const auto & null_value_ref = std::get<String>(attribute.null_values);

View File

@ -49,7 +49,7 @@ ExternalTableDataPtr BaseExternalTable::getData(ContextPtr context)
{
initReadBuffer();
initSampleBlock();
auto input = context->getInputFormat(format, *read_buffer, sample_block, context->getSettingsRef().get("max_block_size").get<UInt64>());
auto input = context->getInputFormat(format, *read_buffer, sample_block, context->getSettingsRef().get("max_block_size").safeGet<UInt64>());
auto data = std::make_unique<ExternalTableData>();
data->pipe = std::make_unique<QueryPipelineBuilder>();

View File

@ -457,15 +457,6 @@ public:
std::string_view getTypeName() const;
bool isNull() const { return which == Types::Null; }
template <typename T>
NearestFieldType<std::decay_t<T>> & get();
template <typename T>
const auto & get() const
{
auto * mutable_this = const_cast<std::decay_t<decltype(*this)> *>(this);
return mutable_this->get<T>();
}
bool isNegativeInfinity() const { return which == Types::Null && get<Null>().isNegativeInfinity(); }
bool isPositiveInfinity() const { return which == Types::Null && get<Null>().isPositiveInfinity(); }
@ -681,6 +672,25 @@ private:
Types::Which which;
/// This function is prone to type punning and should never be used outside of Field class,
/// whenever it is used within this class the stored type should be checked in advance.
template <typename T>
NearestFieldType<std::decay_t<T>> & get()
{
// Before storing the value in the Field, we static_cast it to the field
// storage type, so here we return the value of storage type as well.
// Otherwise, it is easy to make a mistake of reinterpret_casting the stored
// value to a different and incompatible type.
// For example, a Float32 value is stored as Float64, and it is incorrect to
// return a reference to this value as Float32.
return *reinterpret_cast<NearestFieldType<std::decay_t<T>>*>(&storage);
}
template <typename T>
NearestFieldType<std::decay_t<T>> & get() const
{
return const_cast<Field *>(this)->get<T>();
}
/// Assuming there was no allocated state or it was deallocated (see destroy).
template <typename T>
@ -859,55 +869,18 @@ constexpr bool isInt64OrUInt64FieldType(Field::Types::Which t)
|| t == Field::Types::UInt64;
}
constexpr bool isInt64OrUInt64orBoolFieldType(Field::Types::Which t)
{
return t == Field::Types::Int64
|| t == Field::Types::UInt64
|| t == Field::Types::Bool;
}
// Field value getter with type checking in debug builds.
template <typename T>
NearestFieldType<std::decay_t<T>> & Field::get()
{
// Before storing the value in the Field, we static_cast it to the field
// storage type, so here we return the value of storage type as well.
// Otherwise, it is easy to make a mistake of reinterpret_casting the stored
// value to a different and incompatible type.
// For example, a Float32 value is stored as Float64, and it is incorrect to
// return a reference to this value as Float32.
using StoredType = NearestFieldType<std::decay_t<T>>;
#ifndef NDEBUG
// Disregard signedness when converting between int64 types.
constexpr Field::Types::Which target = TypeToEnum<StoredType>::value;
if (target != which
&& (!isInt64OrUInt64orBoolFieldType(target) || !isInt64OrUInt64orBoolFieldType(which)) && target != Field::Types::IPv4)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Invalid Field get from type {} to type {}", which, target);
#endif
StoredType * MAY_ALIAS ptr = reinterpret_cast<StoredType *>(&storage);
return *ptr;
}
template <typename T>
auto & Field::safeGet()
{
const Types::Which target = TypeToEnum<NearestFieldType<std::decay_t<T>>>::value;
/// We allow converting int64 <-> uint64, int64 <-> bool, uint64 <-> bool in safeGet().
if (target != which
&& (!isInt64OrUInt64orBoolFieldType(target) || !isInt64OrUInt64orBoolFieldType(which)))
throw Exception(ErrorCodes::BAD_GET,
"Bad get: has {}, requested {}", getTypeName(), target);
/// bool is stored as uint64, will be returned as UInt64 when requested as bool or UInt64, as Int64 when requested as Int64
if (target != which && !(which == Field::Types::Bool && (target == Field::Types::UInt64 || target == Field::Types::Int64)))
throw Exception(ErrorCodes::BAD_GET, "Bad get: has {}, requested {}", getTypeName(), target);
return get<T>();
}
template <typename T>
requires not_field_or_bool_or_stringlike<T>
Field::Field(T && rhs)

View File

@ -62,27 +62,27 @@ void Range::shrinkToIncludedIfPossible()
{
if (left.isExplicit() && !left_included)
{
if (left.getType() == Field::Types::UInt64 && left.get<UInt64>() != std::numeric_limits<UInt64>::max())
if (left.getType() == Field::Types::UInt64 && left.safeGet<UInt64>() != std::numeric_limits<UInt64>::max())
{
++left.get<UInt64 &>();
++left.safeGet<UInt64 &>();
left_included = true;
}
if (left.getType() == Field::Types::Int64 && left.get<Int64>() != std::numeric_limits<Int64>::max())
if (left.getType() == Field::Types::Int64 && left.safeGet<Int64>() != std::numeric_limits<Int64>::max())
{
++left.get<Int64 &>();
++left.safeGet<Int64 &>();
left_included = true;
}
}
if (right.isExplicit() && !right_included)
{
if (right.getType() == Field::Types::UInt64 && right.get<UInt64>() != std::numeric_limits<UInt64>::min())
if (right.getType() == Field::Types::UInt64 && right.safeGet<UInt64>() != std::numeric_limits<UInt64>::min())
{
--right.get<UInt64 &>();
--right.safeGet<UInt64 &>();
right_included = true;
}
if (right.getType() == Field::Types::Int64 && right.get<Int64>() != std::numeric_limits<Int64>::min())
if (right.getType() == Field::Types::Int64 && right.safeGet<Int64>() != std::numeric_limits<Int64>::min())
{
--right.get<Int64 &>();
--right.safeGet<Int64 &>();
right_included = true;
}
}

View File

@ -118,7 +118,7 @@ void Settings::set(std::string_view name, const Field & value)
{
if (value.getType() != Field::Types::Which::String)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected type of value for setting 'compatibility'. Expected String, got {}", value.getTypeName());
applyCompatibilitySetting(value.get<String>());
applyCompatibilitySetting(value.safeGet<String>());
}
/// If we change setting that was changed by compatibility setting before
/// we should remove it from settings_changed_by_compatibility_setting,

View File

@ -53,29 +53,29 @@ namespace
{
if (f.getType() == Field::Types::String)
{
return stringToNumber<T>(f.get<const String &>());
return stringToNumber<T>(f.safeGet<const String &>());
}
else if (f.getType() == Field::Types::UInt64)
{
T result;
if (!accurate::convertNumeric(f.get<UInt64>(), result))
if (!accurate::convertNumeric(f.safeGet<UInt64>(), result))
throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Field value {} is out of range of {} type", f, demangle(typeid(T).name()));
return result;
}
else if (f.getType() == Field::Types::Int64)
{
T result;
if (!accurate::convertNumeric(f.get<Int64>(), result))
if (!accurate::convertNumeric(f.safeGet<Int64>(), result))
throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Field value {} is out of range of {} type", f, demangle(typeid(T).name()));
return result;
}
else if (f.getType() == Field::Types::Bool)
{
return T(f.get<bool>());
return T(f.safeGet<bool>());
}
else if (f.getType() == Field::Types::Float64)
{
Float64 x = f.get<Float64>();
Float64 x = f.safeGet<Float64>();
if constexpr (std::is_floating_point_v<T>)
{
return T(x);
@ -120,7 +120,7 @@ namespace
if (f.getType() == Field::Types::String)
{
/// Allow to parse Map from string field. For the convenience.
const auto & str = f.get<const String &>();
const auto & str = f.safeGet<const String &>();
return stringToMap(str);
}
@ -218,7 +218,7 @@ namespace
UInt64 fieldToMaxThreads(const Field & f)
{
if (f.getType() == Field::Types::String)
return stringToMaxThreads(f.get<const String &>());
return stringToMaxThreads(f.safeGet<const String &>());
else
return fieldToNumber<UInt64>(f);
}

View File

@ -100,7 +100,7 @@ void doSettingsSanityCheckClamp(Settings & current_settings, LoggerPtr log)
return current_value;
};
UInt64 max_threads = get_current_value("max_threads").get<UInt64>();
UInt64 max_threads = get_current_value("max_threads").safeGet<UInt64>();
UInt64 max_threads_max_value = 256 * getNumberOfPhysicalCPUCores();
if (max_threads > max_threads_max_value)
{
@ -120,7 +120,7 @@ void doSettingsSanityCheckClamp(Settings & current_settings, LoggerPtr log)
"input_format_parquet_max_block_size"};
for (auto const & setting : block_rows_settings)
{
if (auto block_size = get_current_value(setting).get<UInt64>();
if (auto block_size = get_current_value(setting).safeGet<UInt64>();
block_size > max_sane_block_rows_size)
{
if (log)
@ -129,7 +129,7 @@ void doSettingsSanityCheckClamp(Settings & current_settings, LoggerPtr log)
}
}
if (auto max_block_size = get_current_value("max_block_size").get<UInt64>(); max_block_size == 0)
if (auto max_block_size = get_current_value("max_block_size").safeGet<UInt64>(); max_block_size == 0)
{
if (log)
LOG_WARNING(log, "Sanity check: 'max_block_size' cannot be 0. Set to default value {}", DEFAULT_BLOCK_SIZE);

View File

@ -37,7 +37,7 @@ int main(int argc, char ** argv)
std::cerr << applyVisitor(to_string, field) << std::endl;
}
field.get<Array &>().push_back(field);
field.safeGet<Array &>().push_back(field);
std::cerr << applyVisitor(to_string, field) << std::endl;
std::cerr << (field < field2) << std::endl;

View File

@ -8,31 +8,31 @@ GTEST_TEST(Field, FromBool)
{
Field f{false};
ASSERT_EQ(f.getType(), Field::Types::Bool);
ASSERT_EQ(f.get<UInt64>(), 0);
ASSERT_EQ(f.get<bool>(), false);
ASSERT_EQ(f.safeGet<UInt64>(), 0);
ASSERT_EQ(f.safeGet<bool>(), false);
}
{
Field f{true};
ASSERT_EQ(f.getType(), Field::Types::Bool);
ASSERT_EQ(f.get<UInt64>(), 1);
ASSERT_EQ(f.get<bool>(), true);
ASSERT_EQ(f.safeGet<UInt64>(), 1);
ASSERT_EQ(f.safeGet<bool>(), true);
}
{
Field f;
f = false;
ASSERT_EQ(f.getType(), Field::Types::Bool);
ASSERT_EQ(f.get<UInt64>(), 0);
ASSERT_EQ(f.get<bool>(), false);
ASSERT_EQ(f.safeGet<UInt64>(), 0);
ASSERT_EQ(f.safeGet<bool>(), false);
}
{
Field f;
f = true;
ASSERT_EQ(f.getType(), Field::Types::Bool);
ASSERT_EQ(f.get<UInt64>(), 1);
ASSERT_EQ(f.get<bool>(), true);
ASSERT_EQ(f.safeGet<UInt64>(), 1);
ASSERT_EQ(f.safeGet<bool>(), true);
}
}
@ -42,15 +42,15 @@ GTEST_TEST(Field, Move)
Field f;
f = Field{String{"Hello, world (1)"}};
ASSERT_EQ(f.get<String>(), "Hello, world (1)");
ASSERT_EQ(f.safeGet<String>(), "Hello, world (1)");
f = Field{String{"Hello, world (2)"}};
ASSERT_EQ(f.get<String>(), "Hello, world (2)");
ASSERT_EQ(f.safeGet<String>(), "Hello, world (2)");
f = Field{Array{Field{String{"Hello, world (3)"}}}};
ASSERT_EQ(f.get<Array>()[0].get<String>(), "Hello, world (3)");
ASSERT_EQ(f.safeGet<Array>()[0].safeGet<String>(), "Hello, world (3)");
f = String{"Hello, world (4)"};
ASSERT_EQ(f.get<String>(), "Hello, world (4)");
ASSERT_EQ(f.safeGet<String>(), "Hello, world (4)");
f = Array{Field{String{"Hello, world (5)"}}};
ASSERT_EQ(f.get<Array>()[0].get<String>(), "Hello, world (5)");
ASSERT_EQ(f.safeGet<Array>()[0].safeGet<String>(), "Hello, world (5)");
f = Array{String{"Hello, world (6)"}};
ASSERT_EQ(f.get<Array>()[0].get<String>(), "Hello, world (6)");
ASSERT_EQ(f.safeGet<Array>()[0].safeGet<String>(), "Hello, world (6)");
}

View File

@ -119,7 +119,7 @@ MutableColumnPtr DataTypeAggregateFunction::createColumn() const
Field DataTypeAggregateFunction::getDefault() const
{
Field field = AggregateFunctionStateData();
field.get<AggregateFunctionStateData &>().name = getName();
field.safeGet<AggregateFunctionStateData &>().name = getName();
AlignedBuffer place_buffer(function->sizeOfData(), function->alignOfData());
AggregateDataPtr place = place_buffer.data();
@ -128,7 +128,7 @@ Field DataTypeAggregateFunction::getDefault() const
try
{
WriteBufferFromString buffer_from_field(field.get<AggregateFunctionStateData &>().data);
WriteBufferFromString buffer_from_field(field.safeGet<AggregateFunctionStateData &>().data);
function->serialize(place, buffer_from_field, version);
}
catch (...)

View File

@ -71,10 +71,10 @@ static DataTypePtr create(const ASTPtr & arguments)
auto * literal = argument->arguments->children[1]->as<ASTLiteral>();
if (!literal || literal->value.getType() != Field::Types::UInt64 || literal->value.get<UInt64>() == 0 || literal->value.get<UInt64>() > ColumnVariant::MAX_NESTED_COLUMNS)
if (!literal || literal->value.getType() != Field::Types::UInt64 || literal->value.safeGet<UInt64>() == 0 || literal->value.safeGet<UInt64>() > ColumnVariant::MAX_NESTED_COLUMNS)
throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "'max_types' argument for Dynamic type should be a positive integer between 1 and 255");
return std::make_shared<DataTypeDynamic>(literal->value.get<UInt64>());
return std::make_shared<DataTypeDynamic>(literal->value.safeGet<UInt64>());
}
void registerDataTypeDynamic(DataTypeFactory & factory)

View File

@ -122,12 +122,12 @@ Field DataTypeEnum<Type>::castToName(const Field & value_or_name) const
{
if (value_or_name.getType() == Field::Types::String)
{
this->getValue(value_or_name.get<String>()); /// Check correctness
return value_or_name.get<String>();
this->getValue(value_or_name.safeGet<String>()); /// Check correctness
return value_or_name.safeGet<String>();
}
else if (value_or_name.getType() == Field::Types::Int64)
{
Int64 value = value_or_name.get<Int64>();
Int64 value = value_or_name.safeGet<Int64>();
checkOverflow<Type>(value);
return this->getNameForValue(static_cast<Type>(value)).toString();
}
@ -141,12 +141,12 @@ Field DataTypeEnum<Type>::castToValue(const Field & value_or_name) const
{
if (value_or_name.getType() == Field::Types::String)
{
return this->getValue(value_or_name.get<String>());
return this->getValue(value_or_name.safeGet<String>());
}
else if (value_or_name.getType() == Field::Types::Int64
|| value_or_name.getType() == Field::Types::UInt64)
{
Int64 value = value_or_name.get<Int64>();
Int64 value = value_or_name.safeGet<Int64>();
checkOverflow<Type>(value);
this->getNameForValue(static_cast<Type>(value)); /// Check correctness
return value;
@ -220,7 +220,7 @@ static void autoAssignNumberForEnum(const ASTPtr & arguments)
"Elements of Enum data type must be of form: "
"'name' = number or 'name', where name is string literal and number is an integer");
literal_child_assign_num = value_literal->value.get<Int64>();
literal_child_assign_num = value_literal->value.safeGet<Int64>();
}
assign_number_child.emplace_back(child);
}
@ -269,8 +269,8 @@ static DataTypePtr createExact(const ASTPtr & arguments)
"Elements of Enum data type must be of form: "
"'name' = number or 'name', where name is string literal and number is an integer");
const String & field_name = name_literal->value.get<String>();
const auto value = value_literal->value.get<FieldType>();
const String & field_name = name_literal->value.safeGet<String>();
const auto value = value_literal->value.safeGet<FieldType>();
if (value > std::numeric_limits<FieldType>::max() || value < std::numeric_limits<FieldType>::min())
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Value {} for element '{}' exceeds range of {}",
@ -302,7 +302,7 @@ static DataTypePtr create(const ASTPtr & arguments)
"Elements of Enum data type must be of form: "
"'name' = number or 'name', where name is string literal and number is an integer");
Int64 value = value_literal->value.get<Int64>();
Int64 value = value_literal->value.safeGet<Int64>();
if (value > std::numeric_limits<Int8>::max() || value < std::numeric_limits<Int8>::min())
return createExact<DataTypeEnum16>(arguments);

View File

@ -51,11 +51,11 @@ static DataTypePtr create(const ASTPtr & arguments)
"FixedString data type family must have exactly one argument - size in bytes");
const auto * argument = arguments->children[0]->as<ASTLiteral>();
if (!argument || argument->value.getType() != Field::Types::UInt64 || argument->value.get<UInt64>() == 0)
if (!argument || argument->value.getType() != Field::Types::UInt64 || argument->value.safeGet<UInt64>() == 0)
throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE,
"FixedString data type family must have a number (positive integer) as its argument");
return std::make_shared<DataTypeFixedString>(argument->value.get<UInt64>());
return std::make_shared<DataTypeFixedString>(argument->value.safeGet<UInt64>());
}

View File

@ -69,7 +69,7 @@ static DataTypePtr create(const ASTPtr & arguments)
throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE,
"Object data type family must have a const string as its schema name parameter");
return std::make_shared<DataTypeObject>(literal->value.get<const String &>(), is_nullable);
return std::make_shared<DataTypeObject>(literal->value.safeGet<const String &>(), is_nullable);
}
void registerDataTypeObject(DataTypeFactory & factory)

View File

@ -80,14 +80,14 @@ static DataTypePtr create(const ASTPtr & arguments)
const auto * precision_arg = arguments->children[0]->as<ASTLiteral>();
if (!precision_arg || precision_arg->value.getType() != Field::Types::UInt64)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Decimal argument precision is invalid");
precision = precision_arg->value.get<UInt64>();
precision = precision_arg->value.safeGet<UInt64>();
if (arguments->children.size() == 2)
{
const auto * scale_arg = arguments->children[1]->as<ASTLiteral>();
if (!scale_arg || !isInt64OrUInt64FieldType(scale_arg->value.getType()))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Decimal argument scale is invalid");
scale = scale_arg->value.get<UInt64>();
scale = scale_arg->value.safeGet<UInt64>();
}
}
@ -107,7 +107,7 @@ static DataTypePtr createExact(const ASTPtr & arguments)
"Decimal32 | Decimal64 | Decimal128 | Decimal256 data type family must have a one number as its argument");
UInt64 precision = DecimalUtils::max_precision<T>;
UInt64 scale = scale_arg->value.get<UInt64>();
UInt64 scale = scale_arg->value.safeGet<UInt64>();
return createDecimal<DataTypeDecimal>(precision, scale);
}

View File

@ -104,7 +104,7 @@ Array createEmptyArrayField(size_t num_dimensions)
for (size_t i = 1; i < num_dimensions; ++i)
{
current_array->push_back(Array());
current_array = &current_array->back().get<Array &>();
current_array = &current_array->back().safeGet<Array &>();
}
return array;

View File

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

View File

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

View File

@ -29,7 +29,7 @@ static constexpr size_t MAX_ARRAYS_SIZE = 1ULL << 40;
void SerializationArray::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const
{
const Array & a = field.get<const Array &>();
const Array & a = field.safeGet<const Array &>();
writeVarUInt(a.size(), ostr);
for (const auto & i : a)
{
@ -51,7 +51,7 @@ void SerializationArray::deserializeBinary(Field & field, ReadBuffer & istr, con
settings.binary.max_binary_string_size);
field = Array();
Array & arr = field.get<Array &>();
Array & arr = field.safeGet<Array &>();
arr.reserve(size);
for (size_t i = 0; i < size; ++i)
nested->deserializeBinary(arr.emplace_back(), istr, settings);

View File

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

View File

@ -28,7 +28,7 @@ static constexpr size_t MAX_STRINGS_SIZE = 1ULL << 30;
void SerializationFixedString::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const
{
const String & s = field.get<const String &>();
const String & s = field.safeGet<const String &>();
ostr.write(s.data(), std::min(s.size(), n));
if (s.size() < n)
for (size_t i = s.size(); i < n; ++i)
@ -39,7 +39,7 @@ void SerializationFixedString::serializeBinary(const Field & field, WriteBuffer
void SerializationFixedString::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const
{
field = String();
String & s = field.get<String &>();
String & s = field.safeGet<String &>();
s.resize(n);
istr.readStrict(s.data(), n);
}

View File

@ -125,7 +125,7 @@ bool SerializationIP<IPv>::tryDeserializeTextCSV(DB::IColumn & column, DB::ReadB
template <typename IPv>
void SerializationIP<IPv>::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const
{
IPv x = field.get<IPv>();
IPv x = field.safeGet<IPv>();
if constexpr (std::is_same_v<IPv, IPv6>)
writeBinary(x, ostr);
else

View File

@ -40,7 +40,7 @@ static IColumn & extractNestedColumn(IColumn & column)
void SerializationMap::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const
{
const auto & map = field.get<const Map &>();
const auto & map = field.safeGet<const Map &>();
writeVarUInt(map.size(), ostr);
for (const auto & elem : map)
{
@ -63,7 +63,7 @@ void SerializationMap::deserializeBinary(Field & field, ReadBuffer & istr, const
size,
settings.binary.max_binary_string_size);
field = Map();
Map & map = field.get<Map &>();
Map & map = field.safeGet<Map &>();
map.reserve(size);
for (size_t i = 0; i < size; ++i)
{

View File

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

View File

@ -32,7 +32,7 @@ namespace ErrorCodes
void SerializationString::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const
{
const String & s = field.get<const String &>();
const String & s = field.safeGet<const String &>();
if (settings.binary.max_binary_string_size && s.size() > settings.binary.max_binary_string_size)
throw Exception(
ErrorCodes::TOO_LARGE_STRING_SIZE,
@ -59,7 +59,7 @@ void SerializationString::deserializeBinary(Field & field, ReadBuffer & istr, co
settings.binary.max_binary_string_size);
field = String();
String & s = field.get<String &>();
String & s = field.safeGet<String &>();
s.resize(size);
istr.readStrict(s.data(), size);
}

View File

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

View File

@ -137,7 +137,7 @@ bool SerializationUUID::tryDeserializeTextCSV(IColumn & column, ReadBuffer & ist
void SerializationUUID::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const
{
UUID x = field.get<UUID>();
UUID x = field.safeGet<UUID>();
writeBinaryLittleEndian(x, ostr);
}

View File

@ -55,7 +55,7 @@ getArgument(const ASTPtr & arguments, size_t argument_index, const char * argume
}
}
return argument->value.get<NearestResultType>();
return argument->value.safeGet<NearestResultType>();
}
static DataTypePtr create(const ASTPtr & arguments)

View File

@ -183,7 +183,7 @@ void DDLLoadingDependencyVisitor::extractTableNameFromArgument(const ASTFunction
if (name->value.getType() != Field::Types::String)
return;
auto maybe_qualified_name = QualifiedTableName::tryParseFromString(name->value.get<String>());
auto maybe_qualified_name = QualifiedTableName::tryParseFromString(name->value.safeGet<String>());
if (!maybe_qualified_name)
return;
@ -194,7 +194,7 @@ void DDLLoadingDependencyVisitor::extractTableNameFromArgument(const ASTFunction
if (literal->value.getType() != Field::Types::String)
return;
auto maybe_qualified_name = QualifiedTableName::tryParseFromString(literal->value.get<String>());
auto maybe_qualified_name = QualifiedTableName::tryParseFromString(literal->value.safeGet<String>());
/// Just return if name if invalid
if (!maybe_qualified_name)
return;

View File

@ -180,7 +180,7 @@ namespace
if (database_name_field && table_name_field)
{
QualifiedTableName qualified_name{database_name_field->get<String>(), table_name_field->get<String>()};
QualifiedTableName qualified_name{database_name_field->safeGet<String>(), table_name_field->safeGet<String>()};
if (!qualified_name.database.empty() && !qualified_name.table.empty())
{
auto new_qualified_name = data.renaming_map.getNewTableName(qualified_name);
@ -207,7 +207,7 @@ namespace
if (literal->value.getType() != Field::Types::String)
return;
auto maybe_qualified_name = QualifiedTableName::tryParseFromString(literal->value.get<String>());
auto maybe_qualified_name = QualifiedTableName::tryParseFromString(literal->value.safeGet<String>());
/// Just return if name if invalid
if (!maybe_qualified_name || maybe_qualified_name->database.empty() || maybe_qualified_name->table.empty())
return;
@ -247,7 +247,7 @@ namespace
if (!literal || (literal->value.getType() != Field::Types::String))
return;
auto database_name = literal->value.get<String>();
auto database_name = literal->value.safeGet<String>();
if (database_name.empty())
return;

View File

@ -802,8 +802,8 @@ void DatabaseReplicated::checkTableEngine(const ASTCreateQuery & query, ASTStora
if (!arg1 || !arg2 || arg1->value.getType() != Field::Types::String || arg2->value.getType() != Field::Types::String)
return;
String maybe_path = arg1->value.get<String>();
String maybe_replica = arg2->value.get<String>();
String maybe_path = arg1->value.safeGet<String>();
String maybe_replica = arg2->value.safeGet<String>();
/// Looks like it's ReplicatedMergeTree with explicit zookeeper_path and replica_name arguments.
/// Let's ensure that some macros are used.

View File

@ -736,11 +736,11 @@ static void writeFieldsToColumn(
{
for (size_t index = 0; index < rows_data.size(); ++index)
{
const Tuple & row_data = rows_data[index].get<const Tuple &>();
const Tuple & row_data = rows_data[index].safeGet<const Tuple &>();
const Field & value = row_data[column_index];
if (write_data_to_null_map(value, index))
casted_column->insertValue(static_cast<decltype(to_type)>(value.template get<decltype(from_type)>()));
casted_column->insertValue(static_cast<decltype(to_type)>(value.template safeGet<decltype(from_type)>()));
}
};
@ -776,17 +776,17 @@ static void writeFieldsToColumn(
{
for (size_t index = 0; index < rows_data.size(); ++index)
{
const Tuple & row_data = rows_data[index].get<const Tuple &>();
const Tuple & row_data = rows_data[index].safeGet<const Tuple &>();
const Field & value = row_data[column_index];
if (write_data_to_null_map(value, index))
{
if (value.getType() == Field::Types::UInt64)
casted_int32_column->insertValue(static_cast<Int32>(value.get<Int32>()));
casted_int32_column->insertValue(static_cast<Int32>(value.safeGet<Int32>()));
else if (value.getType() == Field::Types::Int64)
{
/// For MYSQL_TYPE_INT24
const Int32 & num = static_cast<Int32>(value.get<Int32>());
const Int32 & num = static_cast<Int32>(value.safeGet<Int32>());
casted_int32_column->insertValue(num & 0x800000 ? num | 0xFF000000 : num);
}
else
@ -798,7 +798,7 @@ static void writeFieldsToColumn(
{
for (size_t index = 0; index < rows_data.size(); ++index)
{
const Tuple & row_data = rows_data[index].get<const Tuple &>();
const Tuple & row_data = rows_data[index].safeGet<const Tuple &>();
const Field & value = row_data[column_index];
if (write_data_to_null_map(value, index))
@ -812,12 +812,12 @@ static void writeFieldsToColumn(
{
for (size_t index = 0; index < rows_data.size(); ++index)
{
const Tuple & row_data = rows_data[index].get<const Tuple &>();
const Tuple & row_data = rows_data[index].safeGet<const Tuple &>();
const Field & value = row_data[column_index];
if (write_data_to_null_map(value, index))
{
const String & data = value.get<const String &>();
const String & data = value.safeGet<const String &>();
casted_fixed_string_column->insertData(data.data(), data.size());
}
}
@ -864,7 +864,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(
rows_data[index].get<const Tuple &>(), rows_data[index + 1].get<const Tuple &>(), sorting_columns_index);
rows_data[index].safeGet<const Tuple &>(), rows_data[index + 1].safeGet<const Tuple &>(), sorting_columns_index);
}
for (size_t column = 0; column < buffer.columns() - 2; ++column)

View File

@ -281,12 +281,12 @@ static void testFile1(IBinlog & binlog, UInt64 timeout, bool filtered = false)
ASSERT_EQ(write_event->table, "a");
ASSERT_EQ(write_event->rows.size(), 1);
ASSERT_EQ(write_event->rows[0].getType(), Field::Types::Tuple);
auto row_data = write_event->rows[0].get<const Tuple &>();
auto row_data = write_event->rows[0].safeGet<const Tuple &>();
ASSERT_EQ(row_data.size(), 4u);
ASSERT_EQ(row_data[0].get<UInt64>(), 1u);
ASSERT_EQ(row_data[1].get<UInt64>(), 1u);
ASSERT_EQ(row_data[2].get<UInt64>(), 1u);
ASSERT_EQ(row_data[3].get<UInt64>(), 1u);
ASSERT_EQ(row_data[0].safeGet<UInt64>(), 1u);
ASSERT_EQ(row_data[1].safeGet<UInt64>(), 1u);
ASSERT_EQ(row_data[2].safeGet<UInt64>(), 1u);
ASSERT_EQ(row_data[3].safeGet<UInt64>(), 1u);
ASSERT_TRUE(binlog.tryReadEvent(event, timeout));
++count;
@ -342,18 +342,18 @@ static void testFile1(IBinlog & binlog, UInt64 timeout, bool filtered = false)
ASSERT_EQ(update_event->table, "a");
ASSERT_EQ(update_event->rows.size(), 2);
ASSERT_EQ(update_event->rows[0].getType(), Field::Types::Tuple);
row_data = update_event->rows[0].get<const Tuple &>();
row_data = update_event->rows[0].safeGet<const Tuple &>();
ASSERT_EQ(row_data.size(), 4u);
ASSERT_EQ(row_data[0].get<UInt64>(), 1u);
ASSERT_EQ(row_data[1].get<UInt64>(), 1u);
ASSERT_EQ(row_data[2].get<UInt64>(), 1u);
ASSERT_EQ(row_data[3].get<UInt64>(), 1u);
row_data = update_event->rows[1].get<const Tuple &>();
ASSERT_EQ(row_data[0].safeGet<UInt64>(), 1u);
ASSERT_EQ(row_data[1].safeGet<UInt64>(), 1u);
ASSERT_EQ(row_data[2].safeGet<UInt64>(), 1u);
ASSERT_EQ(row_data[3].safeGet<UInt64>(), 1u);
row_data = update_event->rows[1].safeGet<const Tuple &>();
ASSERT_EQ(row_data.size(), 4u);
ASSERT_EQ(row_data[0].get<UInt64>(), 1u);
ASSERT_EQ(row_data[1].get<UInt64>(), 2u);
ASSERT_EQ(row_data[2].get<UInt64>(), 1u);
ASSERT_EQ(row_data[3].get<UInt64>(), 1u);
ASSERT_EQ(row_data[0].safeGet<UInt64>(), 1u);
ASSERT_EQ(row_data[1].safeGet<UInt64>(), 2u);
ASSERT_EQ(row_data[2].safeGet<UInt64>(), 1u);
ASSERT_EQ(row_data[3].safeGet<UInt64>(), 1u);
ASSERT_TRUE(binlog.tryReadEvent(event, timeout));
++count;

View File

@ -395,13 +395,13 @@ private:
}
else if constexpr (std::is_same_v<ElementType, StringRef>)
{
const String & string_value = column_value.get<String>();
const String & string_value = column_value.safeGet<String>();
StringRef inserted_value = copyStringInArena(arena, string_value);
container.back() = inserted_value;
}
else
{
container.back() = static_cast<ElementType>(column_value.get<ElementType>());
container.back() = static_cast<ElementType>(column_value.safeGet<ElementType>());
}
});
}
@ -441,7 +441,7 @@ private:
}
else if constexpr (std::is_same_v<ElementType, StringRef>)
{
const String & string_value = column_value.get<String>();
const String & string_value = column_value.safeGet<String>();
StringRef inserted_value = copyStringInArena(arena, string_value);
if (!cell_was_default)
@ -454,7 +454,7 @@ private:
}
else
{
container[index_to_use] = static_cast<ElementType>(column_value.get<ElementType>());
container[index_to_use] = static_cast<ElementType>(column_value.safeGet<ElementType>());
}
});
}
@ -651,12 +651,12 @@ private:
}
else if constexpr (std::is_same_v<ValueType, StringRef>)
{
auto & value = default_value.get<String>();
auto & value = default_value.safeGet<String>();
value_setter(value);
}
else
{
value_setter(default_value.get<ValueType>());
value_setter(default_value.safeGet<ValueType>());
}
}
else

View File

@ -345,7 +345,7 @@ public:
if (attribute_default_value.isNull())
default_value_is_null = true;
else
default_value = static_cast<DictionaryAttributeType>(attribute_default_value.get<DictionaryAttributeType>());
default_value = static_cast<DictionaryAttributeType>(attribute_default_value.safeGet<DictionaryAttributeType>());
}
else
{
@ -377,7 +377,7 @@ public:
if constexpr (std::is_same_v<DefaultColumnType, ColumnArray>)
{
Field field = (*default_values_column)[row];
return field.get<Array>();
return field.safeGet<Array>();
}
else if constexpr (std::is_same_v<DefaultColumnType, ColumnString>)
return default_values_column->getDataAt(row);

View File

@ -245,7 +245,7 @@ ColumnPtr FlatDictionary::getHierarchy(ColumnPtr key_column, const DataTypePtr &
std::optional<UInt64> null_value;
if (!dictionary_attribute.null_value.isNull())
null_value = dictionary_attribute.null_value.get<UInt64>();
null_value = dictionary_attribute.null_value.safeGet<UInt64>();
const ContainerType<UInt64> & parent_keys = std::get<ContainerType<UInt64>>(hierarchical_attribute.container);
@ -300,7 +300,7 @@ ColumnUInt8::Ptr FlatDictionary::isInHierarchy(
std::optional<UInt64> null_value;
if (!dictionary_attribute.null_value.isNull())
null_value = dictionary_attribute.null_value.get<UInt64>();
null_value = dictionary_attribute.null_value.safeGet<UInt64>();
const ContainerType<UInt64> & parent_keys = std::get<ContainerType<UInt64>>(hierarchical_attribute.container);
@ -701,7 +701,7 @@ void FlatDictionary::setAttributeValue(Attribute & attribute, const UInt64 key,
return;
}
auto & attribute_value = value.get<AttributeType>();
auto & attribute_value = value.safeGet<AttributeType>();
auto & container = std::get<ContainerType<ValueType>>(attribute.container);
loaded_keys[key] = true;

View File

@ -240,7 +240,7 @@ ColumnPtr HashedArrayDictionary<dictionary_key_type, sharded>::getHierarchy(Colu
std::optional<UInt64> null_value;
if (!dictionary_attribute.null_value.isNull())
null_value = dictionary_attribute.null_value.get<UInt64>();
null_value = dictionary_attribute.null_value.safeGet<UInt64>();
auto is_key_valid_func = [&, this](auto & key)
@ -313,7 +313,7 @@ ColumnUInt8::Ptr HashedArrayDictionary<dictionary_key_type, sharded>::isInHierar
std::optional<UInt64> null_value;
if (!dictionary_attribute.null_value.isNull())
null_value = dictionary_attribute.null_value.get<UInt64>();
null_value = dictionary_attribute.null_value.safeGet<UInt64>();
auto is_key_valid_func = [&](auto & key)
@ -581,13 +581,13 @@ void HashedArrayDictionary<dictionary_key_type, sharded>::blockToAttributes(cons
if constexpr (std::is_same_v<AttributeValueType, StringRef>)
{
String & value_to_insert = column_value_to_insert.get<String>();
String & value_to_insert = column_value_to_insert.safeGet<String>();
StringRef string_in_arena_reference = copyStringInArena(*string_arenas[shard], value_to_insert);
attribute_container.back() = string_in_arena_reference;
}
else
{
auto value_to_insert = static_cast<AttributeValueType>(column_value_to_insert.get<AttributeValueType>());
auto value_to_insert = static_cast<AttributeValueType>(column_value_to_insert.safeGet<AttributeValueType>());
attribute_container.back() = value_to_insert;
}
};

View File

@ -636,7 +636,7 @@ ColumnPtr HashedDictionary<dictionary_key_type, sparse, sharded>::getHierarchy(C
std::optional<UInt64> null_value;
if (!dictionary_attribute.null_value.isNull())
null_value = dictionary_attribute.null_value.get<UInt64>();
null_value = dictionary_attribute.null_value.safeGet<UInt64>();
const CollectionsHolder<UInt64> & child_key_to_parent_key_maps = std::get<CollectionsHolder<UInt64>>(hierarchical_attribute.containers);
@ -710,7 +710,7 @@ ColumnUInt8::Ptr HashedDictionary<dictionary_key_type, sparse, sharded>::isInHie
std::optional<UInt64> null_value;
if (!dictionary_attribute.null_value.isNull())
null_value = dictionary_attribute.null_value.get<UInt64>();
null_value = dictionary_attribute.null_value.safeGet<UInt64>();
const CollectionsHolder<UInt64> & child_key_to_parent_key_maps = std::get<CollectionsHolder<UInt64>>(hierarchical_attribute.containers);
@ -1004,13 +1004,13 @@ void HashedDictionary<dictionary_key_type, sparse, sharded>::blockToAttributes(c
if constexpr (std::is_same_v<AttributeValueType, StringRef>)
{
String & value_to_insert = column_value_to_insert.get<String>();
String & value_to_insert = column_value_to_insert.safeGet<String>();
StringRef arena_value = copyStringInArena(*string_arenas[shard], value_to_insert);
container.insert({key, arena_value});
}
else
{
auto value_to_insert = static_cast<AttributeValueType>(column_value_to_insert.get<AttributeValueType>());
auto value_to_insert = static_cast<AttributeValueType>(column_value_to_insert.safeGet<AttributeValueType>());
container.insert({key, value_to_insert});
}

View File

@ -50,7 +50,7 @@ namespace
std::optional<UInt64> null_value;
if (!hierarchical_attribute.null_value.isNull())
null_value = hierarchical_attribute.null_value.get<UInt64>();
null_value = hierarchical_attribute.null_value.safeGet<UInt64>();
ColumnPtr key_to_request_column = ColumnVector<UInt64>::create();
auto * key_to_request_column_typed = static_cast<ColumnVector<UInt64> *>(key_to_request_column->assumeMutable().get());
@ -190,7 +190,7 @@ ColumnPtr getKeysHierarchyDefaultImplementation(
std::optional<UInt64> null_value;
if (!hierarchical_attribute.null_value.isNull())
null_value = hierarchical_attribute.null_value.get<UInt64>();
null_value = hierarchical_attribute.null_value.safeGet<UInt64>();
auto get_parent_key_func = [&](auto & key)
{
@ -252,7 +252,7 @@ ColumnUInt8::Ptr getKeysIsInHierarchyDefaultImplementation(
std::optional<UInt64> null_value;
if (!hierarchical_attribute.null_value.isNull())
null_value = hierarchical_attribute.null_value.get<UInt64>();
null_value = hierarchical_attribute.null_value.safeGet<UInt64>();
auto get_parent_key_func = [&](auto & key)
{

View File

@ -613,14 +613,14 @@ void IPAddressDictionary::calculateBytesAllocated()
template <typename T>
void IPAddressDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value)
{
attribute.null_values = null_value.isNull() ? T{} : T(null_value.get<T>());
attribute.null_values = null_value.isNull() ? T{} : T(null_value.safeGet<T>());
attribute.maps.emplace<ContainerType<T>>();
}
template <>
void IPAddressDictionary::createAttributeImpl<String>(Attribute & attribute, const Field & null_value)
{
attribute.null_values = null_value.isNull() ? String() : null_value.get<String>();
attribute.null_values = null_value.isNull() ? String() : null_value.safeGet<String>();
attribute.maps.emplace<ContainerType<StringRef>>();
attribute.string_arena = std::make_unique<Arena>();
}
@ -976,13 +976,13 @@ void IPAddressDictionary::setAttributeValue(Attribute & attribute, const Field &
if constexpr (std::is_same_v<AttributeType, String>)
{
const auto & string = value.get<String>();
const auto & string = value.safeGet<String>();
const auto * string_in_arena = attribute.string_arena->insert(string.data(), string.size());
setAttributeValueImpl<StringRef>(attribute, StringRef{string_in_arena, string.size()});
}
else
{
setAttributeValueImpl<AttributeType>(attribute, static_cast<AttributeType>(value.get<AttributeType>()));
setAttributeValueImpl<AttributeType>(attribute, static_cast<AttributeType>(value.safeGet<AttributeType>()));
}
};

View File

@ -1,7 +1,6 @@
#include "MongoDBDictionarySource.h"
#include "DictionarySourceFactory.h"
#include "DictionaryStructure.h"
#include "registerDictionaries.h"
#include <Storages/ExternalDataSourceConfiguration.h>
#include <Storages/StorageMongoDBSocketFactory.h>
@ -233,7 +232,7 @@ QueryPipeline MongoDBDictionarySource::loadKeys(const Columns & key_columns, con
}
case AttributeUnderlyingType::String:
{
String loaded_str((*key_columns[attribute_index])[row_idx].get<String>());
String loaded_str((*key_columns[attribute_index])[row_idx].safeGet<String>());
/// Convert string to ObjectID
if (key_attribute.is_object_id)
{

View File

@ -141,7 +141,7 @@ ColumnPtr IPolygonDictionary::getColumn(
{
getItemsShortCircuitImpl<ValueType>(
requested_key_points,
[&](size_t row) { return (*attribute_values_column)[row].get<Array>(); },
[&](size_t row) { return (*attribute_values_column)[row].safeGet<Array>(); },
[&](Array & value) { result_column_typed.insert(value); },
default_mask.value());
}
@ -149,7 +149,7 @@ ColumnPtr IPolygonDictionary::getColumn(
{
getItemsImpl<ValueType>(
requested_key_points,
[&](size_t row) { return (*attribute_values_column)[row].get<Array>(); },
[&](size_t row) { return (*attribute_values_column)[row].safeGet<Array>(); },
[&](Array & value) { result_column_typed.insert(value); },
default_value_provider.value());
}
@ -432,16 +432,16 @@ void IPolygonDictionary::getItemsImpl(
}
else if constexpr (std::is_same_v<AttributeType, Array>)
{
set_value(default_value.get<Array>());
set_value(default_value.safeGet<Array>());
}
else if constexpr (std::is_same_v<AttributeType, StringRef>)
{
auto default_value_string = default_value.get<String>();
auto default_value_string = default_value.safeGet<String>();
set_value(default_value_string);
}
else
{
set_value(default_value.get<NearestFieldType<AttributeType>>());
set_value(default_value.safeGet<NearestFieldType<AttributeType>>());
}
}
}

View File

@ -906,13 +906,13 @@ void RangeHashedDictionary<dictionary_key_type>::setAttributeValue(Attribute & a
if constexpr (std::is_same_v<AttributeType, String>)
{
const auto & string = value.get<String>();
const auto & string = value.safeGet<String>();
StringRef string_ref = copyStringInArena(string_arena, string);
value_to_insert = string_ref;
}
else
{
value_to_insert = static_cast<ValueType>(value.get<ValueType>());
value_to_insert = static_cast<ValueType>(value.safeGet<ValueType>());
}
container.back() = value_to_insert;

View File

@ -1,7 +1,6 @@
#include "RedisDictionarySource.h"
#include "DictionarySourceFactory.h"
#include "DictionaryStructure.h"
#include "registerDictionaries.h"
#include <Poco/Util/AbstractConfiguration.h>
#include <Interpreters/Context.h>
@ -160,7 +159,7 @@ namespace DB
if (isInteger(type))
key << DB::toString(key_columns[i]->get64(row));
else if (isString(type))
key << (*key_columns[i])[row].get<const String &>();
key << (*key_columns[i])[row].safeGet<const String &>();
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected type of key in Redis dictionary");
}

View File

@ -113,7 +113,7 @@ std::string getOrCreateDiskFromDiskAST(const ASTPtr & disk_function, ContextPtr
FlattenDiskConfigurationVisitor::Data data{context, attach};
FlattenDiskConfigurationVisitor{data}.visit(ast);
auto disk_name = assert_cast<const ASTLiteral &>(*ast).value.get<String>();
auto disk_name = assert_cast<const ASTLiteral &>(*ast).value.safeGet<String>();
LOG_TRACE(getLogger("getOrCreateDiskFromDiskAST"), "Result disk name: {}", disk_name);
return disk_name;
}

View File

@ -1196,7 +1196,7 @@ struct ToYearImpl
{
if (point.getType() != Field::Types::UInt64) return std::nullopt;
auto year = point.get<UInt64>();
auto year = point.safeGet<UInt64>();
if (year < DATE_LUT_MIN_YEAR || year >= DATE_LUT_MAX_YEAR) return std::nullopt;
const DateLUTImpl & date_lut = DateLUT::instance("UTC");
@ -2001,7 +2001,7 @@ struct ToYYYYMMImpl
{
if (point.getType() != Field::Types::UInt64) return std::nullopt;
auto year_month = point.get<UInt64>();
auto year_month = point.safeGet<UInt64>();
auto year = year_month / 100;
auto month = year_month % 100;

View File

@ -101,9 +101,9 @@ private:
BucketsType num_buckets;
if (buckets_field.getType() == Field::Types::Int64)
num_buckets = checkBucketsRange(buckets_field.get<Int64>());
num_buckets = checkBucketsRange(buckets_field.safeGet<Int64>());
else if (buckets_field.getType() == Field::Types::UInt64)
num_buckets = checkBucketsRange(buckets_field.get<UInt64>());
num_buckets = checkBucketsRange(buckets_field.safeGet<UInt64>());
else
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of the second argument of function {}",

View File

@ -117,7 +117,7 @@ UInt32 extractToDecimalScale(const ColumnWithTypeAndName & named_column)
Field field;
named_column.column->get(0, field);
return static_cast<UInt32>(field.get<UInt32>());
return static_cast<UInt32>(field.safeGet<UInt32>());
}
@ -2604,8 +2604,8 @@ struct ToNumberMonotonicity
if (left.isNull() || right.isNull())
return {};
Float64 left_float = left.get<Float64>();
Float64 right_float = right.get<Float64>();
Float64 left_float = left.safeGet<Float64>();
Float64 right_float = right.safeGet<Float64>();
if (left_float >= static_cast<Float64>(std::numeric_limits<T>::min())
&& left_float <= static_cast<Float64>(std::numeric_limits<T>::max())
@ -2633,11 +2633,11 @@ struct ToNumberMonotonicity
const bool left_in_first_half = left.isNull()
? from_is_unsigned
: (left.get<Int64>() >= 0);
: (left.safeGet<Int64>() >= 0);
const bool right_in_first_half = right.isNull()
? !from_is_unsigned
: (right.get<Int64>() >= 0);
: (right.safeGet<Int64>() >= 0);
/// Size of type is the same.
if (size_of_from == size_of_to)
@ -2675,7 +2675,7 @@ struct ToNumberMonotonicity
return {};
/// Function cannot be monotonic when left and right are not on the same ranges.
if (divideByRangeOfType(left.get<UInt64>()) != divideByRangeOfType(right.get<UInt64>()))
if (divideByRangeOfType(left.safeGet<UInt64>()) != divideByRangeOfType(right.safeGet<UInt64>()))
return {};
if (to_is_unsigned)
@ -2683,7 +2683,7 @@ struct ToNumberMonotonicity
else
{
// If To is signed, it's possible that the signedness is different after conversion. So we check it explicitly.
const bool is_monotonic = (T(left.get<UInt64>()) >= 0) == (T(right.get<UInt64>()) >= 0);
const bool is_monotonic = (T(left.safeGet<UInt64>()) >= 0) == (T(right.safeGet<UInt64>()) >= 0);
return { .is_monotonic = is_monotonic };
}
@ -2707,13 +2707,13 @@ struct ToDateMonotonicity
}
else if (
((left.getType() == Field::Types::UInt64 || left.isNull()) && (right.getType() == Field::Types::UInt64 || right.isNull())
&& ((left.isNull() || left.get<UInt64>() < 0xFFFF) && (right.isNull() || right.get<UInt64>() >= 0xFFFF)))
&& ((left.isNull() || left.safeGet<UInt64>() < 0xFFFF) && (right.isNull() || right.safeGet<UInt64>() >= 0xFFFF)))
|| ((left.getType() == Field::Types::Int64 || left.isNull()) && (right.getType() == Field::Types::Int64 || right.isNull())
&& ((left.isNull() || left.get<Int64>() < 0xFFFF) && (right.isNull() || right.get<Int64>() >= 0xFFFF)))
&& ((left.isNull() || left.safeGet<Int64>() < 0xFFFF) && (right.isNull() || right.safeGet<Int64>() >= 0xFFFF)))
|| ((
(left.getType() == Field::Types::Float64 || left.isNull())
&& (right.getType() == Field::Types::Float64 || right.isNull())
&& ((left.isNull() || left.get<Float64>() < 0xFFFF) && (right.isNull() || right.get<Float64>() >= 0xFFFF))))
&& ((left.isNull() || left.safeGet<Float64>() < 0xFFFF) && (right.isNull() || right.safeGet<Float64>() >= 0xFFFF))))
|| !isNativeNumber(type))
{
return {};
@ -2768,16 +2768,16 @@ struct ToStringMonotonicity
if (left.getType() == Field::Types::UInt64
&& right.getType() == Field::Types::UInt64)
{
return (left.get<Int64>() == 0 && right.get<Int64>() == 0)
|| (floor(log10(left.get<UInt64>())) == floor(log10(right.get<UInt64>())))
return (left.safeGet<Int64>() == 0 && right.safeGet<Int64>() == 0)
|| (floor(log10(left.safeGet<UInt64>())) == floor(log10(right.safeGet<UInt64>())))
? positive : not_monotonic;
}
if (left.getType() == Field::Types::Int64
&& right.getType() == Field::Types::Int64)
{
return (left.get<Int64>() == 0 && right.get<Int64>() == 0)
|| (left.get<Int64>() > 0 && right.get<Int64>() > 0 && floor(log10(left.get<Int64>())) == floor(log10(right.get<Int64>())))
return (left.safeGet<Int64>() == 0 && right.safeGet<Int64>() == 0)
|| (left.safeGet<Int64>() > 0 && right.safeGet<Int64>() > 0 && floor(log10(left.safeGet<Int64>())) == floor(log10(right.safeGet<Int64>())))
? positive : not_monotonic;
}
@ -4673,7 +4673,7 @@ private:
return [function_name] (
ColumnsWithTypeAndName & arguments, const DataTypePtr & res_type, const ColumnNullable * nullable_col, size_t /*input_rows_count*/)
{
using ColumnEnumType = EnumType::ColumnType;
using ColumnEnumType = typename EnumType::ColumnType;
const auto & first_col = arguments.front().column.get();
const auto & first_type = arguments.front().type.get();

View File

@ -243,7 +243,7 @@ private:
}
case MoveType::Index:
{
Int64 index = (*arguments[j + 1].column)[row].get<Int64>();
Int64 index = (*arguments[j + 1].column)[row].safeGet<Int64>();
if (!moveToElementByIndex<JSONParser>(res_element, static_cast<int>(index), key))
return false;
break;

View File

@ -701,11 +701,11 @@ ColumnPtr FunctionAnyArityLogical<Impl, Name>::getConstantResultForNonConstArgum
bool constant_value_bool = false;
if (field_type == Field::Types::Float64)
constant_value_bool = static_cast<bool>(constant_field_value.get<Float64>());
constant_value_bool = static_cast<bool>(constant_field_value.safeGet<Float64>());
else if (field_type == Field::Types::Int64)
constant_value_bool = static_cast<bool>(constant_field_value.get<Int64>());
constant_value_bool = static_cast<bool>(constant_field_value.safeGet<Int64>());
else if (field_type == Field::Types::UInt64)
constant_value_bool = static_cast<bool>(constant_field_value.get<UInt64>());
constant_value_bool = static_cast<bool>(constant_field_value.safeGet<UInt64>());
has_true_constant = has_true_constant || constant_value_bool;
has_false_constant = has_false_constant || !constant_value_bool;

Some files were not shown because too many files have changed in this diff Show More