You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by yi...@apache.org on 2022/10/28 00:45:18 UTC

[doris] branch master updated: [enhancement](java-udf) Support loading libjvm at runtime (#13660)

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

yiguolei pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/master by this push:
     new 2ef8f3f6f4 [enhancement](java-udf) Support loading libjvm at runtime (#13660)
2ef8f3f6f4 is described below

commit 2ef8f3f6f42e5dabc650049dd66fcf6a1e7befe2
Author: Adonis Ling <ad...@gmail.com>
AuthorDate: Fri Oct 28 08:45:12 2022 +0800

    [enhancement](java-udf) Support loading libjvm at runtime (#13660)
---
 be/CMakeLists.txt             | 23 +++-------
 be/src/util/CMakeLists.txt    |  4 ++
 be/src/util/jni-util.cpp      |  7 ++-
 be/src/util/libjvm_loader.cpp | 99 +++++++++++++++++++++++++++++++++++++++++++
 be/src/util/libjvm_loader.h   | 54 +++++++++++++++++++++++
 bin/check_be_version.sh       | 93 ----------------------------------------
 bin/start_be.sh               | 29 -------------
 build.sh                      |  1 -
 run-be-ut.sh                  | 29 -------------
 9 files changed, 167 insertions(+), 172 deletions(-)

diff --git a/be/CMakeLists.txt b/be/CMakeLists.txt
index 3d6f7e4b27..00067b66c7 100644
--- a/be/CMakeLists.txt
+++ b/be/CMakeLists.txt
@@ -603,26 +603,13 @@ include_directories(
 )
 
 if (BUILD_JAVA_UDF)
-    execute_process(COMMAND chmod 755 ${BASE_DIR}/../tools/find_libjvm.sh)
-    execute_process(COMMAND ${BASE_DIR}/../tools/find_libjvm.sh OUTPUT_VARIABLE LIBJVM_PATH OUTPUT_STRIP_TRAILING_WHITESPACE)
-    FILE(GLOB_RECURSE LIB_JVM ${LIBJVM_PATH})
-    if("${LIB_JVM}" STREQUAL "")
-        message(STATUS "Disable JAVA UDF because there is no libjvm found!")
+    include_directories($ENV{JAVA_HOME}/include)
+    if (NOT OS_MACOSX)
+        include_directories($ENV{JAVA_HOME}/include/linux)
     else()
-        set(DORIS_DEPENDENCIES
-            ${DORIS_DEPENDENCIES}
-            jvm
-        )
-        add_library(jvm SHARED IMPORTED)
-        set_target_properties(jvm PROPERTIES IMPORTED_LOCATION ${LIB_JVM})
-        include_directories($ENV{JAVA_HOME}/include)
-        if (NOT OS_MACOSX)
-            include_directories($ENV{JAVA_HOME}/include/linux)
-        else()
-            include_directories($ENV{JAVA_HOME}/include/darwin)
-        endif()
-        add_definitions("-DLIBJVM")
+        include_directories($ENV{JAVA_HOME}/include/darwin)
     endif()
+    add_definitions("-DLIBJVM")
 endif()
 
 if (NOT OS_MACOSX)
diff --git a/be/src/util/CMakeLists.txt b/be/src/util/CMakeLists.txt
index 8b96a6f13f..dc9509a9ec 100644
--- a/be/src/util/CMakeLists.txt
+++ b/be/src/util/CMakeLists.txt
@@ -127,6 +127,10 @@ if (OS_MACOSX)
     list(APPEND UTIL_FILES perf_counters_mac.cpp disk_info_mac.cpp)
 endif()
 
+if (BUILD_JAVA_UDF)
+    list(APPEND UTIL_FILES libjvm_loader.cpp)
+endif()
+
 add_library(Util STATIC
     ${UTIL_FILES}
 )
diff --git a/be/src/util/jni-util.cpp b/be/src/util/jni-util.cpp
index 0509989406..7c56381052 100644
--- a/be/src/util/jni-util.cpp
+++ b/be/src/util/jni-util.cpp
@@ -29,6 +29,7 @@
 #include "common/config.h"
 #include "gutil/once.h"
 #include "gutil/strings/substitute.h"
+#include "libjvm_loader.h"
 
 using std::string;
 
@@ -67,7 +68,7 @@ const std::string GetDorisJNIClasspath() {
 
 void FindOrCreateJavaVM() {
     int num_vms;
-    int rv = JNI_GetCreatedJavaVMs(&g_vm, 1, &num_vms);
+    int rv = LibJVMLoader::JNI_GetCreatedJavaVMs(&g_vm, 1, &num_vms);
     if (rv == 0) {
         auto classpath = GetDorisJNIClasspath();
         std::string heap_size = fmt::format("-Xmx{}", config::jvm_max_heap_size);
@@ -91,7 +92,7 @@ void FindOrCreateJavaVM() {
         // Set it to JNI_FALSE because JNI_TRUE will let JVM ignore the max size config.
         vm_args.ignoreUnrecognized = JNI_FALSE;
 
-        jint res = JNI_CreateJavaVM(&g_vm, (void**)&env, &vm_args);
+        jint res = LibJVMLoader::JNI_CreateJavaVM(&g_vm, (void**)&env, &vm_args);
         if (JNI_OK != res) {
             DCHECK(false) << "Failed to create JVM, code= " << res;
         }
@@ -213,6 +214,8 @@ Status JniUtil::LocalToGlobalRef(JNIEnv* env, jobject local_ref, jobject* global
 }
 
 Status JniUtil::Init() {
+    RETURN_IF_ERROR(LibJVMLoader::instance().load());
+
     // Get the JNIEnv* corresponding to current thread.
     JNIEnv* env;
     RETURN_IF_ERROR(JniUtil::GetJNIEnv(&env));
diff --git a/be/src/util/libjvm_loader.cpp b/be/src/util/libjvm_loader.cpp
new file mode 100644
index 0000000000..abd0a4863b
--- /dev/null
+++ b/be/src/util/libjvm_loader.cpp
@@ -0,0 +1,99 @@
+// 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 "util/libjvm_loader.h"
+
+#include <dlfcn.h>
+
+#include <cstdlib>
+#include <filesystem>
+#include <mutex>
+
+#include "common/status.h"
+
+namespace {
+
+#ifndef __APPLE__
+#define LIBJVM_SO "libjvm.so"
+#else
+#define LIBJVM_SO "libjvm.dylib"
+#endif
+
+template <typename T>
+doris::Status resolve_symbol(T& pointer, void* handle, const char* symbol) {
+    pointer = reinterpret_cast<T>(dlsym(handle, symbol));
+    return (pointer != nullptr)
+                   ? doris::Status::OK()
+                   : doris::Status::RuntimeError("Failed to resolve the symbol %s", symbol);
+}
+
+} // namespace
+
+namespace doris {
+
+LibJVMLoader::JNI_GetCreatedJavaVMsPointer LibJVMLoader::JNI_GetCreatedJavaVMs = nullptr;
+LibJVMLoader::JNI_CreateJavaVMPointer LibJVMLoader::JNI_CreateJavaVM = nullptr;
+
+LibJVMLoader& LibJVMLoader::instance() {
+    static std::once_flag find_library;
+    static std::string library;
+    std::call_once(find_library, []() {
+        const auto* java_home = getenv("JAVA_HOME");
+        if (!java_home) {
+            return;
+        }
+        std::string path(java_home);
+        for (const auto& entry : std::filesystem::recursive_directory_iterator(path)) {
+            if (entry.path().filename() == LIBJVM_SO) {
+                library = entry.path().string();
+                break;
+            }
+        }
+    });
+
+    static LibJVMLoader loader(library);
+    return loader;
+}
+
+Status LibJVMLoader::load() {
+    if (_library.empty()) {
+        return Status::RuntimeError("Failed to find the library %s.", LIBJVM_SO);
+    }
+
+    static std::once_flag resolve_symbols;
+    static Status status;
+    std::call_once(resolve_symbols, [this]() {
+        _handle = std::unique_ptr<void, void (*)(void*)>(dlopen(_library.c_str(), RTLD_LAZY),
+                                                         [](void* handle) { dlclose(handle); });
+        if (!_handle) {
+            status = Status::RuntimeError(dlerror());
+            return;
+        }
+
+        if (status = resolve_symbol(JNI_GetCreatedJavaVMs, _handle.get(), "JNI_GetCreatedJavaVMs");
+            !status.ok()) {
+            return;
+        }
+        if (status = resolve_symbol(JNI_CreateJavaVM, _handle.get(), "JNI_CreateJavaVM");
+            !status.ok()) {
+            return;
+        }
+    });
+    return status;
+}
+
+} // namespace doris
diff --git a/be/src/util/libjvm_loader.h b/be/src/util/libjvm_loader.h
new file mode 100644
index 0000000000..2982cbea7f
--- /dev/null
+++ b/be/src/util/libjvm_loader.h
@@ -0,0 +1,54 @@
+// 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.
+
+#pragma once
+
+#include <dlfcn.h>
+#include <jni.h>
+
+#include <memory>
+#include <string>
+#include <string_view>
+#include <type_traits>
+
+namespace doris {
+
+class Status;
+
+class LibJVMLoader {
+public:
+    LibJVMLoader(const LibJVMLoader&) = delete;
+    LibJVMLoader& operator=(const LibJVMLoader&) = delete;
+
+    static LibJVMLoader& instance();
+    Status load();
+
+    using JNI_GetCreatedJavaVMsPointer = std::add_pointer_t<decltype(::JNI_GetCreatedJavaVMs)>;
+    static JNI_GetCreatedJavaVMsPointer JNI_GetCreatedJavaVMs;
+
+    using JNI_CreateJavaVMPointer = std::add_pointer_t<decltype(::JNI_CreateJavaVM)>;
+    static JNI_CreateJavaVMPointer JNI_CreateJavaVM;
+
+private:
+    explicit LibJVMLoader(std::string_view library)
+            : _library(library), _handle(nullptr, nullptr) {}
+
+    const std::string _library;
+    std::unique_ptr<void, void (*)(void*)> _handle;
+};
+
+} // namespace doris
diff --git a/bin/check_be_version.sh b/bin/check_be_version.sh
deleted file mode 100755
index b44d0c8f64..0000000000
--- a/bin/check_be_version.sh
+++ /dev/null
@@ -1,93 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-
-set -eo pipefail
-
-curdir="$(cd "$(dirname "${BASH_SOURCE[0]}")" &>/dev/null && pwd)"
-
-if [[ "$(uname -s)" == 'Darwin' ]] && command -v brew &>/dev/null; then
-    PATH="$(brew --prefix)/opt/gnu-getopt/bin:${PATH}"
-    export PATH
-fi
-
-DORIS_HOME="$(
-    cd "${curdir}/.."
-    pwd
-)"
-export DORIS_HOME
-
-jdk_version() {
-    local java_cmd="${1}"
-    local result
-    local IFS=$'\n'
-
-    if [[ -z "${java_cmd}" ]]; then
-        result=no_java
-        return 1
-    else
-        local version
-        # remove \r for Cygwin
-        version="$("${java_cmd}" -Xms32M -Xmx32M -version 2>&1 | tr '\r' '\n' | grep version | awk '{print $3}')"
-        version="${version//\"/}"
-        if [[ "${version}" =~ ^1\. ]]; then
-            result="$(echo "${version}" | awk -F '.' '{print $2}')"
-        else
-            result="$(echo "${version}" | awk -F '.' '{print $1}')"
-        fi
-    fi
-    echo "${result}"
-    return 0
-}
-
-setup_java_env() {
-    local java_version
-
-    if [[ -z "${JAVA_HOME}" ]]; then
-        return 1
-    fi
-
-    local jvm_arch='amd64'
-    if [[ "$(uname -m)" == 'aarch64' ]]; then
-        jvm_arch='aarch64'
-    fi
-    java_version="$(
-        set -e
-        jdk_version "${JAVA_HOME}/bin/java"
-    )"
-    if [[ "${java_version}" -gt 8 ]]; then
-        export LD_LIBRARY_PATH="${JAVA_HOME}/lib/server:${JAVA_HOME}/lib:${LD_LIBRARY_PATH}"
-        # JAVA_HOME is jdk
-    elif [[ -d "${JAVA_HOME}/jre" ]]; then
-        export LD_LIBRARY_PATH="${JAVA_HOME}/jre/lib/${jvm_arch}/server:${JAVA_HOME}/jre/lib/${jvm_arch}:${LD_LIBRARY_PATH}"
-        # JAVA_HOME is jre
-    else
-        export LD_LIBRARY_PATH="${JAVA_HOME}/lib/${jvm_arch}/server:${JAVA_HOME}/lib/${jvm_arch}:${LD_LIBRARY_PATH}"
-    fi
-}
-
-# prepare jvm if needed
-setup_java_env || true
-
-if [[ -e "${DORIS_HOME}/bin/palo_env.sh" ]]; then
-    # shellcheck disable=1091
-    source "${DORIS_HOME}/bin/palo_env.sh"
-fi
-
-chmod 755 "${DORIS_HOME}/lib/doris_be"
-
-"${DORIS_HOME}"/lib/doris_be --version
diff --git a/bin/start_be.sh b/bin/start_be.sh
index 49d231e1c3..cdd051bf7d 100755
--- a/bin/start_be.sh
+++ b/bin/start_be.sh
@@ -104,35 +104,6 @@ jdk_version() {
     return 0
 }
 
-setup_java_env() {
-    local java_version
-
-    if [[ -z "${JAVA_HOME}" ]]; then
-        return 1
-    fi
-
-    local jvm_arch='amd64'
-    if [[ "$(uname -m)" == 'aarch64' ]]; then
-        jvm_arch='aarch64'
-    fi
-    java_version="$(
-        set -e
-        jdk_version "${JAVA_HOME}/bin/java"
-    )"
-    if [[ "${java_version}" -gt 8 ]]; then
-        export LD_LIBRARY_PATH="${JAVA_HOME}/lib/server:${JAVA_HOME}/lib:${LD_LIBRARY_PATH}"
-        # JAVA_HOME is jdk
-    elif [[ -d "${JAVA_HOME}/jre" ]]; then
-        export LD_LIBRARY_PATH="${JAVA_HOME}/jre/lib/${jvm_arch}/server:${JAVA_HOME}/jre/lib/${jvm_arch}:${LD_LIBRARY_PATH}"
-        # JAVA_HOME is jre
-    else
-        export LD_LIBRARY_PATH="${JAVA_HOME}/lib/${jvm_arch}/server:${JAVA_HOME}/lib/${jvm_arch}:${LD_LIBRARY_PATH}"
-    fi
-}
-
-# prepare jvm if needed
-setup_java_env || true
-
 # export env variables from be.conf
 #
 # UDF_RUNTIME_DIR
diff --git a/build.sh b/build.sh
index 4f394a7ee7..65559f026e 100755
--- a/build.sh
+++ b/build.sh
@@ -508,7 +508,6 @@ if [[ "${BUILD_BE}" -eq 1 ]]; then
         "${DORIS_OUTPUT}/udf/include"
 
     cp -r -p "${DORIS_HOME}/be/output/bin"/* "${DORIS_OUTPUT}/be/bin"/
-    cp -r -p "${DORIS_HOME}/bin/check_be_version.sh" "${DORIS_OUTPUT}/be/bin"/
     cp -r -p "${DORIS_HOME}/be/output/conf"/* "${DORIS_OUTPUT}/be/conf"/
 
     # Fix Killed: 9 error on MacOS (arm64).
diff --git a/run-be-ut.sh b/run-be-ut.sh
index a0d4daa37c..f66d65a64e 100755
--- a/run-be-ut.sh
+++ b/run-be-ut.sh
@@ -230,35 +230,6 @@ jdk_version() {
     return 0
 }
 
-setup_java_env() {
-    local java_version
-
-    if [[ -z "${JAVA_HOME}" ]]; then
-        return 1
-    fi
-
-    local jvm_arch='amd64'
-    if [[ "$(uname -m)" == 'aarch64' ]]; then
-        jvm_arch='aarch64'
-    fi
-    java_version="$(
-        set -e
-        jdk_version "${JAVA_HOME}/bin/java"
-    )"
-    if [[ "${java_version}" -gt 8 ]]; then
-        export LD_LIBRARY_PATH="${JAVA_HOME}/lib/server:${JAVA_HOME}/lib:${LD_LIBRARY_PATH}"
-        # JAVA_HOME is jdk
-    elif [[ -d "${JAVA_HOME}/jre" ]]; then
-        export LD_LIBRARY_PATH="${JAVA_HOME}/jre/lib/${jvm_arch}/server:${JAVA_HOME}/jre/lib/${jvm_arch}:${LD_LIBRARY_PATH}"
-        # JAVA_HOME is jre
-    else
-        export LD_LIBRARY_PATH="${JAVA_HOME}/lib/${jvm_arch}/server:${JAVA_HOME}/lib/${jvm_arch}:${LD_LIBRARY_PATH}"
-    fi
-}
-
-# prepare jvm if needed
-setup_java_env || true
-
 # prepare gtest output dir
 GTEST_OUTPUT_DIR="${CMAKE_BUILD_DIR}/gtest_output"
 rm -rf "${GTEST_OUTPUT_DIR}"


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