Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[improve](udf) support load data with udf functions #43029

Merged
merged 5 commits into from
Nov 6, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -2622,6 +2622,13 @@ public class Config extends ConfigBase {
})
public static boolean enable_java_udf = true;

@ConfField(mutable = true, masterOnly = true, description = {
"开启后,可以在导入时,利用创建的全局java_udf函数处理数据, 默认为false。",
"When enabled, data can be processed using the globally created java_udf function during import."
+ " The default setting is false."
})
public static boolean enable_udf_in_load = false;

@ConfField(description = {
"是否忽略 Image 文件中未知的模块。如果为 true,不在 PersistMetaModules.MODULE_NAMES 中的元数据模块将被忽略并跳过。"
+ "默认为 false,如果 Image 文件中包含未知的模块,Doris 将会抛出异常。"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import org.apache.doris.catalog.StructType;
import org.apache.doris.catalog.Type;
import org.apache.doris.common.AnalysisException;
import org.apache.doris.common.Config;
import org.apache.doris.common.ErrorCode;
import org.apache.doris.common.ErrorReport;
import org.apache.doris.datasource.InternalCatalog;
Expand Down Expand Up @@ -2456,7 +2457,11 @@ public Function findUdf(FunctionName fnName, Analyzer analyzer) throws AnalysisE
}

Function fn = null;
String dbName = fnName.analyzeDb(analyzer);
String dbName = null;
// when enable_udf_in_load == true, and db is null, maybe it's load, should find global function
if (!(Config.enable_udf_in_load && fnName.getDb() == null)) {
dbName = fnName.analyzeDb(analyzer);
}
if (!Strings.isNullOrEmpty(dbName)) {
// check operation privilege
if (!analyzer.isReplay() && !Env.getCurrentEnv().getAccessManager().checkDbPriv(ConnectContext.get(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -113,7 +113,7 @@ public LoadingTaskPlanner(Long loadJobId, long txnId, long dbId, OlapTable table
PrivPredicate.SELECT)) {
this.analyzer.setUDFAllowed(true);
} else {
this.analyzer.setUDFAllowed(false);
this.analyzer.setUDFAllowed(Config.enable_udf_in_load);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -101,7 +101,7 @@ private void resetAnalyzer() {
analyzer = new Analyzer(Env.getCurrentEnv(), null);
// TODO(cmy): currently we do not support UDF in stream load command.
// Because there is no way to check the privilege of accessing UDF..
analyzer.setUDFAllowed(false);
analyzer.setUDFAllowed(Config.enable_udf_in_load);
descTable = analyzer.getDescTbl();
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,4 @@
-- This file is automatically generated. You should know what you did if you want to edit this
-- !sql_topic_udf --
1 eab 2023-07-15 def 2023-07-20T05:48:31 defdoris udf load

Original file line number Diff line number Diff line change
@@ -0,0 +1,3 @@
1,asd
2,xxx
3,\N
Original file line number Diff line number Diff line change
@@ -0,0 +1,6 @@
-- This file is automatically generated. You should know what you did if you want to edit this
-- !sql --
1001 asd
1002 xxx
1003 \N

Original file line number Diff line number Diff line change
@@ -0,0 +1,24 @@
// 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.

package org.apache.doris.udf;

public class IntLoadTest {
public Integer evaluate(Integer value) {
return value == null? null: value + 1000;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
// 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.

package org.apache.doris.udf;

public class StringLoadTest {
public String evaluate(String str) {
if (str == null) {
return null;
}
return str + "doris udf load";
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
1,eab,2023-07-15,def,2023-07-20:05:48:31,"ghi"
Original file line number Diff line number Diff line change
@@ -0,0 +1,126 @@
// 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.

import org.apache.kafka.clients.admin.AdminClient
import org.apache.kafka.clients.producer.KafkaProducer
import org.apache.kafka.clients.producer.ProducerRecord
import org.apache.kafka.clients.producer.ProducerConfig

suite("test_routine_load_with_udf","p0") {
def kafkaCsvTpoics = [
"test_routine_load_udf",
]
String enabled = context.config.otherConfigs.get("enableKafkaTest")
String kafka_port = context.config.otherConfigs.get("kafka_port")
String externalEnvIp = context.config.otherConfigs.get("externalEnvIp")
def kafka_broker = "${externalEnvIp}:${kafka_port}"
if (enabled != null && enabled.equalsIgnoreCase("true")) {
// define kafka
def props = new Properties()
props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "${kafka_broker}".toString())
props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer")
props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer")
// Create kafka producer
def producer = new KafkaProducer<>(props)

for (String kafkaCsvTopic in kafkaCsvTpoics) {
def txt = new File("""${context.file.parent}/data/${kafkaCsvTopic}.csv""").text
def lines = txt.readLines()
lines.each { line ->
logger.info("=====${line}========")
def record = new ProducerRecord<>(kafkaCsvTopic, null, line)
producer.send(record)
}
}
}

if (enabled != null && enabled.equalsIgnoreCase("true")) {
def tableName = "test_routine_load_with_udf"
sql """ DROP TABLE IF EXISTS ${tableName} """
sql """
CREATE TABLE IF NOT EXISTS ${tableName} (
`k1` int(20) NULL,
`k2` string NULL,
`v1` date NULL,
`v2` string NULL,
`v3` datetime NULL,
`v4` string NULL
) ENGINE=OLAP
DUPLICATE KEY(`k1`)
COMMENT 'OLAP'
DISTRIBUTED BY HASH(`k1`) BUCKETS 3
PROPERTIES ("replication_allocation" = "tag.location.default: 1");
"""
def jarPath = """${context.file.parent}/../../javaudf_p0/jars/java-udf-case-jar-with-dependencies.jar"""
scp_udf_file_to_all_be(jarPath)
log.info("Jar path: ${jarPath}".toString())

sql """ ADMIN SET FRONTEND CONFIG ("enable_udf_in_load" = "true"); """
try_sql("DROP GLOBAL FUNCTION IF EXISTS java_udf_string_load_global(string);")
sql """ CREATE GLOBAL FUNCTION java_udf_string_load_global(string) RETURNS string PROPERTIES (
"file"="file://${jarPath}",
"symbol"="org.apache.doris.udf.StringLoadTest",
"type"="JAVA_UDF"
); """

try {
sql """
CREATE ROUTINE LOAD test_udf_load ON ${tableName}
COLUMNS TERMINATED BY ",",
COLUMNS(k1, k2, v1, v2, v3, tmp, v4=java_udf_string_load_global(v2))
PROPERTIES
(
"max_batch_interval" = "5",
"max_batch_rows" = "300000",
"max_batch_size" = "209715200"
)
FROM KAFKA
(
"kafka_broker_list" = "${externalEnvIp}:${kafka_port}",
"kafka_topic" = "${kafkaCsvTpoics[0]}",
"property.kafka_default_offsets" = "OFFSET_BEGINNING"
);
"""
sql "sync"

String db = context.config.getDbNameByFile(context.file)
log.info("reason of state changed: ${db}".toString())

def count = 0
while (true) {
res = sql "select count(*) from ${tableName}"
def state = sql "show routine load for test_udf_load"
log.info("routine load state: ${state[0][8].toString()}".toString())
log.info("routine load statistic: ${state[0][14].toString()}".toString())
log.info("reason of state changed: ${state[0][17].toString()}".toString())
if (res[0][0] > 0) {
break
}
if (count >= 120) {
log.error("routine load can not visible for long time")
assertEquals(20, res[0][0])
break
}
sleep(5000)
count++
}
qt_sql_topic_udf "select * from ${tableName} order by k1"
} finally {
sql "stop routine load for test_udf_load"
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,65 @@
// 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.

suite("test_stream_load_with_udf", "p0") {
def tableName = "test_stream_load_with_udf"
sql """ set enable_fallback_to_original_planner=false;"""
sql """ DROP TABLE IF EXISTS ${tableName} """
sql """ CREATE TABLE ${tableName} (
id int,
v1 string
) ENGINE=OLAP
duplicate key (`id`)
DISTRIBUTED BY HASH(`id`) BUCKETS 1
PROPERTIES (
"replication_num" = "1"
);
"""
def jarPath = """${context.file.parent}/../../javaudf_p0/jars/java-udf-case-jar-with-dependencies.jar"""
scp_udf_file_to_all_be(jarPath)
log.info("Jar path: ${jarPath}".toString())

sql """ ADMIN SET FRONTEND CONFIG ("enable_udf_in_load" = "true"); """
try_sql("DROP GLOBAL FUNCTION IF EXISTS java_udf_int_load_global(int);")
sql """ CREATE GLOBAL FUNCTION java_udf_int_load_global(int) RETURNS int PROPERTIES (
"file"="file://${jarPath}",
"symbol"="org.apache.doris.udf.IntLoadTest",
"type"="JAVA_UDF"
); """

streamLoad {
table "${tableName}"

set 'column_separator', ','
set 'columns', """ tmp,v1, id=java_udf_int_load_global(tmp) """
file 'test_stream_load_udf.csv'
check { result, exception, startTime, endTime ->
if (exception != null) {
throw exception
}
log.info("Stream load result: ${result}".toString())
def json = parseJson(result)
assertEquals("success", json.Status.toLowerCase())
assertEquals(3, json.NumberLoadedRows)
}
}

sql """sync"""

qt_sql """select * from ${tableName} order by id;"""

}
Loading