Trim down the Catalog implementations to a single one
Catalog is now a final class rather than an interface with different implementations. relates elastic/x-pack-elasticsearch#3179 Original commit: elastic/x-pack-elasticsearch@4cc927e113
This commit is contained in:
parent
8c61cabe87
commit
90aee54251
|
@ -128,12 +128,12 @@ public abstract class RestSqlTestCase extends ESRestTestCase implements ErrorsTe
|
|||
}
|
||||
|
||||
@Override
|
||||
public void testSelectInvalidSql() throws Exception {
|
||||
public void testSelectInvalidSql() {
|
||||
expectBadRequest(() -> runSql("SELECT * FRO"), containsString("1:8: Cannot determine columns for *"));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void testSelectFromMissingIndex() throws IOException {
|
||||
public void testSelectFromMissingIndex() {
|
||||
expectBadRequest(() -> runSql("SELECT * FROM missing"), containsString("1:15: Unknown index [missing]"));
|
||||
}
|
||||
|
||||
|
@ -161,7 +161,7 @@ public abstract class RestSqlTestCase extends ESRestTestCase implements ErrorsTe
|
|||
|
||||
private void expectBadRequest(ThrowingRunnable code, Matcher<String> errorMessageMatcher) {
|
||||
ResponseException e = expectThrows(ResponseException.class, code);
|
||||
assertEquals(400, e.getResponse().getStatusLine().getStatusCode());
|
||||
assertEquals(e.getMessage(), 400, e.getResponse().getStatusLine().getStatusCode());
|
||||
assertThat(e.getMessage(), errorMessageMatcher);
|
||||
}
|
||||
|
||||
|
|
|
@ -253,17 +253,13 @@ public class Analyzer extends RuleExecutor<LogicalPlan> {
|
|||
@Override
|
||||
protected LogicalPlan rule(UnresolvedRelation plan) {
|
||||
TableIdentifier table = plan.table();
|
||||
EsIndex found = null;
|
||||
|
||||
GetIndexResult index = SqlSession.currentContext().catalog.getIndex(table.index());
|
||||
if (index.isValid()) {
|
||||
found = index.get();
|
||||
} else {
|
||||
if (index.isValid() == false) {
|
||||
return plan.unresolvedMessage().equals(index.toString()) ? plan : new UnresolvedRelation(plan.location(), plan.table(),
|
||||
plan.alias(), index.toString());
|
||||
}
|
||||
|
||||
LogicalPlan catalogTable = new EsRelation(plan.location(), found);
|
||||
LogicalPlan catalogTable = new EsRelation(plan.location(), index.get());
|
||||
SubQueryAlias sa = new SubQueryAlias(plan.location(), catalogTable, table.index());
|
||||
|
||||
if (plan.alias() != null) {
|
||||
|
|
|
@ -5,26 +5,39 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.sql.analysis.catalog;
|
||||
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
|
||||
import java.util.Objects;
|
||||
import java.util.function.Function;
|
||||
|
||||
/**
|
||||
* Converts from Elasticsearch's internal metadata ({@link ClusterState})
|
||||
* into a representation that is compatible with SQL (@{link {@link EsIndex}).
|
||||
* Index representation that is compatible with SQL ({@link EsIndex}).
|
||||
*/
|
||||
public interface Catalog {
|
||||
public final class Catalog {
|
||||
|
||||
Catalog EMPTY = GetIndexResult::notFound;
|
||||
public static final Catalog EMPTY = new Catalog(GetIndexResult::notFound);
|
||||
|
||||
private final Function<String, GetIndexResult> resultFunction;
|
||||
|
||||
//TODO given that this always holds a single index, we cana probably get rid of the whole idea of Catalog
|
||||
public Catalog(GetIndexResult result) {
|
||||
assert result != null;
|
||||
this.resultFunction = index -> result.matches(index) ? result : GetIndexResult.notFound(index);
|
||||
}
|
||||
|
||||
private Catalog(Function<String, GetIndexResult> resultFunction) {
|
||||
assert resultFunction != null;
|
||||
this.resultFunction = resultFunction;
|
||||
}
|
||||
|
||||
/**
|
||||
* Lookup the information for a table.
|
||||
*/
|
||||
@Nullable
|
||||
GetIndexResult getIndex(String index);
|
||||
public GetIndexResult getIndex(String index) {
|
||||
return resultFunction.apply(index);
|
||||
}
|
||||
|
||||
class GetIndexResult {
|
||||
public static final class GetIndexResult {
|
||||
public static GetIndexResult valid(EsIndex index) {
|
||||
Objects.requireNonNull(index, "index must not be null if it was found");
|
||||
return new GetIndexResult(index, null);
|
||||
|
@ -47,6 +60,10 @@ public interface Catalog {
|
|||
this.invalid = invalid;
|
||||
}
|
||||
|
||||
private boolean matches(String index) {
|
||||
return isValid() && this.index.name().equals(index);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the {@linkplain EsIndex} built by the {@linkplain Catalog}.
|
||||
* @throws MappingException if the index is invalid for
|
||||
|
|
|
@ -20,6 +20,8 @@ public class EsIndex {
|
|||
private final Map<String, DataType> mapping;
|
||||
|
||||
public EsIndex(String name, Map<String, DataType> mapping) {
|
||||
assert name != null;
|
||||
assert mapping != null;
|
||||
this.name = name;
|
||||
this.mapping = mapping;
|
||||
}
|
||||
|
|
|
@ -38,10 +38,10 @@ public class IndexResolver {
|
|||
public void asCatalog(final String index, ActionListener<Catalog> listener) {
|
||||
GetIndexRequest getIndexRequest = createGetIndexRequest(index);
|
||||
client.admin().indices().getIndex(getIndexRequest, ActionListener.wrap(getIndexResponse -> {
|
||||
Map<String, GetIndexResult> results = new HashMap<>();
|
||||
GetIndexResult result;
|
||||
if (getIndexResponse.getMappings().size() > 1) {
|
||||
results.put(index, GetIndexResult.invalid(
|
||||
"[" + index + "] is an alias pointing to more than one index which is currently incompatible with sql"));
|
||||
result = GetIndexResult.invalid(
|
||||
"[" + index + "] is an alias pointing to more than one index which is currently incompatible with sql");
|
||||
} else if (getIndexResponse.getMappings().size() == 1){
|
||||
ObjectObjectCursor<String, ImmutableOpenMap<String, MappingMetaData>> indexMappings =
|
||||
getIndexResponse.getMappings().iterator().next();
|
||||
|
@ -53,9 +53,11 @@ public class IndexResolver {
|
|||
* make sure that the search is executed against the same alias name from the original command, rather than
|
||||
* the resolved concrete index that we get back from the get index API
|
||||
*/
|
||||
results.put(index, buildGetIndexResult(concreteIndex, index, indexMappings.value));
|
||||
result = buildGetIndexResult(concreteIndex, index, indexMappings.value);
|
||||
} else {
|
||||
result = GetIndexResult.notFound(index);
|
||||
}
|
||||
listener.onResponse(new PreloadedCatalog(results));
|
||||
listener.onResponse(new Catalog(result));
|
||||
}, listener::onFailure));
|
||||
}
|
||||
|
||||
|
|
|
@ -1,23 +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.catalog;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
|
||||
public class PreloadedCatalog implements Catalog {
|
||||
|
||||
private final Map<String, GetIndexResult> map;
|
||||
|
||||
public PreloadedCatalog(Map<String, GetIndexResult> map) {
|
||||
this.map = map;
|
||||
}
|
||||
|
||||
@Override
|
||||
public GetIndexResult getIndex(String index) {
|
||||
return map.getOrDefault(index, GetIndexResult.notFound(index));
|
||||
}
|
||||
}
|
|
@ -176,6 +176,7 @@ public class SqlSession {
|
|||
listener.onFailure(new SqlIllegalArgumentException("Queries with multiple indices are not supported"));
|
||||
return;
|
||||
}
|
||||
//TODO why do we have a list if we only support one single element? Seems like it's the wrong data structure?
|
||||
if (preAnalysis.indices.size() == 1) {
|
||||
indexResolver.asCatalog(preAnalysis.indices.get(0),
|
||||
wrap(c -> listener.onResponse(action.apply(c)), listener::onFailure));
|
||||
|
|
|
@ -10,7 +10,6 @@ import org.elasticsearch.test.junit.annotations.TestLogging;
|
|||
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;
|
||||
|
@ -23,8 +22,6 @@ import org.junit.Before;
|
|||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import static java.util.Collections.singletonList;
|
||||
|
||||
@TestLogging("org.elasticsearch.xpack.sql:TRACE")
|
||||
public class VerifierErrorMessagesTests extends ESTestCase {
|
||||
|
||||
|
@ -43,7 +40,7 @@ public class VerifierErrorMessagesTests extends ESTestCase {
|
|||
mapping.put("text", DataTypes.TEXT);
|
||||
mapping.put("keyword", DataTypes.KEYWORD);
|
||||
EsIndex test = new EsIndex("test", mapping);
|
||||
catalog = new InMemoryCatalog(singletonList(test));
|
||||
catalog = new Catalog(Catalog.GetIndexResult.valid(test));
|
||||
analyzer = new Analyzer(functionRegistry);
|
||||
}
|
||||
|
||||
|
|
|
@ -1,29 +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.catalog;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.Function;
|
||||
|
||||
import static java.util.stream.Collectors.toMap;
|
||||
|
||||
/**
|
||||
* In memory implementation of catalog designed for testing.
|
||||
*/
|
||||
public class InMemoryCatalog implements Catalog {
|
||||
private final Map<String, EsIndex> indices;
|
||||
|
||||
public InMemoryCatalog(List<EsIndex> indices) {
|
||||
this.indices = indices.stream().collect(toMap(EsIndex::name, Function.identity()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public GetIndexResult getIndex(String index) {
|
||||
EsIndex result = indices.get(index);
|
||||
return result == null ? GetIndexResult.notFound(index) : GetIndexResult.valid(result);
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue