You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by GitBox <gi...@apache.org> on 2022/06/01 10:41:14 UTC

[GitHub] [incubator-doris] Gabriel39 opened a new pull request, #9916: [Feature-wip] support datev2/datetimev2/timev2 in FE

Gabriel39 opened a new pull request, #9916:
URL: https://github.com/apache/incubator-doris/pull/9916

   # Proposed changes
   
   Issue Number: close #9575 
   
   ## Problem Summary:
   
   Describe the overview of changes.
   
   ## Checklist(Required)
   
   1. Does it affect the original behavior: (Yes/No/I Don't know)
   2. Has unit tests been added: (Yes/No/No Need)
   3. Has document been added or modified: (Yes/No/No Need)
   4. Does it need to update dependencies: (Yes/No)
   5. Are there any changes that cannot be rolled back: (Yes/No)
   
   ## 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] Gabriel39 commented on a diff in pull request #9916: [Feature-wip] support datev2

Posted by GitBox <gi...@apache.org>.
Gabriel39 commented on code in PR #9916:
URL: https://github.com/apache/doris/pull/9916#discussion_r905699230


##########
be/src/olap/types.h:
##########
@@ -974,6 +988,104 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DATE> : public BaseFieldtypeTraits<OLAP_F
     }
 };
 
+template <>
+struct FieldTypeTraits<OLAP_FIELD_TYPE_DATEV2>
+        : public BaseFieldtypeTraits<OLAP_FIELD_TYPE_DATEV2> {
+    static Status from_string(void* buf, const std::string& scan_key) {
+        tm time_tm;
+        char* res = strptime(scan_key.c_str(), "%Y-%m-%d", &time_tm);
+
+        if (nullptr != res) {
+            uint32_t value = ((time_tm.tm_year + 1900) << 16) | ((time_tm.tm_mon + 1) << 8) |
+                             time_tm.tm_mday;
+            *reinterpret_cast<CppType*>(buf) = value;
+        } else {
+            *reinterpret_cast<CppType*>(buf) = doris::vectorized::MIN_DATE_V2;
+        }
+
+        return Status::OK();
+    }
+    static std::string to_string(const void* src) {
+        CppType tmp = *reinterpret_cast<const CppType*>(src);
+        doris::vectorized::DateV2Value value =
+                binary_cast<CppType, doris::vectorized::DateV2Value>(tmp);
+        string format = "%Y-%m-%d";
+        string res;
+        res.resize(12);
+        res.reserve(12);
+        value.to_format_string(format.c_str(), format.size(), res.data());
+        return res;
+    }
+    static Status convert_from(void* dest, const void* src, const TypeInfo* src_type,
+                               MemPool* mem_pool, size_t variable_len = 0) {
+        if (src_type->type() == FieldType::OLAP_FIELD_TYPE_DATETIME) {
+            using SrcType = typename CppTypeTraits<OLAP_FIELD_TYPE_DATETIME>::CppType;
+            SrcType src_value = *reinterpret_cast<const SrcType*>(src);
+            //only need part one
+            SrcType part1 = (src_value / 1000000L);
+            CppType year = static_cast<CppType>((part1 / 10000L) % 10000);
+            CppType mon = static_cast<CppType>((part1 / 100) % 100);
+            CppType mday = static_cast<CppType>(part1 % 100);
+            *reinterpret_cast<CppType*>(dest) = (year << 16) | (mon << 8) | mday;
+            return Status::OK();
+        }
+
+        if (src_type->type() == FieldType::OLAP_FIELD_TYPE_DATE) {
+            using SrcType = typename CppTypeTraits<OLAP_FIELD_TYPE_DATE>::CppType;
+            SrcType value = *reinterpret_cast<const SrcType*>(src);
+            int day = static_cast<int>(value & 31);
+            int mon = static_cast<int>(value >> 5 & 15);
+            int year = static_cast<int>(value >> 9);
+            *reinterpret_cast<CppType*>(dest) = (year << 16) | (mon << 8) | day;
+            return Status::OK();
+        }
+
+        if (src_type->type() == FieldType::OLAP_FIELD_TYPE_INT) {
+            using SrcType = typename CppTypeTraits<OLAP_FIELD_TYPE_INT>::CppType;
+            SrcType src_value = *reinterpret_cast<const SrcType*>(src);
+            doris::vectorized::DateV2Value dt;
+            if (!dt.from_date_int64(src_value)) {
+                return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+            }
+            CppType year = static_cast<CppType>(src_value / 10000);
+            CppType month = static_cast<CppType>((src_value % 10000) / 100);
+            CppType day = static_cast<CppType>(src_value % 100);
+            *reinterpret_cast<CppType*>(dest) = (year << 16) | (month << 8) | day;
+            return Status::OK();
+        }
+
+        if (src_type->type() == OLAP_FIELD_TYPE_VARCHAR ||
+            src_type->type() == OLAP_FIELD_TYPE_CHAR ||
+            src_type->type() == OLAP_FIELD_TYPE_STRING) {
+            if (src_type->type() == OLAP_FIELD_TYPE_CHAR) {
+                prepare_char_before_convert(src);
+            }
+            using SrcType = typename CppTypeTraits<OLAP_FIELD_TYPE_VARCHAR>::CppType;
+            auto src_value = *reinterpret_cast<const SrcType*>(src);
+            doris::vectorized::DateV2Value dt;
+            for (const auto& format : DATE_FORMATS) {
+                if (dt.from_date_format_str(format.c_str(), format.length(), src_value.get_data(),
+                                            src_value.get_size())) {
+                    *reinterpret_cast<CppType*>(dest) =
+                            (dt.year() << 16) | (dt.month() << 8) | dt.day();
+                    return Status::OK();
+                }
+            }
+            return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+        }
+
+        return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+    }
+    static void set_to_max(void* buf) {
+        // max is 9999 * 16 * 32 + 12 * 32 + 31;
+        *reinterpret_cast<CppType*>(buf) = doris::vectorized::MAX_DATE_V2;
+    }
+    static void set_to_min(void* buf) {
+        // min is 0 * 16 * 32 + 1 * 32 + 1;
+        *reinterpret_cast<CppType*>(buf) = doris::vectorized::MIN_DATE_V2;

Review Comment:
   > The DATE type is used for values with a date part but no time part. MySQL retrieves and displays DATE values in 'YYYY-MM-DD' format. The supported range is '1000-01-01' to '9999-12-31'.
   
   > The DATETIME type is used for values that contain both date and time parts. MySQL retrieves and displays DATETIME values in 'YYYY-MM-DD hh:mm:ss' format. The supported range is '1000-01-01 00:00:00' to '9999-12-31 23:59:59'.
   
   This is described in document[1]
   
   [1] https://dev.mysql.com/doc/refman/8.0/en/datetime.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 #9916: [Feature-wip] support datev2

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #9916:
URL: https://github.com/apache/doris/pull/9916#issuecomment-1166448322

   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 #9916: [Feature-wip] support datev2

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #9916:
URL: https://github.com/apache/doris/pull/9916#issuecomment-1166448329

   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] HappenLee commented on a diff in pull request #9916: [Feature-wip] support datev2

Posted by GitBox <gi...@apache.org>.
HappenLee commented on code in PR #9916:
URL: https://github.com/apache/doris/pull/9916#discussion_r905690100


##########
be/src/olap/types.h:
##########
@@ -974,6 +988,104 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DATE> : public BaseFieldtypeTraits<OLAP_F
     }
 };
 
+template <>
+struct FieldTypeTraits<OLAP_FIELD_TYPE_DATEV2>
+        : public BaseFieldtypeTraits<OLAP_FIELD_TYPE_DATEV2> {
+    static Status from_string(void* buf, const std::string& scan_key) {
+        tm time_tm;
+        char* res = strptime(scan_key.c_str(), "%Y-%m-%d", &time_tm);
+
+        if (nullptr != res) {
+            uint32_t value = ((time_tm.tm_year + 1900) << 16) | ((time_tm.tm_mon + 1) << 8) |
+                             time_tm.tm_mday;
+            *reinterpret_cast<CppType*>(buf) = value;
+        } else {
+            *reinterpret_cast<CppType*>(buf) = doris::vectorized::MIN_DATE_V2;
+        }
+
+        return Status::OK();
+    }
+    static std::string to_string(const void* src) {
+        CppType tmp = *reinterpret_cast<const CppType*>(src);
+        doris::vectorized::DateV2Value value =
+                binary_cast<CppType, doris::vectorized::DateV2Value>(tmp);
+        string format = "%Y-%m-%d";
+        string res;
+        res.resize(12);
+        res.reserve(12);
+        value.to_format_string(format.c_str(), format.size(), res.data());
+        return res;
+    }
+    static Status convert_from(void* dest, const void* src, const TypeInfo* src_type,
+                               MemPool* mem_pool, size_t variable_len = 0) {
+        if (src_type->type() == FieldType::OLAP_FIELD_TYPE_DATETIME) {
+            using SrcType = typename CppTypeTraits<OLAP_FIELD_TYPE_DATETIME>::CppType;
+            SrcType src_value = *reinterpret_cast<const SrcType*>(src);
+            //only need part one
+            SrcType part1 = (src_value / 1000000L);
+            CppType year = static_cast<CppType>((part1 / 10000L) % 10000);
+            CppType mon = static_cast<CppType>((part1 / 100) % 100);
+            CppType mday = static_cast<CppType>(part1 % 100);
+            *reinterpret_cast<CppType*>(dest) = (year << 16) | (mon << 8) | mday;
+            return Status::OK();
+        }
+
+        if (src_type->type() == FieldType::OLAP_FIELD_TYPE_DATE) {
+            using SrcType = typename CppTypeTraits<OLAP_FIELD_TYPE_DATE>::CppType;
+            SrcType value = *reinterpret_cast<const SrcType*>(src);
+            int day = static_cast<int>(value & 31);
+            int mon = static_cast<int>(value >> 5 & 15);
+            int year = static_cast<int>(value >> 9);
+            *reinterpret_cast<CppType*>(dest) = (year << 16) | (mon << 8) | day;
+            return Status::OK();
+        }
+
+        if (src_type->type() == FieldType::OLAP_FIELD_TYPE_INT) {
+            using SrcType = typename CppTypeTraits<OLAP_FIELD_TYPE_INT>::CppType;
+            SrcType src_value = *reinterpret_cast<const SrcType*>(src);
+            doris::vectorized::DateV2Value dt;
+            if (!dt.from_date_int64(src_value)) {
+                return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+            }
+            CppType year = static_cast<CppType>(src_value / 10000);
+            CppType month = static_cast<CppType>((src_value % 10000) / 100);
+            CppType day = static_cast<CppType>(src_value % 100);
+            *reinterpret_cast<CppType*>(dest) = (year << 16) | (month << 8) | day;
+            return Status::OK();
+        }
+
+        if (src_type->type() == OLAP_FIELD_TYPE_VARCHAR ||
+            src_type->type() == OLAP_FIELD_TYPE_CHAR ||
+            src_type->type() == OLAP_FIELD_TYPE_STRING) {
+            if (src_type->type() == OLAP_FIELD_TYPE_CHAR) {
+                prepare_char_before_convert(src);
+            }
+            using SrcType = typename CppTypeTraits<OLAP_FIELD_TYPE_VARCHAR>::CppType;
+            auto src_value = *reinterpret_cast<const SrcType*>(src);
+            doris::vectorized::DateV2Value dt;
+            for (const auto& format : DATE_FORMATS) {
+                if (dt.from_date_format_str(format.c_str(), format.length(), src_value.get_data(),
+                                            src_value.get_size())) {
+                    *reinterpret_cast<CppType*>(dest) =
+                            (dt.year() << 16) | (dt.month() << 8) | dt.day();
+                    return Status::OK();
+                }
+            }
+            return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+        }
+
+        return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+    }
+    static void set_to_max(void* buf) {
+        // max is 9999 * 16 * 32 + 12 * 32 + 31;
+        *reinterpret_cast<CppType*>(buf) = doris::vectorized::MAX_DATE_V2;
+    }
+    static void set_to_min(void* buf) {
+        // min is 0 * 16 * 32 + 1 * 32 + 1;
+        *reinterpret_cast<CppType*>(buf) = doris::vectorized::MIN_DATE_V2;

Review Comment:
   mysql seems support "0000-00-00", rethink whether we need to support the data



##########
be/src/exec/olap_scan_node.cpp:
##########
@@ -1000,7 +1001,8 @@ std::pair<bool, void*> OlapScanNode::should_push_down_eq_predicate(doris::SlotDe
 
 template <typename T, typename ChangeFixedValueRangeFunc>
 Status OlapScanNode::change_fixed_value_range(ColumnValueRange<T>& temp_range, PrimitiveType type,
-                                              void* value, const ChangeFixedValueRangeFunc& func) {
+                                              void* value, const ChangeFixedValueRangeFunc& func,
+                                              PrimitiveType to_type) {

Review Comment:
   add param of `to_type` seems do not use ?



-- 
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 pull request #9916: [Feature-wip] support datev2

Posted by GitBox <gi...@apache.org>.
yiguolei commented on PR #9916:
URL: https://github.com/apache/doris/pull/9916#issuecomment-1166448241

   I will merge it and we can do decimal v3 based on this.


-- 
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] Gabriel39 commented on a diff in pull request #9916: [Feature-wip] support datev2

Posted by GitBox <gi...@apache.org>.
Gabriel39 commented on code in PR #9916:
URL: https://github.com/apache/doris/pull/9916#discussion_r905698872


##########
be/src/exec/olap_scan_node.cpp:
##########
@@ -1000,7 +1001,8 @@ std::pair<bool, void*> OlapScanNode::should_push_down_eq_predicate(doris::SlotDe
 
 template <typename T, typename ChangeFixedValueRangeFunc>
 Status OlapScanNode::change_fixed_value_range(ColumnValueRange<T>& temp_range, PrimitiveType type,
-                                              void* value, const ChangeFixedValueRangeFunc& func) {
+                                              void* value, const ChangeFixedValueRangeFunc& func,
+                                              PrimitiveType to_type) {

Review Comment:
   This argument is added in order to meet patterns like `cast(datev2 as datetimev2)`. Since datetimev2 has not been  supported, this argument is indeed not used now.



-- 
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 #9916: [Feature-wip] support datev2

Posted by GitBox <gi...@apache.org>.
yiguolei commented on code in PR #9916:
URL: https://github.com/apache/doris/pull/9916#discussion_r906662183


##########
be/src/olap/types.h:
##########
@@ -974,6 +988,104 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DATE> : public BaseFieldtypeTraits<OLAP_F
     }
 };
 
+template <>
+struct FieldTypeTraits<OLAP_FIELD_TYPE_DATEV2>
+        : public BaseFieldtypeTraits<OLAP_FIELD_TYPE_DATEV2> {
+    static Status from_string(void* buf, const std::string& scan_key) {
+        tm time_tm;
+        char* res = strptime(scan_key.c_str(), "%Y-%m-%d", &time_tm);
+
+        if (nullptr != res) {
+            uint32_t value = ((time_tm.tm_year + 1900) << 16) | ((time_tm.tm_mon + 1) << 8) |
+                             time_tm.tm_mday;
+            *reinterpret_cast<CppType*>(buf) = value;
+        } else {
+            *reinterpret_cast<CppType*>(buf) = doris::vectorized::MIN_DATE_V2;
+        }
+
+        return Status::OK();
+    }
+    static std::string to_string(const void* src) {
+        CppType tmp = *reinterpret_cast<const CppType*>(src);
+        doris::vectorized::DateV2Value value =
+                binary_cast<CppType, doris::vectorized::DateV2Value>(tmp);
+        string format = "%Y-%m-%d";
+        string res;
+        res.resize(12);
+        res.reserve(12);
+        value.to_format_string(format.c_str(), format.size(), res.data());
+        return res;
+    }
+    static Status convert_from(void* dest, const void* src, const TypeInfo* src_type,
+                               MemPool* mem_pool, size_t variable_len = 0) {
+        if (src_type->type() == FieldType::OLAP_FIELD_TYPE_DATETIME) {
+            using SrcType = typename CppTypeTraits<OLAP_FIELD_TYPE_DATETIME>::CppType;
+            SrcType src_value = *reinterpret_cast<const SrcType*>(src);
+            //only need part one
+            SrcType part1 = (src_value / 1000000L);
+            CppType year = static_cast<CppType>((part1 / 10000L) % 10000);
+            CppType mon = static_cast<CppType>((part1 / 100) % 100);
+            CppType mday = static_cast<CppType>(part1 % 100);
+            *reinterpret_cast<CppType*>(dest) = (year << 16) | (mon << 8) | mday;
+            return Status::OK();
+        }
+
+        if (src_type->type() == FieldType::OLAP_FIELD_TYPE_DATE) {
+            using SrcType = typename CppTypeTraits<OLAP_FIELD_TYPE_DATE>::CppType;
+            SrcType value = *reinterpret_cast<const SrcType*>(src);
+            int day = static_cast<int>(value & 31);
+            int mon = static_cast<int>(value >> 5 & 15);
+            int year = static_cast<int>(value >> 9);
+            *reinterpret_cast<CppType*>(dest) = (year << 16) | (mon << 8) | day;
+            return Status::OK();
+        }
+
+        if (src_type->type() == FieldType::OLAP_FIELD_TYPE_INT) {
+            using SrcType = typename CppTypeTraits<OLAP_FIELD_TYPE_INT>::CppType;
+            SrcType src_value = *reinterpret_cast<const SrcType*>(src);
+            doris::vectorized::DateV2Value dt;
+            if (!dt.from_date_int64(src_value)) {
+                return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+            }
+            CppType year = static_cast<CppType>(src_value / 10000);
+            CppType month = static_cast<CppType>((src_value % 10000) / 100);
+            CppType day = static_cast<CppType>(src_value % 100);
+            *reinterpret_cast<CppType*>(dest) = (year << 16) | (month << 8) | day;
+            return Status::OK();
+        }
+
+        if (src_type->type() == OLAP_FIELD_TYPE_VARCHAR ||
+            src_type->type() == OLAP_FIELD_TYPE_CHAR ||
+            src_type->type() == OLAP_FIELD_TYPE_STRING) {
+            if (src_type->type() == OLAP_FIELD_TYPE_CHAR) {
+                prepare_char_before_convert(src);
+            }
+            using SrcType = typename CppTypeTraits<OLAP_FIELD_TYPE_VARCHAR>::CppType;
+            auto src_value = *reinterpret_cast<const SrcType*>(src);
+            doris::vectorized::DateV2Value dt;
+            for (const auto& format : DATE_FORMATS) {
+                if (dt.from_date_format_str(format.c_str(), format.length(), src_value.get_data(),
+                                            src_value.get_size())) {
+                    *reinterpret_cast<CppType*>(dest) =
+                            (dt.year() << 16) | (dt.month() << 8) | dt.day();
+                    return Status::OK();
+                }
+            }
+            return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+        }
+
+        return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+    }
+    static void set_to_max(void* buf) {
+        // max is 9999 * 16 * 32 + 12 * 32 + 31;
+        *reinterpret_cast<CppType*>(buf) = doris::vectorized::MAX_DATE_V2;
+    }
+    static void set_to_min(void* buf) {
+        // min is 0 * 16 * 32 + 1 * 32 + 1;
+        *reinterpret_cast<CppType*>(buf) = doris::vectorized::MIN_DATE_V2;

Review Comment:
   Maybe we do not support years before 1000. Although we could save it but could not support calculations. And maybe we could support it in the future.



-- 
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 #9916: [Feature-wip] support datev2

Posted by GitBox <gi...@apache.org>.
yiguolei merged PR #9916:
URL: https://github.com/apache/doris/pull/9916


-- 
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