Rework unresolved items messages
Update URLs in embedded HttpServers
Add antlr-runtime for embedded classpath

Original commit: elastic/x-pack-elasticsearch@36f0331d90
This commit is contained in:
Costin Leau 2017-09-28 22:27:20 +03:00 committed by GitHub
parent 33717f477f
commit aca8a5b6c0
24 changed files with 344 additions and 117 deletions

View File

@ -21,6 +21,8 @@ dependencies {
}
compile "net.sourceforge.csvjdbc:csvjdbc:1.0.31"
runtime "com.h2database:h2:1.4.194"
runtime 'org.antlr:antlr4-runtime:4.5.1-1'
// There are *no* CLI testing dependencies because we
// communicate fork a new CLI process when we need it.

View File

@ -53,7 +53,7 @@ public abstract class ProtoHttpServer {
}
public String url() {
return server != null ? "localhost:" + address().getPort() + protoSuffix : "<not started>";
return server != null ? "localhost:" + address().getPort() : "<not started>";
}
public Client client() {

View File

@ -34,7 +34,7 @@ import static org.hamcrest.Matchers.arrayWithSize;
* Tests comparing sql queries executed against our jdbc client
* with hard coded result sets.
*/
public abstract class CsvSpecTestCase extends SpecBaseIntegrationTestCase {
public class CsvSpecTestCase extends SpecBaseIntegrationTestCase {
private final CsvTestCase testCase;
@ParametersFactory(argumentFormatting = PARAM_FORMATTING)

View File

@ -25,7 +25,6 @@ import static java.util.Collections.emptyList;
import static java.util.Collections.singletonList;
import static java.util.Collections.singletonMap;
import static java.util.Collections.unmodifiableMap;
import static org.hamcrest.Matchers.both;
import static org.hamcrest.Matchers.containsString;
/**
@ -131,20 +130,7 @@ public abstract class RestSqlTestCase extends ESRestTestCase {
client().performRequest("POST", "/test/test/_bulk", singletonMap("refresh", "true"),
new StringEntity(bulk.toString(), ContentType.APPLICATION_JSON));
// NOCOMMIT "unresolved" should probably be changed to something users will understand like "missing"
expectBadRequest(() -> runSql("SELECT foo FROM test"), containsString("1:8: Unresolved item 'foo'"));
// NOCOMMIT the ones below one should include (foo) but it looks like the function is missing
expectBadRequest(() -> runSql("SELECT DAY_OF_YEAR(foo) FROM test"), containsString("1:20: Unresolved item 'DAY_OF_YEAR'"));
expectBadRequest(() -> runSql("SELECT foo, * FROM test GROUP BY DAY_OF_YEAR(foo)"),
both(containsString("1:8: Unresolved item 'foo'"))
.and(containsString("1:46: Unresolved item 'DAY_OF_YEAR'")));
// NOCOMMIT broken because we bail on the resolution phase if we can't resolve something in a previous phase
// expectBadRequest(() -> runSql("SELECT * FROM test WHERE foo = 1"), containsString("500"));
// expectBadRequest(() -> runSql("SELECT * FROM test WHERE DAY_OF_YEAR(foo) = 1"), containsString("500"));
// NOCOMMIT this should point to the column, no the (incorrectly capitalized) start or ORDER BY
expectBadRequest(() -> runSql("SELECT * FROM test ORDER BY foo"), containsString("line 1:29: Unresolved item 'Order'"));
expectBadRequest(() -> runSql("SELECT * FROM test ORDER BY DAY_OF_YEAR(foo)"),
containsString("line 1:41: Unresolved item 'Order'"));
expectBadRequest(() -> runSql("SELECT foo FROM test"), containsString("1:8: Unknown column [foo]"));
}
private void expectBadRequest(ThrowingRunnable code, Matcher<String> errorMessageMatcher) {

View File

@ -1,15 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.sql.analysis;
import org.elasticsearch.xpack.sql.tree.Node;
public class UnknownFunctionException extends AnalysisException {
public UnknownFunctionException(String function, Node<?> source) {
super(source, "Cannot resolve function %s", function);
}
}

View File

@ -1,15 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.sql.analysis;
import org.elasticsearch.xpack.sql.tree.Node;
public class UnknownIndexException extends AnalysisException {
public UnknownIndexException(String index, Node<?> source) {
super(source, "Cannot resolve index [%s]", index);
}
}

View File

@ -5,11 +5,10 @@
*/
package org.elasticsearch.xpack.sql.analysis.analyzer;
import org.elasticsearch.xpack.sql.SqlIllegalArgumentException;
import org.elasticsearch.xpack.sql.analysis.AnalysisException;
import org.elasticsearch.xpack.sql.analysis.UnknownFunctionException;
import org.elasticsearch.xpack.sql.analysis.UnknownIndexException;
import org.elasticsearch.xpack.sql.analysis.analyzer.Verifier.Failure;
import org.elasticsearch.xpack.sql.analysis.catalog.Catalog;
import org.elasticsearch.xpack.sql.analysis.catalog.Catalog.GetIndexResult;
import org.elasticsearch.xpack.sql.analysis.catalog.EsIndex;
import org.elasticsearch.xpack.sql.capabilities.Resolvables;
import org.elasticsearch.xpack.sql.expression.Alias;
@ -29,6 +28,7 @@ import org.elasticsearch.xpack.sql.expression.UnresolvedAlias;
import org.elasticsearch.xpack.sql.expression.UnresolvedAttribute;
import org.elasticsearch.xpack.sql.expression.UnresolvedStar;
import org.elasticsearch.xpack.sql.expression.function.Function;
import org.elasticsearch.xpack.sql.expression.function.FunctionDefinition;
import org.elasticsearch.xpack.sql.expression.function.FunctionRegistry;
import org.elasticsearch.xpack.sql.expression.function.Functions;
import org.elasticsearch.xpack.sql.expression.function.UnresolvedFunction;
@ -61,10 +61,12 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.LinkedHashMap;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.stream.Stream;
import static java.util.Collections.emptyList;
@ -75,12 +77,12 @@ import static org.elasticsearch.xpack.sql.util.CollectionUtils.combine;
public class Analyzer extends RuleExecutor<LogicalPlan> {
private final SqlSession session;
private final FunctionRegistry functionRegistry;
private final Catalog catalog;
public Analyzer(SqlSession session, FunctionRegistry functionRegistry) {
this.session = session;
public Analyzer(FunctionRegistry functionRegistry, Catalog catalog) {
this.functionRegistry = functionRegistry;
this.catalog = catalog;
}
@Override
@ -256,14 +258,15 @@ public class Analyzer extends RuleExecutor<LogicalPlan> {
@Override
protected LogicalPlan rule(UnresolvedRelation plan) {
TableIdentifier table = plan.table();
EsIndex found;
try {
found = session.getIndexSync(table.index());
} catch (SqlIllegalArgumentException e) {
throw new AnalysisException(plan, e.getMessage(), e);
EsIndex found = null;
GetIndexResult index = catalog.getIndex(table.index());
if (index.isValid()) {
found = index.get();
}
if (found == null) {
throw new UnknownIndexException(table.index(), plan);
return plan;
//throw new UnknownIndexException(table.index(), plan);
}
LogicalPlan catalogTable = new EsRelation(plan.location(), found);
@ -666,7 +669,9 @@ public class Analyzer extends RuleExecutor<LogicalPlan> {
}
// need to normalize the name for the lookup
List<Function> list = getList(seen, StringUtils.camelCaseToUnderscore(name));
String normalizedName = StringUtils.camelCaseToUnderscore(name);
List<Function> list = getList(seen, normalizedName);
// first try to resolve from seen functions
if (!list.isEmpty()) {
for (Function seenFunction : list) {
@ -678,10 +683,24 @@ public class Analyzer extends RuleExecutor<LogicalPlan> {
// not seen before, use the registry
if (!functionRegistry.functionExists(name)) {
throw new UnknownFunctionException(name, uf);
// try to find alternatives
Set<String> names = new LinkedHashSet<>();
for (FunctionDefinition def : functionRegistry.listFunctions()) {
names.add(def.name());
names.addAll(def.aliases());
}
List<String> matches = StringUtils.findSimilar(normalizedName, names);
if (!matches.isEmpty()) {
return new UnresolvedFunction(uf.location(), uf.name(), uf.distinct(), uf.children(), UnresolvedFunction.errorMessage(normalizedName, matches));
}
else {
return uf;
}
}
// TODO: look into Generator for significant terms, etc..
Function f = functionRegistry.resolveFunction(uf, SqlSession.CURRENT.get());
Function f = functionRegistry.resolveFunction(uf, SqlSession.CURRENT_SETTINGS.get());
list.add(f);
return f;

View File

@ -5,16 +5,18 @@
*/
package org.elasticsearch.xpack.sql.analysis.analyzer;
import org.elasticsearch.xpack.sql.capabilities.Unresolvable;
import org.elasticsearch.xpack.sql.expression.Attribute;
import org.elasticsearch.xpack.sql.expression.AttributeSet;
import org.elasticsearch.xpack.sql.expression.Expressions;
import org.elasticsearch.xpack.sql.expression.NamedExpression;
import org.elasticsearch.xpack.sql.expression.UnresolvedAttribute;
import org.elasticsearch.xpack.sql.expression.function.Functions;
import org.elasticsearch.xpack.sql.expression.function.aggregate.AggregateFunction;
import org.elasticsearch.xpack.sql.plan.logical.Aggregate;
import org.elasticsearch.xpack.sql.plan.logical.Filter;
import org.elasticsearch.xpack.sql.plan.logical.LogicalPlan;
import org.elasticsearch.xpack.sql.tree.Node;
import org.elasticsearch.xpack.sql.util.StringUtils;
import java.util.ArrayList;
import java.util.Collection;
@ -78,34 +80,66 @@ abstract class Verifier {
static Collection<Failure> verify(LogicalPlan plan) {
Set<Failure> failures = new LinkedHashSet<>();
// start bottom-up
plan.forEachUp(p -> {
if (p.analyzed()) {
return;
}
// if the children are unresolved, this node will also so counting it will only add noise
if (!p.childrenResolved()) {
return;
}
Set<Failure> localFailures = new LinkedHashSet<>();
//
// Handle unresolved items first
// First handle usual suspects
//
// first look at expressions
p.forEachExpressions(e -> e.forEachUp(ae -> {
if (ae.typeResolved().unresolved()) {
localFailures.add(fail(ae, ae.typeResolved().message()));
}
else if (ae.childrenResolved() && !ae.resolved()) {
localFailures.add(fail(ae, "Unresolved item '%s'", Expressions.name(e)));
}
else if (ae instanceof Attribute && !ae.resolved()) {
localFailures.add(fail(e, "Cannot resolved '%s' from columns %s", Expressions.name(ae), p.intputSet()));
}
}));
// consider only nodes that are by themselves unresolved (to avoid unresolved dependencies)
if (p.childrenResolved() && p.expressionsResolved() && !p.resolved()) {
localFailures.add(fail(p, "Unresolved item '%s'", p.nodeString()));
if (p instanceof Unresolvable) {
localFailures.add(fail(p, ((Unresolvable) p).unresolvedMessage()));
}
else {
// then take a look at the expressions
p.forEachExpressions(e -> {
// everything is fine, skip expression
if (e.resolved()) {
return;
}
e.forEachUp(ae -> {
// we're only interested in the children
if (!ae.childrenResolved()) {
return;
}
// again the usual suspects
if (ae instanceof Unresolvable) {
// handle Attributes different to provide more context
if (ae instanceof UnresolvedAttribute) {
UnresolvedAttribute ua = (UnresolvedAttribute) ae;
boolean useQualifier = ua.qualifier() != null;
List<String> potentialMatches = new ArrayList<>();
for (Attribute a : p.intputSet()) {
potentialMatches.add(useQualifier ? a.qualifiedName() : a.name());
}
List<String> matches = StringUtils.findSimilar(ua.qualifiedName(), potentialMatches);
if (!matches.isEmpty()) {
ae = new UnresolvedAttribute(ua.location(), ua.name(), ua.qualifier(), UnresolvedAttribute.errorMessage(ua.qualifiedName(), matches));
}
}
localFailures.add(fail(ae, ((Unresolvable) ae).unresolvedMessage()));
return;
}
// type resolution
if (ae.typeResolved().unresolved()) {
localFailures.add(fail(ae, ae.typeResolved().message()));
}
});
});
}
//

View File

@ -33,7 +33,7 @@ public interface Catalog {
}
public static GetIndexResult notFound(String name) {
Objects.requireNonNull(name, "name must not be null");
return invalid("index [" + name + "] does not exist");
return invalid("Index '" + name + "' does not exist");
}
@Nullable

View File

@ -14,4 +14,6 @@ public interface Unresolvable extends Resolvable {
default boolean resolved() {
return false;
}
String unresolvedMessage();
}

View File

@ -45,13 +45,14 @@ public class PlanExecutor {
this.parser = new SqlParser();
this.functionRegistry = new DefaultFunctionRegistry();
this.optimizer = new Optimizer();
this.planner = new Planner();
}
public SqlSession newSession(SqlSettings settings) {
return new SqlSession(settings, client, catalogSupplier.apply(stateSupplier.get()), parser,
functionRegistry, optimizer, planner);
Catalog catalog = catalogSupplier.apply(stateSupplier.get());
return new SqlSession(settings, client, catalog, functionRegistry, parser, optimizer, planner);
}

View File

@ -5,11 +5,11 @@
*/
package org.elasticsearch.xpack.sql.expression;
import java.util.Objects;
import org.elasticsearch.xpack.sql.tree.Location;
import org.elasticsearch.xpack.sql.type.DataType;
import java.util.Objects;
import static java.util.Collections.emptyList;
public abstract class Attribute extends NamedExpression {
@ -40,7 +40,7 @@ public abstract class Attribute extends NamedExpression {
}
public String qualifiedName() {
return qualifier == null ? name() : qualifier.concat(".").concat(name());
return qualifier == null ? name() : qualifier + "." + name();
}
public boolean nullable() {

View File

@ -5,10 +5,10 @@
*/
package org.elasticsearch.xpack.sql.expression;
import java.util.Objects;
import org.elasticsearch.xpack.sql.capabilities.UnresolvedException;
import java.util.Objects;
import static java.util.Collections.singletonList;
public class UnresolvedAlias extends UnresolvedNamedExpression {
@ -24,6 +24,11 @@ public class UnresolvedAlias extends UnresolvedNamedExpression {
return child;
}
@Override
public String unresolvedMessage() {
return "Unknown alias [" + name() + "]";
}
@Override
public boolean nullable() {
throw new UnresolvedException("nullable", this);

View File

@ -5,28 +5,35 @@
*/
package org.elasticsearch.xpack.sql.expression;
import java.util.Arrays;
import java.util.List;
import java.util.Locale;
import org.elasticsearch.xpack.sql.capabilities.Unresolvable;
import org.elasticsearch.xpack.sql.capabilities.UnresolvedException;
import org.elasticsearch.xpack.sql.tree.Location;
import org.elasticsearch.xpack.sql.type.DataType;
import org.elasticsearch.xpack.sql.util.CollectionUtils;
import java.util.Arrays;
import java.util.List;
import java.util.Locale;
import static java.lang.String.format;
public class UnresolvedAttribute extends Attribute implements Unresolvable {
private List<String> nameParts;
private final List<String> nameParts;
private final String unresolvedMsg;
public UnresolvedAttribute(Location location, String name) {
this(location, name, null);
}
public UnresolvedAttribute(Location location, String name, String qualifier) {
this(location, name, qualifier, null);
}
public UnresolvedAttribute(Location location, String name, String qualifier, String unresolvedMessage) {
super(location, name, qualifier, null);
nameParts = Arrays.asList(name.split("\\."));
this.unresolvedMsg = unresolvedMessage == null ? errorMessage(qualifiedName(), null) : unresolvedMessage;
}
public List<String> nameParts() {
@ -48,7 +55,6 @@ public class UnresolvedAttribute extends Attribute implements Unresolvable {
throw new UnresolvedException("dataType", this);
}
@Override
public String nodeString() {
return format(Locale.ROOT, "unknown column '%s'", name());
@ -56,11 +62,24 @@ public class UnresolvedAttribute extends Attribute implements Unresolvable {
@Override
public String toString() {
return UNRESOLVED_PREFIX + (qualifier() != null ? qualifier() + "." : "") + name();
return UNRESOLVED_PREFIX + qualifiedName();
}
@Override
protected String label() {
return UNRESOLVED_PREFIX;
}
}
@Override
public String unresolvedMessage() {
return unresolvedMsg;
}
public static String errorMessage(String name, List<String> potentialMatches) {
String msg = "Unknown column [" + name + "]";
if (!CollectionUtils.isEmpty(potentialMatches)) {
msg += ", did you mean " + (potentialMatches.size() == 1 ? "[" + potentialMatches.get(0) + "]": "any of " + potentialMatches.toString()) + "?";
}
return msg;
}
}

View File

@ -5,11 +5,11 @@
*/
package org.elasticsearch.xpack.sql.expression;
import java.util.Objects;
import org.elasticsearch.xpack.sql.capabilities.UnresolvedException;
import org.elasticsearch.xpack.sql.tree.Location;
import java.util.Objects;
import static java.util.Collections.emptyList;
public class UnresolvedStar extends UnresolvedNamedExpression {
@ -45,8 +45,17 @@ public class UnresolvedStar extends UnresolvedNamedExpression {
return Objects.equals(qualifier, other.qualifier);
}
private String message() {
return (qualifier() != null ? qualifier() + "." : "") + "*";
}
@Override
public String unresolvedMessage() {
return "Cannot determine columns for " + message();
}
@Override
public String toString() {
return UNRESOLVED_PREFIX + (qualifier() != null ? qualifier() + "." : "") + "*";
return UNRESOLVED_PREFIX + message();
}
}

View File

@ -38,7 +38,8 @@ abstract class AbstractFunctionRegistry implements FunctionRegistry {
defs.put(def.name(), def);
for (String alias : def.aliases()) {
Assert.isTrue(defs.containsKey(alias) == false, "Alias %s already exists", alias);
defs.put(alias, def);
// alias should be already normalized but to be double sure
defs.put(normalize(alias), def);
}
}
}

View File

@ -29,7 +29,7 @@ public class FunctionDefinition {
return name;
}
List<String> aliases() {
public List<String> aliases() {
return aliases;
}

View File

@ -11,6 +11,7 @@ import org.elasticsearch.xpack.sql.expression.Attribute;
import org.elasticsearch.xpack.sql.expression.Expression;
import org.elasticsearch.xpack.sql.tree.Location;
import org.elasticsearch.xpack.sql.type.DataType;
import org.elasticsearch.xpack.sql.util.CollectionUtils;
import java.util.List;
@ -18,11 +19,17 @@ public class UnresolvedFunction extends Function implements Unresolvable {
private final String name;
private final boolean distinct;
private final String unresolvedMsg;
public UnresolvedFunction(Location location, String name, boolean distinct, List<Expression> children) {
this(location, name, distinct, children, null);
}
public UnresolvedFunction(Location location, String name, boolean distinct, List<Expression> children, String unresolvedMessage) {
super(location, children);
this.name = name;
this.distinct = distinct;
this.unresolvedMsg = unresolvedMessage == null ? errorMessage(name, null) : unresolvedMessage;
}
@Override
@ -59,8 +66,21 @@ public class UnresolvedFunction extends Function implements Unresolvable {
throw new UnresolvedException("attribute", this);
}
@Override
public String unresolvedMessage() {
return unresolvedMsg;
}
@Override
public String toString() {
return UNRESOLVED_PREFIX + functionName() + functionArgs();
}
public static String errorMessage(String name, List<String> potentialMatches) {
String msg = "Unknown function [" + name + "]";
if (!CollectionUtils.isEmpty(potentialMatches)) {
msg += ", did you mean " + (potentialMatches.size() == 1 ? "[" + potentialMatches.get(0) + "]": "any of " + potentialMatches.toString()) + "?";
}
return msg;
}
}

View File

@ -110,7 +110,7 @@ public enum Extract {
};
public DateTimeFunction toFunction(Location source, Expression argument) {
return toFunction(source, argument, SqlSession.CURRENT.get().timeZone());
return toFunction(source, argument, SqlSession.CURRENT_SETTINGS.get().timeZone());
}
public abstract DateTimeFunction toFunction(Location source, Expression argument, DateTimeZone timeZone);

View File

@ -5,15 +5,15 @@
*/
package org.elasticsearch.xpack.sql.plan.logical;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
import org.elasticsearch.xpack.sql.capabilities.Unresolvable;
import org.elasticsearch.xpack.sql.expression.Attribute;
import org.elasticsearch.xpack.sql.plan.TableIdentifier;
import org.elasticsearch.xpack.sql.tree.Location;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
public class UnresolvedRelation extends LeafPlan implements Unresolvable {
private final TableIdentifier table;
@ -38,6 +38,11 @@ public class UnresolvedRelation extends LeafPlan implements Unresolvable {
return false;
}
@Override
public String unresolvedMessage() {
return "Unknown index [" + table.index() + "]";
}
@Override
public boolean expressionsResolved() {
return false;

View File

@ -8,6 +8,7 @@ package org.elasticsearch.xpack.sql.session;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.xpack.sql.analysis.analyzer.Analyzer;
import org.elasticsearch.xpack.sql.analysis.catalog.Catalog;
import org.elasticsearch.xpack.sql.analysis.catalog.EsIndex;
@ -21,6 +22,7 @@ import org.elasticsearch.xpack.sql.planner.Planner;
import org.elasticsearch.xpack.sql.plugin.SqlGetIndicesAction;
import java.util.List;
import java.util.function.Function;
public class SqlSession {
@ -31,13 +33,14 @@ public class SqlSession {
private final FunctionRegistry functionRegistry;
private final Optimizer optimizer;
private final Planner planner;
private final Analyzer analyzer;
private final SqlSettings defaults; // NOCOMMIT this doesn't look used
private final SqlSettings defaults; // NOCOMMIT this doesn't look used - it is for RESET SESSION
private SqlSettings settings;
// thread-local used for sharing settings across the plan compilation
// TODO investigate removing
public static final ThreadLocal<SqlSettings> CURRENT = new ThreadLocal<SqlSettings>() {
public static final ThreadLocal<SqlSettings> CURRENT_SETTINGS = new ThreadLocal<SqlSettings>() {
@Override
public String toString() {
return "SQL Session";
@ -45,18 +48,22 @@ public class SqlSession {
};
public SqlSession(SqlSession other) {
this(other.defaults(), other.client(), other.catalog(), other.parser,
other.functionRegistry(), other.optimizer(), other.planner());
this(other.defaults(), other.client(), other.catalog(), other.functionRegistry(),
other.parser, other.optimizer(), other.planner());
}
public SqlSession(SqlSettings defaults, Client client,
Catalog catalog, SqlParser parser, FunctionRegistry functionRegistry, Optimizer optimizer,
Catalog catalog, FunctionRegistry functionRegistry,
SqlParser parser,
Optimizer optimizer,
Planner planner) {
this.client = client;
this.catalog = catalog;
this.parser = parser;
this.functionRegistry = functionRegistry;
//TODO: analyzer should really be a singleton
this.analyzer = new Analyzer(functionRegistry, catalog);
this.optimizer = optimizer;
this.planner = planner;
@ -98,7 +105,7 @@ public class SqlSession {
}
public Analyzer analyzer() {
return new Analyzer(this, functionRegistry);
return analyzer;
}
public Optimizer optimizer() {
@ -127,11 +134,11 @@ public class SqlSession {
}
public PhysicalPlan executable(String sql) {
CURRENT.set(settings);
CURRENT_SETTINGS.set(settings);
try {
return physicalPlan(parse(sql), true);
} finally {
CURRENT.remove();
CURRENT_SETTINGS.remove();
}
}
@ -147,6 +154,12 @@ public class SqlSession {
return settings;
}
// session SET
public SqlSettings updateSettings(Function<Settings, Settings> transformer) {
settings = new SqlSettings(transformer.apply(settings.cfg()));
return settings;
}
public void execute(PhysicalPlan plan, ActionListener<RowSet> listener) {
plan.execute(this, listener);
}

View File

@ -5,13 +5,17 @@
*/
package org.elasticsearch.xpack.sql.util;
import org.apache.lucene.search.spell.LevensteinDistance;
import org.apache.lucene.util.CollectionUtil;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
@ -19,6 +23,7 @@ import java.util.Locale;
import java.util.regex.Pattern;
import static java.util.stream.Collectors.joining;
import static java.util.stream.Collectors.toList;
public abstract class StringUtils {
@ -172,4 +177,19 @@ public abstract class StringUtils {
throw new RuntimeException("error rendering", e);
}
}
public static List<String> findSimilar(String match, Iterable<String> potentialMatches) {
LevensteinDistance ld = new LevensteinDistance();
List<Tuple<Float, String>> scoredMatches = new ArrayList<>();
for (String potentialMatch : potentialMatches) {
float distance = ld.getDistance(match, potentialMatch);
if (distance >= 0.5f) {
scoredMatches.add(new Tuple<>(distance, potentialMatch));
}
}
CollectionUtil.timSort(scoredMatches, (a,b) -> b.v1().compareTo(a.v1()));
return scoredMatches.stream()
.map(a -> a.v2())
.collect(toList());
}
}

View File

@ -0,0 +1,121 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.sql.analysis.analyzer;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.xpack.sql.analysis.AnalysisException;
import org.elasticsearch.xpack.sql.analysis.catalog.Catalog;
import org.elasticsearch.xpack.sql.analysis.catalog.EsIndex;
import org.elasticsearch.xpack.sql.analysis.catalog.InMemoryCatalog;
import org.elasticsearch.xpack.sql.expression.function.DefaultFunctionRegistry;
import org.elasticsearch.xpack.sql.expression.function.FunctionRegistry;
import org.elasticsearch.xpack.sql.parser.SqlParser;
import org.elasticsearch.xpack.sql.type.DataType;
import org.elasticsearch.xpack.sql.type.DataTypes;
import org.junit.Test;
import java.util.LinkedHashMap;
import java.util.Map;
import static java.util.Collections.emptyList;
import static java.util.Collections.singletonList;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
public class VerifierErrorMessagesTest {
private SqlParser parser;
private FunctionRegistry functionRegistry;
private Catalog catalog;
private Analyzer analyzer;
public VerifierErrorMessagesTest() {
parser = new SqlParser();
functionRegistry = new DefaultFunctionRegistry();
Map<String, DataType> mapping = new LinkedHashMap<>();
mapping.put("bool", DataTypes.BOOLEAN);
mapping.put("int", DataTypes.INTEGER);
mapping.put("text", DataTypes.TEXT);
mapping.put("keyword", DataTypes.KEYWORD);
EsIndex test = new EsIndex("test", mapping, emptyList(), Settings.EMPTY);
catalog = new InMemoryCatalog(singletonList(test));
analyzer = new Analyzer(functionRegistry, catalog);
}
private String verify(String sql) {
try {
analyzer.analyze(parser.createStatement(sql), true);
fail("query is valid; expected an invalid one");
return "";
} catch (AnalysisException ae) {
String message = ae.getMessage();
assertTrue(message.startsWith("Found "));
// test uses 1 or multiple
String header = "Found 1 problem(s)\nline ";
return message.substring(header.length());
}
}
@Test
public void testMissingIndex() {
assertEquals("1:17: Unknown index [missing]", verify("SELECT foo FROM missing"));
}
@Test
public void testMissingColumn() {
assertEquals("1:8: Unknown column [xxx]", verify("SELECT xxx FROM test"));
}
@Test
public void testMispelledColumn() {
assertEquals("1:8: Unknown column [txt], did you mean [text]?", verify("SELECT txt FROM test"));
}
@Test
public void testFunctionOverMissingField() {
assertEquals("1:12: Unknown column [xxx]", verify("SELECT ABS(xxx) FROM test"));
}
@Test
public void testMissingFunction() {
assertEquals("1:8: Unknown function [ZAZ]", verify("SELECT ZAZ(bool) FROM test"));
}
@Test
public void testMispelledFunction() {
assertEquals("1:8: Unknown function [COONT], did you mean [COUNT]?", verify("SELECT COONT(bool) FROM test"));
}
@Test
public void testMissingColumnInGroupBy() {
assertEquals("1:41: Unknown column [xxx]", verify("SELECT * FROM test GROUP BY DAY_OF_YEAR(xxx)"));
}
@Test
public void testFilterOnUnknownColumn() {
assertEquals("1:26: Unknown column [xxx]", verify("SELECT * FROM test WHERE xxx = 1"));
}
@Test
public void testMissingColumnInOrderby() {
// xxx offset is that of the order by field
assertEquals("1:29: Unknown column [xxx]", verify("SELECT * FROM test ORDER BY xxx"));
}
@Test
public void testMissingColumnFunctionInOrderby() {
// xxx offset is that of the order by field
assertEquals("1:41: Unknown column [xxx]", verify("SELECT * FROM test ORDER BY DAY_oF_YEAR(xxx)"));
}
@Test
public void testMultipleColumns() {
// xxx offset is that of the order by field
assertEquals("1:43: Unknown column [xxx]\nline 1:8: Unknown column [xxx]", verify("SELECT xxx FROM test GROUP BY DAY_oF_YEAR(xxx)"));
}
}

View File

@ -14,10 +14,10 @@ import static java.util.stream.Collectors.toMap;
/**
* In memory implementation of catalog designed for testing.
*/
class InMemoryCatalog implements Catalog {
public class InMemoryCatalog implements Catalog {
private final Map<String, EsIndex> indices;
InMemoryCatalog(List<EsIndex> indices) {
public InMemoryCatalog(List<EsIndex> indices) {
this.indices = indices.stream().collect(toMap(EsIndex::name, Function.identity()));
}