Flink SQL无疑是实时数仓领域一个最耀眼的明星,他对于统一流批一体的设计可谓是居功至伟。鉴于Flink SQL在实时数仓领域的卓越表现,我们很有必要对Flink SQL在ETL场景下的表现要有深刻的理解。本文聚焦于Flink SQL UDF使用场景下由于SQL重写导致UDF翻倍调用的原理分析及对应的应对策略。
一 场景复现
这里通过一个案例来说明问题,本次依然采用《网上书店项目实时数仓学习模拟数据源》案例进行问题说明。数据源更多的说明请参考笔者对应的博客。
-
这里以实时维表关联为例,首先来看UDF定义
private static JedisUtil jedis; @Override public void open(FunctionContext context) throws Exception { super.open(context); jedis = JedisUtil.getInstance(); } /** * Flink 维表查询 UDF 定义 * * @param nkey * @param timestamp */ public String eval(String nkey,Long timestamp) { // 1 构造Redis Sorted set集合key String key = "dim_data:"+nkey; // 2 从Sorted set集合获取key对应的版本数据集合 Set<String> values = jedis.SORTSETS.zrange(key,0,-1); // 3 获取对应的版本数据 String versionKey = null; for(String value:values){ String[] split = value.split(":"); if(Long.valueOf(split[2]) > timestamp){ break; }else { versionKey = value; } } // 4 构造hash对应的key String hashKey = null; if(versionKey != null){ hashKey = "versions:"+versionKey; } // 5 获取版本数据 Map<String, String> versionValue = jedis.HASHS.hgetAll(hashKey); // 6 返回数据 String result = JSONUtils.getJSONObjectFromMap(versionValue).toJSONString(); // *** 这里的打印条件用来跟踪UDF被调用次数 *** System.out.println("Log => " + System.currentTimeMillis() + " : " + result); return result; }
这个UDF支持订单明细数据源从Redis中查询对应的图书维度数据,其中
System.out.println("Log => " + System.currentTimeMillis() + " : " + result);
用来跟踪UDF被调用次数,理论上每调用一次UDF都会打印一条日志记录。 -
Flink SQL主程序代码如下
public static void redisUDFDimQueryWithSubQueryDemo(){ StreamExecutionEnvironment executionEnvironment = StreamExecutionEnvironment.getExecutionEnvironment(); StreamTableEnvironment tableEnvironment = StreamTableEnvironment.create(executionEnvironment); executionEnvironment.setParallelism(1); tableEnvironment.getConfig().set("table.exec.source.idle-timeout","3s"); // 注册UDF tableEnvironment.createTemporarySystemFunction("dim_product_with_versions", DimProductWithVersionsForRedis.class); // 定义订单明细数据源 tableEnvironment.executeSql("create table tbl_order_detail(\n" + " order_book_id int comment '订单明细ID',\n" + " order_id int comment '订单ID',\n" + " book_id int comment '图书ID',\n" + " book_number int comment '图书下单数量',\n" + " original_price double comment '原始交易额',\n" + " actual_price double comment '实付交易额',\n" + " discount_price double comment '折扣金额',\n" + " create_time string comment '下单时间',\n" + " update_time bigint comment '更新时间戳'\n" + ")with(\n" + " 'connector' = 'kafka',\n" + " 'topic' = 'tbl_order_detail',\n" + " 'properties.bootstrap.servers' = 'localhost:9092',\n" + " 'properties.group.id' = 'testGroup',\n" + " 'scan.startup.mode' = 'latest-offset',\n" + " 'format' = 'json',\n" + " 'json.fail-on-missing-field' = 'false',\n" + " 'json.ignore-parse-errors' = 'true'\n" + " );"); // 打印SQL执行计划 String explainSql = tableEnvironment.explainSql("select\n" + "\n" + " order_book_id,\n" + " order_id,\n" + " book_id,\n" + " json_value(dim_product,'$.price') as price,\n" + " json_value(dim_product,'$.book_name') as book_name\n" + "\n" + "from (\n" + " select\n" + "\n" + " order_book_id as order_book_id,\n" + " order_id as order_id,\n" + " book_id as book_id,\n" + " book_number as book_number,\n" + " original_price as original_price,\n" + " actual_price as actual_price,\n" + " discount_price as discount_price,\n" + " create_time as create_time,\n" + " update_time as update_time,\n" + " dim_product_with_versions(concat('dim_book:',cast(book_id as string)),cast(update_time as bigint)) as dim_product\n" + "\n" + " from tbl_order_detail\n" + ") tmp\n" + "where json_value(dim_product,'$.price') > 5\n" + ";"); System.out.println("SQL Explain Plan: "); System.out.println(explainSql); // 定义ETL逻辑,支持订单明细数据源每一条数据查询Redis维表获取对应的维度信息 tableEnvironment.executeSql("select\n" + "\n" + " order_book_id,\n" + " order_id,\n" + " book_id,\n" + " json_value(dim_product,'$.price') as price,\n" + " json_value(dim_product,'$.book_name') as book_name\n" + "\n" + "from (\n" + " select\n" + "\n" + " order_book_id as order_book_id,\n" + " order_id as order_id,\n" + " book_id as book_id,\n" + " book_number as book_number,\n" + " original_price as original_price,\n" + " actual_price as actual_price,\n" + " discount_price as discount_price,\n" + " create_time as create_time,\n" + " update_time as update_time,\n" + " dim_product_with_versions(concat('dim_book:',cast(book_id as string)),cast(update_time as bigint)) as dim_product\n" + "\n" + " from tbl_order_detail\n" + ") tmp\n" + "where json_value(dim_product,'$.price') > 5\n" + ";").print(); }
从ETL逻辑可以发现,这是一个嵌套SQL查询语句,其中子查询通过自定义的UDF函数
dim_product_with_versions
查询远程存储Redis中缓存的图书维度数据,同时在外层查询中存在一个针对UDF查询结果的where过滤条件和两个针对UDF查询结果的二次处理逻辑。从SQL代码来看,理想状态应该仅去Redis获取一次数据,外层查询条件可以根据子查询结果进行处理,这种情况是最优解。 -
测试主程序
-
测试数据如下(
Redis维度数据已经准备好了
){"book_id":"2","update_time":1714895519000,"original_price":"63.81","create_time":"2024-05-05 14:52:07","actual_price":"63.81","discount_price":"0.0","book_number":"3","order_id":"1","order_book_id":"1"}
-
测试结果
SQL Explain Plan: == Abstract Syntax Tree == LogicalProject(order_book_id=[$0], order_id=[$1], book_id=[$2], price=[JSON_VALUE($9, _UTF-16LE'$.price')], book_name=[JSON_VALUE($9, _UTF-16LE'$.book_name')]) +- LogicalFilter(condition=[>(CAST(JSON_VALUE($9, _UTF-16LE'$.price')):BIGINT, 5)]) +- LogicalProject(order_book_id=[$0], order_id=[$1], book_id=[$2], book_number=[$3], original_price=[$4], actual_price=[$5], discount_price=[$6], create_time=[$7], update_time=[$8], dim_product=[dim_product_with_versions(CONCAT(_UTF-16LE'dim_book:', CAST($2):VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), $8)]) +- LogicalTableScan(table=[[default_catalog, default_database, tbl_order_detail]]) == Optimized Physical Plan == Calc(select=[order_book_id, order_id, book_id, JSON_VALUE(dim_product_with_versions(CONCAT(_UTF-16LE'dim_book:', CAST(book_id AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE")), update_time), _UTF-16LE'$.price') AS price, JSON_VALUE(dim_product_with_versions(CONCAT(_UTF-16LE'dim_book:', CAST(book_id AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE")), update_time), _UTF-16LE'$.book_name') AS book_name], where=[>(CAST(JSON_VALUE(dim_product_with_versions(CONCAT(_UTF-16LE'dim_book:', CAST(book_id AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE")), update_time), _UTF-16LE'$.price') AS BIGINT), 5:BIGINT)]) +- TableSourceScan(table=[[default_catalog, default_database, tbl_order_detail]], fields=[order_book_id, order_id, book_id, book_number, original_price, actual_price, discount_price, create_time, update_time]) == Optimized Execution Plan == Calc(select=[order_book_id, order_id, book_id, JSON_VALUE(dim_product_with_versions(CONCAT('dim_book:', CAST(book_id AS VARCHAR(2147483647))), update_time), '$.price') AS price, JSON_VALUE(dim_product_with_versions(CONCAT('dim_book:', CAST(book_id AS VARCHAR(2147483647))), update_time), '$.book_name') AS book_name], where=[(CAST(JSON_VALUE(dim_product_with_versions(CONCAT('dim_book:', CAST(book_id AS VARCHAR(2147483647))), update_time), '$.price') AS BIGINT) > 5)]) +- TableSourceScan(table=[[default_catalog, default_database, tbl_order_detail]], fields=[order_book_id, order_id, book_id, book_number, original_price, actual_price, discount_price, create_time, update_time]) SQL Execute Result: Log => 1714915876672 : {"update_time":"2024-05-05 15:51:53","category_name":"Java","create_time":"2024-05-05 14:51:53","publisher_date":"2024-05-05","category_id":"1","price":"12.03","author":"Java-author:1393","publisher":"Java-publisher:7202","book_id":"2","book_name":"Java-8727-版"} Log => 1714915876735 : {"update_time":"2024-05-05 15:51:53","category_name":"Java","create_time":"2024-05-05 14:51:53","publisher_date":"2024-05-05","category_id":"1","price":"12.03","author":"Java-author:1393","publisher":"Java-publisher:7202","book_id":"2","book_name":"Java-8727-版"} Log => 1714915876736 : {"update_time":"2024-05-05 15:51:53","category_name":"Java","create_time":"2024-05-05 14:51:53","publisher_date":"2024-05-05","category_id":"1","price":"12.03","author":"Java-author:1393","publisher":"Java-publisher:7202","book_id":"2","book_name":"Java-8727-版"} +----+---------------+-------------+-------------+--------------------------------+--------------------------------+ | op | order_book_id | order_id | book_id | price | book_name | +----+---------------+-------------+-------------+--------------------------------+--------------------------------+ | +I | 1 | 1 | 2 | 12.03 | Java-8727-版 |
从结果可以看出,处理一行订单明细输入要3次调用UDF函数,这一点从输出的执行计划也能看出来,并且每次调用的时间还存在一定的窗口期,这和我们代码里定义的逻辑明显存在差异,这是非常危险的,如果我们当前的函数是计算型的(
即存在中间状态
),那么很可能导致最终的结果不正确。并且我们应该知道,调用外部链接成本是非常高昂的,这对于性能来说也不友好。
-
二 问题分析
那么是什么原因造成的这个结果呢?本文从SQL执行层进行解释,更深入的原因要追踪到Flink执行引擎层进行深挖(待时间充分或写付费博客时笔者再加以说明)。
这里的分析还要从上面的执行计划着手,首先我们要了解基本知识,上面执行计划其实是分成了三个部分,分别是抽象语法树
、优化的物理计划
和优化的执行计划
,那么真正提交执行的是优化的执行计划
。
不管是什么执行引擎,SQL执行计划一般来说都是从下到上,从右向左的顺序进行解读的。
-
首先来看抽象语法树
== Abstract Syntax Tree == LogicalProject(order_book_id=[$0], order_id=[$1], book_id=[$2], price=[JSON_VALUE($9, _UTF-16LE'$.price')], book_name=[JSON_VALUE($9, _UTF-16LE'$.book_name')]) +- LogicalFilter(condition=[>(CAST(JSON_VALUE($9, _UTF-16LE'$.price')):BIGINT, 5)]) +- LogicalProject(order_book_id=[$0], order_id=[$1], book_id=[$2], book_number=[$3], original_price=[$4], actual_price=[$5], discount_price=[$6], create_time=[$7], update_time=[$8], dim_product=[dim_product_with_versions(CONCAT(_UTF-16LE'dim_book:', CAST($2):VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), $8)]) +- LogicalTableScan(table=[[default_catalog, default_database, tbl_order_detail]])
这里我稍加解释上面代表的含义:
- 首先读取底表
LogicalTableScan(table=[[default_catalog, default_database, tbl_order_detail]])
,读取数据根据LogicalProject(order_book_id=[$0], order_id=[$1], book_id=[$2], book_number=[$3], original_price=[$4], actual_price=[$5], discount_price=[$6], create_time=[$7], update_time=[$8], dim_product=[dim_product_with_versions(CONCAT(_UTF-16LE'dim_book:', CAST($2):VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), $8)])
选择列集合,一共9列;中间结果集定义result1; - 基于步骤1的中间结果集result1,通过过滤条件
LogicalFilter(condition=[>(CAST(JSON_VALUE($9, _UTF-16LE'$.price')):BIGINT, 5)])
过滤符合条件的数据,定义结果集为result2; - 根据result2通过条件
LogicalProject(order_book_id=[$0], order_id=[$1], book_id=[$2], price=[JSON_VALUE($9, _UTF-16LE'$.price')], book_name=[JSON_VALUE($9, _UTF-16LE'$.book_name')])
过滤最后服务结果的数据返回,其中最后两列通过UDF函数计算获取。
从上述执行过程解析可以看出,抽象语法树的逻辑还是符合我们SQL代码执行逻辑的。
- 首先读取底表
-
然后是优化的物理计划
== Optimized Physical Plan == Calc(select=[order_book_id, order_id, book_id, JSON_VALUE(dim_product_with_versions(CONCAT(_UTF-16LE'dim_book:', CAST(book_id AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE")), update_time), _UTF-16LE'$.price') AS price, JSON_VALUE(dim_product_with_versions(CONCAT(_UTF-16LE'dim_book:', CAST(book_id AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE")), update_time), _UTF-16LE'$.book_name') AS book_name], where=[>(CAST(JSON_VALUE(dim_product_with_versions(CONCAT(_UTF-16LE'dim_book:', CAST(book_id AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE")), update_time), _UTF-16LE'$.price') AS BIGINT), 5:BIGINT)]) +- TableSourceScan(table=[[default_catalog, default_database, tbl_order_detail]], fields=[order_book_id, order_id, book_id, book_number, original_price, actual_price, discount_price, create_time, update_time])
首先从物理计划展开我们就可以发现,他比抽象语法树步骤少了些许,下面让我们解析优化后的物理计划:
- 根据
TableSourceScan(table=[[default_catalog, default_database, tbl_order_detail]], fields=[order_book_id, order_id, book_id, book_number, original_price, actual_price, discount_price, create_time, update_time])
可以得到,第一步是读取数据源获取对应的列数据集合,中间结果记作result1; - 步骤
Calc(select=[order_book_id, order_id, book_id, JSON_VALUE(dim_product_with_versions(CONCAT(_UTF-16LE'dim_book:', CAST(book_id AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE")), update_time), _UTF-16LE'$.price') AS price, JSON_VALUE(dim_product_with_versions(CONCAT(_UTF-16LE'dim_book:', CAST(book_id AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE")), update_time), _UTF-16LE'$.book_name') AS book_name], where=[>(CAST(JSON_VALUE(dim_product_with_versions(CONCAT(_UTF-16LE'dim_book:', CAST(book_id AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE")), update_time), _UTF-16LE'$.price') AS BIGINT), 5:BIGINT)])
可以得到,根据where过滤条件dim_product_with_versions
过滤result1数据,符合条件的记录通过dim_product_with_versions
函数调用远程存储获取数据,在这一步dim_product_with_versions
函数先后一共出现了3次,这也是为什么最终的SQL处理一行实时数据要调用3次UDF。
- 根据
-
最后是真正提交运行的执行计划
== Optimized Execution Plan == Calc(select=[order_book_id, order_id, book_id, JSON_VALUE(dim_product_with_versions(CONCAT('dim_book:', CAST(book_id AS VARCHAR(2147483647))), update_time), '$.price') AS price, JSON_VALUE(dim_product_with_versions(CONCAT('dim_book:', CAST(book_id AS VARCHAR(2147483647))), update_time), '$.book_name') AS book_name], where=[(CAST(JSON_VALUE(dim_product_with_versions(CONCAT('dim_book:', CAST(book_id AS VARCHAR(2147483647))), update_time), '$.price') AS BIGINT) > 5)]) +- TableSourceScan(table=[[default_catalog, default_database, tbl_order_detail]], fields=[order_book_id, order_id, book_id, book_number, original_price, actual_price, discount_price, create_time, update_time])
我们继续根据上述规则分析执行计划:
-
根据
TableSourceScan(table=[[default_catalog, default_database, tbl_order_detail]], fields=[order_book_id, order_id, book_id, book_number, original_price, actual_price, discount_price, create_time, update_time])
可以得到,第一步是读取数据源获取对应的列数据集合,中间结果记作result1; -
基于result1,根据执行计划
Calc(select=[order_book_id, order_id, book_id, JSON_VALUE(dim_product_with_versions(CONCAT('dim_book:', CAST(book_id AS VARCHAR(2147483647))), update_time), '$.price') AS price, JSON_VALUE(dim_product_with_versions(CONCAT('dim_book:', CAST(book_id AS VARCHAR(2147483647))), update_time), '$.book_name') AS book_name], where=[(CAST(JSON_VALUE(dim_product_with_versions(CONCAT('dim_book:', CAST(book_id AS VARCHAR(2147483647))), update_time), '$.price') AS BIGINT) > 5)])
执行过滤条件和投影条件,计算最终结果。这里自定义UDF一共出现了3次,分别是:
- JSON_VALUE(dim_product_with_versions(CONCAT(‘dim_book:’, CAST(book_id AS VARCHAR(2147483647))), update_time), ‘$.price’) AS price
- JSON_VALUE(dim_product_with_versions(CONCAT(‘dim_book:’, CAST(book_id AS VARCHAR(2147483647))), update_time), ‘$.book_name’) AS book_name
- CAST(JSON_VALUE(dim_product_with_versions(CONCAT(‘dim_book:’, CAST(book_id AS VARCHAR(2147483647))), update_time), ‘$.price’) AS BIGINT) > 5
因为最终的执行计划分别调用了3次自定义UDF函数
dim_product_with_versions
,所以也就出现了场景复现中问题,每当处理一条实时明细数据,要调用3次自定义UDF。 -
上面我们已经根据SQL执行计划,从SQL执行计划角度分析了产生UDF多次调用的原因,那么是什么因素驱动了执行计划的调整呢?这里又到了另一个大的主题,关于SQL优化基础知识。这里就不对SQL优化的知识进行展开说明,后续有机会可以开专题专项讨论SQL优化器的原理。在这里笔者直接给出SQL改写的依据,那就是谓语下推
改写了我们提交的SQL语句,整个过程简化大概过程如下:
-- 1 原始SQL: 提交执行的SQL语句
select
order_book_id,
order_id,
book_id,
json_value(dim_product,'$.price') as price,
json_value(dim_product,'$.book_name') as book_name
from (
select
order_book_id as order_book_id,
order_id as order_id,
book_id as book_id,
book_number as book_number,
original_price as original_price,
actual_price as actual_price,
discount_price as discount_price,
create_time as create_time,
update_time as update_time,
dim_product_with_versions(concat('dim_book:',cast(book_id as string)),cast(update_time as bigint)) as dim_product
from tbl_order_detail
) tmp
where json_value(dim_product,'$.price') > 5
;
-- 2 改写SQL: 从提交的语句不难发现,这个SQL的子查询是多余的,完全可以使用更简化的方式改写SQL语句
select
order_book_id as order_book_id,
order_id as order_id,
book_id as book_id,
json_value(dim_product_with_versions(concat('dim_book:',cast(book_id as string)),cast(update_time as bigint)),'$.price') as price,
json_value(dim_product_with_versions(concat('dim_book:',cast(book_id as string)),cast(update_time as bigint)),'$.book_name') as book_name
from tbl_order_detail
where json_value(dim_product_with_versions(concat('dim_book:',cast(book_id as string)),cast(update_time as bigint)),'$.price') > 5
;
改写后的SQL语句更简单可读了,但是会从我们期望的1次调用外部存储扩展到3次调用外部存储。优化器查询重写技术在大部分场景下都会提升SQL执行性能,但在实时数仓领域,当调用外部存储被查询重写破坏原始语义就很麻烦,对于幂等调用会损害性能,对于计算任务可能直接导致结果错误。
三 解决方案
我们已经知道了问题的原因,那么就要想办法去绕过影响因素来解决问题,这里首先想到的解决方案是如何让SQL不进行改写以语法树格式执行(即让dim_product_with_versions(concat('dim_book:',cast(book_id as string)),cast(update_time as bigint)) as dim_product
生成物理的中间结果),很显然这种方案需要深入Flink内核源码,成本比较高,且在普通企业内很难落地。笔者还获得了另一种解决方案,没有对内核源码侵入,但能绕过SQL改写麻烦,下面进行介绍。
通过增加一层专用于透传的UDTF来绕过多次调用,让面让我们一起来逐步解决问题。
-
首先开发专用透传UDTF函数
@FunctionHint(output = @DataTypeHint("STRING")) public class PassThroughFunction extends TableFunction { public void eval(String item) { collect(item); } }
-
改造后用于ETL的SQL主程序
public static void redisUDFDimQueryWithSubQueryPTDemo(){ StreamExecutionEnvironment executionEnvironment = StreamExecutionEnvironment.getExecutionEnvironment(); StreamTableEnvironment tableEnvironment = StreamTableEnvironment.create(executionEnvironment); executionEnvironment.setParallelism(1); tableEnvironment.getConfig().set("table.exec.source.idle-timeout","3s"); // 注册UDF tableEnvironment.createTemporarySystemFunction("dim_product_with_versions", DimProductWithVersionsForRedis.class); tableEnvironment.createTemporarySystemFunction("pass_through", PassThroughFunction.class); // 定义订单明细数据源 tableEnvironment.executeSql("create table tbl_order_detail(\n" + " order_book_id int comment '订单明细ID',\n" + " order_id int comment '订单ID',\n" + " book_id int comment '图书ID',\n" + " book_number int comment '图书下单数量',\n" + " original_price double comment '原始交易额',\n" + " actual_price double comment '实付交易额',\n" + " discount_price double comment '折扣金额',\n" + " create_time string comment '下单时间',\n" + " update_time bigint comment '更新时间戳'\n" + ")with(\n" + " 'connector' = 'kafka',\n" + " 'topic' = 'tbl_order_detail',\n" + " 'properties.bootstrap.servers' = 'localhost:9092',\n" + " 'properties.group.id' = 'testGroup',\n" + " 'scan.startup.mode' = 'latest-offset',\n" + " 'format' = 'json',\n" + " 'json.fail-on-missing-field' = 'false',\n" + " 'json.ignore-parse-errors' = 'true'\n" + " );"); // 打印执行计划 String explainSql = tableEnvironment.explainSql("select\n" + "\n" + " order_book_id,\n" + " order_id,\n" + " book_id,\n" + " json_value(product,'$.price') as price,\n" + " json_value(product,'$.book_name') as book_name\n" + "\n" + "from (\n" + " select\n" + "\n" + " order_book_id as order_book_id,\n" + " order_id as order_id,\n" + " book_id as book_id,\n" + " book_number as book_number,\n" + " original_price as original_price,\n" + " actual_price as actual_price,\n" + " discount_price as discount_price,\n" + " create_time as create_time,\n" + " update_time as update_time,\n" + " product as product \n" + "\n" + " from tbl_order_detail\n" + " left join lateral table(pass_through(dim_product_with_versions(concat('dim_book:',cast(book_id as string)),cast(update_time as bigint)))) t(product) on true \n" + ") tmp\n" + "where json_value(product,'$.price') > 5\n" + ";"); System.out.println("SQL Explain Plan: "); System.out.println(explainSql); // 定义ETL逻辑,支持订单明细数据源每一条数据查询Redis维表获取对应的维度信息 tableEnvironment.executeSql("select\n" + "\n" + " order_book_id,\n" + " order_id,\n" + " book_id,\n" + " json_value(product,'$.price') as price,\n" + " json_value(product,'$.book_name') as book_name\n" + "\n" + "from (\n" + " select\n" + "\n" + " order_book_id as order_book_id,\n" + " order_id as order_id,\n" + " book_id as book_id,\n" + " book_number as book_number,\n" + " original_price as original_price,\n" + " actual_price as actual_price,\n" + " discount_price as discount_price,\n" + " create_time as create_time,\n" + " update_time as update_time,\n" + " product as product \n" + "\n" + " from tbl_order_detail\n" + " left join lateral table(pass_through(dim_product_with_versions(concat('dim_book:',cast(book_id as string)),cast(update_time as bigint)))) t(product) on true \n" + ") tmp\n" + "where json_value(product,'$.price') > 5\n" + ";").print(); }
这段代码执行逻辑通过自定义的透传UDTF函数进行了封装,其他语义与之前一致。
-
执行结果
Log => 1714920821117 : {"update_time":"2024-05-05 15:51:53","category_name":"Java","create_time":"2024-05-05 14:51:53","publisher_date":"2024-05-05","category_id":"1","price":"12.03","author":"Java-author:1393","publisher":"Java-publisher:7202","book_id":"2","book_name":"Java-8727-版"} +----+---------------+-------------+-------------+--------------------------------+--------------------------------+ | op | order_book_id | order_id | book_id | price | book_name | +----+---------------+-------------+-------------+--------------------------------+--------------------------------+ | +I | 1 | 1 | 2 | 12.03 | Java-8727-版 |
从结果看,这次的确变成了一次Redis的访问。通过执行计划也可以发现,SQL代码中的product属性在执行计划层被物化成了列
EXPR$0
,在最终结果投影中都是基于这个生成的表达式进行操作的。