From 552453f270bfe71cc78d72da35251755ef6c850b Mon Sep 17 00:00:00 2001 From: paulb Date: Thu, 8 Feb 2024 14:09:55 +0100 Subject: [PATCH 01/15] Suggestion implementation QuerqyAggregationBuilder --- .../querqy/elasticsearch/QuerqyPlugin.java | 31 +++- .../querqy/elasticsearch/QuerqyProcessor.java | 42 ++++- .../aggregation/InternalQuerqy.java | 128 +++++++++++++ .../aggregation/ParsedQuerqy.java | 80 ++++++++ .../elasticsearch/aggregation/Querqy.java | 15 ++ .../aggregation/QuerqyAggregationBuilder.java | 173 ++++++++++++++++++ .../aggregation/QuerqyAggregator.java | 52 ++++++ .../aggregation/QuerqyAggregatorFactory.java | 58 ++++++ .../RewriterIntegrationTest.java | 55 ++++-- 9 files changed, 609 insertions(+), 25 deletions(-) create mode 100644 src/main/java/querqy/elasticsearch/aggregation/InternalQuerqy.java create mode 100644 src/main/java/querqy/elasticsearch/aggregation/ParsedQuerqy.java create mode 100644 src/main/java/querqy/elasticsearch/aggregation/Querqy.java create mode 100644 src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregationBuilder.java create mode 100644 src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregator.java create mode 100644 src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregatorFactory.java diff --git a/src/main/java/querqy/elasticsearch/QuerqyPlugin.java b/src/main/java/querqy/elasticsearch/QuerqyPlugin.java index d5bf8a4..d517df0 100644 --- a/src/main/java/querqy/elasticsearch/QuerqyPlugin.java +++ b/src/main/java/querqy/elasticsearch/QuerqyPlugin.java @@ -1,9 +1,5 @@ package querqy.elasticsearch; -import static java.util.Arrays.asList; -import static java.util.Collections.unmodifiableList; -import static querqy.elasticsearch.rewriterstore.Constants.SETTINGS_QUERQY_INDEX_NUM_REPLICAS; - import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.client.Client; @@ -16,7 +12,6 @@ import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.SettingsFilter; -import org.elasticsearch.xcontent.NamedXContentRegistry; import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.IndexModule; @@ -30,25 +25,33 @@ import org.elasticsearch.script.ScriptService; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; +import org.elasticsearch.xcontent.NamedXContentRegistry; +import querqy.elasticsearch.aggregation.InternalQuerqy; +import querqy.elasticsearch.aggregation.QuerqyAggregationBuilder; import querqy.elasticsearch.infologging.Log4jSink; import querqy.elasticsearch.query.QuerqyQueryBuilder; import querqy.elasticsearch.rewriterstore.DeleteRewriterAction; import querqy.elasticsearch.rewriterstore.NodesClearRewriterCacheAction; import querqy.elasticsearch.rewriterstore.NodesReloadRewriterAction; +import querqy.elasticsearch.rewriterstore.PutRewriterAction; import querqy.elasticsearch.rewriterstore.RestDeleteRewriterAction; import querqy.elasticsearch.rewriterstore.RestPutRewriterAction; -import querqy.elasticsearch.rewriterstore.PutRewriterAction; import querqy.elasticsearch.rewriterstore.TransportDeleteRewriterAction; import querqy.elasticsearch.rewriterstore.TransportNodesClearRewriterCacheAction; import querqy.elasticsearch.rewriterstore.TransportNodesReloadRewriterAction; import querqy.elasticsearch.rewriterstore.TransportPutRewriterAction; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.function.Supplier; +import static java.util.Arrays.asList; +import static java.util.Collections.unmodifiableList; +import static querqy.elasticsearch.rewriterstore.Constants.SETTINGS_QUERQY_INDEX_NUM_REPLICAS; + public class QuerqyPlugin extends Plugin implements SearchPlugin, ActionPlugin { @@ -57,7 +60,7 @@ public class QuerqyPlugin extends Plugin implements SearchPlugin, ActionPlugin { public QuerqyPlugin(final Settings settings) { rewriterShardContexts = new RewriterShardContexts(settings); - querqyProcessor = new QuerqyProcessor(rewriterShardContexts, new Log4jSink()); + querqyProcessor = new QuerqyProcessor(rewriterShardContexts, new Log4jSink(), settings); } @Override @@ -122,4 +125,18 @@ public List> getSettings() { Setting.Property.NodeScope)); } + + @Override + public ArrayList getAggregations() { + ArrayList r = new ArrayList<>(); + r.add( + new AggregationSpec( + QuerqyAggregationBuilder.NAME, + (in) -> new QuerqyAggregationBuilder(in, querqyProcessor), + QuerqyAggregationBuilder.PARSER + ).addResultReader(InternalQuerqy::new) + ); + return r; + } + } diff --git a/src/main/java/querqy/elasticsearch/QuerqyProcessor.java b/src/main/java/querqy/elasticsearch/QuerqyProcessor.java index a81d94d..1bb910c 100644 --- a/src/main/java/querqy/elasticsearch/QuerqyProcessor.java +++ b/src/main/java/querqy/elasticsearch/QuerqyProcessor.java @@ -3,6 +3,10 @@ import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.Query; +import org.elasticsearch.common.cache.Cache; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.query.SearchExecutionContext; import querqy.elasticsearch.infologging.LogPayloadType; import querqy.elasticsearch.infologging.SingleSinkInfoLogging; @@ -15,23 +19,46 @@ import querqy.lucene.LuceneSearchEngineRequestAdapter; import querqy.lucene.QueryParsingController; import querqy.rewrite.RewriteChain; +import querqy.rewrite.commonrules.model.DecorateInstruction; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.stream.Collectors; public class QuerqyProcessor { + public static final Setting CACHE_DECORATIONS_EXPIRE_AFTER_WRITE = Setting.timeSetting( + "querqy.caches.decorations.expire_after_write", + TimeValue.timeValueSeconds(10L), // 10s + TimeValue.timeValueSeconds(10L), + Setting.Property.NodeScope); + + public static final Setting CACHE_DECORATIONS_EXPIRE_AFTER_READ = Setting.timeSetting( + "querqy.caches.decorations.expire_after_read", + TimeValue.timeValueSeconds(10L), // 10ns + TimeValue.timeValueSeconds(10L), + Setting.Property.NodeScope); + private static final RewriteChain EMPTY_REWRITE_CHAIN = new RewriteChain(Collections.emptyList()); + //private final Map> querqyQueryCache; + private final Cache> querqyQueryCache; + private RewriterShardContexts rewriterShardContexts; private Sink infoLoggingSink; public QuerqyProcessor(final RewriterShardContexts rewriterShardContexts, final Sink infoLoggingSink) { + this(rewriterShardContexts, infoLoggingSink, Settings.EMPTY); + } + + public QuerqyProcessor(final RewriterShardContexts rewriterShardContexts, final Sink infoLoggingSink, final Settings settings) { this.rewriterShardContexts = rewriterShardContexts; this.infoLoggingSink = infoLoggingSink; + //querqyInfoCache = new HashMap<>(); + querqyQueryCache = Caches.buildCache(CACHE_DECORATIONS_EXPIRE_AFTER_READ.get(settings), CACHE_DECORATIONS_EXPIRE_AFTER_WRITE.get(settings)); } public Query parseQuery(final QuerqyQueryBuilder queryBuilder, final SearchExecutionContext context) @@ -75,10 +102,6 @@ public Query parseQuery(final QuerqyQueryBuilder queryBuilder, final SearchExecu final QueryParsingController controller = new QueryParsingController(requestAdapter); final LuceneQueries queries = controller.process(); - -// // TODO: make decos part of the general Querqy object model -// final Set decorations = (Set) requestAdapter.getContext().get(DecorateInstruction.CONTEXT_KEY); - if ((queries.querqyBoostQueries == null || queries.querqyBoostQueries.isEmpty()) && (queries.filterQueries == null || queries.filterQueries.isEmpty()) && queries.mainQuery instanceof BooleanQuery) { @@ -108,10 +131,21 @@ public Query parseQuery(final QuerqyQueryBuilder queryBuilder, final SearchExecu if (infoLogging != null) { infoLogging.endOfRequest(requestAdapter); } + + final Set decorations = (Set) requestAdapter.getContext().get(DecorateInstruction.DECORATION_CONTEXT_KEY); + if (decorations != null) { + querqyQueryCache.put(query, Collections.singletonMap("decorations", decorations)); + } + return query; } + public Map getQuerqyInfoForQuery(Query query) { + final Map querqyInfo = querqyQueryCache.get(query); + querqyQueryCache.invalidate(query); + return querqyInfo; + } void appendFilterQueries(final LuceneQueries queries, final BooleanQuery.Builder builder) { diff --git a/src/main/java/querqy/elasticsearch/aggregation/InternalQuerqy.java b/src/main/java/querqy/elasticsearch/aggregation/InternalQuerqy.java new file mode 100644 index 0000000..036a414 --- /dev/null +++ b/src/main/java/querqy/elasticsearch/aggregation/InternalQuerqy.java @@ -0,0 +1,128 @@ +package querqy.elasticsearch.aggregation; + +import org.elasticsearch.Version; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.metrics.ScriptedMetric; +import org.elasticsearch.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import static java.util.Collections.singletonList; + +public class InternalQuerqy extends InternalAggregation implements ScriptedMetric { + + private final List aggregations; + + InternalQuerqy(String name, List aggregations, Map metadata) { + super(name, metadata); + this.aggregations = aggregations; + } + + public InternalQuerqy(StreamInput in) throws IOException { + super(in); + if (in.getVersion().before(Version.V_7_8_0)) { + aggregations = singletonList(in.readGenericValue()); + } else { + aggregations = in.readList(StreamInput::readGenericValue); + } + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException { + if (out.getVersion().before(Version.V_7_8_0)) { + if (aggregations.size() > 1) { + /* + * If aggregations has more than one entry we're trying to + * serialize an unreduced aggregation. This *should* only + * happen when we're returning a scripted_metric over cross + * cluster search. + */ + throw new IllegalArgumentException("querqy doesn't support cross cluster search until 7.8.0"); + } + out.writeGenericValue(aggregations.get(0)); + } else { + out.writeCollection(aggregations, StreamOutput::writeGenericValue); + } + + } + + @Override + public String getWriteableName() { + return QuerqyAggregationBuilder.NAME; + } + + @Override + public Object aggregation() { + if (aggregations.size() != 1) { + throw new IllegalStateException("aggregation was not reduced"); + } + return aggregations.get(0); + } + + List aggregationsList() { + return aggregations; + } + + @Override + public InternalAggregation reduce(List aggregations, ReduceContext reduceContext) { + List aggregationObjects = new ArrayList<>(); + for (InternalAggregation aggregation : aggregations) { + InternalQuerqy mapReduceAggregation = (InternalQuerqy) aggregation; + aggregationObjects.addAll(mapReduceAggregation.aggregations); + } + InternalQuerqy firstAggregation = ((InternalQuerqy) aggregations.get(0)); + List aggregation; + if (reduceContext.isFinalReduce()) { + aggregation = Collections.singletonList(aggregationObjects); + } else { + // if we are not an final reduce we have to maintain all the aggs from all the incoming one + // until we hit the final reduce phase. + aggregation = aggregationObjects; + } + return new InternalQuerqy(firstAggregation.getName(), aggregation, getMetadata()); + } + + @Override + protected boolean mustReduceOnSingleInternalAgg() { + return true; + } + + @Override + public Object getProperty(List path) { + if (path.isEmpty()) { + return this; + } else if (path.size() == 1 && "value".equals(path.get(0))) { + return aggregation(); + } else { + throw new IllegalArgumentException("path not supported for [" + getName() + "]: " + path); + } + } + + @Override + public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + return builder.field(CommonFields.VALUE.getPreferredName(), aggregation()); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + + InternalQuerqy other = (InternalQuerqy) obj; + return Objects.equals(aggregations, other.aggregations); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), aggregations); + } + +} diff --git a/src/main/java/querqy/elasticsearch/aggregation/ParsedQuerqy.java b/src/main/java/querqy/elasticsearch/aggregation/ParsedQuerqy.java new file mode 100644 index 0000000..2aa0520 --- /dev/null +++ b/src/main/java/querqy/elasticsearch/aggregation/ParsedQuerqy.java @@ -0,0 +1,80 @@ +package querqy.elasticsearch.aggregation; + +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.search.aggregations.ParsedAggregation; +import org.elasticsearch.xcontent.ObjectParser; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.XContentParser; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +public class ParsedQuerqy extends ParsedAggregation implements Querqy { + + private List aggregation; + + @Override + public String getType() { + return QuerqyAggregationBuilder.NAME; + } + + @Override + public Object aggregation() { + assert aggregation.size() == 1; + return aggregation.get(0); + } + + @Override + public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + return builder.field(CommonFields.VALUE.getPreferredName(), aggregation()); + } + + private static final ObjectParser PARSER = new ObjectParser<>( + ParsedQuerqy.class.getSimpleName(), + true, + ParsedQuerqy::new + ); + + static { + declareAggregationFields(PARSER); + PARSER.declareField( + (agg, value) -> agg.aggregation = Collections.singletonList(value), + ParsedQuerqy::parseValue, + CommonFields.VALUE, + ObjectParser.ValueType.VALUE_OBJECT_ARRAY + ); + } + + private static Object parseValue(XContentParser parser) throws IOException { + XContentParser.Token token = parser.currentToken(); + Object value = null; + if (token == XContentParser.Token.VALUE_NULL) { + value = null; + } else if (token.isValue()) { + if (token == XContentParser.Token.VALUE_STRING) { + // binary values will be parsed back and returned as base64 strings when reading from json and yaml + value = parser.text(); + } else if (token == XContentParser.Token.VALUE_NUMBER) { + value = parser.numberValue(); + } else if (token == XContentParser.Token.VALUE_BOOLEAN) { + value = parser.booleanValue(); + } else if (token == XContentParser.Token.VALUE_EMBEDDED_OBJECT) { + // binary values will be parsed back and returned as BytesArray when reading from cbor and smile + value = new BytesArray(parser.binaryValue()); + } + } else if (token == XContentParser.Token.START_OBJECT) { + value = parser.map(); + } else if (token == XContentParser.Token.START_ARRAY) { + value = parser.list(); + } + return value; + } + + public static ParsedQuerqy fromXContent(XContentParser parser, final String name) { + ParsedQuerqy aggregation = PARSER.apply(parser, null); + aggregation.setName(name); + return aggregation; + } + +} diff --git a/src/main/java/querqy/elasticsearch/aggregation/Querqy.java b/src/main/java/querqy/elasticsearch/aggregation/Querqy.java new file mode 100644 index 0000000..dee6d24 --- /dev/null +++ b/src/main/java/querqy/elasticsearch/aggregation/Querqy.java @@ -0,0 +1,15 @@ +package querqy.elasticsearch.aggregation; + +import org.elasticsearch.search.aggregations.Aggregation; + +/** + * A {@code Querqy} aggregation. Defines a single bucket the holds all the querqy info in the search context. + */ +public interface Querqy extends Aggregation { + + /** + * The result of the aggregation. The type of the object depends on the aggregation that was run. + */ + Object aggregation(); + +} diff --git a/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregationBuilder.java b/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregationBuilder.java new file mode 100644 index 0000000..28a15bd --- /dev/null +++ b/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregationBuilder.java @@ -0,0 +1,173 @@ +package querqy.elasticsearch.aggregation; + +import org.elasticsearch.common.ParsingException; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.search.aggregations.AbstractAggregationBuilder; +import org.elasticsearch.search.aggregations.AggregationBuilder; +import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; +import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.support.AggregationContext; +import org.elasticsearch.xcontent.ConstructingObjectParser; +import org.elasticsearch.xcontent.ObjectParser; +import org.elasticsearch.xcontent.ParseField; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.XContentParser; +import querqy.elasticsearch.QuerqyProcessor; +import querqy.elasticsearch.query.QuerqyQueryBuilder; +import querqy.elasticsearch.query.Rewriter; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; + +import static querqy.elasticsearch.query.RequestUtils.fieldBoostModelToString; + +public class QuerqyAggregationBuilder extends AbstractAggregationBuilder { + + public static final String NAME = "querqy"; + + private static final ParseField PARAMS_FIELD = new ParseField("params"); + + private QuerqyProcessor querqyProcessor; + + public static final ObjectParser PARSER = + new ObjectParser<>(NAME, QuerqyAggregationBuilder::new); + + static { + PARSER.declareObject(QuerqyAggregationBuilder::params, (p, name) -> p.map(), PARAMS_FIELD); + } + + private Map params; + + public QuerqyAggregationBuilder() { + super(NAME); + } + + public QuerqyAggregationBuilder(String name) { + super(name); + } + + public QuerqyAggregationBuilder(QuerqyProcessor querqyProcessor) { + super(NAME); + setQuerqyProcessor(querqyProcessor); + } + + protected QuerqyAggregationBuilder(QuerqyAggregationBuilder clone, Builder factoriesBuilder, Map metadata) { + super(clone, factoriesBuilder, metadata); + this.params = clone.params; + } + + public QuerqyAggregationBuilder(StreamInput in) throws IOException { + super(in); + if (in.readBoolean()) { + params = in.readMap(); + } + } + + @Override + protected AggregationBuilder shallowCopy(Builder factoriesBuilder, Map metadata) { + return new QuerqyAggregationBuilder(this, factoriesBuilder, metadata); + } + + /** + * Read from a stream. + */ + public QuerqyAggregationBuilder(StreamInput in, QuerqyProcessor querqyProcessor) throws IOException { + super(in); + if (in.readBoolean()) { + params = in.readMap(); + } + setQuerqyProcessor(querqyProcessor); + } + + public QuerqyAggregationBuilder setQuerqyProcessor(final QuerqyProcessor querqyProcessor) { + this.querqyProcessor = Objects.requireNonNull(querqyProcessor); + return this; + } + + /** + * Set parameters that will be available in the {@code init}, + * {@code map} and {@code combine} phases. + */ + public QuerqyAggregationBuilder params(Map params) { + if (params == null) { + throw new IllegalArgumentException("[params] must not be null: [" + name + "]"); + } + this.params = params; + return this; + } + + /** + * Get parameters that will be available in the {@code init}, + * {@code map} and {@code combine} phases. + */ + public Map params() { + return params; + } + + @Override + public BucketCardinality bucketCardinality() { + return BucketCardinality.NONE; + } + + @Override + protected AggregatorFactory doBuild(AggregationContext context, AggregatorFactory parent, Builder subFactoriesBuilder) + throws IOException { + return new QuerqyAggregatorFactory(name, context, parent, subFactoriesBuilder, metadata, querqyProcessor); + } + + @Override + protected XContentBuilder internalXContent(XContentBuilder builder, Params builderParams) throws IOException { + builder.startObject(); + if (params != null) { + builder.field(PARAMS_FIELD.getPreferredName()); + builder.map(params); + } + builder.endObject(); + return builder; + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException { + boolean hasParams = params != null; + out.writeBoolean(hasParams); + if (hasParams) { + out.writeMap(params); + } + } + + public static QuerqyAggregationBuilder fromXContent(final XContentParser parser, final QuerqyProcessor querqyProcessor) { + final QuerqyAggregationBuilder builder; + try { + builder = PARSER.apply(parser, null); + } catch (final IllegalArgumentException e) { + throw new ParsingException(parser.getTokenLocation(), e.getMessage(), e); + } + builder.setQuerqyProcessor(querqyProcessor); + return builder; + } + + @Override + public String getType() { + return NAME; + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), params); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + QuerqyAggregationBuilder other = (QuerqyAggregationBuilder) obj; + return Objects.equals(params, other.params); + } + +} diff --git a/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregator.java b/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregator.java new file mode 100644 index 0000000..31601fc --- /dev/null +++ b/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregator.java @@ -0,0 +1,52 @@ +package querqy.elasticsearch.aggregation; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.ScoreMode; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.LeafBucketCollector; +import org.elasticsearch.search.aggregations.metrics.MetricsAggregator; +import org.elasticsearch.search.aggregations.support.AggregationContext; + +import java.io.IOException; +import java.util.Collections; +import java.util.Map; + +public final class QuerqyAggregator extends MetricsAggregator { + + public QuerqyAggregator(String name, AggregationContext context, Map metadata) + throws IOException { + super(name, context, null, metadata); + } + + @Override + public ScoreMode scoreMode() { + return ScoreMode.COMPLETE; + } + + @Override + protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException { + // No sub-aggregations + return LeafBucketCollector.NO_OP_COLLECTOR; + } + + @Override + public InternalAggregation buildAggregation(long l) { + Map metadata = metadata(); + Map result = metadata != null && !metadata.isEmpty() ? metadata : Collections.emptyMap(); + StreamOutput.checkWriteable(result); + return new InternalQuerqy( + name, + Collections.singletonList(result), + null + ); + } + + @Override + public InternalAggregation buildEmptyAggregation() { + throw new UnsupportedOperationException( + "querqy_decoration aggregations cannot serve as sub-aggregations, hence should never be called on #buildEmptyAggregations" + ); + } + +} diff --git a/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregatorFactory.java b/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregatorFactory.java new file mode 100644 index 0000000..3168d06 --- /dev/null +++ b/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregatorFactory.java @@ -0,0 +1,58 @@ +package querqy.elasticsearch.aggregation; + +import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.Aggregator; +import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.CardinalityUpperBound; +import org.elasticsearch.search.aggregations.support.AggregationContext; +import querqy.elasticsearch.QuerqyProcessor; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +public class QuerqyAggregatorFactory extends AggregatorFactory { + + final private QuerqyProcessor querqyProcessor; + + public QuerqyAggregatorFactory( + String name, + AggregationContext context, + AggregatorFactory parent, + AggregatorFactories.Builder subFactories, + Map metadata, + QuerqyProcessor querqyProcessor + ) throws IOException { + super(name, context, parent, subFactories, metadata); + this.querqyProcessor = Objects.requireNonNull(querqyProcessor); + } + + @Override + public Aggregator createInternal(Aggregator parent, CardinalityUpperBound cardinality, Map metadata) + throws IOException { + if (parent != null) { + throw new IllegalArgumentException( + "Aggregation [" + + parent.name() + + "] cannot have a querqy_decoration " + + "sub-aggregation [" + + name + + "]. querqy_decoration aggregations can only be defined as top level aggregations" + ); + } + if (cardinality != CardinalityUpperBound.ONE) { + throw new AggregationExecutionException("Aggregation [" + name() + "] must have cardinality 1 but was [" + cardinality + "]"); + } + Map info = querqyProcessor.getQuerqyInfoForQuery(context.subSearchContext().query()); + if (info != null && !info.isEmpty()) { + if (metadata == null) { + metadata = new HashMap<>(); + } + metadata.putAll(info); + } + return new QuerqyAggregator(name, context, metadata); + } +} diff --git a/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java b/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java index 5394c34..9f75133 100644 --- a/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java +++ b/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java @@ -1,23 +1,17 @@ package querqy.elasticsearch; -import java.io.BufferedReader; -import java.io.InputStreamReader; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.ExecutionException; -import java.util.stream.Collectors; - import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.ESSingleNodeTestCase; +import org.elasticsearch.xcontent.XContentType; import org.junit.After; import org.junit.Test; +import querqy.elasticsearch.aggregation.QuerqyAggregationBuilder; import querqy.elasticsearch.query.MatchingQuery; import querqy.elasticsearch.query.QuerqyQueryBuilder; import querqy.elasticsearch.query.Rewriter; @@ -27,6 +21,16 @@ import querqy.elasticsearch.rewriterstore.PutRewriterRequest; import querqy.elasticsearch.rewriterstore.PutRewriterResponse; +import java.io.BufferedReader; +import java.io.InputStreamReader; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; + public class RewriterIntegrationTest extends ESSingleNodeTestCase { private final String INDEX_NAME = "test_index"; @@ -44,7 +48,7 @@ public void testSearchWithConfig() throws Exception { content.put("class", querqy.elasticsearch.rewriter.SimpleCommonRulesRewriterFactory.class.getName()); final Map config = new HashMap<>(); - config.put("rules", "k =>\nSYNONYM: c"); + config.put("rules", "k =>\nSYNONYM: c\na =>\nDECORATE: REDIRECT /faq/a"); config.put("ignoreCase", true); config.put("querqyParser", querqy.rewrite.commonrules.WhiteSpaceQuerqyParserFactory.class.getName()); content.put("config", config); @@ -53,18 +57,41 @@ public void testSearchWithConfig() throws Exception { client().execute(PutRewriterAction.INSTANCE, request).get(); - QuerqyQueryBuilder querqyQuery = new QuerqyQueryBuilder(getInstanceFromNode(QuerqyProcessor.class)); + QuerqyProcessor querqyProcessor = getInstanceFromNode(QuerqyProcessor.class); + QuerqyQueryBuilder querqyQuery = new QuerqyQueryBuilder(querqyProcessor); querqyQuery.setRewriters(Collections.singletonList(new Rewriter("common_rules"))); querqyQuery.setMatchingQuery(new MatchingQuery("a k")); querqyQuery.setQueryFieldsAndBoostings(Arrays.asList("field1", "field2")); querqyQuery.setMinimumShouldMatch("1"); - final SearchRequestBuilder searchRequestBuilder = client().prepareSearch(INDEX_NAME); + QuerqyAggregationBuilder querqyAggregationBuilder = + QuerqyAggregationBuilder.fromXContent( + XContentHelper.createParser( + null, + null, + new BytesArray("{" + + "\"params\": {" + + "\"test-query\": \"a k\"" + + "}}" + ), + XContentType.JSON), querqyProcessor + ); + + SearchRequestBuilder searchRequestBuilder = client().prepareSearch(INDEX_NAME); searchRequestBuilder.setQuery(querqyQuery); - + searchRequestBuilder.addAggregation(querqyAggregationBuilder); SearchResponse response = client().search(searchRequestBuilder.request()).get(); +// System.out.println(response); assertEquals(2L, response.getHits().getTotalHits().value); + assertEquals("{\"querqy\":{\"value\":[{\"decorations\":[\"REDIRECT /faq/a\"]}]}}", response.getAggregations().getAsMap().get(QuerqyAggregationBuilder.NAME).toString()); + + querqyQuery.setMatchingQuery(new MatchingQuery("x z")); + searchRequestBuilder.setQuery(querqyQuery); + response = client().search(searchRequestBuilder.request()).get(); +// System.out.println(response); + assertEquals(0L, response.getHits().getTotalHits().value); + assertEquals("{\"querqy\":{\"value\":[{}]}}", response.getAggregations().getAsMap().get(QuerqyAggregationBuilder.NAME).toString()); } @Test From 6db8746847675d1b0413056ed08ed1aa0645b1ad Mon Sep 17 00:00:00 2001 From: paulb Date: Thu, 8 Feb 2024 20:41:10 +0100 Subject: [PATCH 02/15] Add dependency with jackson-annotations --- pom.xml | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 2fd64ff..34215bf 100644 --- a/pom.xml +++ b/pom.xml @@ -61,6 +61,7 @@ 17 6 ${es.major.version}.${es.minor.version}.${es.bugfix.version} + 2.9.9 0.8.5 @@ -77,7 +78,13 @@ - + + + com.fasterxml.jackson.core + jackson-annotations + ${jackson.version} + + org.elasticsearch elasticsearch From d48d7617b8ff8d2c1164ba0ec7dd7f83dbd5241a Mon Sep 17 00:00:00 2001 From: paulb Date: Mon, 19 Feb 2024 13:09:36 +0100 Subject: [PATCH 03/15] Suggested implementation via wrapped query --- .../querqy/elasticsearch/QuerqyPlugin.java | 8 +- .../querqy/elasticsearch/QuerqyProcessor.java | 45 ++------- .../aggregation/DecoratedQuery.java | 91 +++++++++++++++++++ .../aggregation/QuerqyAggregationBuilder.java | 38 +------- .../aggregation/QuerqyAggregator.java | 15 +-- .../aggregation/QuerqyAggregatorFactory.java | 25 ++--- .../RewriterIntegrationTest.java | 8 +- 7 files changed, 126 insertions(+), 104 deletions(-) create mode 100644 src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java diff --git a/src/main/java/querqy/elasticsearch/QuerqyPlugin.java b/src/main/java/querqy/elasticsearch/QuerqyPlugin.java index d517df0..9eeffa3 100644 --- a/src/main/java/querqy/elasticsearch/QuerqyPlugin.java +++ b/src/main/java/querqy/elasticsearch/QuerqyPlugin.java @@ -60,7 +60,7 @@ public class QuerqyPlugin extends Plugin implements SearchPlugin, ActionPlugin { public QuerqyPlugin(final Settings settings) { rewriterShardContexts = new RewriterShardContexts(settings); - querqyProcessor = new QuerqyProcessor(rewriterShardContexts, new Log4jSink(), settings); + querqyProcessor = new QuerqyProcessor(rewriterShardContexts, new Log4jSink()); } @Override @@ -127,12 +127,12 @@ public List> getSettings() { } @Override - public ArrayList getAggregations() { - ArrayList r = new ArrayList<>(); + public List getAggregations() { + List r = new ArrayList<>(); r.add( new AggregationSpec( QuerqyAggregationBuilder.NAME, - (in) -> new QuerqyAggregationBuilder(in, querqyProcessor), + QuerqyAggregationBuilder::new, QuerqyAggregationBuilder.PARSER ).addResultReader(InternalQuerqy::new) ); diff --git a/src/main/java/querqy/elasticsearch/QuerqyProcessor.java b/src/main/java/querqy/elasticsearch/QuerqyProcessor.java index 1bb910c..c429337 100644 --- a/src/main/java/querqy/elasticsearch/QuerqyProcessor.java +++ b/src/main/java/querqy/elasticsearch/QuerqyProcessor.java @@ -3,11 +3,8 @@ import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.Query; -import org.elasticsearch.common.cache.Cache; -import org.elasticsearch.common.settings.Setting; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.query.SearchExecutionContext; +import querqy.elasticsearch.aggregation.DecoratedQuery; import querqy.elasticsearch.infologging.LogPayloadType; import querqy.elasticsearch.infologging.SingleSinkInfoLogging; import querqy.elasticsearch.query.InfoLoggingSpec; @@ -21,44 +18,23 @@ import querqy.rewrite.RewriteChain; import querqy.rewrite.commonrules.model.DecorateInstruction; +import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Map; import java.util.Set; import java.util.stream.Collectors; public class QuerqyProcessor { - public static final Setting CACHE_DECORATIONS_EXPIRE_AFTER_WRITE = Setting.timeSetting( - "querqy.caches.decorations.expire_after_write", - TimeValue.timeValueSeconds(10L), // 10s - TimeValue.timeValueSeconds(10L), - Setting.Property.NodeScope); - - public static final Setting CACHE_DECORATIONS_EXPIRE_AFTER_READ = Setting.timeSetting( - "querqy.caches.decorations.expire_after_read", - TimeValue.timeValueSeconds(10L), // 10ns - TimeValue.timeValueSeconds(10L), - Setting.Property.NodeScope); - private static final RewriteChain EMPTY_REWRITE_CHAIN = new RewriteChain(Collections.emptyList()); - //private final Map> querqyQueryCache; - private final Cache> querqyQueryCache; - private RewriterShardContexts rewriterShardContexts; private Sink infoLoggingSink; public QuerqyProcessor(final RewriterShardContexts rewriterShardContexts, final Sink infoLoggingSink) { - this(rewriterShardContexts, infoLoggingSink, Settings.EMPTY); - } - - public QuerqyProcessor(final RewriterShardContexts rewriterShardContexts, final Sink infoLoggingSink, final Settings settings) { this.rewriterShardContexts = rewriterShardContexts; this.infoLoggingSink = infoLoggingSink; - //querqyInfoCache = new HashMap<>(); - querqyQueryCache = Caches.buildCache(CACHE_DECORATIONS_EXPIRE_AFTER_READ.get(settings), CACHE_DECORATIONS_EXPIRE_AFTER_WRITE.get(settings)); } public Query parseQuery(final QuerqyQueryBuilder queryBuilder, final SearchExecutionContext context) @@ -127,26 +103,19 @@ public Query parseQuery(final QuerqyQueryBuilder queryBuilder, final SearchExecu appendFilterQueries(queries, builder); - final BooleanQuery query = builder.build(); + final Set decorations = (Set) requestAdapter.getContext().get(DecorateInstruction.DECORATION_CONTEXT_KEY); + final Query query = + decorations != null && !decorations.isEmpty() ? + DecoratedQuery.from(builder.build(), new ArrayList<>(decorations)) : + builder.build(); if (infoLogging != null) { infoLogging.endOfRequest(requestAdapter); } - final Set decorations = (Set) requestAdapter.getContext().get(DecorateInstruction.DECORATION_CONTEXT_KEY); - if (decorations != null) { - querqyQueryCache.put(query, Collections.singletonMap("decorations", decorations)); - } - return query; } - public Map getQuerqyInfoForQuery(Query query) { - final Map querqyInfo = querqyQueryCache.get(query); - querqyQueryCache.invalidate(query); - return querqyInfo; - } - void appendFilterQueries(final LuceneQueries queries, final BooleanQuery.Builder builder) { if (queries.filterQueries != null) { diff --git a/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java b/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java new file mode 100644 index 0000000..340e020 --- /dev/null +++ b/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java @@ -0,0 +1,91 @@ +package querqy.elasticsearch.aggregation; + +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.QueryVisitor; +import org.apache.lucene.search.ScoreMode; +import org.apache.lucene.search.Weight; + +import java.io.IOException; +import java.util.List; +import java.util.Objects; + +public class DecoratedQuery extends Query { + + final private BooleanQuery booleanQuery; + final private List decorations; + + private DecoratedQuery(BooleanQuery booleanQuery, List decorations) { + this.booleanQuery = Objects.requireNonNull(booleanQuery); + this.decorations = Objects.requireNonNull(decorations); + } + + public static Query from(BooleanQuery booleanQuery, List decorations) { + return new DecoratedQuery(booleanQuery, decorations); + } + + public BooleanQuery getBooleanQuery() { + return booleanQuery; + } + + public List getDecorations() { + return decorations; + } + + @Override + public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException { + return booleanQuery.createWeight(searcher, scoreMode, boost); + } + + @Override + public Query rewrite(IndexReader reader) throws IOException { + return booleanQuery.rewrite(reader); + } + + @Override + public void visit(QueryVisitor visitor) { + booleanQuery.visit(visitor); + } + + @Override + public String toString(String field) { + return booleanQuery.toString(); + } + + @Override + public boolean equals(Object o) { + return sameClassAs(o) && + equalsTo(getClass().cast(o)); + } + + private boolean equalsTo(DecoratedQuery other) { + return booleanQuery.getMinimumNumberShouldMatch() == other.getBooleanQuery().getMinimumNumberShouldMatch() && + booleanQuery.clauses().equals(other.getBooleanQuery().clauses()) && + decorations.equals(other.getDecorations()); + } + + private int computeHashCode() { + int hashCode = Objects.hash(booleanQuery, decorations); + if (hashCode == 0) { + hashCode = 1; + } + return hashCode; + } + + // cached hash code is ok since boolean queries are immutable + private int hashCode; + + @Override + public int hashCode() { + // no need for synchronization, in the worst case we would just compute the hash several times. + if (hashCode == 0) { + hashCode = computeHashCode(); + assert hashCode != 0; + } + assert hashCode == computeHashCode(); + return hashCode; + } + +} \ No newline at end of file diff --git a/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregationBuilder.java b/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregationBuilder.java index 28a15bd..df8faec 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregationBuilder.java +++ b/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregationBuilder.java @@ -8,32 +8,21 @@ import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.support.AggregationContext; -import org.elasticsearch.xcontent.ConstructingObjectParser; import org.elasticsearch.xcontent.ObjectParser; import org.elasticsearch.xcontent.ParseField; import org.elasticsearch.xcontent.XContentBuilder; import org.elasticsearch.xcontent.XContentParser; -import querqy.elasticsearch.QuerqyProcessor; -import querqy.elasticsearch.query.QuerqyQueryBuilder; -import querqy.elasticsearch.query.Rewriter; import java.io.IOException; -import java.util.Collections; -import java.util.HashMap; import java.util.Map; import java.util.Objects; -import java.util.Optional; - -import static querqy.elasticsearch.query.RequestUtils.fieldBoostModelToString; public class QuerqyAggregationBuilder extends AbstractAggregationBuilder { - public static final String NAME = "querqy"; + public static final String NAME = "decorations"; private static final ParseField PARAMS_FIELD = new ParseField("params"); - private QuerqyProcessor querqyProcessor; - public static final ObjectParser PARSER = new ObjectParser<>(NAME, QuerqyAggregationBuilder::new); @@ -51,23 +40,11 @@ public QuerqyAggregationBuilder(String name) { super(name); } - public QuerqyAggregationBuilder(QuerqyProcessor querqyProcessor) { - super(NAME); - setQuerqyProcessor(querqyProcessor); - } - protected QuerqyAggregationBuilder(QuerqyAggregationBuilder clone, Builder factoriesBuilder, Map metadata) { super(clone, factoriesBuilder, metadata); this.params = clone.params; } - public QuerqyAggregationBuilder(StreamInput in) throws IOException { - super(in); - if (in.readBoolean()) { - params = in.readMap(); - } - } - @Override protected AggregationBuilder shallowCopy(Builder factoriesBuilder, Map metadata) { return new QuerqyAggregationBuilder(this, factoriesBuilder, metadata); @@ -76,17 +53,11 @@ protected AggregationBuilder shallowCopy(Builder factoriesBuilder, Map metadata) + final List decorations; + + public QuerqyAggregator(String name, AggregationContext context, Map metadata, List decorations) throws IOException { super(name, context, null, metadata); + this.decorations = decorations; } @Override @@ -32,13 +35,11 @@ protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucket @Override public InternalAggregation buildAggregation(long l) { - Map metadata = metadata(); - Map result = metadata != null && !metadata.isEmpty() ? metadata : Collections.emptyMap(); - StreamOutput.checkWriteable(result); + StreamOutput.checkWriteable(decorations); return new InternalQuerqy( name, - Collections.singletonList(result), - null + decorations, + metadata() ); } diff --git a/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregatorFactory.java b/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregatorFactory.java index 3168d06..f0686fc 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregatorFactory.java +++ b/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregatorFactory.java @@ -1,33 +1,28 @@ package querqy.elasticsearch.aggregation; +import org.apache.lucene.search.Query; import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.CardinalityUpperBound; import org.elasticsearch.search.aggregations.support.AggregationContext; -import querqy.elasticsearch.QuerqyProcessor; import java.io.IOException; import java.util.Collections; -import java.util.HashMap; +import java.util.List; import java.util.Map; -import java.util.Objects; public class QuerqyAggregatorFactory extends AggregatorFactory { - final private QuerqyProcessor querqyProcessor; - public QuerqyAggregatorFactory( String name, AggregationContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactories, - Map metadata, - QuerqyProcessor querqyProcessor + Map metadata ) throws IOException { super(name, context, parent, subFactories, metadata); - this.querqyProcessor = Objects.requireNonNull(querqyProcessor); } @Override @@ -46,13 +41,11 @@ public Aggregator createInternal(Aggregator parent, CardinalityUpperBound cardin if (cardinality != CardinalityUpperBound.ONE) { throw new AggregationExecutionException("Aggregation [" + name() + "] must have cardinality 1 but was [" + cardinality + "]"); } - Map info = querqyProcessor.getQuerqyInfoForQuery(context.subSearchContext().query()); - if (info != null && !info.isEmpty()) { - if (metadata == null) { - metadata = new HashMap<>(); - } - metadata.putAll(info); - } - return new QuerqyAggregator(name, context, metadata); + Query query = context.subSearchContext() == null ? null : context.subSearchContext().query(); + List decorations = + query instanceof DecoratedQuery ? + ((DecoratedQuery) query).getDecorations() : + Collections.emptyList(); + return new QuerqyAggregator(name, context, metadata, decorations); } } diff --git a/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java b/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java index 9f75133..190115d 100644 --- a/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java +++ b/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java @@ -74,24 +74,22 @@ public void testSearchWithConfig() throws Exception { "\"test-query\": \"a k\"" + "}}" ), - XContentType.JSON), querqyProcessor + XContentType.JSON) ); SearchRequestBuilder searchRequestBuilder = client().prepareSearch(INDEX_NAME); searchRequestBuilder.setQuery(querqyQuery); searchRequestBuilder.addAggregation(querqyAggregationBuilder); SearchResponse response = client().search(searchRequestBuilder.request()).get(); -// System.out.println(response); assertEquals(2L, response.getHits().getTotalHits().value); - assertEquals("{\"querqy\":{\"value\":[{\"decorations\":[\"REDIRECT /faq/a\"]}]}}", response.getAggregations().getAsMap().get(QuerqyAggregationBuilder.NAME).toString()); + assertEquals("{\"decorations\":{\"value\":[\"REDIRECT /faq/a\"]}}", response.getAggregations().getAsMap().get(QuerqyAggregationBuilder.NAME).toString()); querqyQuery.setMatchingQuery(new MatchingQuery("x z")); searchRequestBuilder.setQuery(querqyQuery); response = client().search(searchRequestBuilder.request()).get(); -// System.out.println(response); assertEquals(0L, response.getHits().getTotalHits().value); - assertEquals("{\"querqy\":{\"value\":[{}]}}", response.getAggregations().getAsMap().get(QuerqyAggregationBuilder.NAME).toString()); + assertEquals("{\"decorations\":{\"value\":[]}}", response.getAggregations().getAsMap().get(QuerqyAggregationBuilder.NAME).toString()); } @Test From b3ecd7473f930a2a71a1aa768cc63971ab8d6cb5 Mon Sep 17 00:00:00 2001 From: paulb Date: Tue, 20 Feb 2024 10:53:18 +0100 Subject: [PATCH 04/15] Refactoring: use generic Query, rename Querqy to DecorationAggregation --- .../querqy/elasticsearch/QuerqyPlugin.java | 12 ++--- .../querqy/elasticsearch/QuerqyProcessor.java | 3 +- .../aggregation/DecoratedQuery.java | 52 ++++++++++--------- ...Querqy.java => DecorationAggregation.java} | 4 +- ...ava => InternalDecorationAggregation.java} | 16 +++--- ....java => ParsedDecorationAggregation.java} | 16 +++--- ...> QuerqyDecorationAggregationBuilder.java} | 28 +++++----- ...r.java => QuerqyDecorationAggregator.java} | 13 ++--- ...=> QuerqyDecorationAggregatorFactory.java} | 16 +++--- .../RewriterIntegrationTest.java | 12 ++--- 10 files changed, 88 insertions(+), 84 deletions(-) rename src/main/java/querqy/elasticsearch/aggregation/{Querqy.java => DecorationAggregation.java} (58%) rename src/main/java/querqy/elasticsearch/aggregation/{InternalQuerqy.java => InternalDecorationAggregation.java} (83%) rename src/main/java/querqy/elasticsearch/aggregation/{ParsedQuerqy.java => ParsedDecorationAggregation.java} (80%) rename src/main/java/querqy/elasticsearch/aggregation/{QuerqyAggregationBuilder.java => QuerqyDecorationAggregationBuilder.java} (73%) rename src/main/java/querqy/elasticsearch/aggregation/{QuerqyAggregator.java => QuerqyDecorationAggregator.java} (78%) rename src/main/java/querqy/elasticsearch/aggregation/{QuerqyAggregatorFactory.java => QuerqyDecorationAggregatorFactory.java} (79%) diff --git a/src/main/java/querqy/elasticsearch/QuerqyPlugin.java b/src/main/java/querqy/elasticsearch/QuerqyPlugin.java index 9eeffa3..b8ce378 100644 --- a/src/main/java/querqy/elasticsearch/QuerqyPlugin.java +++ b/src/main/java/querqy/elasticsearch/QuerqyPlugin.java @@ -26,8 +26,8 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; import org.elasticsearch.xcontent.NamedXContentRegistry; -import querqy.elasticsearch.aggregation.InternalQuerqy; -import querqy.elasticsearch.aggregation.QuerqyAggregationBuilder; +import querqy.elasticsearch.aggregation.InternalDecorationAggregation; +import querqy.elasticsearch.aggregation.QuerqyDecorationAggregationBuilder; import querqy.elasticsearch.infologging.Log4jSink; import querqy.elasticsearch.query.QuerqyQueryBuilder; import querqy.elasticsearch.rewriterstore.DeleteRewriterAction; @@ -131,10 +131,10 @@ public List getAggregations() { List r = new ArrayList<>(); r.add( new AggregationSpec( - QuerqyAggregationBuilder.NAME, - QuerqyAggregationBuilder::new, - QuerqyAggregationBuilder.PARSER - ).addResultReader(InternalQuerqy::new) + QuerqyDecorationAggregationBuilder.NAME, + QuerqyDecorationAggregationBuilder::new, + QuerqyDecorationAggregationBuilder.PARSER + ).addResultReader(InternalDecorationAggregation::new) ); return r; } diff --git a/src/main/java/querqy/elasticsearch/QuerqyProcessor.java b/src/main/java/querqy/elasticsearch/QuerqyProcessor.java index c429337..1990d5f 100644 --- a/src/main/java/querqy/elasticsearch/QuerqyProcessor.java +++ b/src/main/java/querqy/elasticsearch/QuerqyProcessor.java @@ -18,7 +18,6 @@ import querqy.rewrite.RewriteChain; import querqy.rewrite.commonrules.model.DecorateInstruction; -import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Set; @@ -106,7 +105,7 @@ public Query parseQuery(final QuerqyQueryBuilder queryBuilder, final SearchExecu final Set decorations = (Set) requestAdapter.getContext().get(DecorateInstruction.DECORATION_CONTEXT_KEY); final Query query = decorations != null && !decorations.isEmpty() ? - DecoratedQuery.from(builder.build(), new ArrayList<>(decorations)) : + DecoratedQuery.from(builder.build(), decorations) : builder.build(); if (infoLogging != null) { infoLogging.endOfRequest(requestAdapter); diff --git a/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java b/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java index 340e020..8b674c9 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java +++ b/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java @@ -1,7 +1,6 @@ package querqy.elasticsearch.aggregation; import org.apache.lucene.index.IndexReader; -import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; import org.apache.lucene.search.QueryVisitor; @@ -9,65 +8,70 @@ import org.apache.lucene.search.Weight; import java.io.IOException; -import java.util.List; import java.util.Objects; +import java.util.Set; -public class DecoratedQuery extends Query { +public class DecoratedQuery extends Query { - final private BooleanQuery booleanQuery; - final private List decorations; + final private T query; + final private Set decorations; - private DecoratedQuery(BooleanQuery booleanQuery, List decorations) { - this.booleanQuery = Objects.requireNonNull(booleanQuery); + private DecoratedQuery(T query, Set decorations) { + this.query = Objects.requireNonNull(query); this.decorations = Objects.requireNonNull(decorations); } - public static Query from(BooleanQuery booleanQuery, List decorations) { - return new DecoratedQuery(booleanQuery, decorations); + public static DecoratedQuery from(T query, Set decorations) { + return new DecoratedQuery<>(query, decorations); } - public BooleanQuery getBooleanQuery() { - return booleanQuery; + public T getQuery() { + return query; } - public List getDecorations() { + public Set getDecorations() { return decorations; } @Override public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException { - return booleanQuery.createWeight(searcher, scoreMode, boost); + return query.createWeight(searcher, scoreMode, boost); } @Override public Query rewrite(IndexReader reader) throws IOException { - return booleanQuery.rewrite(reader); + return query.rewrite(reader); } @Override public void visit(QueryVisitor visitor) { - booleanQuery.visit(visitor); + query.visit(visitor); } @Override public String toString(String field) { - return booleanQuery.toString(); + return query.toString(); } @Override - public boolean equals(Object o) { - return sameClassAs(o) && - equalsTo(getClass().cast(o)); + public boolean equals(Object object) { + if (!sameClassAs(object)) return false; + DecoratedQuery other = castObject(object); + return isEqualQueriesAndDecorations(other); } - private boolean equalsTo(DecoratedQuery other) { - return booleanQuery.getMinimumNumberShouldMatch() == other.getBooleanQuery().getMinimumNumberShouldMatch() && - booleanQuery.clauses().equals(other.getBooleanQuery().clauses()) && - decorations.equals(other.getDecorations()); + private boolean isEqualQueriesAndDecorations(DecoratedQuery other) { + Query otherQuery = other.getQuery(); + Set otherDecorations = other.getDecorations(); + return getQuery().equals(otherQuery) && getDecorations().equals(otherDecorations); + } + + private DecoratedQuery castObject(Object object) { + return getClass().cast(object); } private int computeHashCode() { - int hashCode = Objects.hash(booleanQuery, decorations); + int hashCode = Objects.hash(query.hashCode(), decorations); if (hashCode == 0) { hashCode = 1; } diff --git a/src/main/java/querqy/elasticsearch/aggregation/Querqy.java b/src/main/java/querqy/elasticsearch/aggregation/DecorationAggregation.java similarity index 58% rename from src/main/java/querqy/elasticsearch/aggregation/Querqy.java rename to src/main/java/querqy/elasticsearch/aggregation/DecorationAggregation.java index dee6d24..e9c2f85 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/Querqy.java +++ b/src/main/java/querqy/elasticsearch/aggregation/DecorationAggregation.java @@ -3,9 +3,9 @@ import org.elasticsearch.search.aggregations.Aggregation; /** - * A {@code Querqy} aggregation. Defines a single bucket the holds all the querqy info in the search context. + * A {@code DecorationAggregation} aggregation. Defines a single bucket the holds all the querqy info in the search context. */ -public interface Querqy extends Aggregation { +public interface DecorationAggregation extends Aggregation { /** * The result of the aggregation. The type of the object depends on the aggregation that was run. diff --git a/src/main/java/querqy/elasticsearch/aggregation/InternalQuerqy.java b/src/main/java/querqy/elasticsearch/aggregation/InternalDecorationAggregation.java similarity index 83% rename from src/main/java/querqy/elasticsearch/aggregation/InternalQuerqy.java rename to src/main/java/querqy/elasticsearch/aggregation/InternalDecorationAggregation.java index 036a414..f5aa9b5 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/InternalQuerqy.java +++ b/src/main/java/querqy/elasticsearch/aggregation/InternalDecorationAggregation.java @@ -16,16 +16,16 @@ import static java.util.Collections.singletonList; -public class InternalQuerqy extends InternalAggregation implements ScriptedMetric { +public class InternalDecorationAggregation extends InternalAggregation implements ScriptedMetric { private final List aggregations; - InternalQuerqy(String name, List aggregations, Map metadata) { + InternalDecorationAggregation(String name, List aggregations, Map metadata) { super(name, metadata); this.aggregations = aggregations; } - public InternalQuerqy(StreamInput in) throws IOException { + public InternalDecorationAggregation(StreamInput in) throws IOException { super(in); if (in.getVersion().before(Version.V_7_8_0)) { aggregations = singletonList(in.readGenericValue()); @@ -55,7 +55,7 @@ protected void doWriteTo(StreamOutput out) throws IOException { @Override public String getWriteableName() { - return QuerqyAggregationBuilder.NAME; + return QuerqyDecorationAggregationBuilder.NAME; } @Override @@ -74,10 +74,10 @@ List aggregationsList() { public InternalAggregation reduce(List aggregations, ReduceContext reduceContext) { List aggregationObjects = new ArrayList<>(); for (InternalAggregation aggregation : aggregations) { - InternalQuerqy mapReduceAggregation = (InternalQuerqy) aggregation; + InternalDecorationAggregation mapReduceAggregation = (InternalDecorationAggregation) aggregation; aggregationObjects.addAll(mapReduceAggregation.aggregations); } - InternalQuerqy firstAggregation = ((InternalQuerqy) aggregations.get(0)); + InternalDecorationAggregation firstAggregation = ((InternalDecorationAggregation) aggregations.get(0)); List aggregation; if (reduceContext.isFinalReduce()) { aggregation = Collections.singletonList(aggregationObjects); @@ -86,7 +86,7 @@ public InternalAggregation reduce(List aggregations, Reduce // until we hit the final reduce phase. aggregation = aggregationObjects; } - return new InternalQuerqy(firstAggregation.getName(), aggregation, getMetadata()); + return new InternalDecorationAggregation(firstAggregation.getName(), aggregation, getMetadata()); } @Override @@ -116,7 +116,7 @@ public boolean equals(Object obj) { if (obj == null || getClass() != obj.getClass()) return false; if (super.equals(obj) == false) return false; - InternalQuerqy other = (InternalQuerqy) obj; + InternalDecorationAggregation other = (InternalDecorationAggregation) obj; return Objects.equals(aggregations, other.aggregations); } diff --git a/src/main/java/querqy/elasticsearch/aggregation/ParsedQuerqy.java b/src/main/java/querqy/elasticsearch/aggregation/ParsedDecorationAggregation.java similarity index 80% rename from src/main/java/querqy/elasticsearch/aggregation/ParsedQuerqy.java rename to src/main/java/querqy/elasticsearch/aggregation/ParsedDecorationAggregation.java index 2aa0520..185b5c7 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/ParsedQuerqy.java +++ b/src/main/java/querqy/elasticsearch/aggregation/ParsedDecorationAggregation.java @@ -10,13 +10,13 @@ import java.util.Collections; import java.util.List; -public class ParsedQuerqy extends ParsedAggregation implements Querqy { +public class ParsedDecorationAggregation extends ParsedAggregation implements DecorationAggregation { private List aggregation; @Override public String getType() { - return QuerqyAggregationBuilder.NAME; + return QuerqyDecorationAggregationBuilder.NAME; } @Override @@ -30,17 +30,17 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th return builder.field(CommonFields.VALUE.getPreferredName(), aggregation()); } - private static final ObjectParser PARSER = new ObjectParser<>( - ParsedQuerqy.class.getSimpleName(), + private static final ObjectParser PARSER = new ObjectParser<>( + ParsedDecorationAggregation.class.getSimpleName(), true, - ParsedQuerqy::new + ParsedDecorationAggregation::new ); static { declareAggregationFields(PARSER); PARSER.declareField( (agg, value) -> agg.aggregation = Collections.singletonList(value), - ParsedQuerqy::parseValue, + ParsedDecorationAggregation::parseValue, CommonFields.VALUE, ObjectParser.ValueType.VALUE_OBJECT_ARRAY ); @@ -71,8 +71,8 @@ private static Object parseValue(XContentParser parser) throws IOException { return value; } - public static ParsedQuerqy fromXContent(XContentParser parser, final String name) { - ParsedQuerqy aggregation = PARSER.apply(parser, null); + public static ParsedDecorationAggregation fromXContent(XContentParser parser, final String name) { + ParsedDecorationAggregation aggregation = PARSER.apply(parser, null); aggregation.setName(name); return aggregation; } diff --git a/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregationBuilder.java b/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationBuilder.java similarity index 73% rename from src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregationBuilder.java rename to src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationBuilder.java index df8faec..a77cc32 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregationBuilder.java +++ b/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationBuilder.java @@ -17,43 +17,43 @@ import java.util.Map; import java.util.Objects; -public class QuerqyAggregationBuilder extends AbstractAggregationBuilder { +public class QuerqyDecorationAggregationBuilder extends AbstractAggregationBuilder { public static final String NAME = "decorations"; private static final ParseField PARAMS_FIELD = new ParseField("params"); - public static final ObjectParser PARSER = - new ObjectParser<>(NAME, QuerqyAggregationBuilder::new); + public static final ObjectParser PARSER = + new ObjectParser<>(NAME, QuerqyDecorationAggregationBuilder::new); static { - PARSER.declareObject(QuerqyAggregationBuilder::params, (p, name) -> p.map(), PARAMS_FIELD); + PARSER.declareObject(QuerqyDecorationAggregationBuilder::params, (p, name) -> p.map(), PARAMS_FIELD); } private Map params; - public QuerqyAggregationBuilder() { + public QuerqyDecorationAggregationBuilder() { super(NAME); } - public QuerqyAggregationBuilder(String name) { + public QuerqyDecorationAggregationBuilder(String name) { super(name); } - protected QuerqyAggregationBuilder(QuerqyAggregationBuilder clone, Builder factoriesBuilder, Map metadata) { + protected QuerqyDecorationAggregationBuilder(QuerqyDecorationAggregationBuilder clone, Builder factoriesBuilder, Map metadata) { super(clone, factoriesBuilder, metadata); this.params = clone.params; } @Override protected AggregationBuilder shallowCopy(Builder factoriesBuilder, Map metadata) { - return new QuerqyAggregationBuilder(this, factoriesBuilder, metadata); + return new QuerqyDecorationAggregationBuilder(this, factoriesBuilder, metadata); } /** * Read from a stream. */ - public QuerqyAggregationBuilder(StreamInput in) throws IOException { + public QuerqyDecorationAggregationBuilder(StreamInput in) throws IOException { super(in); if (in.readBoolean()) { params = in.readMap(); @@ -64,7 +64,7 @@ public QuerqyAggregationBuilder(StreamInput in) throws IOException { * Set parameters that will be available in the {@code init}, * {@code map} and {@code combine} phases. */ - public QuerqyAggregationBuilder params(Map params) { + public QuerqyDecorationAggregationBuilder params(Map params) { if (params == null) { throw new IllegalArgumentException("[params] must not be null: [" + name + "]"); } @@ -88,7 +88,7 @@ public BucketCardinality bucketCardinality() { @Override protected AggregatorFactory doBuild(AggregationContext context, AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { - return new QuerqyAggregatorFactory(name, context, parent, subFactoriesBuilder, metadata); + return new QuerqyDecorationAggregatorFactory(name, context, parent, subFactoriesBuilder, metadata); } @Override @@ -111,8 +111,8 @@ protected void doWriteTo(StreamOutput out) throws IOException { } } - public static QuerqyAggregationBuilder fromXContent(final XContentParser parser) { - final QuerqyAggregationBuilder builder; + public static QuerqyDecorationAggregationBuilder fromXContent(final XContentParser parser) { + final QuerqyDecorationAggregationBuilder builder; try { builder = PARSER.apply(parser, null); } catch (final IllegalArgumentException e) { @@ -136,7 +136,7 @@ public boolean equals(Object obj) { if (this == obj) return true; if (obj == null || getClass() != obj.getClass()) return false; if (super.equals(obj) == false) return false; - QuerqyAggregationBuilder other = (QuerqyAggregationBuilder) obj; + QuerqyDecorationAggregationBuilder other = (QuerqyDecorationAggregationBuilder) obj; return Objects.equals(params, other.params); } diff --git a/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregator.java b/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregator.java similarity index 78% rename from src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregator.java rename to src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregator.java index a4e3696..3ebea0c 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregator.java +++ b/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregator.java @@ -9,14 +9,15 @@ import org.elasticsearch.search.aggregations.support.AggregationContext; import java.io.IOException; -import java.util.List; +import java.util.ArrayList; import java.util.Map; +import java.util.Set; -public final class QuerqyAggregator extends MetricsAggregator { +public final class QuerqyDecorationAggregator extends MetricsAggregator { - final List decorations; + final Set decorations; - public QuerqyAggregator(String name, AggregationContext context, Map metadata, List decorations) + public QuerqyDecorationAggregator(String name, AggregationContext context, Map metadata, Set decorations) throws IOException { super(name, context, null, metadata); this.decorations = decorations; @@ -36,9 +37,9 @@ protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucket @Override public InternalAggregation buildAggregation(long l) { StreamOutput.checkWriteable(decorations); - return new InternalQuerqy( + return new InternalDecorationAggregation( name, - decorations, + new ArrayList<>(decorations), metadata() ); } diff --git a/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregatorFactory.java b/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregatorFactory.java similarity index 79% rename from src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregatorFactory.java rename to src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregatorFactory.java index f0686fc..45bbbb3 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregatorFactory.java +++ b/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregatorFactory.java @@ -10,12 +10,12 @@ import java.io.IOException; import java.util.Collections; -import java.util.List; import java.util.Map; +import java.util.Set; -public class QuerqyAggregatorFactory extends AggregatorFactory { +public class QuerqyDecorationAggregatorFactory extends AggregatorFactory { - public QuerqyAggregatorFactory( + public QuerqyDecorationAggregatorFactory( String name, AggregationContext context, AggregatorFactory parent, @@ -42,10 +42,10 @@ public Aggregator createInternal(Aggregator parent, CardinalityUpperBound cardin throw new AggregationExecutionException("Aggregation [" + name() + "] must have cardinality 1 but was [" + cardinality + "]"); } Query query = context.subSearchContext() == null ? null : context.subSearchContext().query(); - List decorations = - query instanceof DecoratedQuery ? - ((DecoratedQuery) query).getDecorations() : - Collections.emptyList(); - return new QuerqyAggregator(name, context, metadata, decorations); + Set decorations = + (query instanceof DecoratedQuery) ? + ((DecoratedQuery) query).getDecorations() : + Collections.emptySet(); + return new QuerqyDecorationAggregator(name, context, metadata, decorations); } } diff --git a/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java b/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java index 190115d..588ec30 100644 --- a/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java +++ b/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java @@ -11,7 +11,7 @@ import org.elasticsearch.xcontent.XContentType; import org.junit.After; import org.junit.Test; -import querqy.elasticsearch.aggregation.QuerqyAggregationBuilder; +import querqy.elasticsearch.aggregation.QuerqyDecorationAggregationBuilder; import querqy.elasticsearch.query.MatchingQuery; import querqy.elasticsearch.query.QuerqyQueryBuilder; import querqy.elasticsearch.query.Rewriter; @@ -64,8 +64,8 @@ public void testSearchWithConfig() throws Exception { querqyQuery.setQueryFieldsAndBoostings(Arrays.asList("field1", "field2")); querqyQuery.setMinimumShouldMatch("1"); - QuerqyAggregationBuilder querqyAggregationBuilder = - QuerqyAggregationBuilder.fromXContent( + QuerqyDecorationAggregationBuilder querqyDecorationAggregationBuilder = + QuerqyDecorationAggregationBuilder.fromXContent( XContentHelper.createParser( null, null, @@ -79,17 +79,17 @@ public void testSearchWithConfig() throws Exception { SearchRequestBuilder searchRequestBuilder = client().prepareSearch(INDEX_NAME); searchRequestBuilder.setQuery(querqyQuery); - searchRequestBuilder.addAggregation(querqyAggregationBuilder); + searchRequestBuilder.addAggregation(querqyDecorationAggregationBuilder); SearchResponse response = client().search(searchRequestBuilder.request()).get(); assertEquals(2L, response.getHits().getTotalHits().value); - assertEquals("{\"decorations\":{\"value\":[\"REDIRECT /faq/a\"]}}", response.getAggregations().getAsMap().get(QuerqyAggregationBuilder.NAME).toString()); + assertEquals("{\"decorations\":{\"value\":[\"REDIRECT /faq/a\"]}}", response.getAggregations().getAsMap().get(QuerqyDecorationAggregationBuilder.NAME).toString()); querqyQuery.setMatchingQuery(new MatchingQuery("x z")); searchRequestBuilder.setQuery(querqyQuery); response = client().search(searchRequestBuilder.request()).get(); assertEquals(0L, response.getHits().getTotalHits().value); - assertEquals("{\"decorations\":{\"value\":[]}}", response.getAggregations().getAsMap().get(QuerqyAggregationBuilder.NAME).toString()); + assertEquals("{\"decorations\":{\"value\":[]}}", response.getAggregations().getAsMap().get(QuerqyDecorationAggregationBuilder.NAME).toString()); } @Test From 53079d2f4fb657057e70ee0930e59ce3b62321e3 Mon Sep 17 00:00:00 2001 From: paulb Date: Thu, 8 Feb 2024 14:09:55 +0100 Subject: [PATCH 05/15] Suggestion implementation QuerqyAggregationBuilder --- .../querqy/elasticsearch/QuerqyPlugin.java | 31 +++- .../querqy/elasticsearch/QuerqyProcessor.java | 42 ++++- .../aggregation/InternalQuerqy.java | 128 +++++++++++++ .../aggregation/ParsedQuerqy.java | 80 ++++++++ .../elasticsearch/aggregation/Querqy.java | 15 ++ .../aggregation/QuerqyAggregationBuilder.java | 173 ++++++++++++++++++ .../aggregation/QuerqyAggregator.java | 52 ++++++ .../aggregation/QuerqyAggregatorFactory.java | 58 ++++++ .../RewriterIntegrationTest.java | 55 ++++-- 9 files changed, 609 insertions(+), 25 deletions(-) create mode 100644 src/main/java/querqy/elasticsearch/aggregation/InternalQuerqy.java create mode 100644 src/main/java/querqy/elasticsearch/aggregation/ParsedQuerqy.java create mode 100644 src/main/java/querqy/elasticsearch/aggregation/Querqy.java create mode 100644 src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregationBuilder.java create mode 100644 src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregator.java create mode 100644 src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregatorFactory.java diff --git a/src/main/java/querqy/elasticsearch/QuerqyPlugin.java b/src/main/java/querqy/elasticsearch/QuerqyPlugin.java index d5bf8a4..d517df0 100644 --- a/src/main/java/querqy/elasticsearch/QuerqyPlugin.java +++ b/src/main/java/querqy/elasticsearch/QuerqyPlugin.java @@ -1,9 +1,5 @@ package querqy.elasticsearch; -import static java.util.Arrays.asList; -import static java.util.Collections.unmodifiableList; -import static querqy.elasticsearch.rewriterstore.Constants.SETTINGS_QUERQY_INDEX_NUM_REPLICAS; - import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.client.Client; @@ -16,7 +12,6 @@ import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.SettingsFilter; -import org.elasticsearch.xcontent.NamedXContentRegistry; import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.IndexModule; @@ -30,25 +25,33 @@ import org.elasticsearch.script.ScriptService; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; +import org.elasticsearch.xcontent.NamedXContentRegistry; +import querqy.elasticsearch.aggregation.InternalQuerqy; +import querqy.elasticsearch.aggregation.QuerqyAggregationBuilder; import querqy.elasticsearch.infologging.Log4jSink; import querqy.elasticsearch.query.QuerqyQueryBuilder; import querqy.elasticsearch.rewriterstore.DeleteRewriterAction; import querqy.elasticsearch.rewriterstore.NodesClearRewriterCacheAction; import querqy.elasticsearch.rewriterstore.NodesReloadRewriterAction; +import querqy.elasticsearch.rewriterstore.PutRewriterAction; import querqy.elasticsearch.rewriterstore.RestDeleteRewriterAction; import querqy.elasticsearch.rewriterstore.RestPutRewriterAction; -import querqy.elasticsearch.rewriterstore.PutRewriterAction; import querqy.elasticsearch.rewriterstore.TransportDeleteRewriterAction; import querqy.elasticsearch.rewriterstore.TransportNodesClearRewriterCacheAction; import querqy.elasticsearch.rewriterstore.TransportNodesReloadRewriterAction; import querqy.elasticsearch.rewriterstore.TransportPutRewriterAction; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.function.Supplier; +import static java.util.Arrays.asList; +import static java.util.Collections.unmodifiableList; +import static querqy.elasticsearch.rewriterstore.Constants.SETTINGS_QUERQY_INDEX_NUM_REPLICAS; + public class QuerqyPlugin extends Plugin implements SearchPlugin, ActionPlugin { @@ -57,7 +60,7 @@ public class QuerqyPlugin extends Plugin implements SearchPlugin, ActionPlugin { public QuerqyPlugin(final Settings settings) { rewriterShardContexts = new RewriterShardContexts(settings); - querqyProcessor = new QuerqyProcessor(rewriterShardContexts, new Log4jSink()); + querqyProcessor = new QuerqyProcessor(rewriterShardContexts, new Log4jSink(), settings); } @Override @@ -122,4 +125,18 @@ public List> getSettings() { Setting.Property.NodeScope)); } + + @Override + public ArrayList getAggregations() { + ArrayList r = new ArrayList<>(); + r.add( + new AggregationSpec( + QuerqyAggregationBuilder.NAME, + (in) -> new QuerqyAggregationBuilder(in, querqyProcessor), + QuerqyAggregationBuilder.PARSER + ).addResultReader(InternalQuerqy::new) + ); + return r; + } + } diff --git a/src/main/java/querqy/elasticsearch/QuerqyProcessor.java b/src/main/java/querqy/elasticsearch/QuerqyProcessor.java index a81d94d..1bb910c 100644 --- a/src/main/java/querqy/elasticsearch/QuerqyProcessor.java +++ b/src/main/java/querqy/elasticsearch/QuerqyProcessor.java @@ -3,6 +3,10 @@ import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.Query; +import org.elasticsearch.common.cache.Cache; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.query.SearchExecutionContext; import querqy.elasticsearch.infologging.LogPayloadType; import querqy.elasticsearch.infologging.SingleSinkInfoLogging; @@ -15,23 +19,46 @@ import querqy.lucene.LuceneSearchEngineRequestAdapter; import querqy.lucene.QueryParsingController; import querqy.rewrite.RewriteChain; +import querqy.rewrite.commonrules.model.DecorateInstruction; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.stream.Collectors; public class QuerqyProcessor { + public static final Setting CACHE_DECORATIONS_EXPIRE_AFTER_WRITE = Setting.timeSetting( + "querqy.caches.decorations.expire_after_write", + TimeValue.timeValueSeconds(10L), // 10s + TimeValue.timeValueSeconds(10L), + Setting.Property.NodeScope); + + public static final Setting CACHE_DECORATIONS_EXPIRE_AFTER_READ = Setting.timeSetting( + "querqy.caches.decorations.expire_after_read", + TimeValue.timeValueSeconds(10L), // 10ns + TimeValue.timeValueSeconds(10L), + Setting.Property.NodeScope); + private static final RewriteChain EMPTY_REWRITE_CHAIN = new RewriteChain(Collections.emptyList()); + //private final Map> querqyQueryCache; + private final Cache> querqyQueryCache; + private RewriterShardContexts rewriterShardContexts; private Sink infoLoggingSink; public QuerqyProcessor(final RewriterShardContexts rewriterShardContexts, final Sink infoLoggingSink) { + this(rewriterShardContexts, infoLoggingSink, Settings.EMPTY); + } + + public QuerqyProcessor(final RewriterShardContexts rewriterShardContexts, final Sink infoLoggingSink, final Settings settings) { this.rewriterShardContexts = rewriterShardContexts; this.infoLoggingSink = infoLoggingSink; + //querqyInfoCache = new HashMap<>(); + querqyQueryCache = Caches.buildCache(CACHE_DECORATIONS_EXPIRE_AFTER_READ.get(settings), CACHE_DECORATIONS_EXPIRE_AFTER_WRITE.get(settings)); } public Query parseQuery(final QuerqyQueryBuilder queryBuilder, final SearchExecutionContext context) @@ -75,10 +102,6 @@ public Query parseQuery(final QuerqyQueryBuilder queryBuilder, final SearchExecu final QueryParsingController controller = new QueryParsingController(requestAdapter); final LuceneQueries queries = controller.process(); - -// // TODO: make decos part of the general Querqy object model -// final Set decorations = (Set) requestAdapter.getContext().get(DecorateInstruction.CONTEXT_KEY); - if ((queries.querqyBoostQueries == null || queries.querqyBoostQueries.isEmpty()) && (queries.filterQueries == null || queries.filterQueries.isEmpty()) && queries.mainQuery instanceof BooleanQuery) { @@ -108,10 +131,21 @@ public Query parseQuery(final QuerqyQueryBuilder queryBuilder, final SearchExecu if (infoLogging != null) { infoLogging.endOfRequest(requestAdapter); } + + final Set decorations = (Set) requestAdapter.getContext().get(DecorateInstruction.DECORATION_CONTEXT_KEY); + if (decorations != null) { + querqyQueryCache.put(query, Collections.singletonMap("decorations", decorations)); + } + return query; } + public Map getQuerqyInfoForQuery(Query query) { + final Map querqyInfo = querqyQueryCache.get(query); + querqyQueryCache.invalidate(query); + return querqyInfo; + } void appendFilterQueries(final LuceneQueries queries, final BooleanQuery.Builder builder) { diff --git a/src/main/java/querqy/elasticsearch/aggregation/InternalQuerqy.java b/src/main/java/querqy/elasticsearch/aggregation/InternalQuerqy.java new file mode 100644 index 0000000..036a414 --- /dev/null +++ b/src/main/java/querqy/elasticsearch/aggregation/InternalQuerqy.java @@ -0,0 +1,128 @@ +package querqy.elasticsearch.aggregation; + +import org.elasticsearch.Version; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.metrics.ScriptedMetric; +import org.elasticsearch.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import static java.util.Collections.singletonList; + +public class InternalQuerqy extends InternalAggregation implements ScriptedMetric { + + private final List aggregations; + + InternalQuerqy(String name, List aggregations, Map metadata) { + super(name, metadata); + this.aggregations = aggregations; + } + + public InternalQuerqy(StreamInput in) throws IOException { + super(in); + if (in.getVersion().before(Version.V_7_8_0)) { + aggregations = singletonList(in.readGenericValue()); + } else { + aggregations = in.readList(StreamInput::readGenericValue); + } + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException { + if (out.getVersion().before(Version.V_7_8_0)) { + if (aggregations.size() > 1) { + /* + * If aggregations has more than one entry we're trying to + * serialize an unreduced aggregation. This *should* only + * happen when we're returning a scripted_metric over cross + * cluster search. + */ + throw new IllegalArgumentException("querqy doesn't support cross cluster search until 7.8.0"); + } + out.writeGenericValue(aggregations.get(0)); + } else { + out.writeCollection(aggregations, StreamOutput::writeGenericValue); + } + + } + + @Override + public String getWriteableName() { + return QuerqyAggregationBuilder.NAME; + } + + @Override + public Object aggregation() { + if (aggregations.size() != 1) { + throw new IllegalStateException("aggregation was not reduced"); + } + return aggregations.get(0); + } + + List aggregationsList() { + return aggregations; + } + + @Override + public InternalAggregation reduce(List aggregations, ReduceContext reduceContext) { + List aggregationObjects = new ArrayList<>(); + for (InternalAggregation aggregation : aggregations) { + InternalQuerqy mapReduceAggregation = (InternalQuerqy) aggregation; + aggregationObjects.addAll(mapReduceAggregation.aggregations); + } + InternalQuerqy firstAggregation = ((InternalQuerqy) aggregations.get(0)); + List aggregation; + if (reduceContext.isFinalReduce()) { + aggregation = Collections.singletonList(aggregationObjects); + } else { + // if we are not an final reduce we have to maintain all the aggs from all the incoming one + // until we hit the final reduce phase. + aggregation = aggregationObjects; + } + return new InternalQuerqy(firstAggregation.getName(), aggregation, getMetadata()); + } + + @Override + protected boolean mustReduceOnSingleInternalAgg() { + return true; + } + + @Override + public Object getProperty(List path) { + if (path.isEmpty()) { + return this; + } else if (path.size() == 1 && "value".equals(path.get(0))) { + return aggregation(); + } else { + throw new IllegalArgumentException("path not supported for [" + getName() + "]: " + path); + } + } + + @Override + public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + return builder.field(CommonFields.VALUE.getPreferredName(), aggregation()); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + + InternalQuerqy other = (InternalQuerqy) obj; + return Objects.equals(aggregations, other.aggregations); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), aggregations); + } + +} diff --git a/src/main/java/querqy/elasticsearch/aggregation/ParsedQuerqy.java b/src/main/java/querqy/elasticsearch/aggregation/ParsedQuerqy.java new file mode 100644 index 0000000..2aa0520 --- /dev/null +++ b/src/main/java/querqy/elasticsearch/aggregation/ParsedQuerqy.java @@ -0,0 +1,80 @@ +package querqy.elasticsearch.aggregation; + +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.search.aggregations.ParsedAggregation; +import org.elasticsearch.xcontent.ObjectParser; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.XContentParser; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +public class ParsedQuerqy extends ParsedAggregation implements Querqy { + + private List aggregation; + + @Override + public String getType() { + return QuerqyAggregationBuilder.NAME; + } + + @Override + public Object aggregation() { + assert aggregation.size() == 1; + return aggregation.get(0); + } + + @Override + public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + return builder.field(CommonFields.VALUE.getPreferredName(), aggregation()); + } + + private static final ObjectParser PARSER = new ObjectParser<>( + ParsedQuerqy.class.getSimpleName(), + true, + ParsedQuerqy::new + ); + + static { + declareAggregationFields(PARSER); + PARSER.declareField( + (agg, value) -> agg.aggregation = Collections.singletonList(value), + ParsedQuerqy::parseValue, + CommonFields.VALUE, + ObjectParser.ValueType.VALUE_OBJECT_ARRAY + ); + } + + private static Object parseValue(XContentParser parser) throws IOException { + XContentParser.Token token = parser.currentToken(); + Object value = null; + if (token == XContentParser.Token.VALUE_NULL) { + value = null; + } else if (token.isValue()) { + if (token == XContentParser.Token.VALUE_STRING) { + // binary values will be parsed back and returned as base64 strings when reading from json and yaml + value = parser.text(); + } else if (token == XContentParser.Token.VALUE_NUMBER) { + value = parser.numberValue(); + } else if (token == XContentParser.Token.VALUE_BOOLEAN) { + value = parser.booleanValue(); + } else if (token == XContentParser.Token.VALUE_EMBEDDED_OBJECT) { + // binary values will be parsed back and returned as BytesArray when reading from cbor and smile + value = new BytesArray(parser.binaryValue()); + } + } else if (token == XContentParser.Token.START_OBJECT) { + value = parser.map(); + } else if (token == XContentParser.Token.START_ARRAY) { + value = parser.list(); + } + return value; + } + + public static ParsedQuerqy fromXContent(XContentParser parser, final String name) { + ParsedQuerqy aggregation = PARSER.apply(parser, null); + aggregation.setName(name); + return aggregation; + } + +} diff --git a/src/main/java/querqy/elasticsearch/aggregation/Querqy.java b/src/main/java/querqy/elasticsearch/aggregation/Querqy.java new file mode 100644 index 0000000..dee6d24 --- /dev/null +++ b/src/main/java/querqy/elasticsearch/aggregation/Querqy.java @@ -0,0 +1,15 @@ +package querqy.elasticsearch.aggregation; + +import org.elasticsearch.search.aggregations.Aggregation; + +/** + * A {@code Querqy} aggregation. Defines a single bucket the holds all the querqy info in the search context. + */ +public interface Querqy extends Aggregation { + + /** + * The result of the aggregation. The type of the object depends on the aggregation that was run. + */ + Object aggregation(); + +} diff --git a/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregationBuilder.java b/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregationBuilder.java new file mode 100644 index 0000000..28a15bd --- /dev/null +++ b/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregationBuilder.java @@ -0,0 +1,173 @@ +package querqy.elasticsearch.aggregation; + +import org.elasticsearch.common.ParsingException; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.search.aggregations.AbstractAggregationBuilder; +import org.elasticsearch.search.aggregations.AggregationBuilder; +import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; +import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.support.AggregationContext; +import org.elasticsearch.xcontent.ConstructingObjectParser; +import org.elasticsearch.xcontent.ObjectParser; +import org.elasticsearch.xcontent.ParseField; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.XContentParser; +import querqy.elasticsearch.QuerqyProcessor; +import querqy.elasticsearch.query.QuerqyQueryBuilder; +import querqy.elasticsearch.query.Rewriter; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; + +import static querqy.elasticsearch.query.RequestUtils.fieldBoostModelToString; + +public class QuerqyAggregationBuilder extends AbstractAggregationBuilder { + + public static final String NAME = "querqy"; + + private static final ParseField PARAMS_FIELD = new ParseField("params"); + + private QuerqyProcessor querqyProcessor; + + public static final ObjectParser PARSER = + new ObjectParser<>(NAME, QuerqyAggregationBuilder::new); + + static { + PARSER.declareObject(QuerqyAggregationBuilder::params, (p, name) -> p.map(), PARAMS_FIELD); + } + + private Map params; + + public QuerqyAggregationBuilder() { + super(NAME); + } + + public QuerqyAggregationBuilder(String name) { + super(name); + } + + public QuerqyAggregationBuilder(QuerqyProcessor querqyProcessor) { + super(NAME); + setQuerqyProcessor(querqyProcessor); + } + + protected QuerqyAggregationBuilder(QuerqyAggregationBuilder clone, Builder factoriesBuilder, Map metadata) { + super(clone, factoriesBuilder, metadata); + this.params = clone.params; + } + + public QuerqyAggregationBuilder(StreamInput in) throws IOException { + super(in); + if (in.readBoolean()) { + params = in.readMap(); + } + } + + @Override + protected AggregationBuilder shallowCopy(Builder factoriesBuilder, Map metadata) { + return new QuerqyAggregationBuilder(this, factoriesBuilder, metadata); + } + + /** + * Read from a stream. + */ + public QuerqyAggregationBuilder(StreamInput in, QuerqyProcessor querqyProcessor) throws IOException { + super(in); + if (in.readBoolean()) { + params = in.readMap(); + } + setQuerqyProcessor(querqyProcessor); + } + + public QuerqyAggregationBuilder setQuerqyProcessor(final QuerqyProcessor querqyProcessor) { + this.querqyProcessor = Objects.requireNonNull(querqyProcessor); + return this; + } + + /** + * Set parameters that will be available in the {@code init}, + * {@code map} and {@code combine} phases. + */ + public QuerqyAggregationBuilder params(Map params) { + if (params == null) { + throw new IllegalArgumentException("[params] must not be null: [" + name + "]"); + } + this.params = params; + return this; + } + + /** + * Get parameters that will be available in the {@code init}, + * {@code map} and {@code combine} phases. + */ + public Map params() { + return params; + } + + @Override + public BucketCardinality bucketCardinality() { + return BucketCardinality.NONE; + } + + @Override + protected AggregatorFactory doBuild(AggregationContext context, AggregatorFactory parent, Builder subFactoriesBuilder) + throws IOException { + return new QuerqyAggregatorFactory(name, context, parent, subFactoriesBuilder, metadata, querqyProcessor); + } + + @Override + protected XContentBuilder internalXContent(XContentBuilder builder, Params builderParams) throws IOException { + builder.startObject(); + if (params != null) { + builder.field(PARAMS_FIELD.getPreferredName()); + builder.map(params); + } + builder.endObject(); + return builder; + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException { + boolean hasParams = params != null; + out.writeBoolean(hasParams); + if (hasParams) { + out.writeMap(params); + } + } + + public static QuerqyAggregationBuilder fromXContent(final XContentParser parser, final QuerqyProcessor querqyProcessor) { + final QuerqyAggregationBuilder builder; + try { + builder = PARSER.apply(parser, null); + } catch (final IllegalArgumentException e) { + throw new ParsingException(parser.getTokenLocation(), e.getMessage(), e); + } + builder.setQuerqyProcessor(querqyProcessor); + return builder; + } + + @Override + public String getType() { + return NAME; + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), params); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + QuerqyAggregationBuilder other = (QuerqyAggregationBuilder) obj; + return Objects.equals(params, other.params); + } + +} diff --git a/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregator.java b/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregator.java new file mode 100644 index 0000000..31601fc --- /dev/null +++ b/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregator.java @@ -0,0 +1,52 @@ +package querqy.elasticsearch.aggregation; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.ScoreMode; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.LeafBucketCollector; +import org.elasticsearch.search.aggregations.metrics.MetricsAggregator; +import org.elasticsearch.search.aggregations.support.AggregationContext; + +import java.io.IOException; +import java.util.Collections; +import java.util.Map; + +public final class QuerqyAggregator extends MetricsAggregator { + + public QuerqyAggregator(String name, AggregationContext context, Map metadata) + throws IOException { + super(name, context, null, metadata); + } + + @Override + public ScoreMode scoreMode() { + return ScoreMode.COMPLETE; + } + + @Override + protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException { + // No sub-aggregations + return LeafBucketCollector.NO_OP_COLLECTOR; + } + + @Override + public InternalAggregation buildAggregation(long l) { + Map metadata = metadata(); + Map result = metadata != null && !metadata.isEmpty() ? metadata : Collections.emptyMap(); + StreamOutput.checkWriteable(result); + return new InternalQuerqy( + name, + Collections.singletonList(result), + null + ); + } + + @Override + public InternalAggregation buildEmptyAggregation() { + throw new UnsupportedOperationException( + "querqy_decoration aggregations cannot serve as sub-aggregations, hence should never be called on #buildEmptyAggregations" + ); + } + +} diff --git a/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregatorFactory.java b/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregatorFactory.java new file mode 100644 index 0000000..3168d06 --- /dev/null +++ b/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregatorFactory.java @@ -0,0 +1,58 @@ +package querqy.elasticsearch.aggregation; + +import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.Aggregator; +import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.CardinalityUpperBound; +import org.elasticsearch.search.aggregations.support.AggregationContext; +import querqy.elasticsearch.QuerqyProcessor; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +public class QuerqyAggregatorFactory extends AggregatorFactory { + + final private QuerqyProcessor querqyProcessor; + + public QuerqyAggregatorFactory( + String name, + AggregationContext context, + AggregatorFactory parent, + AggregatorFactories.Builder subFactories, + Map metadata, + QuerqyProcessor querqyProcessor + ) throws IOException { + super(name, context, parent, subFactories, metadata); + this.querqyProcessor = Objects.requireNonNull(querqyProcessor); + } + + @Override + public Aggregator createInternal(Aggregator parent, CardinalityUpperBound cardinality, Map metadata) + throws IOException { + if (parent != null) { + throw new IllegalArgumentException( + "Aggregation [" + + parent.name() + + "] cannot have a querqy_decoration " + + "sub-aggregation [" + + name + + "]. querqy_decoration aggregations can only be defined as top level aggregations" + ); + } + if (cardinality != CardinalityUpperBound.ONE) { + throw new AggregationExecutionException("Aggregation [" + name() + "] must have cardinality 1 but was [" + cardinality + "]"); + } + Map info = querqyProcessor.getQuerqyInfoForQuery(context.subSearchContext().query()); + if (info != null && !info.isEmpty()) { + if (metadata == null) { + metadata = new HashMap<>(); + } + metadata.putAll(info); + } + return new QuerqyAggregator(name, context, metadata); + } +} diff --git a/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java b/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java index 5394c34..9f75133 100644 --- a/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java +++ b/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java @@ -1,23 +1,17 @@ package querqy.elasticsearch; -import java.io.BufferedReader; -import java.io.InputStreamReader; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.ExecutionException; -import java.util.stream.Collectors; - import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.ESSingleNodeTestCase; +import org.elasticsearch.xcontent.XContentType; import org.junit.After; import org.junit.Test; +import querqy.elasticsearch.aggregation.QuerqyAggregationBuilder; import querqy.elasticsearch.query.MatchingQuery; import querqy.elasticsearch.query.QuerqyQueryBuilder; import querqy.elasticsearch.query.Rewriter; @@ -27,6 +21,16 @@ import querqy.elasticsearch.rewriterstore.PutRewriterRequest; import querqy.elasticsearch.rewriterstore.PutRewriterResponse; +import java.io.BufferedReader; +import java.io.InputStreamReader; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; + public class RewriterIntegrationTest extends ESSingleNodeTestCase { private final String INDEX_NAME = "test_index"; @@ -44,7 +48,7 @@ public void testSearchWithConfig() throws Exception { content.put("class", querqy.elasticsearch.rewriter.SimpleCommonRulesRewriterFactory.class.getName()); final Map config = new HashMap<>(); - config.put("rules", "k =>\nSYNONYM: c"); + config.put("rules", "k =>\nSYNONYM: c\na =>\nDECORATE: REDIRECT /faq/a"); config.put("ignoreCase", true); config.put("querqyParser", querqy.rewrite.commonrules.WhiteSpaceQuerqyParserFactory.class.getName()); content.put("config", config); @@ -53,18 +57,41 @@ public void testSearchWithConfig() throws Exception { client().execute(PutRewriterAction.INSTANCE, request).get(); - QuerqyQueryBuilder querqyQuery = new QuerqyQueryBuilder(getInstanceFromNode(QuerqyProcessor.class)); + QuerqyProcessor querqyProcessor = getInstanceFromNode(QuerqyProcessor.class); + QuerqyQueryBuilder querqyQuery = new QuerqyQueryBuilder(querqyProcessor); querqyQuery.setRewriters(Collections.singletonList(new Rewriter("common_rules"))); querqyQuery.setMatchingQuery(new MatchingQuery("a k")); querqyQuery.setQueryFieldsAndBoostings(Arrays.asList("field1", "field2")); querqyQuery.setMinimumShouldMatch("1"); - final SearchRequestBuilder searchRequestBuilder = client().prepareSearch(INDEX_NAME); + QuerqyAggregationBuilder querqyAggregationBuilder = + QuerqyAggregationBuilder.fromXContent( + XContentHelper.createParser( + null, + null, + new BytesArray("{" + + "\"params\": {" + + "\"test-query\": \"a k\"" + + "}}" + ), + XContentType.JSON), querqyProcessor + ); + + SearchRequestBuilder searchRequestBuilder = client().prepareSearch(INDEX_NAME); searchRequestBuilder.setQuery(querqyQuery); - + searchRequestBuilder.addAggregation(querqyAggregationBuilder); SearchResponse response = client().search(searchRequestBuilder.request()).get(); +// System.out.println(response); assertEquals(2L, response.getHits().getTotalHits().value); + assertEquals("{\"querqy\":{\"value\":[{\"decorations\":[\"REDIRECT /faq/a\"]}]}}", response.getAggregations().getAsMap().get(QuerqyAggregationBuilder.NAME).toString()); + + querqyQuery.setMatchingQuery(new MatchingQuery("x z")); + searchRequestBuilder.setQuery(querqyQuery); + response = client().search(searchRequestBuilder.request()).get(); +// System.out.println(response); + assertEquals(0L, response.getHits().getTotalHits().value); + assertEquals("{\"querqy\":{\"value\":[{}]}}", response.getAggregations().getAsMap().get(QuerqyAggregationBuilder.NAME).toString()); } @Test From 31f38c57e514ca3f716ad4960f6a60cd007d8e86 Mon Sep 17 00:00:00 2001 From: paulb Date: Mon, 19 Feb 2024 13:09:36 +0100 Subject: [PATCH 06/15] Suggested implementation via wrapped query --- .../querqy/elasticsearch/QuerqyPlugin.java | 8 +- .../querqy/elasticsearch/QuerqyProcessor.java | 45 ++------- .../aggregation/DecoratedQuery.java | 91 +++++++++++++++++++ .../aggregation/QuerqyAggregationBuilder.java | 38 +------- .../aggregation/QuerqyAggregator.java | 15 +-- .../aggregation/QuerqyAggregatorFactory.java | 25 ++--- .../RewriterIntegrationTest.java | 8 +- 7 files changed, 126 insertions(+), 104 deletions(-) create mode 100644 src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java diff --git a/src/main/java/querqy/elasticsearch/QuerqyPlugin.java b/src/main/java/querqy/elasticsearch/QuerqyPlugin.java index d517df0..9eeffa3 100644 --- a/src/main/java/querqy/elasticsearch/QuerqyPlugin.java +++ b/src/main/java/querqy/elasticsearch/QuerqyPlugin.java @@ -60,7 +60,7 @@ public class QuerqyPlugin extends Plugin implements SearchPlugin, ActionPlugin { public QuerqyPlugin(final Settings settings) { rewriterShardContexts = new RewriterShardContexts(settings); - querqyProcessor = new QuerqyProcessor(rewriterShardContexts, new Log4jSink(), settings); + querqyProcessor = new QuerqyProcessor(rewriterShardContexts, new Log4jSink()); } @Override @@ -127,12 +127,12 @@ public List> getSettings() { } @Override - public ArrayList getAggregations() { - ArrayList r = new ArrayList<>(); + public List getAggregations() { + List r = new ArrayList<>(); r.add( new AggregationSpec( QuerqyAggregationBuilder.NAME, - (in) -> new QuerqyAggregationBuilder(in, querqyProcessor), + QuerqyAggregationBuilder::new, QuerqyAggregationBuilder.PARSER ).addResultReader(InternalQuerqy::new) ); diff --git a/src/main/java/querqy/elasticsearch/QuerqyProcessor.java b/src/main/java/querqy/elasticsearch/QuerqyProcessor.java index 1bb910c..c429337 100644 --- a/src/main/java/querqy/elasticsearch/QuerqyProcessor.java +++ b/src/main/java/querqy/elasticsearch/QuerqyProcessor.java @@ -3,11 +3,8 @@ import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.Query; -import org.elasticsearch.common.cache.Cache; -import org.elasticsearch.common.settings.Setting; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.query.SearchExecutionContext; +import querqy.elasticsearch.aggregation.DecoratedQuery; import querqy.elasticsearch.infologging.LogPayloadType; import querqy.elasticsearch.infologging.SingleSinkInfoLogging; import querqy.elasticsearch.query.InfoLoggingSpec; @@ -21,44 +18,23 @@ import querqy.rewrite.RewriteChain; import querqy.rewrite.commonrules.model.DecorateInstruction; +import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Map; import java.util.Set; import java.util.stream.Collectors; public class QuerqyProcessor { - public static final Setting CACHE_DECORATIONS_EXPIRE_AFTER_WRITE = Setting.timeSetting( - "querqy.caches.decorations.expire_after_write", - TimeValue.timeValueSeconds(10L), // 10s - TimeValue.timeValueSeconds(10L), - Setting.Property.NodeScope); - - public static final Setting CACHE_DECORATIONS_EXPIRE_AFTER_READ = Setting.timeSetting( - "querqy.caches.decorations.expire_after_read", - TimeValue.timeValueSeconds(10L), // 10ns - TimeValue.timeValueSeconds(10L), - Setting.Property.NodeScope); - private static final RewriteChain EMPTY_REWRITE_CHAIN = new RewriteChain(Collections.emptyList()); - //private final Map> querqyQueryCache; - private final Cache> querqyQueryCache; - private RewriterShardContexts rewriterShardContexts; private Sink infoLoggingSink; public QuerqyProcessor(final RewriterShardContexts rewriterShardContexts, final Sink infoLoggingSink) { - this(rewriterShardContexts, infoLoggingSink, Settings.EMPTY); - } - - public QuerqyProcessor(final RewriterShardContexts rewriterShardContexts, final Sink infoLoggingSink, final Settings settings) { this.rewriterShardContexts = rewriterShardContexts; this.infoLoggingSink = infoLoggingSink; - //querqyInfoCache = new HashMap<>(); - querqyQueryCache = Caches.buildCache(CACHE_DECORATIONS_EXPIRE_AFTER_READ.get(settings), CACHE_DECORATIONS_EXPIRE_AFTER_WRITE.get(settings)); } public Query parseQuery(final QuerqyQueryBuilder queryBuilder, final SearchExecutionContext context) @@ -127,26 +103,19 @@ public Query parseQuery(final QuerqyQueryBuilder queryBuilder, final SearchExecu appendFilterQueries(queries, builder); - final BooleanQuery query = builder.build(); + final Set decorations = (Set) requestAdapter.getContext().get(DecorateInstruction.DECORATION_CONTEXT_KEY); + final Query query = + decorations != null && !decorations.isEmpty() ? + DecoratedQuery.from(builder.build(), new ArrayList<>(decorations)) : + builder.build(); if (infoLogging != null) { infoLogging.endOfRequest(requestAdapter); } - final Set decorations = (Set) requestAdapter.getContext().get(DecorateInstruction.DECORATION_CONTEXT_KEY); - if (decorations != null) { - querqyQueryCache.put(query, Collections.singletonMap("decorations", decorations)); - } - return query; } - public Map getQuerqyInfoForQuery(Query query) { - final Map querqyInfo = querqyQueryCache.get(query); - querqyQueryCache.invalidate(query); - return querqyInfo; - } - void appendFilterQueries(final LuceneQueries queries, final BooleanQuery.Builder builder) { if (queries.filterQueries != null) { diff --git a/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java b/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java new file mode 100644 index 0000000..340e020 --- /dev/null +++ b/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java @@ -0,0 +1,91 @@ +package querqy.elasticsearch.aggregation; + +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.QueryVisitor; +import org.apache.lucene.search.ScoreMode; +import org.apache.lucene.search.Weight; + +import java.io.IOException; +import java.util.List; +import java.util.Objects; + +public class DecoratedQuery extends Query { + + final private BooleanQuery booleanQuery; + final private List decorations; + + private DecoratedQuery(BooleanQuery booleanQuery, List decorations) { + this.booleanQuery = Objects.requireNonNull(booleanQuery); + this.decorations = Objects.requireNonNull(decorations); + } + + public static Query from(BooleanQuery booleanQuery, List decorations) { + return new DecoratedQuery(booleanQuery, decorations); + } + + public BooleanQuery getBooleanQuery() { + return booleanQuery; + } + + public List getDecorations() { + return decorations; + } + + @Override + public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException { + return booleanQuery.createWeight(searcher, scoreMode, boost); + } + + @Override + public Query rewrite(IndexReader reader) throws IOException { + return booleanQuery.rewrite(reader); + } + + @Override + public void visit(QueryVisitor visitor) { + booleanQuery.visit(visitor); + } + + @Override + public String toString(String field) { + return booleanQuery.toString(); + } + + @Override + public boolean equals(Object o) { + return sameClassAs(o) && + equalsTo(getClass().cast(o)); + } + + private boolean equalsTo(DecoratedQuery other) { + return booleanQuery.getMinimumNumberShouldMatch() == other.getBooleanQuery().getMinimumNumberShouldMatch() && + booleanQuery.clauses().equals(other.getBooleanQuery().clauses()) && + decorations.equals(other.getDecorations()); + } + + private int computeHashCode() { + int hashCode = Objects.hash(booleanQuery, decorations); + if (hashCode == 0) { + hashCode = 1; + } + return hashCode; + } + + // cached hash code is ok since boolean queries are immutable + private int hashCode; + + @Override + public int hashCode() { + // no need for synchronization, in the worst case we would just compute the hash several times. + if (hashCode == 0) { + hashCode = computeHashCode(); + assert hashCode != 0; + } + assert hashCode == computeHashCode(); + return hashCode; + } + +} \ No newline at end of file diff --git a/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregationBuilder.java b/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregationBuilder.java index 28a15bd..df8faec 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregationBuilder.java +++ b/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregationBuilder.java @@ -8,32 +8,21 @@ import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.support.AggregationContext; -import org.elasticsearch.xcontent.ConstructingObjectParser; import org.elasticsearch.xcontent.ObjectParser; import org.elasticsearch.xcontent.ParseField; import org.elasticsearch.xcontent.XContentBuilder; import org.elasticsearch.xcontent.XContentParser; -import querqy.elasticsearch.QuerqyProcessor; -import querqy.elasticsearch.query.QuerqyQueryBuilder; -import querqy.elasticsearch.query.Rewriter; import java.io.IOException; -import java.util.Collections; -import java.util.HashMap; import java.util.Map; import java.util.Objects; -import java.util.Optional; - -import static querqy.elasticsearch.query.RequestUtils.fieldBoostModelToString; public class QuerqyAggregationBuilder extends AbstractAggregationBuilder { - public static final String NAME = "querqy"; + public static final String NAME = "decorations"; private static final ParseField PARAMS_FIELD = new ParseField("params"); - private QuerqyProcessor querqyProcessor; - public static final ObjectParser PARSER = new ObjectParser<>(NAME, QuerqyAggregationBuilder::new); @@ -51,23 +40,11 @@ public QuerqyAggregationBuilder(String name) { super(name); } - public QuerqyAggregationBuilder(QuerqyProcessor querqyProcessor) { - super(NAME); - setQuerqyProcessor(querqyProcessor); - } - protected QuerqyAggregationBuilder(QuerqyAggregationBuilder clone, Builder factoriesBuilder, Map metadata) { super(clone, factoriesBuilder, metadata); this.params = clone.params; } - public QuerqyAggregationBuilder(StreamInput in) throws IOException { - super(in); - if (in.readBoolean()) { - params = in.readMap(); - } - } - @Override protected AggregationBuilder shallowCopy(Builder factoriesBuilder, Map metadata) { return new QuerqyAggregationBuilder(this, factoriesBuilder, metadata); @@ -76,17 +53,11 @@ protected AggregationBuilder shallowCopy(Builder factoriesBuilder, Map metadata) + final List decorations; + + public QuerqyAggregator(String name, AggregationContext context, Map metadata, List decorations) throws IOException { super(name, context, null, metadata); + this.decorations = decorations; } @Override @@ -32,13 +35,11 @@ protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucket @Override public InternalAggregation buildAggregation(long l) { - Map metadata = metadata(); - Map result = metadata != null && !metadata.isEmpty() ? metadata : Collections.emptyMap(); - StreamOutput.checkWriteable(result); + StreamOutput.checkWriteable(decorations); return new InternalQuerqy( name, - Collections.singletonList(result), - null + decorations, + metadata() ); } diff --git a/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregatorFactory.java b/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregatorFactory.java index 3168d06..f0686fc 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregatorFactory.java +++ b/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregatorFactory.java @@ -1,33 +1,28 @@ package querqy.elasticsearch.aggregation; +import org.apache.lucene.search.Query; import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.CardinalityUpperBound; import org.elasticsearch.search.aggregations.support.AggregationContext; -import querqy.elasticsearch.QuerqyProcessor; import java.io.IOException; import java.util.Collections; -import java.util.HashMap; +import java.util.List; import java.util.Map; -import java.util.Objects; public class QuerqyAggregatorFactory extends AggregatorFactory { - final private QuerqyProcessor querqyProcessor; - public QuerqyAggregatorFactory( String name, AggregationContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactories, - Map metadata, - QuerqyProcessor querqyProcessor + Map metadata ) throws IOException { super(name, context, parent, subFactories, metadata); - this.querqyProcessor = Objects.requireNonNull(querqyProcessor); } @Override @@ -46,13 +41,11 @@ public Aggregator createInternal(Aggregator parent, CardinalityUpperBound cardin if (cardinality != CardinalityUpperBound.ONE) { throw new AggregationExecutionException("Aggregation [" + name() + "] must have cardinality 1 but was [" + cardinality + "]"); } - Map info = querqyProcessor.getQuerqyInfoForQuery(context.subSearchContext().query()); - if (info != null && !info.isEmpty()) { - if (metadata == null) { - metadata = new HashMap<>(); - } - metadata.putAll(info); - } - return new QuerqyAggregator(name, context, metadata); + Query query = context.subSearchContext() == null ? null : context.subSearchContext().query(); + List decorations = + query instanceof DecoratedQuery ? + ((DecoratedQuery) query).getDecorations() : + Collections.emptyList(); + return new QuerqyAggregator(name, context, metadata, decorations); } } diff --git a/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java b/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java index 9f75133..190115d 100644 --- a/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java +++ b/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java @@ -74,24 +74,22 @@ public void testSearchWithConfig() throws Exception { "\"test-query\": \"a k\"" + "}}" ), - XContentType.JSON), querqyProcessor + XContentType.JSON) ); SearchRequestBuilder searchRequestBuilder = client().prepareSearch(INDEX_NAME); searchRequestBuilder.setQuery(querqyQuery); searchRequestBuilder.addAggregation(querqyAggregationBuilder); SearchResponse response = client().search(searchRequestBuilder.request()).get(); -// System.out.println(response); assertEquals(2L, response.getHits().getTotalHits().value); - assertEquals("{\"querqy\":{\"value\":[{\"decorations\":[\"REDIRECT /faq/a\"]}]}}", response.getAggregations().getAsMap().get(QuerqyAggregationBuilder.NAME).toString()); + assertEquals("{\"decorations\":{\"value\":[\"REDIRECT /faq/a\"]}}", response.getAggregations().getAsMap().get(QuerqyAggregationBuilder.NAME).toString()); querqyQuery.setMatchingQuery(new MatchingQuery("x z")); searchRequestBuilder.setQuery(querqyQuery); response = client().search(searchRequestBuilder.request()).get(); -// System.out.println(response); assertEquals(0L, response.getHits().getTotalHits().value); - assertEquals("{\"querqy\":{\"value\":[{}]}}", response.getAggregations().getAsMap().get(QuerqyAggregationBuilder.NAME).toString()); + assertEquals("{\"decorations\":{\"value\":[]}}", response.getAggregations().getAsMap().get(QuerqyAggregationBuilder.NAME).toString()); } @Test From 31f2ea4d81669e1da9553f8416af5ad0d0a87246 Mon Sep 17 00:00:00 2001 From: paulb Date: Tue, 20 Feb 2024 10:53:18 +0100 Subject: [PATCH 07/15] Refactoring: use generic Query, rename Querqy to DecorationAggregation --- .../querqy/elasticsearch/QuerqyPlugin.java | 12 ++--- .../querqy/elasticsearch/QuerqyProcessor.java | 3 +- .../aggregation/DecoratedQuery.java | 52 ++++++++++--------- ...Querqy.java => DecorationAggregation.java} | 4 +- ...ava => InternalDecorationAggregation.java} | 16 +++--- ....java => ParsedDecorationAggregation.java} | 16 +++--- ...> QuerqyDecorationAggregationBuilder.java} | 28 +++++----- ...r.java => QuerqyDecorationAggregator.java} | 13 ++--- ...=> QuerqyDecorationAggregatorFactory.java} | 16 +++--- .../RewriterIntegrationTest.java | 12 ++--- 10 files changed, 88 insertions(+), 84 deletions(-) rename src/main/java/querqy/elasticsearch/aggregation/{Querqy.java => DecorationAggregation.java} (58%) rename src/main/java/querqy/elasticsearch/aggregation/{InternalQuerqy.java => InternalDecorationAggregation.java} (83%) rename src/main/java/querqy/elasticsearch/aggregation/{ParsedQuerqy.java => ParsedDecorationAggregation.java} (80%) rename src/main/java/querqy/elasticsearch/aggregation/{QuerqyAggregationBuilder.java => QuerqyDecorationAggregationBuilder.java} (73%) rename src/main/java/querqy/elasticsearch/aggregation/{QuerqyAggregator.java => QuerqyDecorationAggregator.java} (78%) rename src/main/java/querqy/elasticsearch/aggregation/{QuerqyAggregatorFactory.java => QuerqyDecorationAggregatorFactory.java} (79%) diff --git a/src/main/java/querqy/elasticsearch/QuerqyPlugin.java b/src/main/java/querqy/elasticsearch/QuerqyPlugin.java index 9eeffa3..b8ce378 100644 --- a/src/main/java/querqy/elasticsearch/QuerqyPlugin.java +++ b/src/main/java/querqy/elasticsearch/QuerqyPlugin.java @@ -26,8 +26,8 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; import org.elasticsearch.xcontent.NamedXContentRegistry; -import querqy.elasticsearch.aggregation.InternalQuerqy; -import querqy.elasticsearch.aggregation.QuerqyAggregationBuilder; +import querqy.elasticsearch.aggregation.InternalDecorationAggregation; +import querqy.elasticsearch.aggregation.QuerqyDecorationAggregationBuilder; import querqy.elasticsearch.infologging.Log4jSink; import querqy.elasticsearch.query.QuerqyQueryBuilder; import querqy.elasticsearch.rewriterstore.DeleteRewriterAction; @@ -131,10 +131,10 @@ public List getAggregations() { List r = new ArrayList<>(); r.add( new AggregationSpec( - QuerqyAggregationBuilder.NAME, - QuerqyAggregationBuilder::new, - QuerqyAggregationBuilder.PARSER - ).addResultReader(InternalQuerqy::new) + QuerqyDecorationAggregationBuilder.NAME, + QuerqyDecorationAggregationBuilder::new, + QuerqyDecorationAggregationBuilder.PARSER + ).addResultReader(InternalDecorationAggregation::new) ); return r; } diff --git a/src/main/java/querqy/elasticsearch/QuerqyProcessor.java b/src/main/java/querqy/elasticsearch/QuerqyProcessor.java index c429337..1990d5f 100644 --- a/src/main/java/querqy/elasticsearch/QuerqyProcessor.java +++ b/src/main/java/querqy/elasticsearch/QuerqyProcessor.java @@ -18,7 +18,6 @@ import querqy.rewrite.RewriteChain; import querqy.rewrite.commonrules.model.DecorateInstruction; -import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Set; @@ -106,7 +105,7 @@ public Query parseQuery(final QuerqyQueryBuilder queryBuilder, final SearchExecu final Set decorations = (Set) requestAdapter.getContext().get(DecorateInstruction.DECORATION_CONTEXT_KEY); final Query query = decorations != null && !decorations.isEmpty() ? - DecoratedQuery.from(builder.build(), new ArrayList<>(decorations)) : + DecoratedQuery.from(builder.build(), decorations) : builder.build(); if (infoLogging != null) { infoLogging.endOfRequest(requestAdapter); diff --git a/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java b/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java index 340e020..8b674c9 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java +++ b/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java @@ -1,7 +1,6 @@ package querqy.elasticsearch.aggregation; import org.apache.lucene.index.IndexReader; -import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; import org.apache.lucene.search.QueryVisitor; @@ -9,65 +8,70 @@ import org.apache.lucene.search.Weight; import java.io.IOException; -import java.util.List; import java.util.Objects; +import java.util.Set; -public class DecoratedQuery extends Query { +public class DecoratedQuery extends Query { - final private BooleanQuery booleanQuery; - final private List decorations; + final private T query; + final private Set decorations; - private DecoratedQuery(BooleanQuery booleanQuery, List decorations) { - this.booleanQuery = Objects.requireNonNull(booleanQuery); + private DecoratedQuery(T query, Set decorations) { + this.query = Objects.requireNonNull(query); this.decorations = Objects.requireNonNull(decorations); } - public static Query from(BooleanQuery booleanQuery, List decorations) { - return new DecoratedQuery(booleanQuery, decorations); + public static DecoratedQuery from(T query, Set decorations) { + return new DecoratedQuery<>(query, decorations); } - public BooleanQuery getBooleanQuery() { - return booleanQuery; + public T getQuery() { + return query; } - public List getDecorations() { + public Set getDecorations() { return decorations; } @Override public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException { - return booleanQuery.createWeight(searcher, scoreMode, boost); + return query.createWeight(searcher, scoreMode, boost); } @Override public Query rewrite(IndexReader reader) throws IOException { - return booleanQuery.rewrite(reader); + return query.rewrite(reader); } @Override public void visit(QueryVisitor visitor) { - booleanQuery.visit(visitor); + query.visit(visitor); } @Override public String toString(String field) { - return booleanQuery.toString(); + return query.toString(); } @Override - public boolean equals(Object o) { - return sameClassAs(o) && - equalsTo(getClass().cast(o)); + public boolean equals(Object object) { + if (!sameClassAs(object)) return false; + DecoratedQuery other = castObject(object); + return isEqualQueriesAndDecorations(other); } - private boolean equalsTo(DecoratedQuery other) { - return booleanQuery.getMinimumNumberShouldMatch() == other.getBooleanQuery().getMinimumNumberShouldMatch() && - booleanQuery.clauses().equals(other.getBooleanQuery().clauses()) && - decorations.equals(other.getDecorations()); + private boolean isEqualQueriesAndDecorations(DecoratedQuery other) { + Query otherQuery = other.getQuery(); + Set otherDecorations = other.getDecorations(); + return getQuery().equals(otherQuery) && getDecorations().equals(otherDecorations); + } + + private DecoratedQuery castObject(Object object) { + return getClass().cast(object); } private int computeHashCode() { - int hashCode = Objects.hash(booleanQuery, decorations); + int hashCode = Objects.hash(query.hashCode(), decorations); if (hashCode == 0) { hashCode = 1; } diff --git a/src/main/java/querqy/elasticsearch/aggregation/Querqy.java b/src/main/java/querqy/elasticsearch/aggregation/DecorationAggregation.java similarity index 58% rename from src/main/java/querqy/elasticsearch/aggregation/Querqy.java rename to src/main/java/querqy/elasticsearch/aggregation/DecorationAggregation.java index dee6d24..e9c2f85 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/Querqy.java +++ b/src/main/java/querqy/elasticsearch/aggregation/DecorationAggregation.java @@ -3,9 +3,9 @@ import org.elasticsearch.search.aggregations.Aggregation; /** - * A {@code Querqy} aggregation. Defines a single bucket the holds all the querqy info in the search context. + * A {@code DecorationAggregation} aggregation. Defines a single bucket the holds all the querqy info in the search context. */ -public interface Querqy extends Aggregation { +public interface DecorationAggregation extends Aggregation { /** * The result of the aggregation. The type of the object depends on the aggregation that was run. diff --git a/src/main/java/querqy/elasticsearch/aggregation/InternalQuerqy.java b/src/main/java/querqy/elasticsearch/aggregation/InternalDecorationAggregation.java similarity index 83% rename from src/main/java/querqy/elasticsearch/aggregation/InternalQuerqy.java rename to src/main/java/querqy/elasticsearch/aggregation/InternalDecorationAggregation.java index 036a414..f5aa9b5 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/InternalQuerqy.java +++ b/src/main/java/querqy/elasticsearch/aggregation/InternalDecorationAggregation.java @@ -16,16 +16,16 @@ import static java.util.Collections.singletonList; -public class InternalQuerqy extends InternalAggregation implements ScriptedMetric { +public class InternalDecorationAggregation extends InternalAggregation implements ScriptedMetric { private final List aggregations; - InternalQuerqy(String name, List aggregations, Map metadata) { + InternalDecorationAggregation(String name, List aggregations, Map metadata) { super(name, metadata); this.aggregations = aggregations; } - public InternalQuerqy(StreamInput in) throws IOException { + public InternalDecorationAggregation(StreamInput in) throws IOException { super(in); if (in.getVersion().before(Version.V_7_8_0)) { aggregations = singletonList(in.readGenericValue()); @@ -55,7 +55,7 @@ protected void doWriteTo(StreamOutput out) throws IOException { @Override public String getWriteableName() { - return QuerqyAggregationBuilder.NAME; + return QuerqyDecorationAggregationBuilder.NAME; } @Override @@ -74,10 +74,10 @@ List aggregationsList() { public InternalAggregation reduce(List aggregations, ReduceContext reduceContext) { List aggregationObjects = new ArrayList<>(); for (InternalAggregation aggregation : aggregations) { - InternalQuerqy mapReduceAggregation = (InternalQuerqy) aggregation; + InternalDecorationAggregation mapReduceAggregation = (InternalDecorationAggregation) aggregation; aggregationObjects.addAll(mapReduceAggregation.aggregations); } - InternalQuerqy firstAggregation = ((InternalQuerqy) aggregations.get(0)); + InternalDecorationAggregation firstAggregation = ((InternalDecorationAggregation) aggregations.get(0)); List aggregation; if (reduceContext.isFinalReduce()) { aggregation = Collections.singletonList(aggregationObjects); @@ -86,7 +86,7 @@ public InternalAggregation reduce(List aggregations, Reduce // until we hit the final reduce phase. aggregation = aggregationObjects; } - return new InternalQuerqy(firstAggregation.getName(), aggregation, getMetadata()); + return new InternalDecorationAggregation(firstAggregation.getName(), aggregation, getMetadata()); } @Override @@ -116,7 +116,7 @@ public boolean equals(Object obj) { if (obj == null || getClass() != obj.getClass()) return false; if (super.equals(obj) == false) return false; - InternalQuerqy other = (InternalQuerqy) obj; + InternalDecorationAggregation other = (InternalDecorationAggregation) obj; return Objects.equals(aggregations, other.aggregations); } diff --git a/src/main/java/querqy/elasticsearch/aggregation/ParsedQuerqy.java b/src/main/java/querqy/elasticsearch/aggregation/ParsedDecorationAggregation.java similarity index 80% rename from src/main/java/querqy/elasticsearch/aggregation/ParsedQuerqy.java rename to src/main/java/querqy/elasticsearch/aggregation/ParsedDecorationAggregation.java index 2aa0520..185b5c7 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/ParsedQuerqy.java +++ b/src/main/java/querqy/elasticsearch/aggregation/ParsedDecorationAggregation.java @@ -10,13 +10,13 @@ import java.util.Collections; import java.util.List; -public class ParsedQuerqy extends ParsedAggregation implements Querqy { +public class ParsedDecorationAggregation extends ParsedAggregation implements DecorationAggregation { private List aggregation; @Override public String getType() { - return QuerqyAggregationBuilder.NAME; + return QuerqyDecorationAggregationBuilder.NAME; } @Override @@ -30,17 +30,17 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th return builder.field(CommonFields.VALUE.getPreferredName(), aggregation()); } - private static final ObjectParser PARSER = new ObjectParser<>( - ParsedQuerqy.class.getSimpleName(), + private static final ObjectParser PARSER = new ObjectParser<>( + ParsedDecorationAggregation.class.getSimpleName(), true, - ParsedQuerqy::new + ParsedDecorationAggregation::new ); static { declareAggregationFields(PARSER); PARSER.declareField( (agg, value) -> agg.aggregation = Collections.singletonList(value), - ParsedQuerqy::parseValue, + ParsedDecorationAggregation::parseValue, CommonFields.VALUE, ObjectParser.ValueType.VALUE_OBJECT_ARRAY ); @@ -71,8 +71,8 @@ private static Object parseValue(XContentParser parser) throws IOException { return value; } - public static ParsedQuerqy fromXContent(XContentParser parser, final String name) { - ParsedQuerqy aggregation = PARSER.apply(parser, null); + public static ParsedDecorationAggregation fromXContent(XContentParser parser, final String name) { + ParsedDecorationAggregation aggregation = PARSER.apply(parser, null); aggregation.setName(name); return aggregation; } diff --git a/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregationBuilder.java b/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationBuilder.java similarity index 73% rename from src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregationBuilder.java rename to src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationBuilder.java index df8faec..a77cc32 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregationBuilder.java +++ b/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationBuilder.java @@ -17,43 +17,43 @@ import java.util.Map; import java.util.Objects; -public class QuerqyAggregationBuilder extends AbstractAggregationBuilder { +public class QuerqyDecorationAggregationBuilder extends AbstractAggregationBuilder { public static final String NAME = "decorations"; private static final ParseField PARAMS_FIELD = new ParseField("params"); - public static final ObjectParser PARSER = - new ObjectParser<>(NAME, QuerqyAggregationBuilder::new); + public static final ObjectParser PARSER = + new ObjectParser<>(NAME, QuerqyDecorationAggregationBuilder::new); static { - PARSER.declareObject(QuerqyAggregationBuilder::params, (p, name) -> p.map(), PARAMS_FIELD); + PARSER.declareObject(QuerqyDecorationAggregationBuilder::params, (p, name) -> p.map(), PARAMS_FIELD); } private Map params; - public QuerqyAggregationBuilder() { + public QuerqyDecorationAggregationBuilder() { super(NAME); } - public QuerqyAggregationBuilder(String name) { + public QuerqyDecorationAggregationBuilder(String name) { super(name); } - protected QuerqyAggregationBuilder(QuerqyAggregationBuilder clone, Builder factoriesBuilder, Map metadata) { + protected QuerqyDecorationAggregationBuilder(QuerqyDecorationAggregationBuilder clone, Builder factoriesBuilder, Map metadata) { super(clone, factoriesBuilder, metadata); this.params = clone.params; } @Override protected AggregationBuilder shallowCopy(Builder factoriesBuilder, Map metadata) { - return new QuerqyAggregationBuilder(this, factoriesBuilder, metadata); + return new QuerqyDecorationAggregationBuilder(this, factoriesBuilder, metadata); } /** * Read from a stream. */ - public QuerqyAggregationBuilder(StreamInput in) throws IOException { + public QuerqyDecorationAggregationBuilder(StreamInput in) throws IOException { super(in); if (in.readBoolean()) { params = in.readMap(); @@ -64,7 +64,7 @@ public QuerqyAggregationBuilder(StreamInput in) throws IOException { * Set parameters that will be available in the {@code init}, * {@code map} and {@code combine} phases. */ - public QuerqyAggregationBuilder params(Map params) { + public QuerqyDecorationAggregationBuilder params(Map params) { if (params == null) { throw new IllegalArgumentException("[params] must not be null: [" + name + "]"); } @@ -88,7 +88,7 @@ public BucketCardinality bucketCardinality() { @Override protected AggregatorFactory doBuild(AggregationContext context, AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { - return new QuerqyAggregatorFactory(name, context, parent, subFactoriesBuilder, metadata); + return new QuerqyDecorationAggregatorFactory(name, context, parent, subFactoriesBuilder, metadata); } @Override @@ -111,8 +111,8 @@ protected void doWriteTo(StreamOutput out) throws IOException { } } - public static QuerqyAggregationBuilder fromXContent(final XContentParser parser) { - final QuerqyAggregationBuilder builder; + public static QuerqyDecorationAggregationBuilder fromXContent(final XContentParser parser) { + final QuerqyDecorationAggregationBuilder builder; try { builder = PARSER.apply(parser, null); } catch (final IllegalArgumentException e) { @@ -136,7 +136,7 @@ public boolean equals(Object obj) { if (this == obj) return true; if (obj == null || getClass() != obj.getClass()) return false; if (super.equals(obj) == false) return false; - QuerqyAggregationBuilder other = (QuerqyAggregationBuilder) obj; + QuerqyDecorationAggregationBuilder other = (QuerqyDecorationAggregationBuilder) obj; return Objects.equals(params, other.params); } diff --git a/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregator.java b/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregator.java similarity index 78% rename from src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregator.java rename to src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregator.java index a4e3696..3ebea0c 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregator.java +++ b/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregator.java @@ -9,14 +9,15 @@ import org.elasticsearch.search.aggregations.support.AggregationContext; import java.io.IOException; -import java.util.List; +import java.util.ArrayList; import java.util.Map; +import java.util.Set; -public final class QuerqyAggregator extends MetricsAggregator { +public final class QuerqyDecorationAggregator extends MetricsAggregator { - final List decorations; + final Set decorations; - public QuerqyAggregator(String name, AggregationContext context, Map metadata, List decorations) + public QuerqyDecorationAggregator(String name, AggregationContext context, Map metadata, Set decorations) throws IOException { super(name, context, null, metadata); this.decorations = decorations; @@ -36,9 +37,9 @@ protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucket @Override public InternalAggregation buildAggregation(long l) { StreamOutput.checkWriteable(decorations); - return new InternalQuerqy( + return new InternalDecorationAggregation( name, - decorations, + new ArrayList<>(decorations), metadata() ); } diff --git a/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregatorFactory.java b/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregatorFactory.java similarity index 79% rename from src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregatorFactory.java rename to src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregatorFactory.java index f0686fc..45bbbb3 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/QuerqyAggregatorFactory.java +++ b/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregatorFactory.java @@ -10,12 +10,12 @@ import java.io.IOException; import java.util.Collections; -import java.util.List; import java.util.Map; +import java.util.Set; -public class QuerqyAggregatorFactory extends AggregatorFactory { +public class QuerqyDecorationAggregatorFactory extends AggregatorFactory { - public QuerqyAggregatorFactory( + public QuerqyDecorationAggregatorFactory( String name, AggregationContext context, AggregatorFactory parent, @@ -42,10 +42,10 @@ public Aggregator createInternal(Aggregator parent, CardinalityUpperBound cardin throw new AggregationExecutionException("Aggregation [" + name() + "] must have cardinality 1 but was [" + cardinality + "]"); } Query query = context.subSearchContext() == null ? null : context.subSearchContext().query(); - List decorations = - query instanceof DecoratedQuery ? - ((DecoratedQuery) query).getDecorations() : - Collections.emptyList(); - return new QuerqyAggregator(name, context, metadata, decorations); + Set decorations = + (query instanceof DecoratedQuery) ? + ((DecoratedQuery) query).getDecorations() : + Collections.emptySet(); + return new QuerqyDecorationAggregator(name, context, metadata, decorations); } } diff --git a/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java b/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java index 190115d..588ec30 100644 --- a/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java +++ b/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java @@ -11,7 +11,7 @@ import org.elasticsearch.xcontent.XContentType; import org.junit.After; import org.junit.Test; -import querqy.elasticsearch.aggregation.QuerqyAggregationBuilder; +import querqy.elasticsearch.aggregation.QuerqyDecorationAggregationBuilder; import querqy.elasticsearch.query.MatchingQuery; import querqy.elasticsearch.query.QuerqyQueryBuilder; import querqy.elasticsearch.query.Rewriter; @@ -64,8 +64,8 @@ public void testSearchWithConfig() throws Exception { querqyQuery.setQueryFieldsAndBoostings(Arrays.asList("field1", "field2")); querqyQuery.setMinimumShouldMatch("1"); - QuerqyAggregationBuilder querqyAggregationBuilder = - QuerqyAggregationBuilder.fromXContent( + QuerqyDecorationAggregationBuilder querqyDecorationAggregationBuilder = + QuerqyDecorationAggregationBuilder.fromXContent( XContentHelper.createParser( null, null, @@ -79,17 +79,17 @@ public void testSearchWithConfig() throws Exception { SearchRequestBuilder searchRequestBuilder = client().prepareSearch(INDEX_NAME); searchRequestBuilder.setQuery(querqyQuery); - searchRequestBuilder.addAggregation(querqyAggregationBuilder); + searchRequestBuilder.addAggregation(querqyDecorationAggregationBuilder); SearchResponse response = client().search(searchRequestBuilder.request()).get(); assertEquals(2L, response.getHits().getTotalHits().value); - assertEquals("{\"decorations\":{\"value\":[\"REDIRECT /faq/a\"]}}", response.getAggregations().getAsMap().get(QuerqyAggregationBuilder.NAME).toString()); + assertEquals("{\"decorations\":{\"value\":[\"REDIRECT /faq/a\"]}}", response.getAggregations().getAsMap().get(QuerqyDecorationAggregationBuilder.NAME).toString()); querqyQuery.setMatchingQuery(new MatchingQuery("x z")); searchRequestBuilder.setQuery(querqyQuery); response = client().search(searchRequestBuilder.request()).get(); assertEquals(0L, response.getHits().getTotalHits().value); - assertEquals("{\"decorations\":{\"value\":[]}}", response.getAggregations().getAsMap().get(QuerqyAggregationBuilder.NAME).toString()); + assertEquals("{\"decorations\":{\"value\":[]}}", response.getAggregations().getAsMap().get(QuerqyDecorationAggregationBuilder.NAME).toString()); } @Test From 6fe8ae147e3077f8a5ee4189087f19fce2f8a837 Mon Sep 17 00:00:00 2001 From: paulb Date: Wed, 21 Feb 2024 14:53:34 +0100 Subject: [PATCH 08/15] Refactoring --- .../querqy/elasticsearch/QuerqyProcessor.java | 2 +- .../aggregation/DecoratedQuery.java | 12 +- .../InternalDecorationAggregation.java | 1 - .../QuerqyDecorationAggregationBuilder.java | 76 +----- .../RewriterIntegrationTest.java | 33 +-- .../InternalDecorationAggregationTests.java | 242 ++++++++++++++++++ ...yDecorationAggregationIntegrationTest.java | 88 +++++++ 7 files changed, 344 insertions(+), 110 deletions(-) create mode 100644 src/test/java/querqy/elasticsearch/aggregation/InternalDecorationAggregationTests.java create mode 100644 src/test/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationIntegrationTest.java diff --git a/src/main/java/querqy/elasticsearch/QuerqyProcessor.java b/src/main/java/querqy/elasticsearch/QuerqyProcessor.java index 1990d5f..32573bd 100644 --- a/src/main/java/querqy/elasticsearch/QuerqyProcessor.java +++ b/src/main/java/querqy/elasticsearch/QuerqyProcessor.java @@ -105,7 +105,7 @@ public Query parseQuery(final QuerqyQueryBuilder queryBuilder, final SearchExecu final Set decorations = (Set) requestAdapter.getContext().get(DecorateInstruction.DECORATION_CONTEXT_KEY); final Query query = decorations != null && !decorations.isEmpty() ? - DecoratedQuery.from(builder.build(), decorations) : + new DecoratedQuery<>(builder.build(), decorations) : builder.build(); if (infoLogging != null) { infoLogging.endOfRequest(requestAdapter); diff --git a/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java b/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java index 8b674c9..6395c6f 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java +++ b/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java @@ -16,15 +16,11 @@ public class DecoratedQuery extends Query { final private T query; final private Set decorations; - private DecoratedQuery(T query, Set decorations) { + public DecoratedQuery(T query, Set decorations) { this.query = Objects.requireNonNull(query); this.decorations = Objects.requireNonNull(decorations); } - public static DecoratedQuery from(T query, Set decorations) { - return new DecoratedQuery<>(query, decorations); - } - public T getQuery() { return query; } @@ -50,7 +46,7 @@ public void visit(QueryVisitor visitor) { @Override public String toString(String field) { - return query.toString(); + return query.toString(field); } @Override @@ -71,7 +67,7 @@ private DecoratedQuery castObject(Object object) { } private int computeHashCode() { - int hashCode = Objects.hash(query.hashCode(), decorations); + int hashCode = Objects.hash(query, decorations); if (hashCode == 0) { hashCode = 1; } @@ -86,9 +82,7 @@ public int hashCode() { // no need for synchronization, in the worst case we would just compute the hash several times. if (hashCode == 0) { hashCode = computeHashCode(); - assert hashCode != 0; } - assert hashCode == computeHashCode(); return hashCode; } diff --git a/src/main/java/querqy/elasticsearch/aggregation/InternalDecorationAggregation.java b/src/main/java/querqy/elasticsearch/aggregation/InternalDecorationAggregation.java index f5aa9b5..41e3c36 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/InternalDecorationAggregation.java +++ b/src/main/java/querqy/elasticsearch/aggregation/InternalDecorationAggregation.java @@ -50,7 +50,6 @@ protected void doWriteTo(StreamOutput out) throws IOException { } else { out.writeCollection(aggregations, StreamOutput::writeGenericValue); } - } @Override diff --git a/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationBuilder.java b/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationBuilder.java index a77cc32..4caf217 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationBuilder.java +++ b/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationBuilder.java @@ -1,6 +1,5 @@ package querqy.elasticsearch.aggregation; -import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.search.aggregations.AbstractAggregationBuilder; @@ -9,9 +8,7 @@ import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.xcontent.ObjectParser; -import org.elasticsearch.xcontent.ParseField; import org.elasticsearch.xcontent.XContentBuilder; -import org.elasticsearch.xcontent.XContentParser; import java.io.IOException; import java.util.Map; @@ -21,28 +18,19 @@ public class QuerqyDecorationAggregationBuilder extends AbstractAggregationBuild public static final String NAME = "decorations"; - private static final ParseField PARAMS_FIELD = new ParseField("params"); - public static final ObjectParser PARSER = new ObjectParser<>(NAME, QuerqyDecorationAggregationBuilder::new); - static { - PARSER.declareObject(QuerqyDecorationAggregationBuilder::params, (p, name) -> p.map(), PARAMS_FIELD); - } - - private Map params; - public QuerqyDecorationAggregationBuilder() { super(NAME); } - public QuerqyDecorationAggregationBuilder(String name) { - super(name); + public QuerqyDecorationAggregationBuilder(StreamInput in) throws IOException { + super(in); } protected QuerqyDecorationAggregationBuilder(QuerqyDecorationAggregationBuilder clone, Builder factoriesBuilder, Map metadata) { super(clone, factoriesBuilder, metadata); - this.params = clone.params; } @Override @@ -50,36 +38,6 @@ protected AggregationBuilder shallowCopy(Builder factoriesBuilder, Map params) { - if (params == null) { - throw new IllegalArgumentException("[params] must not be null: [" + name + "]"); - } - this.params = params; - return this; - } - - /** - * Get parameters that will be available in the {@code init}, - * {@code map} and {@code combine} phases. - */ - public Map params() { - return params; - } - @Override public BucketCardinality bucketCardinality() { return BucketCardinality.NONE; @@ -92,33 +50,13 @@ protected AggregatorFactory doBuild(AggregationContext context, AggregatorFactor } @Override - protected XContentBuilder internalXContent(XContentBuilder builder, Params builderParams) throws IOException { - builder.startObject(); - if (params != null) { - builder.field(PARAMS_FIELD.getPreferredName()); - builder.map(params); - } - builder.endObject(); + protected XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException { return builder; } @Override protected void doWriteTo(StreamOutput out) throws IOException { - boolean hasParams = params != null; - out.writeBoolean(hasParams); - if (hasParams) { - out.writeMap(params); - } - } - - public static QuerqyDecorationAggregationBuilder fromXContent(final XContentParser parser) { - final QuerqyDecorationAggregationBuilder builder; - try { - builder = PARSER.apply(parser, null); - } catch (final IllegalArgumentException e) { - throw new ParsingException(parser.getTokenLocation(), e.getMessage(), e); - } - return builder; + // no state to write out } @Override @@ -128,16 +66,14 @@ public String getType() { @Override public int hashCode() { - return Objects.hash(super.hashCode(), params); + return Objects.hash(super.hashCode()); } @Override public boolean equals(Object obj) { if (this == obj) return true; if (obj == null || getClass() != obj.getClass()) return false; - if (super.equals(obj) == false) return false; - QuerqyDecorationAggregationBuilder other = (QuerqyDecorationAggregationBuilder) obj; - return Objects.equals(params, other.params); + return super.equals(obj); } } diff --git a/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java b/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java index 588ec30..14ee20e 100644 --- a/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java +++ b/src/test/java/querqy/elasticsearch/RewriterIntegrationTest.java @@ -3,15 +3,11 @@ import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.WriteRequest; -import org.elasticsearch.common.bytes.BytesArray; -import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.ESSingleNodeTestCase; -import org.elasticsearch.xcontent.XContentType; import org.junit.After; import org.junit.Test; -import querqy.elasticsearch.aggregation.QuerqyDecorationAggregationBuilder; import querqy.elasticsearch.query.MatchingQuery; import querqy.elasticsearch.query.QuerqyQueryBuilder; import querqy.elasticsearch.query.Rewriter; @@ -48,7 +44,7 @@ public void testSearchWithConfig() throws Exception { content.put("class", querqy.elasticsearch.rewriter.SimpleCommonRulesRewriterFactory.class.getName()); final Map config = new HashMap<>(); - config.put("rules", "k =>\nSYNONYM: c\na =>\nDECORATE: REDIRECT /faq/a"); + config.put("rules", "k =>\nSYNONYM: c"); config.put("ignoreCase", true); config.put("querqyParser", querqy.rewrite.commonrules.WhiteSpaceQuerqyParserFactory.class.getName()); content.put("config", config); @@ -57,39 +53,18 @@ public void testSearchWithConfig() throws Exception { client().execute(PutRewriterAction.INSTANCE, request).get(); - QuerqyProcessor querqyProcessor = getInstanceFromNode(QuerqyProcessor.class); - QuerqyQueryBuilder querqyQuery = new QuerqyQueryBuilder(querqyProcessor); + QuerqyQueryBuilder querqyQuery = new QuerqyQueryBuilder(getInstanceFromNode(QuerqyProcessor.class)); querqyQuery.setRewriters(Collections.singletonList(new Rewriter("common_rules"))); querqyQuery.setMatchingQuery(new MatchingQuery("a k")); querqyQuery.setQueryFieldsAndBoostings(Arrays.asList("field1", "field2")); querqyQuery.setMinimumShouldMatch("1"); - QuerqyDecorationAggregationBuilder querqyDecorationAggregationBuilder = - QuerqyDecorationAggregationBuilder.fromXContent( - XContentHelper.createParser( - null, - null, - new BytesArray("{" + - "\"params\": {" + - "\"test-query\": \"a k\"" + - "}}" - ), - XContentType.JSON) - ); - - SearchRequestBuilder searchRequestBuilder = client().prepareSearch(INDEX_NAME); + final SearchRequestBuilder searchRequestBuilder = client().prepareSearch(INDEX_NAME); searchRequestBuilder.setQuery(querqyQuery); - searchRequestBuilder.addAggregation(querqyDecorationAggregationBuilder); + SearchResponse response = client().search(searchRequestBuilder.request()).get(); assertEquals(2L, response.getHits().getTotalHits().value); - assertEquals("{\"decorations\":{\"value\":[\"REDIRECT /faq/a\"]}}", response.getAggregations().getAsMap().get(QuerqyDecorationAggregationBuilder.NAME).toString()); - - querqyQuery.setMatchingQuery(new MatchingQuery("x z")); - searchRequestBuilder.setQuery(querqyQuery); - response = client().search(searchRequestBuilder.request()).get(); - assertEquals(0L, response.getHits().getTotalHits().value); - assertEquals("{\"decorations\":{\"value\":[]}}", response.getAggregations().getAsMap().get(QuerqyDecorationAggregationBuilder.NAME).toString()); } @Test diff --git a/src/test/java/querqy/elasticsearch/aggregation/InternalDecorationAggregationTests.java b/src/test/java/querqy/elasticsearch/aggregation/InternalDecorationAggregationTests.java new file mode 100644 index 0000000..c84f636 --- /dev/null +++ b/src/test/java/querqy/elasticsearch/aggregation/InternalDecorationAggregationTests.java @@ -0,0 +1,242 @@ +package querqy.elasticsearch.aggregation; + +import org.elasticsearch.Version; +import org.elasticsearch.common.geo.GeoPoint; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.plugins.SearchPlugin; +import org.elasticsearch.search.aggregations.Aggregation; +import org.elasticsearch.search.aggregations.Aggregation.CommonFields; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.InternalAggregation.ReduceContext; +import org.elasticsearch.search.aggregations.ParsedAggregation; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator.PipelineTree; +import org.elasticsearch.test.InternalAggregationTestCase; +import org.elasticsearch.test.VersionUtils; +import org.elasticsearch.xcontent.ContextParser; +import org.elasticsearch.xcontent.NamedXContentRegistry; +import org.elasticsearch.xcontent.ParseField; +import querqy.elasticsearch.QuerqyPlugin; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.function.Predicate; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +import static java.util.Collections.singletonList; +import static org.hamcrest.Matchers.equalTo; + +public class InternalDecorationAggregationTests extends InternalAggregationTestCase { + + private Supplier[] valueTypes; + @SuppressWarnings({ "rawtypes", "unchecked" }) + private final Supplier[] leafValueSuppliers = new Supplier[] { + () -> randomInt(), + () -> randomLong(), + () -> randomDouble(), + () -> randomFloat(), + () -> randomBoolean(), + () -> randomAlphaOfLength(5), + () -> new GeoPoint(randomDouble(), randomDouble()), + () -> null }; + @SuppressWarnings({ "rawtypes", "unchecked" }) + private final Supplier[] nestedValueSuppliers = new Supplier[] { () -> new HashMap(), () -> new ArrayList<>() }; + + private static final List namedXContents = getDefaultNamedXContents(); + static { + Map> map = new HashMap<>(); + map.put(QuerqyDecorationAggregationBuilder.NAME, (p, c) -> ParsedDecorationAggregation.fromXContent(p, (String) c)); + + List namedXContentsToAdd = map.entrySet() + .stream() + .map(entry -> new NamedXContentRegistry.Entry(Aggregation.class, new ParseField(entry.getKey()), entry.getValue())) + .collect(Collectors.toList()); + namedXContents.addAll(namedXContentsToAdd); + } + + @Override + @SuppressWarnings({ "rawtypes", "unchecked" }) + public void setUp() throws Exception { + super.setUp(); + // we want the same value types (also for nested lists, maps) for all random aggregations + int levels = randomIntBetween(1, 3); + valueTypes = new Supplier[levels]; + for (int i = 0; i < levels; i++) { + if (i < levels - 1) { + valueTypes[i] = randomFrom(nestedValueSuppliers); + } else { + // the last one needs to be a leaf value, not map or list + valueTypes[i] = randomFrom(leafValueSuppliers); + } + } + } + + @Override + protected SearchPlugin registerPlugin() { + return new QuerqyPlugin(Settings.EMPTY); + } + + @Override + protected InternalDecorationAggregation createTestInstance(String name, Map metadata) { + Map params = new HashMap<>(); + if (randomBoolean()) { + params.put(randomAlphaOfLength(5), randomAlphaOfLength(5)); + } + return new InternalDecorationAggregation(name, randomAggregations(), metadata); + } + + private List randomAggregations() { + return randomList(randomBoolean() ? 1 : 5, this::randomAggregation); + } + + @SuppressWarnings({ "rawtypes", "unchecked" }) + private Object randomAggregation() { + int levels = randomIntBetween(1, 3); + Supplier[] valueTypes = new Supplier[levels]; + for (int l = 0; l < levels; l++) { + if (l < levels - 1) { + valueTypes[l] = randomFrom(nestedValueSuppliers); + } else { + // the last one needs to be a leaf value, not map or + // list + valueTypes[l] = randomFrom(leafValueSuppliers); + } + } + return randomValue(valueTypes, 0); + } + + @SuppressWarnings("unchecked") + private static Object randomValue(Supplier[] valueTypes, int level) { + Object value = valueTypes[level].get(); + if (value instanceof Map) { + int elements = randomIntBetween(1, 5); + Map map = (Map) value; + for (int i = 0; i < elements; i++) { + map.put(randomAlphaOfLength(5), randomValue(valueTypes, level + 1)); + } + } else if (value instanceof List) { + int elements = randomIntBetween(1, 5); + List list = (List) value; + for (int i = 0; i < elements; i++) { + list.add(randomValue(valueTypes, level + 1)); + } + } + return value; + } + + @Override + protected void assertReduced(InternalDecorationAggregation reduced, List inputs) { + InternalDecorationAggregation firstAgg = inputs.get(0); + assertEquals(firstAgg.getName(), reduced.getName()); + assertEquals(firstAgg.getMetadata(), reduced.getMetadata()); + int size = (int) inputs.stream().mapToLong(i -> i.aggregationsList().size()).sum(); + assertEquals(size, ((List) reduced.aggregation()).size()); + } + + @Override + public InternalDecorationAggregation createTestInstanceForXContent() { + InternalDecorationAggregation aggregation = createTestInstance(); + return (InternalDecorationAggregation) aggregation.reduce( + singletonList(aggregation), + ReduceContext.forFinalReduction(null, mockScriptService(), null, PipelineTree.EMPTY, () -> false) + ); + } + + @Override + protected void assertFromXContent(InternalDecorationAggregation aggregation, ParsedAggregation parsedAggregation) throws IOException { + assertTrue(parsedAggregation instanceof ParsedDecorationAggregation); + ParsedDecorationAggregation parsed = (ParsedDecorationAggregation) parsedAggregation; + + assertValues(aggregation.aggregation(), parsed.aggregation()); + } + + private static void assertValues(Object expected, Object actual) { + if (expected instanceof Long) { + // longs that fit into the integer range are parsed back as integer + if (actual instanceof Integer) { + assertEquals(((Long) expected).intValue(), actual); + } else { + assertEquals(expected, actual); + } + } else if (expected instanceof Float) { + // based on the xContent type, floats are sometimes parsed back as doubles + if (actual instanceof Double) { + assertEquals(expected, ((Double) actual).floatValue()); + } else { + assertEquals(expected, actual); + } + } else if (expected instanceof GeoPoint) { + assertTrue(actual instanceof Map); + GeoPoint point = (GeoPoint) expected; + @SuppressWarnings("unchecked") + Map pointMap = (Map) actual; + assertEquals(point.getLat(), pointMap.get("lat")); + assertEquals(point.getLon(), pointMap.get("lon")); + } else if (expected instanceof Map) { + @SuppressWarnings("unchecked") + Map expectedMap = (Map) expected; + @SuppressWarnings("unchecked") + Map actualMap = (Map) actual; + assertEquals(expectedMap.size(), actualMap.size()); + for (String key : expectedMap.keySet()) { + assertValues(expectedMap.get(key), actualMap.get(key)); + } + } else if (expected instanceof List) { + @SuppressWarnings("unchecked") + List expectedList = (List) expected; + @SuppressWarnings("unchecked") + List actualList = (List) actual; + assertEquals(expectedList.size(), actualList.size()); + Iterator actualIterator = actualList.iterator(); + for (Object element : expectedList) { + assertValues(element, actualIterator.next()); + } + } else { + assertEquals(expected, actual); + } + } + + @Override + protected Predicate excludePathsFromXContentInsertion() { + return path -> path.contains(CommonFields.VALUE.getPreferredName()); + } + + public void testOldSerialization() throws IOException { + // A single element list looks like a fully reduced agg + InternalDecorationAggregation original = new InternalDecorationAggregation( + "test", + org.elasticsearch.core.List.of("foo"), + null + ); + original.mergePipelineTreeForBWCSerialization(PipelineTree.EMPTY); + InternalDecorationAggregation roundTripped = (InternalDecorationAggregation) copyNamedWriteable( + original, + getNamedWriteableRegistry(), + InternalAggregation.class, + VersionUtils.randomVersionBetween(random(), Version.V_7_0_0, VersionUtils.getPreviousVersion(Version.V_7_8_0)) + ); + assertThat(roundTripped, equalTo(original)); + + // A multi-element list looks like a non-reduced agg + InternalDecorationAggregation unreduced = new InternalDecorationAggregation( + "test", + org.elasticsearch.core.List.of("foo", "bar"), + null + ); + unreduced.mergePipelineTreeForBWCSerialization(PipelineTree.EMPTY); + Exception e = expectThrows( + IllegalArgumentException.class, + () -> copyNamedWriteable( + unreduced, + getNamedWriteableRegistry(), + InternalAggregation.class, + VersionUtils.randomVersionBetween(random(), Version.V_7_0_0, VersionUtils.getPreviousVersion(Version.V_7_8_0)) + ) + ); + assertThat(e.getMessage(), equalTo("querqy doesn't support cross cluster search until 7.8.0")); + } +} diff --git a/src/test/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationIntegrationTest.java b/src/test/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationIntegrationTest.java new file mode 100644 index 0000000..c13b584 --- /dev/null +++ b/src/test/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationIntegrationTest.java @@ -0,0 +1,88 @@ +package querqy.elasticsearch.aggregation; + +import org.elasticsearch.action.search.SearchRequestBuilder; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.test.ESSingleNodeTestCase; +import org.junit.After; +import querqy.elasticsearch.QuerqyPlugin; +import querqy.elasticsearch.QuerqyProcessor; +import querqy.elasticsearch.query.MatchingQuery; +import querqy.elasticsearch.query.QuerqyQueryBuilder; +import querqy.elasticsearch.query.Rewriter; +import querqy.elasticsearch.rewriterstore.PutRewriterAction; +import querqy.elasticsearch.rewriterstore.PutRewriterRequest; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +public class QuerqyDecorationAggregationIntegrationTest extends ESSingleNodeTestCase { + + private final String INDEX_NAME = "test_index"; + + @Override + protected Collection> getPlugins() { + return Collections.singleton(QuerqyPlugin.class); + } + + public void testSearchWithConfig() throws Exception { + + index(); + + final Map content = new HashMap<>(); + content.put("class", querqy.elasticsearch.rewriter.SimpleCommonRulesRewriterFactory.class.getName()); + + final Map config = new HashMap<>(); + config.put("rules", "k =>\nSYNONYM: c\na =>\nDECORATE: REDIRECT /faq/a"); + config.put("ignoreCase", true); + config.put("querqyParser", querqy.rewrite.commonrules.WhiteSpaceQuerqyParserFactory.class.getName()); + content.put("config", config); + + final PutRewriterRequest request = new PutRewriterRequest("common_rules", content); + + client().execute(PutRewriterAction.INSTANCE, request).get(); + + QuerqyProcessor querqyProcessor = getInstanceFromNode(QuerqyProcessor.class); + QuerqyQueryBuilder querqyQuery = new QuerqyQueryBuilder(querqyProcessor); + querqyQuery.setRewriters(Collections.singletonList(new Rewriter("common_rules"))); + querqyQuery.setMatchingQuery(new MatchingQuery("a k")); + querqyQuery.setQueryFieldsAndBoostings(Arrays.asList("field1", "field2")); + querqyQuery.setMinimumShouldMatch("1"); + + QuerqyDecorationAggregationBuilder aggregationBuilder = new QuerqyDecorationAggregationBuilder(); + // with decorations + SearchRequestBuilder searchRequestBuilder = client().prepareSearch(INDEX_NAME); + searchRequestBuilder.setQuery(querqyQuery); + searchRequestBuilder.addAggregation(aggregationBuilder); + SearchResponse response = client().search(searchRequestBuilder.request()).get(); + assertEquals(2L, response.getHits().getTotalHits().value); + assertEquals("{\"decorations\":{\"value\":[\"REDIRECT /faq/a\"]}}", response.getAggregations().getAsMap().get(QuerqyDecorationAggregationBuilder.NAME).toString()); + + // without decorations + querqyQuery.setMatchingQuery(new MatchingQuery("x z")); + searchRequestBuilder.setQuery(querqyQuery); + response = client().search(searchRequestBuilder.request()).get(); + assertEquals(0L, response.getHits().getTotalHits().value); + assertEquals("{\"decorations\":{\"value\":[]}}", response.getAggregations().getAsMap().get(QuerqyDecorationAggregationBuilder.NAME).toString()); + } + + @After + public void deleteRewriterIndex() { + client().admin().indices().prepareDelete(".querqy").get(); + } + + public void index() { + client().admin().indices().prepareCreate(INDEX_NAME).get(); + client().prepareIndex(INDEX_NAME, null) + .setSource("field1", "a b", "field2", "a c") + .get(); + client().prepareIndex(INDEX_NAME, null) + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) + .setSource("field1", "b c") + .get(); + } +} From bd15b0e696c4dee5aca295c0608d31a6a6d42bef Mon Sep 17 00:00:00 2001 From: paulb Date: Mon, 26 Feb 2024 12:48:49 +0100 Subject: [PATCH 09/15] Add test decorations without hits --- ...rqyDecorationAggregationIntegrationTest.java | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/src/test/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationIntegrationTest.java b/src/test/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationIntegrationTest.java index c13b584..331673e 100644 --- a/src/test/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationIntegrationTest.java +++ b/src/test/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationIntegrationTest.java @@ -1,5 +1,7 @@ package querqy.elasticsearch.aggregation; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.WriteRequest; @@ -22,6 +24,8 @@ public class QuerqyDecorationAggregationIntegrationTest extends ESSingleNodeTestCase { + private final static Logger LOGGER = LogManager.getLogger(QuerqyDecorationAggregationIntegrationTest.class); + private final String INDEX_NAME = "test_index"; @Override @@ -37,7 +41,7 @@ public void testSearchWithConfig() throws Exception { content.put("class", querqy.elasticsearch.rewriter.SimpleCommonRulesRewriterFactory.class.getName()); final Map config = new HashMap<>(); - config.put("rules", "k =>\nSYNONYM: c\na =>\nDECORATE: REDIRECT /faq/a"); + config.put("rules", "k =>\nSYNONYM: c\na =>\nDECORATE: REDIRECT /faq/a\ny =>\nDECORATE: REDIRECT /faq/y"); config.put("ignoreCase", true); config.put("querqyParser", querqy.rewrite.commonrules.WhiteSpaceQuerqyParserFactory.class.getName()); content.put("config", config); @@ -61,13 +65,22 @@ public void testSearchWithConfig() throws Exception { SearchResponse response = client().search(searchRequestBuilder.request()).get(); assertEquals(2L, response.getHits().getTotalHits().value); assertEquals("{\"decorations\":{\"value\":[\"REDIRECT /faq/a\"]}}", response.getAggregations().getAsMap().get(QuerqyDecorationAggregationBuilder.NAME).toString()); + LOGGER.info("Response:\n{}", response); - // without decorations + // without hits, without decorations querqyQuery.setMatchingQuery(new MatchingQuery("x z")); searchRequestBuilder.setQuery(querqyQuery); response = client().search(searchRequestBuilder.request()).get(); assertEquals(0L, response.getHits().getTotalHits().value); assertEquals("{\"decorations\":{\"value\":[]}}", response.getAggregations().getAsMap().get(QuerqyDecorationAggregationBuilder.NAME).toString()); + + // without hits, with decorations + querqyQuery.setMatchingQuery(new MatchingQuery("x y")); + searchRequestBuilder.setQuery(querqyQuery); + response = client().search(searchRequestBuilder.request()).get(); + assertEquals(0L, response.getHits().getTotalHits().value); + assertEquals("{\"decorations\":{\"value\":[\"REDIRECT /faq/y\"]}}", response.getAggregations().getAsMap().get(QuerqyDecorationAggregationBuilder.NAME).toString()); + } @After From 6d6364624ea8578e456db4977785267ee15a57bc Mon Sep 17 00:00:00 2001 From: paulb Date: Mon, 26 Feb 2024 12:51:20 +0100 Subject: [PATCH 10/15] Add test without decorations with hits --- .../QuerqyDecorationAggregationIntegrationTest.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/test/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationIntegrationTest.java b/src/test/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationIntegrationTest.java index 331673e..165b302 100644 --- a/src/test/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationIntegrationTest.java +++ b/src/test/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationIntegrationTest.java @@ -81,6 +81,13 @@ public void testSearchWithConfig() throws Exception { assertEquals(0L, response.getHits().getTotalHits().value); assertEquals("{\"decorations\":{\"value\":[\"REDIRECT /faq/y\"]}}", response.getAggregations().getAsMap().get(QuerqyDecorationAggregationBuilder.NAME).toString()); + // with hits, without decorations + querqyQuery.setMatchingQuery(new MatchingQuery("k x")); + searchRequestBuilder.setQuery(querqyQuery); + response = client().search(searchRequestBuilder.request()).get(); + assertEquals(2L, response.getHits().getTotalHits().value); + assertEquals("{\"decorations\":{\"value\":[]}}", response.getAggregations().getAsMap().get(QuerqyDecorationAggregationBuilder.NAME).toString()); + } @After From 0239fef382c7d58b9cc54bff81253c17ccec8141 Mon Sep 17 00:00:00 2001 From: paulb Date: Mon, 26 Feb 2024 16:53:02 +0100 Subject: [PATCH 11/15] Add test with inner queries --- .../QuerqyDecorationAggregatorFactory.java | 32 +++++++-- ...yDecorationAggregationIntegrationTest.java | 69 +++++++++++++++---- 2 files changed, 82 insertions(+), 19 deletions(-) diff --git a/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregatorFactory.java b/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregatorFactory.java index 45bbbb3..20755bf 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregatorFactory.java +++ b/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregatorFactory.java @@ -1,5 +1,7 @@ package querqy.elasticsearch.aggregation; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.ConstantScoreQuery; import org.apache.lucene.search.Query; import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.Aggregator; @@ -10,6 +12,7 @@ import java.io.IOException; import java.util.Collections; +import java.util.HashSet; import java.util.Map; import java.util.Set; @@ -42,10 +45,31 @@ public Aggregator createInternal(Aggregator parent, CardinalityUpperBound cardin throw new AggregationExecutionException("Aggregation [" + name() + "] must have cardinality 1 but was [" + cardinality + "]"); } Query query = context.subSearchContext() == null ? null : context.subSearchContext().query(); - Set decorations = - (query instanceof DecoratedQuery) ? - ((DecoratedQuery) query).getDecorations() : - Collections.emptySet(); + Set> decoratedQueries = getDecoratedQueries(query); + Set decorations; + if (decoratedQueries.isEmpty()) { + decorations = Collections.emptySet(); + } else { + decorations = new HashSet<>(); + decoratedQueries.forEach(decoratedQuery -> decorations.addAll(decoratedQuery.getDecorations())); + } return new QuerqyDecorationAggregator(name, context, metadata, decorations); } + + private Set> getDecoratedQueries(Query query) { + if (query instanceof DecoratedQuery) { + return Collections.singleton((DecoratedQuery) query); + } + if (query instanceof ConstantScoreQuery) { + return getDecoratedQueries(((ConstantScoreQuery) query).getQuery()); + } + if (query instanceof BooleanQuery) { + Set> decoratedQueries = new HashSet<>(); + ((BooleanQuery) query).clauses().forEach( + booleanClause -> decoratedQueries.addAll(getDecoratedQueries(booleanClause.getQuery())) + ); + return decoratedQueries; + } + return Collections.emptySet(); + } } diff --git a/src/test/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationIntegrationTest.java b/src/test/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationIntegrationTest.java index 165b302..a14a01c 100644 --- a/src/test/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationIntegrationTest.java +++ b/src/test/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationIntegrationTest.java @@ -5,8 +5,13 @@ import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.index.query.BoolQueryBuilder; +import org.elasticsearch.index.query.ConstantScoreQueryBuilder; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESSingleNodeTestCase; +import org.elasticsearch.xcontent.XContentType; import org.junit.After; import querqy.elasticsearch.QuerqyPlugin; import querqy.elasticsearch.QuerqyProcessor; @@ -20,10 +25,15 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; +import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.concurrent.ExecutionException; public class QuerqyDecorationAggregationIntegrationTest extends ESSingleNodeTestCase { + public static String DECORATIONS_TEMPLATE = "{\"decorations\":{\"value\":[%s]}}"; private final static Logger LOGGER = LogManager.getLogger(QuerqyDecorationAggregationIntegrationTest.class); private final String INDEX_NAME = "test_index"; @@ -51,43 +61,72 @@ public void testSearchWithConfig() throws Exception { client().execute(PutRewriterAction.INSTANCE, request).get(); QuerqyProcessor querqyProcessor = getInstanceFromNode(QuerqyProcessor.class); + + SearchRequestBuilder searchRequestBuilder = client().prepareSearch(INDEX_NAME); + QuerqyQueryBuilder querqyQuery = new QuerqyQueryBuilder(querqyProcessor); querqyQuery.setRewriters(Collections.singletonList(new Rewriter("common_rules"))); - querqyQuery.setMatchingQuery(new MatchingQuery("a k")); querqyQuery.setQueryFieldsAndBoostings(Arrays.asList("field1", "field2")); querqyQuery.setMinimumShouldMatch("1"); QuerqyDecorationAggregationBuilder aggregationBuilder = new QuerqyDecorationAggregationBuilder(); + String[] expectedDecoration = new String[] {"REDIRECT /faq/a","REDIRECT /faq/y"}; + // with decorations - SearchRequestBuilder searchRequestBuilder = client().prepareSearch(INDEX_NAME); + querqyQuery.setMatchingQuery(new MatchingQuery("a k")); searchRequestBuilder.setQuery(querqyQuery); searchRequestBuilder.addAggregation(aggregationBuilder); - SearchResponse response = client().search(searchRequestBuilder.request()).get(); - assertEquals(2L, response.getHits().getTotalHits().value); - assertEquals("{\"decorations\":{\"value\":[\"REDIRECT /faq/a\"]}}", response.getAggregations().getAsMap().get(QuerqyDecorationAggregationBuilder.NAME).toString()); - LOGGER.info("Response:\n{}", response); + testSearchRequest(searchRequestBuilder, 2L, Collections.singleton(expectedDecoration[0])); // without hits, without decorations querqyQuery.setMatchingQuery(new MatchingQuery("x z")); searchRequestBuilder.setQuery(querqyQuery); - response = client().search(searchRequestBuilder.request()).get(); - assertEquals(0L, response.getHits().getTotalHits().value); - assertEquals("{\"decorations\":{\"value\":[]}}", response.getAggregations().getAsMap().get(QuerqyDecorationAggregationBuilder.NAME).toString()); + testSearchRequest(searchRequestBuilder, 0L, Collections.emptySet()); // without hits, with decorations querqyQuery.setMatchingQuery(new MatchingQuery("x y")); searchRequestBuilder.setQuery(querqyQuery); - response = client().search(searchRequestBuilder.request()).get(); - assertEquals(0L, response.getHits().getTotalHits().value); - assertEquals("{\"decorations\":{\"value\":[\"REDIRECT /faq/y\"]}}", response.getAggregations().getAsMap().get(QuerqyDecorationAggregationBuilder.NAME).toString()); + testSearchRequest(searchRequestBuilder, 0L, Collections.singleton(expectedDecoration[1])); // with hits, without decorations querqyQuery.setMatchingQuery(new MatchingQuery("k x")); searchRequestBuilder.setQuery(querqyQuery); - response = client().search(searchRequestBuilder.request()).get(); - assertEquals(2L, response.getHits().getTotalHits().value); - assertEquals("{\"decorations\":{\"value\":[]}}", response.getAggregations().getAsMap().get(QuerqyDecorationAggregationBuilder.NAME).toString()); + testSearchRequest(searchRequestBuilder, 2L, Collections.emptySet()); + // inner boolean query + querqyQuery.setMatchingQuery(new MatchingQuery("a k")); + BoolQueryBuilder boolQueryBuilder = new BoolQueryBuilder(); + boolQueryBuilder.should(querqyQuery); + searchRequestBuilder.setQuery(boolQueryBuilder); + testSearchRequest(searchRequestBuilder, 2L, Collections.singleton(expectedDecoration[0])); + + // inner constant score && inner bool query + querqyQuery.setMatchingQuery(new MatchingQuery("a k")); + + QuerqyQueryBuilder querqyQuery2 = new QuerqyQueryBuilder(querqyProcessor); + querqyQuery2.setRewriters(Collections.singletonList(new Rewriter("common_rules"))); + querqyQuery2.setQueryFieldsAndBoostings(Arrays.asList("field1", "field2")); + querqyQuery2.setMinimumShouldMatch("1"); + querqyQuery2.setMatchingQuery(new MatchingQuery("x y")); + + ConstantScoreQueryBuilder constantScoreQueryBuilder = new ConstantScoreQueryBuilder(querqyQuery2); + boolQueryBuilder = new BoolQueryBuilder(); + boolQueryBuilder.should(querqyQuery); + boolQueryBuilder.should(constantScoreQueryBuilder); + searchRequestBuilder.setQuery(boolQueryBuilder); + testSearchRequest(searchRequestBuilder, 2L, new HashSet<>(Arrays.asList(expectedDecoration))); + + } + + private void testSearchRequest(SearchRequestBuilder searchRequestBuilder, long expectedHits, Set expectedDecorations) throws ExecutionException, InterruptedException { + SearchResponse response = client().search(searchRequestBuilder.request()).get(); + LOGGER.info("Response:\n{}", response); + assertEquals(expectedHits, response.getHits().getTotalHits().value); + InternalDecorationAggregation aggregation = response.getAggregations().get(QuerqyDecorationAggregationBuilder.NAME); + assertEquals( + expectedDecorations, + new HashSet<>((List) aggregation.aggregation()) + ); } @After From 8bc83d6c02ee88dd7c536742f01837aee55a3ec1 Mon Sep 17 00:00:00 2001 From: paulb Date: Tue, 27 Feb 2024 11:36:43 +0100 Subject: [PATCH 12/15] Enhance collecting Set via QueryVisitor --- .../aggregation/DecoratedQuery.java | 6 --- .../QuerqyDecorationAggregatorFactory.java | 44 +++++++++---------- 2 files changed, 21 insertions(+), 29 deletions(-) diff --git a/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java b/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java index 6395c6f..2711bda 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java +++ b/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java @@ -3,7 +3,6 @@ import org.apache.lucene.index.IndexReader; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; -import org.apache.lucene.search.QueryVisitor; import org.apache.lucene.search.ScoreMode; import org.apache.lucene.search.Weight; @@ -39,11 +38,6 @@ public Query rewrite(IndexReader reader) throws IOException { return query.rewrite(reader); } - @Override - public void visit(QueryVisitor visitor) { - query.visit(visitor); - } - @Override public String toString(String field) { return query.toString(field); diff --git a/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregatorFactory.java b/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregatorFactory.java index 20755bf..d5979e0 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregatorFactory.java +++ b/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregatorFactory.java @@ -1,8 +1,7 @@ package querqy.elasticsearch.aggregation; -import org.apache.lucene.search.BooleanQuery; -import org.apache.lucene.search.ConstantScoreQuery; import org.apache.lucene.search.Query; +import org.apache.lucene.search.QueryVisitor; import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; @@ -15,6 +14,7 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; public class QuerqyDecorationAggregatorFactory extends AggregatorFactory { @@ -46,30 +46,28 @@ public Aggregator createInternal(Aggregator parent, CardinalityUpperBound cardin } Query query = context.subSearchContext() == null ? null : context.subSearchContext().query(); Set> decoratedQueries = getDecoratedQueries(query); - Set decorations; - if (decoratedQueries.isEmpty()) { - decorations = Collections.emptySet(); - } else { - decorations = new HashSet<>(); - decoratedQueries.forEach(decoratedQuery -> decorations.addAll(decoratedQuery.getDecorations())); - } - return new QuerqyDecorationAggregator(name, context, metadata, decorations); + return new QuerqyDecorationAggregator(name, context, metadata, collectAllDecorations(decoratedQueries)); + } + + + private Set collectAllDecorations(Set> decoratedQueries) { + return decoratedQueries.stream().flatMap(decoratedQuery -> decoratedQuery.getDecorations().stream()).collect(Collectors.toSet()); } private Set> getDecoratedQueries(Query query) { - if (query instanceof DecoratedQuery) { - return Collections.singleton((DecoratedQuery) query); + if (query == null) { + return Collections.emptySet(); } - if (query instanceof ConstantScoreQuery) { - return getDecoratedQueries(((ConstantScoreQuery) query).getQuery()); - } - if (query instanceof BooleanQuery) { - Set> decoratedQueries = new HashSet<>(); - ((BooleanQuery) query).clauses().forEach( - booleanClause -> decoratedQueries.addAll(getDecoratedQueries(booleanClause.getQuery())) - ); - return decoratedQueries; - } - return Collections.emptySet(); + Set> decoratedQueries = new HashSet<>(); + query.visit(new QueryVisitor() { + @Override + public void visitLeaf(Query query) { + if (query instanceof DecoratedQuery) { + decoratedQueries.add((DecoratedQuery) query); + } + } + }); + return decoratedQueries; } + } From d3dc22e4ac29248217f7c847b1d8bc1b539ab19b Mon Sep 17 00:00:00 2001 From: paulb Date: Wed, 28 Feb 2024 09:25:50 +0100 Subject: [PATCH 13/15] Add final for local variable and parameter --- .../querqy/elasticsearch/QuerqyPlugin.java | 6 +-- .../aggregation/DecoratedQuery.java | 10 ++--- .../InternalDecorationAggregation.java | 28 ++++++------ .../ParsedDecorationAggregation.java | 44 +++++++++---------- .../QuerqyDecorationAggregationBuilder.java | 14 +++--- .../QuerqyDecorationAggregator.java | 6 +-- .../QuerqyDecorationAggregatorFactory.java | 24 +++++----- 7 files changed, 66 insertions(+), 66 deletions(-) diff --git a/src/main/java/querqy/elasticsearch/QuerqyPlugin.java b/src/main/java/querqy/elasticsearch/QuerqyPlugin.java index b8ce378..7a818f7 100644 --- a/src/main/java/querqy/elasticsearch/QuerqyPlugin.java +++ b/src/main/java/querqy/elasticsearch/QuerqyPlugin.java @@ -114,8 +114,8 @@ public Collection createComponents(final Client client, final ClusterSer final NamedXContentRegistry xContentRegistry, final Environment environment, final NodeEnvironment nodeEnvironment, final NamedWriteableRegistry namedWriteableRegistry, - IndexNameExpressionResolver indexNameExpressionResolver, - Supplier repositoriesServiceSupplier) { + final IndexNameExpressionResolver indexNameExpressionResolver, + final Supplier repositoriesServiceSupplier) { return Arrays.asList(rewriterShardContexts, querqyProcessor); } @@ -128,7 +128,7 @@ public List> getSettings() { @Override public List getAggregations() { - List r = new ArrayList<>(); + final List r = new ArrayList<>(); r.add( new AggregationSpec( QuerqyDecorationAggregationBuilder.NAME, diff --git a/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java b/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java index 2711bda..0b50d69 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java +++ b/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java @@ -15,7 +15,7 @@ public class DecoratedQuery extends Query { final private T query; final private Set decorations; - public DecoratedQuery(T query, Set decorations) { + public DecoratedQuery(final T query, final Set decorations) { this.query = Objects.requireNonNull(query); this.decorations = Objects.requireNonNull(decorations); } @@ -29,17 +29,17 @@ public Set getDecorations() { } @Override - public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException { + public Weight createWeight(final IndexSearcher searcher, final ScoreMode scoreMode, final float boost) throws IOException { return query.createWeight(searcher, scoreMode, boost); } @Override - public Query rewrite(IndexReader reader) throws IOException { + public Query rewrite(final IndexReader reader) throws IOException { return query.rewrite(reader); } @Override - public String toString(String field) { + public String toString(final String field) { return query.toString(field); } @@ -56,7 +56,7 @@ private boolean isEqualQueriesAndDecorations(DecoratedQuery other) { return getQuery().equals(otherQuery) && getDecorations().equals(otherDecorations); } - private DecoratedQuery castObject(Object object) { + private DecoratedQuery castObject(final Object object) { return getClass().cast(object); } diff --git a/src/main/java/querqy/elasticsearch/aggregation/InternalDecorationAggregation.java b/src/main/java/querqy/elasticsearch/aggregation/InternalDecorationAggregation.java index 41e3c36..b723a9d 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/InternalDecorationAggregation.java +++ b/src/main/java/querqy/elasticsearch/aggregation/InternalDecorationAggregation.java @@ -20,12 +20,12 @@ public class InternalDecorationAggregation extends InternalAggregation implement private final List aggregations; - InternalDecorationAggregation(String name, List aggregations, Map metadata) { + InternalDecorationAggregation(final String name, final List aggregations, final Map metadata) { super(name, metadata); this.aggregations = aggregations; } - public InternalDecorationAggregation(StreamInput in) throws IOException { + public InternalDecorationAggregation(final StreamInput in) throws IOException { super(in); if (in.getVersion().before(Version.V_7_8_0)) { aggregations = singletonList(in.readGenericValue()); @@ -35,7 +35,7 @@ public InternalDecorationAggregation(StreamInput in) throws IOException { } @Override - protected void doWriteTo(StreamOutput out) throws IOException { + protected void doWriteTo(final StreamOutput out) throws IOException { if (out.getVersion().before(Version.V_7_8_0)) { if (aggregations.size() > 1) { /* @@ -70,14 +70,14 @@ List aggregationsList() { } @Override - public InternalAggregation reduce(List aggregations, ReduceContext reduceContext) { - List aggregationObjects = new ArrayList<>(); - for (InternalAggregation aggregation : aggregations) { - InternalDecorationAggregation mapReduceAggregation = (InternalDecorationAggregation) aggregation; + public InternalAggregation reduce(final List aggregations, final ReduceContext reduceContext) { + final List aggregationObjects = new ArrayList<>(); + for (final InternalAggregation aggregation : aggregations) { + final InternalDecorationAggregation mapReduceAggregation = (InternalDecorationAggregation) aggregation; aggregationObjects.addAll(mapReduceAggregation.aggregations); } - InternalDecorationAggregation firstAggregation = ((InternalDecorationAggregation) aggregations.get(0)); - List aggregation; + final InternalDecorationAggregation firstAggregation = ((InternalDecorationAggregation) aggregations.get(0)); + final List aggregation; if (reduceContext.isFinalReduce()) { aggregation = Collections.singletonList(aggregationObjects); } else { @@ -94,7 +94,7 @@ protected boolean mustReduceOnSingleInternalAgg() { } @Override - public Object getProperty(List path) { + public Object getProperty(final List path) { if (path.isEmpty()) { return this; } else if (path.size() == 1 && "value".equals(path.get(0))) { @@ -105,17 +105,17 @@ public Object getProperty(List path) { } @Override - public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + public XContentBuilder doXContentBody(final XContentBuilder builder, final Params params) throws IOException { return builder.field(CommonFields.VALUE.getPreferredName(), aggregation()); } @Override - public boolean equals(Object obj) { + public boolean equals(final Object obj) { if (this == obj) return true; if (obj == null || getClass() != obj.getClass()) return false; - if (super.equals(obj) == false) return false; + if (!super.equals(obj)) return false; - InternalDecorationAggregation other = (InternalDecorationAggregation) obj; + final InternalDecorationAggregation other = (InternalDecorationAggregation) obj; return Objects.equals(aggregations, other.aggregations); } diff --git a/src/main/java/querqy/elasticsearch/aggregation/ParsedDecorationAggregation.java b/src/main/java/querqy/elasticsearch/aggregation/ParsedDecorationAggregation.java index 185b5c7..3b08947 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/ParsedDecorationAggregation.java +++ b/src/main/java/querqy/elasticsearch/aggregation/ParsedDecorationAggregation.java @@ -26,7 +26,7 @@ public Object aggregation() { } @Override - public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + public XContentBuilder doXContentBody(final XContentBuilder builder, final Params params) throws IOException { return builder.field(CommonFields.VALUE.getPreferredName(), aggregation()); } @@ -46,33 +46,33 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th ); } - private static Object parseValue(XContentParser parser) throws IOException { - XContentParser.Token token = parser.currentToken(); + private static Object parseValue(final XContentParser parser) throws IOException { + final XContentParser.Token token = parser.currentToken(); Object value = null; - if (token == XContentParser.Token.VALUE_NULL) { - value = null; - } else if (token.isValue()) { - if (token == XContentParser.Token.VALUE_STRING) { - // binary values will be parsed back and returned as base64 strings when reading from json and yaml - value = parser.text(); - } else if (token == XContentParser.Token.VALUE_NUMBER) { - value = parser.numberValue(); - } else if (token == XContentParser.Token.VALUE_BOOLEAN) { - value = parser.booleanValue(); - } else if (token == XContentParser.Token.VALUE_EMBEDDED_OBJECT) { - // binary values will be parsed back and returned as BytesArray when reading from cbor and smile - value = new BytesArray(parser.binaryValue()); + if (token != XContentParser.Token.VALUE_NULL) { + if (token.isValue()) { + if (token == XContentParser.Token.VALUE_STRING) { + // binary values will be parsed back and returned as base64 strings when reading from json and yaml + value = parser.text(); + } else if (token == XContentParser.Token.VALUE_NUMBER) { + value = parser.numberValue(); + } else if (token == XContentParser.Token.VALUE_BOOLEAN) { + value = parser.booleanValue(); + } else if (token == XContentParser.Token.VALUE_EMBEDDED_OBJECT) { + // binary values will be parsed back and returned as BytesArray when reading from cbor and smile + value = new BytesArray(parser.binaryValue()); + } + } else if (token == XContentParser.Token.START_OBJECT) { + value = parser.map(); + } else if (token == XContentParser.Token.START_ARRAY) { + value = parser.list(); } - } else if (token == XContentParser.Token.START_OBJECT) { - value = parser.map(); - } else if (token == XContentParser.Token.START_ARRAY) { - value = parser.list(); } return value; } - public static ParsedDecorationAggregation fromXContent(XContentParser parser, final String name) { - ParsedDecorationAggregation aggregation = PARSER.apply(parser, null); + public static ParsedDecorationAggregation fromXContent(final XContentParser parser, final String name) { + final ParsedDecorationAggregation aggregation = PARSER.apply(parser, null); aggregation.setName(name); return aggregation; } diff --git a/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationBuilder.java b/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationBuilder.java index 4caf217..028a34e 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationBuilder.java +++ b/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregationBuilder.java @@ -25,16 +25,16 @@ public QuerqyDecorationAggregationBuilder() { super(NAME); } - public QuerqyDecorationAggregationBuilder(StreamInput in) throws IOException { + public QuerqyDecorationAggregationBuilder(final StreamInput in) throws IOException { super(in); } - protected QuerqyDecorationAggregationBuilder(QuerqyDecorationAggregationBuilder clone, Builder factoriesBuilder, Map metadata) { + protected QuerqyDecorationAggregationBuilder(final QuerqyDecorationAggregationBuilder clone, final Builder factoriesBuilder, final Map metadata) { super(clone, factoriesBuilder, metadata); } @Override - protected AggregationBuilder shallowCopy(Builder factoriesBuilder, Map metadata) { + protected AggregationBuilder shallowCopy(final Builder factoriesBuilder, final Map metadata) { return new QuerqyDecorationAggregationBuilder(this, factoriesBuilder, metadata); } @@ -44,18 +44,18 @@ public BucketCardinality bucketCardinality() { } @Override - protected AggregatorFactory doBuild(AggregationContext context, AggregatorFactory parent, Builder subFactoriesBuilder) + protected AggregatorFactory doBuild(final AggregationContext context, final AggregatorFactory parent, final Builder subFactoriesBuilder) throws IOException { return new QuerqyDecorationAggregatorFactory(name, context, parent, subFactoriesBuilder, metadata); } @Override - protected XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException { + protected XContentBuilder internalXContent(final XContentBuilder builder, final Params params) { return builder; } @Override - protected void doWriteTo(StreamOutput out) throws IOException { + protected void doWriteTo(final StreamOutput out) { // no state to write out } @@ -70,7 +70,7 @@ public int hashCode() { } @Override - public boolean equals(Object obj) { + public boolean equals(final Object obj) { if (this == obj) return true; if (obj == null || getClass() != obj.getClass()) return false; return super.equals(obj); diff --git a/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregator.java b/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregator.java index 3ebea0c..68f4c4f 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregator.java +++ b/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregator.java @@ -17,7 +17,7 @@ public final class QuerqyDecorationAggregator extends MetricsAggregator { final Set decorations; - public QuerqyDecorationAggregator(String name, AggregationContext context, Map metadata, Set decorations) + public QuerqyDecorationAggregator(final String name, final AggregationContext context, final Map metadata, final Set decorations) throws IOException { super(name, context, null, metadata); this.decorations = decorations; @@ -29,13 +29,13 @@ public ScoreMode scoreMode() { } @Override - protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException { + protected LeafBucketCollector getLeafCollector(final LeafReaderContext ctx, final LeafBucketCollector sub) { // No sub-aggregations return LeafBucketCollector.NO_OP_COLLECTOR; } @Override - public InternalAggregation buildAggregation(long l) { + public InternalAggregation buildAggregation(final long l) { StreamOutput.checkWriteable(decorations); return new InternalDecorationAggregation( name, diff --git a/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregatorFactory.java b/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregatorFactory.java index d5979e0..13e78d5 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregatorFactory.java +++ b/src/main/java/querqy/elasticsearch/aggregation/QuerqyDecorationAggregatorFactory.java @@ -19,17 +19,17 @@ public class QuerqyDecorationAggregatorFactory extends AggregatorFactory { public QuerqyDecorationAggregatorFactory( - String name, - AggregationContext context, - AggregatorFactory parent, - AggregatorFactories.Builder subFactories, - Map metadata + final String name, + final AggregationContext context, + final AggregatorFactory parent, + final AggregatorFactories.Builder subFactories, + final Map metadata ) throws IOException { super(name, context, parent, subFactories, metadata); } @Override - public Aggregator createInternal(Aggregator parent, CardinalityUpperBound cardinality, Map metadata) + public Aggregator createInternal(final Aggregator parent, final CardinalityUpperBound cardinality, final Map metadata) throws IOException { if (parent != null) { throw new IllegalArgumentException( @@ -44,24 +44,24 @@ public Aggregator createInternal(Aggregator parent, CardinalityUpperBound cardin if (cardinality != CardinalityUpperBound.ONE) { throw new AggregationExecutionException("Aggregation [" + name() + "] must have cardinality 1 but was [" + cardinality + "]"); } - Query query = context.subSearchContext() == null ? null : context.subSearchContext().query(); - Set> decoratedQueries = getDecoratedQueries(query); + final Query query = context.subSearchContext() == null ? null : context.subSearchContext().query(); + final Set> decoratedQueries = getDecoratedQueries(query); return new QuerqyDecorationAggregator(name, context, metadata, collectAllDecorations(decoratedQueries)); } - private Set collectAllDecorations(Set> decoratedQueries) { + private Set collectAllDecorations(final Set> decoratedQueries) { return decoratedQueries.stream().flatMap(decoratedQuery -> decoratedQuery.getDecorations().stream()).collect(Collectors.toSet()); } - private Set> getDecoratedQueries(Query query) { + private Set> getDecoratedQueries(final Query query) { if (query == null) { return Collections.emptySet(); } - Set> decoratedQueries = new HashSet<>(); + final Set> decoratedQueries = new HashSet<>(); query.visit(new QueryVisitor() { @Override - public void visitLeaf(Query query) { + public void visitLeaf(final Query query) { if (query instanceof DecoratedQuery) { decoratedQueries.add((DecoratedQuery) query); } From cb7fae9c838e3e73b488feaa79326d5444b1ccec Mon Sep 17 00:00:00 2001 From: paulb Date: Wed, 28 Feb 2024 10:08:16 +0100 Subject: [PATCH 14/15] Add final for local variable and parameter --- .../elasticsearch/aggregation/DecoratedQuery.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java b/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java index 0b50d69..014c227 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java +++ b/src/main/java/querqy/elasticsearch/aggregation/DecoratedQuery.java @@ -44,15 +44,15 @@ public String toString(final String field) { } @Override - public boolean equals(Object object) { + public boolean equals(final Object object) { if (!sameClassAs(object)) return false; - DecoratedQuery other = castObject(object); + final DecoratedQuery other = castObject(object); return isEqualQueriesAndDecorations(other); } - private boolean isEqualQueriesAndDecorations(DecoratedQuery other) { - Query otherQuery = other.getQuery(); - Set otherDecorations = other.getDecorations(); + private boolean isEqualQueriesAndDecorations(final DecoratedQuery other) { + final Query otherQuery = other.getQuery(); + final Set otherDecorations = other.getDecorations(); return getQuery().equals(otherQuery) && getDecorations().equals(otherDecorations); } From 5c09ccf5bca6c7eea49bf00443c9f507fb446583 Mon Sep 17 00:00:00 2001 From: paulb Date: Mon, 4 Mar 2024 09:00:08 +0100 Subject: [PATCH 15/15] Remove version check before V_7_8_0 --- .../InternalDecorationAggregation.java | 21 ++----------------- 1 file changed, 2 insertions(+), 19 deletions(-) diff --git a/src/main/java/querqy/elasticsearch/aggregation/InternalDecorationAggregation.java b/src/main/java/querqy/elasticsearch/aggregation/InternalDecorationAggregation.java index b723a9d..9b9535d 100644 --- a/src/main/java/querqy/elasticsearch/aggregation/InternalDecorationAggregation.java +++ b/src/main/java/querqy/elasticsearch/aggregation/InternalDecorationAggregation.java @@ -27,29 +27,12 @@ public class InternalDecorationAggregation extends InternalAggregation implement public InternalDecorationAggregation(final StreamInput in) throws IOException { super(in); - if (in.getVersion().before(Version.V_7_8_0)) { - aggregations = singletonList(in.readGenericValue()); - } else { - aggregations = in.readList(StreamInput::readGenericValue); - } + aggregations = in.readList(StreamInput::readGenericValue); } @Override protected void doWriteTo(final StreamOutput out) throws IOException { - if (out.getVersion().before(Version.V_7_8_0)) { - if (aggregations.size() > 1) { - /* - * If aggregations has more than one entry we're trying to - * serialize an unreduced aggregation. This *should* only - * happen when we're returning a scripted_metric over cross - * cluster search. - */ - throw new IllegalArgumentException("querqy doesn't support cross cluster search until 7.8.0"); - } - out.writeGenericValue(aggregations.get(0)); - } else { - out.writeCollection(aggregations, StreamOutput::writeGenericValue); - } + out.writeCollection(aggregations, StreamOutput::writeGenericValue); } @Override