Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Remaining queries for script keyword fields #59630

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -6,9 +6,13 @@

package org.elasticsearch.xpack.runtimefields.mapper;

import org.apache.lucene.search.MultiTermQuery.RewriteMethod;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.geo.ShapeRelation;
import org.elasticsearch.common.lucene.BytesRefs;
import org.elasticsearch.common.time.DateMathParser;
import org.elasticsearch.common.unit.Fuzziness;
import org.elasticsearch.common.xcontent.ToXContent.Params;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.mapper.MappedFieldType;
Expand All @@ -18,10 +22,16 @@
import org.elasticsearch.xpack.runtimefields.StringScriptFieldScript;
import org.elasticsearch.xpack.runtimefields.fielddata.ScriptBinaryFieldData;
import org.elasticsearch.xpack.runtimefields.query.StringScriptFieldExistsQuery;
import org.elasticsearch.xpack.runtimefields.query.StringScriptFieldFuzzyQuery;
import org.elasticsearch.xpack.runtimefields.query.StringScriptFieldPrefixQuery;
import org.elasticsearch.xpack.runtimefields.query.StringScriptFieldRangeQuery;
import org.elasticsearch.xpack.runtimefields.query.StringScriptFieldRegexpQuery;
import org.elasticsearch.xpack.runtimefields.query.StringScriptFieldTermQuery;
import org.elasticsearch.xpack.runtimefields.query.StringScriptFieldTermsQuery;
import org.elasticsearch.xpack.runtimefields.query.StringScriptFieldWildcardQuery;

import java.io.IOException;
import java.time.ZoneId;
import java.util.List;
import java.util.Map;
import java.util.Objects;
Expand Down Expand Up @@ -69,18 +79,75 @@ private StringScriptFieldScript.LeafFactory leafFactory(QueryShardContext contex

@Override
public Query existsQuery(QueryShardContext context) {
return new StringScriptFieldExistsQuery(leafFactory(context), name());
return new StringScriptFieldExistsQuery(script, leafFactory(context), name());
}

@Override
public Query fuzzyQuery(
Object value,
Fuzziness fuzziness,
int prefixLength,
int maxExpansions,
boolean transpositions,
QueryShardContext context
) {
return StringScriptFieldFuzzyQuery.build(
script,
leafFactory(context),
name(),
BytesRefs.toString(Objects.requireNonNull(value)),
fuzziness.asDistance(BytesRefs.toString(value)),
prefixLength,
transpositions
);
}

@Override
public Query prefixQuery(String value, RewriteMethod method, org.elasticsearch.index.query.QueryShardContext context) {
return new StringScriptFieldPrefixQuery(script, leafFactory(context), name(), value);
}

@Override
public Query rangeQuery(
Object lowerTerm,
Object upperTerm,
boolean includeLower,
boolean includeUpper,
ShapeRelation relation,
ZoneId timeZone,
DateMathParser parser,
QueryShardContext context
) {
return new StringScriptFieldRangeQuery(
script,
leafFactory(context),
name(),
BytesRefs.toString(Objects.requireNonNull(lowerTerm)),
BytesRefs.toString(Objects.requireNonNull(upperTerm)),
includeLower,
includeUpper
);
}

@Override
public Query regexpQuery(String value, int flags, int maxDeterminizedStates, RewriteMethod method, QueryShardContext context) {
return new StringScriptFieldRegexpQuery(script, leafFactory(context), name(), value, flags, maxDeterminizedStates);
}

@Override
public Query termQuery(Object value, QueryShardContext context) {
return new StringScriptFieldTermQuery(leafFactory(context), name(), BytesRefs.toString(Objects.requireNonNull(value)));
return new StringScriptFieldTermQuery(script, leafFactory(context), name(), BytesRefs.toString(Objects.requireNonNull(value)));
}

@Override
public Query termsQuery(List<?> values, QueryShardContext context) {
Set<String> terms = values.stream().map(v -> BytesRefs.toString(Objects.requireNonNull(v))).collect(toSet());
return new StringScriptFieldTermsQuery(leafFactory(context), name(), terms);
return new StringScriptFieldTermsQuery(script, leafFactory(context), name(), terms);
}

@Override
public Query wildcardQuery(String value, RewriteMethod method, QueryShardContext context) {
return new StringScriptFieldWildcardQuery(script, leafFactory(context), name(), value);
}

void doXContentBody(XContentBuilder builder, boolean includeDefaults, Params params) throws IOException {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
/*
* 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.runtimefields.query;

import org.apache.lucene.search.QueryVisitor;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.automaton.ByteRunAutomaton;
import org.elasticsearch.script.Script;
import org.elasticsearch.xpack.runtimefields.StringScriptFieldScript;

import java.util.List;

public abstract class AbstractStringScriptFieldAutomatonQuery extends AbstractStringScriptFieldQuery {
private final BytesRefBuilder scratch = new BytesRefBuilder();
private final ByteRunAutomaton automaton;

public AbstractStringScriptFieldAutomatonQuery(
Script script,
StringScriptFieldScript.LeafFactory leafFactory,
String fieldName,
ByteRunAutomaton automaton
) {
super(script, leafFactory, fieldName);
this.automaton = automaton;
}

@Override
protected final boolean matches(List<String> values) {
for (String value : values) {
scratch.copyChars(value);
if (automaton.run(scratch.bytes(), 0, scratch.length())) {
return true;
}
}
return false;
}

@Override
public final void visit(QueryVisitor visitor) {
if (visitor.acceptField(fieldName())) {
visitor.consumeTermsMatching(this, fieldName(), () -> automaton);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.TwoPhaseIterator;
import org.apache.lucene.search.Weight;
import org.elasticsearch.script.Script;
import org.elasticsearch.xpack.runtimefields.StringScriptFieldScript;
import org.elasticsearch.xpack.runtimefields.StringScriptFieldScript.LeafFactory;

Expand All @@ -27,18 +28,20 @@
* Abstract base class for building queries based on {@link StringScriptFieldScript}.
*/
abstract class AbstractStringScriptFieldQuery extends Query {
private final Script script;
private final StringScriptFieldScript.LeafFactory leafFactory;
private final String fieldName;

AbstractStringScriptFieldQuery(LeafFactory leafFactory, String fieldName) {
AbstractStringScriptFieldQuery(Script script, LeafFactory leafFactory, String fieldName) {
this.script = script;
this.leafFactory = Objects.requireNonNull(leafFactory);
this.fieldName = Objects.requireNonNull(fieldName);
}

/**
* Does the value match this query?
*/
public abstract boolean matches(List<String> values);
protected abstract boolean matches(List<String> values);

@Override
public final Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
Expand Down Expand Up @@ -69,23 +72,25 @@ public float matchCost() {
};
}

final Script script() {
return script;
}

protected final String fieldName() {
return fieldName;
}

@Override
public int hashCode() {
// TODO should leafFactory be here? Something about the script probably should be!
return Objects.hash(getClass(), fieldName);
return Objects.hash(getClass(), script, fieldName);
}

@Override
public boolean equals(Object obj) {
if (obj == null || getClass() != obj.getClass()) {
return false;
}
// TODO should leafFactory be here? Something about the script probably should be!
AbstractStringScriptFieldQuery other = (AbstractStringScriptFieldQuery) obj;
return fieldName.equals(other.fieldName);
return script.equals(other.script) && fieldName.equals(other.fieldName);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -6,26 +6,27 @@

package org.elasticsearch.xpack.runtimefields.query;

import org.elasticsearch.script.Script;
import org.elasticsearch.xpack.runtimefields.StringScriptFieldScript;

import java.util.List;

public class StringScriptFieldExistsQuery extends AbstractStringScriptFieldQuery {
public StringScriptFieldExistsQuery(StringScriptFieldScript.LeafFactory leafFactory, String fieldName) {
super(leafFactory, fieldName);
public StringScriptFieldExistsQuery(Script script, StringScriptFieldScript.LeafFactory leafFactory, String fieldName) {
super(script, leafFactory, fieldName);
}

@Override
public boolean matches(List<String> values) {
protected boolean matches(List<String> values) {
return false == values.isEmpty();
}

@Override
public final String toString(String field) {
if (fieldName().contentEquals(field)) {
return "*";
return "ScriptFieldExists";
}
return fieldName() + ":*";
return fieldName() + ":ScriptFieldExists";
}

// Superclass's equals and hashCode are great for this class
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,68 @@
/*
* 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.runtimefields.query;

import org.apache.lucene.index.Term;
import org.apache.lucene.search.FuzzyQuery;
import org.apache.lucene.util.automaton.ByteRunAutomaton;
import org.elasticsearch.script.Script;
import org.elasticsearch.xpack.runtimefields.StringScriptFieldScript;

import java.util.Objects;

public class StringScriptFieldFuzzyQuery extends AbstractStringScriptFieldAutomatonQuery {
public static StringScriptFieldFuzzyQuery build(
Script script,
StringScriptFieldScript.LeafFactory leafFactory,
String fieldName,
String term,
int maxEdits,
int prefixLength,
boolean transpositions
) {
int maxExpansions = 1; // We don't actually expand anything so the value here doesn't matter
FuzzyQuery delegate = new FuzzyQuery(new Term(fieldName, term), maxEdits, prefixLength, maxExpansions, transpositions);
ByteRunAutomaton automaton = delegate.getAutomata().runAutomaton;
return new StringScriptFieldFuzzyQuery(script, leafFactory, fieldName, automaton, delegate);
}

private final FuzzyQuery delegate;

private StringScriptFieldFuzzyQuery(
Script script,
StringScriptFieldScript.LeafFactory leafFactory,
String fieldName,
ByteRunAutomaton automaton,
FuzzyQuery delegate
) {
super(script, leafFactory, fieldName, automaton);
this.delegate = delegate;
}

@Override
public final String toString(String field) {
return delegate.toString(field);
}

@Override
public int hashCode() {
return Objects.hash(super.hashCode(), delegate);
}

@Override
public boolean equals(Object obj) {
if (false == super.equals(obj)) {
return false;
}
StringScriptFieldFuzzyQuery other = (StringScriptFieldFuzzyQuery) obj;
return delegate.equals(other.delegate);
}

FuzzyQuery delegate() {
return delegate;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,69 @@
/*
* 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.runtimefields.query;

import org.apache.lucene.search.PrefixQuery;
import org.apache.lucene.search.QueryVisitor;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.automaton.ByteRunAutomaton;
import org.elasticsearch.script.Script;
import org.elasticsearch.xpack.runtimefields.StringScriptFieldScript;

import java.util.List;
import java.util.Objects;

public class StringScriptFieldPrefixQuery extends AbstractStringScriptFieldQuery {
private final String prefix;

public StringScriptFieldPrefixQuery(Script script, StringScriptFieldScript.LeafFactory leafFactory, String fieldName, String prefix) {
super(script, leafFactory, fieldName);
this.prefix = Objects.requireNonNull(prefix);
}

@Override
protected boolean matches(List<String> values) {
for (String value : values) {
if (value != null && value.startsWith(prefix)) {
return true;
}
}
return false;
}

@Override
public void visit(QueryVisitor visitor) {
if (visitor.acceptField(fieldName())) {
visitor.consumeTermsMatching(this, fieldName(), () -> new ByteRunAutomaton(PrefixQuery.toAutomaton(new BytesRef(prefix))));
}
}

@Override
public final String toString(String field) {
if (fieldName().contentEquals(field)) {
return prefix + "*";
}
return fieldName() + ":" + prefix + "*";
}

@Override
public int hashCode() {
return Objects.hash(super.hashCode(), prefix);
}

@Override
public boolean equals(Object obj) {
if (false == super.equals(obj)) {
return false;
}
StringScriptFieldPrefixQuery other = (StringScriptFieldPrefixQuery) obj;
return prefix.equals(other.prefix);
}

String prefix() {
return prefix;
}
}
Loading