Skip to content

Commit

Permalink
[Refactor] Fix clang compile (#24832)
Browse files Browse the repository at this point in the history
Fixes #issue

Signed-off-by: imay <buaa.zhaoc@gmail.com>
  • Loading branch information
imay authored Jun 8, 2023
1 parent e770adb commit d843c7d
Show file tree
Hide file tree
Showing 34 changed files with 306 additions and 288 deletions.
4 changes: 3 additions & 1 deletion be/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -467,7 +467,9 @@ set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -Wno-deprecated -Wno-vla -Wno-comment"

if (CMAKE_CXX_COMPILER_ID STREQUAL "Clang")
set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -Wno-unused-parameter -Wno-documentation -Wno-weak-vtables")
set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -Wno-reserved-identifier -Wno-suggest-destructor-override")
if (CMAKE_CXX_COMPILER_VERSION VERSION_GREATER_EQUAL "11.0.0")
set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -Wno-reserved-identifier -Wno-suggest-destructor-override")
endif()
set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -Wno-documentation-unknown-command -Wno-old-style-cast")
set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -Wno-c++20-designator -Wno-mismatched-tags")
if (CMAKE_CXX_COMPILER_VERSION VERSION_GREATER_EQUAL "14.0.0")
Expand Down
3 changes: 2 additions & 1 deletion be/src/exec/parquet_scanner.h
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,8 @@ class ParquetScanner : public FileScanner {
Status open() override;

StatusOr<ChunkPtr> get_next() override;
Status get_schema(std::vector<SlotDescriptor>* schema);

Status get_schema(std::vector<SlotDescriptor>* schema) override;

void close() override;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,8 @@ void LazyInstantiateDriversOperator::close(RuntimeState* state) {
auto* fragment_ctx = state->fragment_ctx();
for (auto& pipeline_group : _unready_pipeline_groups) {
for (auto& pipeline : pipeline_group.pipelines) {
if (typeid(*pipeline->sink_operator_factory()) != typeid(ResultSinkOperatorFactory)) {
auto sink_factory = pipeline->sink_operator_factory();
if (typeid(*sink_factory) != typeid(ResultSinkOperatorFactory)) {
fragment_ctx->count_down_pipeline(state);
} else {
// Closing ResultSinkOperator notifies FE not to wait fetch_data anymore.
Expand Down
4 changes: 1 addition & 3 deletions be/src/exec/pipeline/set/except_output_source_operator.h
Original file line number Diff line number Diff line change
Expand Up @@ -27,8 +27,7 @@ class ExceptOutputSourceOperator final : public SourceOperator {
ExceptOutputSourceOperator(OperatorFactory* factory, int32_t id, int32_t plan_node_id, int32_t driver_sequence,
std::shared_ptr<ExceptContext> except_ctx, const int32_t dependency_index)
: SourceOperator(factory, id, "except_output_source", plan_node_id, driver_sequence),
_except_ctx(std::move(except_ctx)),
_dependency_index(dependency_index) {
_except_ctx(std::move(except_ctx)) {
_except_ctx->ref();
}

Expand All @@ -44,7 +43,6 @@ class ExceptOutputSourceOperator final : public SourceOperator {

private:
std::shared_ptr<ExceptContext> _except_ctx;
const int32_t _dependency_index;
};

class ExceptOutputSourceOperatorFactory final : public SourceOperatorFactory {
Expand Down
2 changes: 0 additions & 2 deletions be/src/exec/pipeline/sink/iceberg_table_sink_operator.h
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,6 @@ class IcebergTableSinkOperator final : public Operator {
_cloud_conf(cloud_conf),
_iceberg_table(iceberg_table),
_parquet_file_schema(std::move(schema)),
_fragment_ctx(fragment_ctx),
_output_expr(output_expr_ctxs),
_partition_expr(partition_output_expr) {}

Expand Down Expand Up @@ -83,7 +82,6 @@ class IcebergTableSinkOperator final : public Operator {

IcebergTableDescriptor* _iceberg_table;
std::shared_ptr<::parquet::schema::GroupNode> _parquet_file_schema;
FragmentContext* _fragment_ctx = nullptr;
std::vector<ExprContext*> _output_expr;
std::vector<ExprContext*> _partition_expr;
std::unordered_map<std::string, std::unique_ptr<starrocks::RollingAsyncParquetWriter>> _partition_writers;
Expand Down
2 changes: 1 addition & 1 deletion be/src/exec/spill/spiller.hpp
Original file line number Diff line number Diff line change
Expand Up @@ -187,7 +187,7 @@ Status SpillerReader::trigger_restore(RuntimeState* state, TaskExecutor&& execut
// if all is well and input stream enable prefetch and not eof
if (!_stream->eof()) {
_running_restore_tasks++;
auto restore_task = [this, state, guard, trace = TraceInfo(state)]() {
auto restore_task = [this, guard, trace = TraceInfo(state)]() {
SCOPED_SET_TRACE_INFO({}, trace.query_id, trace.fragment_id);
RETURN_IF(!guard.scoped_begin(), Status::OK());
auto defer = DeferOp([&]() { _running_restore_tasks--; });
Expand Down
10 changes: 5 additions & 5 deletions be/src/exprs/cast_expr.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -576,7 +576,7 @@ UNARY_FN_CAST_VALID(TYPE_LARGEINT, TYPE_INT, ImplicitToNumber);

DIAGNOSTIC_PUSH
#if defined(__clang__)
DIAGNOSTIC_IGNORE("-Wimplicit-const-int-float-conversion")
DIAGNOSTIC_IGNORE("-Wimplicit-int-float-conversion")
#endif
UNARY_FN_CAST_VALID(TYPE_FLOAT, TYPE_INT, ImplicitToNumber);
DIAGNOSTIC_POP
Expand All @@ -599,7 +599,7 @@ UNARY_FN_CAST_VALID(TYPE_LARGEINT, TYPE_BIGINT, ImplicitToNumber);

DIAGNOSTIC_PUSH
#if defined(__clang__)
DIAGNOSTIC_IGNORE("-Wimplicit-const-int-float-conversion")
DIAGNOSTIC_IGNORE("-Wimplicit-int-float-conversion")
#endif
UNARY_FN_CAST_VALID(TYPE_FLOAT, TYPE_BIGINT, ImplicitToNumber);
UNARY_FN_CAST_VALID(TYPE_DOUBLE, TYPE_BIGINT, ImplicitToNumber);
Expand All @@ -622,7 +622,7 @@ UNARY_FN_CAST_VALID(TYPE_BIGINT, TYPE_LARGEINT, ImplicitToNumber);

DIAGNOSTIC_PUSH
#if defined(__clang__)
DIAGNOSTIC_IGNORE("-Wimplicit-const-int-float-conversion")
DIAGNOSTIC_IGNORE("-Wimplicit-int-float-conversion")
#endif
UNARY_FN_CAST_VALID(TYPE_FLOAT, TYPE_LARGEINT, ImplicitToNumber);
UNARY_FN_CAST_VALID(TYPE_DOUBLE, TYPE_LARGEINT, ImplicitToNumber);
Expand All @@ -643,7 +643,7 @@ UNARY_FN_CAST_VALID(TYPE_SMALLINT, TYPE_FLOAT, ImplicitToNumber);

DIAGNOSTIC_PUSH
#if defined(__clang__)
DIAGNOSTIC_IGNORE("-Wimplicit-const-int-float-conversion")
DIAGNOSTIC_IGNORE("-Wimplicit-int-float-conversion")
#endif
UNARY_FN_CAST_VALID(TYPE_INT, TYPE_FLOAT, ImplicitToNumber);
UNARY_FN_CAST_VALID(TYPE_BIGINT, TYPE_FLOAT, ImplicitToNumber);
Expand All @@ -667,7 +667,7 @@ UNARY_FN_CAST_VALID(TYPE_INT, TYPE_DOUBLE, ImplicitToNumber);

DIAGNOSTIC_PUSH
#if defined(__clang__)
DIAGNOSTIC_IGNORE("-Wimplicit-const-int-float-conversion")
DIAGNOSTIC_IGNORE("-Wimplicit-int-float-conversion")
#endif
UNARY_FN_CAST_VALID(TYPE_BIGINT, TYPE_DOUBLE, ImplicitToNumber);
UNARY_FN_CAST_VALID(TYPE_LARGEINT, TYPE_DOUBLE, ImplicitToNumber);
Expand Down
2 changes: 1 addition & 1 deletion be/src/formats/avro/numeric_column.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ static inline bool checked_cast(const FromType& from, ToType* to) {
// NOTE: use lowest() because float and double needed.
DIAGNOSTIC_PUSH
#if defined(__clang__)
DIAGNOSTIC_IGNORE("-Wimplicit-const-int-float-conversion")
DIAGNOSTIC_IGNORE("-Wimplicit-int-float-conversion")
#endif
return (from < std::numeric_limits<ToType>::lowest() || from > std::numeric_limits<ToType>::max());
DIAGNOSTIC_POP
Expand Down
2 changes: 1 addition & 1 deletion be/src/formats/json/numeric_column.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ static inline bool checked_cast(const FromType& from, ToType* to) {
// Needs to covnert long and int128_t to double to compare, so disable compiler's complain
DIAGNOSTIC_PUSH
#if defined(__clang__)
DIAGNOSTIC_IGNORE("-Wimplicit-const-int-float-conversion")
DIAGNOSTIC_IGNORE("-Wimplicit-int-float-conversion")
#endif
return (from < std::numeric_limits<ToType>::lowest() || from > std::numeric_limits<ToType>::max());
DIAGNOSTIC_POP
Expand Down
2 changes: 0 additions & 2 deletions be/src/runtime/iceberg_table_sink.h
Original file line number Diff line number Diff line change
Expand Up @@ -42,10 +42,8 @@ class IcebergTableSink : public DataSink {
std::vector<TExpr> get_output_expr() const { return _t_output_expr; }

private:
RuntimeState* _state;
ObjectPool* _pool;
const std::vector<TExpr>& _t_output_expr;
int _chunk_size;
std::vector<ExprContext*> _output_expr_ctxs;
RuntimeProfile* _profile = nullptr;
};
Expand Down
2 changes: 1 addition & 1 deletion be/src/storage/options.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -180,7 +180,7 @@ Status parse_conf_block_cache_paths(const std::string& config_path, std::vector<
LOG(WARNING) << "invalid block cache path. path=" << item;
continue;
}
paths->emplace_back(std::move(local_path.string()));
paths->emplace_back(local_path.string());
}
if ((path_vec.size() != paths->size() && !config::ignore_broken_disk)) {
LOG(WARNING) << "fail to parse block_cache_disk_path config. value=[" << config_path << "]";
Expand Down
20 changes: 10 additions & 10 deletions be/src/storage/rowset/segment_group.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,16 @@ ShortKeyIndexGroupIterator ShortKeyIndexDecoderGroup::back() const {
return {this, num_blocks() - 1};
}

template <bool is_lower_bound>
ShortKeyIndexGroupIterator ShortKeyIndexDecoderGroup::_seek(const Slice& key) const {
auto comparator = [](const Slice& lhs, const Slice& rhs) { return lhs.compare(rhs) < 0; };
if constexpr (is_lower_bound) {
return std::lower_bound(begin(), end(), key, comparator);
} else {
return std::upper_bound(begin(), end(), key, comparator);
}
}

ShortKeyIndexGroupIterator ShortKeyIndexDecoderGroup::lower_bound(const Slice& key) const {
return _seek<true>(key);
}
Expand All @@ -76,16 +86,6 @@ Slice ShortKeyIndexDecoderGroup::key(ssize_t ordinal) const {
return _sk_index_decoders[size_t(decoder_id)]->key(block_id_in_decoder);
}

template <bool lower_bound>
ShortKeyIndexGroupIterator ShortKeyIndexDecoderGroup::_seek(const Slice& key) const {
auto comparator = [](const Slice& lhs, const Slice& rhs) { return lhs.compare(rhs) < 0; };
if constexpr (lower_bound) {
return std::lower_bound(begin(), end(), key, comparator);
} else {
return std::upper_bound(begin(), end(), key, comparator);
}
}

void ShortKeyIndexDecoderGroup::_find_position(ssize_t ordinal, ssize_t* decoder_id,
ssize_t* block_id_in_decoder) const {
auto it = std::upper_bound(_decoder_start_ordinals.begin(), _decoder_start_ordinals.end(), ordinal);
Expand Down
3 changes: 2 additions & 1 deletion be/test/column/map_column_test.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -1308,7 +1308,8 @@ PARALLEL_TEST(MapColumnTest, test_remove_duplicated_keys) {
nest_offsets->get_data().push_back(2);
nest_offsets->get_data().push_back(4);

auto nest_map = MapColumn::create(std::move(nest_keys), std::move(ColumnHelper::cast_to_nullable_column(column)), nest_offsets);
auto nest_map = MapColumn::create(std::move(nest_keys),
std::move(ColumnHelper::cast_to_nullable_column(column)), nest_offsets);
nest_map->remove_duplicated_keys(true);

ASSERT_EQ("{1:{4:66}}", nest_map->debug_item(0));
Expand Down
5 changes: 2 additions & 3 deletions be/test/exec/arrow_converter_test.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -1262,7 +1262,6 @@ static std::shared_ptr<arrow::Array> create_map_array(int64_t num_elements, cons
return builder.Finish().ValueOrDie();
}


static std::shared_ptr<arrow::Array> create_struct_array(int elemnts_num, bool is_null) {
auto int1_builder = std::make_shared<arrow::Int32Builder>();
auto str_builder = std::make_shared<arrow::StringBuilder>();
Expand All @@ -1275,7 +1274,8 @@ static std::shared_ptr<arrow::Array> create_struct_array(int elemnts_num, bool i

auto data_type = std::make_shared<arrow::StructType>(fields);

arrow::TypeTraits<arrow::StructType>::BuilderType builder(data_type, arrow::default_memory_pool(), {int1_builder, str_builder, int2_builder});
arrow::TypeTraits<arrow::StructType>::BuilderType builder(data_type, arrow::default_memory_pool(),
{int1_builder, str_builder, int2_builder});

for (int i = 0; i < elemnts_num; i++) {
if (is_null && i % 2 == 0) {
Expand Down Expand Up @@ -1575,7 +1575,6 @@ PARALLEL_TEST(ArrowConverterTest, test_convert_struct_less_column) {
ASSERT_EQ(st_col->debug_item(9), "{col1:9,col2:'char-9',col3:90,col4:NULL}");
}


PARALLEL_TEST(ArrowConverterTest, test_convert_struct_more_column) {
TypeDescriptor struct_type(TYPE_STRUCT);
struct_type.children.emplace_back(TYPE_INT);
Expand Down
3 changes: 1 addition & 2 deletions be/test/exec/iceberg/iceberg_delete_builder_test.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,6 @@ class IcebergDeleteBuilderTest : public testing::Test {
~IcebergDeleteBuilderTest() override = default;

protected:

std::string _parquet_delete_path = "./be/test/exec/test_data/parquet_scanner/parquet_delete_file.parquet";
std::string _parquet_data_path = "parquet_data_file.parquet";

Expand All @@ -60,4 +59,4 @@ TEST_F(IcebergDeleteBuilderTest, TestParquetBuilder) {
ASSERT_EQ(1, _need_skip_rowids.size());
}

}
} // namespace starrocks
5 changes: 2 additions & 3 deletions be/test/exec/parquet_scanner_test.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -274,8 +274,8 @@ class ParquetScannerTest : public ::testing::Test {

template <bool is_nullable>
ChunkPtr get_chunk(const std::vector<std::string>& columns_from_file,
const std::unordered_map<size_t, ::starrocks::TExpr>& dst_slot_exprs,
std::string specific_file, size_t expected_rows) {
const std::unordered_map<size_t, ::starrocks::TExpr>& dst_slot_exprs, std::string specific_file,
size_t expected_rows) {
std::vector<std::string> file_names{std::move(specific_file)};
const std::vector<std::string>& column_names = columns_from_file;

Expand Down Expand Up @@ -668,7 +668,6 @@ TEST_F(ParquetScannerTest, int96_timestamp) {
}
}


TEST_F(ParquetScannerTest, get_file_schema) {
const std::vector<std::pair<std::string, std::vector<std::pair<std::string, LogicalType>>>> test_cases = {
{test_exec_dir + "/test_data/parquet_data/int96_timestamp.parquet", {{"col_datetime", TYPE_DATETIME}}},
Expand Down
4 changes: 1 addition & 3 deletions be/test/exec/pipeline/pipeline_test_base.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -117,9 +117,7 @@ void PipelineTestBase::_prepare() {

void PipelineTestBase::_execute() {
Status prepare_status = _fragment_ctx->iterate_drivers(
[state = _fragment_ctx->runtime_state()](const DriverPtr& driver) {
return driver->prepare(state);
});
[state = _fragment_ctx->runtime_state()](const DriverPtr& driver) { return driver->prepare(state); });
ASSERT_TRUE(prepare_status.ok());

_fragment_ctx->iterate_drivers([exec_env = _exec_env](const DriverPtr& driver) {
Expand Down
2 changes: 1 addition & 1 deletion be/test/exprs/coalesce_expr_test.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -140,7 +140,7 @@ TEST_F(VectorizedCoalesceExprTest, coalesceNull) {
{
Chunk chunk;
ColumnPtr ptr = expr->evaluate(nullptr, &chunk);
if(ptr->is_nullable()) {
if (ptr->is_nullable()) {
ptr = down_cast<NullableColumn*>(ptr.get())->data_column();
}
ASSERT_TRUE(ptr->is_numeric());
Expand Down
2 changes: 1 addition & 1 deletion be/test/exprs/condition_expr_test.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -283,7 +283,7 @@ TEST_F(VectorizedConditionExprTest, ifExpr) {
expr0->_children.push_back(&col2);

ColumnPtr ptr = expr0->evaluate(nullptr, nullptr);
if(ptr->is_nullable()) {
if (ptr->is_nullable()) {
ptr = down_cast<NullableColumn*>(ptr.get())->data_column();
}
auto* res_col0 = down_cast<Int32Column*>(ptr.get());
Expand Down
56 changes: 33 additions & 23 deletions be/test/exprs/decimal_cast_expr_float_test.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -811,18 +811,19 @@ PARALLEL_TEST(VectorizedDecimalCastExprFloatTest, testCastFromFloatToDecimal32p7
}

PARALLEL_TEST(VectorizedDecimalCastExprFloatTest, testCastFromFloatToDecimal32p7s4Normal) {
CastTestCaseArray test_cases = {{-1, -1, "91617.46", 7, 4, "91617.4609", "false"}, //
{-1, -1, "-13741.9795", 7, 4, "-13741.9795", "false"},
{-1, -1, "2805.812", 7, 4, "2805.8120", "false"},
{-1, -1, "6.9628773", 7, 4, "6.9629", "false"},
{-1, -1, "-123.74408", 7, 4, "-123.7441", "false"},
{-1, -1, "-144.42905", 7, 4, "-144.4290", "false"}, // string to double -144.429046630859
{-1, -1, "-77489.55", 7, 4, "-77489.5469", "false"},
{-1, -1, "-4.8811874", 7, 4, "-4.8812", "false"},
{-1, -1, "0.0", 7, 4, "0.0000", "false"},
{-1, -1, "-2.724122", 7, 4, "-2.7241", "false"},
{-1, -1, "534.44507", 7, 4, "534.4451", "false"},
{-1, -1, "1.1364293", 7, 4, "1.1364", "false"}};
CastTestCaseArray test_cases = {
{-1, -1, "91617.46", 7, 4, "91617.4609", "false"}, //
{-1, -1, "-13741.9795", 7, 4, "-13741.9795", "false"},
{-1, -1, "2805.812", 7, 4, "2805.8120", "false"},
{-1, -1, "6.9628773", 7, 4, "6.9629", "false"},
{-1, -1, "-123.74408", 7, 4, "-123.7441", "false"},
{-1, -1, "-144.42905", 7, 4, "-144.4290", "false"}, // string to double -144.429046630859
{-1, -1, "-77489.55", 7, 4, "-77489.5469", "false"},
{-1, -1, "-4.8811874", 7, 4, "-4.8812", "false"},
{-1, -1, "0.0", 7, 4, "0.0000", "false"},
{-1, -1, "-2.724122", 7, 4, "-2.7241", "false"},
{-1, -1, "534.44507", 7, 4, "534.4451", "false"},
{-1, -1, "1.1364293", 7, 4, "1.1364", "false"}};
test_cast_all<TYPE_FLOAT, TYPE_DECIMAL32>(test_cases);
}

Expand Down Expand Up @@ -851,17 +852,26 @@ PARALLEL_TEST(VectorizedDecimalCastExprFloatTest, testCastFromFloatToDecimal64p1
}

PARALLEL_TEST(VectorizedDecimalCastExprFloatTest, testCastFromFloatToDecimal64p18s2Normal) {
CastTestCaseArray test_cases = {
{-1, -1, "6403.0576", 18, 2, "6403.06", "false"}, {-1, -1, "-416908.6", 18, 2, "-416908.59", "false"},
{-1, -1, "-374.52243", 18, 2, "-374.52", "false"}, {-1, -1, "-7448.2847", 18, 2, "-7448.28", "false"},
{-1, -1, "54660.094", 18, 2, "54660.09", "false"}, {-1, -1, "1.6501439", 18, 2, "1.65", "false"},
{-1, -1, "-934552.3", 18, 2, "-934552.31", "false"}, {-1, -1, "0.020804286", 18, 2, "0.02", "false"},
{-1, -1, "7.1221914", 18, 2, "7.12", "false"}, {-1, -1, "-21574.484", 18, 2, "-21574.48", "false"},
{-1, -1, "-98968.22", 18, 2, "-98968.22", "false"}, {-1, -1, "0.46264833", 18, 2, "0.46", "false"},
{-1, -1, "221174.38", 18, 2, "221174.38", "false"}, {-1, -1, "1.76979072E8", 18, 2, "176979072.00", "false"},
{-1, -1, "-5.987686", 18, 2, "-5.99", "false"}, {-1, -1, "2553868.2", 18, 2, "2553868.25", "false"},
{-1, -1, "25450.646", 18, 2, "25450.65", "false"}, {-1, -1, "-76036.64", 18, 2, "-76036.64", "false"},
{-1, -1, "-24.997913", 18, 2, "-25.00", "false"}, {-1, -1, "-61.181725", 18, 2, "-61.18", "false"}};
CastTestCaseArray test_cases = {{-1, -1, "6403.0576", 18, 2, "6403.06", "false"},
{-1, -1, "-416908.6", 18, 2, "-416908.59", "false"},
{-1, -1, "-374.52243", 18, 2, "-374.52", "false"},
{-1, -1, "-7448.2847", 18, 2, "-7448.28", "false"},
{-1, -1, "54660.094", 18, 2, "54660.09", "false"},
{-1, -1, "1.6501439", 18, 2, "1.65", "false"},
{-1, -1, "-934552.3", 18, 2, "-934552.31", "false"},
{-1, -1, "0.020804286", 18, 2, "0.02", "false"},
{-1, -1, "7.1221914", 18, 2, "7.12", "false"},
{-1, -1, "-21574.484", 18, 2, "-21574.48", "false"},
{-1, -1, "-98968.22", 18, 2, "-98968.22", "false"},
{-1, -1, "0.46264833", 18, 2, "0.46", "false"},
{-1, -1, "221174.38", 18, 2, "221174.38", "false"},
{-1, -1, "1.76979072E8", 18, 2, "176979072.00", "false"},
{-1, -1, "-5.987686", 18, 2, "-5.99", "false"},
{-1, -1, "2553868.2", 18, 2, "2553868.25", "false"},
{-1, -1, "25450.646", 18, 2, "25450.65", "false"},
{-1, -1, "-76036.64", 18, 2, "-76036.64", "false"},
{-1, -1, "-24.997913", 18, 2, "-25.00", "false"},
{-1, -1, "-61.181725", 18, 2, "-61.18", "false"}};
test_cast_all<TYPE_FLOAT, TYPE_DECIMAL64>(test_cases);
}

Expand Down
Loading

0 comments on commit d843c7d

Please sign in to comment.