You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by mo...@apache.org on 2023/01/23 13:49:48 UTC

[doris] branch branch-1.2-lts updated (e23788e0a7 -> 768ff43c65)

This is an automated email from the ASF dual-hosted git repository.

morningman pushed a change to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git


    from e23788e0a7 [Chore](build) enable -Wextra and remove some -Wno (#15760)
     new f59626a32e [opt](planner) return bigint literal when cast date literal to bigint type (#15613)
     new 776255018e [refactor](rpc fn) decouple vectorized remote function from row-based one (#15871)
     new 12faf8a99d [Bug](function)  catch function calculation error on aggregate node to avoid core dump (#15903)
     new 521bed9ca1 [enhancement](load) change the publish version log to VLOG_CRITICAL (#15673)
     new 768ff43c65 [Compile](lzo) fix lzo decompressor compiler error (#15956)

The 5 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 be/src/exec/decompressor.h                         |   2 +-
 be/src/olap/task/engine_publish_version_task.cpp   |   8 +-
 be/src/vec/exec/vaggregation_node.cpp              |  36 +-
 be/src/vec/exec/vaggregation_node.h                |  35 +-
 be/src/vec/exprs/vectorized_agg_fn.cpp             |  38 +-
 be/src/vec/exprs/vectorized_agg_fn.h               |  20 +-
 be/src/vec/functions/function_rpc.cpp              | 505 ++++++++++++++++++++-
 be/src/vec/functions/function_rpc.h                |  42 +-
 .../org/apache/doris/analysis/DateLiteral.java     |   9 +-
 .../bitmap_functions/test_bitmap_function.out      |   3 +
 .../bitmap_functions/test_bitmap_function.groovy   |  16 +
 11 files changed, 642 insertions(+), 72 deletions(-)


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 03/05: [Bug](function) catch function calculation error on aggregate node to avoid core dump (#15903)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 12faf8a99ded291deae17e9cdb5c6ff1104b5f9a
Author: Pxl <px...@qq.com>
AuthorDate: Mon Jan 16 11:21:28 2023 +0800

    [Bug](function)  catch function calculation error on aggregate node to avoid core dump (#15903)
---
 be/src/vec/exec/vaggregation_node.cpp              | 36 ++++++++++----------
 be/src/vec/exec/vaggregation_node.h                | 35 ++++++++++++--------
 be/src/vec/exprs/vectorized_agg_fn.cpp             | 38 +++++++++++++---------
 be/src/vec/exprs/vectorized_agg_fn.h               | 20 ++++++------
 .../bitmap_functions/test_bitmap_function.out      |  3 ++
 .../bitmap_functions/test_bitmap_function.groovy   | 16 +++++++++
 6 files changed, 91 insertions(+), 57 deletions(-)

diff --git a/be/src/vec/exec/vaggregation_node.cpp b/be/src/vec/exec/vaggregation_node.cpp
index 9056dee8ca..e93fc89591 100644
--- a/be/src/vec/exec/vaggregation_node.cpp
+++ b/be/src/vec/exec/vaggregation_node.cpp
@@ -81,12 +81,11 @@ AggregationNode::AggregationNode(ObjectPool* pool, const TPlanNode& tnode,
                                  const DescriptorTbl& descs)
         : ExecNode(pool, tnode, descs),
           _intermediate_tuple_id(tnode.agg_node.intermediate_tuple_id),
-          _intermediate_tuple_desc(NULL),
+          _intermediate_tuple_desc(nullptr),
           _output_tuple_id(tnode.agg_node.output_tuple_id),
-          _output_tuple_desc(NULL),
+          _output_tuple_desc(nullptr),
           _needs_finalize(tnode.agg_node.need_finalize),
           _is_merge(false),
-          _agg_data(),
           _build_timer(nullptr),
           _serialize_key_timer(nullptr),
           _exec_timer(nullptr),
@@ -671,9 +670,9 @@ Status AggregationNode::_execute_without_key(Block* block) {
     DCHECK(_agg_data->without_key != nullptr);
     SCOPED_TIMER(_build_timer);
     for (int i = 0; i < _aggregate_evaluators.size(); ++i) {
-        _aggregate_evaluators[i]->execute_single_add(
+        RETURN_IF_ERROR(_aggregate_evaluators[i]->execute_single_add(
                 block, _agg_data->without_key + _offsets_of_aggregate_states[i],
-                _agg_arena_pool.get());
+                _agg_arena_pool.get()));
     }
     return Status::OK();
 }
@@ -706,9 +705,9 @@ Status AggregationNode::_merge_without_key(Block* block) {
                 }
             }
         } else {
-            _aggregate_evaluators[i]->execute_single_add(
+            RETURN_IF_ERROR(_aggregate_evaluators[i]->execute_single_add(
                     block, _agg_data->without_key + _offsets_of_aggregate_states[i],
-                    _agg_arena_pool.get());
+                    _agg_arena_pool.get()));
         }
     }
     return Status::OK();
@@ -974,8 +973,8 @@ Status AggregationNode::_pre_agg_with_serialized_key(doris::vectorized::Block* i
     // to avoid wasting memory.
     // But for fixed hash map, it never need to expand
     bool ret_flag = false;
-    std::visit(
-            [&](auto&& agg_method) -> void {
+    RETURN_IF_ERROR(std::visit(
+            [&](auto&& agg_method) -> Status {
                 if (auto& hash_tbl = agg_method.data; hash_tbl.add_elem_size_overflow(rows)) {
                     // do not try to do agg, just init and serialize directly return the out_block
                     if (!_should_expand_preagg_hash_tables()) {
@@ -1007,8 +1006,10 @@ Status AggregationNode::_pre_agg_with_serialized_key(doris::vectorized::Block* i
 
                             for (int i = 0; i != _aggregate_evaluators.size(); ++i) {
                                 SCOPED_TIMER(_serialize_data_timer);
-                                _aggregate_evaluators[i]->streaming_agg_serialize_to_column(
-                                        in_block, value_columns[i], rows, _agg_arena_pool.get());
+                                RETURN_IF_ERROR(
+                                        _aggregate_evaluators[i]->streaming_agg_serialize_to_column(
+                                                in_block, value_columns[i], rows,
+                                                _agg_arena_pool.get()));
                             }
                         } else {
                             std::vector<VectorBufferWriter> value_buffer_writers;
@@ -1031,9 +1032,9 @@ Status AggregationNode::_pre_agg_with_serialized_key(doris::vectorized::Block* i
 
                             for (int i = 0; i != _aggregate_evaluators.size(); ++i) {
                                 SCOPED_TIMER(_serialize_data_timer);
-                                _aggregate_evaluators[i]->streaming_agg_serialize(
+                                RETURN_IF_ERROR(_aggregate_evaluators[i]->streaming_agg_serialize(
                                         in_block, value_buffer_writers[i], rows,
-                                        _agg_arena_pool.get());
+                                        _agg_arena_pool.get()));
                             }
                         }
 
@@ -1059,16 +1060,17 @@ Status AggregationNode::_pre_agg_with_serialized_key(doris::vectorized::Block* i
                         }
                     }
                 }
+                return Status::OK();
             },
-            _agg_data->_aggregated_method_variant);
+            _agg_data->_aggregated_method_variant));
 
     if (!ret_flag) {
         RETURN_IF_CATCH_BAD_ALLOC(_emplace_into_hash_table(_places.data(), key_columns, rows));
 
         for (int i = 0; i < _aggregate_evaluators.size(); ++i) {
-            _aggregate_evaluators[i]->execute_batch_add(in_block, _offsets_of_aggregate_states[i],
-                                                        _places.data(), _agg_arena_pool.get(),
-                                                        _should_expand_hash_table);
+            RETURN_IF_ERROR(_aggregate_evaluators[i]->execute_batch_add(
+                    in_block, _offsets_of_aggregate_states[i], _places.data(),
+                    _agg_arena_pool.get(), _should_expand_hash_table));
         }
     }
 
diff --git a/be/src/vec/exec/vaggregation_node.h b/be/src/vec/exec/vaggregation_node.h
index 23203380ff..7a636227d6 100644
--- a/be/src/vec/exec/vaggregation_node.h
+++ b/be/src/vec/exec/vaggregation_node.h
@@ -106,12 +106,16 @@ struct AggregationMethodSerialized {
     static void insert_key_into_columns(const StringRef& key, MutableColumns& key_columns,
                                         const Sizes&) {
         auto pos = key.data;
-        for (auto& column : key_columns) pos = column->deserialize_and_insert_from_arena(pos);
+        for (auto& column : key_columns) {
+            pos = column->deserialize_and_insert_from_arena(pos);
+        }
     }
 
     static void insert_keys_into_columns(std::vector<StringRef>& keys, MutableColumns& key_columns,
                                          const size_t num_rows, const Sizes&) {
-        for (auto& column : key_columns) column->deserialize_vec(keys, num_rows);
+        for (auto& column : key_columns) {
+            column->deserialize_vec(keys, num_rows);
+        }
     }
 
     void init_once() {
@@ -260,7 +264,7 @@ struct AggregationMethodKeysFixed {
     Iterator iterator;
     bool inited = false;
 
-    AggregationMethodKeysFixed() {}
+    AggregationMethodKeysFixed() = default;
 
     template <typename Other>
     AggregationMethodKeysFixed(const Other& other) : data(other.data) {}
@@ -282,7 +286,9 @@ struct AggregationMethodKeysFixed {
             ColumnUInt8* null_map;
 
             bool column_nullable = false;
-            if constexpr (has_nullable_keys) column_nullable = is_column_nullable(*key_columns[i]);
+            if constexpr (has_nullable_keys) {
+                column_nullable = is_column_nullable(*key_columns[i]);
+            }
 
             /// If we have a nullable column, get its nested column and its null map.
             if (column_nullable) {
@@ -305,9 +311,9 @@ struct AggregationMethodKeysFixed {
                 is_null = val == 1;
             }
 
-            if (has_nullable_keys && is_null)
+            if (has_nullable_keys && is_null) {
                 observed_column->insert_default();
-            else {
+            } else {
                 size_t size = key_sizes[i];
                 observed_column->insert_data(reinterpret_cast<const char*>(&key) + pos, size);
                 pos += size;
@@ -901,16 +907,17 @@ private:
             _find_in_hash_table(_places.data(), key_columns, rows);
 
             for (int i = 0; i < _aggregate_evaluators.size(); ++i) {
-                _aggregate_evaluators[i]->execute_batch_add_selected(
+                RETURN_IF_ERROR(_aggregate_evaluators[i]->execute_batch_add_selected(
                         block, _offsets_of_aggregate_states[i], _places.data(),
-                        _agg_arena_pool.get());
+                        _agg_arena_pool.get()));
             }
         } else {
             _emplace_into_hash_table(_places.data(), key_columns, rows);
 
             for (int i = 0; i < _aggregate_evaluators.size(); ++i) {
-                _aggregate_evaluators[i]->execute_batch_add(block, _offsets_of_aggregate_states[i],
-                                                            _places.data(), _agg_arena_pool.get());
+                RETURN_IF_ERROR(_aggregate_evaluators[i]->execute_batch_add(
+                        block, _offsets_of_aggregate_states[i], _places.data(),
+                        _agg_arena_pool.get()));
             }
 
             if (_should_limit_output) {
@@ -985,9 +992,9 @@ private:
                                                                       rows);
 
                 } else {
-                    _aggregate_evaluators[i]->execute_batch_add_selected(
+                    RETURN_IF_ERROR(_aggregate_evaluators[i]->execute_batch_add_selected(
                             block, _offsets_of_aggregate_states[i], _places.data(),
-                            _agg_arena_pool.get());
+                            _agg_arena_pool.get()));
                 }
             }
         } else {
@@ -1025,9 +1032,9 @@ private:
                                                                       rows);
 
                 } else {
-                    _aggregate_evaluators[i]->execute_batch_add(
+                    RETURN_IF_ERROR(_aggregate_evaluators[i]->execute_batch_add(
                             block, _offsets_of_aggregate_states[i], _places.data(),
-                            _agg_arena_pool.get());
+                            _agg_arena_pool.get()));
                 }
             }
 
diff --git a/be/src/vec/exprs/vectorized_agg_fn.cpp b/be/src/vec/exprs/vectorized_agg_fn.cpp
index 0895eae775..101717d4cd 100644
--- a/be/src/vec/exprs/vectorized_agg_fn.cpp
+++ b/be/src/vec/exprs/vectorized_agg_fn.cpp
@@ -154,38 +154,43 @@ void AggFnEvaluator::destroy(AggregateDataPtr place) {
     _function->destroy(place);
 }
 
-void AggFnEvaluator::execute_single_add(Block* block, AggregateDataPtr place, Arena* arena) {
-    _calc_argment_columns(block);
+Status AggFnEvaluator::execute_single_add(Block* block, AggregateDataPtr place, Arena* arena) {
+    RETURN_IF_ERROR(_calc_argment_columns(block));
     SCOPED_TIMER(_exec_timer);
     _function->add_batch_single_place(block->rows(), place, _agg_columns.data(), arena);
+    return Status::OK();
 }
 
-void AggFnEvaluator::execute_batch_add(Block* block, size_t offset, AggregateDataPtr* places,
-                                       Arena* arena, bool agg_many) {
-    _calc_argment_columns(block);
+Status AggFnEvaluator::execute_batch_add(Block* block, size_t offset, AggregateDataPtr* places,
+                                         Arena* arena, bool agg_many) {
+    RETURN_IF_ERROR(_calc_argment_columns(block));
     SCOPED_TIMER(_exec_timer);
     _function->add_batch(block->rows(), places, offset, _agg_columns.data(), arena, agg_many);
+    return Status::OK();
 }
 
-void AggFnEvaluator::execute_batch_add_selected(Block* block, size_t offset,
-                                                AggregateDataPtr* places, Arena* arena) {
-    _calc_argment_columns(block);
+Status AggFnEvaluator::execute_batch_add_selected(Block* block, size_t offset,
+                                                  AggregateDataPtr* places, Arena* arena) {
+    RETURN_IF_ERROR(_calc_argment_columns(block));
     SCOPED_TIMER(_exec_timer);
     _function->add_batch_selected(block->rows(), places, offset, _agg_columns.data(), arena);
+    return Status::OK();
 }
 
-void AggFnEvaluator::streaming_agg_serialize(Block* block, BufferWritable& buf,
-                                             const size_t num_rows, Arena* arena) {
-    _calc_argment_columns(block);
+Status AggFnEvaluator::streaming_agg_serialize(Block* block, BufferWritable& buf,
+                                               const size_t num_rows, Arena* arena) {
+    RETURN_IF_ERROR(_calc_argment_columns(block));
     SCOPED_TIMER(_exec_timer);
     _function->streaming_agg_serialize(_agg_columns.data(), buf, num_rows, arena);
+    return Status::OK();
 }
 
-void AggFnEvaluator::streaming_agg_serialize_to_column(Block* block, MutableColumnPtr& dst,
-                                                       const size_t num_rows, Arena* arena) {
-    _calc_argment_columns(block);
+Status AggFnEvaluator::streaming_agg_serialize_to_column(Block* block, MutableColumnPtr& dst,
+                                                         const size_t num_rows, Arena* arena) {
+    RETURN_IF_ERROR(_calc_argment_columns(block));
     SCOPED_TIMER(_exec_timer);
     _function->streaming_agg_serialize_to_column(_agg_columns.data(), dst, num_rows, arena);
+    return Status::OK();
 }
 
 void AggFnEvaluator::insert_result_info(AggregateDataPtr place, IColumn* column) {
@@ -220,19 +225,20 @@ std::string AggFnEvaluator::debug_string() const {
     return out.str();
 }
 
-void AggFnEvaluator::_calc_argment_columns(Block* block) {
+Status AggFnEvaluator::_calc_argment_columns(Block* block) {
     SCOPED_TIMER(_expr_timer);
     _agg_columns.resize(_input_exprs_ctxs.size());
     int column_ids[_input_exprs_ctxs.size()];
     for (int i = 0; i < _input_exprs_ctxs.size(); ++i) {
         int column_id = -1;
-        _input_exprs_ctxs[i]->execute(block, &column_id);
+        RETURN_IF_ERROR(_input_exprs_ctxs[i]->execute(block, &column_id));
         column_ids[i] = column_id;
     }
     materialize_block_inplace(*block, column_ids, column_ids + _input_exprs_ctxs.size());
     for (int i = 0; i < _input_exprs_ctxs.size(); ++i) {
         _agg_columns[i] = block->get_by_position(column_ids[i]).column.get();
     }
+    return Status::OK();
 }
 
 } // namespace doris::vectorized
diff --git a/be/src/vec/exprs/vectorized_agg_fn.h b/be/src/vec/exprs/vectorized_agg_fn.h
index fa025edec1..1fe6214654 100644
--- a/be/src/vec/exprs/vectorized_agg_fn.h
+++ b/be/src/vec/exprs/vectorized_agg_fn.h
@@ -53,19 +53,19 @@ public:
     void destroy(AggregateDataPtr place);
 
     // agg_function
-    void execute_single_add(Block* block, AggregateDataPtr place, Arena* arena = nullptr);
+    Status execute_single_add(Block* block, AggregateDataPtr place, Arena* arena = nullptr);
 
-    void execute_batch_add(Block* block, size_t offset, AggregateDataPtr* places,
-                           Arena* arena = nullptr, bool agg_many = false);
+    Status execute_batch_add(Block* block, size_t offset, AggregateDataPtr* places,
+                             Arena* arena = nullptr, bool agg_many = false);
 
-    void execute_batch_add_selected(Block* block, size_t offset, AggregateDataPtr* places,
-                                    Arena* arena = nullptr);
+    Status execute_batch_add_selected(Block* block, size_t offset, AggregateDataPtr* places,
+                                      Arena* arena = nullptr);
 
-    void streaming_agg_serialize(Block* block, BufferWritable& buf, const size_t num_rows,
-                                 Arena* arena);
+    Status streaming_agg_serialize(Block* block, BufferWritable& buf, const size_t num_rows,
+                                   Arena* arena);
 
-    void streaming_agg_serialize_to_column(Block* block, MutableColumnPtr& dst,
-                                           const size_t num_rows, Arena* arena);
+    Status streaming_agg_serialize_to_column(Block* block, MutableColumnPtr& dst,
+                                             const size_t num_rows, Arena* arena);
 
     void insert_result_info(AggregateDataPtr place, IColumn* column);
 
@@ -89,7 +89,7 @@ private:
 
     AggFnEvaluator(const TExprNode& desc);
 
-    void _calc_argment_columns(Block* block);
+    Status _calc_argment_columns(Block* block);
 
     DataTypes _argument_types_with_sort;
     DataTypes _real_argument_types;
diff --git a/regression-test/data/query_p0/sql_functions/bitmap_functions/test_bitmap_function.out b/regression-test/data/query_p0/sql_functions/bitmap_functions/test_bitmap_function.out
index ed771ea325..d7b68d929b 100644
--- a/regression-test/data/query_p0/sql_functions/bitmap_functions/test_bitmap_function.out
+++ b/regression-test/data/query_p0/sql_functions/bitmap_functions/test_bitmap_function.out
@@ -271,3 +271,6 @@ false
 -- !sql --
 20221103
 
+-- !sql --
+\N
+
diff --git a/regression-test/suites/query_p0/sql_functions/bitmap_functions/test_bitmap_function.groovy b/regression-test/suites/query_p0/sql_functions/bitmap_functions/test_bitmap_function.groovy
index 8669f961a4..9a61f874fc 100644
--- a/regression-test/suites/query_p0/sql_functions/bitmap_functions/test_bitmap_function.groovy
+++ b/regression-test/suites/query_p0/sql_functions/bitmap_functions/test_bitmap_function.groovy
@@ -191,4 +191,20 @@ suite("test_bitmap_function") {
     qt_sql """ select bitmap_to_string(sub_bitmap(bitmap_from_string('1'), 0, 3)) value;  """
     qt_sql """ select bitmap_to_string(bitmap_subset_limit(bitmap_from_string('100'), 0, 3)) value;  """
     qt_sql """ select bitmap_to_string(bitmap_subset_in_range(bitmap_from_string('20221103'), 0, 20221104)) date_list_bitmap;  """
+
+    sql "drop table if exists d_table;"
+    sql """
+        create table d_table (
+            k1 int null,
+            k2 int not null,
+            k3 bigint null,
+            k4 varchar(100) null
+        )
+        duplicate key (k1,k2,k3)
+        distributed BY hash(k1) buckets 3
+        properties("replication_num" = "1");
+    """
+    sql "insert into d_table select -4,-4,-4,'d';"
+    try_sql "select bitmap_union(to_bitmap_with_check(k2)) from d_table;"
+    qt_sql "select bitmap_union(to_bitmap(k2)) from d_table;"
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 02/05: [refactor](rpc fn) decouple vectorized remote function from row-based one (#15871)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 776255018e49de42f6c9d5d003076903880ee7ba
Author: Gabriel <ga...@gmail.com>
AuthorDate: Fri Jan 13 09:21:33 2023 +0800

    [refactor](rpc fn) decouple vectorized remote function from row-based one (#15871)
---
 be/src/vec/functions/function_rpc.cpp | 505 +++++++++++++++++++++++++++++++++-
 be/src/vec/functions/function_rpc.h   |  42 ++-
 2 files changed, 538 insertions(+), 9 deletions(-)

diff --git a/be/src/vec/functions/function_rpc.cpp b/be/src/vec/functions/function_rpc.cpp
index 97d31710ca..2dfcfce8a7 100644
--- a/be/src/vec/functions/function_rpc.cpp
+++ b/be/src/vec/functions/function_rpc.cpp
@@ -21,17 +21,516 @@
 
 #include <memory>
 
-#include "exprs/rpc_fn.h"
+#include "gen_cpp/Exprs_types.h"
+#include "json2pb/json_to_pb.h"
+#include "json2pb/pb_to_json.h"
 
 namespace doris::vectorized {
+
+RPCFnImpl::RPCFnImpl(const TFunction& fn) : _fn(fn) {
+    _client = ExecEnv::GetInstance()->brpc_function_client_cache()->get_client(_server_addr);
+    _function_name = _fn.scalar_fn.symbol;
+    _server_addr = _fn.hdfs_location;
+    _signature = fmt::format("{}: [{}/{}]", _fn.name.function_name, _fn.hdfs_location,
+                             _fn.scalar_fn.symbol);
+}
+
+void RPCFnImpl::_convert_nullable_col_to_pvalue(const ColumnPtr& column,
+                                                const DataTypePtr& data_type,
+                                                const ColumnUInt8& null_col, PValues* arg,
+                                                int start, int end) {
+    int row_count = end - start;
+    if (column->has_null(row_count)) {
+        auto* null_map = arg->mutable_null_map();
+        null_map->Reserve(row_count);
+        const auto* col = check_and_get_column<ColumnUInt8>(null_col);
+        auto& data = col->get_data();
+        null_map->Add(data.begin() + start, data.begin() + end);
+        this->_convert_col_to_pvalue<true>(column, data_type, arg, start, end);
+    } else {
+        this->_convert_col_to_pvalue<false>(column, data_type, arg, start, end);
+    }
+}
+
+template <bool nullable>
+void RPCFnImpl::_convert_col_to_pvalue(const ColumnPtr& column, const DataTypePtr& data_type,
+                                       PValues* arg, int start, int end) {
+    int row_count = end - start;
+    PGenericType* ptype = arg->mutable_type();
+    switch (data_type->get_type_id()) {
+    case TypeIndex::UInt8: {
+        ptype->set_id(PGenericType::UINT8);
+        auto* values = arg->mutable_bool_value();
+        values->Reserve(row_count);
+        const auto* col = check_and_get_column<ColumnUInt8>(column);
+        auto& data = col->get_data();
+        values->Add(data.begin() + start, data.begin() + end);
+        break;
+    }
+    case TypeIndex::UInt16: {
+        ptype->set_id(PGenericType::UINT16);
+        auto* values = arg->mutable_uint32_value();
+        values->Reserve(row_count);
+        const auto* col = check_and_get_column<ColumnUInt16>(column);
+        auto& data = col->get_data();
+        values->Add(data.begin() + start, data.begin() + end);
+        break;
+    }
+    case TypeIndex::UInt32: {
+        ptype->set_id(PGenericType::UINT32);
+        auto* values = arg->mutable_uint32_value();
+        values->Reserve(row_count);
+        const auto* col = check_and_get_column<ColumnUInt32>(column);
+        auto& data = col->get_data();
+        values->Add(data.begin() + start, data.begin() + end);
+        break;
+    }
+    case TypeIndex::UInt64: {
+        ptype->set_id(PGenericType::UINT64);
+        auto* values = arg->mutable_uint64_value();
+        values->Reserve(row_count);
+        const auto* col = check_and_get_column<ColumnUInt64>(column);
+        auto& data = col->get_data();
+        values->Add(data.begin() + start, data.begin() + end);
+        break;
+    }
+    case TypeIndex::UInt128: {
+        ptype->set_id(PGenericType::UINT128);
+        arg->mutable_bytes_value()->Reserve(row_count);
+        for (size_t row_num = start; row_num < end; ++row_num) {
+            if constexpr (nullable) {
+                if (column->is_null_at(row_num)) {
+                    arg->add_bytes_value(nullptr);
+                } else {
+                    StringRef data = column->get_data_at(row_num);
+                    arg->add_bytes_value(data.data, data.size);
+                }
+            } else {
+                StringRef data = column->get_data_at(row_num);
+                arg->add_bytes_value(data.data, data.size);
+            }
+        }
+        break;
+    }
+    case TypeIndex::Int8: {
+        ptype->set_id(PGenericType::INT8);
+        auto* values = arg->mutable_int32_value();
+        values->Reserve(row_count);
+        const auto* col = check_and_get_column<ColumnInt8>(column);
+        auto& data = col->get_data();
+        values->Add(data.begin() + start, data.begin() + end);
+        break;
+    }
+    case TypeIndex::Int16: {
+        ptype->set_id(PGenericType::INT16);
+        auto* values = arg->mutable_int32_value();
+        values->Reserve(row_count);
+        const auto* col = check_and_get_column<ColumnInt16>(column);
+        auto& data = col->get_data();
+        values->Add(data.begin() + start, data.begin() + end);
+        break;
+    }
+    case TypeIndex::Int32: {
+        ptype->set_id(PGenericType::INT32);
+        auto* values = arg->mutable_int32_value();
+        values->Reserve(row_count);
+        const auto* col = check_and_get_column<ColumnInt32>(column);
+        auto& data = col->get_data();
+        values->Add(data.begin() + start, data.begin() + end);
+        break;
+    }
+    case TypeIndex::Int64: {
+        ptype->set_id(PGenericType::INT64);
+        auto* values = arg->mutable_int64_value();
+        values->Reserve(row_count);
+        const auto* col = check_and_get_column<ColumnInt64>(column);
+        auto& data = col->get_data();
+        values->Add(data.begin() + start, data.begin() + end);
+        break;
+    }
+    case TypeIndex::Int128: {
+        ptype->set_id(PGenericType::INT128);
+        arg->mutable_bytes_value()->Reserve(row_count);
+        for (size_t row_num = start; row_num < end; ++row_num) {
+            if constexpr (nullable) {
+                if (column->is_null_at(row_num)) {
+                    arg->add_bytes_value(nullptr);
+                } else {
+                    StringRef data = column->get_data_at(row_num);
+                    arg->add_bytes_value(data.data, data.size);
+                }
+            } else {
+                StringRef data = column->get_data_at(row_num);
+                arg->add_bytes_value(data.data, data.size);
+            }
+        }
+        break;
+    }
+    case TypeIndex::Float32: {
+        ptype->set_id(PGenericType::FLOAT);
+        auto* values = arg->mutable_float_value();
+        values->Reserve(row_count);
+        const auto* col = check_and_get_column<ColumnFloat32>(column);
+        auto& data = col->get_data();
+        values->Add(data.begin() + start, data.begin() + end);
+        break;
+    }
+
+    case TypeIndex::Float64: {
+        ptype->set_id(PGenericType::DOUBLE);
+        auto* values = arg->mutable_double_value();
+        values->Reserve(row_count);
+        const auto* col = check_and_get_column<ColumnFloat64>(column);
+        auto& data = col->get_data();
+        values->Add(data.begin() + start, data.begin() + end);
+        break;
+    }
+    case TypeIndex::String: {
+        ptype->set_id(PGenericType::STRING);
+        arg->mutable_bytes_value()->Reserve(row_count);
+        for (size_t row_num = start; row_num < end; ++row_num) {
+            if constexpr (nullable) {
+                if (column->is_null_at(row_num)) {
+                    arg->add_string_value(nullptr);
+                } else {
+                    StringRef data = column->get_data_at(row_num);
+                    arg->add_string_value(data.to_string());
+                }
+            } else {
+                StringRef data = column->get_data_at(row_num);
+                arg->add_string_value(data.to_string());
+            }
+        }
+        break;
+    }
+    case TypeIndex::Date: {
+        ptype->set_id(PGenericType::DATE);
+        arg->mutable_datetime_value()->Reserve(row_count);
+        for (size_t row_num = start; row_num < end; ++row_num) {
+            PDateTime* date_time = arg->add_datetime_value();
+            if constexpr (nullable) {
+                if (!column->is_null_at(row_num)) {
+                    VecDateTimeValue v =
+                            VecDateTimeValue::create_from_olap_date(column->get_int(row_num));
+                    date_time->set_day(v.day());
+                    date_time->set_month(v.month());
+                    date_time->set_year(v.year());
+                }
+            } else {
+                VecDateTimeValue v =
+                        VecDateTimeValue::create_from_olap_date(column->get_int(row_num));
+                date_time->set_day(v.day());
+                date_time->set_month(v.month());
+                date_time->set_year(v.year());
+            }
+        }
+        break;
+    }
+    case TypeIndex::DateTime: {
+        ptype->set_id(PGenericType::DATETIME);
+        arg->mutable_datetime_value()->Reserve(row_count);
+        for (size_t row_num = start; row_num < end; ++row_num) {
+            PDateTime* date_time = arg->add_datetime_value();
+            if constexpr (nullable) {
+                if (!column->is_null_at(row_num)) {
+                    VecDateTimeValue v =
+                            VecDateTimeValue::create_from_olap_datetime(column->get_int(row_num));
+                    date_time->set_day(v.day());
+                    date_time->set_month(v.month());
+                    date_time->set_year(v.year());
+                    date_time->set_hour(v.hour());
+                    date_time->set_minute(v.minute());
+                    date_time->set_second(v.second());
+                }
+            } else {
+                VecDateTimeValue v =
+                        VecDateTimeValue::create_from_olap_datetime(column->get_int(row_num));
+                date_time->set_day(v.day());
+                date_time->set_month(v.month());
+                date_time->set_year(v.year());
+                date_time->set_hour(v.hour());
+                date_time->set_minute(v.minute());
+                date_time->set_second(v.second());
+            }
+        }
+        break;
+    }
+    case TypeIndex::BitMap: {
+        ptype->set_id(PGenericType::BITMAP);
+        arg->mutable_bytes_value()->Reserve(row_count);
+        for (size_t row_num = start; row_num < end; ++row_num) {
+            if constexpr (nullable) {
+                if (column->is_null_at(row_num)) {
+                    arg->add_bytes_value(nullptr);
+                } else {
+                    StringRef data = column->get_data_at(row_num);
+                    arg->add_bytes_value(data.data, data.size);
+                }
+            } else {
+                StringRef data = column->get_data_at(row_num);
+                arg->add_bytes_value(data.data, data.size);
+            }
+        }
+        break;
+    }
+    case TypeIndex::HLL: {
+        ptype->set_id(PGenericType::HLL);
+        arg->mutable_bytes_value()->Reserve(row_count);
+        for (size_t row_num = start; row_num < end; ++row_num) {
+            if constexpr (nullable) {
+                if (column->is_null_at(row_num)) {
+                    arg->add_bytes_value(nullptr);
+                } else {
+                    StringRef data = column->get_data_at(row_num);
+                    arg->add_bytes_value(data.data, data.size);
+                }
+            } else {
+                StringRef data = column->get_data_at(row_num);
+                arg->add_bytes_value(data.data, data.size);
+            }
+        }
+        break;
+    }
+    default:
+        LOG(INFO) << "unknown type: " << data_type->get_name();
+        ptype->set_id(PGenericType::UNKNOWN);
+        break;
+    }
+}
+
+template <bool nullable>
+void RPCFnImpl::_convert_to_column(MutableColumnPtr& column, const PValues& result) {
+    switch (result.type().id()) {
+    case PGenericType::UINT8: {
+        column->reserve(result.uint32_value_size());
+        column->resize(result.uint32_value_size());
+        auto& data = reinterpret_cast<ColumnUInt8*>(column.get())->get_data();
+        for (int i = 0; i < result.uint32_value_size(); ++i) {
+            data[i] = result.uint32_value(i);
+        }
+        break;
+    }
+    case PGenericType::UINT16: {
+        column->reserve(result.uint32_value_size());
+        column->resize(result.uint32_value_size());
+        auto& data = reinterpret_cast<ColumnUInt16*>(column.get())->get_data();
+        for (int i = 0; i < result.uint32_value_size(); ++i) {
+            data[i] = result.uint32_value(i);
+        }
+        break;
+    }
+    case PGenericType::UINT32: {
+        column->reserve(result.uint32_value_size());
+        column->resize(result.uint32_value_size());
+        auto& data = reinterpret_cast<ColumnUInt32*>(column.get())->get_data();
+        for (int i = 0; i < result.uint32_value_size(); ++i) {
+            data[i] = result.uint32_value(i);
+        }
+        break;
+    }
+    case PGenericType::UINT64: {
+        column->reserve(result.uint64_value_size());
+        column->resize(result.uint64_value_size());
+        auto& data = reinterpret_cast<ColumnUInt64*>(column.get())->get_data();
+        for (int i = 0; i < result.uint64_value_size(); ++i) {
+            data[i] = result.uint64_value(i);
+        }
+        break;
+    }
+    case PGenericType::INT8: {
+        column->reserve(result.int32_value_size());
+        column->resize(result.int32_value_size());
+        auto& data = reinterpret_cast<ColumnInt16*>(column.get())->get_data();
+        for (int i = 0; i < result.int32_value_size(); ++i) {
+            data[i] = result.int32_value(i);
+        }
+        break;
+    }
+    case PGenericType::INT16: {
+        column->reserve(result.int32_value_size());
+        column->resize(result.int32_value_size());
+        auto& data = reinterpret_cast<ColumnInt16*>(column.get())->get_data();
+        for (int i = 0; i < result.int32_value_size(); ++i) {
+            data[i] = result.int32_value(i);
+        }
+        break;
+    }
+    case PGenericType::INT32: {
+        column->reserve(result.int32_value_size());
+        column->resize(result.int32_value_size());
+        auto& data = reinterpret_cast<ColumnInt32*>(column.get())->get_data();
+        for (int i = 0; i < result.int32_value_size(); ++i) {
+            data[i] = result.int32_value(i);
+        }
+        break;
+    }
+    case PGenericType::INT64: {
+        column->reserve(result.int64_value_size());
+        column->resize(result.int64_value_size());
+        auto& data = reinterpret_cast<ColumnInt64*>(column.get())->get_data();
+        for (int i = 0; i < result.int64_value_size(); ++i) {
+            data[i] = result.int64_value(i);
+        }
+        break;
+    }
+    case PGenericType::DATE:
+    case PGenericType::DATETIME: {
+        column->reserve(result.datetime_value_size());
+        column->resize(result.datetime_value_size());
+        auto& data = reinterpret_cast<ColumnInt64*>(column.get())->get_data();
+        for (int i = 0; i < result.datetime_value_size(); ++i) {
+            VecDateTimeValue v;
+            PDateTime pv = result.datetime_value(i);
+            v.set_time(pv.year(), pv.month(), pv.day(), pv.hour(), pv.minute(), pv.minute());
+            data[i] = binary_cast<VecDateTimeValue, Int64>(v);
+        }
+        break;
+    }
+    case PGenericType::FLOAT: {
+        column->reserve(result.float_value_size());
+        column->resize(result.float_value_size());
+        auto& data = reinterpret_cast<ColumnFloat32*>(column.get())->get_data();
+        for (int i = 0; i < result.float_value_size(); ++i) {
+            data[i] = result.float_value(i);
+        }
+        break;
+    }
+    case PGenericType::DOUBLE: {
+        column->reserve(result.double_value_size());
+        column->resize(result.double_value_size());
+        auto& data = reinterpret_cast<ColumnFloat64*>(column.get())->get_data();
+        for (int i = 0; i < result.double_value_size(); ++i) {
+            data[i] = result.double_value(i);
+        }
+        break;
+    }
+    case PGenericType::INT128: {
+        column->reserve(result.bytes_value_size());
+        column->resize(result.bytes_value_size());
+        auto& data = reinterpret_cast<ColumnInt128*>(column.get())->get_data();
+        for (int i = 0; i < result.bytes_value_size(); ++i) {
+            data[i] = *(int128_t*)(result.bytes_value(i).c_str());
+        }
+        break;
+    }
+    case PGenericType::STRING: {
+        column->reserve(result.string_value_size());
+        for (int i = 0; i < result.string_value_size(); ++i) {
+            column->insert_data(result.string_value(i).c_str(), result.string_value(i).size());
+        }
+        break;
+    }
+    case PGenericType::DECIMAL128: {
+        column->reserve(result.bytes_value_size());
+        column->resize(result.bytes_value_size());
+        auto& data = reinterpret_cast<ColumnDecimal128*>(column.get())->get_data();
+        for (int i = 0; i < result.bytes_value_size(); ++i) {
+            data[i] = *(int128_t*)(result.bytes_value(i).c_str());
+        }
+        break;
+    }
+    case PGenericType::BITMAP: {
+        column->reserve(result.bytes_value_size());
+        for (int i = 0; i < result.bytes_value_size(); ++i) {
+            column->insert_data(result.bytes_value(i).c_str(), result.bytes_value(i).size());
+        }
+        break;
+    }
+    case PGenericType::HLL: {
+        column->reserve(result.bytes_value_size());
+        for (int i = 0; i < result.bytes_value_size(); ++i) {
+            column->insert_data(result.bytes_value(i).c_str(), result.bytes_value(i).size());
+        }
+        break;
+    }
+    default: {
+        LOG(WARNING) << "unknown PGenericType: " << result.type().DebugString();
+        break;
+    }
+    }
+}
+
+Status RPCFnImpl::vec_call(FunctionContext* context, Block& block, const ColumnNumbers& arguments,
+                           size_t result, size_t input_rows_count) {
+    PFunctionCallRequest request;
+    PFunctionCallResponse response;
+    request.set_function_name(_function_name);
+    _convert_block_to_proto(block, arguments, input_rows_count, &request);
+    brpc::Controller cntl;
+    _client->fn_call(&cntl, &request, &response, nullptr);
+    if (cntl.Failed()) {
+        return Status::InternalError("call to rpc function {} failed: {}", _signature,
+                                     cntl.ErrorText());
+    }
+    if (!response.has_status() || response.result_size() == 0) {
+        return Status::InternalError("call rpc function {} failed: status or result is not set.",
+                                     _signature);
+    }
+    if (response.status().status_code() != 0) {
+        return Status::InternalError("call to rpc function {} failed: {}", _signature,
+                                     response.status().DebugString());
+    }
+    _convert_to_block(block, response.result(0), result);
+    return Status::OK();
+}
+
+void RPCFnImpl::_convert_block_to_proto(Block& block, const ColumnNumbers& arguments,
+                                        size_t input_rows_count, PFunctionCallRequest* request) {
+    size_t row_count = std::min(block.rows(), input_rows_count);
+    for (size_t col_idx : arguments) {
+        PValues* arg = request->add_args();
+        ColumnWithTypeAndName& column = block.get_by_position(col_idx);
+        arg->set_has_null(column.column->has_null(row_count));
+        auto col = column.column->convert_to_full_column_if_const();
+        if (auto* nullable = check_and_get_column<const ColumnNullable>(*col)) {
+            auto data_col = nullable->get_nested_column_ptr();
+            auto& null_col = nullable->get_null_map_column();
+            auto data_type = std::reinterpret_pointer_cast<const DataTypeNullable>(column.type);
+            _convert_nullable_col_to_pvalue(data_col->convert_to_full_column_if_const(),
+                                            data_type->get_nested_type(), null_col, arg, 0,
+                                            row_count);
+        } else {
+            _convert_col_to_pvalue<false>(col, column.type, arg, 0, row_count);
+        }
+    }
+}
+
+void RPCFnImpl::_convert_to_block(Block& block, const PValues& result, size_t pos) {
+    auto data_type = block.get_data_type(pos);
+    if (data_type->is_nullable()) {
+        auto null_type = std::reinterpret_pointer_cast<const DataTypeNullable>(data_type);
+        auto data_col = null_type->get_nested_type()->create_column();
+        _convert_to_column<true>(data_col, result);
+        auto null_col = ColumnUInt8::create(data_col->size(), 0);
+        auto& null_map_data = null_col->get_data();
+        null_col->reserve(data_col->size());
+        null_col->resize(data_col->size());
+        if (result.has_null()) {
+            for (int i = 0; i < data_col->size(); ++i) {
+                null_map_data[i] = result.null_map(i);
+            }
+        } else {
+            for (int i = 0; i < data_col->size(); ++i) {
+                null_map_data[i] = false;
+            }
+        }
+        block.replace_by_position(pos,
+                                  ColumnNullable::create(std::move(data_col), std::move(null_col)));
+    } else {
+        auto column = data_type->create_column();
+        _convert_to_column<false>(column, result);
+        block.replace_by_position(pos, std::move(column));
+    }
+}
+
 FunctionRPC::FunctionRPC(const TFunction& fn, const DataTypes& argument_types,
                          const DataTypePtr& return_type)
         : _argument_types(argument_types), _return_type(return_type), _tfn(fn) {}
 
 Status FunctionRPC::prepare(FunctionContext* context, FunctionContext::FunctionStateScope scope) {
-    _fn = std::make_unique<RPCFn>(_tfn, false);
+    _fn = std::make_unique<RPCFnImpl>(_tfn);
 
-    if (!_fn->avliable()) {
+    if (!_fn->available()) {
         return Status::InternalError("rpc env init error");
     }
     return Status::OK();
diff --git a/be/src/vec/functions/function_rpc.h b/be/src/vec/functions/function_rpc.h
index a4037958dd..56d953744f 100644
--- a/be/src/vec/functions/function_rpc.h
+++ b/be/src/vec/functions/function_rpc.h
@@ -17,12 +17,43 @@
 
 #pragma once
 
+#include "gen_cpp/function_service.pb.h"
+#include "util/brpc_client_cache.h"
 #include "vec/functions/function.h"
 
-namespace doris {
-class RPCFn;
+namespace doris::vectorized {
+
+class RPCFnImpl {
+public:
+    RPCFnImpl(const TFunction& fn);
+    ~RPCFnImpl() = default;
+    Status vec_call(FunctionContext* context, vectorized::Block& block,
+                    const std::vector<size_t>& arguments, size_t result, size_t input_rows_count);
+    bool available() { return _client != nullptr; }
+
+private:
+    void _convert_block_to_proto(vectorized::Block& block,
+                                 const vectorized::ColumnNumbers& arguments,
+                                 size_t input_rows_count, PFunctionCallRequest* request);
+    void _convert_to_block(vectorized::Block& block, const PValues& result, size_t pos);
+    void _convert_nullable_col_to_pvalue(const vectorized::ColumnPtr& column,
+                                         const vectorized::DataTypePtr& data_type,
+                                         const vectorized::ColumnUInt8& null_col, PValues* arg,
+                                         int start, int end);
+    template <bool nullable>
+    void _convert_col_to_pvalue(const vectorized::ColumnPtr& column,
+                                const vectorized::DataTypePtr& data_type, PValues* arg, int start,
+                                int end);
+    template <bool nullable>
+    void _convert_to_column(vectorized::MutableColumnPtr& column, const PValues& result);
+
+    std::shared_ptr<PFunctionService_Stub> _client;
+    std::string _function_name;
+    std::string _server_addr;
+    std::string _signature;
+    TFunction _fn;
+};
 
-namespace vectorized {
 class FunctionRPC : public IFunctionBase {
 public:
     FunctionRPC(const TFunction& fn, const DataTypes& argument_types,
@@ -64,8 +95,7 @@ private:
     DataTypes _argument_types;
     DataTypePtr _return_type;
     TFunction _tfn;
-    std::unique_ptr<RPCFn> _fn;
+    std::unique_ptr<RPCFnImpl> _fn;
 };
 
-} // namespace vectorized
-} // namespace doris
+} // namespace doris::vectorized


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 01/05: [opt](planner) return bigint literal when cast date literal to bigint type (#15613)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit f59626a32ebd9f60207a2daa358e7948a93e2a01
Author: minghong <en...@gmail.com>
AuthorDate: Fri Jan 13 12:58:04 2023 +0800

    [opt](planner) return bigint literal when cast date literal to bigint type (#15613)
---
 .../src/main/java/org/apache/doris/analysis/DateLiteral.java     | 9 +++++++--
 1 file changed, 7 insertions(+), 2 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java
index 2377351822..229769e6f2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java
@@ -647,8 +647,13 @@ public class DateLiteral extends LiteralExpr {
             }
         } else if (targetType.isStringType()) {
             return new StringLiteral(getStringValue());
-        } else if (Type.isImplicitlyCastable(this.type, targetType, true)) {
-            return new CastExpr(targetType, this);
+        } else if (targetType.isBigIntType()) {
+            long value = getYear() * 1000 + getMonth() * 100 + getDay();
+            return new IntLiteral(value, Type.BIGINT);
+        } else {
+            if (Type.isImplicitlyCastable(this.type, targetType, true)) {
+                return new CastExpr(targetType, this);
+            }
         }
         Preconditions.checkState(false);
         return this;


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 04/05: [enhancement](load) change the publish version log to VLOG_CRITICAL (#15673)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 521bed9ca122447baef4b61cdd7003f57f1b61c9
Author: zhannngchen <48...@users.noreply.github.com>
AuthorDate: Mon Jan 16 16:22:33 2023 +0800

    [enhancement](load) change the publish version log to VLOG_CRITICAL (#15673)
---
 be/src/olap/task/engine_publish_version_task.cpp | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)

diff --git a/be/src/olap/task/engine_publish_version_task.cpp b/be/src/olap/task/engine_publish_version_task.cpp
index 4e786a0455..f937645c0a 100644
--- a/be/src/olap/task/engine_publish_version_task.cpp
+++ b/be/src/olap/task/engine_publish_version_task.cpp
@@ -181,9 +181,11 @@ Status EnginePublishVersionTask::finish() {
         }
     }
 
-    LOG(INFO) << "finish to publish version on transaction."
-              << "transaction_id=" << transaction_id << ", cost(us): " << watch.get_elapse_time_us()
-              << ", error_tablet_size=" << _error_tablet_ids->size() << ", res=" << res.to_string();
+    VLOG_CRITICAL << "finish to publish version on transaction."
+                  << "transaction_id=" << transaction_id
+                  << ", cost(us): " << watch.get_elapse_time_us()
+                  << ", error_tablet_size=" << _error_tablet_ids->size()
+                  << ", res=" << res.to_string();
     return res;
 }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 05/05: [Compile](lzo) fix lzo decompressor compiler error (#15956)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 768ff43c65226a966a2734a37b74de7ff9a33d85
Author: HappenLee <ha...@hotmail.com>
AuthorDate: Tue Jan 17 09:56:07 2023 +0800

    [Compile](lzo) fix lzo decompressor compiler error (#15956)
---
 be/src/exec/decompressor.h | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/be/src/exec/decompressor.h b/be/src/exec/decompressor.h
index 0b10b874fc..a9e6ef7b02 100644
--- a/be/src/exec/decompressor.h
+++ b/be/src/exec/decompressor.h
@@ -150,7 +150,7 @@ public:
 private:
     friend class Decompressor;
     LzopDecompressor()
-            : Decompressor(CompressType::LZOP), _header_info({0}), _is_header_loaded(false) {}
+            : Decompressor(CompressType::LZOP), _header_info(), _is_header_loaded(false) {}
     Status init() override;
 
 private:


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org