[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:
@ -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 = "";
|
||||
|
||||
@ -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>
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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 {
|
||||
|
||||
|
||||
@ -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) {
|
||||
|
||||
@ -18,7 +18,7 @@
|
||||
package org.apache.doris.nereids.parser;
|
||||
|
||||
/**
|
||||
* Trino complex function transformer
|
||||
* Complex complex function transformer
|
||||
*/
|
||||
public abstract class ComplexFnCallTransformer extends AbstractFnCallTransformer {
|
||||
|
||||
|
||||
@ -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;
|
||||
}
|
||||
}
|
||||
@ -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 {
|
||||
|
||||
|
||||
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
@ -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);
|
||||
}
|
||||
@ -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;
|
||||
}
|
||||
|
||||
@ -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() {
|
||||
|
||||
@ -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();
|
||||
|
||||
@ -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) {
|
||||
|
||||
@ -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) {
|
||||
|
||||
@ -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();
|
||||
|
||||
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
@ -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;
|
||||
}
|
||||
}
|
||||
@ -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());
|
||||
}
|
||||
}
|
||||
@ -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);
|
||||
}
|
||||
}
|
||||
@ -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;
|
||||
}
|
||||
}
|
||||
@ -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>
|
||||
|
||||
119
fe_plugins/http-dialect-converter/pom.xml
Normal file
119
fe_plugins/http-dialect-converter/pom.xml
Normal 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>
|
||||
22
fe_plugins/http-dialect-converter/src/main/assembly/plugin.conf
Executable file
22
fe_plugins/http-dialect-converter/src/main/assembly/plugin.conf
Executable 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
|
||||
23
fe_plugins/http-dialect-converter/src/main/assembly/plugin.properties
Executable file
23
fe_plugins/http-dialect-converter/src/main/assembly/plugin.properties
Executable 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
|
||||
43
fe_plugins/http-dialect-converter/src/main/assembly/zip.xml
Normal file
43
fe_plugins/http-dialect-converter/src/main/assembly/zip.xml
Normal 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>
|
||||
@ -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;
|
||||
}
|
||||
}
|
||||
@ -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;
|
||||
@ -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;
|
||||
}
|
||||
}
|
||||
@ -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;
|
||||
@ -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>
|
||||
|
||||
132
fe_plugins/sparksql-converter/pom.xml
Normal file
132
fe_plugins/sparksql-converter/pom.xml
Normal 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>
|
||||
18
fe_plugins/sparksql-converter/src/main/assembly/plugin.conf
Executable file
18
fe_plugins/sparksql-converter/src/main/assembly/plugin.conf
Executable 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
|
||||
23
fe_plugins/sparksql-converter/src/main/assembly/plugin.properties
Executable file
23
fe_plugins/sparksql-converter/src/main/assembly/plugin.properties
Executable 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
|
||||
43
fe_plugins/sparksql-converter/src/main/assembly/zip.xml
Normal file
43
fe_plugins/sparksql-converter/src/main/assembly/zip.xml
Normal 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>
|
||||
@ -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)));
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -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 {
|
||||
|
||||
@ -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
|
||||
@ -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());
|
||||
}
|
||||
}
|
||||
@ -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",
|
||||
@ -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);
|
||||
}
|
||||
}
|
||||
159
fe_plugins/trino-converter/pom.xml
Normal file
159
fe_plugins/trino-converter/pom.xml
Normal 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>
|
||||
18
fe_plugins/trino-converter/src/main/assembly/plugin.conf
Executable file
18
fe_plugins/trino-converter/src/main/assembly/plugin.conf
Executable 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
|
||||
23
fe_plugins/trino-converter/src/main/assembly/plugin.properties
Executable file
23
fe_plugins/trino-converter/src/main/assembly/plugin.properties
Executable 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
|
||||
43
fe_plugins/trino-converter/src/main/assembly/zip.xml
Normal file
43
fe_plugins/trino-converter/src/main/assembly/zip.xml
Normal 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>
|
||||
@ -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;
|
||||
@ -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);
|
||||
}
|
||||
}
|
||||
@ -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;
|
||||
|
||||
@ -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;
|
||||
@ -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;
|
||||
}
|
||||
}
|
||||
}}
|
||||
@ -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)"));
|
||||
}
|
||||
}
|
||||
@ -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);
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user