[Refactor](dialect) Add sql dialect converter plugins (#28890)

The current logic for SQL dialect conversion is all in the `fe-core` module, which may lead to the following issues:
- Changes to the dialect conversion logic may occur frequently, requiring users to upgrade the Doris version frequently within the fe-core module, leading to a longer change cycle.
- The cost of customized development is high, requiring users to replace the fe-core JAR package.

Turning it into a plugin can address the above issues properly.
This commit is contained in:
Xiangyu Wang
2024-01-08 22:56:17 +08:00
committed by yiguolei
parent 28dca66c06
commit 2ca90b2bf1
54 changed files with 1599 additions and 487 deletions

View File

@ -2397,6 +2397,9 @@ public class Config extends ConfigBase {
"Whether to enable the function of getting log files through http interface"})
public static boolean enable_get_log_file_api = false;
// This config is deprecated and has not taken effect anymore,
// please use dialect plugin: fe_plugins/http-dialect-converter for instead
@Deprecated
@ConfField(description = {"用于SQL方言转换的服务地址。",
"The service address for SQL dialect conversion."})
public static String sql_convertor_service = "";

View File

@ -680,12 +680,6 @@ under the License.
<artifactId>kryo-shaded</artifactId>
</dependency>
<!-- trino-parser -->
<dependency>
<groupId>io.trino</groupId>
<artifactId>trino-parser</artifactId>
</dependency>
<!-- for arrow flight sql -->
<dependency>
<groupId>org.apache.arrow</groupId>

View File

@ -27,8 +27,7 @@ import org.apache.doris.common.AnalysisException;
import org.apache.doris.common.ErrorCode;
import org.apache.doris.common.ErrorReport;
import org.apache.doris.common.UserException;
import org.apache.doris.nereids.parser.ParseDialect;
import org.apache.doris.nereids.parser.spark.SparkSql3LogicalPlanBuilder;
import org.apache.doris.nereids.parser.Dialect;
import org.apache.doris.qe.ConnectContext;
import org.apache.doris.rewrite.ExprRewriter;
import org.apache.doris.thrift.TNullSide;
@ -50,6 +49,8 @@ import java.util.Set;
public class InlineViewRef extends TableRef {
private static final Logger LOG = LogManager.getLogger(InlineViewRef.class);
private static final String DEFAULT_TABLE_ALIAS_FOR_SPARK_SQL = "__auto_generated_subquery_name";
// Catalog or local view that is referenced.
// Null for inline views parsed directly from a query string.
private final View view;
@ -198,12 +199,12 @@ public class InlineViewRef extends TableRef {
if (view == null && !hasExplicitAlias()) {
String dialect = ConnectContext.get().getSessionVariable().getSqlDialect();
ParseDialect.Dialect sqlDialect = ParseDialect.Dialect.getByName(dialect);
if (ParseDialect.Dialect.SPARK_SQL != sqlDialect) {
Dialect sqlDialect = Dialect.getByName(dialect);
if (Dialect.SPARK_SQL != sqlDialect) {
ErrorReport.reportAnalysisException(ErrorCode.ERR_DERIVED_MUST_HAVE_ALIAS);
}
hasExplicitAlias = true;
aliases = new String[] { SparkSql3LogicalPlanBuilder.DEFAULT_TABLE_ALIAS };
aliases = new String[] { DEFAULT_TABLE_ALIAS_FOR_SPARK_SQL };
}
// Analyze the inline view query statement with its own analyzer

View File

@ -18,8 +18,7 @@
package org.apache.doris.nereids.exceptions;
/**
* DialectTransformException when have not supported transforming for the
* {@link io.trino.sql.tree.Node}.
* DialectTransformException when have not supported transforming for dialect converters.
*/
public class DialectTransformException extends UnsupportedOperationException {

View File

@ -17,17 +17,16 @@
package org.apache.doris.nereids.exceptions;
import org.apache.doris.nereids.parser.ParseDialect;
import org.apache.doris.nereids.parser.Dialect;
/**
* UnsupportedDialectException when not match any in
* {@link org.apache.doris.nereids.parser.ParseDialect}.
* {@link Dialect}.
*/
public class UnsupportedDialectException extends UnsupportedOperationException {
public UnsupportedDialectException(ParseDialect dialect) {
super(String.format("Unsupported dialect name is %s, version is %s",
dialect.getDialect().getDialectName(), dialect.getVersion().getVersionName()));
public UnsupportedDialectException(Dialect dialect) {
super(String.format("Unsupported dialect name is %s", dialect.getDialectName()));
}
public UnsupportedDialectException(String type, String msg) {

View File

@ -18,7 +18,7 @@
package org.apache.doris.nereids.parser;
/**
* Trino complex function transformer
* Complex complex function transformer
*/
public abstract class ComplexFnCallTransformer extends AbstractFnCallTransformer {

View File

@ -0,0 +1,101 @@
// 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.nereids.parser;
import javax.annotation.Nullable;
/**
* ParseDialect enum, maybe support other dialect.
*/
public enum Dialect {
/**
* Doris parser dialect
*/
DORIS("doris"),
/**
* Trino parser dialect
*/
TRINO("trino"),
/**
* Presto parser dialect
*/
PRESTO("presto"),
/**
* Spark sql parser dialect
*/
SPARK_SQL("spark_sql"),
/**
* Hive parser dialect
*/
HIVE("hive"),
/**
* Alibaba max compute parser dialect
*/
MAX_COMPUTE("max_compute"),
/**
* Mysql parser dialect
*/
MYSQL("mysql"),
/**
* Postgresql parser dialect
*/
POSTGRESQL("postgresql"),
/**
* Sqlserver parser dialect
*/
SQLSERVER("sqlserver"),
/**
* Clickhouse parser dialect
*/
CLICKHOUSE("clickhouse"),
/**
* Sap hana parser dialect
*/
SAP_HANA("sap_hana"),
/**
* OceanBase parser dialect
*/
OCEANBASE("oceanbase");
public static final int MAX_DIALECT_SIZE = Dialect.values().length;
private final String dialectName;
Dialect(String dialectName) {
this.dialectName = dialectName;
}
public String getDialectName() {
return dialectName;
}
/**
* Get dialect by name
*/
public static @Nullable Dialect getByName(String dialectName) {
if (dialectName == null) {
return null;
}
for (Dialect dialect : Dialect.values()) {
if (dialect.getDialectName().equals(dialectName.toLowerCase())) {
return dialect;
}
}
return null;
}
}

View File

@ -31,8 +31,7 @@ import java.math.BigInteger;
/**
* Logical plan builder assistant for buildIn dialect and other dialect.
* The same logical in {@link org.apache.doris.nereids.parser.LogicalPlanBuilder}
* and {@link org.apache.doris.nereids.parser.trino.TrinoLogicalPlanBuilder} can be
* extracted to here.
* can be extracted to here.
*/
public class LogicalPlanBuilderAssistant {

View File

@ -18,20 +18,19 @@
package org.apache.doris.nereids.parser;
import org.apache.doris.analysis.StatementBase;
import org.apache.doris.common.Config;
import org.apache.doris.catalog.Env;
import org.apache.doris.common.Pair;
import org.apache.doris.nereids.DorisLexer;
import org.apache.doris.nereids.DorisParser;
import org.apache.doris.nereids.StatementContext;
import org.apache.doris.nereids.glue.LogicalPlanAdapter;
import org.apache.doris.nereids.parser.spark.SparkSql3LogicalPlanBuilder;
import org.apache.doris.nereids.parser.trino.TrinoParser;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
import org.apache.doris.nereids.types.DataType;
import org.apache.doris.plugin.DialectConverterPlugin;
import org.apache.doris.plugin.PluginMgr;
import org.apache.doris.qe.SessionVariable;
import com.google.common.base.Strings;
import com.google.common.collect.Lists;
import org.antlr.v4.runtime.CharStreams;
import org.antlr.v4.runtime.CommonTokenStream;
@ -67,11 +66,13 @@ public class NereidsParser {
* ParseSQL with dialect.
*/
public List<StatementBase> parseSQL(String sql, SessionVariable sessionVariable) {
@Nullable ParseDialect.Dialect sqlDialect = ParseDialect.Dialect.getByName(sessionVariable.getSqlDialect());
return parseSQLWithDialect(sql, sqlDialect, sessionVariable);
return parseSQLWithDialect(sql, sessionVariable);
}
private List<StatementBase> parseSQL(String originStr, @Nullable LogicalPlanBuilder logicalPlanBuilder) {
/**
* ParseSQL with logicalPlanBuilder.
*/
public List<StatementBase> parseSQL(String originStr, @Nullable LogicalPlanBuilder logicalPlanBuilder) {
List<Pair<LogicalPlan, StatementContext>> logicalPlans = parseMultiple(originStr, logicalPlanBuilder);
List<StatementBase> statementBases = Lists.newArrayList();
for (Pair<LogicalPlan, StatementContext> parsedPlanToContext : logicalPlans) {
@ -81,26 +82,28 @@ public class NereidsParser {
}
private List<StatementBase> parseSQLWithDialect(String sql,
@Nullable ParseDialect.Dialect sqlDialect,
SessionVariable sessionVariable) {
if (!Strings.isNullOrEmpty(Config.sql_convertor_service)) {
// if sql convertor service is enabled, no need to parse sql again by specific dialect.
@Nullable Dialect sqlDialect = Dialect.getByName(sessionVariable.getSqlDialect());
if (sqlDialect == null) {
return parseSQL(sql);
}
switch (sqlDialect) {
case TRINO:
final List<StatementBase> logicalPlans = TrinoParser.parse(sql, sessionVariable);
if (CollectionUtils.isEmpty(logicalPlans)) {
return parseSQL(sql);
PluginMgr pluginMgr = Env.getCurrentEnv().getPluginMgr();
List<DialectConverterPlugin> plugins = pluginMgr.getActiveDialectPluginList(sqlDialect);
for (DialectConverterPlugin plugin : plugins) {
try {
List<StatementBase> statementBases = plugin.parseSqlWithDialect(sql, sessionVariable);
if (CollectionUtils.isNotEmpty(statementBases)) {
return statementBases;
}
return logicalPlans;
case SPARK_SQL:
return parseSQL(sql, new SparkSql3LogicalPlanBuilder());
default:
return parseSQL(sql);
} catch (Throwable throwable) {
LOG.warn("Parse sql with dialect {} failed, plugin: {}, sql: {}.",
sqlDialect, plugin.getClass().getSimpleName(), sql, throwable);
}
}
// fallback if any exception occurs before
return parseSQL(sql);
}
/**

View File

@ -1,129 +0,0 @@
// 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.nereids.parser;
import javax.annotation.Nullable;
/**
* ParseDialect enum, maybe support other dialect.
*/
public enum ParseDialect {
/**
* Trino parser and it's version is 395.
*/
TRINO_395(Dialect.TRINO, Version.TRINO_395),
/**
* Doris parser and it's version is 2.0.0.
*/
DORIS_2_ALL(Dialect.DORIS, Version.DORIS_2_ALL),
/**
* Spark parser and it's version is 3.x.
*/
SPARK_SQL_3_ALL(Dialect.SPARK_SQL, Version.SPARK_SQL_3_ALL);
private final Dialect dialect;
private final Version version;
ParseDialect(Dialect dialect, Version version) {
this.dialect = dialect;
this.version = version;
}
public Version getVersion() {
return version;
}
public Dialect getDialect() {
return dialect;
}
/**
* The version of parse dialect.
*/
public enum Version {
/**
* Trino parser and it's version is 395.
*/
TRINO_395("395"),
/**
* Doris parser and it's version is 2.0.0.
*/
DORIS_2_ALL("2.*"),
/**
* Spark sql parser and it's version is 3.x.
*/
SPARK_SQL_3_ALL("3.*");
private final String version;
Version(String version) {
this.version = version;
}
public String getVersionName() {
return version;
}
}
/**
* The dialect name of parse dialect.
*/
public enum Dialect {
/**
* Trino parser dialect
*/
TRINO("trino"),
/**
* Presto parser dialect
*/
PRESTO("presto"),
/**
* Doris parser dialect
*/
DORIS("doris"),
/**
* Spark sql parser dialect
*/
SPARK_SQL("spark_sql");
private String dialectName;
Dialect(String dialectName) {
this.dialectName = dialectName;
}
public String getDialectName() {
return dialectName;
}
/**
* Get dialect by name
*/
public static @Nullable Dialect getByName(String dialectName) {
if (dialectName == null) {
return null;
}
for (Dialect dialect : Dialect.values()) {
if (dialect.getDialectName().equals(dialectName.toLowerCase())) {
return dialect;
}
}
return null;
}
}
}

View File

@ -22,13 +22,13 @@ package org.apache.doris.nereids.parser;
*/
public class ParserContext {
private final ParseDialect parseDialect;
private final Dialect parseDialect;
public ParserContext(ParseDialect parseDialect) {
public ParserContext(Dialect parseDialect) {
this.parseDialect = parseDialect;
}
public ParseDialect getParserDialect() {
public Dialect getParserDialect() {
return parseDialect;
}
}

View File

@ -0,0 +1,61 @@
// 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.plugin;
import org.apache.doris.analysis.StatementBase;
import org.apache.doris.nereids.parser.Dialect;
import org.apache.doris.qe.SessionVariable;
import com.google.common.collect.ImmutableSet;
import java.util.List;
import javax.annotation.Nullable;
/**
* Sql dialect adapter interface.
*/
public interface DialectConverterPlugin {
/**
* Set dialect set which this plugin can handle
*/
ImmutableSet<Dialect> acceptDialects();
/**
* <pre>
* Override this method if you want to convert sql before parse.
* Fallback to next dialect plugin if this method returns null, empty string or any exception throws.
* If all plugins fail to convert (returns null, empty string or any exception throws),
* Nereids parser will use the original SQL.
* </pre>
* */
default @Nullable String convertSql(String originSql, SessionVariable sessionVariable) {
return null;
}
/**
* <pre>
* Parse sql with dialect.
* Fallback to next dialect plugin if this method returns null, empty string or any exception throws.
* If all plugins fail to parse (returns null, empty string or any exception throws),
* Nereids parser will fallback to invoke {@link org.apache.doris.nereids.parser.NereidsParser#parseSQL(String)}.
* Use Dialect.getByName(sessionVariable.getSqlDialect()) to extract the dialect parameter.
* </pre>
* */
@Nullable List<StatementBase> parseSqlWithDialect(String sql, SessionVariable sessionVariable);
}

View File

@ -52,7 +52,8 @@ public class PluginInfo implements Writable {
public enum PluginType {
AUDIT,
IMPORT,
STORAGE;
STORAGE,
DIALECT;
public static int MAX_PLUGIN_TYPE_SIZE = PluginType.values().length;
}

View File

@ -24,11 +24,13 @@ import org.apache.doris.common.DdlException;
import org.apache.doris.common.UserException;
import org.apache.doris.common.io.Writable;
import org.apache.doris.common.util.PrintableMap;
import org.apache.doris.nereids.parser.Dialect;
import org.apache.doris.plugin.PluginInfo.PluginType;
import org.apache.doris.plugin.PluginLoader.PluginStatus;
import org.apache.doris.qe.AuditLogBuilder;
import com.google.common.base.Strings;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
@ -51,15 +53,26 @@ public class PluginMgr implements Writable {
public static final String BUILTIN_PLUGIN_PREFIX = "__builtin_";
private final Map<String, PluginLoader>[] plugins;
// use this for
private final Map<String, DialectConverterPlugin>[] dialectPlugins;
// all dynamic plugins should have unique names,
private final Set<String> dynamicPluginNames;
public PluginMgr() {
plugins = new Map[PluginType.MAX_PLUGIN_TYPE_SIZE];
for (int i = 0; i < PluginType.MAX_PLUGIN_TYPE_SIZE; i++) {
plugins[i] = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
// use synchronized wrapper for thread-safe
plugins[i] = Collections.synchronizedSortedMap(Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER));
}
// use synchronized wrapper for thread-safe
dynamicPluginNames = Collections.synchronizedSortedSet(Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER));
dialectPlugins = new Map[Dialect.MAX_DIALECT_SIZE];
for (int i = 0; i < Dialect.MAX_DIALECT_SIZE; i++) {
// use synchronized wrapper for thread-safe
dialectPlugins[i] = Collections.synchronizedSortedMap(Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER));
}
dynamicPluginNames = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER);
}
// create the plugin dir if missing
@ -79,21 +92,15 @@ public class PluginMgr implements Writable {
}
private boolean checkDynamicPluginNameExist(String name) {
synchronized (dynamicPluginNames) {
return dynamicPluginNames.contains(name);
}
return dynamicPluginNames.contains(name);
}
private boolean addDynamicPluginNameIfAbsent(String name) {
synchronized (dynamicPluginNames) {
return dynamicPluginNames.add(name);
}
return dynamicPluginNames.add(name);
}
private boolean removeDynamicPluginName(String name) {
synchronized (dynamicPluginNames) {
return dynamicPluginNames.remove(name);
}
return dynamicPluginNames.remove(name);
}
private void initBuiltinPlugins() {
@ -130,7 +137,11 @@ public class PluginMgr implements Writable {
throw new UserException("plugin " + info.getName() + " has already been installed.");
}
plugins[info.getTypeId()].put(info.getName(), pluginLoader);
// add dialect plugin
Plugin plugin = pluginLoader.getPlugin();
if (plugin instanceof DialectConverterPlugin) {
addDialectPlugin((DialectConverterPlugin) plugin, info);
}
Env.getCurrentEnv().getEditLog().logInstallPlugin(info);
LOG.info("install plugin {}", info.getName());
return info;
@ -140,6 +151,18 @@ public class PluginMgr implements Writable {
}
}
private void addDialectPlugin(DialectConverterPlugin plugin, PluginInfo info) {
for (Dialect dialect : plugin.acceptDialects()) {
dialectPlugins[dialect.ordinal()].put(info.getName(), plugin);
}
}
private void removeDialectPlugin(String name) {
for (int i = 0; i < Dialect.MAX_DIALECT_SIZE; i++) {
dialectPlugins[i].remove(name);
}
}
/**
* Dynamic uninstall plugin.
* If uninstall failed, the plugin should NOT be removed from plugin manager.
@ -153,7 +176,7 @@ public class PluginMgr implements Writable {
if (plugins[i].containsKey(name)) {
PluginLoader loader = plugins[i].get(name);
if (loader == null) {
// this is not a atomic operation, so even if containsKey() is true,
// this is not an atomic operation, so even if containsKey() is true,
// we may still get null object by get() method
continue;
}
@ -171,7 +194,7 @@ public class PluginMgr implements Writable {
plugins[i].remove(name);
loader.setStatus(PluginStatus.UNINSTALLED);
removeDynamicPluginName(name);
removeDialectPlugin(name);
// do not get plugin info by calling loader.getPluginInfo(). That method will try to
// reload the plugin properties from source if this plugin is not installed successfully.
// Here we only need the plugin's name for persisting.
@ -195,7 +218,10 @@ public class PluginMgr implements Writable {
PluginLoader loader = new BuiltinPluginLoader(Config.plugin_dir, pluginInfo, plugin);
PluginLoader checkLoader = plugins[pluginInfo.getTypeId()].putIfAbsent(pluginInfo.getName(), loader);
// add dialect plugin
if (plugin instanceof DialectConverterPlugin) {
addDialectPlugin((DialectConverterPlugin) plugin, pluginInfo);
}
return checkLoader == null;
}
@ -217,6 +243,11 @@ public class PluginMgr implements Writable {
// install plugin
pluginLoader.reload();
pluginLoader.setStatus(PluginStatus.INSTALLED);
// add dialect plugin
Plugin plugin = pluginLoader.getPlugin();
if (plugin instanceof DialectConverterPlugin) {
addDialectPlugin((DialectConverterPlugin) plugin, info);
}
} catch (IOException | UserException e) {
pluginLoader.setStatus(PluginStatus.ERROR, e.getMessage());
throw e;
@ -246,7 +277,12 @@ public class PluginMgr implements Writable {
}
});
return Collections.unmodifiableList(l);
return ImmutableList.copyOf(l);
}
public final List<DialectConverterPlugin> getActiveDialectPluginList(Dialect dialect) {
Map<String, DialectConverterPlugin> m = dialectPlugins[dialect.ordinal()];
return ImmutableList.copyOf(m.values());
}
public final List<PluginInfo> getAllDynamicPluginInfo() {

View File

@ -84,7 +84,7 @@ public class AuditEventProcessor {
AuditEvent auditEvent;
while (!isStopped) {
// update audit plugin list every UPDATE_PLUGIN_INTERVAL_MS.
// because some of plugins may be installed or uninstalled at runtime.
// because some plugins may be installed or uninstalled at runtime.
if (auditPlugins == null || System.currentTimeMillis() - lastUpdateTime > UPDATE_PLUGIN_INTERVAL_MS) {
auditPlugins = pluginMgr.getActivePluginList(PluginType.AUDIT);
lastUpdateTime = System.currentTimeMillis();

View File

@ -36,7 +36,6 @@ import org.apache.doris.common.ErrorCode;
import org.apache.doris.common.NotImplementedException;
import org.apache.doris.common.UserException;
import org.apache.doris.common.util.DebugUtil;
import org.apache.doris.common.util.SQLDialectUtils;
import org.apache.doris.common.util.SqlParserUtils;
import org.apache.doris.common.util.SqlUtils;
import org.apache.doris.common.util.Util;
@ -50,8 +49,11 @@ import org.apache.doris.mysql.MysqlServerStatusFlag;
import org.apache.doris.nereids.exceptions.NotSupportedException;
import org.apache.doris.nereids.glue.LogicalPlanAdapter;
import org.apache.doris.nereids.minidump.MinidumpUtils;
import org.apache.doris.nereids.parser.Dialect;
import org.apache.doris.nereids.parser.NereidsParser;
import org.apache.doris.nereids.stats.StatsErrorEstimator;
import org.apache.doris.plugin.DialectConverterPlugin;
import org.apache.doris.plugin.PluginMgr;
import org.apache.doris.proto.Data;
import org.apache.doris.qe.QueryState.MysqlStateType;
import org.apache.doris.thrift.TMasterOpRequest;
@ -61,6 +63,7 @@ import org.apache.doris.thrift.TUniqueId;
import com.google.common.base.Preconditions;
import com.google.common.base.Strings;
import org.apache.commons.codec.digest.DigestUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@ -70,6 +73,7 @@ import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import java.util.UUID;
import javax.annotation.Nullable;
/**
* Process one connection, the life cycle is the same as connection
@ -174,8 +178,7 @@ public abstract class ConnectProcessor {
MetricRepo.COUNTER_REQUEST_ALL.increase(1L);
}
String convertedStmt = SQLDialectUtils.convertStmtWithDialect(originStmt, ctx, mysqlCommand);
String convertedStmt = convertOriginStmt(originStmt);
String sqlHash = DigestUtils.md5Hex(convertedStmt);
ctx.setSqlHash(sqlHash);
ctx.getAuditEventBuilder().reset();
@ -284,6 +287,28 @@ public abstract class ConnectProcessor {
}
private String convertOriginStmt(String originStmt) {
String convertedStmt = originStmt;
@Nullable Dialect sqlDialect = Dialect.getByName(ctx.getSessionVariable().getSqlDialect());
if (sqlDialect != null) {
PluginMgr pluginMgr = Env.getCurrentEnv().getPluginMgr();
List<DialectConverterPlugin> plugins = pluginMgr.getActiveDialectPluginList(sqlDialect);
for (DialectConverterPlugin plugin : plugins) {
try {
String convertedSql = plugin.convertSql(originStmt, ctx.getSessionVariable());
if (StringUtils.isNotEmpty(convertedSql)) {
convertedStmt = convertedSql;
break;
}
} catch (Throwable throwable) {
LOG.warn("Convert sql with dialect {} failed, plugin: {}, sql: {}, use origin sql.",
sqlDialect, plugin.getClass().getSimpleName(), originStmt, throwable);
}
}
}
return convertedStmt;
}
// Use a handler for exception to avoid big try catch block which is a little hard to understand
protected void handleQueryException(Throwable throwable, String origStmt,
StatementBase parsedStmt, Data.PQueryStatistics statistics) {

View File

@ -28,8 +28,7 @@ import org.apache.doris.common.io.Writable;
import org.apache.doris.common.util.TimeUtils;
import org.apache.doris.nereids.metrics.Event;
import org.apache.doris.nereids.metrics.EventSwitchParser;
import org.apache.doris.nereids.parser.ParseDialect;
import org.apache.doris.nereids.parser.ParseDialect.Dialect;
import org.apache.doris.nereids.parser.Dialect;
import org.apache.doris.nereids.rules.RuleType;
import org.apache.doris.planner.GroupCommitBlockSink;
import org.apache.doris.qe.VariableMgr.VarAttr;
@ -2333,8 +2332,8 @@ public class SessionVariable implements Serializable, Writable {
return waitFullBlockScheduleTimes;
}
public ParseDialect.Dialect getSqlParseDialect() {
return ParseDialect.Dialect.getByName(sqlDialect);
public Dialect getSqlParseDialect() {
return Dialect.getByName(sqlDialect);
}
public void setSqlDialect(String sqlDialect) {

View File

@ -21,7 +21,6 @@ import org.apache.doris.analysis.StatementBase;
import org.apache.doris.common.Config;
import org.apache.doris.common.Pair;
import org.apache.doris.nereids.StatementContext;
import org.apache.doris.nereids.analyzer.UnboundResultSink;
import org.apache.doris.nereids.exceptions.AnalysisException;
import org.apache.doris.nereids.exceptions.ParseException;
import org.apache.doris.nereids.glue.LogicalPlanAdapter;
@ -42,7 +41,6 @@ import org.apache.doris.nereids.types.DateTimeType;
import org.apache.doris.nereids.types.DateType;
import org.apache.doris.nereids.types.DecimalV2Type;
import org.apache.doris.nereids.types.DecimalV3Type;
import org.apache.doris.qe.SessionVariable;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
@ -189,55 +187,6 @@ public class NereidsParserTest extends ParserTestBase {
Assertions.assertTrue(logicalPlan1 instanceof ExplainCommand);
}
@Test
public void testParseSQLWithTrinoDialect() {
String sql = "select `AD``D` from t1 where a = 1;explain graph select `AD``D` from t1 where a = 1;";
NereidsParser nereidsParser = new NereidsParser();
SessionVariable sessionVariable = new SessionVariable();
sessionVariable.setSqlDialect("trino");
// test fall back to doris parser
List<StatementBase> statementBases = nereidsParser.parseSQL(sql, sessionVariable);
Assertions.assertEquals(2, statementBases.size());
Assertions.assertTrue(statementBases.get(0) instanceof LogicalPlanAdapter);
Assertions.assertTrue(statementBases.get(1) instanceof LogicalPlanAdapter);
LogicalPlan logicalPlan0 = ((LogicalPlanAdapter) statementBases.get(0)).getLogicalPlan();
LogicalPlan logicalPlan1 = ((LogicalPlanAdapter) statementBases.get(1)).getLogicalPlan();
Assertions.assertTrue(logicalPlan0 instanceof UnboundResultSink);
Assertions.assertTrue(logicalPlan1 instanceof ExplainCommand);
}
@Test
public void testParseSingleStmtWithTrinoDialect() {
String sql = "select `AD``D` from t1 where a = 1";
NereidsParser nereidsParser = new NereidsParser();
SessionVariable sessionVariable = new SessionVariable();
sessionVariable.setSqlDialect("trino");
// test fall back to doris parser
List<StatementBase> statementBases = nereidsParser.parseSQL(sql, sessionVariable);
Assertions.assertEquals(1, statementBases.size());
Assertions.assertTrue(statementBases.get(0) instanceof LogicalPlanAdapter);
LogicalPlan logicalPlan0 = ((LogicalPlanAdapter) statementBases.get(0)).getLogicalPlan();
Assertions.assertTrue(logicalPlan0 instanceof UnboundResultSink);
}
@Test
public void testParseSQLWithSparkSqlDialect() {
// doris parser will throw a ParseException when derived table does not have alias
String sql1 = "select * from (select * from t1);";
NereidsParser nereidsParser = new NereidsParser();
Assertions.assertThrows(ParseException.class, () -> nereidsParser.parseSQL(sql1),
"Every derived table must have its own alias");
// test parse with spark-sql dialect
SessionVariable sessionVariable = new SessionVariable();
sessionVariable.setSqlDialect("spark_sql");
List<StatementBase> statementBases = nereidsParser.parseSQL(sql1, sessionVariable);
Assertions.assertEquals(1, statementBases.size());
Assertions.assertTrue(statementBases.get(0) instanceof LogicalPlanAdapter);
LogicalPlan logicalPlan = ((LogicalPlanAdapter) statementBases.get(0)).getLogicalPlan();
Assertions.assertTrue(logicalPlan instanceof UnboundResultSink);
}
@Test
public void testParseJoin() {
NereidsParser nereidsParser = new NereidsParser();
@ -440,7 +389,6 @@ public class NereidsParserTest extends ParserTestBase {
}
@Test
void testParseExprDepthWidth() {
String sql = "SELECT 1+2 = 3 from t";
NereidsParser nereidsParser = new NereidsParser();

View File

@ -20,7 +20,6 @@ package org.apache.doris.nereids.parser;
import org.apache.doris.nereids.util.ExpressionParseChecker;
import org.apache.doris.nereids.util.MemoPatternMatchSupported;
import org.apache.doris.nereids.util.PlanParseChecker;
import org.apache.doris.nereids.util.TrinoDialectPlanParseChecker;
import org.apache.doris.qe.ConnectContext;
import mockit.Mock;
@ -51,7 +50,4 @@ public abstract class ParserTestBase implements MemoPatternMatchSupported {
return new ExpressionParseChecker(sql);
}
public TrinoDialectPlanParseChecker trinoDialectParsePlan(String sql) {
return new TrinoDialectPlanParseChecker(sql);
}
}

View File

@ -1,60 +0,0 @@
// 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.nereids.util;
import org.apache.doris.nereids.parser.ParseDialect;
import org.apache.doris.nereids.parser.ParserContext;
import org.apache.doris.nereids.parser.trino.TrinoParser;
import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
import com.google.common.base.Supplier;
import com.google.common.base.Suppliers;
import org.apache.commons.lang3.StringUtils;
import org.junit.jupiter.api.Assertions;
/**
* Plan parse checker for trino.
* It supports equals or contain pattern match assert and so on.
*/
public class TrinoDialectPlanParseChecker extends ParseChecker {
private final Supplier<LogicalPlan> parsedPlanSupplier;
public TrinoDialectPlanParseChecker(String sql) {
super(sql);
this.parsedPlanSupplier =
Suppliers.memoize(() -> TrinoParser.parseSingle(sql, new ParserContext(ParseDialect.TRINO_395)));
}
public TrinoDialectPlanParseChecker assertEquals(LogicalPlan plan) {
LogicalPlan target = parsedPlanSupplier.get();
Assertions.assertEquals(plan, target);
return this;
}
public TrinoDialectPlanParseChecker assertContains(String... expects) {
LogicalPlan logicalPlan = parsedPlanSupplier.get();
Assertions.assertNotNull(logicalPlan);
String targetPlanString = logicalPlan.toString();
for (String expected : expects) {
Assertions.assertTrue(StringUtils.containsIgnoreCase(targetPlanString.toLowerCase(), expected),
"expected contain is: " + expected + " but plan is \n" + targetPlanString);
}
return this;
}
}

View File

@ -0,0 +1,76 @@
// 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.plugin;
import org.apache.doris.analysis.StatementBase;
import org.apache.doris.catalog.Env;
import org.apache.doris.nereids.glue.LogicalPlanAdapter;
import org.apache.doris.nereids.parser.Dialect;
import org.apache.doris.nereids.parser.NereidsParser;
import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
import org.apache.doris.qe.ConnectContext;
import org.apache.doris.utframe.TestWithFeService;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.util.List;
public class DialectPluginTest extends TestWithFeService {
private static final TestDialectPlugin1 sparkPlugin = new TestDialectPlugin1();
private static final TestDialectPlugin2 hivePlugin = new TestDialectPlugin2();
private static final String TEST_SQL = "select * from test_hive_table";
@Override
public void runBeforeAll() throws IOException, InterruptedException {
PluginInfo sparkPluginInfo = new PluginInfo("sparkDialectPlugin", PluginInfo.PluginType.DIALECT, "test");
Env.getCurrentEnv().getPluginMgr().registerBuiltinPlugin(sparkPluginInfo, sparkPlugin);
PluginInfo hivePluginInfo = new PluginInfo("hiveDialectPlugin", PluginInfo.PluginType.DIALECT, "test");
Env.getCurrentEnv().getPluginMgr().registerBuiltinPlugin(hivePluginInfo, hivePlugin);
}
@Test
public void testHivePlugin() {
ConnectContext.get().getSessionVariable().setSqlDialect(Dialect.HIVE.getDialectName());
NereidsParser parser = new NereidsParser();
List<StatementBase> stmts = parser.parseSQL(TEST_SQL, ConnectContext.get().getSessionVariable());
Assertions.assertEquals(1, stmts.size());
Assertions.assertTrue(stmts.get(0) instanceof LogicalPlanAdapter);
LogicalPlan logicalPlan = ((LogicalPlanAdapter) stmts.get(0)).getLogicalPlan();
String convertedSql = hivePlugin.convertSql(TEST_SQL, ConnectContext.get().getSessionVariable());
Assertions.assertEquals(logicalPlan, parser.parseSingle(convertedSql));
}
@Test
public void testSparkPlugin() {
ConnectContext.get().getSessionVariable().setSqlDialect(Dialect.SPARK_SQL.getDialectName());
NereidsParser parser = new NereidsParser();
List<StatementBase> stmts = parser.parseSQL(TEST_SQL, ConnectContext.get().getSessionVariable());
Assertions.assertEquals(1, stmts.size());
Assertions.assertTrue(stmts.get(0) instanceof LogicalPlanAdapter);
LogicalPlan logicalPlan = ((LogicalPlanAdapter) stmts.get(0)).getLogicalPlan();
List<StatementBase> expectedStmts = sparkPlugin.parseSqlWithDialect(TEST_SQL,
ConnectContext.get().getSessionVariable());
Assertions.assertTrue(expectedStmts != null && expectedStmts.size() == 1);
Assertions.assertTrue(expectedStmts.get(0) instanceof LogicalPlanAdapter);
Assertions.assertEquals(logicalPlan, ((LogicalPlanAdapter) expectedStmts.get(0)).getLogicalPlan());
}
}

View File

@ -0,0 +1,42 @@
// 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.plugin;
import org.apache.doris.analysis.StatementBase;
import org.apache.doris.nereids.parser.Dialect;
import org.apache.doris.nereids.parser.NereidsParser;
import org.apache.doris.qe.SessionVariable;
import com.google.common.collect.ImmutableSet;
import java.util.List;
public class TestDialectPlugin1 extends Plugin implements DialectConverterPlugin {
private static final String TEST_CONVERTED_SQL = "select * from rename_hive_table";
@Override
public ImmutableSet<Dialect> acceptDialects() {
return ImmutableSet.of(Dialect.SPARK_SQL);
}
@Override
public List<StatementBase> parseSqlWithDialect(String sql, SessionVariable sessionVariable) {
return new NereidsParser().parseSQL(TEST_CONVERTED_SQL);
}
}

View File

@ -17,43 +17,30 @@
package org.apache.doris.plugin;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import org.apache.doris.analysis.StatementBase;
import org.apache.doris.nereids.parser.Dialect;
import org.apache.doris.qe.SessionVariable;
public class PluginTest extends Plugin {
import com.google.common.collect.ImmutableSet;
private Map<String, String> map = new HashMap<>();
import java.util.List;
public class TestDialectPlugin2 extends Plugin implements DialectConverterPlugin {
private static final String TEST_CONVERTED_SQL = "select * from rename_hive_table";
@Override
public void init(PluginInfo info, PluginContext ctx) {
System.out.println("this is init");
}
@Override
public void close() throws IOException {
super.close();
System.out.println("this is close");
public ImmutableSet<Dialect> acceptDialects() {
return ImmutableSet.of(Dialect.HIVE);
}
@Override
public int flags() {
return 2;
public String convertSql(String originSql, SessionVariable sessionVariable) {
return TEST_CONVERTED_SQL;
}
@Override
public void setVariable(String key, String value) {
map.put(key, value);
}
@Override
public Map<String, String> variable() {
return map;
}
@Override
public Map<String, String> status() {
return new HashMap<>();
public List<StatementBase> parseSqlWithDialect(String sql, SessionVariable sessionVariable) {
return null;
}
}

View File

@ -345,7 +345,6 @@ under the License.
<!-- paimon -->
<paimon.version>0.6.0-incubating</paimon.version>
<disruptor.version>3.4.4</disruptor.version>
<trino.parser.version>395</trino.parser.version>
<!-- arrow flight sql -->
<arrow.vector.classifier>shade-format-flatbuffers</arrow.vector.classifier>
<flatbuffers.version>1.12.0</flatbuffers.version>
@ -1496,13 +1495,6 @@ under the License.
<artifactId>client</artifactId>
<version>${vesoft.client.version}</version>
</dependency>
<!-- trino-parser The version of antlr-runtime in trino parser is need to be consistent with doris,
when upgrade trino-parser antlr-runtime version, should take care of trino-parser doris.-->
<dependency>
<groupId>io.trino</groupId>
<artifactId>trino-parser</artifactId>
<version>${trino.parser.version}</version>
</dependency>
<dependency>
<groupId>io.grpc</groupId>
<artifactId>grpc-netty</artifactId>

View File

@ -0,0 +1,119 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
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.
-->
<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xmlns="http://maven.apache.org/POM/4.0.0"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<parent>
<groupId>org.apache.doris</groupId>
<artifactId>fe-plugins</artifactId>
<version>1.0-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>
<modelVersion>4.0.0</modelVersion>
<artifactId>http-dialect-converter</artifactId>
<packaging>jar</packaging>
<dependencies>
<dependency>
<groupId>org.apache.doris</groupId>
<artifactId>fe-core</artifactId>
<version>${doris.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.doris</groupId>
<artifactId>fe-common</artifactId>
<version>${doris.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.projectlombok</groupId>
<artifactId>lombok</artifactId>
<version>${lombok.version}</version>
<scope>provided</scope>
</dependency>
<!-- https://mvnrepository.com/artifact/org.apache.logging.log4j/log4j-api -->
<dependency>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-api</artifactId>
</dependency>
<!-- https://mvnrepository.com/artifact/org.apache.logging.log4j/log4j-core -->
<dependency>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-core</artifactId>
</dependency>
<!-- https://mvnrepository.com/artifact/org.apache.logging.log4j/log4j-slf4j-impl -->
<dependency>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-slf4j-impl</artifactId>
</dependency>
<!-- https://mvnrepository.com/artifact/log4j/log4j -->
<dependency>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</dependency>
<!-- https://mvnrepository.com/artifact/org.junit.jupiter/junit-jupiter-engine -->
<dependency>
<groupId>org.junit.jupiter</groupId>
<artifactId>junit-jupiter-engine</artifactId>
<scope>test</scope>
</dependency>
<!-- https://mvnrepository.com/artifact/org.junit.vintage/junit-vintage-engine -->
<dependency>
<groupId>org.junit.vintage</groupId>
<artifactId>junit-vintage-engine</artifactId>
<scope>test</scope>
</dependency>
<!-- https://mvnrepository.com/artifact/org.junit.jupiter/junit-jupiter-params -->
<dependency>
<groupId>org.junit.jupiter</groupId>
<artifactId>junit-jupiter-params</artifactId>
<scope>test</scope>
</dependency>
<!-- https://mvnrepository.com/artifact/org.jmockit/jmockit -->
<dependency>
<groupId>org.jmockit</groupId>
<artifactId>jmockit</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
<build>
<finalName>presto-converter</finalName>
<plugins>
<plugin>
<artifactId>maven-assembly-plugin</artifactId>
<version>2.4.1</version>
<configuration>
<appendAssemblyId>false</appendAssemblyId>
<descriptors>
<descriptor>src/main/assembly/zip.xml</descriptor>
</descriptors>
</configuration>
<executions>
<execution>
<id>make-assembly</id>
<phase>package</phase>
<goals>
<goal>single</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
</project>

View File

@ -0,0 +1,22 @@
# 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.
### plugin configuration
# Replace the target url, set your sql converter service url here
target_url=http://127.0.0.1:8080/api/v1/sql/convert
# Replace the dialects if you need, use comma to split the value, for instance: presto,trino,hive
accept_dialects=presto

View File

@ -0,0 +1,23 @@
# 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.
name=HttpDialectConverter
type=DIALECT
description=SQL dialect converter plugin using http protocol.
version=1.0.0
java.version=1.8.0
classname=org.apache.doris.plugin.dialect.http.HttpDialectConverterPlugin

View File

@ -0,0 +1,43 @@
<?xml version="1.0"?>
<!--
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.
-->
<assembly>
<id>plugin</id>
<formats>
<format>zip</format>
</formats>
<includeBaseDirectory>false</includeBaseDirectory>
<fileSets>
<fileSet>
<directory>target</directory>
<includes>
<include>*.jar</include>
</includes>
<outputDirectory>/</outputDirectory>
</fileSet>
<fileSet>
<directory>src/main/assembly</directory>
<includes>
<include>plugin.properties</include>
<include>plugin.conf</include>
</includes>
<outputDirectory>/</outputDirectory>
</fileSet>
</fileSets>
</assembly>

View File

@ -0,0 +1,144 @@
// 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.plugin.dialect.http;
import org.apache.doris.analysis.StatementBase;
import org.apache.doris.nereids.parser.Dialect;
import org.apache.doris.plugin.DialectConverterPlugin;
import org.apache.doris.plugin.Plugin;
import org.apache.doris.plugin.PluginContext;
import org.apache.doris.plugin.PluginException;
import org.apache.doris.plugin.PluginInfo;
import org.apache.doris.qe.SessionVariable;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableSet;
import java.io.IOException;
import java.io.InputStream;
import java.nio.file.FileSystems;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Properties;
import java.util.function.Function;
import java.util.stream.Collectors;
import javax.annotation.Nullable;
/**
* Currently, there are many frameworks and services that support SQL dialect conversion,
* but they may not be implemented in Java.
* Therefore, we can encapsulate these SQL dialect conversion frameworks or services into an HTTP service,
* and combine them with this plugin to provide dialect conversion capabilities.
* Note that the protocol request/response for the wrapped HTTP service must comply with the following rules:
* <pre>
* Request body:
* {
* "version": "v1",
* "sql": "select * from t",
* "from": "presto",
* "to": "doris",
* "source": "text",
* "case_sensitive": "0"
* }
*
* Response body:
* {
* "version": "v1",
* "data": "select * from t",
* "code": 0,
* "message": ""
* }
* </pre>
* */
public class HttpDialectConverterPlugin extends Plugin implements DialectConverterPlugin {
private volatile boolean isInit = false;
private volatile boolean isClosed = false;
private volatile String targetURL = null;
private volatile ImmutableSet<Dialect> acceptDialects = null;
@Override
public void init(PluginInfo info, PluginContext ctx) throws PluginException {
super.init(info, ctx);
synchronized (this) {
if (isInit) {
return;
}
loadConfig(ctx, info.getProperties());
isInit = true;
}
}
private void loadConfig(PluginContext ctx, Map<String, String> pluginInfoProperties) throws PluginException {
Path pluginPath = FileSystems.getDefault().getPath(ctx.getPluginPath());
if (!Files.exists(pluginPath)) {
throw new PluginException("plugin path does not exist: " + pluginPath);
}
Path confFile = pluginPath.resolve("plugin.conf");
if (!Files.exists(confFile)) {
throw new PluginException("plugin conf file does not exist: " + confFile);
}
final Properties props = new Properties();
try (InputStream stream = Files.newInputStream(confFile)) {
props.load(stream);
} catch (IOException e) {
throw new PluginException(e.getMessage());
}
for (Map.Entry<String, String> entry : pluginInfoProperties.entrySet()) {
props.setProperty(entry.getKey(), entry.getValue());
}
final Map<String, String> properties = props.stringPropertyNames().stream()
.collect(Collectors.toMap(Function.identity(), props::getProperty));
targetURL = properties.get("target_url");
String acceptDialectsStr = Objects.requireNonNull(properties.get("accept_dialects"));
acceptDialects = ImmutableSet.copyOf(Arrays.stream(acceptDialectsStr.split(","))
.map(Dialect::getByName).collect(Collectors.toSet()));
}
@Override
public void close() throws IOException {
super.close();
isClosed = true;
}
@Override
public ImmutableSet<Dialect> acceptDialects() {
return acceptDialects;
}
@Override
public @Nullable String convertSql(String originSql, SessionVariable sessionVariable) {
Preconditions.checkNotNull(targetURL);
return HttpDialectUtils.convertSql(targetURL, originSql);
}
// no need to override parseSqlWithDialect, just return null
@Override
public List<StatementBase> parseSqlWithDialect(String sql, SessionVariable sessionVariable) {
return null;
}
}

View File

@ -15,12 +15,7 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.common.util;
import org.apache.doris.common.Config;
import org.apache.doris.mysql.MysqlCommand;
import org.apache.doris.nereids.parser.ParseDialect;
import org.apache.doris.qe.ConnectContext;
package org.apache.doris.plugin.dialect.http;
import com.google.gson.Gson;
import com.google.gson.reflect.TypeToken;
@ -37,51 +32,13 @@ import java.net.URL;
import java.nio.charset.StandardCharsets;
/**
* This class is used to convert sql with different dialects
* using sql convertor service.
* This class is used to convert sql with different dialects using sql convertor service.
* The sql convertor service is a http service which is used to convert sql.
* Request body:
* {
* "version": "v1",
* "sql": "select * from t",
* "from": "presto",
* "to": "doris",
* "source": "text",
* "case_sensitive": "0"
* }
* <p>
* Response body:
* {
* "version": "v1",
* "data": "select * from t",
* "code": 0,
* "message": ""
*/
public class SQLDialectUtils {
private static final Logger LOG = LogManager.getLogger(SQLDialectUtils.class);
public class HttpDialectUtils {
private static final Logger LOG = LogManager.getLogger(HttpDialectUtils.class);
public static String convertStmtWithDialect(String originStmt, ConnectContext ctx, MysqlCommand mysqlCommand) {
if (mysqlCommand != MysqlCommand.COM_QUERY) {
return originStmt;
}
if (Config.sql_convertor_service.isEmpty()) {
return originStmt;
}
ParseDialect.Dialect dialect = ctx.getSessionVariable().getSqlParseDialect();
if (dialect == null) {
return originStmt;
}
switch (dialect) {
case PRESTO:
return convertStmtWithPresto(originStmt);
default:
LOG.debug("only support presto dialect now.");
return originStmt;
}
}
private static String convertStmtWithPresto(String originStmt) {
String targetURL = Config.sql_convertor_service;
public static String convertSql(String targetURL, String originStmt) {
ConvertRequest convertRequest = new ConvertRequest(originStmt, "presto");
HttpURLConnection connection = null;

View File

@ -15,13 +15,7 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.common.util;
import org.apache.doris.common.Config;
import org.apache.doris.mysql.MysqlCommand;
import org.apache.doris.qe.ConnectContext;
import org.apache.doris.utframe.SimpleHttpServer;
import org.apache.doris.utframe.TestWithFeService;
package org.apache.doris.plugin.dialect.http;
import org.junit.After;
import org.junit.Assert;
@ -29,15 +23,18 @@ import org.junit.Before;
import org.junit.Test;
import java.io.IOException;
import java.net.DatagramSocket;
import java.net.ServerSocket;
import java.net.SocketException;
public class SQLDialectUtilsTest {
public class HttpDialectUtilsTest {
int port;
SimpleHttpServer server;
private int port;
private SimpleHttpServer server;
@Before
public void setUp() throws Exception {
port = TestWithFeService.findValidPort();
port = findValidPort();
server = new SimpleHttpServer(port);
server.start("/api/v1/convert");
}
@ -50,37 +47,44 @@ public class SQLDialectUtilsTest {
}
@Test
public void testSqlConvert() throws IOException {
public void testSqlConvert() {
String originSql = "select * from t1 where \"k1\" = 1";
String expectedSql = "select * from t1 where `k1` = 1";
ConnectContext ctx = TestWithFeService.createDefaultCtx();
// 1. not COM_QUERY
String res = SQLDialectUtils.convertStmtWithDialect(originSql, ctx, MysqlCommand.COM_STMT_RESET);
String targetURL = "http://127.0.0.1:" + port + "/api/v1/convert";
String res = HttpDialectUtils.convertSql(targetURL, originSql);
Assert.assertEquals(originSql, res);
// 2. config sql_convertor_service not set
res = SQLDialectUtils.convertStmtWithDialect(originSql, ctx, MysqlCommand.COM_QUERY);
Assert.assertEquals(originSql, res);
// 3. session var sql_dialect not set
Config.sql_convertor_service = "http://127.0.0.1:" + port + "/api/v1/convert";
res = SQLDialectUtils.convertStmtWithDialect(originSql, ctx, MysqlCommand.COM_QUERY);
Assert.assertEquals(originSql, res);
// 4. not support dialect
ctx.getSessionVariable().setSqlDialect("sqlserver");
res = SQLDialectUtils.convertStmtWithDialect(originSql, ctx, MysqlCommand.COM_QUERY);
Assert.assertEquals(originSql, res);
// 5. test presto
ctx.getSessionVariable().setSqlDialect("presto");
// test presto
server.setResponse("{\"version\": \"v1\", \"data\": \"" + expectedSql + "\", \"code\": 0, \"message\": \"\"}");
res = SQLDialectUtils.convertStmtWithDialect(originSql, ctx, MysqlCommand.COM_QUERY);
res = HttpDialectUtils.convertSql(targetURL, originSql);
Assert.assertEquals(expectedSql, res);
// 6. test response version error
// test response version error
server.setResponse("{\"version\": \"v2\", \"data\": \"" + expectedSql + "\", \"code\": 0, \"message\": \"\"}");
res = SQLDialectUtils.convertStmtWithDialect(originSql, ctx, MysqlCommand.COM_QUERY);
res = HttpDialectUtils.convertSql(targetURL, originSql);
Assert.assertEquals(originSql, res);
// 7. test response code error
server.setResponse(
"{\"version\": \"v1\", \"data\": \"" + expectedSql + "\", \"code\": 400, \"message\": \"\"}");
res = SQLDialectUtils.convertStmtWithDialect(originSql, ctx, MysqlCommand.COM_QUERY);
res = HttpDialectUtils.convertSql(targetURL, originSql);
Assert.assertEquals(originSql, res);
}
private static int findValidPort() {
int port;
while (true) {
try (ServerSocket socket = new ServerSocket(0)) {
socket.setReuseAddress(true);
port = socket.getLocalPort();
try (DatagramSocket datagramSocket = new DatagramSocket(port)) {
datagramSocket.setReuseAddress(true);
break;
} catch (SocketException e) {
System.out.println("The port " + port + " is invalid and try another port.");
}
} catch (IOException e) {
throw new IllegalStateException("Could not find a free TCP/IP port to start HTTP Server on");
}
}
return port;
}
}

View File

@ -15,7 +15,7 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.utframe;
package org.apache.doris.plugin.dialect.http;
import com.sun.net.httpserver.HttpExchange;
import com.sun.net.httpserver.HttpHandler;

View File

@ -65,11 +65,20 @@ under the License.
<modules>
<module>auditdemo</module>
<module>auditloader</module>
<module>http-dialect-converter</module>
<module>trino-converter</module>
<module>sparksql-converter</module>
</modules>
<properties>
<log4j2.version>2.18.0</log4j2.version>
<doris.version>2.0-SNAPSHOT</doris.version>
<project.scm.id>github</project.scm.id>
<plugin-ut.parallel>1</plugin-ut.parallel>
<doris.version>1.2-SNAPSHOT</doris.version>
<trino.parser.version>395</trino.parser.version>
<log4j2.version>2.18.0</log4j2.version>
<log4j.verion>1.2.17</log4j.verion>
<lombok.version>1.18.24</lombok.version>
<junit.version>5.8.2</junit.version>
<jmockit.version>1.49</jmockit.version>
</properties>
<profiles>
<!-- for custom internal repository -->
@ -137,7 +146,35 @@ under the License.
<dependency>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
<version>1.2.17</version>
<version>${log4j.verion}</version>
</dependency>
<!-- https://mvnrepository.com/artifact/org.jmockit/jmockit -->
<dependency>
<groupId>org.jmockit</groupId>
<artifactId>jmockit</artifactId>
<version>${jmockit.version}</version>
<scope>test</scope>
</dependency>
<!-- https://mvnrepository.com/artifact/org.junit.jupiter/junit-jupiter-engine -->
<dependency>
<groupId>org.junit.jupiter</groupId>
<artifactId>junit-jupiter-engine</artifactId>
<version>${junit.version}</version>
<scope>test</scope>
</dependency>
<!-- https://mvnrepository.com/artifact/org.junit.vintage/junit-vintage-engine -->
<dependency>
<groupId>org.junit.vintage</groupId>
<artifactId>junit-vintage-engine</artifactId>
<version>${junit.version}</version>
<scope>test</scope>
</dependency>
<!-- https://mvnrepository.com/artifact/org.junit.jupiter/junit-jupiter-params -->
<dependency>
<groupId>org.junit.jupiter</groupId>
<artifactId>junit-jupiter-params</artifactId>
<version>${junit.version}</version>
<scope>test</scope>
</dependency>
</dependencies>
</dependencyManagement>

View File

@ -0,0 +1,132 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
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.
-->
<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xmlns="http://maven.apache.org/POM/4.0.0"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<parent>
<groupId>org.apache.doris</groupId>
<artifactId>fe-plugins</artifactId>
<version>1.0-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>
<modelVersion>4.0.0</modelVersion>
<artifactId>sparksql-converter</artifactId>
<packaging>jar</packaging>
<dependencies>
<dependency>
<groupId>org.apache.doris</groupId>
<artifactId>fe-core</artifactId>
<version>${doris.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.doris</groupId>
<artifactId>fe-common</artifactId>
<version>${doris.version}</version>
<scope>provided</scope>
</dependency>
<!-- https://mvnrepository.com/artifact/org.apache.logging.log4j/log4j-api -->
<dependency>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-api</artifactId>
<scope>provided</scope>
</dependency>
<!-- https://mvnrepository.com/artifact/org.apache.logging.log4j/log4j-core -->
<dependency>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-core</artifactId>
<scope>provided</scope>
</dependency>
<!-- https://mvnrepository.com/artifact/org.apache.logging.log4j/log4j-slf4j-impl -->
<dependency>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-slf4j-impl</artifactId>
<scope>provided</scope>
</dependency>
<!-- https://mvnrepository.com/artifact/log4j/log4j -->
<dependency>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
<scope>provided</scope>
</dependency>
<!-- https://mvnrepository.com/artifact/org.junit.jupiter/junit-jupiter-engine -->
<dependency>
<groupId>org.junit.jupiter</groupId>
<artifactId>junit-jupiter-engine</artifactId>
<scope>test</scope>
</dependency>
<!-- https://mvnrepository.com/artifact/org.junit.vintage/junit-vintage-engine -->
<dependency>
<groupId>org.junit.vintage</groupId>
<artifactId>junit-vintage-engine</artifactId>
<scope>test</scope>
</dependency>
<!-- https://mvnrepository.com/artifact/org.junit.jupiter/junit-jupiter-params -->
<dependency>
<groupId>org.junit.jupiter</groupId>
<artifactId>junit-jupiter-params</artifactId>
<scope>test</scope>
</dependency>
<!-- https://mvnrepository.com/artifact/org.jmockit/jmockit -->
<dependency>
<groupId>org.jmockit</groupId>
<artifactId>jmockit</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
<build>
<finalName>sparksql-converter</finalName>
<plugins>
<plugin>
<artifactId>maven-assembly-plugin</artifactId>
<version>2.4.1</version>
<configuration>
<appendAssemblyId>false</appendAssemblyId>
<descriptors>
<descriptor>src/main/assembly/zip.xml</descriptor>
</descriptors>
</configuration>
<executions>
<execution>
<id>make-assembly</id>
<phase>package</phase>
<goals>
<goal>single</goal>
</goals>
</execution>
</executions>
</plugin>
<!-- jmockit -->
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<configuration>
<!-->set larger, eg, 3, to reduce the time or running plugin unit tests<-->
<forkCount>${plugin-ut.parallel}</forkCount>
<!-->not reuse forked jvm, so that each unit test will run in separate jvm. to avoid singleton conflict<-->
<reuseForks>false</reuseForks>
<useFile>false</useFile>
<argLine>
-javaagent:${settings.localRepository}/org/jmockit/jmockit/${jmockit.version}/jmockit-${jmockit.version}.jar
</argLine>
</configuration>
</plugin>
</plugins>
</build>
</project>

View File

@ -0,0 +1,18 @@
# 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.
### plugin configuration

View File

@ -0,0 +1,23 @@
# 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.
name=SparkSqlConverter
type=DIALECT
description=SQL dialect converter plugin for spark sql.
version=1.0.0
java.version=1.8.0
classname=org.apache.doris.plugin.dialect.spark.SparkSqlDialectConverterPlugin

View File

@ -0,0 +1,43 @@
<?xml version="1.0"?>
<!--
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.
-->
<assembly>
<id>plugin</id>
<formats>
<format>zip</format>
</formats>
<includeBaseDirectory>false</includeBaseDirectory>
<fileSets>
<fileSet>
<directory>target</directory>
<includes>
<include>*.jar</include>
</includes>
<outputDirectory>/</outputDirectory>
</fileSet>
<fileSet>
<directory>src/main/assembly</directory>
<includes>
<include>plugin.properties</include>
<include>plugin.conf</include>
</includes>
<outputDirectory>/</outputDirectory>
</fileSet>
</fileSets>
</assembly>

View File

@ -15,7 +15,7 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.parser.spark;
package org.apache.doris.plugin.dialect.spark;
import org.apache.doris.nereids.analyzer.UnboundFunction;
import org.apache.doris.nereids.parser.ComplexFnCallTransformer;
@ -55,13 +55,13 @@ public class DateTruncFnCallTransformer extends ComplexFnCallTransformer {
@Override
protected boolean check(String sourceFnName, List<Expression> sourceFnTransformedArguments,
ParserContext context) {
ParserContext context) {
return getSourceFnName().equalsIgnoreCase(sourceFnName) && (sourceFnTransformedArguments.size() == 2);
}
@Override
protected Function transform(String sourceFnName, List<Expression> sourceFnTransformedArguments,
ParserContext context) {
ParserContext context) {
VarcharLiteral fmtLiteral = (VarcharLiteral) sourceFnTransformedArguments.get(1);
if (YEAR.contains(fmtLiteral.getValue().toUpperCase())) {
return new UnboundFunction(
@ -78,4 +78,4 @@ public class DateTruncFnCallTransformer extends ComplexFnCallTransformer {
"date_trunc",
ImmutableList.of(sourceFnTransformedArguments.get(0), sourceFnTransformedArguments.get(1)));
}
}
}

View File

@ -15,16 +15,18 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.parser.spark;
package org.apache.doris.plugin.dialect.spark;
import org.apache.doris.nereids.analyzer.PlaceholderExpression;
import org.apache.doris.nereids.parser.AbstractFnCallTransformer;
import org.apache.doris.nereids.parser.AbstractFnCallTransformers;
import org.apache.doris.nereids.trees.expressions.Expression;
import com.google.common.collect.Lists;
/**
* The builder and factory for spark-sql 3.x FnCallTransformers, supply transform facade ability.
* The builder and factory for spark-sql 3.x {@link AbstractFnCallTransformer},
* and supply transform facade ability.
*/
public class SparkSql3FnCallTransformers extends AbstractFnCallTransformers {

View File

@ -15,13 +15,13 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.parser.spark;
package org.apache.doris.plugin.dialect.spark;
import org.apache.doris.nereids.DorisParser;
import org.apache.doris.nereids.analyzer.UnboundFunction;
import org.apache.doris.nereids.exceptions.ParseException;
import org.apache.doris.nereids.parser.Dialect;
import org.apache.doris.nereids.parser.LogicalPlanBuilder;
import org.apache.doris.nereids.parser.ParseDialect;
import org.apache.doris.nereids.parser.ParserContext;
import org.apache.doris.nereids.parser.ParserUtils;
import org.apache.doris.nereids.trees.expressions.Expression;
@ -42,7 +42,7 @@ public class SparkSql3LogicalPlanBuilder extends LogicalPlanBuilder {
private final ParserContext parserContext;
public SparkSql3LogicalPlanBuilder() {
this.parserContext = new ParserContext(ParseDialect.SPARK_SQL_3_ALL);
this.parserContext = new ParserContext(Dialect.SPARK_SQL);
}
@Override

View File

@ -0,0 +1,47 @@
// 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.plugin.dialect.spark;
import org.apache.doris.analysis.StatementBase;
import org.apache.doris.nereids.parser.Dialect;
import org.apache.doris.nereids.parser.NereidsParser;
import org.apache.doris.plugin.DialectConverterPlugin;
import org.apache.doris.plugin.Plugin;
import org.apache.doris.qe.SessionVariable;
import com.google.common.collect.ImmutableSet;
import java.util.List;
import javax.annotation.Nullable;
/**
* Extends from {@link org.apache.doris.plugin.DialectConverterPlugin},
* just focus on the syntax difference between spark-sql and doris.
*/
public class SparkSqlDialectConverterPlugin extends Plugin implements DialectConverterPlugin {
@Override
public ImmutableSet<Dialect> acceptDialects() {
return ImmutableSet.of(Dialect.SPARK_SQL);
}
@Override
public @Nullable List<StatementBase> parseSqlWithDialect(String sql, SessionVariable sessionVariable) {
return new NereidsParser().parseSQL(sql, new SparkSql3LogicalPlanBuilder());
}
}

View File

@ -15,21 +15,39 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.parser.spark;
package org.apache.doris.plugin.dialect.spark;
import org.apache.doris.nereids.parser.NereidsParser;
import org.apache.doris.nereids.parser.ParserTestBase;
import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
import org.apache.doris.qe.ConnectContext;
import org.apache.doris.qe.SessionVariable;
import org.apache.commons.lang3.StringUtils;
import mockit.Mock;
import mockit.MockUp;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
/**
* Spark SQL to Doris function mapping test.
*/
public class FnTransformTest extends ParserTestBase {
public class FnTransformTest {
@BeforeAll
public static void init() {
ConnectContext ctx = new ConnectContext();
SessionVariable sessionVariable = new SessionVariable();
ctx.setSessionVariable(sessionVariable);
new MockUp<ConnectContext>() {
@Mock
public ConnectContext get() {
return ctx;
}
};
}
@Test
public void testCommonFnTransformers() {
// test json functions
@ -40,35 +58,34 @@ public class FnTransformTest extends ParserTestBase {
testFunction("SELECT json_extract(c1, '$.c1') as b FROM t",
"SELECT get_json_object(c1, '$.c1') as b FROM t",
"json_extract('c1, '$.c1')");
// test string functions
testFunction("SELECT str_to_date('2023-12-16', 'yyyy-MM-dd') as b FROM t",
"SELECT to_date('2023-12-16', 'yyyy-MM-dd') as b FROM t",
"str_to_date('2023-12-16', 'yyyy-MM-dd')");
testFunction("SELECT str_to_date(c1, 'yyyy-MM-dd') as b FROM t",
"SELECT to_date(c1, 'yyyy-MM-dd') as b FROM t",
"str_to_date('c1, 'yyyy-MM-dd')");
"SELECT to_date(c1, 'yyyy-MM-dd') as b FROM t",
"str_to_date('c1, 'yyyy-MM-dd')");
testFunction("SELECT date_trunc('2023-12-16', 'YEAR') as a FROM t",
"SELECT trunc('2023-12-16', 'YEAR') as a FROM t",
"date_trunc('2023-12-16', 'YEAR')");
testFunction("SELECT date_trunc(c1, 'YEAR') as a FROM t",
"SELECT trunc(c1, 'YEAR') as a FROM t",
"date_trunc('c1, 'YEAR')");
"SELECT trunc(c1, 'YEAR') as a FROM t",
"date_trunc('c1, 'YEAR')");
testFunction("SELECT date_trunc('2023-12-16', 'YEAR') as a FROM t",
"SELECT trunc('2023-12-16', 'YY') as a FROM t",
"date_trunc('2023-12-16', 'YEAR')");
testFunction("SELECT date_trunc(c1, 'YEAR') as a FROM t",
"SELECT trunc(c1, 'YY') as a FROM t",
"date_trunc('c1, 'YEAR')");
"SELECT trunc(c1, 'YY') as a FROM t",
"date_trunc('c1, 'YEAR')");
testFunction("SELECT date_trunc('2023-12-16', 'MONTH') as a FROM t",
"SELECT trunc('2023-12-16', 'MON') as a FROM t",
"date_trunc('2023-12-16', 'MONTH')");
testFunction("SELECT date_trunc(c1, 'MONTH') as a FROM t",
"SELECT trunc(c1, 'MON') as a FROM t",
"date_trunc('c1, 'MONTH')");
"SELECT trunc(c1, 'MON') as a FROM t",
"date_trunc('c1, 'MONTH')");
// test numeric functions
testFunction("SELECT avg(c1) as a from t",

View File

@ -0,0 +1,64 @@
// 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.plugin.dialect.spark;
import org.apache.doris.nereids.analyzer.UnboundResultSink;
import org.apache.doris.nereids.exceptions.ParseException;
import org.apache.doris.nereids.parser.NereidsParser;
import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
import org.apache.doris.qe.ConnectContext;
import org.apache.doris.qe.SessionVariable;
import mockit.Mock;
import mockit.MockUp;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
/**
* Spark-sql query tests.
*/
public class ParserTest {
@BeforeAll
public static void init() {
ConnectContext ctx = new ConnectContext();
SessionVariable sessionVariable = new SessionVariable();
ctx.setSessionVariable(sessionVariable);
new MockUp<ConnectContext>() {
@Mock
public ConnectContext get() {
return ctx;
}
};
}
@Test
public void testParseCast1() {
// doris parser will throw a ParseException when derived table does not have alias
String sql1 = "select * from (select * from t1);";
NereidsParser nereidsParser = new NereidsParser();
Assertions.assertThrows(ParseException.class, () -> nereidsParser.parseSQL(sql1),
"Every derived table must have its own alias");
// test parse with spark-sql dialect
LogicalPlan dialectLogicalPlan = nereidsParser.parseSingle(sql1, new SparkSql3LogicalPlanBuilder());
Assertions.assertTrue(dialectLogicalPlan instanceof UnboundResultSink);
}
}

View File

@ -0,0 +1,159 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
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.
-->
<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xmlns="http://maven.apache.org/POM/4.0.0"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<parent>
<groupId>org.apache.doris</groupId>
<artifactId>fe-plugins</artifactId>
<version>1.0-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>
<modelVersion>4.0.0</modelVersion>
<artifactId>trino-converter</artifactId>
<packaging>jar</packaging>
<dependencies>
<dependency>
<groupId>org.apache.doris</groupId>
<artifactId>fe-core</artifactId>
<version>${doris.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.doris</groupId>
<artifactId>fe-common</artifactId>
<version>${doris.version}</version>
<scope>provided</scope>
</dependency>
<!-- trino-parser The version of antlr-runtime in trino parser is need to be consistent with doris,
when upgrade trino-parser antlr-runtime version, should take care of trino-parser doris.-->
<dependency>
<groupId>io.trino</groupId>
<artifactId>trino-parser</artifactId>
<version>${trino.parser.version}</version>
<scope>compile</scope>
</dependency>
<!-- https://mvnrepository.com/artifact/org.apache.logging.log4j/log4j-api -->
<dependency>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-api</artifactId>
<scope>provided</scope>
</dependency>
<!-- https://mvnrepository.com/artifact/org.apache.logging.log4j/log4j-core -->
<dependency>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-core</artifactId>
<scope>provided</scope>
</dependency>
<!-- https://mvnrepository.com/artifact/org.apache.logging.log4j/log4j-slf4j-impl -->
<dependency>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-slf4j-impl</artifactId>
<scope>provided</scope>
</dependency>
<!-- https://mvnrepository.com/artifact/log4j/log4j -->
<dependency>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
<scope>provided</scope>
</dependency>
<!-- https://mvnrepository.com/artifact/org.junit.jupiter/junit-jupiter-engine -->
<dependency>
<groupId>org.junit.jupiter</groupId>
<artifactId>junit-jupiter-engine</artifactId>
<scope>test</scope>
</dependency>
<!-- https://mvnrepository.com/artifact/org.junit.vintage/junit-vintage-engine -->
<dependency>
<groupId>org.junit.vintage</groupId>
<artifactId>junit-vintage-engine</artifactId>
<scope>test</scope>
</dependency>
<!-- https://mvnrepository.com/artifact/org.junit.jupiter/junit-jupiter-params -->
<dependency>
<groupId>org.junit.jupiter</groupId>
<artifactId>junit-jupiter-params</artifactId>
<scope>test</scope>
</dependency>
<!-- https://mvnrepository.com/artifact/org.jmockit/jmockit -->
<dependency>
<groupId>org.jmockit</groupId>
<artifactId>jmockit</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
<build>
<finalName>trino-converter</finalName>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-dependency-plugin</artifactId>
<version>3.1.2</version>
<executions>
<execution>
<id>copy-dependencies</id>
<phase>package</phase>
<goals>
<goal>copy-dependencies</goal>
</goals>
<configuration>
<outputDirectory>${project.build.directory}</outputDirectory>
<overWriteIfNewer>true</overWriteIfNewer>
<includeArtifactIds>trino-parser</includeArtifactIds>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<artifactId>maven-assembly-plugin</artifactId>
<version>2.4.1</version>
<configuration>
<appendAssemblyId>false</appendAssemblyId>
<descriptors>
<descriptor>src/main/assembly/zip.xml</descriptor>
</descriptors>
</configuration>
<executions>
<execution>
<id>make-assembly</id>
<phase>package</phase>
<goals>
<goal>single</goal>
</goals>
</execution>
</executions>
</plugin>
<!-- jmockit -->
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<configuration>
<!-->set larger, eg, 3, to reduce the time or running plugin unit tests<-->
<forkCount>${plugin-ut.parallel}</forkCount>
<!-->not reuse forked jvm, so that each unit test will run in separate jvm. to avoid singleton conflict<-->
<reuseForks>false</reuseForks>
<useFile>false</useFile>
<argLine>
-javaagent:${settings.localRepository}/org/jmockit/jmockit/${jmockit.version}/jmockit-${jmockit.version}.jar
</argLine>
</configuration>
</plugin>
</plugins>
</build>
</project>

View File

@ -0,0 +1,18 @@
# 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.
### plugin configuration

View File

@ -0,0 +1,23 @@
# 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.
name=TrinoConverter
type=DIALECT
description=SQL dialect converter plugin for trino.
version=1.0.0
java.version=1.8.0
classname=org.apache.doris.plugin.dialect.trino.TrinoDialectConverterPlugin

View File

@ -0,0 +1,43 @@
<?xml version="1.0"?>
<!--
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.
-->
<assembly>
<id>plugin</id>
<formats>
<format>zip</format>
</formats>
<includeBaseDirectory>false</includeBaseDirectory>
<fileSets>
<fileSet>
<directory>target</directory>
<includes>
<include>*.jar</include>
</includes>
<outputDirectory>/</outputDirectory>
</fileSet>
<fileSet>
<directory>src/main/assembly</directory>
<includes>
<include>plugin.properties</include>
<include>plugin.conf</include>
</includes>
<outputDirectory>/</outputDirectory>
</fileSet>
</fileSets>
</assembly>

View File

@ -15,7 +15,7 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.parser.trino;
package org.apache.doris.plugin.dialect.trino;
import org.apache.doris.nereids.analyzer.UnboundFunction;
import org.apache.doris.nereids.parser.ComplexFnCallTransformer;

View File

@ -0,0 +1,45 @@
// 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.plugin.dialect.trino;
import org.apache.doris.analysis.StatementBase;
import org.apache.doris.nereids.parser.Dialect;
import org.apache.doris.plugin.DialectConverterPlugin;
import org.apache.doris.plugin.Plugin;
import org.apache.doris.qe.SessionVariable;
import com.google.common.collect.ImmutableSet;
import javax.annotation.Nullable;
import java.util.List;
/**
* Extends from {@link org.apache.doris.plugin.DialectConverterPlugin}, focus on Trino dialect.
*/
public class TrinoDialectConverterPlugin extends Plugin implements DialectConverterPlugin {
@Override
public ImmutableSet<Dialect> acceptDialects() {
return ImmutableSet.of(Dialect.TRINO);
}
@Override
public @Nullable List<StatementBase> parseSqlWithDialect(String sql, SessionVariable sessionVariable) {
return TrinoParser.parse(sql, sessionVariable);
}
}

View File

@ -15,9 +15,10 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.parser.trino;
package org.apache.doris.plugin.dialect.trino;
import org.apache.doris.nereids.analyzer.PlaceholderExpression;
import org.apache.doris.nereids.parser.AbstractFnCallTransformer;
import org.apache.doris.nereids.parser.AbstractFnCallTransformers;
import org.apache.doris.nereids.trees.expressions.Expression;

View File

@ -15,7 +15,8 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.parser.trino;
package org.apache.doris.plugin.dialect.trino;
import org.apache.doris.nereids.analyzer.UnboundAlias;
import org.apache.doris.nereids.analyzer.UnboundFunction;

View File

@ -15,18 +15,23 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.parser.trino;
package org.apache.doris.plugin.dialect.trino;
import org.apache.doris.analysis.StatementBase;
import org.apache.doris.nereids.StatementContext;
import org.apache.doris.nereids.exceptions.UnsupportedDialectException;
import org.apache.doris.nereids.glue.LogicalPlanAdapter;
import org.apache.doris.nereids.parser.ParseDialect;
import org.apache.doris.nereids.parser.Dialect;
import org.apache.doris.nereids.parser.ParserContext;
import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
import org.apache.doris.qe.SessionVariable;
import com.google.common.base.Preconditions;
import io.trino.sql.parser.ParsingException;
import io.trino.sql.parser.ParsingOptions;
import io.trino.sql.parser.SqlParser;
import io.trino.sql.parser.StatementSplitter;
import io.trino.sql.tree.Statement;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@ -42,8 +47,8 @@ public class TrinoParser {
public static final Logger LOG = LogManager.getLogger(TrinoParser.class);
private static final io.trino.sql.parser.ParsingOptions PARSING_OPTIONS = new io.trino.sql.parser.ParsingOptions(
io.trino.sql.parser.ParsingOptions.DecimalLiteralTreatment.AS_DECIMAL);
private static final ParsingOptions PARSING_OPTIONS = new ParsingOptions(
ParsingOptions.DecimalLiteralTreatment.AS_DECIMAL);
/**
* Parse with trino syntax, return null if parse failed
@ -51,11 +56,10 @@ public class TrinoParser {
public static @Nullable List<StatementBase> parse(String sql, SessionVariable sessionVariable) {
final List<StatementBase> logicalPlans = new ArrayList<>();
try {
io.trino.sql.parser.StatementSplitter splitter = new io.trino.sql.parser.StatementSplitter(
addDelimiterIfNeeded(sql));
ParserContext parserContext = new ParserContext(ParseDialect.TRINO_395);
StatementSplitter splitter = new StatementSplitter(addDelimiterIfNeeded(sql));
ParserContext parserContext = new ParserContext(Dialect.TRINO);
StatementContext statementContext = new StatementContext();
for (io.trino.sql.parser.StatementSplitter.Statement statement : splitter.getCompleteStatements()) {
for (StatementSplitter.Statement statement : splitter.getCompleteStatements()) {
Object parsedPlan = parseSingle(statement.statement(), parserContext);
logicalPlans.add(parsedPlan == null
? null : new LogicalPlanAdapter((LogicalPlan) parsedPlan, statementContext));
@ -64,14 +68,14 @@ public class TrinoParser {
return null;
}
return logicalPlans;
} catch (io.trino.sql.parser.ParsingException | UnsupportedDialectException e) {
} catch (ParsingException | UnsupportedDialectException e) {
LOG.debug("Failed to parse logical plan from trino, sql is :{}", sql, e);
return null;
}
}
private static io.trino.sql.tree.Statement parse(String sql) {
io.trino.sql.parser.SqlParser sqlParser = new io.trino.sql.parser.SqlParser();
private static Statement parse(String sql) {
SqlParser sqlParser = new io.trino.sql.parser.SqlParser();
return sqlParser.createStatement(sql, PARSING_OPTIONS);
}
@ -83,8 +87,8 @@ public class TrinoParser {
* @return logical plan
*/
public static <T> T parseSingle(String sql, ParserContext parserContext) {
Preconditions.checkArgument(parserContext.getParserDialect() == ParseDialect.TRINO_395);
io.trino.sql.tree.Statement statement = TrinoParser.parse(sql);
Preconditions.checkArgument(parserContext.getParserDialect() == Dialect.TRINO);
Statement statement = TrinoParser.parse(sql);
return (T) new TrinoLogicalPlanBuilder().visit(statement, parserContext);
}
@ -98,5 +102,4 @@ public class TrinoParser {
return sql + ";";
}
return sql;
}
}
}}

View File

@ -15,32 +15,59 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.parser.trino;
package org.apache.doris.plugin.dialect.trino;
import org.apache.doris.nereids.parser.Dialect;
import org.apache.doris.nereids.parser.NereidsParser;
import org.apache.doris.nereids.parser.ParserTestBase;
import org.apache.doris.nereids.parser.ParserContext;
import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
import org.apache.doris.qe.ConnectContext;
import org.apache.doris.qe.SessionVariable;
import mockit.Mock;
import mockit.MockUp;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
/**
* Trino to Doris function mapping test.
*/
public class FnTransformTest extends ParserTestBase {
public class FnTransformTest {
@BeforeAll
public static void init() {
ConnectContext ctx = new ConnectContext();
SessionVariable sessionVariable = new SessionVariable();
ctx.setSessionVariable(sessionVariable);
new MockUp<ConnectContext>() {
@Mock
public ConnectContext get() {
return ctx;
}
};
}
@Test
public void testStringFnTransform() {
String sql = "SELECT ascii('a') as b FROM t";
ParserContext parserContext = new ParserContext(Dialect.TRINO);
NereidsParser nereidsParser = new NereidsParser();
String sql = "SELECT ascii('a') as b FROM t";
LogicalPlan logicalPlan = nereidsParser.parseSingle(sql);
String dialectSql = "SELECT codepoint('a') as b FROM t";
trinoDialectParsePlan(dialectSql).assertEquals(logicalPlan);
LogicalPlan dialectLogicalPlan = TrinoParser.parseSingle(dialectSql, parserContext);
Assertions.assertEquals(dialectLogicalPlan, logicalPlan);
}
@Test
public void testDateDiffFnTransform() {
ParserContext parserContext = new ParserContext(Dialect.TRINO);
String dialectSql = "SELECT date_diff('second', TIMESTAMP '2020-12-25 22:00:00', TIMESTAMP '2020-12-25 21:00:00')";
trinoDialectParsePlan(dialectSql).assertContains("seconds_diff(2020-12-25 22:00:00, 2020-12-25 21:00:00)");
LogicalPlan logicalPlan = TrinoParser.parseSingle(dialectSql, parserContext);
Assertions.assertTrue(logicalPlan.toString().toLowerCase()
.contains("seconds_diff(2020-12-25 22:00:00, 2020-12-25 21:00:00)"));
}
}

View File

@ -15,28 +15,52 @@
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.parser.trino;
package org.apache.doris.plugin.dialect.trino;
import org.apache.doris.nereids.parser.Dialect;
import org.apache.doris.nereids.parser.NereidsParser;
import org.apache.doris.nereids.parser.ParserTestBase;
import org.apache.doris.nereids.parser.ParserContext;
import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
import mockit.Mock;
import mockit.MockUp;
import org.apache.doris.qe.ConnectContext;
import org.apache.doris.qe.SessionVariable;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
/**
* Trino query tests.
*/
public class QueryTest extends ParserTestBase {
public class ParserTest {
@BeforeAll
public static void init() {
ConnectContext ctx = new ConnectContext();
SessionVariable sessionVariable = new SessionVariable();
ctx.setSessionVariable(sessionVariable);
new MockUp<ConnectContext>() {
@Mock
public ConnectContext get() {
return ctx;
}
};
}
@Test
public void testParseCast1() {
ParserContext parserContext = new ParserContext(Dialect.TRINO);
String sql = "SELECT CAST(1 AS DECIMAL(20, 6)) FROM t";
NereidsParser nereidsParser = new NereidsParser();
LogicalPlan logicalPlan = nereidsParser.parseSingle(sql);
trinoDialectParsePlan(sql).assertEquals(logicalPlan);
LogicalPlan dialectLogicalPlan = TrinoParser.parseSingle(sql, parserContext);
Assertions.assertEquals(dialectLogicalPlan, logicalPlan);
sql = "SELECT CAST(a AS DECIMAL(20, 6)) FROM t";
logicalPlan = nereidsParser.parseSingle(sql);
trinoDialectParsePlan(sql).assertEquals(logicalPlan);
dialectLogicalPlan = TrinoParser.parseSingle(sql, parserContext);
Assertions.assertEquals(dialectLogicalPlan, logicalPlan);
}
}