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 2019/12/30 12:38:52 UTC

[GitHub] [incubator-doris] Seaven opened a new pull request #2618: Add BE plugin framework (#2348)

Seaven opened a new pull request #2618: Add BE plugin framework (#2348)
URL: https://github.com/apache/incubator-doris/pull/2618
 
 
   support BE plugin framework, include:
   
   * update Plugin Manager, support Plugin find method
   
   * support Builtin-Plugin register method
   
   * plugin install/uninstall process
   	* PluginLoader:
   		* dynamic install and check Plugin .so file
   		* dynamic uninstall and check Plugin status
   		
   	* PluginZip:
   		* support plugin remote/local .zip file download and extract
   
   and Todo:
   
   * We should support a PluginContext to transmit necessary system variable when the plugin's init/close method invoke
   * Add the entry which is BE dynamic Plugin install/uninstall process, include:
   
   	* The FE send install/uninstall Plugin statement (RPC way)
   	* The FE meta update request with Plugin list information
   	* The FE operation request(update/query) with Plugin (maybe don't need)
   * Add the plugin status upload way
   * Load already install Plugin when BE start

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] morningman merged pull request #2618: Add BE plugin framework (#2348)

Posted by GitBox <gi...@apache.org>.
morningman merged pull request #2618: Add BE plugin framework (#2348)
URL: https://github.com/apache/incubator-doris/pull/2618
 
 
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] morningman commented on a change in pull request #2618: Add BE plugin framework (#2348)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2618: Add BE plugin framework (#2348)
URL: https://github.com/apache/incubator-doris/pull/2618#discussion_r396936273
 
 

 ##########
 File path: gensrc/thrift/AgentService.thrift
 ##########
 @@ -279,6 +294,8 @@ struct TAgentTaskRequest {
     24: optional TAlterTabletReqV2 alter_tablet_req_v2
     25: optional i64 recv_time // time the task is inserted to queue
     26: optional TUpdateTabletMetaInfoReq update_tablet_meta_info_req
+    27: optional TInstallPluginReq install_plugin_req
 
 Review comment:
   You can currently remove these 2 fields, add them when implement them.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] Seaven commented on a change in pull request #2618: Add BE plugin framework (#2348)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2618: Add BE plugin framework (#2348)
URL: https://github.com/apache/incubator-doris/pull/2618#discussion_r396333316
 
 

 ##########
 File path: be/src/plugin/plugin.h
 ##########
 @@ -15,11 +15,17 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#ifndef DORIS_BE_PLUGIN_PLUGIN_H
+#define DORIS_BE_PLUGIN_PLUGIN_H
+
 namespace doris {
 
-#define PLUGIN_NOT_DYNAMIC_INSTALL 1UL
+#define PLUGIN_TYPE_IMPORT 1
+#define PLUGIN_TYPE_STORAGE 2
+#define PLUGIN_TYPE_MAX 3
 
 Review comment:
   Temporary, we can add later

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] morningman commented on a change in pull request #2618: Add BE plugin framework (#2348)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2618: Add BE plugin framework (#2348)
URL: https://github.com/apache/incubator-doris/pull/2618#discussion_r396931141
 
 

 ##########
 File path: be/src/plugin/plugin_loader.cpp
 ##########
 @@ -0,0 +1,194 @@
+// 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 <boost/algorithm/string/predicate.hpp>
+#include <cstring>
+
+#include "plugin/plugin_loader.h"
+#include "plugin/plugin_zip.h"
+
+#include "gutil/strings/substitute.h"
+#include "gutil/strings/util.h"
+#include "util/dynamic_util.h"
+#include "util/file_utils.h"
+#include "http/http_client.h"
+#include "util/time.h"
+#include "util/md5.h"
+#include "env/env.h"
+
+namespace doris {
+
+using namespace strings;
+
+static const std::string PLUGIN_VERSION_SYMBOL = "_plugin_interface_version";
+static const std::string PLUGIN_SIZE_SYMBOL = "_sizeof_plugin";
+static const std::string PLUGIN_STRUCT_SYMBOL = "_plugin";
+
+
+Status PluginLoader::open_valid() {
+    return Status::OK();
+}
+
+Status PluginLoader::close_valid() {
+    if (_plugin.get() != nullptr && _plugin->flags & PLUGIN_NOT_DYNAMIC_UNINSTALL) {
+        return Status::InternalError(Substitute("plugin $0 not allow dynamic uninstall", _name));
+    }
+
+    return Status::OK();
+}
+
+Status DynamicPluginLoader::install() {
+    // check already install
+    std::string so_path = _install_path + "/" + _name + "/" + _so_name;
+    if (!FileUtils::check_exist(so_path)) {
+        // no, need download zip install
+        PluginZip zip(_source);
+
+        RETURN_IF_ERROR(zip.extract(_install_path, _name));
+    }
+
+    // open plugin
+    RETURN_IF_ERROR(open_plugin());
+
+    RETURN_IF_ERROR(open_valid());
+
+    // plugin init
+    // todo: what should be send?
+    if (_plugin->init != nullptr) {
+        _plugin->init(&_plugin->handler);
+    }
+
+    return Status::OK();
+}
+
+/**
+ * open & valid Plugin:
+ * 1. check .so file exists
+ * 2. check .so version symbol
+ * 3. check .so plugin symbol
+ */
+Status DynamicPluginLoader::open_plugin() {
+    // check .so file
+    std::string so_path = _install_path + "/" + _name + "/" + _so_name;
+    if (!FileUtils::check_exist(so_path)) {
+        return Status::InternalError("plugin install not found " + _so_name);
+    }
+
+    RETURN_IF_ERROR(dynamic_open(so_path.c_str(), &_plugin_handler));
+
+    void* symbol;
+    // check version symbol
+    RETURN_IF_ERROR(dynamic_lookup(_plugin_handler, (_name + PLUGIN_VERSION_SYMBOL).c_str(), &symbol));
+
+    if (DORIS_PLUGIN_VERSION > *(int*) symbol) {
+        return Status::InternalError("plugin compile version too old");
+    }
+
+    RETURN_IF_ERROR(dynamic_lookup(_plugin_handler, (_name + PLUGIN_SIZE_SYMBOL).c_str(), &symbol));
+
+    int plugin_size = *(int*) symbol;
+    if (plugin_size != sizeof(Plugin)) {
+        return Status::InternalError("plugin struct error");
+    }
+
+    // check Plugin declaration
+    RETURN_IF_ERROR(dynamic_lookup(_plugin_handler, (_name + PLUGIN_STRUCT_SYMBOL).c_str(), &symbol));
+
+    Plugin* end_plugin = (Plugin*) ((char*) symbol + plugin_size);
+
+    if (end_plugin->handler != nullptr || end_plugin->init != nullptr || end_plugin->close != nullptr) {
 
 Review comment:
   What does this mean?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] morningman commented on a change in pull request #2618: Add BE plugin framework (#2348)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2618: Add BE plugin framework (#2348)
URL: https://github.com/apache/incubator-doris/pull/2618#discussion_r396928032
 
 

 ##########
 File path: be/src/plugin/plugin_mgr.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
+//
+// 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 <boost/foreach.hpp>
+
+#include "plugin/plugin_mgr.h"
+#include "gutil/strings/substitute.h"
+
+namespace doris {
+
+using namespace strings;
+
+#define PLUGIN_TYPE_CHECK(_type) {   \
+    if (_type >= PLUGIN_TYPE_MAX) {   \
+        return Status::InvalidArgument(Substitute("error plugin type: $0", _type));  \
+    }   \
+}
+
+
+Status PluginMgr::install_plugin(const TPluginMetaInfo& info) {
+    {
+        std::lock_guard<std::mutex> l(_lock);
+        auto iter = _plugins[info.type].find(info.name);
+
+        if (iter != _plugins[info.type].end()) {
+            return Status::AlreadyExist("plugin " + info.name + " is already install");
+        }
+    }
+    
+    DCHECK(info.__isset.so_name);
+    DCHECK(info.__isset.source);
+
+    std::unique_ptr<PluginLoader> loader = std::unique_ptr<PluginLoader>(
+            new DynamicPluginLoader(info.name, info.type, info.source, info.so_name, config::plugin_path));
+
+    Status st = loader->install();
+    
+    if (!st.ok() && !st.is_already_exist()) {
+        RETURN_IF_ERROR(loader->uninstall());
+        return st;
+    }
+
+    {
+        std::lock_guard<std::mutex> l(_lock);
+        auto iter = _plugins[info.type].find(info.name);
+        
+        if (iter != _plugins[info.type].end()) {
+            return Status::AlreadyExist("plugin " + info.name + " is already install");
 
 Review comment:
   ```suggestion
               return Status::AlreadyExist("plugin " + info.name + " is already installed");
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] Seaven commented on a change in pull request #2618: Add BE plugin framework (#2348)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2618: Add BE plugin framework (#2348)
URL: https://github.com/apache/incubator-doris/pull/2618#discussion_r397139828
 
 

 ##########
 File path: be/src/plugin/plugin_loader.cpp
 ##########
 @@ -0,0 +1,194 @@
+// 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 <boost/algorithm/string/predicate.hpp>
+#include <cstring>
+
+#include "plugin/plugin_loader.h"
+#include "plugin/plugin_zip.h"
+
+#include "gutil/strings/substitute.h"
+#include "gutil/strings/util.h"
+#include "util/dynamic_util.h"
+#include "util/file_utils.h"
+#include "http/http_client.h"
+#include "util/time.h"
+#include "util/md5.h"
+#include "env/env.h"
+
+namespace doris {
+
+using namespace strings;
+
+static const std::string PLUGIN_VERSION_SYMBOL = "_plugin_interface_version";
+static const std::string PLUGIN_SIZE_SYMBOL = "_sizeof_plugin";
+static const std::string PLUGIN_STRUCT_SYMBOL = "_plugin";
+
+
+Status PluginLoader::open_valid() {
+    return Status::OK();
+}
+
+Status PluginLoader::close_valid() {
+    if (_plugin.get() != nullptr && _plugin->flags & PLUGIN_NOT_DYNAMIC_UNINSTALL) {
+        return Status::InternalError(Substitute("plugin $0 not allow dynamic uninstall", _name));
+    }
+
+    return Status::OK();
+}
+
+Status DynamicPluginLoader::install() {
+    // check already install
+    std::string so_path = _install_path + "/" + _name + "/" + _so_name;
+    if (!FileUtils::check_exist(so_path)) {
+        // no, need download zip install
+        PluginZip zip(_source);
+
+        RETURN_IF_ERROR(zip.extract(_install_path, _name));
+    }
+
+    // open plugin
+    RETURN_IF_ERROR(open_plugin());
+
+    RETURN_IF_ERROR(open_valid());
+
+    // plugin init
+    // todo: what should be send?
+    if (_plugin->init != nullptr) {
+        _plugin->init(&_plugin->handler);
+    }
+
+    return Status::OK();
+}
+
+/**
+ * open & valid Plugin:
+ * 1. check .so file exists
+ * 2. check .so version symbol
+ * 3. check .so plugin symbol
+ */
+Status DynamicPluginLoader::open_plugin() {
+    // check .so file
+    std::string so_path = _install_path + "/" + _name + "/" + _so_name;
+    if (!FileUtils::check_exist(so_path)) {
+        return Status::InternalError("plugin install not found " + _so_name);
+    }
+
+    RETURN_IF_ERROR(dynamic_open(so_path.c_str(), &_plugin_handler));
+
+    void* symbol;
+    // check version symbol
+    RETURN_IF_ERROR(dynamic_lookup(_plugin_handler, (_name + PLUGIN_VERSION_SYMBOL).c_str(), &symbol));
+
+    if (DORIS_PLUGIN_VERSION > *(int*) symbol) {
+        return Status::InternalError("plugin compile version too old");
+    }
+
+    RETURN_IF_ERROR(dynamic_lookup(_plugin_handler, (_name + PLUGIN_SIZE_SYMBOL).c_str(), &symbol));
+
+    int plugin_size = *(int*) symbol;
+    if (plugin_size != sizeof(Plugin)) {
+        return Status::InternalError("plugin struct error");
+    }
+
+    // check Plugin declaration
+    RETURN_IF_ERROR(dynamic_lookup(_plugin_handler, (_name + PLUGIN_STRUCT_SYMBOL).c_str(), &symbol));
+
+    Plugin* end_plugin = (Plugin*) ((char*) symbol + plugin_size);
+
+    if (end_plugin->handler != nullptr || end_plugin->init != nullptr || end_plugin->close != nullptr) {
 
 Review comment:
   For check plugin struct

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] morningman commented on a change in pull request #2618: Add BE plugin framework (#2348)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2618: Add BE plugin framework (#2348)
URL: https://github.com/apache/incubator-doris/pull/2618#discussion_r396928822
 
 

 ##########
 File path: be/src/plugin/plugin_mgr.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
+//
+// 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 <boost/foreach.hpp>
+
+#include "plugin/plugin_mgr.h"
+#include "gutil/strings/substitute.h"
+
+namespace doris {
+
+using namespace strings;
+
+#define PLUGIN_TYPE_CHECK(_type) {   \
+    if (_type >= PLUGIN_TYPE_MAX) {   \
+        return Status::InvalidArgument(Substitute("error plugin type: $0", _type));  \
+    }   \
+}
+
+
+Status PluginMgr::install_plugin(const TPluginMetaInfo& info) {
+    {
+        std::lock_guard<std::mutex> l(_lock);
+        auto iter = _plugins[info.type].find(info.name);
+
+        if (iter != _plugins[info.type].end()) {
+            return Status::AlreadyExist("plugin " + info.name + " is already install");
+        }
+    }
+    
+    DCHECK(info.__isset.so_name);
+    DCHECK(info.__isset.source);
+
+    std::unique_ptr<PluginLoader> loader = std::unique_ptr<PluginLoader>(
+            new DynamicPluginLoader(info.name, info.type, info.source, info.so_name, config::plugin_path));
+
+    Status st = loader->install();
+    
+    if (!st.ok() && !st.is_already_exist()) {
+        RETURN_IF_ERROR(loader->uninstall());
+        return st;
+    }
+
+    {
+        std::lock_guard<std::mutex> l(_lock);
+        auto iter = _plugins[info.type].find(info.name);
+        
+        if (iter != _plugins[info.type].end()) {
+            return Status::AlreadyExist("plugin " + info.name + " is already install");
+        } else {
+            _plugins[info.type][info.name] = std::move(loader);
+        };
+    }
+    
+    return Status::OK();
+}
+
+Status PluginMgr::uninstall_plugin(const TPluginMetaInfo& info) {
+    std::lock_guard<std::mutex> l(_lock);
+
+    auto iter = _plugins[info.type].find(info.name);
+
+    if (iter != _plugins[info.type].end()) {
+        _plugins[info.type].erase(iter);
+    }
+
+    return Status::OK();
+}
+
+Status PluginMgr::get_plugin(const std::string& name, int type, std::shared_ptr<Plugin>* plugin) {
+    PLUGIN_TYPE_CHECK(type);
+
+    std::lock_guard<std::mutex> l(_lock);
+
+    auto iter = _plugins[type].find(name);
+
+    if (iter != _plugins[type].end()) {
+        *plugin = iter->second->plugin();
+        return Status::OK();
+    }
+
+    return Status::NotFound(Substitute("not found type $0 plugin $1", type, name));
+}
+
+Status PluginMgr::get_plugin(const std::string& name, std::shared_ptr<Plugin>* plugin) {
+    for (int i = 0; i < PLUGIN_TYPE_MAX; ++i) {
+        std::lock_guard<std::mutex> l(_lock);
+
+        auto iter = _plugins[i].find(name);
+
+        if (iter != _plugins[i].end()) {
+            *plugin = iter->second->plugin();
+            return Status::OK();
+        }
+    }
+
+    return Status::NotFound(Substitute("not found plugin $0", name));
+}
+
+Status PluginMgr::get_plugin_list(int type, std::vector<std::shared_ptr<Plugin>>* plugin_list) {
+    PLUGIN_TYPE_CHECK(type);
+
+    std::lock_guard<std::mutex> l(_lock);
+
+    BOOST_FOREACH(const PluginLoaderMap::value_type& iter, _plugins[type]){
+        plugin_list->push_back(iter.second->plugin());
+    }
+    
+    return Status::OK();
+}
+
+Status PluginMgr::register_builtin_plugin(const std::string& name, int type, doris::Plugin* plugin) {
+    PLUGIN_TYPE_CHECK(type);
+
+    std::lock_guard<std::mutex> l(_lock);
+
+    auto iter = _plugins[type].find(name);
+    if (iter != _plugins[type].end()) {
+        return Status::AlreadyExist(Substitute("the type $0 plugin $1 already register"));
 
 Review comment:
   Missing parameter for `Substitute `

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] morningman commented on a change in pull request #2618: Add BE plugin framework (#2348)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2618: Add BE plugin framework (#2348)
URL: https://github.com/apache/incubator-doris/pull/2618#discussion_r393044587
 
 

 ##########
 File path: be/src/plugin/plugin_loader.cpp
 ##########
 @@ -0,0 +1,179 @@
+// 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 <boost/algorithm/string/predicate.hpp>
+#include <cstring>
+
+#include "plugin/plugin_loader.h"
+#include "plugin/plugin_zip.h"
+
+#include "gutil/strings/substitute.h"
+#include "gutil/strings/util.h"
+#include "util/dynamic_util.h"
+#include "util/file_utils.h"
+#include "http/http_client.h"
+#include "util/time.h"
+#include "util/md5.h"
+#include "env/env.h"
+
+namespace doris {
+
+using namespace strings;
+
+static const std::string PLUGIN_VERSION_SYMBOL = "_plugin_interface_version";
+static const std::string PLUGIN_SIZE_SYMBOL = "_sizeof_plugin";
+static const std::string PLUGIN_STRUCT_SYMBOL = "_plugin";
+
+
+Status PluginLoader::open_valid() {
+    return Status::OK();
+}
+
+Status PluginLoader::close_valid() {
+    if (_plugin.get() != nullptr && _plugin->flags & PLUGIN_NOT_DYNAMIC_UNINSTALL) {
+        return Status::InternalError(Substitute("plugin $0 not allow dynamic uninstall", _name));
+    }
+
+    return Status::OK();
+}
+
+Status DynamicPluginLoader::install() {
+    // check already install
+    std::string so_path = _install_path + "/" + _name + "/" + _so_name;
+    if (!FileUtils::check_exist(so_path)) {
+        // no, need download zip install
+        PluginZip zip(_source);
+
+        RETURN_IF_ERROR(zip.extract(_install_path, _name));
+    }
+
+    // open plugin
+    RETURN_IF_ERROR(open_plugin());
+
+    RETURN_IF_ERROR(open_valid());
+
+    // plugin init
+    // todo: what should be send?
+    _plugin->init(nullptr);
+
+    return Status::OK();
+}
+
+/**
+ * open & valid Plugin:
+ * 1. check .so file exists
+ * 2. check .so version symbol
+ * 3. check .so plugin symbol
+ */
+Status DynamicPluginLoader::open_plugin() {
+    // check .so file
+    std::string so_path = _install_path + "/" + _name + "/" + _so_name;
+    if (!FileUtils::check_exist(so_path)) {
+        return Status::InternalError("plugin install not found " + _so_name);
+    }
+
+    RETURN_IF_ERROR(dynamic_open(so_path.c_str(), &_plugin_handler));
+
+    void* symbol;
+    // check version symbol
+    RETURN_IF_ERROR(dynamic_lookup(_plugin_handler, (_name + PLUGIN_VERSION_SYMBOL).c_str(), &symbol));
+
+    if (DORIS_PLUGIN_VERSION > *(int*) symbol) {
+        return Status::InternalError("plugin compile version to old");
 
 Review comment:
   ```suggestion
           return Status::InternalError("plugin compile version too old");
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] morningman commented on a change in pull request #2618: Add BE plugin framework (#2348)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2618: Add BE plugin framework (#2348)
URL: https://github.com/apache/incubator-doris/pull/2618#discussion_r396937579
 
 

 ##########
 File path: be/src/plugin/plugin_mgr.h
 ##########
 @@ -0,0 +1,69 @@
+// 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.
+
+#ifndef DORIS_BE_PLUGIN_PLUGIN_MGR_H
+#define DORIS_BE_PLUGIN_PLUGIN_MGR_H
+
+
+#include <string>
+#include <unordered_map>
+#include <memory>
+#include <mutex>
+
+#include "gen_cpp/MasterService_types.h"
+#include "gen_cpp/AgentService_types.h"
+
+#include "common/status.h"
+#include "plugin/plugin_loader.h"
+#include "plugin/plugin.h"
+
+
+namespace doris {
+
+typedef std::unordered_map<std::string, std::unique_ptr<PluginLoader>> PluginLoaderMap;
+
+class PluginMgr {
+
+public:
+    
+    PluginMgr() {}
+    
+    ~PluginMgr() {}
+    
+    Status install_plugin(const TPluginMetaInfo& info);
+    
+    Status uninstall_plugin(const TPluginMetaInfo& info);
+
+    Status register_builtin_plugin(const std::string& name, int type, Plugin* plugin);
 
 Review comment:
   Can the parameter be : `const Plugin* plugin`?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] morningman commented on a change in pull request #2618: Add BE plugin framework (#2348)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2618: Add BE plugin framework (#2348)
URL: https://github.com/apache/incubator-doris/pull/2618#discussion_r396927988
 
 

 ##########
 File path: be/src/plugin/plugin_mgr.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
+//
+// 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 <boost/foreach.hpp>
+
+#include "plugin/plugin_mgr.h"
+#include "gutil/strings/substitute.h"
+
+namespace doris {
+
+using namespace strings;
+
+#define PLUGIN_TYPE_CHECK(_type) {   \
+    if (_type >= PLUGIN_TYPE_MAX) {   \
+        return Status::InvalidArgument(Substitute("error plugin type: $0", _type));  \
+    }   \
+}
+
+
+Status PluginMgr::install_plugin(const TPluginMetaInfo& info) {
+    {
+        std::lock_guard<std::mutex> l(_lock);
+        auto iter = _plugins[info.type].find(info.name);
+
+        if (iter != _plugins[info.type].end()) {
+            return Status::AlreadyExist("plugin " + info.name + " is already install");
 
 Review comment:
   ```suggestion
               return Status::AlreadyExist("plugin " + info.name + " is already installed");
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] morningman commented on a change in pull request #2618: Add BE plugin framework (#2348)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2618: Add BE plugin framework (#2348)
URL: https://github.com/apache/incubator-doris/pull/2618#discussion_r393056810
 
 

 ##########
 File path: be/src/plugin/plugin.h
 ##########
 @@ -15,11 +15,17 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#ifndef DORIS_BE_PLUGIN_PLUGIN_H
+#define DORIS_BE_PLUGIN_PLUGIN_H
+
 namespace doris {
 
-#define PLUGIN_NOT_DYNAMIC_INSTALL 1UL
+#define PLUGIN_TYPE_IMPORT 1
+#define PLUGIN_TYPE_STORAGE 2
+#define PLUGIN_TYPE_MAX 3
 
 Review comment:
   Only 3?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] morningman commented on a change in pull request #2618: Add BE plugin framework (#2348)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2618: Add BE plugin framework (#2348)
URL: https://github.com/apache/incubator-doris/pull/2618#discussion_r396936407
 
 

 ##########
 File path: gensrc/thrift/MasterService.thrift
 ##########
 @@ -84,6 +89,8 @@ struct TReportRequest {
     // the max compaction score of all tablets on a backend,
     // this field should be set along with tablet report
     8: optional i64 tablet_max_compaction_score
+    // plugin list
+    9: optional list<TPluginInfo> plugins
 
 Review comment:
   Remove this field, add it later

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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