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 2022/06/20 02:52:58 UTC

[doris] branch dev-1.0.1 updated (2e04e4cc63 -> 239fbada6a)

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

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


    from 2e04e4cc63 [opt](compaction) optimize compaction in concurrent load (#10153)
     new 688c083b98 [bug](be) fix be block_reader.cc::_update_agg_value() mem leak.(#10216) (#10218)
     new 9a28b26f72 [Improvement][ASAN] make BE can exit normally and ASAN memory leak checking work (#9620)
     new 239fbada6a [improvement](variables) change session variable when set global variable (#10238)

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


Summary of changes:
 be/src/util/priority_thread_pool.hpp               | 20 ++++++++++----------
 be/src/util/priority_work_stealing_thread_pool.hpp | 19 +++++--------------
 be/src/vec/olap/block_reader.cpp                   |  4 +++-
 docs/en/administrator-guide/variables.md           |  4 ++--
 .../main/java/org/apache/doris/qe/VariableMgr.java | 22 +++++++++++-----------
 .../java/org/apache/doris/qe/VariableMgrTest.java  |  2 ++
 6 files changed, 33 insertions(+), 38 deletions(-)


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


[doris] 02/03: [Improvement][ASAN] make BE can exit normally and ASAN memory leak checking work (#9620)

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

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

commit 9a28b26f723f8a8aa6176c00b7b7febe9616d51d
Author: jacktengg <18...@users.noreply.github.com>
AuthorDate: Wed May 18 07:40:57 2022 +0800

    [Improvement][ASAN] make BE can exit normally and ASAN memory leak checking work (#9620)
---
 be/src/util/priority_thread_pool.hpp               | 20 ++++++++++----------
 be/src/util/priority_work_stealing_thread_pool.hpp | 19 +++++--------------
 2 files changed, 15 insertions(+), 24 deletions(-)

diff --git a/be/src/util/priority_thread_pool.hpp b/be/src/util/priority_thread_pool.hpp
index ed0fe90157..db5700137e 100644
--- a/be/src/util/priority_thread_pool.hpp
+++ b/be/src/util/priority_thread_pool.hpp
@@ -53,7 +53,6 @@ public:
     //  -- queue_size: the maximum size of the queue on which work items are offered. If the
     //     queue exceeds this size, subsequent calls to Offer will block until there is
     //     capacity available.
-    //  -- work_function: the function to run every time an item is consumed from the queue
     PriorityThreadPool(uint32_t num_threads, uint32_t queue_size)
             : _work_queue(queue_size), _shutdown(false) {
         for (int i = 0; i < num_threads; ++i) {
@@ -117,6 +116,16 @@ public:
     }
 protected:
     virtual bool is_shutdown() { return _shutdown; }
+
+    // Collection of worker threads that process work from the queue.
+    ThreadGroup _threads;
+
+    // Guards _empty_cv
+    std::mutex _lock;
+
+    // Signalled when the queue becomes empty
+    std::condition_variable _empty_cv;
+
 private:
     // Driver method for each thread in the pool. Continues to read work from the queue
     // until the pool is shutdown.
@@ -136,17 +145,8 @@ private:
     // FIFO order.
     BlockingPriorityQueue<Task> _work_queue;
 
-    // Collection of worker threads that process work from the queue.
-    ThreadGroup _threads;
-
-    // Guards _empty_cv
-    std::mutex _lock;
-
     // Set to true when threads should stop doing work and terminate.
     std::atomic<bool> _shutdown;
-
-    // Signalled when the queue becomes empty
-    std::condition_variable _empty_cv;
 };
 
 } // namespace doris
diff --git a/be/src/util/priority_work_stealing_thread_pool.hpp b/be/src/util/priority_work_stealing_thread_pool.hpp
index 3c86098034..c11ee7d198 100644
--- a/be/src/util/priority_work_stealing_thread_pool.hpp
+++ b/be/src/util/priority_work_stealing_thread_pool.hpp
@@ -36,7 +36,6 @@ public:
     //  -- queue_size: the maximum size of the queue on which work items are offered. If the
     //     queue exceeds this size, subsequent calls to Offer will block until there is
     //     capacity available.
-    //  -- work_function: the function to run every time an item is consumed from the queue
     PriorityWorkStealingThreadPool(uint32_t num_threads, uint32_t num_queues, uint32_t queue_size)
             : PriorityThreadPool(0, 0) {
         DCHECK_GT(num_queues, 0);
@@ -51,6 +50,11 @@ public:
         }
     }
 
+    virtual ~PriorityWorkStealingThreadPool() {
+        shutdown();
+        join();
+    }
+
     // Blocking operation that puts a work item on the queue. If the queue is full, blocks
     // until there is capacity available.
     //
@@ -82,10 +86,6 @@ public:
         }
     }
 
-    // Blocks until all threads are finished. shutdown does not need to have been called,
-    // since it may be called on a separate thread.
-    void join() override { _threads.join_all(); }
-
     uint32_t get_queue_size() const override {
         uint32_t size = 0;
         for (auto work_queue : _work_queues) {
@@ -141,15 +141,6 @@ private:
     // Queue on which work items are held until a thread is available to process them in
     // FIFO order.
     std::vector<std::shared_ptr<BlockingPriorityQueue<Task>>> _work_queues;
-
-    // Collection of worker threads that process work from the queues.
-    ThreadGroup _threads;
-
-    // Guards _empty_cv
-    std::mutex _lock;
-
-    // Signalled when the queue becomes empty
-    std::condition_variable _empty_cv;
 };
 
 } // namespace doris
\ No newline at end of file


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


[doris] 03/03: [improvement](variables) change session variable when set global variable (#10238)

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

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

commit 239fbada6a8b557d7778b35d4e15089be108badd
Author: Mingyu Chen <mo...@gmail.com>
AuthorDate: Mon Jun 20 09:05:50 2022 +0800

    [improvement](variables) change session variable when set global variable (#10238)
    
    Currently, when setting variables with `global` keywords, it will not affect the
    current session variable's value. That is always make user confused.
    
    This CL mainly changes:
    
    1. Change session variable when set global variable
---
 docs/en/administrator-guide/variables.md           |  4 ++--
 .../main/java/org/apache/doris/qe/VariableMgr.java | 22 +++++++++++-----------
 .../java/org/apache/doris/qe/VariableMgrTest.java  |  2 ++
 3 files changed, 15 insertions(+), 13 deletions(-)

diff --git a/docs/en/administrator-guide/variables.md b/docs/en/administrator-guide/variables.md
index e6d6e22d28..3eb2ac3121 100644
--- a/docs/en/administrator-guide/variables.md
+++ b/docs/en/administrator-guide/variables.md
@@ -43,7 +43,8 @@ SHOW VARIABLES LIKE '%time_zone%';
 
 ### Settings
 
-Some variables can be set at global-level or session-only. For global-level, the set value will be used in subsequent new session connections. For session-only, the variable only works for the current session.
+Note that before version 1.1, after the setting takes effect globally, the setting value will be inherited in subsequent new session connections, but the value in the current session will remain unchanged.
+After version 1.1 (inclusive), after the setting takes effect globally, the setting value will be used in subsequent new session connections, and the value in the current session will also change.
 
 For session-only, set by the `SET var_name=xxx;` statement. Such as:
 
@@ -60,7 +61,6 @@ SET GLOBAL exec_mem_limit = 137438953472
 ```
 
 > Note 1: Only ADMIN users can set variable at global-level.
-> Note 2: Global-level variables do not affect variable values in the current session, only variables in new sessions.
 
 Variables that support both session-level and global-level setting include:
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/VariableMgr.java b/fe/fe-core/src/main/java/org/apache/doris/qe/VariableMgr.java
index 924f923e85..1016bc0a87 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/VariableMgr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/VariableMgr.java
@@ -250,19 +250,19 @@ public class VariableMgr {
 
         if (setVar.getType() == SetType.GLOBAL) {
             setGlobalVarAndWriteEditLog(ctx, attr.name(), setVar.getValue().getStringValue());
-        } else {
-            // set session variable
-            Field field = ctx.getField();
-            // if stmt is "Select /*+ SET_VAR(...)*/"
-            if (sessionVariable.getIsSingleSetVar()) {
-                try {
-                    sessionVariable.addSessionOriginValue(field, field.get(sessionVariable).toString());
-                } catch (Exception e) {
-                    LOG.warn("failed to collect origin session value ", e);
-                }
+        }
+
+        // No matter this is a global setting or not, always set session variable.
+        Field field = ctx.getField();
+        // if stmt is "Select /*+ SET_VAR(...)*/"
+        if (sessionVariable.getIsSingleSetVar()) {
+            try {
+                sessionVariable.addSessionOriginValue(field, field.get(sessionVariable).toString());
+            } catch (Exception e) {
+                LOG.warn("failed to collect origin session value ", e);
             }
-            setValue(sessionVariable, field, value);
         }
+        setValue(sessionVariable, field, value);
     }
 
     private static void setGlobalVarAndWriteEditLog(VarContext ctx, String name, String value) throws DdlException {
diff --git a/fe/fe-core/src/test/java/org/apache/doris/qe/VariableMgrTest.java b/fe/fe-core/src/test/java/org/apache/doris/qe/VariableMgrTest.java
index 38e235ce0f..560f2a70d2 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/qe/VariableMgrTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/qe/VariableMgrTest.java
@@ -163,6 +163,8 @@ public class VariableMgrTest {
         SetExecutor executor = new SetExecutor(ctx, stmt);
         executor.execute();
         Assert.assertEquals(5678, VariableMgr.newSessionVariable().getMaxExecMemByte());
+        // the session var is also changed.
+        Assert.assertEquals(5678, ctx.getSessionVariable().getMaxExecMemByte());
 
         Config.edit_log_roll_num = 100;
         stmt = (SetStmt) UtFrameUtils.parseAndAnalyzeStmt("set global exec_mem_limit=7890", ctx);


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


[doris] 01/03: [bug](be) fix be block_reader.cc::_update_agg_value() mem leak.(#10216) (#10218)

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

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

commit 688c083b98ae6d58f44f853d1ed718c433b501b3
Author: Lei Zhang <10...@qq.com>
AuthorDate: Fri Jun 17 21:25:52 2022 +0800

    [bug](be) fix be block_reader.cc::_update_agg_value() mem leak.(#10216) (#10218)
---
 be/src/vec/olap/block_reader.cpp | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)

diff --git a/be/src/vec/olap/block_reader.cpp b/be/src/vec/olap/block_reader.cpp
index e9da92f1ba..2d5c542379 100644
--- a/be/src/vec/olap/block_reader.cpp
+++ b/be/src/vec/olap/block_reader.cpp
@@ -361,7 +361,9 @@ void BlockReader::_update_agg_value(MutableColumns& columns, int begin, int end,
 
         if (is_close) {
             function->insert_result_into(place, *columns[_return_columns_loc[idx]]);
-            function->create(place); // reset aggregate data
+            // reset aggregate data
+            function->destroy(place);
+            function->create(place);
         }
     }
 }


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