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

yiguolei pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/master by this push:
     new a77921d767 [refactor](typesystem) remove unused rpc common file and 
using function rpc (#18270)
a77921d767 is described below

commit a77921d767809715c880da19b80b9da46eecc86d
Author: yiguolei <676222...@qq.com>
AuthorDate: Fri Mar 31 18:13:25 2023 +0800

    [refactor](typesystem) remove unused rpc common file and using function rpc 
(#18270)
    
    rpc common is duplicate, all its method is included in function rpc. So 
that I remove it.
    get_field_type is never used, remove it.
    ---------
    
    Co-authored-by: yiguolei <yiguo...@gmail.com>
---
 be/src/exprs/CMakeLists.txt                        |   1 -
 be/src/exprs/rpc_fn_comm.cpp                       |  89 ----
 be/src/exprs/rpc_fn_comm.h                         | 482 ---------------------
 .../aggregate_functions/aggregate_function_rpc.h   |  13 +-
 be/src/vec/common/schema_util.cpp                  |  41 --
 be/src/vec/functions/function_rpc.cpp              |  24 +-
 be/src/vec/functions/function_rpc.h                |  17 +-
 7 files changed, 28 insertions(+), 639 deletions(-)

diff --git a/be/src/exprs/CMakeLists.txt b/be/src/exprs/CMakeLists.txt
index a5a7d969bb..7903d07048 100644
--- a/be/src/exprs/CMakeLists.txt
+++ b/be/src/exprs/CMakeLists.txt
@@ -27,7 +27,6 @@ add_library(Exprs
   runtime_filter.cpp
   runtime_filter_rpc.cpp
   math_functions.cpp
-  rpc_fn_comm.cpp
   string_functions.cpp
   json_functions.cpp
 )
diff --git a/be/src/exprs/rpc_fn_comm.cpp b/be/src/exprs/rpc_fn_comm.cpp
deleted file mode 100644
index 6b548ad3c0..0000000000
--- a/be/src/exprs/rpc_fn_comm.cpp
+++ /dev/null
@@ -1,89 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-
-#include "exprs/rpc_fn_comm.h"
-namespace doris::vectorized {
-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) {
-    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 = 
vectorized::check_and_get_column<vectorized::ColumnUInt8>(null_col);
-        auto& data = col->get_data();
-        null_map->Add(data.begin() + start, data.begin() + end);
-        convert_col_to_pvalue<true>(column, data_type, arg, start, end);
-    } else {
-        convert_col_to_pvalue<false>(column, data_type, arg, start, end);
-    }
-}
-
-void convert_block_to_proto(vectorized::Block& block, const 
vectorized::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();
-        vectorized::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 =
-                    vectorized::check_and_get_column<const 
vectorized::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 
vectorized::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 convert_to_block(vectorized::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 
vectorized::DataTypeNullable>(data_type);
-        auto data_col = null_type->get_nested_type()->create_column();
-        convert_to_column<true>(data_col, result);
-        auto null_col = vectorized::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, vectorized::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));
-    }
-}
-
-} // namespace doris::vectorized
\ No newline at end of file
diff --git a/be/src/exprs/rpc_fn_comm.h b/be/src/exprs/rpc_fn_comm.h
deleted file mode 100644
index 1352c3d06c..0000000000
--- a/be/src/exprs/rpc_fn_comm.h
+++ /dev/null
@@ -1,482 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-
-#pragma once
-#include <cstdint>
-#include <memory>
-
-#include "common/status.h"
-#include "gen_cpp/Exprs_types.h"
-#include "json2pb/json_to_pb.h"
-#include "json2pb/pb_to_json.h"
-#include "runtime/exec_env.h"
-#include "runtime/user_function_cache.h"
-#include "util/brpc_client_cache.h"
-#include "util/jni-util.h"
-#include "vec/aggregate_functions/aggregate_function.h"
-#include "vec/columns/column_string.h"
-#include "vec/columns/column_vector.h"
-#include "vec/columns/columns_number.h"
-#include "vec/common/string_ref.h"
-#include "vec/core/block.h"
-#include "vec/core/column_numbers.h"
-#include "vec/core/field.h"
-#include "vec/core/types.h"
-#include "vec/data_types/data_type_string.h"
-#include "vec/io/io_helper.h"
-
-namespace doris::vectorized {
-
-template <bool nullable>
-void convert_col_to_pvalue(const vectorized::ColumnPtr& column,
-                           const vectorized::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 vectorized::TypeIndex::UInt8: {
-        ptype->set_id(PGenericType::UINT8);
-        auto* values = arg->mutable_bool_value();
-        values->Reserve(row_count);
-        const auto* col = 
vectorized::check_and_get_column<vectorized::ColumnUInt8>(column);
-        auto& data = col->get_data();
-        values->Add(data.begin() + start, data.begin() + end);
-        break;
-    }
-    case vectorized::TypeIndex::UInt16: {
-        ptype->set_id(PGenericType::UINT16);
-        auto* values = arg->mutable_uint32_value();
-        values->Reserve(row_count);
-        const auto* col = 
vectorized::check_and_get_column<vectorized::ColumnUInt16>(column);
-        auto& data = col->get_data();
-        values->Add(data.begin() + start, data.begin() + end);
-        break;
-    }
-    case vectorized::TypeIndex::UInt32: {
-        ptype->set_id(PGenericType::UINT32);
-        auto* values = arg->mutable_uint32_value();
-        values->Reserve(row_count);
-        const auto* col = 
vectorized::check_and_get_column<vectorized::ColumnUInt32>(column);
-        auto& data = col->get_data();
-        values->Add(data.begin() + start, data.begin() + end);
-        break;
-    }
-    case vectorized::TypeIndex::UInt64: {
-        ptype->set_id(PGenericType::UINT64);
-        auto* values = arg->mutable_uint64_value();
-        values->Reserve(row_count);
-        const auto* col = 
vectorized::check_and_get_column<vectorized::ColumnUInt64>(column);
-        auto& data = col->get_data();
-        values->Add(data.begin() + start, data.begin() + end);
-        break;
-    }
-    case vectorized::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 vectorized::TypeIndex::Int8: {
-        ptype->set_id(PGenericType::INT8);
-        auto* values = arg->mutable_int32_value();
-        values->Reserve(row_count);
-        const auto* col = 
vectorized::check_and_get_column<vectorized::ColumnInt8>(column);
-        auto& data = col->get_data();
-        values->Add(data.begin() + start, data.begin() + end);
-        break;
-    }
-    case vectorized::TypeIndex::Int16: {
-        ptype->set_id(PGenericType::INT16);
-        auto* values = arg->mutable_int32_value();
-        values->Reserve(row_count);
-        const auto* col = 
vectorized::check_and_get_column<vectorized::ColumnInt16>(column);
-        auto& data = col->get_data();
-        values->Add(data.begin() + start, data.begin() + end);
-        break;
-    }
-    case vectorized::TypeIndex::Int32: {
-        ptype->set_id(PGenericType::INT32);
-        auto* values = arg->mutable_int32_value();
-        values->Reserve(row_count);
-        const auto* col = 
vectorized::check_and_get_column<vectorized::ColumnInt32>(column);
-        auto& data = col->get_data();
-        values->Add(data.begin() + start, data.begin() + end);
-        break;
-    }
-    case vectorized::TypeIndex::Int64: {
-        ptype->set_id(PGenericType::INT64);
-        auto* values = arg->mutable_int64_value();
-        values->Reserve(row_count);
-        const auto* col = 
vectorized::check_and_get_column<vectorized::ColumnInt64>(column);
-        auto& data = col->get_data();
-        values->Add(data.begin() + start, data.begin() + end);
-        break;
-    }
-    case vectorized::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 vectorized::TypeIndex::Float32: {
-        ptype->set_id(PGenericType::FLOAT);
-        auto* values = arg->mutable_float_value();
-        values->Reserve(row_count);
-        const auto* col = 
vectorized::check_and_get_column<vectorized::ColumnFloat32>(column);
-        auto& data = col->get_data();
-        values->Add(data.begin() + start, data.begin() + end);
-        break;
-    }
-
-    case vectorized::TypeIndex::Float64: {
-        ptype->set_id(PGenericType::DOUBLE);
-        auto* values = arg->mutable_double_value();
-        values->Reserve(row_count);
-        const auto* col = 
vectorized::check_and_get_column<vectorized::ColumnFloat64>(column);
-        auto& data = col->get_data();
-        values->Add(data.begin() + start, data.begin() + end);
-        break;
-    }
-    case vectorized::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 vectorized::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)) {
-                    vectorized::VecDateTimeValue v =
-                            
vectorized::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 {
-                vectorized::VecDateTimeValue v =
-                        vectorized::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 vectorized::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)) {
-                    vectorized::VecDateTimeValue v =
-                            
vectorized::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 {
-                vectorized::VecDateTimeValue v =
-                        
vectorized::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 vectorized::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 vectorized::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;
-    }
-    case vectorized::TypeIndex::QuantileState: {
-        ptype->set_id(PGenericType::QUANTILE_STATE);
-        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 convert_to_column(vectorized::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<vectorized::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<vectorized::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<vectorized::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<vectorized::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<vectorized::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<vectorized::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<vectorized::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<vectorized::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<vectorized::ColumnInt64*>(column.get())->get_data();
-        for (int i = 0; i < result.datetime_value_size(); ++i) {
-            vectorized::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<vectorized::VecDateTimeValue, 
vectorized::Int64>(v);
-        }
-        break;
-    }
-    case PGenericType::FLOAT: {
-        column->reserve(result.float_value_size());
-        column->resize(result.float_value_size());
-        auto& data = 
reinterpret_cast<vectorized::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<vectorized::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<vectorized::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<vectorized::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;
-    }
-    case PGenericType::QUANTILE_STATE: {
-        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;
-    }
-    }
-}
-
-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);
-
-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);
-
-} // namespace doris::vectorized
diff --git a/be/src/vec/aggregate_functions/aggregate_function_rpc.h 
b/be/src/vec/aggregate_functions/aggregate_function_rpc.h
index ea106150ee..937c936d86 100644
--- a/be/src/vec/aggregate_functions/aggregate_function_rpc.h
+++ b/be/src/vec/aggregate_functions/aggregate_function_rpc.h
@@ -21,7 +21,6 @@
 #include <memory>
 
 #include "common/status.h"
-#include "exprs/rpc_fn_comm.h"
 #include "gen_cpp/Exprs_types.h"
 #include "json2pb/json_to_pb.h"
 #include "json2pb/pb_to_json.h"
@@ -39,6 +38,7 @@
 #include "vec/core/field.h"
 #include "vec/core/types.h"
 #include "vec/data_types/data_type_string.h"
+#include "vec/functions/function_rpc.h"
 #include "vec/io/io_helper.h"
 namespace doris::vectorized {
 
@@ -158,13 +158,14 @@ public:
                 auto data_type = std::reinterpret_pointer_cast<const 
vectorized::DataTypeNullable>(
                         argument_types[i]);
                 data_col->get_data_at(0);
-                
convert_nullable_col_to_pvalue(data_col->convert_to_full_column_if_const(),
-                                               data_type->get_nested_type(), 
null_col, arg, start,
-                                               end);
+                RPCFnImpl::convert_nullable_col_to_pvalue(
+                        data_col->convert_to_full_column_if_const(), 
data_type->get_nested_type(),
+                        null_col, arg, start, end);
 
             } else {
-                
convert_col_to_pvalue<false>(columns[i]->convert_to_full_column_if_const(),
-                                             argument_types[i], arg, start, 
end);
+                RPCFnImpl::convert_col_to_pvalue<false>(
+                        columns[i]->convert_to_full_column_if_const(), 
argument_types[i], arg,
+                        start, end);
             }
         }
         return Status::OK();
diff --git a/be/src/vec/common/schema_util.cpp 
b/be/src/vec/common/schema_util.cpp
index f90345dab3..8a74ebf476 100644
--- a/be/src/vec/common/schema_util.cpp
+++ b/be/src/vec/common/schema_util.cpp
@@ -79,47 +79,6 @@ Array create_empty_array_field(size_t num_dimensions) {
     return array;
 }
 
-FieldType get_field_type(const IDataType* data_type) {
-    switch (data_type->get_type_id()) {
-    case TypeIndex::UInt8:
-        return FieldType::OLAP_FIELD_TYPE_UNSIGNED_TINYINT;
-    case TypeIndex::UInt16:
-        return FieldType::OLAP_FIELD_TYPE_UNSIGNED_SMALLINT;
-    case TypeIndex::UInt32:
-        return FieldType::OLAP_FIELD_TYPE_UNSIGNED_INT;
-    case TypeIndex::UInt64:
-        return FieldType::OLAP_FIELD_TYPE_UNSIGNED_BIGINT;
-    case TypeIndex::Int8:
-        return FieldType::OLAP_FIELD_TYPE_TINYINT;
-    case TypeIndex::Int16:
-        return FieldType::OLAP_FIELD_TYPE_SMALLINT;
-    case TypeIndex::Int32:
-        return FieldType::OLAP_FIELD_TYPE_INT;
-    case TypeIndex::Int64:
-        return FieldType::OLAP_FIELD_TYPE_BIGINT;
-    case TypeIndex::Float32:
-        return FieldType::OLAP_FIELD_TYPE_FLOAT;
-    case TypeIndex::Float64:
-        return FieldType::OLAP_FIELD_TYPE_DOUBLE;
-    case TypeIndex::Decimal32:
-        return FieldType::OLAP_FIELD_TYPE_DECIMAL;
-    case TypeIndex::Array:
-        return FieldType::OLAP_FIELD_TYPE_ARRAY;
-    case TypeIndex::String:
-        return FieldType::OLAP_FIELD_TYPE_STRING;
-    case TypeIndex::Date:
-        return FieldType::OLAP_FIELD_TYPE_DATE;
-    case TypeIndex::DateTime:
-        return FieldType::OLAP_FIELD_TYPE_DATETIME;
-    case TypeIndex::Tuple:
-        return FieldType::OLAP_FIELD_TYPE_STRUCT;
-    // TODO add more types
-    default:
-        LOG(FATAL) << "unknow type";
-        return FieldType::OLAP_FIELD_TYPE_UNKNOWN;
-    }
-}
-
 bool is_conversion_required_between_integers(const IDataType& lhs, const 
IDataType& rhs) {
     WhichDataType which_lhs(lhs);
     WhichDataType which_rhs(rhs);
diff --git a/be/src/vec/functions/function_rpc.cpp 
b/be/src/vec/functions/function_rpc.cpp
index c85c16d5c6..5f591991ff 100644
--- a/be/src/vec/functions/function_rpc.cpp
+++ b/be/src/vec/functions/function_rpc.cpp
@@ -35,10 +35,10 @@ RPCFnImpl::RPCFnImpl(const TFunction& fn) : _fn(fn) {
                              _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) {
+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();
@@ -46,15 +46,15 @@ void RPCFnImpl::_convert_nullable_col_to_pvalue(const 
ColumnPtr& column,
         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);
+        RPCFnImpl::convert_col_to_pvalue<true>(column, data_type, arg, start, 
end);
     } else {
-        this->_convert_col_to_pvalue<false>(column, data_type, arg, start, 
end);
+        RPCFnImpl::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) {
+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()) {
@@ -511,11 +511,11 @@ void RPCFnImpl::_convert_block_to_proto(Block& block, 
const ColumnNumbers& argum
             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);
+            
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);
+            convert_col_to_pvalue<false>(col, column.type, arg, 0, row_count);
         }
     }
 }
diff --git a/be/src/vec/functions/function_rpc.h 
b/be/src/vec/functions/function_rpc.h
index 116c213b44..30f2dbaf8d 100644
--- a/be/src/vec/functions/function_rpc.h
+++ b/be/src/vec/functions/function_rpc.h
@@ -31,19 +31,20 @@ public:
                     const std::vector<size_t>& arguments, size_t result, 
size_t input_rows_count);
     bool available() { return _client != nullptr; }
 
+    static 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>
+    static void convert_col_to_pvalue(const vectorized::ColumnPtr& column,
+                                      const vectorized::DataTypePtr& 
data_type, PValues* arg,
+                                      int start, int end);
+
 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);
 


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


Reply via email to