From a348d72a903fd1f9f8e78a99b672392577ec82ef Mon Sep 17 00:00:00 2001 From: Chang Chen Date: Wed, 14 Aug 2024 22:11:11 +0800 Subject: [PATCH] Fix Build due to https://github.com/ClickHouse/ClickHouse/pull/68135 --- .../AggregateFunctionGroupBloomFilter.cpp | 4 +- .../AggregateFunctionSparkAvg.cpp | 2 +- .../Functions/SparkFunctionArraySort.cpp | 2 +- .../SparkFunctionCheckDecimalOverflow.h | 14 ++-- .../Functions/SparkFunctionFloor.h | 2 +- .../Functions/SparkFunctionHashingExtended.h | 40 +++++------ .../Functions/SparkFunctionMakeDecimal.cpp | 2 +- .../Functions/SparkFunctionRoundHalfUp.h | 2 +- .../Functions/SparkFunctionToDateTime.h | 2 +- .../local-engine/Operator/ExpandTransform.cpp | 2 +- .../Parser/CHColumnToSparkRow.cpp | 66 +++++++++---------- .../Parser/SerializedPlanParser.cpp | 2 +- cpp-ch/local-engine/Parser/WriteRelParser.cpp | 4 +- .../ApproxPercentileParser.cpp | 2 +- .../BloomFilterAggParser.cpp | 4 +- .../aggregate_function_parser/NtileParser.cpp | 2 +- .../arrayHighOrderFunctions.cpp | 4 +- .../scalar_function_parser/lambdaFunction.cpp | 4 +- .../scalar_function_parser/tupleElement.cpp | 2 +- .../Mergetree/SparkMergeTreeWriter.cpp | 8 +-- .../Storages/Parquet/ParquetConverter.h | 8 +-- .../tests/gtest_clickhouse_pr_verify.cpp | 4 +- 22 files changed, 91 insertions(+), 91 deletions(-) diff --git a/cpp-ch/local-engine/AggregateFunctions/AggregateFunctionGroupBloomFilter.cpp b/cpp-ch/local-engine/AggregateFunctions/AggregateFunctionGroupBloomFilter.cpp index 5555302a5c2fd..1b853cc67c691 100644 --- a/cpp-ch/local-engine/AggregateFunctions/AggregateFunctionGroupBloomFilter.cpp +++ b/cpp-ch/local-engine/AggregateFunctions/AggregateFunctionGroupBloomFilter.cpp @@ -62,10 +62,10 @@ createAggregateFunctionBloomFilter(const std::string & name, const DataTypes & a if (type != Field::Types::Int64 && type != Field::Types::UInt64) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be Int64 or UInt64", name); - if ((type == Field::Types::Int64 && parameters[i].get() < 0)) + if ((type == Field::Types::Int64 && parameters[i].safeGet() < 0)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be non-negative number", name); - return parameters[i].get(); + return parameters[i].safeGet(); }; filter_size = get_parameter(0); diff --git a/cpp-ch/local-engine/AggregateFunctions/AggregateFunctionSparkAvg.cpp b/cpp-ch/local-engine/AggregateFunctions/AggregateFunctionSparkAvg.cpp index 5eb3a0b360575..0aa2331457284 100644 --- a/cpp-ch/local-engine/AggregateFunctions/AggregateFunctionSparkAvg.cpp +++ b/cpp-ch/local-engine/AggregateFunctions/AggregateFunctionSparkAvg.cpp @@ -140,7 +140,7 @@ createAggregateFunctionSparkAvg(const std::string & name, const DataTypes & argu throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for aggregate function {}", data_type->getName(), name); - bool allowPrecisionLoss = settings->get(DECIMAL_OPERATIONS_ALLOW_PREC_LOSS).get(); + bool allowPrecisionLoss = settings->get(DECIMAL_OPERATIONS_ALLOW_PREC_LOSS).safeGet(); const UInt32 p1 = DB::getDecimalPrecision(*data_type); const UInt32 s1 = DB::getDecimalScale(*data_type); auto [p2, s2] = GlutenDecimalUtils::LONG_DECIMAL; diff --git a/cpp-ch/local-engine/Functions/SparkFunctionArraySort.cpp b/cpp-ch/local-engine/Functions/SparkFunctionArraySort.cpp index 1371ec60e1796..cf9d67f1696b5 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionArraySort.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionArraySort.cpp @@ -60,7 +60,7 @@ struct LambdaLess auto compare_res_col = lambda_->reduce(); DB::Field field; compare_res_col.column->get(0, field); - return field.get() < 0; + return field.safeGet() < 0; } private: ALWAYS_INLINE DB::ColumnPtr oneRowColumn(size_t i) const diff --git a/cpp-ch/local-engine/Functions/SparkFunctionCheckDecimalOverflow.h b/cpp-ch/local-engine/Functions/SparkFunctionCheckDecimalOverflow.h index 32bf79a563a70..e501c7fc5ffbd 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionCheckDecimalOverflow.h +++ b/cpp-ch/local-engine/Functions/SparkFunctionCheckDecimalOverflow.h @@ -50,17 +50,17 @@ template Field convertNumericType(const Field & from) { if (from.getType() == Field::Types::UInt64) - return convertNumericTypeImpl(from.get()); + return convertNumericTypeImpl(from.safeGet()); if (from.getType() == Field::Types::Int64) - return convertNumericTypeImpl(from.get()); + return convertNumericTypeImpl(from.safeGet()); if (from.getType() == Field::Types::UInt128) - return convertNumericTypeImpl(from.get()); + return convertNumericTypeImpl(from.safeGet()); if (from.getType() == Field::Types::Int128) - return convertNumericTypeImpl(from.get()); + return convertNumericTypeImpl(from.safeGet()); if (from.getType() == Field::Types::UInt256) - return convertNumericTypeImpl(from.get()); + return convertNumericTypeImpl(from.safeGet()); if (from.getType() == Field::Types::Int256) - return convertNumericTypeImpl(from.get()); + return convertNumericTypeImpl(from.safeGet()); throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch. Expected: Integer. Got: {}", from.getType()); } @@ -81,7 +81,7 @@ inline UInt32 extractArgument(const ColumnWithTypeAndName & named_column) throw Exception( ErrorCodes::DECIMAL_OVERFLOW, "{} convert overflow, precision/scale value must in UInt32", named_column.type->getName()); } - return static_cast(to.get()); + return static_cast(to.safeGet()); } } diff --git a/cpp-ch/local-engine/Functions/SparkFunctionFloor.h b/cpp-ch/local-engine/Functions/SparkFunctionFloor.h index ce33d11dbd8cf..4a3f99a9a3567 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionFloor.h +++ b/cpp-ch/local-engine/Functions/SparkFunctionFloor.h @@ -197,7 +197,7 @@ class SparkFunctionFloor : public DB::FunctionFloor if (scale_field.getType() != Field::Types::UInt64 && scale_field.getType() != Field::Types::Int64) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Scale argument for rounding functions must have integer type"); - Int64 scale64 = scale_field.get(); + Int64 scale64 = scale_field.safeGet(); if (scale64 > std::numeric_limits::max() || scale64 < std::numeric_limits::min()) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale argument for rounding function is too large"); diff --git a/cpp-ch/local-engine/Functions/SparkFunctionHashingExtended.h b/cpp-ch/local-engine/Functions/SparkFunctionHashingExtended.h index 57bf00ba99044..c6499031492ed 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionHashingExtended.h +++ b/cpp-ch/local-engine/Functions/SparkFunctionHashingExtended.h @@ -101,42 +101,42 @@ class SparkFunctionAnyHash : public IFunction if (which.isNothing()) return seed; else if (which.isUInt8()) - return applyNumber(field.get(), seed); + return applyNumber(field.safeGet(), seed); else if (which.isUInt16()) - return applyNumber(field.get(), seed); + return applyNumber(field.safeGet(), seed); else if (which.isUInt32()) - return applyNumber(field.get(), seed); + return applyNumber(field.safeGet(), seed); else if (which.isUInt64()) - return applyNumber(field.get(), seed); + return applyNumber(field.safeGet(), seed); else if (which.isInt8()) - return applyNumber(field.get(), seed); + return applyNumber(field.safeGet(), seed); else if (which.isInt16()) - return applyNumber(field.get(), seed); + return applyNumber(field.safeGet(), seed); else if (which.isInt32()) - return applyNumber(field.get(), seed); + return applyNumber(field.safeGet(), seed); else if (which.isInt64()) - return applyNumber(field.get(), seed); + return applyNumber(field.safeGet(), seed); else if (which.isFloat32()) - return applyNumber(field.get(), seed); + return applyNumber(field.safeGet(), seed); else if (which.isFloat64()) - return applyNumber(field.get(), seed); + return applyNumber(field.safeGet(), seed); else if (which.isDate()) - return applyNumber(field.get(), seed); + return applyNumber(field.safeGet(), seed); else if (which.isDate32()) - return applyNumber(field.get(), seed); + return applyNumber(field.safeGet(), seed); else if (which.isDateTime()) - return applyNumber(field.get(), seed); + return applyNumber(field.safeGet(), seed); else if (which.isDateTime64()) - return applyDecimal(field.get(), seed); + return applyDecimal(field.safeGet(), seed); else if (which.isDecimal32()) - return applyDecimal(field.get(), seed); + return applyDecimal(field.safeGet(), seed); else if (which.isDecimal64()) - return applyDecimal(field.get(), seed); + return applyDecimal(field.safeGet(), seed); else if (which.isDecimal128()) - return applyDecimal(field.get(), seed); + return applyDecimal(field.safeGet(), seed); else if (which.isStringOrFixedString()) { - const String & str = field.get(); + const String & str = field.safeGet(); return applyUnsafeBytes(str.data(), str.size(), seed); } else if (which.isTuple()) @@ -145,7 +145,7 @@ class SparkFunctionAnyHash : public IFunction assert(tuple_type); const auto & elements = tuple_type->getElements(); - const Tuple & tuple = field.get(); + const Tuple & tuple = field.safeGet(); assert(tuple.size() == elements.size()); for (size_t i = 0; i < elements.size(); ++i) @@ -160,7 +160,7 @@ class SparkFunctionAnyHash : public IFunction assert(array_type); const auto & nested_type = array_type->getNestedType(); - const Array & array = field.get(); + const Array & array = field.safeGet(); for (size_t i=0; i < array.size(); ++i) { seed = applyGeneric(array[i], seed, nested_type); diff --git a/cpp-ch/local-engine/Functions/SparkFunctionMakeDecimal.cpp b/cpp-ch/local-engine/Functions/SparkFunctionMakeDecimal.cpp index 231856b0288f5..795e2b0be329a 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionMakeDecimal.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionMakeDecimal.cpp @@ -205,7 +205,7 @@ namespace else return false; } - result = static_cast(convert_to.get()); + result = static_cast(convert_to.safeGet()); ToNativeType pow10 = intExp10OfSize(precision_value); if ((result < 0 && result <= -pow10) || result >= pow10) diff --git a/cpp-ch/local-engine/Functions/SparkFunctionRoundHalfUp.h b/cpp-ch/local-engine/Functions/SparkFunctionRoundHalfUp.h index 441842d4e7e1a..0bd28b116d9aa 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionRoundHalfUp.h +++ b/cpp-ch/local-engine/Functions/SparkFunctionRoundHalfUp.h @@ -271,7 +271,7 @@ class FunctionRoundingHalfUp : public IFunction if (scale_field.getType() != Field::Types::UInt64 && scale_field.getType() != Field::Types::Int64) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Scale argument for rounding functions must have integer type"); - Int64 scale64 = scale_field.get(); + Int64 scale64 = scale_field.safeGet(); if (scale64 > std::numeric_limits::max() || scale64 < std::numeric_limits::min()) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale argument for rounding function is too large"); diff --git a/cpp-ch/local-engine/Functions/SparkFunctionToDateTime.h b/cpp-ch/local-engine/Functions/SparkFunctionToDateTime.h index 980af85bd9838..aab8aabc3a8d3 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionToDateTime.h +++ b/cpp-ch/local-engine/Functions/SparkFunctionToDateTime.h @@ -128,7 +128,7 @@ class SparkFunctionConvertToDateTime : public IFunction Field field; named_column.column->get(0, field); - return static_cast(field.get()); + return static_cast(field.safeGet()); } DB::DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override diff --git a/cpp-ch/local-engine/Operator/ExpandTransform.cpp b/cpp-ch/local-engine/Operator/ExpandTransform.cpp index f5787163c5a1e..a3bb545602577 100644 --- a/cpp-ch/local-engine/Operator/ExpandTransform.cpp +++ b/cpp-ch/local-engine/Operator/ExpandTransform.cpp @@ -104,7 +104,7 @@ void ExpandTransform::work() if (kind == EXPAND_FIELD_KIND_SELECTION) { - const auto & original_col = original_cols.at(field.get()); + const auto & original_col = original_cols.at(field.safeGet()); if (type->isNullable() == original_col->isNullable()) { cols.push_back(original_col); diff --git a/cpp-ch/local-engine/Parser/CHColumnToSparkRow.cpp b/cpp-ch/local-engine/Parser/CHColumnToSparkRow.cpp index 3d5a7731bffb9..80902b22324b9 100644 --- a/cpp-ch/local-engine/Parser/CHColumnToSparkRow.cpp +++ b/cpp-ch/local-engine/Parser/CHColumnToSparkRow.cpp @@ -501,7 +501,7 @@ int64_t BackingDataLengthCalculator::calculate(const Field & field) const if (which.isStringOrFixedString()) { - const auto & str = field.get(); + const auto & str = field.safeGet(); return roundNumberOfBytesToNearestWord(str.size()); } @@ -511,7 +511,7 @@ int64_t BackingDataLengthCalculator::calculate(const Field & field) const if (which.isArray()) { /// 内存布局:numElements(8B) | null_bitmap(与numElements成正比) | values(每个值长度与类型有关) | backing buffer - const auto & array = field.get(); /// Array can not be wrapped with Nullable + const auto & array = field.safeGet(); /// Array can not be wrapped with Nullable const auto num_elems = array.size(); int64_t res = 8 + calculateBitSetWidthInBytes(num_elems); @@ -531,7 +531,7 @@ int64_t BackingDataLengthCalculator::calculate(const Field & field) const int64_t res = 8; /// Construct Array of keys and values from Map - const auto & map = field.get(); /// Map can not be wrapped with Nullable + const auto & map = field.safeGet(); /// Map can not be wrapped with Nullable const auto num_keys = map.size(); auto array_key = Array(); auto array_val = Array(); @@ -539,7 +539,7 @@ int64_t BackingDataLengthCalculator::calculate(const Field & field) const array_val.reserve(num_keys); for (size_t i = 0; i < num_keys; ++i) { - const auto & pair = map[i].get(); + const auto & pair = map[i].safeGet(); array_key.push_back(pair[0]); array_val.push_back(pair[1]); } @@ -561,7 +561,7 @@ int64_t BackingDataLengthCalculator::calculate(const Field & field) const if (which.isTuple()) { /// 内存布局:null_bitmap(字节数与字段数成正比) | field1 value(8B) | field2 value(8B) | ... | fieldn value(8B) | backing buffer - const auto & tuple = field.get(); /// Tuple can not be wrapped with Nullable + const auto & tuple = field.safeGet(); /// Tuple can not be wrapped with Nullable const auto * type_tuple = typeid_cast(type_without_nullable.get()); const auto & type_fields = type_tuple->getElements(); const auto num_fields = type_fields.size(); @@ -689,27 +689,27 @@ int64_t VariableLengthDataWriter::writeArray(size_t row_idx, const DB::Array & a { if (writer.getWhichDataType().isFloat32()) { - // We can not use get() directly here to process Float32 field, + // We can not use safeGet() directly here to process Float32 field, // because it will get 8 byte data, but Float32 is 4 byte, which will cause error conversion. - auto v = static_cast(elem.get()); + auto v = static_cast(elem.safeGet()); writer.unsafeWrite( reinterpret_cast(&v), buffer_address + offset + start + 8 + len_null_bitmap + i * elem_size); } else if (writer.getWhichDataType().isFloat64()) { // Fix 'Invalid Field get from type Float64 to type Int64' in debug build. - auto v = elem.get(); + auto v = elem.safeGet(); writer.unsafeWrite(reinterpret_cast(&v), buffer_address + offset + start + 8 + len_null_bitmap + i * elem_size); } else if (writer.getWhichDataType().isDecimal32()) { - // We can not use get() directly here to process Decimal32 field, + // We can not use safeGet() directly here to process Decimal32 field, // because it will get 4 byte data, but Decimal32 is 8 byte in Spark, which will cause error conversion. writer.write(elem, buffer_address + offset + start + 8 + len_null_bitmap + i * elem_size); } else writer.unsafeWrite( - reinterpret_cast(&elem.get()), + reinterpret_cast(&elem.safeGet()), buffer_address + offset + start + 8 + len_null_bitmap + i * elem_size); } } @@ -754,7 +754,7 @@ int64_t VariableLengthDataWriter::writeMap(size_t row_idx, const DB::Map & map, val_array.reserve(num_pairs); for (size_t i = 0; i < num_pairs; ++i) { - const auto & pair = map[i].get(); + const auto & pair = map[i].safeGet(); key_array.push_back(pair[0]); val_array.push_back(pair[1]); } @@ -814,25 +814,25 @@ int64_t VariableLengthDataWriter::writeStruct(size_t row_idx, const DB::Tuple & FixedLengthDataWriter writer(field_type); if (writer.getWhichDataType().isFloat32()) { - // We can not use get() directly here to process Float32 field, + // We can not use safeGet() directly here to process Float32 field, // because it will get 8 byte data, but Float32 is 4 byte, which will cause error conversion. - auto v = static_cast(field_value.get()); + auto v = static_cast(field_value.safeGet()); writer.unsafeWrite(reinterpret_cast(&v), buffer_address + offset + start + len_null_bitmap + i * 8); } else if (writer.getWhichDataType().isFloat64()) { // Fix 'Invalid Field get from type Float64 to type Int64' in debug build. - auto v = field_value.get(); + auto v = field_value.safeGet(); writer.unsafeWrite(reinterpret_cast(&v), buffer_address + offset + start + len_null_bitmap + i * 8); } else if (writer.getWhichDataType().isDecimal64() || writer.getWhichDataType().isDateTime64()) { - auto v = field_value.get(); + auto v = field_value.safeGet(); writer.unsafeWrite(reinterpret_cast(&v), buffer_address + offset + start + len_null_bitmap + i * 8); } else writer.unsafeWrite( - reinterpret_cast(&field_value.get()), buffer_address + offset + start + len_null_bitmap + i * 8); + reinterpret_cast(&field_value.safeGet()), buffer_address + offset + start + len_null_bitmap + i * 8); } else { @@ -853,7 +853,7 @@ int64_t VariableLengthDataWriter::write(size_t row_idx, const DB::Field & field, if (which.isStringOrFixedString()) { - const auto & str = field.get(); + const auto & str = field.safeGet(); return writeUnalignedBytes(row_idx, str.data(), str.size(), parent_offset); } @@ -868,19 +868,19 @@ int64_t VariableLengthDataWriter::write(size_t row_idx, const DB::Field & field, if (which.isArray()) { - const auto & array = field.get(); + const auto & array = field.safeGet(); return writeArray(row_idx, array, parent_offset); } if (which.isMap()) { - const auto & map = field.get(); + const auto & map = field.safeGet(); return writeMap(row_idx, map, parent_offset); } if (which.isTuple()) { - const auto & tuple = field.get(); + const auto & tuple = field.safeGet(); return writeStruct(row_idx, tuple, parent_offset); } @@ -926,64 +926,64 @@ void FixedLengthDataWriter::write(const DB::Field & field, char * buffer) if (which.isUInt8()) { - const auto value = UInt8(field.get()); + const auto value = static_cast(field.safeGet()); memcpy(buffer, &value, 1); } else if (which.isUInt16() || which.isDate()) { - const auto value = UInt16(field.get()); + const auto value = static_cast(field.safeGet()); memcpy(buffer, &value, 2); } else if (which.isUInt32() || which.isDate32()) { - const auto value = UInt32(field.get()); + const auto value = static_cast(field.safeGet()); memcpy(buffer, &value, 4); } else if (which.isUInt64()) { - const auto & value = field.get(); + const auto & value = field.safeGet(); memcpy(buffer, &value, 8); } else if (which.isInt8()) { - const auto value = Int8(field.get()); + const auto value = static_cast(field.safeGet()); memcpy(buffer, &value, 1); } else if (which.isInt16()) { - const auto value = Int16(field.get()); + const auto value = static_cast(field.safeGet()); memcpy(buffer, &value, 2); } else if (which.isInt32()) { - const auto value = Int32(field.get()); + const auto value = static_cast(field.safeGet()); memcpy(buffer, &value, 4); } else if (which.isInt64()) { - const auto & value = field.get(); + const auto & value = field.safeGet(); memcpy(buffer, &value, 8); } else if (which.isFloat32()) { - const auto value = Float32(field.get()); + const auto value = static_cast(field.safeGet()); memcpy(buffer, &value, 4); } else if (which.isFloat64()) { - const auto & value = field.get(); + const auto & value = field.safeGet(); memcpy(buffer, &value, 8); } else if (which.isDecimal32()) { - const auto & value = field.get(); + const auto & value = field.safeGet(); const Int64 decimal = static_cast(value.getValue()); memcpy(buffer, &decimal, 8); } else if (which.isDecimal64() || which.isDateTime64()) { - const auto & value = field.get(); - auto decimal = value.getValue(); + const auto & value = field.safeGet(); + const auto decimal = value.getValue(); memcpy(buffer, &decimal, 8); } else diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp index fffe3ed13191a..445cae973ab25 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp @@ -772,7 +772,7 @@ std::pair SerializedPlanParser::convertStructFieldType(const #define UINT_CONVERT(type_ptr, field, type_name) \ if ((type_ptr)->getTypeId() == TypeIndex::type_name) \ { \ - return {std::make_shared(), static_cast((field).get()) + 1}; \ + return {std::make_shared(), static_cast((field).safeGet()) + 1}; \ } auto type_id = type->getTypeId(); diff --git a/cpp-ch/local-engine/Parser/WriteRelParser.cpp b/cpp-ch/local-engine/Parser/WriteRelParser.cpp index 9b6226adbed81..1a468a41eef27 100644 --- a/cpp-ch/local-engine/Parser/WriteRelParser.cpp +++ b/cpp-ch/local-engine/Parser/WriteRelParser.cpp @@ -137,12 +137,12 @@ void addSinkTransfrom(const DB::ContextPtr & context, const substrait::WriteRel DB::Field field_tmp_dir; if (!settings.tryGet(SPARK_TASK_WRITE_TMEP_DIR, field_tmp_dir)) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Write Pipeline need inject temp directory."); - const auto & tmp_dir = field_tmp_dir.get(); + const auto & tmp_dir = field_tmp_dir.safeGet(); DB::Field field_filename; if (!settings.tryGet(SPARK_TASK_WRITE_FILENAME, field_filename)) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Write Pipeline need inject file name."); - const auto & filename = field_filename.get(); + const auto & filename = field_filename.safeGet(); assert(write_rel.has_named_table()); const substrait::NamedObjectWrite & named_table = write_rel.named_table(); diff --git a/cpp-ch/local-engine/Parser/aggregate_function_parser/ApproxPercentileParser.cpp b/cpp-ch/local-engine/Parser/aggregate_function_parser/ApproxPercentileParser.cpp index 237da650c8e12..ceddbd2aef809 100644 --- a/cpp-ch/local-engine/Parser/aggregate_function_parser/ApproxPercentileParser.cpp +++ b/cpp-ch/local-engine/Parser/aggregate_function_parser/ApproxPercentileParser.cpp @@ -98,7 +98,7 @@ DB::Array ApproxPercentileParser::parseFunctionParameters( if (isArray(type2)) { /// Multiple percentages for quantilesGK - const Array & percentags = field2.get(); + const Array & percentags = field2.safeGet(); for (const auto & percentage : percentags) params.emplace_back(percentage); } diff --git a/cpp-ch/local-engine/Parser/aggregate_function_parser/BloomFilterAggParser.cpp b/cpp-ch/local-engine/Parser/aggregate_function_parser/BloomFilterAggParser.cpp index 8788abb6dcf79..10bf0b09482e0 100644 --- a/cpp-ch/local-engine/Parser/aggregate_function_parser/BloomFilterAggParser.cpp +++ b/cpp-ch/local-engine/Parser/aggregate_function_parser/BloomFilterAggParser.cpp @@ -63,8 +63,8 @@ DB::Array AggregateFunctionParserBloomFilterAgg::parseFunctionParameters( node->column->get(0, ret); return ret; }; - Int64 insert_num = get_parameter_field(arg_nodes[1], 1).get(); - Int64 bits_num = get_parameter_field(arg_nodes[2], 2).get(); + Int64 insert_num = get_parameter_field(arg_nodes[1], 1).safeGet(); + Int64 bits_num = get_parameter_field(arg_nodes[2], 2).safeGet(); // Delete all args except the first arg. arg_nodes.resize(1); diff --git a/cpp-ch/local-engine/Parser/aggregate_function_parser/NtileParser.cpp b/cpp-ch/local-engine/Parser/aggregate_function_parser/NtileParser.cpp index 62f83223c06f7..1a24e320609e7 100644 --- a/cpp-ch/local-engine/Parser/aggregate_function_parser/NtileParser.cpp +++ b/cpp-ch/local-engine/Parser/aggregate_function_parser/NtileParser.cpp @@ -32,7 +32,7 @@ NtileParser::parseFunctionArguments(const CommonFunctionInfo & func_info, DB::Ac auto [data_type, field] = parseLiteral(arg0.literal()); if (!(DB::WhichDataType(data_type).isInt32())) throw Exception(ErrorCodes::BAD_ARGUMENTS, "ntile's argument must be i32"); - Int32 field_index = static_cast(field.get()); + Int32 field_index = static_cast(field.safeGet()); // For CH, the data type of the args[0] must be the UInt32 const auto * index_node = addColumnToActionsDAG(actions_dag, std::make_shared(), field_index); args.emplace_back(index_node); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayHighOrderFunctions.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayHighOrderFunctions.cpp index 675a4fdf0fe9b..aa82b33a7a3c8 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayHighOrderFunctions.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayHighOrderFunctions.cpp @@ -216,14 +216,14 @@ class ArraySort : public FunctionParser if (!var_expr.has_literal()) return false; auto [_, name] = plan_parser->parseLiteral(var_expr.literal()); - return var == name.get(); + return var == name.safeGet(); }; auto is_int_value = [&](const substrait::Expression & expr, Int32 val) { if (!expr.has_literal()) return false; auto [_, x] = plan_parser->parseLiteral(expr.literal()); - return val == x.get(); + return val == x.safeGet(); }; auto is_variable_null = [&](const substrait::Expression & expr, const String & var) { diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/lambdaFunction.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/lambdaFunction.cpp index 547ffd971fcd4..c2841564e8c3f 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/lambdaFunction.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/lambdaFunction.cpp @@ -43,7 +43,7 @@ DB::NamesAndTypesList collectLambdaArguments(const SerializedPlanParser & plan_p && plan_parser_.getFunctionSignatureName(arg.value().scalar_function().function_reference()) == "namedlambdavariable") { auto [_, col_name_field] = plan_parser_.parseLiteral(arg.value().scalar_function().arguments()[0].value().literal()); - String col_name = col_name_field.get(); + String col_name = col_name_field.safeGet(); if (collected_names.contains(col_name)) { continue; @@ -187,7 +187,7 @@ class NamedLambdaVariable : public FunctionParser const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { auto [_, col_name_field] = parseLiteral(substrait_func.arguments()[0].value().literal()); - String col_name = col_name_field.get(); + String col_name = col_name_field.safeGet(); auto type = TypeParser::parseType(substrait_func.output_type()); const auto & inputs = actions_dag.getInputs(); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/tupleElement.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/tupleElement.cpp index 179aa7860484d..4809cc887b8df 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/tupleElement.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/tupleElement.cpp @@ -45,7 +45,7 @@ namespace local_engine auto [data_type, field] = parseLiteral(args[1].value().literal()); \ if (!DB::WhichDataType(data_type).isInt32()) \ throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "{}'s second argument must be i32", #substrait_name); \ - Int32 field_index = static_cast(field.get() + 1); \ + Int32 field_index = static_cast(field.safeGet() + 1); \ const auto * index_node = addColumnToActionsDAG(actions_dag, std::make_shared(), field_index); \ parsed_args.emplace_back(index_node); \ const auto * func_node = toFunctionNode(actions_dag, ch_function_name, parsed_args); \ diff --git a/cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.cpp b/cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.cpp index 6fee65efe5931..93f4374d4ce1c 100644 --- a/cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.cpp +++ b/cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.cpp @@ -71,16 +71,16 @@ SparkMergeTreeWriter::SparkMergeTreeWriter( , thread_pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled, 1, 1, 100000) { const DB::Settings & settings = context->getSettingsRef(); - merge_after_insert = settings.get(MERGETREE_MERGE_AFTER_INSERT).get(); - insert_without_local_storage = settings.get(MERGETREE_INSERT_WITHOUT_LOCAL_STORAGE).get(); + merge_after_insert = settings.get(MERGETREE_MERGE_AFTER_INSERT).safeGet(); + insert_without_local_storage = settings.get(MERGETREE_INSERT_WITHOUT_LOCAL_STORAGE).safeGet(); Field limit_size_field; if (settings.tryGet("optimize.minFileSize", limit_size_field)) - merge_min_size = limit_size_field.get() <= 0 ? merge_min_size : limit_size_field.get(); + merge_min_size = limit_size_field.safeGet() <= 0 ? merge_min_size : limit_size_field.safeGet(); Field limit_cnt_field; if (settings.tryGet("mergetree.max_num_part_per_merge_task", limit_cnt_field)) - merge_limit_parts = limit_cnt_field.get() <= 0 ? merge_limit_parts : limit_cnt_field.get(); + merge_limit_parts = limit_cnt_field.safeGet() <= 0 ? merge_limit_parts : limit_cnt_field.safeGet(); dest_storage = MergeTreeRelParser::parseStorage(merge_tree_table, SerializedPlanParser::global_context); isRemoteStorage = dest_storage->getStoragePolicy()->getAnyDisk()->isRemote(); diff --git a/cpp-ch/local-engine/Storages/Parquet/ParquetConverter.h b/cpp-ch/local-engine/Storages/Parquet/ParquetConverter.h index 89e83e668aebc..312cea7efc0af 100644 --- a/cpp-ch/local-engine/Storages/Parquet/ParquetConverter.h +++ b/cpp-ch/local-engine/Storages/Parquet/ParquetConverter.h @@ -38,9 +38,9 @@ struct ToParquet T as(const DB::Field & value, const parquet::ColumnDescriptor &) { if constexpr (std::is_same_v) - return static_cast(value.get()); + return static_cast(value.safeGet()); // parquet::BooleanType, parquet::Int64Type, parquet::FloatType, parquet::DoubleType - return value.get(); // FLOAT, DOUBLE, INT64 + return value.safeGet(); // FLOAT, DOUBLE, INT64 } }; @@ -51,7 +51,7 @@ struct ToParquet T as(const DB::Field & value, const parquet::ColumnDescriptor &) { assert(value.getType() == DB::Field::Types::String); - const std::string & s = value.get(); + const std::string & s = value.safeGet(); const auto * const ptr = reinterpret_cast(s.data()); return parquet::ByteArray(static_cast(s.size()), ptr); } @@ -74,7 +74,7 @@ struct ToParquet "descriptor.type_length() = {} , which is > {}, e.g. sizeof(Int128)", descriptor.type_length(), sizeof(Int128)); - Int128 val = value.get>().getValue(); + Int128 val = value.safeGet>().getValue(); std::reverse(reinterpret_cast(&val), reinterpret_cast(&val) + sizeof(val)); const int offset = sizeof(Int128) - descriptor.type_length(); memcpy(buf, reinterpret_cast(&val) + offset, descriptor.type_length()); diff --git a/cpp-ch/local-engine/tests/gtest_clickhouse_pr_verify.cpp b/cpp-ch/local-engine/tests/gtest_clickhouse_pr_verify.cpp index 6352a819927c6..e1fa3a34c4c3c 100644 --- a/cpp-ch/local-engine/tests/gtest_clickhouse_pr_verify.cpp +++ b/cpp-ch/local-engine/tests/gtest_clickhouse_pr_verify.cpp @@ -63,7 +63,7 @@ TEST(Clickhouse, PR54881) Field field; const auto & col_1 = *(block.getColumns()[1]); col_1.get(0, field); - const Tuple & row_0 = field.get(); + const Tuple & row_0 = field.safeGet(); EXPECT_EQ(2, row_0.size()); Int64 actual{-1}; @@ -74,7 +74,7 @@ TEST(Clickhouse, PR54881) EXPECT_EQ(10, actual); col_1.get(1, field); - const Tuple & row_1 = field.get(); + const Tuple & row_1 = field.safeGet(); EXPECT_EQ(2, row_1.size()); EXPECT_TRUE(row_1[0].tryGet(actual)); EXPECT_EQ(10, actual);