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

[doris] branch branch-1.2-lts updated: [fix]fixed dangerous usage of namespace std (#15741)

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

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


The following commit(s) were added to refs/heads/branch-1.2-lts by this push:
     new 981e18dfc7 [fix]fixed dangerous usage of namespace std (#15741)
981e18dfc7 is described below

commit 981e18dfc74d4966288016c7c00c02d899c39350
Author: zclllyybb <zh...@vip.qq.com>
AuthorDate: Tue Jan 10 16:10:49 2023 +0800

    [fix]fixed dangerous usage of namespace std (#15741)
    
    Co-authored-by: zhaochangle <zh...@selectdb.com>
---
 be/src/agent/task_worker_pool.cpp         | 18 +++----
 be/src/exec/es/es_predicate.cpp           |  2 -
 be/src/gutil/hash/hash.h                  | 12 ++---
 be/src/gutil/strings/stringpiece.cc       |  4 +-
 be/src/gutil/strings/stringpiece.h        |  4 +-
 be/src/olap/aggregate_func.cpp            |  9 ++--
 be/src/olap/in_list_predicate.h           | 14 +++---
 be/src/olap/task/engine_batch_load_task.h |  2 -
 be/src/runtime/datetime_value.h           |  4 +-
 be/src/runtime/decimalv2_value.h          |  4 +-
 be/src/util/hash_util.hpp                 | 11 ++---
 be/src/util/mustache/mustache.cc          | 82 ++++++++++++++++---------------
 be/src/util/time.cpp                      | 44 ++++++++---------
 be/src/util/uid_util.h                    |  6 +--
 be/src/vec/common/string_ref.h            |  4 +-
 be/src/vec/common/strong_typedef.h        |  4 +-
 be/src/vec/common/uint128.h               | 14 +++---
 be/src/vec/core/types.h                   |  8 ++-
 be/src/vec/functions/function_totype.h    |  4 +-
 be/src/vec/runtime/vdatetime_value.h      |  8 ++-
 20 files changed, 111 insertions(+), 147 deletions(-)

diff --git a/be/src/agent/task_worker_pool.cpp b/be/src/agent/task_worker_pool.cpp
index 3f5ca98021..020bcd1543 100644
--- a/be/src/agent/task_worker_pool.cpp
+++ b/be/src/agent/task_worker_pool.cpp
@@ -264,16 +264,16 @@ void TaskWorkerPool::notify_thread() {
 }
 
 bool TaskWorkerPool::_register_task_info(const TTaskType::type task_type, int64_t signature) {
-    lock_guard<std::mutex> task_signatures_lock(_s_task_signatures_lock);
-    set<int64_t>& signature_set = _s_task_signatures[task_type];
+    std::lock_guard<std::mutex> task_signatures_lock(_s_task_signatures_lock);
+    std::set<int64_t>& signature_set = _s_task_signatures[task_type];
     return signature_set.insert(signature).second;
 }
 
 void TaskWorkerPool::_remove_task_info(const TTaskType::type task_type, int64_t signature) {
     size_t queue_size;
     {
-        lock_guard<std::mutex> task_signatures_lock(_s_task_signatures_lock);
-        set<int64_t>& signature_set = _s_task_signatures[task_type];
+        std::lock_guard<std::mutex> task_signatures_lock(_s_task_signatures_lock);
+        std::set<int64_t>& signature_set = _s_task_signatures[task_type];
         signature_set.erase(signature);
         queue_size = signature_set.size();
     }
@@ -313,7 +313,7 @@ uint32_t TaskWorkerPool::_get_next_task_index(int32_t thread_count,
                                               std::deque<TAgentTaskRequest>& tasks,
                                               TPriority::type priority) {
     int32_t index = -1;
-    deque<TAgentTaskRequest>::size_type task_count = tasks.size();
+    std::deque<TAgentTaskRequest>::size_type task_count = tasks.size();
     for (uint32_t i = 0; i < task_count; ++i) {
         TAgentTaskRequest task = tasks[i];
         if (priority == TPriority::HIGH) {
@@ -1115,7 +1115,7 @@ void TaskWorkerPool::_report_task_worker_thread_callback() {
         // See _random_sleep() comment in _report_disk_state_worker_thread_callback
         _random_sleep(5);
         {
-            lock_guard<std::mutex> task_signatures_lock(_s_task_signatures_lock);
+            std::lock_guard<std::mutex> task_signatures_lock(_s_task_signatures_lock);
             request.__set_tasks(_s_task_signatures);
         }
         _handle_report(request, ReportType::TASK);
@@ -1159,7 +1159,7 @@ void TaskWorkerPool::_report_disk_state_worker_thread_callback() {
         std::vector<DataDirInfo> data_dir_infos;
         _env->storage_engine()->get_all_data_dir_info(&data_dir_infos, true /* update */);
 
-        map<string, TDisk> disks;
+        std::map<string, TDisk> disks;
         for (auto& root_path_info : data_dir_infos) {
             TDisk disk;
             disk.__set_root_path(root_path_info.path);
@@ -1629,7 +1629,7 @@ void TaskWorkerPool::_storage_refresh_storage_policy_worker_thread_callback() {
             // update storage policy mgr.
             StoragePolicyMgr* spm = ExecEnv::GetInstance()->storage_policy_mgr();
             for (const auto& iter : result.result_entrys) {
-                shared_ptr<StoragePolicy> policy_ptr = make_shared<StoragePolicy>();
+                auto policy_ptr = std::make_shared<StoragePolicy>();
                 policy_ptr->storage_policy_name = iter.policy_name;
                 policy_ptr->cooldown_datetime = iter.cooldown_datetime;
                 policy_ptr->cooldown_ttl = iter.cooldown_ttl;
@@ -1669,7 +1669,7 @@ void TaskWorkerPool::_storage_update_storage_policy_worker_thread_callback() {
         }
 
         StoragePolicyMgr* spm = ExecEnv::GetInstance()->storage_policy_mgr();
-        shared_ptr<StoragePolicy> policy_ptr = make_shared<StoragePolicy>();
+        auto policy_ptr = std::make_shared<StoragePolicy>();
         policy_ptr->storage_policy_name = get_storage_policy_req.policy_name;
         policy_ptr->cooldown_datetime = get_storage_policy_req.cooldown_datetime;
         policy_ptr->cooldown_ttl = get_storage_policy_req.cooldown_ttl;
diff --git a/be/src/exec/es/es_predicate.cpp b/be/src/exec/es/es_predicate.cpp
index cbf21d1651..9773cc32c6 100644
--- a/be/src/exec/es/es_predicate.cpp
+++ b/be/src/exec/es/es_predicate.cpp
@@ -36,8 +36,6 @@
 
 namespace doris {
 
-using namespace std;
-
 #define RETURN_ERROR_IF_EXPR_IS_NOT_SLOTREF(expr)                                          \
     do {                                                                                   \
         const Expr* expr_without_cast = Expr::expr_without_cast(expr);                     \
diff --git a/be/src/gutil/hash/hash.h b/be/src/gutil/hash/hash.h
index 1eee35ab96..dc9898d313 100644
--- a/be/src/gutil/hash/hash.h
+++ b/be/src/gutil/hash/hash.h
@@ -118,7 +118,7 @@
 extern uint64 FingerprintReferenceImplementation(const char* s, uint32 len);
 extern uint64 FingerprintInterleavedImplementation(const char* s, uint32 len);
 inline uint64 Fingerprint(const char* s, uint32 len) {
-    if (sizeof(s) == 8) { // 64-bit systems have 8-byte pointers.
+    if constexpr (sizeof(s) == 8) { // 64-bit systems have 8-byte pointers.
         // The better choice when we have a decent number of registers.
         return FingerprintInterleavedImplementation(s, len);
     } else {
@@ -126,7 +126,7 @@ inline uint64 Fingerprint(const char* s, uint32 len) {
     }
 }
 
-// Routine that combines together the hi/lo part of a fingerprint
+// Routine that combines the hi/lo part of a fingerprint
 // and changes the result appropriately to avoid returning 0/1.
 inline uint64 CombineFingerprintHalves(uint32 hi, uint32 lo) {
     uint64 result = (static_cast<uint64>(hi) << 32) | static_cast<uint64>(lo);
@@ -181,11 +181,9 @@ inline uint64 FingerprintCat(uint64 fp1, uint64 fp2) {
     return Hash64NumWithSeed(fp1, fp2);
 }
 
-namespace std {
-
 // This intended to be a "good" hash function.  It may change from time to time.
 template <>
-struct hash<uint128> {
+struct std::hash<uint128> {
     size_t operator()(const uint128& x) const {
         if (sizeof(&x) == 8) { // 64-bit systems have 8-byte pointers.
             return Hash128to64(x);
@@ -205,7 +203,7 @@ struct hash<uint128> {
 
 // Hasher for STL pairs. Requires hashers for both members to be defined
 template <class First, class Second>
-struct hash<pair<First, Second> > {
+struct std::hash<std::pair<First, Second> > {
     size_t operator()(const pair<First, Second>& p) const {
         size_t h1 = std::hash<First>()(p.first);
         size_t h2 = std::hash<Second>()(p.second);
@@ -216,8 +214,6 @@ struct hash<pair<First, Second> > {
     static const size_t min_buckets = 8; // 4 and 8 are defaults.
 };
 
-} // namespace std
-
 // If you want an excellent string hash function, and you don't mind if it
 // might change when you sync and recompile, please use GoodFastHash<>.
 // For most applications, GoodFastHash<> is a good choice, better than
diff --git a/be/src/gutil/strings/stringpiece.cc b/be/src/gutil/strings/stringpiece.cc
index 094e8e27b6..c5c58b1772 100644
--- a/be/src/gutil/strings/stringpiece.cc
+++ b/be/src/gutil/strings/stringpiece.cc
@@ -23,11 +23,9 @@ using std::sort;
 using std::swap;
 using std::string;
 
-namespace std {
-size_t hash<StringPiece>::operator()(StringPiece s) const {
+size_t std::hash<StringPiece>::operator()(StringPiece s) const {
     return HashTo32(s.data(), s.size());
 }
-} // namespace std
 
 std::ostream& operator<<(std::ostream& o, StringPiece piece) {
     o.write(piece.data(), piece.size());
diff --git a/be/src/gutil/strings/stringpiece.h b/be/src/gutil/strings/stringpiece.h
index 143eb8d3e3..de1f291363 100644
--- a/be/src/gutil/strings/stringpiece.h
+++ b/be/src/gutil/strings/stringpiece.h
@@ -334,12 +334,10 @@ struct GoodFastHash;
 // SWIG doesn't know how to parse this stuff properly. Omit it.
 #ifndef SWIG
 
-namespace std {
 template <>
-struct hash<StringPiece> {
+struct std::hash<StringPiece> {
     size_t operator()(StringPiece s) const;
 };
-} // namespace std
 
 // An implementation of GoodFastHash for StringPiece.  See
 // GoodFastHash values.
diff --git a/be/src/olap/aggregate_func.cpp b/be/src/olap/aggregate_func.cpp
index 027105952c..d943e94015 100644
--- a/be/src/olap/aggregate_func.cpp
+++ b/be/src/olap/aggregate_func.cpp
@@ -17,11 +17,10 @@
 
 #include "olap/aggregate_func.h"
 
-namespace std {
-namespace {
 // algorithm from boost: http://www.boost.org/doc/libs/1_61_0/doc/html/hash/reference.html#boost.hash_combine
+// `static` make them with internal linkage.
 template <class T>
-inline void hash_combine(std::size_t& seed, T const& v) {
+inline static void hash_combine(std::size_t& seed, T const& v) {
     seed ^= std::hash<T>()(v) + 0x9e3779b9 + (seed << 6) + (seed >> 2);
 }
 
@@ -37,17 +36,15 @@ template <class Tuple>
 struct HashValueImpl<Tuple, 0> {
     static void apply(size_t& seed, Tuple const& tuple) { hash_combine(seed, std::get<0>(tuple)); }
 };
-} // namespace
 
 template <typename... TT>
-struct hash<std::tuple<TT...>> {
+struct std::hash<std::tuple<TT...>> {
     size_t operator()(std::tuple<TT...> const& tt) const {
         size_t seed = 0;
         HashValueImpl<std::tuple<TT...>>::apply(seed, tt);
         return seed;
     }
 };
-} // namespace std
 
 namespace doris {
 
diff --git a/be/src/olap/in_list_predicate.h b/be/src/olap/in_list_predicate.h
index eed5260a6a..b78afa271e 100644
--- a/be/src/olap/in_list_predicate.h
+++ b/be/src/olap/in_list_predicate.h
@@ -36,36 +36,35 @@
 #include "vec/columns/column_dictionary.h"
 #include "vec/core/types.h"
 
-namespace std {
 // for string value
 template <>
-struct hash<doris::StringValue> {
+struct std::hash<doris::StringValue> {
     uint64_t operator()(const doris::StringValue& rhs) const { return hash_value(rhs); }
 };
 
 template <>
-struct equal_to<doris::StringValue> {
+struct std::equal_to<doris::StringValue> {
     bool operator()(const doris::StringValue& lhs, const doris::StringValue& rhs) const {
         return lhs == rhs;
     }
 };
 // for decimal12_t
 template <>
-struct hash<doris::decimal12_t> {
+struct std::hash<doris::decimal12_t> {
     int64_t operator()(const doris::decimal12_t& rhs) const {
         return hash<int64_t>()(rhs.integer) ^ hash<int32_t>()(rhs.fraction);
     }
 };
 
 template <>
-struct equal_to<doris::decimal12_t> {
+struct std::equal_to<doris::decimal12_t> {
     bool operator()(const doris::decimal12_t& lhs, const doris::decimal12_t& rhs) const {
         return lhs == rhs;
     }
 };
 // for uint24_t
 template <>
-struct hash<doris::uint24_t> {
+struct std::hash<doris::uint24_t> {
     size_t operator()(const doris::uint24_t& rhs) const {
         uint32_t val(rhs);
         return hash<int>()(val);
@@ -73,12 +72,11 @@ struct hash<doris::uint24_t> {
 };
 
 template <>
-struct equal_to<doris::uint24_t> {
+struct std::equal_to<doris::uint24_t> {
     bool operator()(const doris::uint24_t& lhs, const doris::uint24_t& rhs) const {
         return lhs == rhs;
     }
 };
-} // namespace std
 
 namespace doris {
 
diff --git a/be/src/olap/task/engine_batch_load_task.h b/be/src/olap/task/engine_batch_load_task.h
index a37d6033b2..14bf611e93 100644
--- a/be/src/olap/task/engine_batch_load_task.h
+++ b/be/src/olap/task/engine_batch_load_task.h
@@ -28,8 +28,6 @@
 #include "olap/olap_define.h"
 #include "olap/task/engine_task.h"
 
-using namespace std;
-
 namespace doris {
 
 const uint32_t PUSH_MAX_RETRY = 1;
diff --git a/be/src/runtime/datetime_value.h b/be/src/runtime/datetime_value.h
index b7f0392097..dec991ece2 100644
--- a/be/src/runtime/datetime_value.h
+++ b/be/src/runtime/datetime_value.h
@@ -719,9 +719,7 @@ std::size_t hash_value(DateTimeValue const& value);
 
 } // namespace doris
 
-namespace std {
 template <>
-struct hash<doris::DateTimeValue> {
+struct std::hash<doris::DateTimeValue> {
     size_t operator()(const doris::DateTimeValue& v) const { return doris::hash_value(v); }
 };
-} // namespace std
diff --git a/be/src/runtime/decimalv2_value.h b/be/src/runtime/decimalv2_value.h
index 12c13bee69..44d93c7e11 100644
--- a/be/src/runtime/decimalv2_value.h
+++ b/be/src/runtime/decimalv2_value.h
@@ -329,9 +329,7 @@ std::size_t hash_value(DecimalV2Value const& value);
 
 } // end namespace doris
 
-namespace std {
 template <>
-struct hash<doris::DecimalV2Value> {
+struct std::hash<doris::DecimalV2Value> {
     size_t operator()(const doris::DecimalV2Value& v) const { return doris::hash_value(v); }
 };
-} // namespace std
diff --git a/be/src/util/hash_util.hpp b/be/src/util/hash_util.hpp
index 330a35d028..b563a902ce 100644
--- a/be/src/util/hash_util.hpp
+++ b/be/src/util/hash_util.hpp
@@ -339,9 +339,8 @@ public:
 
 } // namespace doris
 
-namespace std {
 template <>
-struct hash<doris::TUniqueId> {
+struct std::hash<doris::TUniqueId> {
     std::size_t operator()(const doris::TUniqueId& id) const {
         std::size_t seed = 0;
         seed = doris::HashUtil::hash(&id.lo, sizeof(id.lo), seed);
@@ -351,7 +350,7 @@ struct hash<doris::TUniqueId> {
 };
 
 template <>
-struct hash<doris::TNetworkAddress> {
+struct std::hash<doris::TNetworkAddress> {
     size_t operator()(const doris::TNetworkAddress& address) const {
         std::size_t seed = 0;
         seed = doris::HashUtil::hash(address.hostname.data(), address.hostname.size(), seed);
@@ -363,7 +362,7 @@ struct hash<doris::TNetworkAddress> {
 #if __GNUC__ < 6 && !defined(__clang__)
 // Cause this is builtin function
 template <>
-struct hash<__int128> {
+struct std::hash<__int128> {
     std::size_t operator()(const __int128& val) const {
         return doris::HashUtil::hash(&val, sizeof(val), 0);
     }
@@ -371,7 +370,7 @@ struct hash<__int128> {
 #endif
 
 template <>
-struct hash<std::pair<doris::TUniqueId, int64_t>> {
+struct std::hash<std::pair<doris::TUniqueId, int64_t>> {
     size_t operator()(const std::pair<doris::TUniqueId, int64_t>& pair) const {
         size_t seed = 0;
         seed = doris::HashUtil::hash(&pair.first.lo, sizeof(pair.first.lo), seed);
@@ -380,5 +379,3 @@ struct hash<std::pair<doris::TUniqueId, int64_t>> {
         return seed;
     }
 };
-
-} // namespace std
diff --git a/be/src/util/mustache/mustache.cc b/be/src/util/mustache/mustache.cc
index 249d28553b..a3947e8463 100644
--- a/be/src/util/mustache/mustache.cc
+++ b/be/src/util/mustache/mustache.cc
@@ -24,7 +24,6 @@
 #include "rapidjson/writer.h"
 
 using namespace rapidjson;
-using namespace std;
 using namespace boost::algorithm;
 
 namespace mustache {
@@ -50,8 +49,8 @@ enum TagOperator {
 
 struct OpCtx {
     TagOperator op;
-    string tag_name;
-    string tag_arg;
+    std::string tag_name;
+    std::string tag_arg;
     bool escaped = false;
 };
 
@@ -60,7 +59,7 @@ struct ContextStack {
     const ContextStack* parent;
 };
 
-TagOperator GetOperator(const string& tag) {
+TagOperator GetOperator(const std::string& tag) {
     if (tag.size() == 0) return SUBSTITUTION;
     switch (tag[0]) {
     case '#':
@@ -87,13 +86,13 @@ TagOperator GetOperator(const string& tag) {
     }
 }
 
-int EvaluateTag(const string& document, const string& document_root, int idx,
-                const ContextStack* context, const OpCtx& op_ctx, stringstream* out);
+int EvaluateTag(const std::string& document, const std::string& document_root, int idx,
+                const ContextStack* context, const OpCtx& op_ctx, std::stringstream* out);
 
-static bool RenderTemplate(const string& document, const string& document_root,
-                           const ContextStack* stack, stringstream* out);
+static bool RenderTemplate(const std::string& document, const std::string& document_root,
+                           const ContextStack* stack, std::stringstream* out);
 
-void EscapeHtml(const string& in, stringstream* out) {
+void EscapeHtml(const std::string& in, std::stringstream* out) {
     for (const char& c : in) {
         switch (c) {
         case '&':
@@ -129,7 +128,7 @@ void Dump(const rapidjson::Value& v) {
 // being a simple split() is that we allow path components to be quoted, e.g.: "foo".bar,
 // and any '.' characters inside those quoted sections aren't considered to be
 // delimiters. This is to allow Json keys that contain periods.
-void FindJsonPathComponents(const string& path, vector<string>* components) {
+void FindJsonPathComponents(const std::string& path, std::vector<std::string>* components) {
     bool in_quote = false;
     bool escape_this_char = false;
     int start = 0;
@@ -166,19 +165,20 @@ void FindJsonPathComponents(const string& path, vector<string>* components) {
 
 // Looks up the json entity at 'path' in 'parent_context', and places it in 'resolved'. If
 // the entity does not exist (i.e. the path is invalid), 'resolved' will be set to nullptr.
-void ResolveJsonContext(const string& path, const ContextStack* stack, const Value** resolved) {
+void ResolveJsonContext(const std::string& path, const ContextStack* stack,
+                        const Value** resolved) {
     if (path == ".") {
         *resolved = stack->value;
         return;
     }
-    vector<string> components;
+    std::vector<std::string> components;
     FindJsonPathComponents(path, &components);
 
     // At each enclosing level of context, try to resolve the path.
     for (; stack != nullptr; stack = stack->parent) {
         const Value* cur = stack->value;
         bool match = true;
-        for (const string& c : components) {
+        for (const std::string& c : components) {
             if (cur->IsObject() && cur->HasMember(c.c_str())) {
                 cur = &(*cur)[c.c_str()];
             } else {
@@ -194,7 +194,7 @@ void ResolveJsonContext(const string& path, const ContextStack* stack, const Val
     *resolved = nullptr;
 }
 
-int FindNextTag(const string& document, int idx, OpCtx* op, stringstream* out) {
+int FindNextTag(const std::string& document, int idx, OpCtx* op, std::stringstream* out) {
     op->op = NONE;
     while (idx < document.size()) {
         if (document[idx] == '{' && idx < (document.size() - 3) && document[idx + 1] == '{') {
@@ -205,7 +205,7 @@ int FindNextTag(const string& document, int idx, OpCtx* op, stringstream* out) {
                 op->escaped = false;
                 idx += 2; // Now at start of template expression
             }
-            stringstream expr;
+            std::stringstream expr;
             while (idx < document.size()) {
                 if (document[idx] != '}') {
                     expr << document[idx];
@@ -224,7 +224,7 @@ int FindNextTag(const string& document, int idx, OpCtx* op, stringstream* out) {
                 }
             }
 
-            string key = expr.str();
+            std::string key = expr.str();
             trim(key);
             if (key != ".") trim_if(key, is_any_of("."));
             if (key.size() == 0) continue;
@@ -238,7 +238,7 @@ int FindNextTag(const string& document, int idx, OpCtx* op, stringstream* out) {
 
             if (op->op == EQUALITY || op->op == INEQUALITY) {
                 // Find an argument
-                vector<string> components;
+                std::vector<std::string> components;
                 split(components, key, is_any_of(" "));
                 key = components[0];
                 components.erase(components.begin());
@@ -262,8 +262,9 @@ int FindNextTag(const string& document, int idx, OpCtx* op, stringstream* out) {
 //
 // If 'is_negation' is true, the behaviour is the opposite of the above: false values
 // cause the section to be normally evaluated etc.
-int EvaluateSection(const string& document, const string& document_root, int idx,
-                    const ContextStack* context_stack, const OpCtx& op_ctx, stringstream* out) {
+int EvaluateSection(const std::string& document, const std::string& document_root, int idx,
+                    const ContextStack* context_stack, const OpCtx& op_ctx,
+                    std::stringstream* out) {
     // Precondition: idx is the immediate next character after an opening {{ #tag_name }}
     const Value* context;
     ResolveJsonContext(op_ctx.tag_name, context_stack, &context);
@@ -291,7 +292,7 @@ int EvaluateSection(const string& document, const string& document_root, int idx
         context = context_stack->value;
     }
 
-    vector<const Value*> values;
+    std::vector<const Value*> values;
     if (!skip_contents && context != nullptr && context->IsArray()) {
         for (int i = 0; i < context->Size(); ++i) {
             values.push_back(&(*context)[i]);
@@ -307,7 +308,7 @@ int EvaluateSection(const string& document, const string& document_root, int idx
     int start_idx = idx;
     for (const Value* v : values) {
         idx = start_idx;
-        stack<OpCtx> section_starts;
+        std::stack<OpCtx> section_starts;
         section_starts.push(op_ctx);
         while (idx < document.size()) {
             OpCtx next_ctx;
@@ -334,8 +335,9 @@ int EvaluateSection(const string& document, const string& document_root, int idx
 
 // Evaluates a SUBSTITUTION tag, by replacing its contents with the value of the tag's
 // name in 'parent_context'.
-int EvaluateSubstitution(const string& document, const int idx, const ContextStack* context_stack,
-                         const OpCtx& op_ctx, stringstream* out) {
+int EvaluateSubstitution(const std::string& document, const int idx,
+                         const ContextStack* context_stack, const OpCtx& op_ctx,
+                         std::stringstream* out) {
     const Value* val;
     ResolveJsonContext(op_ctx.tag_name, context_stack, &val);
     if (val == nullptr) return idx;
@@ -353,15 +355,15 @@ int EvaluateSubstitution(const string& document, const int idx, const ContextSta
     } else if (val->IsDouble()) {
         (*out) << val->GetDouble();
     } else if (val->IsBool()) {
-        (*out) << boolalpha << val->GetBool();
+        (*out) << std::boolalpha << val->GetBool();
     }
     return idx;
 }
 
 // Evaluates a LENGTH tag by replacing its contents with the type-dependent 'size' of the
 // value.
-int EvaluateLength(const string& document, const int idx, const ContextStack* context_stack,
-                   const string& tag_name, stringstream* out) {
+int EvaluateLength(const std::string& document, const int idx, const ContextStack* context_stack,
+                   const std::string& tag_name, std::stringstream* out) {
     const Value* val;
     ResolveJsonContext(tag_name, context_stack, &val);
     if (val == nullptr) return idx;
@@ -374,8 +376,8 @@ int EvaluateLength(const string& document, const int idx, const ContextStack* co
     return idx;
 }
 
-int EvaluateLiteral(const string& document, const int idx, const ContextStack* context_stack,
-                    const string& tag_name, stringstream* out) {
+int EvaluateLiteral(const std::string& document, const int idx, const ContextStack* context_stack,
+                    const std::string& tag_name, std::stringstream* out) {
     const Value* val;
     ResolveJsonContext(tag_name, context_stack, &val);
     if (val == nullptr) return idx;
@@ -392,17 +394,17 @@ int EvaluateLiteral(const string& document, const int idx, const ContextStack* c
 //
 // TODO: This could obviously be more efficient (and there are lots of file accesses in a
 // long list context).
-void EvaluatePartial(const string& tag_name, const string& document_root, const ContextStack* stack,
-                     stringstream* out) {
-    stringstream ss;
+void EvaluatePartial(const std::string& tag_name, const std::string& document_root,
+                     const ContextStack* stack, std::stringstream* out) {
+    std::stringstream ss;
     ss << document_root << tag_name;
-    ifstream tmpl(ss.str().c_str());
+    std::ifstream tmpl(ss.str().c_str());
     if (!tmpl.is_open()) {
         ss << ".mustache";
         tmpl.open(ss.str().c_str());
         if (!tmpl.is_open()) return;
     }
-    stringstream file_ss;
+    std::stringstream file_ss;
     file_ss << tmpl.rdbuf();
     RenderTemplate(file_ss.str(), document_root, stack, out);
 }
@@ -410,8 +412,8 @@ void EvaluatePartial(const string& tag_name, const string& document_root, const
 // Given a tag name, and its operator, evaluate the tag in the given context and write the
 // output to 'out'. The heavy-lifting is delegated to specific Evaluate*()
 // methods. Returns the new cursor position within 'document', or -1 on error.
-int EvaluateTag(const string& document, const string& document_root, int idx,
-                const ContextStack* context, const OpCtx& op_ctx, stringstream* out) {
+int EvaluateTag(const std::string& document, const std::string& document_root, int idx,
+                const ContextStack* context, const OpCtx& op_ctx, std::stringstream* out) {
     if (idx == -1) return idx;
     switch (op_ctx.op) {
     case SECTION_START:
@@ -436,13 +438,13 @@ int EvaluateTag(const string& document, const string& document_root, int idx,
     case SECTION_END:
         return idx;
     default:
-        cout << "Unknown tag: " << op_ctx.op << endl;
+        std::cout << "Unknown tag: " << op_ctx.op << std::endl;
         return -1;
     }
 }
 
-static bool RenderTemplate(const string& document, const string& document_root,
-                           const ContextStack* stack, stringstream* out) {
+static bool RenderTemplate(const std::string& document, const std::string& document_root,
+                           const ContextStack* stack, std::stringstream* out) {
     int idx = 0;
     while (idx < document.size() && idx != -1) {
         OpCtx op;
@@ -453,8 +455,8 @@ static bool RenderTemplate(const string& document, const string& document_root,
     return idx != -1;
 }
 
-bool RenderTemplate(const string& document, const string& document_root, const Value& context,
-                    stringstream* out) {
+bool RenderTemplate(const std::string& document, const std::string& document_root,
+                    const Value& context, std::stringstream* out) {
     ContextStack stack = {&context, nullptr};
     return RenderTemplate(document, document_root, &stack, out);
 }
diff --git a/be/src/util/time.cpp b/be/src/util/time.cpp
index 89d5631eee..b75ddeeb47 100644
--- a/be/src/util/time.cpp
+++ b/be/src/util/time.cpp
@@ -29,19 +29,19 @@
 #include "common/logging.h"
 
 using namespace doris;
-using namespace std;
+using namespace std::chrono;
 
 void doris::SleepForMs(const int64_t duration_ms) {
-    this_thread::sleep_for(chrono::milliseconds(duration_ms));
+    std::this_thread::sleep_for(milliseconds(duration_ms));
 }
 
 // Convert the given time_point, 't', into a date-time string in the
 // UTC time zone if 'utc' is true, or the local time zone if it is false.
 // The returned string is of the form yyy-MM-dd HH::mm::SS.
-static string TimepointToString(const chrono::system_clock::time_point& t, bool utc) {
+static std::string TimepointToString(const system_clock::time_point& t, bool utc) {
     char buf[256];
     struct tm tmp;
-    auto input_time = chrono::system_clock::to_time_t(t);
+    auto input_time = system_clock::to_time_t(t);
 
     // gcc 4.9 does not support C++14 get_time and put_time functions, so we're
     // stuck with strftime() for now.
@@ -50,7 +50,7 @@ static string TimepointToString(const chrono::system_clock::time_point& t, bool
     } else {
         strftime(buf, sizeof(buf), "%F %T", localtime_r(&input_time, &tmp));
     }
-    return string(buf);
+    return std::string(buf);
 }
 
 // Format the sub-second part of the input time point object 't', at the
@@ -58,17 +58,17 @@ static string TimepointToString(const chrono::system_clock::time_point& t, bool
 // the string returned by TimePointToString() above.
 // Note the use of abs(). This is to make sure we correctly format negative times,
 // i.e., times before the Unix epoch.
-static string FormatSubSecond(const chrono::system_clock::time_point& t, TimePrecision p) {
+static std::string FormatSubSecond(const system_clock::time_point& t, TimePrecision p) {
     std::stringstream ss;
     auto frac = t.time_since_epoch();
     if (p == TimePrecision::Millisecond) {
-        auto subsec = chrono::duration_cast<chrono::milliseconds>(frac) % MILLIS_PER_SEC;
+        auto subsec = duration_cast<milliseconds>(frac) % MILLIS_PER_SEC;
         ss << "." << std::setfill('0') << std::setw(3) << abs(subsec.count());
     } else if (p == TimePrecision::Microsecond) {
-        auto subsec = chrono::duration_cast<chrono::microseconds>(frac) % MICROS_PER_SEC;
+        auto subsec = duration_cast<microseconds>(frac) % MICROS_PER_SEC;
         ss << "." << std::setfill('0') << std::setw(6) << abs(subsec.count());
     } else if (p == TimePrecision::Nanosecond) {
-        auto subsec = chrono::duration_cast<chrono::nanoseconds>(frac) % NANOS_PER_SEC;
+        auto subsec = duration_cast<nanoseconds>(frac) % NANOS_PER_SEC;
         ss << "." << std::setfill('0') << std::setw(9) << abs(subsec.count());
     } else {
         // 1-second precision or unknown unit. Return empty string.
@@ -81,7 +81,7 @@ static string FormatSubSecond(const chrono::system_clock::time_point& t, TimePre
 // Convert time point 't' into date-time string at precision 'p'.
 // Output string is in UTC time zone if 'utc' is true, else it is in the
 // local time zone.
-static string ToString(const chrono::system_clock::time_point& t, TimePrecision p, bool utc) {
+static std::string ToString(const system_clock::time_point& t, TimePrecision p, bool utc) {
     std::stringstream ss;
     ss << TimepointToString(t, utc);
     ss << FormatSubSecond(t, p);
@@ -90,48 +90,48 @@ static string ToString(const chrono::system_clock::time_point& t, TimePrecision
 
 // Convenience function to convert Unix time, specified as seconds since
 // the Unix epoch, into a C++ time_point object.
-static chrono::system_clock::time_point TimepointFromUnix(int64_t s) {
-    return chrono::system_clock::time_point(chrono::seconds(s));
+static system_clock::time_point TimepointFromUnix(int64_t s) {
+    return system_clock::time_point(seconds(s));
 }
 
 // Convenience function to convert Unix time, specified as milliseconds since
 // the Unix epoch, into a C++ time_point object.
-static chrono::system_clock::time_point TimepointFromUnixMillis(int64_t ms) {
-    return chrono::system_clock::time_point(chrono::milliseconds(ms));
+static system_clock::time_point TimepointFromUnixMillis(int64_t ms) {
+    return system_clock::time_point(milliseconds(ms));
 }
 
 // Convenience function to convert Unix time, specified as microseconds since
 // the Unix epoch, into a C++ time_point object.
-static chrono::system_clock::time_point TimepointFromUnixMicros(int64_t us) {
-    return chrono::system_clock::time_point(chrono::microseconds(us));
+static system_clock::time_point TimepointFromUnixMicros(int64_t us) {
+    return system_clock::time_point(microseconds(us));
 }
 
 std::string doris::ToStringFromUnix(int64_t s, TimePrecision p) {
-    chrono::system_clock::time_point t = TimepointFromUnix(s);
+    system_clock::time_point t = TimepointFromUnix(s);
     return ToString(t, p, false);
 }
 
 std::string doris::ToUtcStringFromUnix(int64_t s, TimePrecision p) {
-    chrono::system_clock::time_point t = TimepointFromUnix(s);
+    system_clock::time_point t = TimepointFromUnix(s);
     return ToString(t, p, true);
 }
 
 std::string doris::ToStringFromUnixMillis(int64_t ms, TimePrecision p) {
-    chrono::system_clock::time_point t = TimepointFromUnixMillis(ms);
+    system_clock::time_point t = TimepointFromUnixMillis(ms);
     return ToString(t, p, false);
 }
 
 std::string doris::ToUtcStringFromUnixMillis(int64_t ms, TimePrecision p) {
-    chrono::system_clock::time_point t = TimepointFromUnixMillis(ms);
+    system_clock::time_point t = TimepointFromUnixMillis(ms);
     return ToString(t, p, true);
 }
 
 std::string doris::ToStringFromUnixMicros(int64_t us, TimePrecision p) {
-    chrono::system_clock::time_point t = TimepointFromUnixMicros(us);
+    system_clock::time_point t = TimepointFromUnixMicros(us);
     return ToString(t, p, false);
 }
 
 std::string doris::ToUtcStringFromUnixMicros(int64_t us, TimePrecision p) {
-    chrono::system_clock::time_point t = TimepointFromUnixMicros(us);
+    system_clock::time_point t = TimepointFromUnixMicros(us);
     return ToString(t, p, true);
 }
diff --git a/be/src/util/uid_util.h b/be/src/util/uid_util.h
index d23258f982..d43367d95f 100644
--- a/be/src/util/uid_util.h
+++ b/be/src/util/uid_util.h
@@ -167,11 +167,7 @@ bool parse_id(const std::string& s, TUniqueId* id);
 
 } // namespace doris
 
-namespace std {
-
 template <>
-struct hash<doris::UniqueId> {
+struct std::hash<doris::UniqueId> {
     size_t operator()(const doris::UniqueId& uid) const { return uid.hash(); }
 };
-
-} // namespace std
diff --git a/be/src/vec/common/string_ref.h b/be/src/vec/common/string_ref.h
index 006274bd7c..625326b747 100644
--- a/be/src/vec/common/string_ref.h
+++ b/be/src/vec/common/string_ref.h
@@ -319,10 +319,8 @@ struct StringRefHash : StringRefHash64 {};
 
 #endif
 
-namespace std {
 template <>
-struct hash<StringRef> : public StringRefHash {};
-} // namespace std
+struct std::hash<StringRef> : public StringRefHash {};
 
 namespace ZeroTraits {
 inline bool check(const StringRef& x) {
diff --git a/be/src/vec/common/strong_typedef.h b/be/src/vec/common/strong_typedef.h
index 1f48d5b82c..c7500a2c51 100644
--- a/be/src/vec/common/strong_typedef.h
+++ b/be/src/vec/common/strong_typedef.h
@@ -70,14 +70,12 @@ public:
     const T& to_under_type() const { return t; }
 };
 
-namespace std {
 template <class T, class Tag>
-struct hash<StrongTypedef<T, Tag>> {
+struct std::hash<StrongTypedef<T, Tag>> {
     size_t operator()(const StrongTypedef<T, Tag>& x) const {
         return std::hash<T>()(x.to_under_type());
     }
 };
-} // namespace std
 
 #define STRONG_TYPEDEF(T, D) \
     struct D##Tag {};        \
diff --git a/be/src/vec/common/uint128.h b/be/src/vec/common/uint128.h
index 039572b17d..028ab61b0f 100644
--- a/be/src/vec/common/uint128.h
+++ b/be/src/vec/common/uint128.h
@@ -190,32 +190,30 @@ struct UInt256 {
 } // namespace doris::vectorized
 
 /// Overload hash for type casting
-namespace std {
 template <>
-struct hash<doris::vectorized::UInt128> {
+struct std::hash<doris::vectorized::UInt128> {
     size_t operator()(const doris::vectorized::UInt128& u) const {
         return Hash128to64({u.low, u.high});
     }
 };
 
 template <>
-struct is_signed<doris::vectorized::UInt128> {
+struct std::is_signed<doris::vectorized::UInt128> {
     static constexpr bool value = false;
 };
 
 template <>
-struct is_unsigned<doris::vectorized::UInt128> {
+struct std::is_unsigned<doris::vectorized::UInt128> {
     static constexpr bool value = true;
 };
 
 template <>
-struct is_integral<doris::vectorized::UInt128> {
+struct std::is_integral<doris::vectorized::UInt128> {
     static constexpr bool value = true;
 };
 
-// Operator +, -, /, *, % aren't implemented so it's not an arithmetic type
+// Operator +, -, /, *, % aren't implemented, so it's not an arithmetic type
 template <>
-struct is_arithmetic<doris::vectorized::UInt128> {
+struct std::is_arithmetic<doris::vectorized::UInt128> {
     static constexpr bool value = false;
 };
-} // namespace std
diff --git a/be/src/vec/core/types.h b/be/src/vec/core/types.h
index 7636d714b3..332d310d57 100644
--- a/be/src/vec/core/types.h
+++ b/be/src/vec/core/types.h
@@ -533,14 +533,13 @@ inline const char* getTypeName(TypeIndex idx) {
 } // namespace doris
 
 /// Specialization of `std::hash` for the Decimal<T> types.
-namespace std {
 template <typename T>
-struct hash<doris::vectorized::Decimal<T>> {
+struct std::hash<doris::vectorized::Decimal<T>> {
     size_t operator()(const doris::vectorized::Decimal<T>& x) const { return hash<T>()(x.value); }
 };
 
 template <>
-struct hash<doris::vectorized::Decimal128> {
+struct std::hash<doris::vectorized::Decimal128> {
     size_t operator()(const doris::vectorized::Decimal128& x) const {
         return std::hash<doris::vectorized::Int64>()(x.value >> 64) ^
                std::hash<doris::vectorized::Int64>()(
@@ -548,7 +547,7 @@ struct hash<doris::vectorized::Decimal128> {
     }
 };
 
-constexpr bool is_integer(doris::vectorized::TypeIndex index) {
+constexpr bool typeindex_is_int(doris::vectorized::TypeIndex index) {
     using TypeIndex = doris::vectorized::TypeIndex;
     switch (index) {
     case TypeIndex::UInt8:
@@ -568,4 +567,3 @@ constexpr bool is_integer(doris::vectorized::TypeIndex index) {
     }
     }
 }
-} // namespace std
diff --git a/be/src/vec/functions/function_totype.h b/be/src/vec/functions/function_totype.h
index 033e168a8e..95cb34eaaa 100644
--- a/be/src/vec/functions/function_totype.h
+++ b/be/src/vec/functions/function_totype.h
@@ -67,7 +67,7 @@ private:
     Status execute_impl(Block& block, const ColumnNumbers& arguments, size_t result,
                         size_t input_rows_count) {
         const ColumnPtr column = block.get_by_position(arguments[0]).column;
-        if constexpr (std::is_integer(Impl::TYPE_INDEX)) {
+        if constexpr (typeindex_is_int(Impl::TYPE_INDEX)) {
             if (auto* col = check_and_get_column<ColumnVector<typename Impl::Type>>(column.get())) {
                 auto col_res = Impl::ReturnColumnType::create();
                 RETURN_IF_ERROR(Impl::vector(col->get_data(), col_res->get_chars(),
@@ -102,7 +102,7 @@ private:
                 block.replace_by_position(result, std::move(col_res));
                 return Status::OK();
             }
-        } else if constexpr (std::is_integer(Impl::TYPE_INDEX)) {
+        } else if constexpr (typeindex_is_int(Impl::TYPE_INDEX)) {
             if (const auto* col =
                         check_and_get_column<ColumnVector<typename Impl::Type>>(column.get())) {
                 auto col_res = Impl::ReturnColumnType::create();
diff --git a/be/src/vec/runtime/vdatetime_value.h b/be/src/vec/runtime/vdatetime_value.h
index bced4f962d..7abdeb0064 100644
--- a/be/src/vec/runtime/vdatetime_value.h
+++ b/be/src/vec/runtime/vdatetime_value.h
@@ -1504,16 +1504,15 @@ struct DateTraits<uint64_t> {
 } // namespace vectorized
 } // namespace doris
 
-namespace std {
 template <>
-struct hash<doris::vectorized::VecDateTimeValue> {
+struct std::hash<doris::vectorized::VecDateTimeValue> {
     size_t operator()(const doris::vectorized::VecDateTimeValue& v) const {
         return doris::vectorized::hash_value(v);
     }
 };
 
 template <>
-struct hash<doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>> {
+struct std::hash<doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>> {
     size_t operator()(
             const doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>& v) const {
         return doris::vectorized::hash_value(v);
@@ -1521,10 +1520,9 @@ struct hash<doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>>
 };
 
 template <>
-struct hash<doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>> {
+struct std::hash<doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>> {
     size_t operator()(
             const doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>& v) const {
         return doris::vectorized::hash_value(v);
     }
 };
-} // namespace std


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