You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by "amorynan (via GitHub)" <gi...@apache.org> on 2023/06/24 09:00:46 UTC

[GitHub] [doris] amorynan opened a new pull request, #21109: [Improve](serde)update serialize and deserialize text for data type

amorynan opened a new pull request, #21109:
URL: https://github.com/apache/doris/pull/21109

   ## Proposed changes
   this pr is for centralize text serde for data types , now we use different from_string and to_string in field, wrapper_field, data_types/ and predict_column ... , but serde at text can be unified for specify data type, not scattered everywhere. 
   Issue Number: close #xxx
   
   <!--Describe your changes.-->
   In this pr :
   1. from_string_test aim to comparing using "from_string" at data_type and wrapper field 
   2. data_type_serde_text_test aim to comparing serde text action can equal to wrapper field in scala type 
   
   after this pr, I will do
   1. replace wrapper field with core/field and using data type serde instead of using  wrapper field from_string method
   2. complete nested datatype (array/map/struct) text serde (support nested type in text) 
   ## Further comments
   
   If this is a relatively large or complex change, kick off the discussion at [dev@doris.apache.org](mailto:dev@doris.apache.org) by explaining why you chose the solution you did and what alternatives you considered, etc...
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] github-actions[bot] commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1646796454

   clang-tidy review says "All clean, LGTM! :+1:"


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] hello-stephen commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "hello-stephen (via GitHub)" <gi...@apache.org>.
hello-stephen commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1627857470

   TeamCity pipeline, clickbench performance test result:
    the sum of best hot time: 45.45 seconds
    stream load tsv:          450 seconds loaded 74807831229 Bytes, about 158 MB/s
    stream load json:         19 seconds loaded 2358488459 Bytes, about 118 MB/s
    stream load orc:          57 seconds loaded 1101869774 Bytes, about 18 MB/s
    stream load parquet:          30 seconds loaded 861443392 Bytes, about 27 MB/s
    insert into select:          25.7 seconds inserted 10000000 Rows, about 389K ops/s
    https://doris-community-test-1308700295.cos.ap-hongkong.myqcloud.com/tmp/20230709231602_clickbench_pr_175359.html


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] github-actions[bot] commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1605325602

   clang-tidy review says "All clean, LGTM! :+1:"


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] eldenmoon commented on a diff in pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "eldenmoon (via GitHub)" <gi...@apache.org>.
eldenmoon commented on code in PR #21109:
URL: https://github.com/apache/doris/pull/21109#discussion_r1241877756


##########
be/src/vec/data_types/serde/data_type_serde.h:
##########
@@ -57,9 +59,20 @@ class IDataType;
 // the developer does not know how many datatypes has to deal.
 
 class DataTypeSerDe {
+public:
+    struct FormatOptions {
+        bool use_lib_format = false;

Review Comment:
   `use_lib_format` could you add some example to explain more details of this flag? 



##########
be/src/vec/data_types/serde/data_type_number_serde.cpp:
##########
@@ -92,6 +95,63 @@ void DataTypeNumberSerDe<T>::write_column_to_arrow(const IColumn& column, const
     }
 }
 
+template <typename T>
+Status DataTypeNumberSerDe<T>::deserialize_one_cell_from_text(IColumn& column, ReadBuffer& rb,
+                                                              const FormatOptions& options) const {
+    auto& column_data = reinterpret_cast<ColumnType&>(column);
+    if constexpr (std::is_same<T, UInt128>::value) {
+        // TODO: support for Uint128
+        return Status::InvalidArgument("uint128 is not support");
+    } else if constexpr (std::is_same_v<T, float> || std::is_same_v<T, double>) {
+        T val = 0;
+        if (!read_float_text_fast_impl(val, rb)) {
+            return Status::InvalidArgument("parse number fail, string: '{}'",
+                                           std::string(rb.position(), rb.count()).c_str());
+        }
+        column_data.insert_value(val);
+    } else if constexpr (std::is_same_v<T, uint8_t>) {
+        // Note: here we should handle the bool type
+        T val = 0;
+        if (!try_read_bool_text(val, rb)) {

Review Comment:
   what if doris provide uint_8 type? The input  rb could be '123' and result Error?



##########
be/src/vec/data_types/serde/data_type_number_serde.cpp:
##########
@@ -92,6 +95,63 @@ void DataTypeNumberSerDe<T>::write_column_to_arrow(const IColumn& column, const
     }
 }
 
+template <typename T>
+Status DataTypeNumberSerDe<T>::deserialize_one_cell_from_text(IColumn& column, ReadBuffer& rb,
+                                                              const FormatOptions& options) const {
+    auto& column_data = reinterpret_cast<ColumnType&>(column);
+    if constexpr (std::is_same<T, UInt128>::value) {

Review Comment:
   why not handle float and double?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] github-actions[bot] commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1617663372

   PR approved by anyone and no changes requested.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] github-actions[bot] commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1627665813

   clang-tidy review says "All clean, LGTM! :+1:"


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] yiguolei commented on a diff in pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "yiguolei (via GitHub)" <gi...@apache.org>.
yiguolei commented on code in PR #21109:
URL: https://github.com/apache/doris/pull/21109#discussion_r1266281350


##########
be/test/vec/data_types/from_string_test.cpp:
##########
@@ -0,0 +1,177 @@
+// 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 "gtest/gtest_pred_impl.h"
+#include "olap/types.h" // for TypeInfo
+#include "olap/wrapper_field.h"
+#include "vec/columns/column.h"
+#include "vec/core/field.h"
+#include "vec/data_types/data_type.h"
+#include "vec/data_types/data_type_factory.hpp"
+#include "vec/data_types/serde_utils.h"
+#include "vec/io/reader_buffer.h"
+
+namespace doris::vectorized {
+
+/**
+ * This test is used to check wrapperField from_string is equal to data type from_string or not
+ *  same string feed to wrapperField and data type from_string, and check the result from
+ *  wrapperField and data type to_string is equal or not
+ */
+TEST(FromStringTest, ScalaWrapperFieldVsDataType) {

Review Comment:
   decimal32
   decimal64
   decimal128
   
   decimalv2



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] yiguolei commented on a diff in pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "yiguolei (via GitHub)" <gi...@apache.org>.
yiguolei commented on code in PR #21109:
URL: https://github.com/apache/doris/pull/21109#discussion_r1266281350


##########
be/test/vec/data_types/from_string_test.cpp:
##########
@@ -0,0 +1,177 @@
+// 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 "gtest/gtest_pred_impl.h"
+#include "olap/types.h" // for TypeInfo
+#include "olap/wrapper_field.h"
+#include "vec/columns/column.h"
+#include "vec/core/field.h"
+#include "vec/data_types/data_type.h"
+#include "vec/data_types/data_type_factory.hpp"
+#include "vec/data_types/serde_utils.h"
+#include "vec/io/reader_buffer.h"
+
+namespace doris::vectorized {
+
+/**
+ * This test is used to check wrapperField from_string is equal to data type from_string or not
+ *  same string feed to wrapperField and data type from_string, and check the result from
+ *  wrapperField and data type to_string is equal or not
+ */
+TEST(FromStringTest, ScalaWrapperFieldVsDataType) {

Review Comment:
   decimalv3
       decimal32
       decimal64
       decimal128
   
   decimalv2



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] github-actions[bot] commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1619356383

   clang-tidy review says "All clean, LGTM! :+1:"


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] yiguolei commented on a diff in pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "yiguolei (via GitHub)" <gi...@apache.org>.
yiguolei commented on code in PR #21109:
URL: https://github.com/apache/doris/pull/21109#discussion_r1248543369


##########
be/src/vec/data_types/serde/data_type_date64_serde.cpp:
##########
@@ -23,10 +23,122 @@
 
 #include "gutil/casts.h"
 #include "vec/columns/column_const.h"
+#include "vec/io/io_helper.h"
 
 namespace doris {
 namespace vectorized {
 
+void DataTypeDate64SerDe::serialize_one_cell_to_text(const IColumn& column, int row_num,
+                                                     BufferWritable& bw,
+                                                     const FormatOptions& options) const {
+    auto result = check_column_const_set_readability(column, row_num);
+    ColumnPtr ptr = result.first;
+    row_num = result.second;
+
+    Int64 int_val = assert_cast<const ColumnInt64&>(*ptr).get_element(row_num);
+    if (options.use_lib_format) {
+        tm time_tm;
+        memset(&time_tm, 0, sizeof(time_tm));
+        time_tm.tm_mday = static_cast<int>(int_val & 31);
+        time_tm.tm_mon = static_cast<int>(int_val >> 5 & 15) - 1;
+        time_tm.tm_year = static_cast<int>(int_val >> 9) - 1900;
+        char buf[20] = {'\0'};

Review Comment:
   这样写太废了,相当于这里先搞一个buf 数组,然后填充;再转string 这样会再copy 一次;然后buffer writable 会继续copy 一次



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] amorynan commented on a diff in pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "amorynan (via GitHub)" <gi...@apache.org>.
amorynan commented on code in PR #21109:
URL: https://github.com/apache/doris/pull/21109#discussion_r1250190815


##########
be/src/vec/data_types/serde/data_type_date64_serde.cpp:
##########
@@ -23,10 +23,122 @@
 
 #include "gutil/casts.h"
 #include "vec/columns/column_const.h"
+#include "vec/io/io_helper.h"
 
 namespace doris {
 namespace vectorized {
 
+void DataTypeDate64SerDe::serialize_one_cell_to_text(const IColumn& column, int row_num,
+                                                     BufferWritable& bw,
+                                                     const FormatOptions& options) const {
+    auto result = check_column_const_set_readability(column, row_num);
+    ColumnPtr ptr = result.first;
+    row_num = result.second;
+
+    Int64 int_val = assert_cast<const ColumnInt64&>(*ptr).get_element(row_num);
+    if (options.use_lib_format) {
+        tm time_tm;
+        memset(&time_tm, 0, sizeof(time_tm));
+        time_tm.tm_mday = static_cast<int>(int_val & 31);
+        time_tm.tm_mon = static_cast<int>(int_val >> 5 & 15) - 1;
+        time_tm.tm_year = static_cast<int>(int_val >> 9) - 1900;
+        char buf[20] = {'\0'};

Review Comment:
   just make same code with type.h to_string() , so not suggest use this if 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] amorynan commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "amorynan (via GitHub)" <gi...@apache.org>.
amorynan commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1617742523

   run beut


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] yiguolei commented on a diff in pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "yiguolei (via GitHub)" <gi...@apache.org>.
yiguolei commented on code in PR #21109:
URL: https://github.com/apache/doris/pull/21109#discussion_r1248536581


##########
be/test/vec/data_types/from_string_test.cpp:
##########
@@ -0,0 +1,162 @@
+// 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

Review Comment:
   add some ut for column const , column nullable const



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] github-actions[bot] commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1648910233

   clang-tidy review says "All clean, LGTM! :+1:"


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] yiguolei merged pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "yiguolei (via GitHub)" <gi...@apache.org>.
yiguolei merged PR #21109:
URL: https://github.com/apache/doris/pull/21109


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] amorynan commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "amorynan (via GitHub)" <gi...@apache.org>.
amorynan commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1648899358

   run buildall


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] github-actions[bot] commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1627667147

   clang-tidy review says "All clean, LGTM! :+1:"


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] github-actions[bot] commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1617669399

   clang-tidy review says "All clean, LGTM! :+1:"


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] github-actions[bot] commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1619356253

   clang-tidy review says "All clean, LGTM! :+1:"


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] amorynan commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "amorynan (via GitHub)" <gi...@apache.org>.
amorynan commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1606928812

   run buildall


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] github-actions[bot] commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1649194662

   PR approved by at least one committer and no changes requested.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] github-actions[bot] commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1649592411

   PR approved by at least one committer and no changes requested.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] amorynan commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "amorynan (via GitHub)" <gi...@apache.org>.
amorynan commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1627711582

   run buildall


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] hello-stephen commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "hello-stephen (via GitHub)" <gi...@apache.org>.
hello-stephen commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1627721819

   (From new machine)TeamCity pipeline, clickbench performance test result:
    the sum of best hot time: 51.46 seconds
    stream load tsv:          500 seconds loaded 74807831229 Bytes, about 142 MB/s
    stream load json:         19 seconds loaded 2358488459 Bytes, about 118 MB/s
    stream load orc:          65 seconds loaded 1101869774 Bytes, about 16 MB/s
    stream load parquet:          30 seconds loaded 861443392 Bytes, about 27 MB/s
    insert into select:          28.6 seconds inserted 10000000 Rows, about 349K ops/s
    storage size: 17168518683 Bytes
    https://doris-community-test-1308700295.cos.ap-hongkong.myqcloud.com/tmp/20230709214937_clickbench_pr_175234.html


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] hello-stephen commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "hello-stephen (via GitHub)" <gi...@apache.org>.
hello-stephen commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1619454794

   TeamCity pipeline, clickbench performance test result:
    the sum of best hot time: 39.28 seconds
    stream load tsv:          468 seconds loaded 74807831229 Bytes, about 152 MB/s
    stream load json:         22 seconds loaded 2358488459 Bytes, about 102 MB/s
    stream load orc:          57 seconds loaded 1101869774 Bytes, about 18 MB/s
    stream load parquet:          29 seconds loaded 861443392 Bytes, about 28 MB/s
    insert into select:          71.1 seconds inserted 10000000 Rows, about 140K ops/s
    https://doris-community-test-1308700295.cos.ap-hongkong.myqcloud.com/tmp/20230704042042_clickbench_pr_171749.html


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] github-actions[bot] commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1641523322

   clang-tidy review says "All clean, LGTM! :+1:"


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] github-actions[bot] commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1648909749

   clang-tidy review says "All clean, LGTM! :+1:"


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] hello-stephen commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "hello-stephen (via GitHub)" <gi...@apache.org>.
hello-stephen commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1649590497

   (From new machine)TeamCity pipeline, clickbench performance test result:
    the sum of best hot time: 44.95 seconds
    stream load tsv:          504 seconds loaded 74807831229 Bytes, about 141 MB/s
    stream load json:         20 seconds loaded 2358488459 Bytes, about 112 MB/s
    stream load orc:          65 seconds loaded 1101869774 Bytes, about 16 MB/s
    stream load parquet:          31 seconds loaded 861443392 Bytes, about 26 MB/s
    insert into select:          29.3 seconds inserted 10000000 Rows, about 341K ops/s
    storage size: 17161641918 Bytes


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] github-actions[bot] commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1647682316

   clang-tidy review says "All clean, LGTM! :+1:"


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] github-actions[bot] commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1619389480

   clang-tidy review says "All clean, LGTM! :+1:"


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] github-actions[bot] commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1617666784

   clang-tidy review says "All clean, LGTM! :+1:"


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] github-actions[bot] commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1627711362

   clang-tidy review says "All clean, LGTM! :+1:"


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] github-actions[bot] commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1639454991

   PR approved by at least one committer and no changes requested.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] yiguolei commented on a diff in pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "yiguolei (via GitHub)" <gi...@apache.org>.
yiguolei commented on code in PR #21109:
URL: https://github.com/apache/doris/pull/21109#discussion_r1266225781


##########
be/test/vec/data_types/from_string_test.cpp:
##########
@@ -0,0 +1,177 @@
+// 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 "gtest/gtest_pred_impl.h"
+#include "olap/types.h" // for TypeInfo
+#include "olap/wrapper_field.h"
+#include "vec/columns/column.h"
+#include "vec/core/field.h"
+#include "vec/data_types/data_type.h"
+#include "vec/data_types/data_type_factory.hpp"
+#include "vec/data_types/serde_utils.h"
+#include "vec/io/reader_buffer.h"
+
+namespace doris::vectorized {
+
+/**
+ * This test is used to check wrapperField from_string is equal to data type from_string or not
+ *  same string feed to wrapperField and data type from_string, and check the result from
+ *  wrapperField and data type to_string is equal or not
+ */
+TEST(FromStringTest, ScalaWrapperFieldVsDataType) {
+    // arithmetic scala field types
+    {
+        std::vector<FieldType> arithmetic_scala_field_types = {
+                FieldType::OLAP_FIELD_TYPE_BOOL,       FieldType::OLAP_FIELD_TYPE_TINYINT,
+                FieldType::OLAP_FIELD_TYPE_SMALLINT,   FieldType::OLAP_FIELD_TYPE_INT,
+                FieldType::OLAP_FIELD_TYPE_FLOAT,      FieldType::OLAP_FIELD_TYPE_DOUBLE,
+                FieldType::OLAP_FIELD_TYPE_BIGINT,     FieldType::OLAP_FIELD_TYPE_LARGEINT,
+                FieldType::OLAP_FIELD_TYPE_CHAR,       FieldType::OLAP_FIELD_TYPE_VARCHAR,
+                FieldType::OLAP_FIELD_TYPE_STRING,     FieldType::OLAP_FIELD_TYPE_DECIMAL,
+                FieldType::OLAP_FIELD_TYPE_DECIMAL32,  FieldType::OLAP_FIELD_TYPE_DECIMAL64,
+                FieldType::OLAP_FIELD_TYPE_DECIMAL128I};
+
+        for (auto type : arithmetic_scala_field_types) {
+            DataTypePtr data_type_ptr;
+            if (type == FieldType::OLAP_FIELD_TYPE_DECIMAL32) {
+                data_type_ptr = DataTypeFactory::instance().create_data_type(type, 9, 0);
+            } else if (type == FieldType::OLAP_FIELD_TYPE_DECIMAL64) {

Review Comment:
   test is too simple。
   For decimal type, please test decimalv2, decimalv3(decimal32,decimal64,decimal128) and the precision should be 0 or 2



##########
be/test/vec/data_types/from_string_test.cpp:
##########
@@ -0,0 +1,177 @@
+// 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 "gtest/gtest_pred_impl.h"
+#include "olap/types.h" // for TypeInfo
+#include "olap/wrapper_field.h"
+#include "vec/columns/column.h"
+#include "vec/core/field.h"
+#include "vec/data_types/data_type.h"
+#include "vec/data_types/data_type_factory.hpp"
+#include "vec/data_types/serde_utils.h"
+#include "vec/io/reader_buffer.h"
+
+namespace doris::vectorized {
+
+/**
+ * This test is used to check wrapperField from_string is equal to data type from_string or not
+ *  same string feed to wrapperField and data type from_string, and check the result from
+ *  wrapperField and data type to_string is equal or not
+ */
+TEST(FromStringTest, ScalaWrapperFieldVsDataType) {
+    // arithmetic scala field types
+    {
+        std::vector<FieldType> arithmetic_scala_field_types = {
+                FieldType::OLAP_FIELD_TYPE_BOOL,       FieldType::OLAP_FIELD_TYPE_TINYINT,
+                FieldType::OLAP_FIELD_TYPE_SMALLINT,   FieldType::OLAP_FIELD_TYPE_INT,
+                FieldType::OLAP_FIELD_TYPE_FLOAT,      FieldType::OLAP_FIELD_TYPE_DOUBLE,
+                FieldType::OLAP_FIELD_TYPE_BIGINT,     FieldType::OLAP_FIELD_TYPE_LARGEINT,
+                FieldType::OLAP_FIELD_TYPE_CHAR,       FieldType::OLAP_FIELD_TYPE_VARCHAR,
+                FieldType::OLAP_FIELD_TYPE_STRING,     FieldType::OLAP_FIELD_TYPE_DECIMAL,
+                FieldType::OLAP_FIELD_TYPE_DECIMAL32,  FieldType::OLAP_FIELD_TYPE_DECIMAL64,
+                FieldType::OLAP_FIELD_TYPE_DECIMAL128I};
+
+        for (auto type : arithmetic_scala_field_types) {
+            DataTypePtr data_type_ptr;
+            if (type == FieldType::OLAP_FIELD_TYPE_DECIMAL32) {
+                data_type_ptr = DataTypeFactory::instance().create_data_type(type, 9, 0);
+            } else if (type == FieldType::OLAP_FIELD_TYPE_DECIMAL64) {

Review Comment:
   test is too simple。
   For decimal type, please test decimalv2, decimalv3(decimal32,decimal64,decimal128) and the precision should be 0 or 2



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] yiguolei commented on a diff in pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "yiguolei (via GitHub)" <gi...@apache.org>.
yiguolei commented on code in PR #21109:
URL: https://github.com/apache/doris/pull/21109#discussion_r1266226483


##########
be/test/vec/data_types/from_string_test.cpp:
##########
@@ -0,0 +1,177 @@
+// 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 "gtest/gtest_pred_impl.h"
+#include "olap/types.h" // for TypeInfo
+#include "olap/wrapper_field.h"
+#include "vec/columns/column.h"
+#include "vec/core/field.h"
+#include "vec/data_types/data_type.h"
+#include "vec/data_types/data_type_factory.hpp"
+#include "vec/data_types/serde_utils.h"
+#include "vec/io/reader_buffer.h"
+
+namespace doris::vectorized {
+
+/**
+ * This test is used to check wrapperField from_string is equal to data type from_string or not
+ *  same string feed to wrapperField and data type from_string, and check the result from
+ *  wrapperField and data type to_string is equal or not
+ */
+TEST(FromStringTest, ScalaWrapperFieldVsDataType) {
+    // arithmetic scala field types
+    {
+        std::vector<FieldType> arithmetic_scala_field_types = {
+                FieldType::OLAP_FIELD_TYPE_BOOL,       FieldType::OLAP_FIELD_TYPE_TINYINT,
+                FieldType::OLAP_FIELD_TYPE_SMALLINT,   FieldType::OLAP_FIELD_TYPE_INT,
+                FieldType::OLAP_FIELD_TYPE_FLOAT,      FieldType::OLAP_FIELD_TYPE_DOUBLE,
+                FieldType::OLAP_FIELD_TYPE_BIGINT,     FieldType::OLAP_FIELD_TYPE_LARGEINT,
+                FieldType::OLAP_FIELD_TYPE_CHAR,       FieldType::OLAP_FIELD_TYPE_VARCHAR,
+                FieldType::OLAP_FIELD_TYPE_STRING,     FieldType::OLAP_FIELD_TYPE_DECIMAL,
+                FieldType::OLAP_FIELD_TYPE_DECIMAL32,  FieldType::OLAP_FIELD_TYPE_DECIMAL64,
+                FieldType::OLAP_FIELD_TYPE_DECIMAL128I};
+
+        for (auto type : arithmetic_scala_field_types) {
+            DataTypePtr data_type_ptr;
+            if (type == FieldType::OLAP_FIELD_TYPE_DECIMAL32) {
+                data_type_ptr = DataTypeFactory::instance().create_data_type(type, 9, 0);
+            } else if (type == FieldType::OLAP_FIELD_TYPE_DECIMAL64) {

Review Comment:
   and datetimev2 also has pricision, should also test different precisions.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] yiguolei commented on a diff in pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "yiguolei (via GitHub)" <gi...@apache.org>.
yiguolei commented on code in PR #21109:
URL: https://github.com/apache/doris/pull/21109#discussion_r1266283211


##########
be/test/vec/data_types/from_string_test.cpp:
##########
@@ -0,0 +1,177 @@
+// 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 "gtest/gtest_pred_impl.h"
+#include "olap/types.h" // for TypeInfo
+#include "olap/wrapper_field.h"
+#include "vec/columns/column.h"
+#include "vec/core/field.h"
+#include "vec/data_types/data_type.h"
+#include "vec/data_types/data_type_factory.hpp"
+#include "vec/data_types/serde_utils.h"
+#include "vec/io/reader_buffer.h"
+
+namespace doris::vectorized {
+
+/**
+ * This test is used to check wrapperField from_string is equal to data type from_string or not
+ *  same string feed to wrapperField and data type from_string, and check the result from
+ *  wrapperField and data type to_string is equal or not
+ */
+TEST(FromStringTest, ScalaWrapperFieldVsDataType) {

Review Comment:
   decimalv2 是固定的
   decimalv2(27,9)
   
   decimalv2(27,8) 的行为应该跟 decimalv2(27,9) 一样的
   



##########
be/test/vec/data_types/from_string_test.cpp:
##########
@@ -0,0 +1,177 @@
+// 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 "gtest/gtest_pred_impl.h"
+#include "olap/types.h" // for TypeInfo
+#include "olap/wrapper_field.h"
+#include "vec/columns/column.h"
+#include "vec/core/field.h"
+#include "vec/data_types/data_type.h"
+#include "vec/data_types/data_type_factory.hpp"
+#include "vec/data_types/serde_utils.h"
+#include "vec/io/reader_buffer.h"
+
+namespace doris::vectorized {
+
+/**
+ * This test is used to check wrapperField from_string is equal to data type from_string or not
+ *  same string feed to wrapperField and data type from_string, and check the result from
+ *  wrapperField and data type to_string is equal or not
+ */
+TEST(FromStringTest, ScalaWrapperFieldVsDataType) {

Review Comment:
   decimalv2 是固定的
   decimalv2(27,9)
   
   decimalv2(27,8) 的行为应该跟 decimalv2(27,9) 一样的
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] hello-stephen commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "hello-stephen (via GitHub)" <gi...@apache.org>.
hello-stephen commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1649034399

   (From new machine)TeamCity pipeline, clickbench performance test result:
    the sum of best hot time: 46.18 seconds
    stream load tsv:          506 seconds loaded 74807831229 Bytes, about 140 MB/s
    stream load json:         21 seconds loaded 2358488459 Bytes, about 107 MB/s
    stream load orc:          65 seconds loaded 1101869774 Bytes, about 16 MB/s
    stream load parquet:          30 seconds loaded 861443392 Bytes, about 27 MB/s
    insert into select:          29.6 seconds inserted 10000000 Rows, about 337K ops/s
    storage size: 17169817249 Bytes


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] github-actions[bot] commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1649545153

   clang-tidy review says "All clean, LGTM! :+1:"


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] amorynan commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "amorynan (via GitHub)" <gi...@apache.org>.
amorynan commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1617656322

   run buildall


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] amorynan commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "amorynan (via GitHub)" <gi...@apache.org>.
amorynan commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1619439790

   run p0


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] amorynan commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "amorynan (via GitHub)" <gi...@apache.org>.
amorynan commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1627669121

   run buildall


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] github-actions[bot] commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1605324238

   clang-tidy review says "All clean, LGTM! :+1:"


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] amorynan commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "amorynan (via GitHub)" <gi...@apache.org>.
amorynan commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1619388217

   run buildall


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] amorynan commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "amorynan (via GitHub)" <gi...@apache.org>.
amorynan commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1619358995

   run buildall


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] amorynan commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "amorynan (via GitHub)" <gi...@apache.org>.
amorynan commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1627851654

   run clickbench


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] hello-stephen commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "hello-stephen (via GitHub)" <gi...@apache.org>.
hello-stephen commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1647873561

   (From new machine)TeamCity pipeline, clickbench performance test result:
    the sum of best hot time: 45.14 seconds
    stream load tsv:          506 seconds loaded 74807831229 Bytes, about 140 MB/s
    stream load json:         21 seconds loaded 2358488459 Bytes, about 107 MB/s
    stream load orc:          65 seconds loaded 1101869774 Bytes, about 16 MB/s
    stream load parquet:          31 seconds loaded 861443392 Bytes, about 26 MB/s
    insert into select:          29.3 seconds inserted 10000000 Rows, about 341K ops/s
    storage size: 17160224993 Bytes


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] amorynan commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "amorynan (via GitHub)" <gi...@apache.org>.
amorynan commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1647775684

   run buildall


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] github-actions[bot] commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1646796311

   clang-tidy review says "All clean, LGTM! :+1:"


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] yiguolei commented on a diff in pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "yiguolei (via GitHub)" <gi...@apache.org>.
yiguolei commented on code in PR #21109:
URL: https://github.com/apache/doris/pull/21109#discussion_r1266280514


##########
be/test/vec/data_types/from_string_test.cpp:
##########
@@ -0,0 +1,177 @@
+// 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 "gtest/gtest_pred_impl.h"
+#include "olap/types.h" // for TypeInfo
+#include "olap/wrapper_field.h"
+#include "vec/columns/column.h"
+#include "vec/core/field.h"
+#include "vec/data_types/data_type.h"
+#include "vec/data_types/data_type_factory.hpp"
+#include "vec/data_types/serde_utils.h"
+#include "vec/io/reader_buffer.h"
+
+namespace doris::vectorized {
+
+/**
+ * This test is used to check wrapperField from_string is equal to data type from_string or not
+ *  same string feed to wrapperField and data type from_string, and check the result from
+ *  wrapperField and data type to_string is equal or not
+ */
+TEST(FromStringTest, ScalaWrapperFieldVsDataType) {

Review Comment:
   std::string  decimal32_val = "32.12"
   auto serde = createSerde(10,4);
   Column col = serde.from_string(col, decimal32_val);
   col.get_row_num(1) == 32.1200
   
   std::string  decimal32_val = "32.1234567891011"
   auto serde = createSerde(10,4);
   Column col = serde.from_string(col, decimal32_val);
   col.get_row_num(1) == 32.1200
   
   serde = createserde(10,1)
   col.get_row_num(1) == 32.1
   
   std::string  decimal32_val = "32.16"
   auto serde = createSerde(10,1);
   col.get_row_num(1) == 32.2 ??
   
   
   datetime
   datetimev2 (precision) ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] github-actions[bot] commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1641521791

   clang-tidy review says "All clean, LGTM! :+1:"


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] amorynan commented on a diff in pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "amorynan (via GitHub)" <gi...@apache.org>.
amorynan commented on code in PR #21109:
URL: https://github.com/apache/doris/pull/21109#discussion_r1243114148


##########
be/src/vec/data_types/serde/data_type_number_serde.cpp:
##########
@@ -92,6 +95,63 @@ void DataTypeNumberSerDe<T>::write_column_to_arrow(const IColumn& column, const
     }
 }
 
+template <typename T>
+Status DataTypeNumberSerDe<T>::deserialize_one_cell_from_text(IColumn& column, ReadBuffer& rb,
+                                                              const FormatOptions& options) const {
+    auto& column_data = reinterpret_cast<ColumnType&>(column);
+    if constexpr (std::is_same<T, UInt128>::value) {
+        // TODO: support for Uint128
+        return Status::InvalidArgument("uint128 is not support");
+    } else if constexpr (std::is_same_v<T, float> || std::is_same_v<T, double>) {
+        T val = 0;
+        if (!read_float_text_fast_impl(val, rb)) {
+            return Status::InvalidArgument("parse number fail, string: '{}'",
+                                           std::string(rb.position(), rb.count()).c_str());
+        }
+        column_data.insert_value(val);
+    } else if constexpr (std::is_same_v<T, uint8_t>) {
+        // Note: here we should handle the bool type
+        T val = 0;
+        if (!try_read_bool_text(val, rb)) {

Review Comment:
   inner try_read_bool_text() we just make judge rb == 0 || rb == 1 , and return true or false , so will not result error, but here now we think uint_8 equals bool , just keep same with data_type_number_base.cpp from_string behavior, but I do not sure when I get a false ,like '123' , can we just insert into vector<uint8> column? is that much ambiguity for uint8 and bool ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] github-actions[bot] commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1617663298

   PR approved by at least one committer and no changes requested.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] github-actions[bot] commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1617664633

   clang-tidy review says "All clean, LGTM! :+1:"


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] amorynan commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "amorynan (via GitHub)" <gi...@apache.org>.
amorynan commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1617740598

   run clickbench


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] amorynan commented on pull request #21109: [Improve](serde)update serialize and deserialize text for data type

Posted by "amorynan (via GitHub)" <gi...@apache.org>.
amorynan commented on PR #21109:
URL: https://github.com/apache/doris/pull/21109#issuecomment-1649532387

   run buildall


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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