Merge branch 'master' into feature/client_aggs_parsing
Conflicts: core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalHistogramTests.java core/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTermsTests.java core/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsTests.java core/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsTests.java
This commit is contained in:
commit
744b1afcb2
|
@ -127,7 +127,6 @@
|
|||
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]bulk[/\\]Retry.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]bulk[/\\]TransportBulkAction.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]bulk[/\\]TransportShardBulkAction.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]bulk[/\\]byscroll[/\\]AbstractAsyncBulkByScrollAction.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]delete[/\\]DeleteRequest.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]explain[/\\]TransportExplainAction.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]get[/\\]GetRequest.java" checks="LineLength" />
|
||||
|
|
|
@ -542,7 +542,7 @@ public abstract class TransportReplicationAction<
|
|||
new TransportChannelResponseHandler<>(logger, channel, extraMessage,
|
||||
() -> TransportResponse.Empty.INSTANCE);
|
||||
transportService.sendRequest(clusterService.localNode(), transportReplicaAction,
|
||||
new ConcreteShardRequest<>(request, targetAllocationID),
|
||||
new ConcreteReplicaRequest<>(request, targetAllocationID, globalCheckpoint),
|
||||
handler);
|
||||
}
|
||||
|
||||
|
|
|
@ -35,6 +35,7 @@ import org.elasticsearch.cli.UserException;
|
|||
import org.elasticsearch.common.PidFile;
|
||||
import org.elasticsearch.common.SuppressForbidden;
|
||||
import org.elasticsearch.common.inject.CreationException;
|
||||
import org.elasticsearch.common.logging.DeprecationLogger;
|
||||
import org.elasticsearch.common.logging.ESLoggerFactory;
|
||||
import org.elasticsearch.common.logging.LogConfigurator;
|
||||
import org.elasticsearch.common.logging.Loggers;
|
||||
|
@ -297,6 +298,7 @@ final class Bootstrap {
|
|||
throw new BootstrapException(e);
|
||||
}
|
||||
checkForCustomConfFile();
|
||||
checkConfigExtension(environment.configExtension());
|
||||
|
||||
if (environment.pidFile() != null) {
|
||||
try {
|
||||
|
@ -412,6 +414,14 @@ final class Bootstrap {
|
|||
}
|
||||
}
|
||||
|
||||
// pkg private for tests
|
||||
static void checkConfigExtension(String extension) {
|
||||
if (".yml".equals(extension) || ".json".equals(extension)) {
|
||||
final DeprecationLogger deprecationLogger = new DeprecationLogger(Loggers.getLogger(Bootstrap.class));
|
||||
deprecationLogger.deprecated("elasticsearch{} is deprecated; rename your configuration file to elasticsearch.yaml", extension);
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressForbidden(reason = "Allowed to exit explicitly in bootstrap phase")
|
||||
private static void exit(int status) {
|
||||
System.exit(status);
|
||||
|
|
|
@ -315,11 +315,11 @@ final class BootstrapChecks {
|
|||
static class MaxNumberOfThreadsCheck implements BootstrapCheck {
|
||||
|
||||
// this should be plenty for machines up to 256 cores
|
||||
private final long maxNumberOfThreadsThreshold = 1 << 12;
|
||||
private static final long MAX_NUMBER_OF_THREADS_THRESHOLD = 1 << 12;
|
||||
|
||||
@Override
|
||||
public boolean check() {
|
||||
return getMaxNumberOfThreads() != -1 && getMaxNumberOfThreads() < maxNumberOfThreadsThreshold;
|
||||
return getMaxNumberOfThreads() != -1 && getMaxNumberOfThreads() < MAX_NUMBER_OF_THREADS_THRESHOLD;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -329,7 +329,7 @@ final class BootstrapChecks {
|
|||
"max number of threads [%d] for user [%s] is too low, increase to at least [%d]",
|
||||
getMaxNumberOfThreads(),
|
||||
BootstrapInfo.getSystemProperties().get("user.name"),
|
||||
maxNumberOfThreadsThreshold);
|
||||
MAX_NUMBER_OF_THREADS_THRESHOLD);
|
||||
}
|
||||
|
||||
// visible for testing
|
||||
|
@ -369,11 +369,11 @@ final class BootstrapChecks {
|
|||
|
||||
static class MaxMapCountCheck implements BootstrapCheck {
|
||||
|
||||
private final long limit = 1 << 18;
|
||||
private static final long LIMIT = 1 << 18;
|
||||
|
||||
@Override
|
||||
public boolean check() {
|
||||
return getMaxMapCount() != -1 && getMaxMapCount() < limit;
|
||||
return getMaxMapCount() != -1 && getMaxMapCount() < LIMIT;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -382,7 +382,7 @@ final class BootstrapChecks {
|
|||
Locale.ROOT,
|
||||
"max virtual memory areas vm.max_map_count [%d] is too low, increase to at least [%d]",
|
||||
getMaxMapCount(),
|
||||
limit);
|
||||
LIMIT);
|
||||
}
|
||||
|
||||
// visible for testing
|
||||
|
|
|
@ -79,21 +79,21 @@ public abstract class ShapeBuilder extends ToXContentToBytes implements NamedWri
|
|||
/** It's possible that some geometries in a MULTI* shape might overlap. With the possible exception of GeometryCollection,
|
||||
* this normally isn't allowed.
|
||||
*/
|
||||
protected final boolean multiPolygonMayOverlap = false;
|
||||
protected static final boolean MULTI_POLYGON_MAY_OVERLAP = false;
|
||||
/** @see org.locationtech.spatial4j.shape.jts.JtsGeometry#validate() */
|
||||
protected final boolean autoValidateJtsGeometry = true;
|
||||
protected static final boolean AUTO_VALIDATE_JTS_GEOMETRY = true;
|
||||
/** @see org.locationtech.spatial4j.shape.jts.JtsGeometry#index() */
|
||||
protected final boolean autoIndexJtsGeometry = true;//may want to turn off once SpatialStrategy impls do it.
|
||||
protected static final boolean AUTO_INDEX_JTS_GEOMETRY = true;//may want to turn off once SpatialStrategy impls do it.
|
||||
|
||||
protected ShapeBuilder() {
|
||||
}
|
||||
|
||||
protected JtsGeometry jtsGeometry(Geometry geom) {
|
||||
//dateline180Check is false because ElasticSearch does it's own dateline wrapping
|
||||
JtsGeometry jtsGeometry = new JtsGeometry(geom, SPATIAL_CONTEXT, false, multiPolygonMayOverlap);
|
||||
if (autoValidateJtsGeometry)
|
||||
JtsGeometry jtsGeometry = new JtsGeometry(geom, SPATIAL_CONTEXT, false, MULTI_POLYGON_MAY_OVERLAP);
|
||||
if (AUTO_VALIDATE_JTS_GEOMETRY)
|
||||
jtsGeometry.validate();
|
||||
if (autoIndexJtsGeometry)
|
||||
if (AUTO_INDEX_JTS_GEOMETRY)
|
||||
jtsGeometry.index();
|
||||
return jtsGeometry;
|
||||
}
|
||||
|
|
|
@ -24,25 +24,25 @@ import java.security.GeneralSecurityException;
|
|||
import java.util.EnumSet;
|
||||
import java.util.Set;
|
||||
|
||||
import org.elasticsearch.common.Booleans;
|
||||
import org.elasticsearch.common.util.ArrayUtils;
|
||||
|
||||
|
||||
/**
|
||||
* A secure setting.
|
||||
*
|
||||
* This class allows access to settings from the Elasticsearch keystore.
|
||||
*/
|
||||
public abstract class SecureSetting<T> extends Setting<T> {
|
||||
|
||||
/** Determines whether legacy settings with sensitive values should be allowed. */
|
||||
private static final boolean ALLOW_INSECURE_SETTINGS = Booleans.parseBoolean(System.getProperty("es.allow_insecure_settings", "false"));
|
||||
|
||||
private static final Set<Property> ALLOWED_PROPERTIES = EnumSet.of(Property.Deprecated, Property.Shared);
|
||||
|
||||
private static final Property[] FIXED_PROPERTIES = {
|
||||
Property.NodeScope
|
||||
};
|
||||
|
||||
private static final Property[] LEGACY_PROPERTIES = {
|
||||
Property.NodeScope, Property.Deprecated, Property.Filtered
|
||||
};
|
||||
|
||||
private SecureSetting(String key, Property... properties) {
|
||||
super(key, (String)null, null, ArrayUtils.concat(properties, FIXED_PROPERTIES, Property.class));
|
||||
assert assertAllowedProperties(properties);
|
||||
|
@ -133,6 +133,23 @@ public abstract class SecureSetting<T> extends Setting<T> {
|
|||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* A setting which contains a sensitive string, but which for legacy reasons must be found outside secure settings.
|
||||
* @see #secureString(String, Setting, Property...)
|
||||
*/
|
||||
public static Setting<SecureString> insecureString(String name) {
|
||||
return new Setting<SecureString>(name, "", SecureString::new, Property.Deprecated, Property.Filtered, Property.NodeScope) {
|
||||
@Override
|
||||
public SecureString get(Settings settings) {
|
||||
if (ALLOW_INSECURE_SETTINGS == false && exists(settings)) {
|
||||
throw new IllegalArgumentException("Setting [" + name + "] is insecure, " +
|
||||
"but property [allow_insecure_settings] is not set");
|
||||
}
|
||||
return super.get(settings);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* A setting which contains a file. Reading the setting opens an input stream to the file.
|
||||
*
|
||||
|
|
|
@ -410,6 +410,7 @@ public final class ObjectParser<Value, Context> extends AbstractObjectParser<Val
|
|||
OBJECT_ARRAY(START_OBJECT, START_ARRAY),
|
||||
OBJECT_OR_BOOLEAN(START_OBJECT, VALUE_BOOLEAN),
|
||||
OBJECT_OR_STRING(START_OBJECT, VALUE_STRING),
|
||||
OBJECT_ARRAY_BOOLEAN_OR_STRING(START_OBJECT, START_ARRAY, VALUE_BOOLEAN, VALUE_STRING),
|
||||
OBJECT_ARRAY_OR_STRING(START_OBJECT, START_ARRAY, VALUE_STRING),
|
||||
VALUE(VALUE_BOOLEAN, VALUE_NULL, VALUE_EMBEDDED_OBJECT, VALUE_NUMBER, VALUE_STRING);
|
||||
|
||||
|
|
|
@ -71,6 +71,8 @@ public class Environment {
|
|||
|
||||
private final Settings settings;
|
||||
|
||||
private final String configExtension;
|
||||
|
||||
private final Path[] dataFiles;
|
||||
|
||||
private final Path[] dataWithClusterFiles;
|
||||
|
@ -102,6 +104,12 @@ public class Environment {
|
|||
private final Path tmpFile = PathUtils.get(System.getProperty("java.io.tmpdir"));
|
||||
|
||||
public Environment(Settings settings) {
|
||||
this(settings, null);
|
||||
}
|
||||
|
||||
// Note: Do not use this ctor, it is for correct deprecation logging in 5.5 and will be removed
|
||||
public Environment(Settings settings, String configExtension) {
|
||||
this.configExtension = configExtension;
|
||||
final Path homeFile;
|
||||
if (PATH_HOME_SETTING.exists(settings)) {
|
||||
homeFile = PathUtils.get(cleanPath(PATH_HOME_SETTING.get(settings)));
|
||||
|
@ -273,8 +281,14 @@ public class Environment {
|
|||
}
|
||||
}
|
||||
|
||||
/** Return then extension of the config file that was loaded, or*/
|
||||
public String configExtension() {
|
||||
return configExtension;
|
||||
}
|
||||
|
||||
// TODO: rename all these "file" methods to "dir"
|
||||
/**
|
||||
* The config location.
|
||||
* The config directory.
|
||||
*/
|
||||
public Path configFile() {
|
||||
return configFile;
|
||||
|
|
|
@ -101,7 +101,7 @@ public final class InnerHitBuilder extends ToXContentToBytes implements Writeabl
|
|||
} catch (IOException e) {
|
||||
throw new ParsingException(p.getTokenLocation(), "Could not parse inner _source definition", e);
|
||||
}
|
||||
}, SearchSourceBuilder._SOURCE_FIELD, ObjectParser.ValueType.OBJECT_OR_BOOLEAN);
|
||||
}, SearchSourceBuilder._SOURCE_FIELD, ObjectParser.ValueType.OBJECT_ARRAY_BOOLEAN_OR_STRING);
|
||||
PARSER.declareObject(InnerHitBuilder::setHighlightBuilder, (p, c) -> HighlightBuilder.fromXContent(c),
|
||||
SearchSourceBuilder.HIGHLIGHT_FIELD);
|
||||
PARSER.declareObject(InnerHitBuilder::setChildInnerHits, (p, c) -> {
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.action.bulk.byscroll;
|
||||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.ActionRequest;
|
||||
|
@ -355,7 +355,7 @@ public abstract class AbstractBulkByScrollRequest<Self extends AbstractBulkByScr
|
|||
/**
|
||||
* Build a new request for a slice of the parent request.
|
||||
*/
|
||||
protected abstract Self forSlice(TaskId slicingTask, SearchRequest slice);
|
||||
public abstract Self forSlice(TaskId slicingTask, SearchRequest slice);
|
||||
|
||||
/**
|
||||
* Setup a clone of this request with the information needed to process a slice of it.
|
|
@ -17,7 +17,7 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.action.bulk.byscroll;
|
||||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.elasticsearch.action.Action;
|
||||
import org.elasticsearch.action.ActionRequestBuilder;
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.elasticsearch.action.bulk.byscroll.AbstractBulkByScrollRequest;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
|
@ -20,8 +20,6 @@
|
|||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.elasticsearch.action.Action;
|
||||
import org.elasticsearch.action.bulk.byscroll.AbstractBulkByScrollRequestBuilder;
|
||||
import org.elasticsearch.action.bulk.byscroll.BulkByScrollResponse;
|
||||
import org.elasticsearch.action.search.SearchRequestBuilder;
|
||||
import org.elasticsearch.client.ElasticsearchClient;
|
||||
import org.elasticsearch.script.Script;
|
|
@ -17,11 +17,10 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.action.bulk.byscroll;
|
||||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.elasticsearch.action.ActionResponse;
|
||||
import org.elasticsearch.action.bulk.BulkItemResponse.Failure;
|
||||
import org.elasticsearch.action.bulk.byscroll.ScrollableHitSource.SearchFailure;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
|
@ -45,14 +44,14 @@ public class BulkByScrollResponse extends ActionResponse implements ToXContent {
|
|||
private TimeValue took;
|
||||
private BulkByScrollTask.Status status;
|
||||
private List<Failure> bulkFailures;
|
||||
private List<SearchFailure> searchFailures;
|
||||
private List<ScrollableHitSource.SearchFailure> searchFailures;
|
||||
private boolean timedOut;
|
||||
|
||||
public BulkByScrollResponse() {
|
||||
}
|
||||
|
||||
public BulkByScrollResponse(TimeValue took, BulkByScrollTask.Status status, List<Failure> bulkFailures,
|
||||
List<SearchFailure> searchFailures, boolean timedOut) {
|
||||
List<ScrollableHitSource.SearchFailure> searchFailures, boolean timedOut) {
|
||||
this.took = took;
|
||||
this.status = requireNonNull(status, "Null status not supported");
|
||||
this.bulkFailures = bulkFailures;
|
||||
|
@ -139,7 +138,7 @@ public class BulkByScrollResponse extends ActionResponse implements ToXContent {
|
|||
/**
|
||||
* All search failures.
|
||||
*/
|
||||
public List<SearchFailure> getSearchFailures() {
|
||||
public List<ScrollableHitSource.SearchFailure> getSearchFailures() {
|
||||
return searchFailures;
|
||||
}
|
||||
|
||||
|
@ -166,7 +165,7 @@ public class BulkByScrollResponse extends ActionResponse implements ToXContent {
|
|||
took = new TimeValue(in);
|
||||
status = new BulkByScrollTask.Status(in);
|
||||
bulkFailures = in.readList(Failure::new);
|
||||
searchFailures = in.readList(SearchFailure::new);
|
||||
searchFailures = in.readList(ScrollableHitSource.SearchFailure::new);
|
||||
timedOut = in.readBoolean();
|
||||
}
|
||||
|
||||
|
@ -181,7 +180,7 @@ public class BulkByScrollResponse extends ActionResponse implements ToXContent {
|
|||
failure.toXContent(builder, params);
|
||||
builder.endObject();
|
||||
}
|
||||
for (SearchFailure failure: searchFailures) {
|
||||
for (ScrollableHitSource.SearchFailure failure: searchFailures) {
|
||||
failure.toXContent(builder, params);
|
||||
}
|
||||
builder.endArray();
|
||||
|
@ -199,4 +198,4 @@ public class BulkByScrollResponse extends ActionResponse implements ToXContent {
|
|||
builder.append(",search_failures=").append(getSearchFailures().subList(0, min(3, getSearchFailures().size())));
|
||||
return builder.append(']').toString();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -17,7 +17,7 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.action.bulk.byscroll;
|
||||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.Version;
|
|
@ -17,7 +17,7 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.action.bulk.byscroll;
|
||||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.apache.logging.log4j.message.ParameterizedMessage;
|
|
@ -20,8 +20,6 @@
|
|||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.elasticsearch.action.Action;
|
||||
import org.elasticsearch.action.bulk.byscroll.BulkByScrollResponse;
|
||||
import org.elasticsearch.action.bulk.byscroll.DeleteByQueryRequest;
|
||||
import org.elasticsearch.client.ElasticsearchClient;
|
||||
|
||||
public class DeleteByQueryAction extends Action<DeleteByQueryRequest, BulkByScrollResponse, DeleteByQueryRequestBuilder> {
|
|
@ -17,7 +17,7 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.action.bulk.byscroll;
|
||||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.elasticsearch.action.ActionRequestValidationException;
|
||||
import org.elasticsearch.action.IndicesRequest;
|
||||
|
@ -81,7 +81,7 @@ public class DeleteByQueryRequest extends AbstractBulkByScrollRequest<DeleteByQu
|
|||
}
|
||||
|
||||
@Override
|
||||
protected DeleteByQueryRequest forSlice(TaskId slicingTask, SearchRequest slice) {
|
||||
public DeleteByQueryRequest forSlice(TaskId slicingTask, SearchRequest slice) {
|
||||
return doForSlice(new DeleteByQueryRequest(slice, false), slicingTask);
|
||||
}
|
||||
|
|
@ -20,9 +20,6 @@
|
|||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.elasticsearch.action.Action;
|
||||
import org.elasticsearch.action.bulk.byscroll.AbstractBulkByScrollRequestBuilder;
|
||||
import org.elasticsearch.action.bulk.byscroll.BulkByScrollResponse;
|
||||
import org.elasticsearch.action.bulk.byscroll.DeleteByQueryRequest;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequestBuilder;
|
||||
import org.elasticsearch.client.ElasticsearchClient;
|
|
@ -17,7 +17,7 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.action.bulk.byscroll;
|
||||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.common.collect.Tuple;
|
||||
|
@ -104,7 +104,7 @@ public class ParentBulkByScrollTask extends BulkByScrollTask {
|
|||
/**
|
||||
* Record a failure from a slice and respond to the listener if the request is finished.
|
||||
*/
|
||||
void onSliceFailure(ActionListener<BulkByScrollResponse> listener, int sliceId, Exception e) {
|
||||
public void onSliceFailure(ActionListener<BulkByScrollResponse> listener, int sliceId, Exception e) {
|
||||
results.setOnce(sliceId, new Result(sliceId, e));
|
||||
recordSliceCompletionAndRespondIfAllDone(listener);
|
||||
// TODO cancel when a slice fails?
|
|
@ -26,7 +26,6 @@ import org.elasticsearch.action.search.SearchRequest;
|
|||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.lucene.uid.Versions;
|
||||
import org.elasticsearch.index.reindex.remote.RemoteInfo;
|
||||
import org.elasticsearch.tasks.TaskId;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -128,7 +127,7 @@ public class ReindexRequest extends AbstractBulkIndexByScrollRequest<ReindexRequ
|
|||
}
|
||||
|
||||
@Override
|
||||
protected ReindexRequest forSlice(TaskId slicingTask, SearchRequest slice) {
|
||||
public ReindexRequest forSlice(TaskId slicingTask, SearchRequest slice) {
|
||||
ReindexRequest sliced = doForSlice(new ReindexRequest(slice, destination, false), slicingTask);
|
||||
sliced.setRemoteInfo(remoteInfo);
|
||||
return sliced;
|
|
@ -20,13 +20,11 @@
|
|||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.elasticsearch.action.Action;
|
||||
import org.elasticsearch.action.bulk.byscroll.BulkByScrollResponse;
|
||||
import org.elasticsearch.action.index.IndexAction;
|
||||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequestBuilder;
|
||||
import org.elasticsearch.client.ElasticsearchClient;
|
||||
import org.elasticsearch.index.reindex.remote.RemoteInfo;
|
||||
|
||||
public class ReindexRequestBuilder extends
|
||||
AbstractBulkIndexByScrollRequestBuilder<ReindexRequest, ReindexRequestBuilder> {
|
|
@ -17,7 +17,7 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.index.reindex.remote;
|
||||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.common.Nullable;
|
|
@ -17,7 +17,7 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.action.bulk.byscroll;
|
||||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
|
@ -111,7 +111,7 @@ public abstract class ScrollableHitSource {
|
|||
/**
|
||||
* Set the id of the last scroll. Used for debugging.
|
||||
*/
|
||||
final void setScroll(String scrollId) {
|
||||
public final void setScroll(String scrollId) {
|
||||
this.scrollId.set(scrollId);
|
||||
}
|
||||
|
|
@ -17,7 +17,7 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.action.bulk.byscroll;
|
||||
package org.elasticsearch.index.reindex;
|
||||
|
||||
/**
|
||||
* Implemented by {@link BulkByScrollTask} and {@link BulkByScrollTask.Status} to consistently implement
|
|
@ -20,7 +20,6 @@
|
|||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.elasticsearch.action.Action;
|
||||
import org.elasticsearch.action.bulk.byscroll.BulkByScrollResponse;
|
||||
import org.elasticsearch.client.ElasticsearchClient;
|
||||
|
||||
public class UpdateByQueryAction extends
|
|
@ -71,7 +71,7 @@ public class UpdateByQueryRequest extends AbstractBulkIndexByScrollRequest<Updat
|
|||
}
|
||||
|
||||
@Override
|
||||
protected UpdateByQueryRequest forSlice(TaskId slicingTask, SearchRequest slice) {
|
||||
public UpdateByQueryRequest forSlice(TaskId slicingTask, SearchRequest slice) {
|
||||
UpdateByQueryRequest request = doForSlice(new UpdateByQueryRequest(slice, false), slicingTask);
|
||||
request.setPipeline(pipeline);
|
||||
return request;
|
|
@ -20,7 +20,6 @@
|
|||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.elasticsearch.action.Action;
|
||||
import org.elasticsearch.action.bulk.byscroll.BulkByScrollResponse;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequestBuilder;
|
||||
import org.elasticsearch.client.ElasticsearchClient;
|
|
@ -17,7 +17,7 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.action.bulk.byscroll;
|
||||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.common.logging.ESLoggerFactory;
|
||||
|
@ -115,15 +115,15 @@ public class WorkingBulkByScrollTask extends BulkByScrollTask implements Success
|
|||
return timeValueNanos(max(0, delayed.future.getDelay(TimeUnit.NANOSECONDS)));
|
||||
}
|
||||
|
||||
void setTotal(long totalHits) {
|
||||
public void setTotal(long totalHits) {
|
||||
total.set(totalHits);
|
||||
}
|
||||
|
||||
void countBatch() {
|
||||
public void countBatch() {
|
||||
batch.incrementAndGet();
|
||||
}
|
||||
|
||||
void countNoop() {
|
||||
public void countNoop() {
|
||||
noops.incrementAndGet();
|
||||
}
|
||||
|
||||
|
@ -132,7 +132,7 @@ public class WorkingBulkByScrollTask extends BulkByScrollTask implements Success
|
|||
return created.get();
|
||||
}
|
||||
|
||||
void countCreated() {
|
||||
public void countCreated() {
|
||||
created.incrementAndGet();
|
||||
}
|
||||
|
||||
|
@ -141,7 +141,7 @@ public class WorkingBulkByScrollTask extends BulkByScrollTask implements Success
|
|||
return updated.get();
|
||||
}
|
||||
|
||||
void countUpdated() {
|
||||
public void countUpdated() {
|
||||
updated.incrementAndGet();
|
||||
}
|
||||
|
||||
|
@ -150,15 +150,15 @@ public class WorkingBulkByScrollTask extends BulkByScrollTask implements Success
|
|||
return deleted.get();
|
||||
}
|
||||
|
||||
void countDeleted() {
|
||||
public void countDeleted() {
|
||||
deleted.incrementAndGet();
|
||||
}
|
||||
|
||||
void countVersionConflict() {
|
||||
public void countVersionConflict() {
|
||||
versionConflicts.incrementAndGet();
|
||||
}
|
||||
|
||||
void countBulkRetry() {
|
||||
public void countBulkRetry() {
|
||||
bulkRetries.incrementAndGet();
|
||||
}
|
||||
|
||||
|
@ -174,8 +174,8 @@ public class WorkingBulkByScrollTask extends BulkByScrollTask implements Success
|
|||
* Schedule prepareBulkRequestRunnable to run after some delay. This is where throttling plugs into reindexing so the request can be
|
||||
* rescheduled over and over again.
|
||||
*/
|
||||
void delayPrepareBulkRequest(ThreadPool threadPool, TimeValue lastBatchStartTime, int lastBatchSize,
|
||||
AbstractRunnable prepareBulkRequestRunnable) {
|
||||
public void delayPrepareBulkRequest(ThreadPool threadPool, TimeValue lastBatchStartTime, int lastBatchSize,
|
||||
AbstractRunnable prepareBulkRequestRunnable) {
|
||||
// Synchronize so we are less likely to schedule the same request twice.
|
||||
synchronized (delayedPrepareBulkRequestReference) {
|
||||
TimeValue delay = throttleWaitTime(lastBatchStartTime, timeValueNanos(System.nanoTime()), lastBatchSize);
|
||||
|
@ -184,7 +184,7 @@ public class WorkingBulkByScrollTask extends BulkByScrollTask implements Success
|
|||
}
|
||||
}
|
||||
|
||||
TimeValue throttleWaitTime(TimeValue lastBatchStartTime, TimeValue now, int lastBatchSize) {
|
||||
public TimeValue throttleWaitTime(TimeValue lastBatchStartTime, TimeValue now, int lastBatchSize) {
|
||||
long earliestNextBatchStartTime = now.nanos() + (long) perfectlyThrottledBatchTime(lastBatchSize);
|
||||
return timeValueNanos(max(0, earliestNextBatchStartTime - System.nanoTime()));
|
||||
}
|
|
@ -18,6 +18,7 @@
|
|||
*/
|
||||
|
||||
/**
|
||||
* Infrastructure for actions that modify documents based on the results of a scrolling query.
|
||||
* Infrastructure for actions that modify documents based on the results of a scrolling query
|
||||
* like reindex, update by query or delete by query.
|
||||
*/
|
||||
package org.elasticsearch.action.bulk.byscroll;
|
||||
package org.elasticsearch.index.reindex;
|
|
@ -541,6 +541,9 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> if an fsync is required to ensure durability of the translogs operations or it's metadata.
|
||||
*/
|
||||
public boolean syncNeeded() {
|
||||
try (ReleasableLock lock = readLock.acquire()) {
|
||||
return current.syncNeeded();
|
||||
|
|
|
@ -209,7 +209,8 @@ public class TranslogWriter extends BaseTranslogReader implements Closeable {
|
|||
}
|
||||
|
||||
/**
|
||||
* returns true if there are buffered ops
|
||||
* Returns <code>true</code> if there are buffered operations that have not been flushed and fsynced to disk or if the latest global
|
||||
* checkpoint has not yet been fsynced
|
||||
*/
|
||||
public boolean syncNeeded() {
|
||||
return totalOffset != lastSyncedCheckpoint.offset || globalCheckpointSupplier.getAsLong() != lastSyncedCheckpoint.globalCheckpoint;
|
||||
|
|
|
@ -70,7 +70,7 @@ public class RecoveryTarget extends AbstractRefCounted implements RecoveryTarget
|
|||
|
||||
private static final AtomicLong idGenerator = new AtomicLong();
|
||||
|
||||
private final String RECOVERY_PREFIX = "recovery.";
|
||||
private static final String RECOVERY_PREFIX = "recovery.";
|
||||
|
||||
private final ShardId shardId;
|
||||
private final long recoveryId;
|
||||
|
|
|
@ -19,14 +19,6 @@
|
|||
|
||||
package org.elasticsearch.node;
|
||||
|
||||
import org.elasticsearch.cli.Terminal;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.collect.Tuple;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.settings.SettingsException;
|
||||
import org.elasticsearch.env.Environment;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
|
@ -38,6 +30,14 @@ import java.util.Map;
|
|||
import java.util.Set;
|
||||
import java.util.function.Function;
|
||||
|
||||
import org.elasticsearch.cli.Terminal;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.collect.Tuple;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.settings.SettingsException;
|
||||
import org.elasticsearch.env.Environment;
|
||||
|
||||
import static org.elasticsearch.common.Strings.cleanPath;
|
||||
|
||||
public class InternalSettingsPreparer {
|
||||
|
@ -116,7 +116,8 @@ public class InternalSettingsPreparer {
|
|||
|
||||
// we put back the path.logs so we can use it in the logging configuration file
|
||||
output.put(Environment.PATH_LOGS_SETTING.getKey(), cleanPath(environment.logsFile().toAbsolutePath().toString()));
|
||||
return new Environment(output.build());
|
||||
String configExtension = foundSuffixes.isEmpty() ? null : foundSuffixes.iterator().next();
|
||||
return new Environment(output.build(), configExtension);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -19,18 +19,30 @@
|
|||
|
||||
package org.elasticsearch.script;
|
||||
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.elasticsearch.common.lucene.ScorerAware;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* A per-segment {@link SearchScript}.
|
||||
*
|
||||
* This is effectively a functional interface, requiring at least implementing {@link #runAsDouble()}.
|
||||
*/
|
||||
public interface LeafSearchScript extends ScorerAware, ExecutableScript {
|
||||
|
||||
void setDocument(int doc);
|
||||
/**
|
||||
* Set the document this script will process next.
|
||||
*/
|
||||
default void setDocument(int doc) {}
|
||||
|
||||
void setSource(Map<String, Object> source);
|
||||
@Override
|
||||
default void setScorer(Scorer scorer) {}
|
||||
|
||||
/**
|
||||
* Set the source for the current document.
|
||||
*/
|
||||
default void setSource(Map<String, Object> source) {}
|
||||
|
||||
/**
|
||||
* Sets per-document aggregation {@code _value}.
|
||||
|
@ -44,8 +56,23 @@ public interface LeafSearchScript extends ScorerAware, ExecutableScript {
|
|||
setNextVar("_value", value);
|
||||
}
|
||||
|
||||
long runAsLong();
|
||||
@Override
|
||||
default void setNextVar(String field, Object value) {}
|
||||
|
||||
/**
|
||||
* Return the result as a long. This is used by aggregation scripts over long fields.
|
||||
*/
|
||||
default long runAsLong() {
|
||||
throw new UnsupportedOperationException("runAsLong is not implemented");
|
||||
}
|
||||
|
||||
@Override
|
||||
default Object run() {
|
||||
return runAsDouble();
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the result as a double. This is the main use case of search script, used for document scoring.
|
||||
*/
|
||||
double runAsDouble();
|
||||
|
||||
}
|
||||
|
|
|
@ -38,7 +38,9 @@ public interface ScriptEngine extends Closeable {
|
|||
/**
|
||||
* The extension for file scripts in this language.
|
||||
*/
|
||||
String getExtension();
|
||||
default String getExtension() {
|
||||
return getType();
|
||||
}
|
||||
|
||||
/**
|
||||
* Compiles a script.
|
||||
|
|
|
@ -0,0 +1,127 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package org.elasticsearch.search.aggregations;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.common.xcontent.ToXContentObject;
|
||||
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation.Bucket;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationPath;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* {@link Bucket} Ordering strategy.
|
||||
*/
|
||||
public abstract class BucketOrder implements ToXContentObject, Writeable {
|
||||
|
||||
/**
|
||||
* Creates a bucket ordering strategy that sorts buckets by their document counts (ascending or descending).
|
||||
*
|
||||
* @param asc direction to sort by: {@code true} for ascending, {@code false} for descending.
|
||||
*/
|
||||
public static BucketOrder count(boolean asc) {
|
||||
return asc ? InternalOrder.COUNT_ASC : InternalOrder.COUNT_DESC;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a bucket ordering strategy that sorts buckets by their keys (ascending or descending). This may be
|
||||
* used as a tie-breaker to avoid non-deterministic ordering.
|
||||
*
|
||||
* @param asc direction to sort by: {@code true} for ascending, {@code false} for descending.
|
||||
*/
|
||||
public static BucketOrder key(boolean asc) {
|
||||
return asc ? InternalOrder.KEY_ASC : InternalOrder.KEY_DESC;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a bucket ordering strategy which sorts buckets based on a single-valued sub-aggregation.
|
||||
*
|
||||
* @param path path to the sub-aggregation to sort on.
|
||||
* @param asc direction to sort by: {@code true} for ascending, {@code false} for descending.
|
||||
* @see AggregationPath
|
||||
*/
|
||||
public static BucketOrder aggregation(String path, boolean asc) {
|
||||
return new InternalOrder.Aggregation(path, asc);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a bucket ordering strategy which sorts buckets based on a metric from a multi-valued sub-aggregation.
|
||||
*
|
||||
* @param path path to the sub-aggregation to sort on.
|
||||
* @param metricName name of the value of the multi-value metric to sort on.
|
||||
* @param asc direction to sort by: {@code true} for ascending, {@code false} for descending.
|
||||
* @see AggregationPath
|
||||
*/
|
||||
public static BucketOrder aggregation(String path, String metricName, boolean asc) {
|
||||
return new InternalOrder.Aggregation(path + "." + metricName, asc);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a bucket ordering strategy which sorts buckets based on multiple criteria. A tie-breaker may be added to
|
||||
* avoid non-deterministic ordering.
|
||||
*
|
||||
* @param orders a list of {@link BucketOrder} objects to sort on, in order of priority.
|
||||
*/
|
||||
public static BucketOrder compound(List<BucketOrder> orders) {
|
||||
return new InternalOrder.CompoundOrder(orders);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a bucket ordering strategy which sorts buckets based on multiple criteria. A tie-breaker may be added to
|
||||
* avoid non-deterministic ordering.
|
||||
*
|
||||
* @param orders a list of {@link BucketOrder} parameters to sort on, in order of priority.
|
||||
*/
|
||||
public static BucketOrder compound(BucketOrder... orders) {
|
||||
return compound(Arrays.asList(orders));
|
||||
}
|
||||
|
||||
/**
|
||||
* @return A comparator for the bucket based on the given aggregator. The comparator is used in two phases:
|
||||
* <p>
|
||||
* - aggregation phase, where each shard builds a list of buckets to be sent to the coordinating node.
|
||||
* In this phase, the passed in aggregator will be the aggregator that aggregates the buckets on the
|
||||
* shard level.
|
||||
* <p>
|
||||
* - reduce phase, where the coordinating node gathers all the buckets from all the shards and reduces them
|
||||
* to a final bucket list. In this case, the passed in aggregator will be {@code null}.
|
||||
*/
|
||||
public abstract Comparator<Bucket> comparator(Aggregator aggregator);
|
||||
|
||||
/**
|
||||
* @return unique internal ID used for reading/writing this order from/to a stream.
|
||||
* @see InternalOrder.Streams
|
||||
*/
|
||||
abstract byte id();
|
||||
|
||||
@Override
|
||||
public abstract int hashCode();
|
||||
|
||||
@Override
|
||||
public abstract boolean equals(Object obj);
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
InternalOrder.Streams.writeOrder(this, out);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,595 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package org.elasticsearch.search.aggregations;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.common.ParsingException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.logging.DeprecationLogger;
|
||||
import org.elasticsearch.common.logging.Loggers;
|
||||
import org.elasticsearch.common.util.Comparators;
|
||||
import org.elasticsearch.common.xcontent.XContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.index.query.QueryParseContext;
|
||||
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation.Bucket;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationPath;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* Implementations for {@link Bucket} ordering strategies.
|
||||
*/
|
||||
public class InternalOrder extends BucketOrder {
|
||||
|
||||
private final byte id;
|
||||
private final String key;
|
||||
protected final boolean asc;
|
||||
protected final Comparator<Bucket> comparator;
|
||||
|
||||
/**
|
||||
* Creates an ordering strategy that sorts {@link Bucket}s by some property.
|
||||
*
|
||||
* @param id unique ID for this ordering strategy.
|
||||
* @param key key of the property to sort on.
|
||||
* @param asc direction to sort by: {@code true} for ascending, {@code false} for descending.
|
||||
* @param comparator determines how buckets will be ordered.
|
||||
*/
|
||||
public InternalOrder(byte id, String key, boolean asc, Comparator<Bucket> comparator) {
|
||||
this.id = id;
|
||||
this.key = key;
|
||||
this.asc = asc;
|
||||
this.comparator = comparator;
|
||||
}
|
||||
|
||||
@Override
|
||||
byte id() {
|
||||
return id;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator<Bucket> comparator(Aggregator aggregator) {
|
||||
return comparator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder.startObject().field(key, asc ? "asc" : "desc").endObject();
|
||||
}
|
||||
|
||||
/**
|
||||
* Validate a bucket ordering strategy for an {@link Aggregator}.
|
||||
*
|
||||
* @param order bucket ordering strategy to sort on.
|
||||
* @param aggregator aggregator to sort.
|
||||
* @return unmodified bucket ordering strategy.
|
||||
* @throws AggregationExecutionException if validation fails
|
||||
*/
|
||||
public static BucketOrder validate(BucketOrder order, Aggregator aggregator) throws AggregationExecutionException {
|
||||
if (order instanceof CompoundOrder) {
|
||||
for (BucketOrder innerOrder : ((CompoundOrder) order).orderElements) {
|
||||
validate(innerOrder, aggregator);
|
||||
}
|
||||
} else if (order instanceof Aggregation) {
|
||||
((Aggregation) order).path().validate(aggregator);
|
||||
}
|
||||
return order;
|
||||
}
|
||||
|
||||
/**
|
||||
* {@link Bucket} ordering strategy to sort by a sub-aggregation.
|
||||
*/
|
||||
public static class Aggregation extends InternalOrder {
|
||||
|
||||
static final byte ID = 0;
|
||||
|
||||
/**
|
||||
* Create a new ordering strategy to sort by a sub-aggregation.
|
||||
*
|
||||
* @param path path to the sub-aggregation to sort on.
|
||||
* @param asc direction to sort by: {@code true} for ascending, {@code false} for descending.
|
||||
* @see AggregationPath
|
||||
*/
|
||||
Aggregation(String path, boolean asc) {
|
||||
super(ID, path, asc, new AggregationComparator(path, asc));
|
||||
}
|
||||
|
||||
/**
|
||||
* @return parsed path to the sub-aggregation to sort on.
|
||||
*/
|
||||
public AggregationPath path() {
|
||||
return ((AggregationComparator) comparator).path;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator<Bucket> comparator(Aggregator aggregator) {
|
||||
if (aggregator instanceof TermsAggregator) {
|
||||
// Internal Optimization for terms aggregation to avoid constructing buckets for ordering purposes
|
||||
return ((TermsAggregator) aggregator).bucketComparator(path(), asc);
|
||||
}
|
||||
return comparator;
|
||||
}
|
||||
|
||||
/**
|
||||
* {@link Bucket} ordering strategy to sort by a sub-aggregation.
|
||||
*/
|
||||
static class AggregationComparator implements Comparator<Bucket> {
|
||||
|
||||
private final AggregationPath path;
|
||||
private final boolean asc;
|
||||
|
||||
/**
|
||||
* Create a new {@link Bucket} ordering strategy to sort by a sub-aggregation.
|
||||
*
|
||||
* @param path path to the sub-aggregation to sort on.
|
||||
* @param asc direction to sort by: {@code true} for ascending, {@code false} for descending.
|
||||
* @see AggregationPath
|
||||
*/
|
||||
AggregationComparator(String path, boolean asc) {
|
||||
this.asc = asc;
|
||||
this.path = AggregationPath.parse(path);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compare(Bucket b1, Bucket b2) {
|
||||
double v1 = path.resolveValue(b1);
|
||||
double v2 = path.resolveValue(b2);
|
||||
return Comparators.compareDiscardNaN(v1, v2, asc);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* {@link Bucket} ordering strategy to sort by multiple criteria.
|
||||
*/
|
||||
public static class CompoundOrder extends BucketOrder {
|
||||
|
||||
static final byte ID = -1;
|
||||
|
||||
final List<BucketOrder> orderElements;
|
||||
|
||||
/**
|
||||
* Create a new ordering strategy to sort by multiple criteria. A tie-breaker may be added to avoid
|
||||
* non-deterministic ordering.
|
||||
*
|
||||
* @param compoundOrder a list of {@link BucketOrder}s to sort on, in order of priority.
|
||||
*/
|
||||
CompoundOrder(List<BucketOrder> compoundOrder) {
|
||||
this(compoundOrder, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new ordering strategy to sort by multiple criteria.
|
||||
*
|
||||
* @param compoundOrder a list of {@link BucketOrder}s to sort on, in order of priority.
|
||||
* @param absoluteOrdering {@code true} to add a tie-breaker to avoid non-deterministic ordering if needed,
|
||||
* {@code false} otherwise.
|
||||
*/
|
||||
CompoundOrder(List<BucketOrder> compoundOrder, boolean absoluteOrdering) {
|
||||
this.orderElements = new LinkedList<>(compoundOrder);
|
||||
BucketOrder lastElement = null;
|
||||
for (BucketOrder order : orderElements) {
|
||||
if (order instanceof CompoundOrder) {
|
||||
throw new IllegalArgumentException("nested compound order not supported");
|
||||
}
|
||||
lastElement = order;
|
||||
}
|
||||
if (absoluteOrdering && isKeyOrder(lastElement) == false) {
|
||||
// add key order ascending as a tie-breaker to avoid non-deterministic ordering
|
||||
// if all user provided comparators return 0.
|
||||
this.orderElements.add(KEY_ASC);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
byte id() {
|
||||
return ID;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return unmodifiable list of {@link BucketOrder}s to sort on.
|
||||
*/
|
||||
public List<BucketOrder> orderElements() {
|
||||
return Collections.unmodifiableList(orderElements);
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startArray();
|
||||
for (BucketOrder order : orderElements) {
|
||||
order.toXContent(builder, params);
|
||||
}
|
||||
return builder.endArray();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator<Bucket> comparator(Aggregator aggregator) {
|
||||
return new CompoundOrderComparator(orderElements, aggregator);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(orderElements);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (obj == null) {
|
||||
return false;
|
||||
}
|
||||
if (getClass() != obj.getClass()) {
|
||||
return false;
|
||||
}
|
||||
CompoundOrder other = (CompoundOrder) obj;
|
||||
return Objects.equals(orderElements, other.orderElements);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@code Comparator} for sorting buckets by multiple criteria.
|
||||
*/
|
||||
static class CompoundOrderComparator implements Comparator<Bucket> {
|
||||
|
||||
private List<BucketOrder> compoundOrder;
|
||||
private Aggregator aggregator;
|
||||
|
||||
/**
|
||||
* Create a new {@code Comparator} for sorting buckets by multiple criteria.
|
||||
*
|
||||
* @param compoundOrder a list of {@link BucketOrder}s to sort on, in order of priority.
|
||||
* @param aggregator {@link BucketOrder#comparator(Aggregator)}
|
||||
*/
|
||||
CompoundOrderComparator(List<BucketOrder> compoundOrder, Aggregator aggregator) {
|
||||
this.compoundOrder = compoundOrder;
|
||||
this.aggregator = aggregator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compare(Bucket b1, Bucket b2) {
|
||||
int result = 0;
|
||||
for (Iterator<BucketOrder> itr = compoundOrder.iterator(); itr.hasNext() && result == 0; ) {
|
||||
result = itr.next().comparator(aggregator).compare(b1, b2);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static final byte COUNT_DESC_ID = 1;
|
||||
private static final byte COUNT_ASC_ID = 2;
|
||||
private static final byte KEY_DESC_ID = 3;
|
||||
private static final byte KEY_ASC_ID = 4;
|
||||
|
||||
/**
|
||||
* Order by the (higher) count of each bucket.
|
||||
*/
|
||||
static final InternalOrder COUNT_DESC = new InternalOrder(COUNT_DESC_ID, "_count", false, comparingCounts().reversed());
|
||||
|
||||
/**
|
||||
* Order by the (lower) count of each bucket.
|
||||
*/
|
||||
static final InternalOrder COUNT_ASC = new InternalOrder(COUNT_ASC_ID, "_count", true, comparingCounts());
|
||||
|
||||
/**
|
||||
* Order by the key of each bucket descending.
|
||||
*/
|
||||
static final InternalOrder KEY_DESC = new InternalOrder(KEY_DESC_ID, "_key", false, comparingKeys().reversed());
|
||||
|
||||
/**
|
||||
* Order by the key of each bucket ascending.
|
||||
*/
|
||||
static final InternalOrder KEY_ASC = new InternalOrder(KEY_ASC_ID, "_key", true, comparingKeys());
|
||||
|
||||
/**
|
||||
* @return compare by {@link Bucket#getDocCount()}.
|
||||
*/
|
||||
private static Comparator<Bucket> comparingCounts() {
|
||||
return Comparator.comparingLong(Bucket::getDocCount);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return compare by {@link Bucket#getKey()} from the appropriate implementation.
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
private static Comparator<Bucket> comparingKeys() {
|
||||
return (b1, b2) -> {
|
||||
if (b1 instanceof KeyComparable) {
|
||||
return ((KeyComparable) b1).compareKey(b2);
|
||||
}
|
||||
throw new IllegalStateException("Unexpected order bucket class [" + b1.getClass() + "]");
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Determine if the ordering strategy is sorting on bucket count descending.
|
||||
*
|
||||
* @param order bucket ordering strategy to check.
|
||||
* @return {@code true} if the ordering strategy is sorting on bucket count descending, {@code false} otherwise.
|
||||
*/
|
||||
public static boolean isCountDesc(BucketOrder order) {
|
||||
return isOrder(order, COUNT_DESC);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determine if the ordering strategy is sorting on bucket key (ascending or descending).
|
||||
*
|
||||
* @param order bucket ordering strategy to check.
|
||||
* @return {@code true} if the ordering strategy is sorting on bucket key, {@code false} otherwise.
|
||||
*/
|
||||
public static boolean isKeyOrder(BucketOrder order) {
|
||||
return isOrder(order, KEY_ASC) || isOrder(order, KEY_DESC);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determine if the ordering strategy is sorting on bucket key ascending.
|
||||
*
|
||||
* @param order bucket ordering strategy to check.
|
||||
* @return {@code true} if the ordering strategy is sorting on bucket key ascending, {@code false} otherwise.
|
||||
*/
|
||||
public static boolean isKeyAsc(BucketOrder order) {
|
||||
return isOrder(order, KEY_ASC);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determine if the ordering strategy is sorting on bucket key descending.
|
||||
*
|
||||
* @param order bucket ordering strategy to check.
|
||||
* @return {@code true} if the ordering strategy is sorting on bucket key descending, {@code false} otherwise.
|
||||
*/
|
||||
public static boolean isKeyDesc(BucketOrder order) {
|
||||
return isOrder(order, KEY_DESC);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determine if the ordering strategy matches the expected one.
|
||||
*
|
||||
* @param order bucket ordering strategy to check. If this is a {@link CompoundOrder} the first element will be
|
||||
* check instead.
|
||||
* @param expected expected bucket ordering strategy.
|
||||
* @return {@code true} if the order matches, {@code false} otherwise.
|
||||
*/
|
||||
private static boolean isOrder(BucketOrder order, BucketOrder expected) {
|
||||
if (order == expected) {
|
||||
return true;
|
||||
} else if (order instanceof CompoundOrder) {
|
||||
// check if its a compound order with the first element that matches
|
||||
List<BucketOrder> orders = ((CompoundOrder) order).orderElements;
|
||||
if (orders.size() >= 1) {
|
||||
return isOrder(orders.get(0), expected);
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Contains logic for reading/writing {@link BucketOrder} from/to streams.
|
||||
*/
|
||||
public static class Streams {
|
||||
|
||||
/**
|
||||
* Read a {@link BucketOrder} from a {@link StreamInput}.
|
||||
*
|
||||
* @param in stream with order data to read.
|
||||
* @return order read from the stream
|
||||
* @throws IOException on error reading from the stream.
|
||||
*/
|
||||
public static BucketOrder readOrder(StreamInput in) throws IOException {
|
||||
byte id = in.readByte();
|
||||
switch (id) {
|
||||
case COUNT_DESC_ID: return COUNT_DESC;
|
||||
case COUNT_ASC_ID: return COUNT_ASC;
|
||||
case KEY_DESC_ID: return KEY_DESC;
|
||||
case KEY_ASC_ID: return KEY_ASC;
|
||||
case Aggregation.ID:
|
||||
boolean asc = in.readBoolean();
|
||||
String key = in.readString();
|
||||
return new Aggregation(key, asc);
|
||||
case CompoundOrder.ID:
|
||||
int size = in.readVInt();
|
||||
List<BucketOrder> compoundOrder = new ArrayList<>(size);
|
||||
for (int i = 0; i < size; i++) {
|
||||
compoundOrder.add(Streams.readOrder(in));
|
||||
}
|
||||
return new CompoundOrder(compoundOrder, false);
|
||||
default:
|
||||
throw new RuntimeException("unknown order id [" + id + "]");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* ONLY FOR HISTOGRAM ORDER: Backwards compatibility logic to read a {@link BucketOrder} from a {@link StreamInput}.
|
||||
*
|
||||
* @param in stream with order data to read.
|
||||
* @param bwcOrderFlag {@code true} to check {@code in.readBoolean()} in the backwards compat logic before reading
|
||||
* the order. {@code false} to skip this flag (order always present).
|
||||
* @return order read from the stream
|
||||
* @throws IOException on error reading from the stream.
|
||||
*/
|
||||
public static BucketOrder readHistogramOrder(StreamInput in, boolean bwcOrderFlag) throws IOException {
|
||||
if (in.getVersion().onOrAfter(Version.V_6_0_0_alpha2_UNRELEASED)) {
|
||||
return Streams.readOrder(in);
|
||||
} else { // backwards compat logic
|
||||
if (bwcOrderFlag == false || in.readBoolean()) {
|
||||
// translate the old histogram order IDs to the new order objects
|
||||
byte id = in.readByte();
|
||||
switch (id) {
|
||||
case 1: return KEY_ASC;
|
||||
case 2: return KEY_DESC;
|
||||
case 3: return COUNT_ASC;
|
||||
case 4: return COUNT_DESC;
|
||||
case 0: // aggregation order stream logic is backwards compatible
|
||||
boolean asc = in.readBoolean();
|
||||
String key = in.readString();
|
||||
return new Aggregation(key, asc);
|
||||
default: // not expecting compound order ID
|
||||
throw new RuntimeException("unknown histogram order id [" + id + "]");
|
||||
}
|
||||
} else { // default to _key asc if no order specified
|
||||
return KEY_ASC;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Write a {@link BucketOrder} to a {@link StreamOutput}.
|
||||
*
|
||||
* @param order order to write to the stream.
|
||||
* @param out stream to write the order to.
|
||||
* @throws IOException on error writing to the stream.
|
||||
*/
|
||||
public static void writeOrder(BucketOrder order, StreamOutput out) throws IOException {
|
||||
out.writeByte(order.id());
|
||||
if (order instanceof Aggregation) {
|
||||
Aggregation aggregationOrder = (Aggregation) order;
|
||||
out.writeBoolean(aggregationOrder.asc);
|
||||
out.writeString(aggregationOrder.path().toString());
|
||||
} else if (order instanceof CompoundOrder) {
|
||||
CompoundOrder compoundOrder = (CompoundOrder) order;
|
||||
out.writeVInt(compoundOrder.orderElements.size());
|
||||
for (BucketOrder innerOrder : compoundOrder.orderElements) {
|
||||
innerOrder.writeTo(out);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* ONLY FOR HISTOGRAM ORDER: Backwards compatibility logic to write a {@link BucketOrder} to a stream.
|
||||
*
|
||||
* @param order order to write to the stream.
|
||||
* @param out stream to write the order to.
|
||||
* @param bwcOrderFlag {@code true} to always {@code out.writeBoolean(true)} for the backwards compat logic before
|
||||
* writing the order. {@code false} to skip this flag.
|
||||
* @throws IOException on error writing to the stream.
|
||||
*/
|
||||
public static void writeHistogramOrder(BucketOrder order, StreamOutput out, boolean bwcOrderFlag) throws IOException {
|
||||
if (out.getVersion().onOrAfter(Version.V_6_0_0_alpha2_UNRELEASED)) {
|
||||
order.writeTo(out);
|
||||
} else { // backwards compat logic
|
||||
if(bwcOrderFlag) { // need to add flag that determines if order exists
|
||||
out.writeBoolean(true); // order always exists
|
||||
}
|
||||
if (order instanceof CompoundOrder) {
|
||||
// older versions do not support histogram compound order; the best we can do here is use the first order.
|
||||
order = ((CompoundOrder) order).orderElements.get(0);
|
||||
}
|
||||
if (order instanceof Aggregation) {
|
||||
// aggregation order stream logic is backwards compatible
|
||||
order.writeTo(out);
|
||||
} else {
|
||||
// convert the new order IDs to the old histogram order IDs.
|
||||
byte id;
|
||||
switch (order.id()) {
|
||||
case COUNT_DESC_ID: id = 4; break;
|
||||
case COUNT_ASC_ID: id = 3; break;
|
||||
case KEY_DESC_ID: id = 2; break;
|
||||
case KEY_ASC_ID: id = 1; break;
|
||||
default: throw new RuntimeException("unknown order id [" + order.id() + "]");
|
||||
}
|
||||
out.writeByte(id);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Contains logic for parsing a {@link BucketOrder} from a {@link XContentParser}.
|
||||
*/
|
||||
public static class Parser {
|
||||
|
||||
private static final DeprecationLogger DEPRECATION_LOGGER =
|
||||
new DeprecationLogger(Loggers.getLogger(Parser.class));
|
||||
|
||||
/**
|
||||
* Parse a {@link BucketOrder} from {@link XContent}.
|
||||
*
|
||||
* @param parser for parsing {@link XContent} that contains the order.
|
||||
* @param context parsing context.
|
||||
* @return bucket ordering strategy
|
||||
* @throws IOException on error a {@link XContent} parsing error.
|
||||
*/
|
||||
public static BucketOrder parseOrderParam(XContentParser parser, QueryParseContext context) throws IOException {
|
||||
XContentParser.Token token;
|
||||
String orderKey = null;
|
||||
boolean orderAsc = false;
|
||||
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
|
||||
if (token == XContentParser.Token.FIELD_NAME) {
|
||||
orderKey = parser.currentName();
|
||||
} else if (token == XContentParser.Token.VALUE_STRING) {
|
||||
String dir = parser.text();
|
||||
if ("asc".equalsIgnoreCase(dir)) {
|
||||
orderAsc = true;
|
||||
} else if ("desc".equalsIgnoreCase(dir)) {
|
||||
orderAsc = false;
|
||||
} else {
|
||||
throw new ParsingException(parser.getTokenLocation(),
|
||||
"Unknown order direction [" + dir + "]");
|
||||
}
|
||||
} else {
|
||||
throw new ParsingException(parser.getTokenLocation(),
|
||||
"Unexpected token [" + token + "] for [order]");
|
||||
}
|
||||
}
|
||||
if (orderKey == null) {
|
||||
throw new ParsingException(parser.getTokenLocation(),
|
||||
"Must specify at least one field for [order]");
|
||||
}
|
||||
// _term and _time order deprecated in 6.0; replaced by _key
|
||||
if ("_term".equals(orderKey) || "_time".equals(orderKey)) {
|
||||
DEPRECATION_LOGGER.deprecated("Deprecated aggregation order key [{}] used, replaced by [_key]", orderKey);
|
||||
}
|
||||
switch (orderKey) {
|
||||
case "_term":
|
||||
case "_time":
|
||||
case "_key":
|
||||
return orderAsc ? KEY_ASC : KEY_DESC;
|
||||
case "_count":
|
||||
return orderAsc ? COUNT_ASC : COUNT_DESC;
|
||||
default: // assume all other orders are sorting on a sub-aggregation. Validation occurs later.
|
||||
return aggregation(orderKey, orderAsc);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(id, key, asc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (obj == null) {
|
||||
return false;
|
||||
}
|
||||
if (getClass() != obj.getClass()) {
|
||||
return false;
|
||||
}
|
||||
InternalOrder other = (InternalOrder) obj;
|
||||
return Objects.equals(id, other.id)
|
||||
&& Objects.equals(key, other.key)
|
||||
&& Objects.equals(asc, other.asc);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,41 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package org.elasticsearch.search.aggregations;
|
||||
|
||||
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation.Bucket;
|
||||
|
||||
/**
|
||||
* Defines behavior for comparing {@link Bucket#getKey() bucket keys} to imposes a total ordering
|
||||
* of buckets of the same type.
|
||||
*
|
||||
* @param <T> {@link Bucket} of the same type that also implements {@link KeyComparable}.
|
||||
* @see BucketOrder#key(boolean)
|
||||
*/
|
||||
public interface KeyComparable<T extends Bucket & KeyComparable<T>> {
|
||||
|
||||
/**
|
||||
* Compare this {@link Bucket}s {@link Bucket#getKey() key} with another bucket.
|
||||
*
|
||||
* @param other the bucket that contains the key to compare to.
|
||||
* @return a negative integer, zero, or a positive integer as this buckets key
|
||||
* is less than, equal to, or greater than the other buckets key.
|
||||
* @see Comparable#compareTo(Object)
|
||||
*/
|
||||
int compareKey(T other);
|
||||
}
|
|
@ -19,12 +19,10 @@
|
|||
|
||||
package org.elasticsearch.search.aggregations.bucket;
|
||||
|
||||
import org.elasticsearch.common.util.Comparators;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.search.aggregations.Aggregation;
|
||||
import org.elasticsearch.search.aggregations.Aggregations;
|
||||
import org.elasticsearch.search.aggregations.HasAggregations;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationPath;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
|
@ -58,31 +56,6 @@ public interface MultiBucketsAggregation extends Aggregation {
|
|||
@Override
|
||||
Aggregations getAggregations();
|
||||
|
||||
class SubAggregationComparator<B extends Bucket> implements java.util.Comparator<B> {
|
||||
|
||||
private final AggregationPath path;
|
||||
private final boolean asc;
|
||||
|
||||
public SubAggregationComparator(String expression, boolean asc) {
|
||||
this.asc = asc;
|
||||
this.path = AggregationPath.parse(expression);
|
||||
}
|
||||
|
||||
public boolean asc() {
|
||||
return asc;
|
||||
}
|
||||
|
||||
public AggregationPath path() {
|
||||
return path;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compare(B b1, B b2) {
|
||||
double v1 = path.resolveValue(b1);
|
||||
double v2 = path.resolveValue(b2);
|
||||
return Comparators.compareDiscardNaN(v1, v2, asc);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.elasticsearch.search.aggregations.bucket.histogram;
|
||||
|
||||
import org.elasticsearch.common.ParsingException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.rounding.DateTimeUnit;
|
||||
|
@ -28,10 +27,12 @@ import org.elasticsearch.common.unit.TimeValue;
|
|||
import org.elasticsearch.common.xcontent.ObjectParser;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.common.xcontent.XContentParser.Token;
|
||||
import org.elasticsearch.index.query.QueryParseContext;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.search.aggregations.InternalOrder;
|
||||
import org.elasticsearch.search.aggregations.InternalOrder.CompoundOrder;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
||||
|
@ -44,6 +45,7 @@ import org.elasticsearch.search.internal.SearchContext;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
|
@ -113,8 +115,8 @@ public class DateHistogramAggregationBuilder
|
|||
PARSER.declareField(DateHistogramAggregationBuilder::extendedBounds, parser -> ExtendedBounds.PARSER.apply(parser, null),
|
||||
ExtendedBounds.EXTENDED_BOUNDS_FIELD, ObjectParser.ValueType.OBJECT);
|
||||
|
||||
PARSER.declareField(DateHistogramAggregationBuilder::order, DateHistogramAggregationBuilder::parseOrder,
|
||||
Histogram.ORDER_FIELD, ObjectParser.ValueType.OBJECT);
|
||||
PARSER.declareObjectArray(DateHistogramAggregationBuilder::order, InternalOrder.Parser::parseOrderParam,
|
||||
Histogram.ORDER_FIELD);
|
||||
}
|
||||
|
||||
public static DateHistogramAggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
|
||||
|
@ -125,7 +127,7 @@ public class DateHistogramAggregationBuilder
|
|||
private DateHistogramInterval dateHistogramInterval;
|
||||
private long offset = 0;
|
||||
private ExtendedBounds extendedBounds;
|
||||
private InternalOrder order = (InternalOrder) Histogram.Order.KEY_ASC;
|
||||
private BucketOrder order = BucketOrder.key(true);
|
||||
private boolean keyed = false;
|
||||
private long minDocCount = 0;
|
||||
|
||||
|
@ -137,9 +139,7 @@ public class DateHistogramAggregationBuilder
|
|||
/** Read from a stream, for internal use only. */
|
||||
public DateHistogramAggregationBuilder(StreamInput in) throws IOException {
|
||||
super(in, ValuesSourceType.NUMERIC, ValueType.DATE);
|
||||
if (in.readBoolean()) {
|
||||
order = InternalOrder.Streams.readOrder(in);
|
||||
}
|
||||
order = InternalOrder.Streams.readHistogramOrder(in, true);
|
||||
keyed = in.readBoolean();
|
||||
minDocCount = in.readVLong();
|
||||
interval = in.readLong();
|
||||
|
@ -150,11 +150,7 @@ public class DateHistogramAggregationBuilder
|
|||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
boolean hasOrder = order != null;
|
||||
out.writeBoolean(hasOrder);
|
||||
if (hasOrder) {
|
||||
InternalOrder.Streams.writeOrder(order, out);
|
||||
}
|
||||
InternalOrder.Streams.writeHistogramOrder(order, out, true);
|
||||
out.writeBoolean(keyed);
|
||||
out.writeVLong(minDocCount);
|
||||
out.writeLong(interval);
|
||||
|
@ -244,17 +240,34 @@ public class DateHistogramAggregationBuilder
|
|||
}
|
||||
|
||||
/** Return the order to use to sort buckets of this histogram. */
|
||||
public Histogram.Order order() {
|
||||
public BucketOrder order() {
|
||||
return order;
|
||||
}
|
||||
|
||||
/** Set a new order on this builder and return the builder so that calls
|
||||
* can be chained. */
|
||||
public DateHistogramAggregationBuilder order(Histogram.Order order) {
|
||||
* can be chained. A tie-breaker may be added to avoid non-deterministic ordering. */
|
||||
public DateHistogramAggregationBuilder order(BucketOrder order) {
|
||||
if (order == null) {
|
||||
throw new IllegalArgumentException("[order] must not be null: [" + name + "]");
|
||||
}
|
||||
this.order = (InternalOrder) order;
|
||||
if(order instanceof CompoundOrder || InternalOrder.isKeyOrder(order)) {
|
||||
this.order = order; // if order already contains a tie-breaker we are good to go
|
||||
} else { // otherwise add a tie-breaker by using a compound order
|
||||
this.order = BucketOrder.compound(order);
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the order in which the buckets will be returned. A tie-breaker may be added to avoid non-deterministic
|
||||
* ordering.
|
||||
*/
|
||||
public DateHistogramAggregationBuilder order(List<BucketOrder> orders) {
|
||||
if (orders == null) {
|
||||
throw new IllegalArgumentException("[orders] must not be null: [" + name + "]");
|
||||
}
|
||||
// if the list only contains one order use that to avoid inconsistent xcontent
|
||||
order(orders.size() > 1 ? BucketOrder.compound(orders) : orders.get(0));
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -370,35 +383,4 @@ public class DateHistogramAggregationBuilder
|
|||
&& Objects.equals(offset, other.offset)
|
||||
&& Objects.equals(extendedBounds, other.extendedBounds);
|
||||
}
|
||||
|
||||
// similar to the parsing oh histogram orders, but also accepts _time as an alias for _key
|
||||
private static InternalOrder parseOrder(XContentParser parser, QueryParseContext context) throws IOException {
|
||||
InternalOrder order = null;
|
||||
Token token;
|
||||
String currentFieldName = null;
|
||||
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
|
||||
if (token == XContentParser.Token.FIELD_NAME) {
|
||||
currentFieldName = parser.currentName();
|
||||
} else if (token == XContentParser.Token.VALUE_STRING) {
|
||||
String dir = parser.text();
|
||||
boolean asc = "asc".equals(dir);
|
||||
if (!asc && !"desc".equals(dir)) {
|
||||
throw new ParsingException(parser.getTokenLocation(), "Unknown order direction: [" + dir
|
||||
+ "]. Should be either [asc] or [desc]");
|
||||
}
|
||||
order = resolveOrder(currentFieldName, asc);
|
||||
}
|
||||
}
|
||||
return order;
|
||||
}
|
||||
|
||||
static InternalOrder resolveOrder(String key, boolean asc) {
|
||||
if ("_key".equals(key) || "_time".equals(key)) {
|
||||
return (InternalOrder) (asc ? InternalOrder.KEY_ASC : InternalOrder.KEY_DESC);
|
||||
}
|
||||
if ("_count".equals(key)) {
|
||||
return (InternalOrder) (asc ? InternalOrder.COUNT_ASC : InternalOrder.COUNT_DESC);
|
||||
}
|
||||
return new InternalOrder.Aggregation(key, asc);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,6 +33,8 @@ import org.elasticsearch.search.aggregations.LeafBucketCollector;
|
|||
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
|
||||
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.search.aggregations.InternalOrder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
|
@ -53,7 +55,7 @@ class DateHistogramAggregator extends BucketsAggregator {
|
|||
private final ValuesSource.Numeric valuesSource;
|
||||
private final DocValueFormat formatter;
|
||||
private final Rounding rounding;
|
||||
private final InternalOrder order;
|
||||
private final BucketOrder order;
|
||||
private final boolean keyed;
|
||||
|
||||
private final long minDocCount;
|
||||
|
@ -62,7 +64,7 @@ class DateHistogramAggregator extends BucketsAggregator {
|
|||
private final LongHash bucketOrds;
|
||||
private long offset;
|
||||
|
||||
DateHistogramAggregator(String name, AggregatorFactories factories, Rounding rounding, long offset, InternalOrder order,
|
||||
DateHistogramAggregator(String name, AggregatorFactories factories, Rounding rounding, long offset, BucketOrder order,
|
||||
boolean keyed,
|
||||
long minDocCount, @Nullable ExtendedBounds extendedBounds, @Nullable ValuesSource.Numeric valuesSource,
|
||||
DocValueFormat formatter, SearchContext aggregationContext,
|
||||
|
@ -71,7 +73,7 @@ class DateHistogramAggregator extends BucketsAggregator {
|
|||
super(name, factories, aggregationContext, parent, pipelineAggregators, metaData);
|
||||
this.rounding = rounding;
|
||||
this.offset = offset;
|
||||
this.order = order;
|
||||
this.order = InternalOrder.validate(order, this);;
|
||||
this.keyed = keyed;
|
||||
this.minDocCount = minDocCount;
|
||||
this.extendedBounds = extendedBounds;
|
||||
|
@ -131,7 +133,7 @@ class DateHistogramAggregator extends BucketsAggregator {
|
|||
}
|
||||
|
||||
// the contract of the histogram aggregation is that shards must return buckets ordered by key in ascending order
|
||||
CollectionUtil.introSort(buckets, InternalOrder.KEY_ASC.comparator());
|
||||
CollectionUtil.introSort(buckets, BucketOrder.key(true).comparator(this));
|
||||
|
||||
// value source will be null for unmapped fields
|
||||
InternalDateHistogram.EmptyBucketInfo emptyBucketInfo = minDocCount == 0
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.elasticsearch.search.aggregations.Aggregator;
|
|||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
|
@ -40,14 +41,14 @@ public final class DateHistogramAggregatorFactory
|
|||
private final DateHistogramInterval dateHistogramInterval;
|
||||
private final long interval;
|
||||
private final long offset;
|
||||
private final InternalOrder order;
|
||||
private final BucketOrder order;
|
||||
private final boolean keyed;
|
||||
private final long minDocCount;
|
||||
private final ExtendedBounds extendedBounds;
|
||||
private Rounding rounding;
|
||||
|
||||
public DateHistogramAggregatorFactory(String name, ValuesSourceConfig<Numeric> config, long interval,
|
||||
DateHistogramInterval dateHistogramInterval, long offset, InternalOrder order, boolean keyed, long minDocCount,
|
||||
DateHistogramInterval dateHistogramInterval, long offset, BucketOrder order, boolean keyed, long minDocCount,
|
||||
Rounding rounding, ExtendedBounds extendedBounds, SearchContext context, AggregatorFactory<?> parent,
|
||||
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
|
||||
super(name, config, context, parent, subFactoriesBuilder, metaData);
|
||||
|
|
|
@ -19,10 +19,8 @@
|
|||
package org.elasticsearch.search.aggregations.bucket.histogram;
|
||||
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation;
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
|
@ -50,83 +48,4 @@ public interface Histogram extends MultiBucketsAggregation {
|
|||
@Override
|
||||
List<? extends Bucket> getBuckets();
|
||||
|
||||
/**
|
||||
* A strategy defining the order in which the buckets in this histogram are ordered.
|
||||
*/
|
||||
abstract class Order implements ToXContent {
|
||||
|
||||
private static int compareKey(Histogram.Bucket b1, Histogram.Bucket b2) {
|
||||
if (b1 instanceof InternalHistogram.Bucket) {
|
||||
return Double.compare(((InternalHistogram.Bucket) b1).key, ((InternalHistogram.Bucket) b2).key);
|
||||
} else if (b1 instanceof InternalDateHistogram.Bucket) {
|
||||
return Long.compare(((InternalDateHistogram.Bucket) b1).key, ((InternalDateHistogram.Bucket) b2).key);
|
||||
} else {
|
||||
throw new IllegalStateException("Unexpected impl: " + b1.getClass());
|
||||
}
|
||||
}
|
||||
|
||||
public static final Order KEY_ASC = new InternalOrder((byte) 1, "_key", true, new Comparator<Histogram.Bucket>() {
|
||||
@Override
|
||||
public int compare(Histogram.Bucket b1, Histogram.Bucket b2) {
|
||||
return compareKey(b1, b2);
|
||||
}
|
||||
});
|
||||
|
||||
public static final Order KEY_DESC = new InternalOrder((byte) 2, "_key", false, new Comparator<Histogram.Bucket>() {
|
||||
@Override
|
||||
public int compare(Histogram.Bucket b1, Histogram.Bucket b2) {
|
||||
return compareKey(b2, b1);
|
||||
}
|
||||
});
|
||||
|
||||
public static final Order COUNT_ASC = new InternalOrder((byte) 3, "_count", true, new Comparator<Histogram.Bucket>() {
|
||||
@Override
|
||||
public int compare(Histogram.Bucket b1, Histogram.Bucket b2) {
|
||||
int cmp = Long.compare(b1.getDocCount(), b2.getDocCount());
|
||||
if (cmp == 0) {
|
||||
cmp = compareKey(b1, b2);
|
||||
}
|
||||
return cmp;
|
||||
}
|
||||
});
|
||||
|
||||
|
||||
public static final Order COUNT_DESC = new InternalOrder((byte) 4, "_count", false, new Comparator<Histogram.Bucket>() {
|
||||
@Override
|
||||
public int compare(Histogram.Bucket b1, Histogram.Bucket b2) {
|
||||
int cmp = Long.compare(b2.getDocCount(), b1.getDocCount());
|
||||
if (cmp == 0) {
|
||||
cmp = compareKey(b1, b2);
|
||||
}
|
||||
return cmp;
|
||||
}
|
||||
});
|
||||
|
||||
/**
|
||||
* Creates a bucket ordering strategy that sorts buckets based on a single-valued calc sug-aggregation
|
||||
*
|
||||
* @param path the name of the aggregation
|
||||
* @param asc The direction of the order (ascending or descending)
|
||||
*/
|
||||
public static Order aggregation(String path, boolean asc) {
|
||||
return new InternalOrder.Aggregation(path, asc);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a bucket ordering strategy that sorts buckets based on a multi-valued calc sug-aggregation
|
||||
*
|
||||
* @param aggregationName the name of the aggregation
|
||||
* @param valueName The name of the value of the multi-value get by which the sorting will be applied
|
||||
* @param asc The direction of the order (ascending or descending)
|
||||
*/
|
||||
public static Order aggregation(String aggregationName, String valueName, boolean asc) {
|
||||
return new InternalOrder.Aggregation(aggregationName + "." + valueName, asc);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The bucket comparator by which the order will be applied.
|
||||
*/
|
||||
abstract Comparator<Histogram.Bucket> comparator();
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,16 +20,16 @@
|
|||
package org.elasticsearch.search.aggregations.bucket.histogram;
|
||||
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.ParsingException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.ObjectParser;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.common.xcontent.XContentParser.Token;
|
||||
import org.elasticsearch.index.query.QueryParseContext;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.search.aggregations.InternalOrder;
|
||||
import org.elasticsearch.search.aggregations.InternalOrder.CompoundOrder;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
||||
|
@ -41,6 +41,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceType;
|
|||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
|
@ -75,8 +76,8 @@ public class HistogramAggregationBuilder
|
|||
histogram.extendedBounds(extendedBounds[0], extendedBounds[1]);
|
||||
}, parser -> EXTENDED_BOUNDS_PARSER.apply(parser, null), ExtendedBounds.EXTENDED_BOUNDS_FIELD, ObjectParser.ValueType.OBJECT);
|
||||
|
||||
PARSER.declareField(HistogramAggregationBuilder::order, HistogramAggregationBuilder::parseOrder,
|
||||
Histogram.ORDER_FIELD, ObjectParser.ValueType.OBJECT);
|
||||
PARSER.declareObjectArray(HistogramAggregationBuilder::order, InternalOrder.Parser::parseOrderParam,
|
||||
Histogram.ORDER_FIELD);
|
||||
}
|
||||
|
||||
public static HistogramAggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
|
||||
|
@ -87,7 +88,7 @@ public class HistogramAggregationBuilder
|
|||
private double offset = 0;
|
||||
private double minBound = Double.POSITIVE_INFINITY;
|
||||
private double maxBound = Double.NEGATIVE_INFINITY;
|
||||
private InternalOrder order = (InternalOrder) Histogram.Order.KEY_ASC;
|
||||
private BucketOrder order = BucketOrder.key(true);
|
||||
private boolean keyed = false;
|
||||
private long minDocCount = 0;
|
||||
|
||||
|
@ -99,9 +100,7 @@ public class HistogramAggregationBuilder
|
|||
/** Read from a stream, for internal use only. */
|
||||
public HistogramAggregationBuilder(StreamInput in) throws IOException {
|
||||
super(in, ValuesSourceType.NUMERIC, ValueType.DOUBLE);
|
||||
if (in.readBoolean()) {
|
||||
order = InternalOrder.Streams.readOrder(in);
|
||||
}
|
||||
order = InternalOrder.Streams.readHistogramOrder(in, true);
|
||||
keyed = in.readBoolean();
|
||||
minDocCount = in.readVLong();
|
||||
interval = in.readDouble();
|
||||
|
@ -112,11 +111,7 @@ public class HistogramAggregationBuilder
|
|||
|
||||
@Override
|
||||
protected void innerWriteTo(StreamOutput out) throws IOException {
|
||||
boolean hasOrder = order != null;
|
||||
out.writeBoolean(hasOrder);
|
||||
if (hasOrder) {
|
||||
InternalOrder.Streams.writeOrder(order, out);
|
||||
}
|
||||
InternalOrder.Streams.writeHistogramOrder(order, out, true);
|
||||
out.writeBoolean(keyed);
|
||||
out.writeVLong(minDocCount);
|
||||
out.writeDouble(interval);
|
||||
|
@ -185,17 +180,34 @@ public class HistogramAggregationBuilder
|
|||
}
|
||||
|
||||
/** Return the order to use to sort buckets of this histogram. */
|
||||
public Histogram.Order order() {
|
||||
public BucketOrder order() {
|
||||
return order;
|
||||
}
|
||||
|
||||
/** Set a new order on this builder and return the builder so that calls
|
||||
* can be chained. */
|
||||
public HistogramAggregationBuilder order(Histogram.Order order) {
|
||||
* can be chained. A tie-breaker may be added to avoid non-deterministic ordering. */
|
||||
public HistogramAggregationBuilder order(BucketOrder order) {
|
||||
if (order == null) {
|
||||
throw new IllegalArgumentException("[order] must not be null: [" + name + "]");
|
||||
}
|
||||
this.order = (InternalOrder) order;
|
||||
if(order instanceof CompoundOrder || InternalOrder.isKeyOrder(order)) {
|
||||
this.order = order; // if order already contains a tie-breaker we are good to go
|
||||
} else { // otherwise add a tie-breaker by using a compound order
|
||||
this.order = BucketOrder.compound(order);
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the order in which the buckets will be returned. A tie-breaker may be added to avoid non-deterministic
|
||||
* ordering.
|
||||
*/
|
||||
public HistogramAggregationBuilder order(List<BucketOrder> orders) {
|
||||
if (orders == null) {
|
||||
throw new IllegalArgumentException("[orders] must not be null: [" + name + "]");
|
||||
}
|
||||
// if the list only contains one order use that to avoid inconsistent xcontent
|
||||
order(orders.size() > 1 ? BucketOrder.compound(orders) : orders.get(0));
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -286,34 +298,4 @@ public class HistogramAggregationBuilder
|
|||
&& Objects.equals(minBound, other.minBound)
|
||||
&& Objects.equals(maxBound, other.maxBound);
|
||||
}
|
||||
|
||||
private static InternalOrder parseOrder(XContentParser parser, QueryParseContext context) throws IOException {
|
||||
InternalOrder order = null;
|
||||
Token token;
|
||||
String currentFieldName = null;
|
||||
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
|
||||
if (token == XContentParser.Token.FIELD_NAME) {
|
||||
currentFieldName = parser.currentName();
|
||||
} else if (token == XContentParser.Token.VALUE_STRING) {
|
||||
String dir = parser.text();
|
||||
boolean asc = "asc".equals(dir);
|
||||
if (!asc && !"desc".equals(dir)) {
|
||||
throw new ParsingException(parser.getTokenLocation(), "Unknown order direction: [" + dir
|
||||
+ "]. Should be either [asc] or [desc]");
|
||||
}
|
||||
order = resolveOrder(currentFieldName, asc);
|
||||
}
|
||||
}
|
||||
return order;
|
||||
}
|
||||
|
||||
static InternalOrder resolveOrder(String key, boolean asc) {
|
||||
if ("_key".equals(key)) {
|
||||
return (InternalOrder) (asc ? InternalOrder.KEY_ASC : InternalOrder.KEY_DESC);
|
||||
}
|
||||
if ("_count".equals(key)) {
|
||||
return (InternalOrder) (asc ? InternalOrder.COUNT_ASC : InternalOrder.COUNT_DESC);
|
||||
}
|
||||
return new InternalOrder.Aggregation(key, asc);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -34,6 +34,8 @@ import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
|
|||
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram.EmptyBucketInfo;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.search.aggregations.InternalOrder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
|
@ -54,7 +56,7 @@ class HistogramAggregator extends BucketsAggregator {
|
|||
private final ValuesSource.Numeric valuesSource;
|
||||
private final DocValueFormat formatter;
|
||||
private final double interval, offset;
|
||||
private final InternalOrder order;
|
||||
private final BucketOrder order;
|
||||
private final boolean keyed;
|
||||
private final long minDocCount;
|
||||
private final double minBound, maxBound;
|
||||
|
@ -62,7 +64,7 @@ class HistogramAggregator extends BucketsAggregator {
|
|||
private final LongHash bucketOrds;
|
||||
|
||||
HistogramAggregator(String name, AggregatorFactories factories, double interval, double offset,
|
||||
InternalOrder order, boolean keyed, long minDocCount, double minBound, double maxBound,
|
||||
BucketOrder order, boolean keyed, long minDocCount, double minBound, double maxBound,
|
||||
@Nullable ValuesSource.Numeric valuesSource, DocValueFormat formatter,
|
||||
SearchContext context, Aggregator parent,
|
||||
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
|
||||
|
@ -73,7 +75,7 @@ class HistogramAggregator extends BucketsAggregator {
|
|||
}
|
||||
this.interval = interval;
|
||||
this.offset = offset;
|
||||
this.order = order;
|
||||
this.order = InternalOrder.validate(order, this);
|
||||
this.keyed = keyed;
|
||||
this.minDocCount = minDocCount;
|
||||
this.minBound = minBound;
|
||||
|
@ -137,7 +139,7 @@ class HistogramAggregator extends BucketsAggregator {
|
|||
}
|
||||
|
||||
// the contract of the histogram aggregation is that shards must return buckets ordered by key in ascending order
|
||||
CollectionUtil.introSort(buckets, InternalOrder.KEY_ASC.comparator());
|
||||
CollectionUtil.introSort(buckets, BucketOrder.key(true).comparator(this));
|
||||
|
||||
EmptyBucketInfo emptyBucketInfo = null;
|
||||
if (minDocCount == 0) {
|
||||
|
|
|
@ -23,6 +23,7 @@ import org.elasticsearch.search.aggregations.Aggregator;
|
|||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
|
@ -36,13 +37,13 @@ import java.util.Map;
|
|||
public final class HistogramAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource.Numeric, HistogramAggregatorFactory> {
|
||||
|
||||
private final double interval, offset;
|
||||
private final InternalOrder order;
|
||||
private final BucketOrder order;
|
||||
private final boolean keyed;
|
||||
private final long minDocCount;
|
||||
private final double minBound, maxBound;
|
||||
|
||||
HistogramAggregatorFactory(String name, ValuesSourceConfig<Numeric> config, double interval, double offset,
|
||||
InternalOrder order, boolean keyed, long minDocCount, double minBound, double maxBound,
|
||||
BucketOrder order, boolean keyed, long minDocCount, double minBound, double maxBound,
|
||||
SearchContext context, AggregatorFactory<?> parent,
|
||||
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
|
||||
super(name, config, context, parent, subFactoriesBuilder, metaData);
|
||||
|
@ -80,4 +81,4 @@ public final class HistogramAggregatorFactory extends ValuesSourceAggregatorFact
|
|||
throws IOException {
|
||||
return createAggregator(null, parent, pipelineAggregators, metaData);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -31,6 +31,9 @@ import org.elasticsearch.search.aggregations.InternalAggregations;
|
|||
import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation;
|
||||
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.search.aggregations.InternalOrder;
|
||||
import org.elasticsearch.search.aggregations.KeyComparable;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.DateTimeZone;
|
||||
|
||||
|
@ -49,7 +52,7 @@ import java.util.Objects;
|
|||
public final class InternalDateHistogram extends InternalMultiBucketAggregation<InternalDateHistogram, InternalDateHistogram.Bucket>
|
||||
implements Histogram, HistogramFactory {
|
||||
|
||||
public static class Bucket extends InternalMultiBucketAggregation.InternalBucket implements Histogram.Bucket {
|
||||
public static class Bucket extends InternalMultiBucketAggregation.InternalBucket implements Histogram.Bucket, KeyComparable<Bucket> {
|
||||
|
||||
final long key;
|
||||
final long docCount;
|
||||
|
@ -151,6 +154,11 @@ public final class InternalDateHistogram extends InternalMultiBucketAggregation<
|
|||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareKey(Bucket other) {
|
||||
return Long.compare(key, other.key);
|
||||
}
|
||||
|
||||
public DocValueFormat getFormatter() {
|
||||
return format;
|
||||
}
|
||||
|
@ -206,14 +214,14 @@ public final class InternalDateHistogram extends InternalMultiBucketAggregation<
|
|||
}
|
||||
|
||||
private final List<Bucket> buckets;
|
||||
private final InternalOrder order;
|
||||
private final BucketOrder order;
|
||||
private final DocValueFormat format;
|
||||
private final boolean keyed;
|
||||
private final long minDocCount;
|
||||
private final long offset;
|
||||
private final EmptyBucketInfo emptyBucketInfo;
|
||||
|
||||
InternalDateHistogram(String name, List<Bucket> buckets, InternalOrder order, long minDocCount, long offset,
|
||||
InternalDateHistogram(String name, List<Bucket> buckets, BucketOrder order, long minDocCount, long offset,
|
||||
EmptyBucketInfo emptyBucketInfo,
|
||||
DocValueFormat formatter, boolean keyed, List<PipelineAggregator> pipelineAggregators,
|
||||
Map<String, Object> metaData) {
|
||||
|
@ -233,7 +241,7 @@ public final class InternalDateHistogram extends InternalMultiBucketAggregation<
|
|||
*/
|
||||
public InternalDateHistogram(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
order = InternalOrder.Streams.readOrder(in);
|
||||
order = InternalOrder.Streams.readHistogramOrder(in, false);
|
||||
minDocCount = in.readVLong();
|
||||
if (minDocCount == 0) {
|
||||
emptyBucketInfo = new EmptyBucketInfo(in);
|
||||
|
@ -248,7 +256,7 @@ public final class InternalDateHistogram extends InternalMultiBucketAggregation<
|
|||
|
||||
@Override
|
||||
protected void doWriteTo(StreamOutput out) throws IOException {
|
||||
InternalOrder.Streams.writeOrder(order, out);
|
||||
InternalOrder.Streams.writeHistogramOrder(order, out, false);
|
||||
out.writeVLong(minDocCount);
|
||||
if (minDocCount == 0) {
|
||||
emptyBucketInfo.writeTo(out);
|
||||
|
@ -416,18 +424,18 @@ public final class InternalDateHistogram extends InternalMultiBucketAggregation<
|
|||
addEmptyBuckets(reducedBuckets, reduceContext);
|
||||
}
|
||||
|
||||
if (order == InternalOrder.KEY_ASC || reduceContext.isFinalReduce() == false) {
|
||||
if (InternalOrder.isKeyAsc(order) || reduceContext.isFinalReduce() == false) {
|
||||
// nothing to do, data are already sorted since shards return
|
||||
// sorted buckets and the merge-sort performed by reduceBuckets
|
||||
// maintains order
|
||||
} else if (order == InternalOrder.KEY_DESC) {
|
||||
} else if (InternalOrder.isKeyDesc(order)) {
|
||||
// we just need to reverse here...
|
||||
List<Bucket> reverse = new ArrayList<>(reducedBuckets);
|
||||
Collections.reverse(reverse);
|
||||
reducedBuckets = reverse;
|
||||
} else {
|
||||
// sorted by sub-aggregation, need to fall back to a costly n*log(n) sort
|
||||
CollectionUtil.introSort(reducedBuckets, order.comparator());
|
||||
// sorted by compound order or sub-aggregation, need to fall back to a costly n*log(n) sort
|
||||
CollectionUtil.introSort(reducedBuckets, order.comparator(null));
|
||||
}
|
||||
|
||||
return new InternalDateHistogram(getName(), reducedBuckets, order, minDocCount, offset, emptyBucketInfo,
|
||||
|
|
|
@ -30,6 +30,9 @@ import org.elasticsearch.search.aggregations.InternalAggregations;
|
|||
import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation;
|
||||
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.search.aggregations.InternalOrder;
|
||||
import org.elasticsearch.search.aggregations.KeyComparable;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -45,7 +48,7 @@ import java.util.Objects;
|
|||
*/
|
||||
public final class InternalHistogram extends InternalMultiBucketAggregation<InternalHistogram, InternalHistogram.Bucket>
|
||||
implements Histogram, HistogramFactory {
|
||||
public static class Bucket extends InternalMultiBucketAggregation.InternalBucket implements Histogram.Bucket {
|
||||
public static class Bucket extends InternalMultiBucketAggregation.InternalBucket implements Histogram.Bucket, KeyComparable<Bucket> {
|
||||
|
||||
final double key;
|
||||
final long docCount;
|
||||
|
@ -147,6 +150,11 @@ public final class InternalHistogram extends InternalMultiBucketAggregation<Inte
|
|||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareKey(Bucket other) {
|
||||
return Double.compare(key, other.key);
|
||||
}
|
||||
|
||||
public DocValueFormat getFormatter() {
|
||||
return format;
|
||||
}
|
||||
|
@ -201,13 +209,13 @@ public final class InternalHistogram extends InternalMultiBucketAggregation<Inte
|
|||
}
|
||||
|
||||
private final List<Bucket> buckets;
|
||||
private final InternalOrder order;
|
||||
private final BucketOrder order;
|
||||
private final DocValueFormat format;
|
||||
private final boolean keyed;
|
||||
private final long minDocCount;
|
||||
private final EmptyBucketInfo emptyBucketInfo;
|
||||
|
||||
InternalHistogram(String name, List<Bucket> buckets, InternalOrder order, long minDocCount, EmptyBucketInfo emptyBucketInfo,
|
||||
InternalHistogram(String name, List<Bucket> buckets, BucketOrder order, long minDocCount, EmptyBucketInfo emptyBucketInfo,
|
||||
DocValueFormat formatter, boolean keyed, List<PipelineAggregator> pipelineAggregators,
|
||||
Map<String, Object> metaData) {
|
||||
super(name, pipelineAggregators, metaData);
|
||||
|
@ -225,7 +233,7 @@ public final class InternalHistogram extends InternalMultiBucketAggregation<Inte
|
|||
*/
|
||||
public InternalHistogram(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
order = InternalOrder.Streams.readOrder(in);
|
||||
order = InternalOrder.Streams.readHistogramOrder(in, false);
|
||||
minDocCount = in.readVLong();
|
||||
if (minDocCount == 0) {
|
||||
emptyBucketInfo = new EmptyBucketInfo(in);
|
||||
|
@ -239,7 +247,7 @@ public final class InternalHistogram extends InternalMultiBucketAggregation<Inte
|
|||
|
||||
@Override
|
||||
protected void doWriteTo(StreamOutput out) throws IOException {
|
||||
InternalOrder.Streams.writeOrder(order, out);
|
||||
InternalOrder.Streams.writeHistogramOrder(order, out, false);
|
||||
out.writeVLong(minDocCount);
|
||||
if (minDocCount == 0) {
|
||||
emptyBucketInfo.writeTo(out);
|
||||
|
@ -400,18 +408,18 @@ public final class InternalHistogram extends InternalMultiBucketAggregation<Inte
|
|||
addEmptyBuckets(reducedBuckets, reduceContext);
|
||||
}
|
||||
|
||||
if (order == InternalOrder.KEY_ASC || reduceContext.isFinalReduce() == false) {
|
||||
if (InternalOrder.isKeyAsc(order) || reduceContext.isFinalReduce() == false) {
|
||||
// nothing to do, data are already sorted since shards return
|
||||
// sorted buckets and the merge-sort performed by reduceBuckets
|
||||
// maintains order
|
||||
} else if (order == InternalOrder.KEY_DESC) {
|
||||
} else if (InternalOrder.isKeyDesc(order)) {
|
||||
// we just need to reverse here...
|
||||
List<Bucket> reverse = new ArrayList<>(reducedBuckets);
|
||||
Collections.reverse(reverse);
|
||||
reducedBuckets = reverse;
|
||||
} else {
|
||||
// sorted by sub-aggregation, need to fall back to a costly n*log(n) sort
|
||||
CollectionUtil.introSort(reducedBuckets, order.comparator());
|
||||
// sorted by compound order or sub-aggregation, need to fall back to a costly n*log(n) sort
|
||||
CollectionUtil.introSort(reducedBuckets, order.comparator(null));
|
||||
}
|
||||
|
||||
return new InternalHistogram(getName(), reducedBuckets, order, minDocCount, emptyBucketInfo, format, keyed, pipelineAggregators(),
|
||||
|
|
|
@ -1,135 +0,0 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package org.elasticsearch.search.aggregations.bucket.histogram;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* An internal {@link Histogram.Order} strategy which is identified by a unique id.
|
||||
*/
|
||||
class InternalOrder extends Histogram.Order {
|
||||
|
||||
final byte id;
|
||||
final String key;
|
||||
final boolean asc;
|
||||
final Comparator<Histogram.Bucket> comparator;
|
||||
|
||||
InternalOrder(byte id, String key, boolean asc, Comparator<Histogram.Bucket> comparator) {
|
||||
this.id = id;
|
||||
this.key = key;
|
||||
this.asc = asc;
|
||||
this.comparator = comparator;
|
||||
}
|
||||
|
||||
byte id() {
|
||||
return id;
|
||||
}
|
||||
|
||||
String key() {
|
||||
return key;
|
||||
}
|
||||
|
||||
boolean asc() {
|
||||
return asc;
|
||||
}
|
||||
|
||||
@Override
|
||||
Comparator<Histogram.Bucket> comparator() {
|
||||
return comparator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder.startObject().field(key, asc ? "asc" : "desc").endObject();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(id, key, asc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (obj == null) {
|
||||
return false;
|
||||
}
|
||||
if (getClass() != obj.getClass()) {
|
||||
return false;
|
||||
}
|
||||
InternalOrder other = (InternalOrder) obj;
|
||||
return Objects.equals(id, other.id)
|
||||
&& Objects.equals(key, other.key)
|
||||
&& Objects.equals(asc, other.asc);
|
||||
}
|
||||
|
||||
static class Aggregation extends InternalOrder {
|
||||
|
||||
static final byte ID = 0;
|
||||
|
||||
Aggregation(String key, boolean asc) {
|
||||
super(ID, key, asc, new MultiBucketsAggregation.Bucket.SubAggregationComparator<Histogram.Bucket>(key, asc));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
static class Streams {
|
||||
|
||||
/**
|
||||
* Writes the given order to the given output (based on the id of the order).
|
||||
*/
|
||||
public static void writeOrder(InternalOrder order, StreamOutput out) throws IOException {
|
||||
out.writeByte(order.id());
|
||||
if (order instanceof InternalOrder.Aggregation) {
|
||||
out.writeBoolean(order.asc());
|
||||
out.writeString(order.key());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads an order from the given input (based on the id of the order).
|
||||
*
|
||||
* @see Streams#writeOrder(InternalOrder, org.elasticsearch.common.io.stream.StreamOutput)
|
||||
*/
|
||||
public static InternalOrder readOrder(StreamInput in) throws IOException {
|
||||
byte id = in.readByte();
|
||||
switch (id) {
|
||||
case 1: return (InternalOrder) Histogram.Order.KEY_ASC;
|
||||
case 2: return (InternalOrder) Histogram.Order.KEY_DESC;
|
||||
case 3: return (InternalOrder) Histogram.Order.COUNT_ASC;
|
||||
case 4: return (InternalOrder) Histogram.Order.COUNT_DESC;
|
||||
case 0:
|
||||
boolean asc = in.readBoolean();
|
||||
String key = in.readString();
|
||||
return new InternalOrder.Aggregation(key, asc);
|
||||
default:
|
||||
throw new RuntimeException("unknown histogram order");
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
}
|
|
@ -24,6 +24,7 @@ import org.elasticsearch.search.aggregations.Aggregator;
|
|||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -37,7 +38,7 @@ abstract class AbstractStringTermsAggregator extends TermsAggregator {
|
|||
protected final boolean showTermDocCountError;
|
||||
|
||||
AbstractStringTermsAggregator(String name, AggregatorFactories factories, SearchContext context, Aggregator parent,
|
||||
Terms.Order order, DocValueFormat format, BucketCountThresholds bucketCountThresholds, SubAggCollectionMode subAggCollectMode,
|
||||
BucketOrder order, DocValueFormat format, BucketCountThresholds bucketCountThresholds, SubAggCollectionMode subAggCollectMode,
|
||||
boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
|
||||
super(name, factories, context, parent, bucketCountThresholds, order, format, subAggCollectMode, pipelineAggregators, metaData);
|
||||
this.showTermDocCountError = showTermDocCountError;
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.elasticsearch.search.DocValueFormat;
|
|||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -76,8 +77,8 @@ public class DoubleTerms extends InternalMappedTerms<DoubleTerms, DoubleTerms.Bu
|
|||
}
|
||||
|
||||
@Override
|
||||
public int compareTerm(Terms.Bucket other) {
|
||||
return Double.compare(term, ((Number) other.getKey()).doubleValue());
|
||||
public int compareKey(Bucket other) {
|
||||
return Double.compare(term, other.term);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -105,7 +106,7 @@ public class DoubleTerms extends InternalMappedTerms<DoubleTerms, DoubleTerms.Bu
|
|||
}
|
||||
}
|
||||
|
||||
public DoubleTerms(String name, Terms.Order order, int requiredSize, long minDocCount, List<PipelineAggregator> pipelineAggregators,
|
||||
public DoubleTerms(String name, BucketOrder order, int requiredSize, long minDocCount, List<PipelineAggregator> pipelineAggregators,
|
||||
Map<String, Object> metaData, DocValueFormat format, int shardSize, boolean showTermDocCountError, long otherDocCount,
|
||||
List<Bucket> buckets, long docCountError) {
|
||||
super(name, order, requiredSize, minDocCount, pipelineAggregators, metaData, format, shardSize, showTermDocCountError,
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.elasticsearch.search.aggregations.Aggregator;
|
|||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
@ -39,7 +40,7 @@ import java.util.stream.Collectors;
|
|||
public class DoubleTermsAggregator extends LongTermsAggregator {
|
||||
|
||||
public DoubleTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, DocValueFormat format,
|
||||
Terms.Order order, BucketCountThresholds bucketCountThresholds, SearchContext aggregationContext, Aggregator parent,
|
||||
BucketOrder order, BucketCountThresholds bucketCountThresholds, SearchContext aggregationContext, Aggregator parent,
|
||||
SubAggCollectionMode collectionMode, boolean showTermDocCountError, IncludeExclude.LongFilter longFilter,
|
||||
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
|
||||
super(name, factories, valuesSource, format, order, bucketCountThresholds, aggregationContext, parent, collectionMode,
|
||||
|
|
|
@ -43,6 +43,7 @@ import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
|
|||
import org.elasticsearch.search.aggregations.bucket.terms.support.BucketPriorityQueue;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
|
@ -70,7 +71,7 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
|
|||
protected SortedSetDocValues globalOrds;
|
||||
|
||||
public GlobalOrdinalsStringTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals valuesSource,
|
||||
Terms.Order order, DocValueFormat format, BucketCountThresholds bucketCountThresholds,
|
||||
BucketOrder order, DocValueFormat format, BucketCountThresholds bucketCountThresholds,
|
||||
IncludeExclude.OrdinalsFilter includeExclude, SearchContext context, Aggregator parent,
|
||||
SubAggCollectionMode collectionMode, boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators,
|
||||
Map<String, Object> metaData) throws IOException {
|
||||
|
@ -122,8 +123,8 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
|
|||
public void collect(int doc, long bucket) throws IOException {
|
||||
assert bucket == 0;
|
||||
if (ords.advanceExact(doc)) {
|
||||
for (long globalOrd = ords.nextOrd();
|
||||
globalOrd != SortedSetDocValues.NO_MORE_ORDS;
|
||||
for (long globalOrd = ords.nextOrd();
|
||||
globalOrd != SortedSetDocValues.NO_MORE_ORDS;
|
||||
globalOrd = ords.nextOrd()) {
|
||||
collectExistingBucket(sub, doc, globalOrd);
|
||||
}
|
||||
|
@ -218,8 +219,8 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
|
|||
}
|
||||
|
||||
@Override
|
||||
public int compareTerm(Terms.Bucket other) {
|
||||
return Long.compare(globalOrd, ((OrdBucket) other).globalOrd);
|
||||
public int compareKey(OrdBucket other) {
|
||||
return Long.compare(globalOrd, other.globalOrd);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -261,7 +262,7 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
|
|||
|
||||
private final LongHash bucketOrds;
|
||||
|
||||
public WithHash(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals valuesSource, Terms.Order order,
|
||||
public WithHash(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals valuesSource, BucketOrder order,
|
||||
DocValueFormat format, BucketCountThresholds bucketCountThresholds, IncludeExclude.OrdinalsFilter includeExclude,
|
||||
SearchContext context, Aggregator parent, SubAggCollectionMode collectionMode,
|
||||
boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
|
||||
|
@ -296,8 +297,8 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
|
|||
@Override
|
||||
public void collect(int doc, long bucket) throws IOException {
|
||||
if (ords.advanceExact(doc)) {
|
||||
for (long globalOrd = ords.nextOrd();
|
||||
globalOrd != SortedSetDocValues.NO_MORE_ORDS;
|
||||
for (long globalOrd = ords.nextOrd();
|
||||
globalOrd != SortedSetDocValues.NO_MORE_ORDS;
|
||||
globalOrd = ords.nextOrd()) {
|
||||
long bucketOrd = bucketOrds.add(globalOrd);
|
||||
if (bucketOrd < 0) {
|
||||
|
@ -337,7 +338,7 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
|
|||
private SortedSetDocValues segmentOrds;
|
||||
|
||||
public LowCardinality(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals valuesSource,
|
||||
Terms.Order order, DocValueFormat format,
|
||||
BucketOrder order, DocValueFormat format,
|
||||
BucketCountThresholds bucketCountThresholds, SearchContext context, Aggregator parent,
|
||||
SubAggCollectionMode collectionMode, boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators,
|
||||
Map<String, Object> metaData) throws IOException {
|
||||
|
@ -371,8 +372,8 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
|
|||
public void collect(int doc, long bucket) throws IOException {
|
||||
assert bucket == 0;
|
||||
if (ords.advanceExact(doc)) {
|
||||
for (long segmentOrd = ords.nextOrd();
|
||||
segmentOrd != SortedSetDocValues.NO_MORE_ORDS;
|
||||
for (long segmentOrd = ords.nextOrd();
|
||||
segmentOrd != SortedSetDocValues.NO_MORE_ORDS;
|
||||
segmentOrd = ords.nextOrd()) {
|
||||
segmentDocCounts.increment(segmentOrd + 1, 1);
|
||||
}
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
|
|||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.DocValueFormat;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
|
@ -46,7 +47,7 @@ public abstract class InternalMappedTerms<A extends InternalTerms<A, B>, B exten
|
|||
|
||||
protected long docCountError;
|
||||
|
||||
protected InternalMappedTerms(String name, Terms.Order order, int requiredSize, long minDocCount,
|
||||
protected InternalMappedTerms(String name, BucketOrder order, int requiredSize, long minDocCount,
|
||||
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData, DocValueFormat format, int shardSize,
|
||||
boolean showTermDocCountError, long otherDocCount, List<B> buckets, long docCountError) {
|
||||
super(name, order, requiredSize, minDocCount, pipelineAggregators, metaData);
|
||||
|
@ -83,7 +84,7 @@ public abstract class InternalMappedTerms<A extends InternalTerms<A, B>, B exten
|
|||
|
||||
@Override
|
||||
protected void setDocCountError(long docCountError) {
|
||||
this.docCountError = docCountError;
|
||||
this.docCountError = docCountError;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -1,385 +0,0 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package org.elasticsearch.search.aggregations.bucket.terms;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.util.Comparators;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
|
||||
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation;
|
||||
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.Terms.Bucket;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order;
|
||||
import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationPath;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
class InternalOrder extends Terms.Order {
|
||||
|
||||
private static final byte COUNT_DESC_ID = 1;
|
||||
private static final byte COUNT_ASC_ID = 2;
|
||||
private static final byte TERM_DESC_ID = 3;
|
||||
private static final byte TERM_ASC_ID = 4;
|
||||
|
||||
/**
|
||||
* Order by the (higher) count of each term.
|
||||
*/
|
||||
public static final InternalOrder COUNT_DESC = new InternalOrder(COUNT_DESC_ID, "_count", false, new Comparator<Terms.Bucket>() {
|
||||
@Override
|
||||
public int compare(Terms.Bucket o1, Terms.Bucket o2) {
|
||||
return Long.compare(o2.getDocCount(), o1.getDocCount());
|
||||
}
|
||||
});
|
||||
|
||||
/**
|
||||
* Order by the (lower) count of each term.
|
||||
*/
|
||||
public static final InternalOrder COUNT_ASC = new InternalOrder(COUNT_ASC_ID, "_count", true, new Comparator<Terms.Bucket>() {
|
||||
|
||||
@Override
|
||||
public int compare(Terms.Bucket o1, Terms.Bucket o2) {
|
||||
return Long.compare(o1.getDocCount(), o2.getDocCount());
|
||||
}
|
||||
});
|
||||
|
||||
/**
|
||||
* Order by the terms.
|
||||
*/
|
||||
public static final InternalOrder TERM_DESC = new InternalOrder(TERM_DESC_ID, "_term", false, new Comparator<Terms.Bucket>() {
|
||||
|
||||
@Override
|
||||
public int compare(Terms.Bucket o1, Terms.Bucket o2) {
|
||||
return o2.compareTerm(o1);
|
||||
}
|
||||
});
|
||||
|
||||
/**
|
||||
* Order by the terms.
|
||||
*/
|
||||
public static final InternalOrder TERM_ASC = new InternalOrder(TERM_ASC_ID, "_term", true, new Comparator<Terms.Bucket>() {
|
||||
|
||||
@Override
|
||||
public int compare(Terms.Bucket o1, Terms.Bucket o2) {
|
||||
return o1.compareTerm(o2);
|
||||
}
|
||||
});
|
||||
|
||||
public static boolean isCountDesc(Terms.Order order) {
|
||||
if (order == COUNT_DESC) {
|
||||
return true;
|
||||
} else if (order instanceof CompoundOrder) {
|
||||
// check if its a compound order with count desc and the tie breaker (term asc)
|
||||
CompoundOrder compoundOrder = (CompoundOrder) order;
|
||||
if (compoundOrder.orderElements.size() == 2 && compoundOrder.orderElements.get(0) == COUNT_DESC && compoundOrder.orderElements.get(1) == TERM_ASC) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
public static boolean isTermOrder(Terms.Order order) {
|
||||
if (order == TERM_ASC) {
|
||||
return true;
|
||||
} else if (order == TERM_DESC) {
|
||||
return true;
|
||||
} else if (order instanceof CompoundOrder) {
|
||||
// check if its a compound order with only a single element ordering
|
||||
// by term
|
||||
CompoundOrder compoundOrder = (CompoundOrder) order;
|
||||
if (compoundOrder.orderElements.size() == 1 && compoundOrder.orderElements.get(0) == TERM_ASC
|
||||
|| compoundOrder.orderElements.get(0) == TERM_DESC) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
final byte id;
|
||||
final String key;
|
||||
final boolean asc;
|
||||
protected final Comparator<Terms.Bucket> comparator;
|
||||
|
||||
InternalOrder(byte id, String key, boolean asc, Comparator<Terms.Bucket> comparator) {
|
||||
this.id = id;
|
||||
this.key = key;
|
||||
this.asc = asc;
|
||||
this.comparator = comparator;
|
||||
}
|
||||
|
||||
@Override
|
||||
byte id() {
|
||||
return id;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Comparator<Terms.Bucket> comparator(Aggregator aggregator) {
|
||||
return comparator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
return builder.startObject().field(key, asc ? "asc" : "desc").endObject();
|
||||
}
|
||||
|
||||
public static Terms.Order validate(Terms.Order order, Aggregator termsAggregator) {
|
||||
if (order instanceof CompoundOrder) {
|
||||
for (Terms.Order innerOrder : ((CompoundOrder)order).orderElements) {
|
||||
validate(innerOrder, termsAggregator);
|
||||
}
|
||||
return order;
|
||||
} else if (!(order instanceof Aggregation)) {
|
||||
return order;
|
||||
}
|
||||
AggregationPath path = ((Aggregation) order).path();
|
||||
path.validate(termsAggregator);
|
||||
return order;
|
||||
}
|
||||
|
||||
static class Aggregation extends InternalOrder {
|
||||
|
||||
static final byte ID = 0;
|
||||
|
||||
Aggregation(String key, boolean asc) {
|
||||
super(ID, key, asc, new MultiBucketsAggregation.Bucket.SubAggregationComparator<Terms.Bucket>(key, asc));
|
||||
}
|
||||
|
||||
AggregationPath path() {
|
||||
return ((MultiBucketsAggregation.Bucket.SubAggregationComparator) comparator).path();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Comparator<Terms.Bucket> comparator(Aggregator termsAggregator) {
|
||||
if (termsAggregator == null) {
|
||||
return comparator;
|
||||
}
|
||||
|
||||
// Internal Optimization:
|
||||
//
|
||||
// in this phase, if the order is based on sub-aggregations, we need to use a different comparator
|
||||
// to avoid constructing buckets for ordering purposes (we can potentially have a lot of buckets and building
|
||||
// them will cause loads of redundant object constructions). The "special" comparators here will fetch the
|
||||
// sub aggregation values directly from the sub aggregators bypassing bucket creation. Note that the comparator
|
||||
// attached to the order will still be used in the reduce phase of the Aggregation.
|
||||
|
||||
AggregationPath path = path();
|
||||
final Aggregator aggregator = path.resolveAggregator(termsAggregator);
|
||||
final String key = path.lastPathElement().key;
|
||||
|
||||
if (aggregator instanceof SingleBucketAggregator) {
|
||||
assert key == null : "this should be picked up before the aggregation is executed - on validate";
|
||||
return new Comparator<Terms.Bucket>() {
|
||||
@Override
|
||||
public int compare(Terms.Bucket o1, Terms.Bucket o2) {
|
||||
int mul = asc ? 1 : -1;
|
||||
int v1 = ((SingleBucketAggregator) aggregator).bucketDocCount(((InternalTerms.Bucket) o1).bucketOrd);
|
||||
int v2 = ((SingleBucketAggregator) aggregator).bucketDocCount(((InternalTerms.Bucket) o2).bucketOrd);
|
||||
return mul * (v1 - v2);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
// with only support single-bucket aggregators
|
||||
assert !(aggregator instanceof BucketsAggregator) : "this should be picked up before the aggregation is executed - on validate";
|
||||
|
||||
if (aggregator instanceof NumericMetricsAggregator.MultiValue) {
|
||||
assert key != null : "this should be picked up before the aggregation is executed - on validate";
|
||||
return new Comparator<Terms.Bucket>() {
|
||||
@Override
|
||||
public int compare(Terms.Bucket o1, Terms.Bucket o2) {
|
||||
double v1 = ((NumericMetricsAggregator.MultiValue) aggregator).metric(key, ((InternalTerms.Bucket) o1).bucketOrd);
|
||||
double v2 = ((NumericMetricsAggregator.MultiValue) aggregator).metric(key, ((InternalTerms.Bucket) o2).bucketOrd);
|
||||
// some metrics may return NaN (eg. avg, variance, etc...) in which case we'd like to push all of those to
|
||||
// the bottom
|
||||
return Comparators.compareDiscardNaN(v1, v2, asc);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
// single-value metrics agg
|
||||
return new Comparator<Terms.Bucket>() {
|
||||
@Override
|
||||
public int compare(Terms.Bucket o1, Terms.Bucket o2) {
|
||||
double v1 = ((NumericMetricsAggregator.SingleValue) aggregator).metric(((InternalTerms.Bucket) o1).bucketOrd);
|
||||
double v2 = ((NumericMetricsAggregator.SingleValue) aggregator).metric(((InternalTerms.Bucket) o2).bucketOrd);
|
||||
// some metrics may return NaN (eg. avg, variance, etc...) in which case we'd like to push all of those to
|
||||
// the bottom
|
||||
return Comparators.compareDiscardNaN(v1, v2, asc);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
static class CompoundOrder extends Terms.Order {
|
||||
|
||||
static final byte ID = -1;
|
||||
|
||||
private final List<Terms.Order> orderElements;
|
||||
|
||||
CompoundOrder(List<Terms.Order> compoundOrder) {
|
||||
this(compoundOrder, true);
|
||||
}
|
||||
|
||||
CompoundOrder(List<Terms.Order> compoundOrder, boolean absoluteOrdering) {
|
||||
this.orderElements = new LinkedList<>(compoundOrder);
|
||||
Terms.Order lastElement = compoundOrder.get(compoundOrder.size() - 1);
|
||||
if (absoluteOrdering && !(InternalOrder.TERM_ASC == lastElement || InternalOrder.TERM_DESC == lastElement)) {
|
||||
// add term order ascending as a tie-breaker to avoid non-deterministic ordering
|
||||
// if all user provided comparators return 0.
|
||||
this.orderElements.add(Order.term(true));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
byte id() {
|
||||
return ID;
|
||||
}
|
||||
|
||||
List<Terms.Order> orderElements() {
|
||||
return Collections.unmodifiableList(orderElements);
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startArray();
|
||||
for (Terms.Order order : orderElements) {
|
||||
order.toXContent(builder, params);
|
||||
}
|
||||
return builder.endArray();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Comparator<Bucket> comparator(Aggregator aggregator) {
|
||||
return new CompoundOrderComparator(orderElements, aggregator);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(orderElements);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (obj == null) {
|
||||
return false;
|
||||
}
|
||||
if (getClass() != obj.getClass()) {
|
||||
return false;
|
||||
}
|
||||
CompoundOrder other = (CompoundOrder) obj;
|
||||
return Objects.equals(orderElements, other.orderElements);
|
||||
}
|
||||
|
||||
public static class CompoundOrderComparator implements Comparator<Terms.Bucket> {
|
||||
|
||||
private List<Terms.Order> compoundOrder;
|
||||
private Aggregator aggregator;
|
||||
|
||||
CompoundOrderComparator(List<Terms.Order> compoundOrder, Aggregator aggregator) {
|
||||
this.compoundOrder = compoundOrder;
|
||||
this.aggregator = aggregator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compare(Bucket o1, Bucket o2) {
|
||||
int result = 0;
|
||||
for (Iterator<Terms.Order> itr = compoundOrder.iterator(); itr.hasNext() && result == 0;) {
|
||||
result = itr.next().comparator(aggregator).compare(o1, o2);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static class Streams {
|
||||
|
||||
public static void writeOrder(Terms.Order order, StreamOutput out) throws IOException {
|
||||
if (order instanceof Aggregation) {
|
||||
out.writeByte(order.id());
|
||||
Aggregation aggregationOrder = (Aggregation) order;
|
||||
out.writeBoolean(((MultiBucketsAggregation.Bucket.SubAggregationComparator) aggregationOrder.comparator).asc());
|
||||
AggregationPath path = ((Aggregation) order).path();
|
||||
out.writeString(path.toString());
|
||||
} else if (order instanceof CompoundOrder) {
|
||||
CompoundOrder compoundOrder = (CompoundOrder) order;
|
||||
out.writeByte(order.id());
|
||||
out.writeVInt(compoundOrder.orderElements.size());
|
||||
for (Terms.Order innerOrder : compoundOrder.orderElements) {
|
||||
Streams.writeOrder(innerOrder, out);
|
||||
}
|
||||
} else {
|
||||
out.writeByte(order.id());
|
||||
}
|
||||
}
|
||||
|
||||
public static Terms.Order readOrder(StreamInput in) throws IOException {
|
||||
return readOrder(in, false);
|
||||
}
|
||||
|
||||
public static Terms.Order readOrder(StreamInput in, boolean absoluteOrder) throws IOException {
|
||||
byte id = in.readByte();
|
||||
switch (id) {
|
||||
case COUNT_DESC_ID: return absoluteOrder ? new CompoundOrder(Collections.singletonList((Terms.Order) InternalOrder.COUNT_DESC)) : InternalOrder.COUNT_DESC;
|
||||
case COUNT_ASC_ID: return absoluteOrder ? new CompoundOrder(Collections.singletonList((Terms.Order) InternalOrder.COUNT_ASC)) : InternalOrder.COUNT_ASC;
|
||||
case TERM_DESC_ID: return InternalOrder.TERM_DESC;
|
||||
case TERM_ASC_ID: return InternalOrder.TERM_ASC;
|
||||
case Aggregation.ID:
|
||||
boolean asc = in.readBoolean();
|
||||
String key = in.readString();
|
||||
return new InternalOrder.Aggregation(key, asc);
|
||||
case CompoundOrder.ID:
|
||||
int size = in.readVInt();
|
||||
List<Terms.Order> compoundOrder = new ArrayList<>(size);
|
||||
for (int i = 0; i < size; i++) {
|
||||
compoundOrder.add(Streams.readOrder(in, false));
|
||||
}
|
||||
return new CompoundOrder(compoundOrder, absoluteOrder);
|
||||
default:
|
||||
throw new RuntimeException("unknown terms order");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(id, asc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (obj == null) {
|
||||
return false;
|
||||
}
|
||||
if (getClass() != obj.getClass()) {
|
||||
return false;
|
||||
}
|
||||
InternalOrder other = (InternalOrder) obj;
|
||||
return Objects.equals(id, other.id)
|
||||
&& Objects.equals(asc, other.asc);
|
||||
}
|
||||
}
|
|
@ -31,6 +31,9 @@ import org.elasticsearch.search.aggregations.InternalAggregations;
|
|||
import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.support.BucketPriorityQueue;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.search.aggregations.InternalOrder;
|
||||
import org.elasticsearch.search.aggregations.KeyComparable;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -46,8 +49,8 @@ public abstract class InternalTerms<A extends InternalTerms<A, B>, B extends Int
|
|||
protected static final ParseField DOC_COUNT_ERROR_UPPER_BOUND_FIELD_NAME = new ParseField("doc_count_error_upper_bound");
|
||||
protected static final ParseField SUM_OF_OTHER_DOC_COUNTS = new ParseField("sum_other_doc_count");
|
||||
|
||||
public abstract static class Bucket<B extends Bucket<B>> extends InternalMultiBucketAggregation.InternalBucket implements Terms.Bucket {
|
||||
|
||||
public abstract static class Bucket<B extends Bucket<B>> extends InternalMultiBucketAggregation.InternalBucket
|
||||
implements Terms.Bucket, KeyComparable<B> {
|
||||
/**
|
||||
* Reads a bucket. Should be a constructor reference.
|
||||
*/
|
||||
|
@ -177,11 +180,11 @@ public abstract class InternalTerms<A extends InternalTerms<A, B>, B extends Int
|
|||
}
|
||||
}
|
||||
|
||||
protected final Terms.Order order;
|
||||
protected final BucketOrder order;
|
||||
protected final int requiredSize;
|
||||
protected final long minDocCount;
|
||||
|
||||
protected InternalTerms(String name, Terms.Order order, int requiredSize, long minDocCount,
|
||||
protected InternalTerms(String name, BucketOrder order, int requiredSize, long minDocCount,
|
||||
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) {
|
||||
super(name, pipelineAggregators, metaData);
|
||||
this.order = order;
|
||||
|
@ -201,7 +204,7 @@ public abstract class InternalTerms<A extends InternalTerms<A, B>, B extends Int
|
|||
|
||||
@Override
|
||||
protected final void doWriteTo(StreamOutput out) throws IOException {
|
||||
InternalOrder.Streams.writeOrder(order, out);
|
||||
order.writeTo(out);
|
||||
writeSize(requiredSize, out);
|
||||
out.writeVLong(minDocCount);
|
||||
writeTermTypeInfoTo(out);
|
||||
|
@ -238,9 +241,9 @@ public abstract class InternalTerms<A extends InternalTerms<A, B>, B extends Int
|
|||
}
|
||||
otherDocCount += terms.getSumOfOtherDocCounts();
|
||||
final long thisAggDocCountError;
|
||||
if (terms.getBuckets().size() < getShardSize() || InternalOrder.isTermOrder(order)) {
|
||||
if (terms.getBuckets().size() < getShardSize() || InternalOrder.isKeyOrder(order)) {
|
||||
thisAggDocCountError = 0;
|
||||
} else if (InternalOrder.isCountDesc(this.order)) {
|
||||
} else if (InternalOrder.isCountDesc(order)) {
|
||||
if (terms.getDocCountError() > 0) {
|
||||
// If there is an existing docCountError for this agg then
|
||||
// use this as the error for this aggregation
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.elasticsearch.search.DocValueFormat;
|
|||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -76,8 +77,8 @@ public class LongTerms extends InternalMappedTerms<LongTerms, LongTerms.Bucket>
|
|||
}
|
||||
|
||||
@Override
|
||||
public int compareTerm(Terms.Bucket other) {
|
||||
return Long.compare(term, ((Number) other.getKey()).longValue());
|
||||
public int compareKey(Bucket other) {
|
||||
return Long.compare(term, other.term);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -105,7 +106,7 @@ public class LongTerms extends InternalMappedTerms<LongTerms, LongTerms.Bucket>
|
|||
}
|
||||
}
|
||||
|
||||
public LongTerms(String name, Terms.Order order, int requiredSize, long minDocCount, List<PipelineAggregator> pipelineAggregators,
|
||||
public LongTerms(String name, BucketOrder order, int requiredSize, long minDocCount, List<PipelineAggregator> pipelineAggregators,
|
||||
Map<String, Object> metaData, DocValueFormat format, int shardSize, boolean showTermDocCountError, long otherDocCount,
|
||||
List<Bucket> buckets, long docCountError) {
|
||||
super(name, order, requiredSize, minDocCount, pipelineAggregators, metaData, format, shardSize, showTermDocCountError,
|
||||
|
|
|
@ -32,6 +32,8 @@ import org.elasticsearch.search.aggregations.bucket.terms.support.BucketPriority
|
|||
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude.LongFilter;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.search.aggregations.InternalOrder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
|
@ -50,7 +52,7 @@ public class LongTermsAggregator extends TermsAggregator {
|
|||
private LongFilter longFilter;
|
||||
|
||||
public LongTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, DocValueFormat format,
|
||||
Terms.Order order, BucketCountThresholds bucketCountThresholds, SearchContext aggregationContext, Aggregator parent,
|
||||
BucketOrder order, BucketCountThresholds bucketCountThresholds, SearchContext aggregationContext, Aggregator parent,
|
||||
SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, IncludeExclude.LongFilter longFilter,
|
||||
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
|
||||
super(name, factories, aggregationContext, parent, bucketCountThresholds, order, format, subAggCollectMode, pipelineAggregators, metaData);
|
||||
|
@ -106,7 +108,7 @@ public class LongTermsAggregator extends TermsAggregator {
|
|||
public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException {
|
||||
assert owningBucketOrdinal == 0;
|
||||
|
||||
if (bucketCountThresholds.getMinDocCount() == 0 && (order != InternalOrder.COUNT_DESC || bucketOrds.size() < bucketCountThresholds.getRequiredSize())) {
|
||||
if (bucketCountThresholds.getMinDocCount() == 0 && (InternalOrder.isCountDesc(order) == false || bucketOrds.size() < bucketCountThresholds.getRequiredSize())) {
|
||||
// we need to fill-in the blanks
|
||||
for (LeafReaderContext ctx : context.searcher().getTopReaderContext().leaves()) {
|
||||
final SortedNumericDocValues values = getValues(valuesSource, ctx);
|
||||
|
|
|
@ -93,11 +93,6 @@ public abstract class ParsedTerms extends ParsedMultiBucketAggregation<ParsedTer
|
|||
boolean showDocCountError = false;
|
||||
protected long docCountError;
|
||||
|
||||
@Override
|
||||
public int compareTerm(Terms.Bucket other) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getDocCountError() {
|
||||
return docCountError;
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
|
|||
import org.elasticsearch.search.DocValueFormat;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
@ -75,8 +76,8 @@ public class StringTerms extends InternalMappedTerms<StringTerms, StringTerms.Bu
|
|||
}
|
||||
|
||||
@Override
|
||||
public int compareTerm(Terms.Bucket other) {
|
||||
return termBytes.compareTo(((Bucket) other).termBytes);
|
||||
public int compareKey(Bucket other) {
|
||||
return termBytes.compareTo(other.termBytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -100,7 +101,7 @@ public class StringTerms extends InternalMappedTerms<StringTerms, StringTerms.Bu
|
|||
}
|
||||
}
|
||||
|
||||
public StringTerms(String name, Terms.Order order, int requiredSize, long minDocCount, List<PipelineAggregator> pipelineAggregators,
|
||||
public StringTerms(String name, BucketOrder order, int requiredSize, long minDocCount, List<PipelineAggregator> pipelineAggregators,
|
||||
Map<String, Object> metaData, DocValueFormat format, int shardSize, boolean showTermDocCountError, long otherDocCount,
|
||||
List<Bucket> buckets, long docCountError) {
|
||||
super(name, order, requiredSize, minDocCount, pipelineAggregators, metaData, format,
|
||||
|
|
|
@ -33,6 +33,8 @@ import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
|
|||
import org.elasticsearch.search.aggregations.bucket.terms.support.BucketPriorityQueue;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.search.aggregations.InternalOrder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
|
@ -51,7 +53,7 @@ public class StringTermsAggregator extends AbstractStringTermsAggregator {
|
|||
private final IncludeExclude.StringFilter includeExclude;
|
||||
|
||||
public StringTermsAggregator(String name, AggregatorFactories factories, ValuesSource valuesSource,
|
||||
Terms.Order order, DocValueFormat format, BucketCountThresholds bucketCountThresholds,
|
||||
BucketOrder order, DocValueFormat format, BucketCountThresholds bucketCountThresholds,
|
||||
IncludeExclude.StringFilter includeExclude, SearchContext context,
|
||||
Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError,
|
||||
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
|
||||
|
@ -110,7 +112,7 @@ public class StringTermsAggregator extends AbstractStringTermsAggregator {
|
|||
public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException {
|
||||
assert owningBucketOrdinal == 0;
|
||||
|
||||
if (bucketCountThresholds.getMinDocCount() == 0 && (order != InternalOrder.COUNT_DESC || bucketOrds.size() < bucketCountThresholds.getRequiredSize())) {
|
||||
if (bucketCountThresholds.getMinDocCount() == 0 && (InternalOrder.isCountDesc(order) == false || bucketOrds.size() < bucketCountThresholds.getRequiredSize())) {
|
||||
// we need to fill-in the blanks
|
||||
for (LeafReaderContext ctx : context.searcher().getTopReaderContext().leaves()) {
|
||||
final SortedBinaryDocValues values = valuesSource.bytesValues(ctx);
|
||||
|
|
|
@ -18,12 +18,8 @@
|
|||
*/
|
||||
package org.elasticsearch.search.aggregations.bucket.terms;
|
||||
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
|
@ -39,8 +35,6 @@ public interface Terms extends MultiBucketsAggregation {
|
|||
|
||||
Number getKeyAsNumber();
|
||||
|
||||
int compareTerm(Terms.Bucket other);
|
||||
|
||||
long getDocCountError();
|
||||
}
|
||||
|
||||
|
@ -65,84 +59,4 @@ public interface Terms extends MultiBucketsAggregation {
|
|||
* it to the top buckets.
|
||||
*/
|
||||
long getSumOfOtherDocCounts();
|
||||
|
||||
/**
|
||||
* Determines the order by which the term buckets will be sorted
|
||||
*/
|
||||
abstract class Order implements ToXContent {
|
||||
|
||||
/**
|
||||
* @return a bucket ordering strategy that sorts buckets by their document counts (ascending or descending)
|
||||
*/
|
||||
public static Order count(boolean asc) {
|
||||
return asc ? InternalOrder.COUNT_ASC : InternalOrder.COUNT_DESC;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return a bucket ordering strategy that sorts buckets by their terms (ascending or descending)
|
||||
*/
|
||||
public static Order term(boolean asc) {
|
||||
return asc ? InternalOrder.TERM_ASC : InternalOrder.TERM_DESC;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a bucket ordering strategy which sorts buckets based on a single-valued calc get
|
||||
*
|
||||
* @param path the name of the get
|
||||
* @param asc The direction of the order (ascending or descending)
|
||||
*/
|
||||
public static Order aggregation(String path, boolean asc) {
|
||||
return new InternalOrder.Aggregation(path, asc);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a bucket ordering strategy which sorts buckets based on a multi-valued calc get
|
||||
*
|
||||
* @param aggregationName the name of the get
|
||||
* @param metricName The name of the value of the multi-value get by which the sorting will be applied
|
||||
* @param asc The direction of the order (ascending or descending)
|
||||
*/
|
||||
public static Order aggregation(String aggregationName, String metricName, boolean asc) {
|
||||
return new InternalOrder.Aggregation(aggregationName + "." + metricName, asc);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a bucket ordering strategy which sorts buckets based multiple criteria
|
||||
*
|
||||
* @param orders a list of {@link Order} objects to sort on, in order of priority
|
||||
*/
|
||||
public static Order compound(List<Order> orders) {
|
||||
return new InternalOrder.CompoundOrder(orders);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a bucket ordering strategy which sorts buckets based multiple criteria
|
||||
*
|
||||
* @param orders a list of {@link Order} parameters to sort on, in order of priority
|
||||
*/
|
||||
public static Order compound(Order... orders) {
|
||||
return compound(Arrays.asList(orders));
|
||||
}
|
||||
|
||||
/**
|
||||
* @return A comparator for the bucket based on the given terms aggregator. The comparator is used in two phases:
|
||||
*
|
||||
* - aggregation phase, where each shard builds a list of term buckets to be sent to the coordinating node.
|
||||
* In this phase, the passed in aggregator will be the terms aggregator that aggregates the buckets on the
|
||||
* shard level.
|
||||
*
|
||||
* - reduce phase, where the coordinating node gathers all the buckets from all the shards and reduces them
|
||||
* to a final bucket list. In this case, the passed in aggregator will be {@code null}
|
||||
*/
|
||||
protected abstract Comparator<Bucket> comparator(Aggregator aggregator);
|
||||
|
||||
abstract byte id();
|
||||
|
||||
@Override
|
||||
public abstract int hashCode();
|
||||
|
||||
@Override
|
||||
public abstract boolean equals(Object obj);
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,20 +19,20 @@
|
|||
package org.elasticsearch.search.aggregations.bucket.terms;
|
||||
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.ParsingException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.ObjectParser;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.index.query.QueryParseContext;
|
||||
import org.elasticsearch.search.aggregations.AggregationBuilder;
|
||||
import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator.BucketCountThresholds;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.search.aggregations.InternalOrder;
|
||||
import org.elasticsearch.search.aggregations.InternalOrder.CompoundOrder;
|
||||
import org.elasticsearch.search.aggregations.support.ValueType;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder;
|
||||
|
@ -82,7 +82,7 @@ public class TermsAggregationBuilder extends ValuesSourceAggregationBuilder<Valu
|
|||
(p, c) -> SubAggCollectionMode.parse(p.text()),
|
||||
SubAggCollectionMode.KEY, ObjectParser.ValueType.STRING);
|
||||
|
||||
PARSER.declareObjectArray(TermsAggregationBuilder::order, TermsAggregationBuilder::parseOrderParam,
|
||||
PARSER.declareObjectArray(TermsAggregationBuilder::order, InternalOrder.Parser::parseOrderParam,
|
||||
TermsAggregationBuilder.ORDER_FIELD);
|
||||
|
||||
PARSER.declareField((b, v) -> b.includeExclude(IncludeExclude.merge(v, b.includeExclude())),
|
||||
|
@ -96,7 +96,7 @@ public class TermsAggregationBuilder extends ValuesSourceAggregationBuilder<Valu
|
|||
return PARSER.parse(context.parser(), new TermsAggregationBuilder(aggregationName, null), context);
|
||||
}
|
||||
|
||||
private Terms.Order order = Terms.Order.compound(Terms.Order.count(false), Terms.Order.term(true));
|
||||
private BucketOrder order = BucketOrder.compound(BucketOrder.count(false)); // automatically adds tie-breaker key asc order
|
||||
private IncludeExclude includeExclude = null;
|
||||
private String executionHint = null;
|
||||
private SubAggCollectionMode collectMode = null;
|
||||
|
@ -132,7 +132,7 @@ public class TermsAggregationBuilder extends ValuesSourceAggregationBuilder<Valu
|
|||
out.writeOptionalWriteable(collectMode);
|
||||
out.writeOptionalString(executionHint);
|
||||
out.writeOptionalWriteable(includeExclude);
|
||||
InternalOrder.Streams.writeOrder(order, out);
|
||||
order.writeTo(out);
|
||||
out.writeBoolean(showTermDocCountError);
|
||||
}
|
||||
|
||||
|
@ -189,32 +189,37 @@ public class TermsAggregationBuilder extends ValuesSourceAggregationBuilder<Valu
|
|||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the order in which the buckets will be returned.
|
||||
*/
|
||||
public TermsAggregationBuilder order(Terms.Order order) {
|
||||
/** Set a new order on this builder and return the builder so that calls
|
||||
* can be chained. A tie-breaker may be added to avoid non-deterministic ordering. */
|
||||
public TermsAggregationBuilder order(BucketOrder order) {
|
||||
if (order == null) {
|
||||
throw new IllegalArgumentException("[order] must not be null: [" + name + "]");
|
||||
}
|
||||
this.order = order;
|
||||
if(order instanceof CompoundOrder || InternalOrder.isKeyOrder(order)) {
|
||||
this.order = order; // if order already contains a tie-breaker we are good to go
|
||||
} else { // otherwise add a tie-breaker by using a compound order
|
||||
this.order = BucketOrder.compound(order);
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the order in which the buckets will be returned.
|
||||
* Sets the order in which the buckets will be returned. A tie-breaker may be added to avoid non-deterministic
|
||||
* ordering.
|
||||
*/
|
||||
public TermsAggregationBuilder order(List<Terms.Order> orders) {
|
||||
public TermsAggregationBuilder order(List<BucketOrder> orders) {
|
||||
if (orders == null) {
|
||||
throw new IllegalArgumentException("[orders] must not be null: [" + name + "]");
|
||||
}
|
||||
order(Terms.Order.compound(orders));
|
||||
// if the list only contains one order use that to avoid inconsistent xcontent
|
||||
order(orders.size() > 1 ? BucketOrder.compound(orders) : orders.get(0));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the order in which the buckets will be returned.
|
||||
*/
|
||||
public Terms.Order order() {
|
||||
public BucketOrder order() {
|
||||
return order;
|
||||
}
|
||||
|
||||
|
@ -327,45 +332,4 @@ public class TermsAggregationBuilder extends ValuesSourceAggregationBuilder<Valu
|
|||
return NAME;
|
||||
}
|
||||
|
||||
private static Terms.Order parseOrderParam(XContentParser parser, QueryParseContext context) throws IOException {
|
||||
XContentParser.Token token;
|
||||
Terms.Order orderParam = null;
|
||||
String orderKey = null;
|
||||
boolean orderAsc = false;
|
||||
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
|
||||
if (token == XContentParser.Token.FIELD_NAME) {
|
||||
orderKey = parser.currentName();
|
||||
} else if (token == XContentParser.Token.VALUE_STRING) {
|
||||
String dir = parser.text();
|
||||
if ("asc".equalsIgnoreCase(dir)) {
|
||||
orderAsc = true;
|
||||
} else if ("desc".equalsIgnoreCase(dir)) {
|
||||
orderAsc = false;
|
||||
} else {
|
||||
throw new ParsingException(parser.getTokenLocation(),
|
||||
"Unknown terms order direction [" + dir + "]");
|
||||
}
|
||||
} else {
|
||||
throw new ParsingException(parser.getTokenLocation(),
|
||||
"Unexpected token " + token + " for [order]");
|
||||
}
|
||||
}
|
||||
if (orderKey == null) {
|
||||
throw new ParsingException(parser.getTokenLocation(),
|
||||
"Must specify at least one field for [order]");
|
||||
} else {
|
||||
orderParam = resolveOrder(orderKey, orderAsc);
|
||||
}
|
||||
return orderParam;
|
||||
}
|
||||
|
||||
static Terms.Order resolveOrder(String key, boolean asc) {
|
||||
if ("_term".equals(key)) {
|
||||
return Order.term(asc);
|
||||
}
|
||||
if ("_count".equals(key)) {
|
||||
return Order.count(asc);
|
||||
}
|
||||
return Order.aggregation(key, asc);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,19 +24,26 @@ import org.elasticsearch.ElasticsearchException;
|
|||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.common.util.Comparators;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.search.DocValueFormat;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.InternalOrder.Aggregation;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.InternalOrder.CompoundOrder;
|
||||
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation.Bucket;
|
||||
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
|
||||
import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationPath;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.search.aggregations.InternalOrder;
|
||||
import org.elasticsearch.search.aggregations.InternalOrder.Aggregation;
|
||||
import org.elasticsearch.search.aggregations.InternalOrder.CompoundOrder;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -168,12 +175,12 @@ public abstract class TermsAggregator extends BucketsAggregator {
|
|||
|
||||
protected final DocValueFormat format;
|
||||
protected final BucketCountThresholds bucketCountThresholds;
|
||||
protected final Terms.Order order;
|
||||
protected final BucketOrder order;
|
||||
protected final Set<Aggregator> aggsUsedForSorting = new HashSet<>();
|
||||
protected final SubAggCollectionMode collectMode;
|
||||
|
||||
public TermsAggregator(String name, AggregatorFactories factories, SearchContext context, Aggregator parent,
|
||||
BucketCountThresholds bucketCountThresholds, Terms.Order order, DocValueFormat format, SubAggCollectionMode collectMode,
|
||||
BucketCountThresholds bucketCountThresholds, BucketOrder order, DocValueFormat format, SubAggCollectionMode collectMode,
|
||||
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
|
||||
super(name, factories, context, parent, pipelineAggregators, metaData);
|
||||
this.bucketCountThresholds = bucketCountThresholds;
|
||||
|
@ -186,7 +193,7 @@ public abstract class TermsAggregator extends BucketsAggregator {
|
|||
aggsUsedForSorting.add(path.resolveTopmostAggregator(this));
|
||||
} else if (order instanceof CompoundOrder) {
|
||||
CompoundOrder compoundOrder = (CompoundOrder) order;
|
||||
for (Terms.Order orderElement : compoundOrder.orderElements()) {
|
||||
for (BucketOrder orderElement : compoundOrder.orderElements()) {
|
||||
if (orderElement instanceof Aggregation) {
|
||||
AggregationPath path = ((Aggregation) orderElement).path();
|
||||
aggsUsedForSorting.add(path.resolveTopmostAggregator(this));
|
||||
|
@ -195,6 +202,58 @@ public abstract class TermsAggregator extends BucketsAggregator {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Internal Optimization for ordering {@link InternalTerms.Bucket}s by a sub aggregation.
|
||||
* <p>
|
||||
* in this phase, if the order is based on sub-aggregations, we need to use a different comparator
|
||||
* to avoid constructing buckets for ordering purposes (we can potentially have a lot of buckets and building
|
||||
* them will cause loads of redundant object constructions). The "special" comparators here will fetch the
|
||||
* sub aggregation values directly from the sub aggregators bypassing bucket creation. Note that the comparator
|
||||
* attached to the order will still be used in the reduce phase of the Aggregation.
|
||||
*
|
||||
* @param path determines which sub aggregation to use for ordering.
|
||||
* @param asc {@code true} for ascending order, {@code false} for descending.
|
||||
* @return {@code Comparator} to order {@link InternalTerms.Bucket}s in the desired order.
|
||||
*/
|
||||
public Comparator<Bucket> bucketComparator(AggregationPath path, boolean asc) {
|
||||
|
||||
final Aggregator aggregator = path.resolveAggregator(this);
|
||||
final String key = path.lastPathElement().key;
|
||||
|
||||
if (aggregator instanceof SingleBucketAggregator) {
|
||||
assert key == null : "this should be picked up before the aggregation is executed - on validate";
|
||||
return (b1, b2) -> {
|
||||
int mul = asc ? 1 : -1;
|
||||
int v1 = ((SingleBucketAggregator) aggregator).bucketDocCount(((InternalTerms.Bucket) b1).bucketOrd);
|
||||
int v2 = ((SingleBucketAggregator) aggregator).bucketDocCount(((InternalTerms.Bucket) b2).bucketOrd);
|
||||
return mul * (v1 - v2);
|
||||
};
|
||||
}
|
||||
|
||||
// with only support single-bucket aggregators
|
||||
assert !(aggregator instanceof BucketsAggregator) : "this should be picked up before the aggregation is executed - on validate";
|
||||
|
||||
if (aggregator instanceof NumericMetricsAggregator.MultiValue) {
|
||||
assert key != null : "this should be picked up before the aggregation is executed - on validate";
|
||||
return (b1, b2) -> {
|
||||
double v1 = ((NumericMetricsAggregator.MultiValue) aggregator).metric(key, ((InternalTerms.Bucket) b1).bucketOrd);
|
||||
double v2 = ((NumericMetricsAggregator.MultiValue) aggregator).metric(key, ((InternalTerms.Bucket) b2).bucketOrd);
|
||||
// some metrics may return NaN (eg. avg, variance, etc...) in which case we'd like to push all of those to
|
||||
// the bottom
|
||||
return Comparators.compareDiscardNaN(v1, v2, asc);
|
||||
};
|
||||
}
|
||||
|
||||
// single-value metrics agg
|
||||
return (b1, b2) -> {
|
||||
double v1 = ((NumericMetricsAggregator.SingleValue) aggregator).metric(((InternalTerms.Bucket) b1).bucketOrd);
|
||||
double v2 = ((NumericMetricsAggregator.SingleValue) aggregator).metric(((InternalTerms.Bucket) b2).bucketOrd);
|
||||
// some metrics may return NaN (eg. avg, variance, etc...) in which case we'd like to push all of those to
|
||||
// the bottom
|
||||
return Comparators.compareDiscardNaN(v1, v2, asc);
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean shouldDefer(Aggregator aggregator) {
|
||||
return collectMode == SubAggCollectionMode.BREADTH_FIRST
|
||||
|
|
|
@ -33,6 +33,8 @@ import org.elasticsearch.search.aggregations.bucket.BucketUtils;
|
|||
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator.BucketCountThresholds;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.search.aggregations.InternalOrder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
|
@ -44,14 +46,14 @@ import java.util.Map;
|
|||
|
||||
public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource, TermsAggregatorFactory> {
|
||||
|
||||
private final Terms.Order order;
|
||||
private final BucketOrder order;
|
||||
private final IncludeExclude includeExclude;
|
||||
private final String executionHint;
|
||||
private final SubAggCollectionMode collectMode;
|
||||
private final TermsAggregator.BucketCountThresholds bucketCountThresholds;
|
||||
private boolean showTermDocCountError;
|
||||
|
||||
public TermsAggregatorFactory(String name, ValuesSourceConfig<ValuesSource> config, Terms.Order order,
|
||||
public TermsAggregatorFactory(String name, ValuesSourceConfig<ValuesSource> config, BucketOrder order,
|
||||
IncludeExclude includeExclude, String executionHint, SubAggCollectionMode collectMode,
|
||||
TermsAggregator.BucketCountThresholds bucketCountThresholds, boolean showTermDocCountError, SearchContext context,
|
||||
AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
|
||||
|
@ -90,7 +92,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
|
|||
return asMultiBucketAggregator(this, context, parent);
|
||||
}
|
||||
BucketCountThresholds bucketCountThresholds = new BucketCountThresholds(this.bucketCountThresholds);
|
||||
if (!(order == InternalOrder.TERM_ASC || order == InternalOrder.TERM_DESC)
|
||||
if (InternalOrder.isKeyOrder(order) == false
|
||||
&& bucketCountThresholds.getShardSize() == TermsAggregationBuilder.DEFAULT_BUCKET_COUNT_THRESHOLDS.getShardSize()) {
|
||||
// The user has not made a shardSize selection. Use default
|
||||
// heuristic to avoid any wrong-ranking caused by distributed
|
||||
|
@ -129,7 +131,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
|
|||
// to be unbounded and most instances may only aggregate few
|
||||
// documents, so use hashed based
|
||||
// global ordinals to keep the bucket ords dense.
|
||||
// Additionally, if using partitioned terms the regular global
|
||||
// Additionally, if using partitioned terms the regular global
|
||||
// ordinals would be sparse so we opt for hash
|
||||
if (Aggregator.descendsFromBucketAggregator(parent) ||
|
||||
(includeExclude != null && includeExclude.isPartitionBased())) {
|
||||
|
@ -223,7 +225,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
|
|||
MAP(new ParseField("map")) {
|
||||
|
||||
@Override
|
||||
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, Terms.Order order,
|
||||
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, BucketOrder order,
|
||||
DocValueFormat format, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
|
||||
SearchContext context, Aggregator parent, SubAggCollectionMode subAggCollectMode,
|
||||
boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
|
||||
|
@ -242,7 +244,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
|
|||
GLOBAL_ORDINALS(new ParseField("global_ordinals")) {
|
||||
|
||||
@Override
|
||||
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, Terms.Order order,
|
||||
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, BucketOrder order,
|
||||
DocValueFormat format, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
|
||||
SearchContext context, Aggregator parent, SubAggCollectionMode subAggCollectMode,
|
||||
boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
|
||||
|
@ -262,7 +264,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
|
|||
GLOBAL_ORDINALS_HASH(new ParseField("global_ordinals_hash")) {
|
||||
|
||||
@Override
|
||||
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, Terms.Order order,
|
||||
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, BucketOrder order,
|
||||
DocValueFormat format, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
|
||||
SearchContext context, Aggregator parent, SubAggCollectionMode subAggCollectMode,
|
||||
boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
|
||||
|
@ -281,7 +283,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
|
|||
GLOBAL_ORDINALS_LOW_CARDINALITY(new ParseField("global_ordinals_low_cardinality")) {
|
||||
|
||||
@Override
|
||||
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, Terms.Order order,
|
||||
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, BucketOrder order,
|
||||
DocValueFormat format, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
|
||||
SearchContext context, Aggregator parent, SubAggCollectionMode subAggCollectMode,
|
||||
boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
|
||||
|
@ -319,7 +321,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
|
|||
this.parseField = parseField;
|
||||
}
|
||||
|
||||
abstract Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, Terms.Order order,
|
||||
abstract Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, BucketOrder order,
|
||||
DocValueFormat format, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
|
||||
SearchContext context, Aggregator parent, SubAggCollectionMode subAggCollectMode,
|
||||
boolean showTermDocCountError, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.elasticsearch.search.DocValueFormat;
|
|||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
|
@ -50,7 +51,7 @@ public class UnmappedTerms extends InternalTerms<UnmappedTerms, UnmappedTerms.Bu
|
|||
}
|
||||
}
|
||||
|
||||
public UnmappedTerms(String name, Terms.Order order, int requiredSize, long minDocCount,
|
||||
public UnmappedTerms(String name, BucketOrder order, int requiredSize, long minDocCount,
|
||||
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) {
|
||||
super(name, order, requiredSize, minDocCount, pipelineAggregators, metaData);
|
||||
}
|
||||
|
|
|
@ -33,7 +33,7 @@ import java.util.ArrayList;
|
|||
import java.util.List;
|
||||
|
||||
/**
|
||||
* A path that can be used to sort/order buckets (in some multi-bucket aggregations, eg terms & histogram) based on
|
||||
* A path that can be used to sort/order buckets (in some multi-bucket aggregations, e.g. terms & histogram) based on
|
||||
* sub-aggregations. The path may point to either a single-bucket aggregation or a metrics aggregation. If the path
|
||||
* points to a single-bucket aggregation, the sort will be applied based on the {@code doc_count} of the bucket. If this
|
||||
* path points to a metrics aggregation, if it's a single-value metrics (eg. avg, max, min, etc..) the sort will be
|
||||
|
@ -281,14 +281,15 @@ public class AggregationPath {
|
|||
/**
|
||||
* Validates this path over the given aggregator as a point of reference.
|
||||
*
|
||||
* @param root The point of reference of this path
|
||||
* @param root The point of reference of this path
|
||||
* @throws AggregationExecutionException on validation error
|
||||
*/
|
||||
public void validate(Aggregator root) {
|
||||
public void validate(Aggregator root) throws AggregationExecutionException {
|
||||
Aggregator aggregator = root;
|
||||
for (int i = 0; i < pathElements.size(); i++) {
|
||||
aggregator = aggregator.subAggregator(pathElements.get(i).name);
|
||||
if (aggregator == null) {
|
||||
throw new AggregationExecutionException("Invalid term-aggregator order path [" + this + "]. Unknown aggregation ["
|
||||
throw new AggregationExecutionException("Invalid aggregator order path [" + this + "]. Unknown aggregation ["
|
||||
+ pathElements.get(i).name + "]");
|
||||
}
|
||||
if (i < pathElements.size() - 1) {
|
||||
|
@ -296,16 +297,16 @@ public class AggregationPath {
|
|||
// we're in the middle of the path, so the aggregator can only be a single-bucket aggregator
|
||||
|
||||
if (!(aggregator instanceof SingleBucketAggregator)) {
|
||||
throw new AggregationExecutionException("Invalid terms aggregation order path [" + this +
|
||||
"]. Terms buckets can only be sorted on a sub-aggregator path " +
|
||||
throw new AggregationExecutionException("Invalid aggregation order path [" + this +
|
||||
"]. Buckets can only be sorted on a sub-aggregator path " +
|
||||
"that is built out of zero or more single-bucket aggregations within the path and a final " +
|
||||
"single-bucket or a metrics aggregation at the path end. Sub-path [" +
|
||||
subPath(0, i + 1) + "] points to non single-bucket aggregation");
|
||||
}
|
||||
|
||||
if (pathElements.get(i).key != null) {
|
||||
throw new AggregationExecutionException("Invalid terms aggregation order path [" + this +
|
||||
"]. Terms buckets can only be sorted on a sub-aggregator path " +
|
||||
throw new AggregationExecutionException("Invalid aggregation order path [" + this +
|
||||
"]. Buckets can only be sorted on a sub-aggregator path " +
|
||||
"that is built out of zero or more single-bucket aggregations within the path and a " +
|
||||
"final single-bucket or a metrics aggregation at the path end. Sub-path [" +
|
||||
subPath(0, i + 1) + "] points to non single-bucket aggregation");
|
||||
|
@ -314,8 +315,8 @@ public class AggregationPath {
|
|||
}
|
||||
boolean singleBucket = aggregator instanceof SingleBucketAggregator;
|
||||
if (!singleBucket && !(aggregator instanceof NumericMetricsAggregator)) {
|
||||
throw new AggregationExecutionException("Invalid terms aggregation order path [" + this +
|
||||
"]. Terms buckets can only be sorted on a sub-aggregator path " +
|
||||
throw new AggregationExecutionException("Invalid aggregation order path [" + this +
|
||||
"]. Buckets can only be sorted on a sub-aggregator path " +
|
||||
"that is built out of zero or more single-bucket aggregations within the path and a final " +
|
||||
"single-bucket or a metrics aggregation at the path end.");
|
||||
}
|
||||
|
@ -324,7 +325,7 @@ public class AggregationPath {
|
|||
|
||||
if (singleBucket) {
|
||||
if (lastToken.key != null && !"doc_count".equals(lastToken.key)) {
|
||||
throw new AggregationExecutionException("Invalid terms aggregation order path [" + this +
|
||||
throw new AggregationExecutionException("Invalid aggregation order path [" + this +
|
||||
"]. Ordering on a single-bucket aggregation can only be done on its doc_count. " +
|
||||
"Either drop the key (a la \"" + lastToken.name + "\") or change it to \"doc_count\" (a la \"" + lastToken.name + ".doc_count\")");
|
||||
}
|
||||
|
@ -333,7 +334,7 @@ public class AggregationPath {
|
|||
|
||||
if (aggregator instanceof NumericMetricsAggregator.SingleValue) {
|
||||
if (lastToken.key != null && !"value".equals(lastToken.key)) {
|
||||
throw new AggregationExecutionException("Invalid terms aggregation order path [" + this +
|
||||
throw new AggregationExecutionException("Invalid aggregation order path [" + this +
|
||||
"]. Ordering on a single-value metrics aggregation can only be done on its value. " +
|
||||
"Either drop the key (a la \"" + lastToken.name + "\") or change it to \"value\" (a la \"" + lastToken.name + ".value\")");
|
||||
}
|
||||
|
@ -342,12 +343,12 @@ public class AggregationPath {
|
|||
|
||||
// the aggregator must be of a multi-value metrics type
|
||||
if (lastToken.key == null) {
|
||||
throw new AggregationExecutionException("Invalid terms aggregation order path [" + this +
|
||||
throw new AggregationExecutionException("Invalid aggregation order path [" + this +
|
||||
"]. When ordering on a multi-value metrics aggregation a metric name must be specified");
|
||||
}
|
||||
|
||||
if (!((NumericMetricsAggregator.MultiValue) aggregator).hasMetric(lastToken.key)) {
|
||||
throw new AggregationExecutionException("Invalid terms aggregation order path [" + this +
|
||||
throw new AggregationExecutionException("Invalid aggregation order path [" + this +
|
||||
"]. Unknown metric name [" + lastToken.key + "] on multi-value metrics aggregation [" + lastToken.name + "]");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,19 +19,31 @@
|
|||
|
||||
package org.elasticsearch.search.fetch.subphase;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.ReaderUtil;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.BooleanClause.Occur;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.CollectionTerminatedException;
|
||||
import org.apache.lucene.search.Collector;
|
||||
import org.apache.lucene.search.ConjunctionDISI;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.DocValuesTermsQuery;
|
||||
import org.apache.lucene.search.LeafCollector;
|
||||
import org.apache.lucene.search.MatchNoDocsQuery;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.ScorerSupplier;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
import org.apache.lucene.search.TopDocsCollector;
|
||||
import org.apache.lucene.search.TopFieldCollector;
|
||||
import org.apache.lucene.search.TopScoreDocCollector;
|
||||
import org.apache.lucene.search.TotalHitCountCollector;
|
||||
import org.apache.lucene.search.Weight;
|
||||
import org.apache.lucene.search.join.BitSetProducer;
|
||||
import org.apache.lucene.search.join.ParentChildrenBlockJoinQuery;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.elasticsearch.common.lucene.Lucene;
|
||||
import org.elasticsearch.common.lucene.search.Queries;
|
||||
import org.elasticsearch.index.mapper.DocumentMapper;
|
||||
|
@ -40,11 +52,11 @@ import org.elasticsearch.index.mapper.ObjectMapper;
|
|||
import org.elasticsearch.index.mapper.ParentFieldMapper;
|
||||
import org.elasticsearch.search.SearchHit;
|
||||
import org.elasticsearch.search.SearchHitField;
|
||||
import org.elasticsearch.search.fetch.FetchSubPhase;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
import org.elasticsearch.search.internal.SubSearchContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
@ -57,7 +69,7 @@ public final class InnerHitsContext {
|
|||
this.innerHits = new HashMap<>();
|
||||
}
|
||||
|
||||
public InnerHitsContext(Map<String, BaseInnerHits> innerHits) {
|
||||
InnerHitsContext(Map<String, BaseInnerHits> innerHits) {
|
||||
this.innerHits = Objects.requireNonNull(innerHits);
|
||||
}
|
||||
|
||||
|
@ -77,14 +89,16 @@ public final class InnerHitsContext {
|
|||
public abstract static class BaseInnerHits extends SubSearchContext {
|
||||
|
||||
private final String name;
|
||||
final SearchContext context;
|
||||
private InnerHitsContext childInnerHits;
|
||||
|
||||
protected BaseInnerHits(String name, SearchContext context) {
|
||||
BaseInnerHits(String name, SearchContext context) {
|
||||
super(context);
|
||||
this.name = name;
|
||||
this.context = context;
|
||||
}
|
||||
|
||||
public abstract TopDocs topDocs(SearchContext context, FetchSubPhase.HitContext hitContext) throws IOException;
|
||||
public abstract TopDocs[] topDocs(SearchHit[] hits) throws IOException;
|
||||
|
||||
public String getName() {
|
||||
return name;
|
||||
|
@ -98,6 +112,12 @@ public final class InnerHitsContext {
|
|||
public void setChildInnerHits(Map<String, InnerHitsContext.BaseInnerHits> childInnerHits) {
|
||||
this.childInnerHits = new InnerHitsContext(childInnerHits);
|
||||
}
|
||||
|
||||
Weight createInnerHitQueryWeight() throws IOException {
|
||||
final boolean needsScores = size() != 0 && (sort() == null || sort().sort.needsScores());
|
||||
return context.searcher().createNormalizedWeight(query(), needsScores);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public static final class NestedInnerHits extends BaseInnerHits {
|
||||
|
@ -112,35 +132,48 @@ public final class InnerHitsContext {
|
|||
}
|
||||
|
||||
@Override
|
||||
public TopDocs topDocs(SearchContext context, FetchSubPhase.HitContext hitContext) throws IOException {
|
||||
Query rawParentFilter;
|
||||
if (parentObjectMapper == null) {
|
||||
rawParentFilter = Queries.newNonNestedFilter();
|
||||
} else {
|
||||
rawParentFilter = parentObjectMapper.nestedTypeFilter();
|
||||
}
|
||||
BitSetProducer parentFilter = context.bitsetFilterCache().getBitSetProducer(rawParentFilter);
|
||||
Query childFilter = childObjectMapper.nestedTypeFilter();
|
||||
int parentDocId = hitContext.readerContext().docBase + hitContext.docId();
|
||||
Query q = Queries.filtered(query(), new ParentChildrenBlockJoinQuery(parentFilter, childFilter, parentDocId));
|
||||
|
||||
if (size() == 0) {
|
||||
return new TopDocs(context.searcher().count(q), Lucene.EMPTY_SCORE_DOCS, 0);
|
||||
} else {
|
||||
int topN = Math.min(from() + size(), context.searcher().getIndexReader().maxDoc());
|
||||
TopDocsCollector<?> topDocsCollector;
|
||||
if (sort() != null) {
|
||||
topDocsCollector = TopFieldCollector.create(sort().sort, topN, true, trackScores(), trackScores());
|
||||
public TopDocs[] topDocs(SearchHit[] hits) throws IOException {
|
||||
Weight innerHitQueryWeight = createInnerHitQueryWeight();
|
||||
TopDocs[] result = new TopDocs[hits.length];
|
||||
for (int i = 0; i < hits.length; i++) {
|
||||
SearchHit hit = hits[i];
|
||||
Query rawParentFilter;
|
||||
if (parentObjectMapper == null) {
|
||||
rawParentFilter = Queries.newNonNestedFilter();
|
||||
} else {
|
||||
topDocsCollector = TopScoreDocCollector.create(topN);
|
||||
rawParentFilter = parentObjectMapper.nestedTypeFilter();
|
||||
}
|
||||
try {
|
||||
context.searcher().search(q, topDocsCollector);
|
||||
} finally {
|
||||
clearReleasables(Lifetime.COLLECTION);
|
||||
|
||||
int parentDocId = hit.docId();
|
||||
final int readerIndex = ReaderUtil.subIndex(parentDocId, searcher().getIndexReader().leaves());
|
||||
// With nested inner hits the nested docs are always in the same segement, so need to use the other segments
|
||||
LeafReaderContext ctx = searcher().getIndexReader().leaves().get(readerIndex);
|
||||
|
||||
Query childFilter = childObjectMapper.nestedTypeFilter();
|
||||
BitSetProducer parentFilter = context.bitsetFilterCache().getBitSetProducer(rawParentFilter);
|
||||
Query q = new ParentChildrenBlockJoinQuery(parentFilter, childFilter, parentDocId);
|
||||
Weight weight = context.searcher().createNormalizedWeight(q, false);
|
||||
if (size() == 0) {
|
||||
TotalHitCountCollector totalHitCountCollector = new TotalHitCountCollector();
|
||||
intersect(weight, innerHitQueryWeight, totalHitCountCollector, ctx);
|
||||
result[i] = new TopDocs(totalHitCountCollector.getTotalHits(), Lucene.EMPTY_SCORE_DOCS, 0);
|
||||
} else {
|
||||
int topN = Math.min(from() + size(), context.searcher().getIndexReader().maxDoc());
|
||||
TopDocsCollector<?> topDocsCollector;
|
||||
if (sort() != null) {
|
||||
topDocsCollector = TopFieldCollector.create(sort().sort, topN, true, trackScores(), trackScores());
|
||||
} else {
|
||||
topDocsCollector = TopScoreDocCollector.create(topN);
|
||||
}
|
||||
try {
|
||||
intersect(weight, innerHitQueryWeight, topDocsCollector, ctx);
|
||||
} finally {
|
||||
clearReleasables(Lifetime.COLLECTION);
|
||||
}
|
||||
result[i] = topDocsCollector.topDocs(from(), size());
|
||||
}
|
||||
return topDocsCollector.topDocs(from(), size());
|
||||
}
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -156,53 +189,65 @@ public final class InnerHitsContext {
|
|||
}
|
||||
|
||||
@Override
|
||||
public TopDocs topDocs(SearchContext context, FetchSubPhase.HitContext hitContext) throws IOException {
|
||||
final Query hitQuery;
|
||||
if (isParentHit(hitContext.hit())) {
|
||||
String field = ParentFieldMapper.joinField(hitContext.hit().getType());
|
||||
hitQuery = new DocValuesTermsQuery(field, hitContext.hit().getId());
|
||||
} else if (isChildHit(hitContext.hit())) {
|
||||
DocumentMapper hitDocumentMapper = mapperService.documentMapper(hitContext.hit().getType());
|
||||
final String parentType = hitDocumentMapper.parentFieldMapper().type();
|
||||
SearchHitField parentField = hitContext.hit().field(ParentFieldMapper.NAME);
|
||||
if (parentField == null) {
|
||||
throw new IllegalStateException("All children must have a _parent");
|
||||
}
|
||||
Term uidTerm = context.mapperService().createUidTerm(parentType, parentField.getValue());
|
||||
if (uidTerm == null) {
|
||||
hitQuery = new MatchNoDocsQuery("Missing type: " + parentType);
|
||||
public TopDocs[] topDocs(SearchHit[] hits) throws IOException {
|
||||
Weight innerHitQueryWeight = createInnerHitQueryWeight();
|
||||
TopDocs[] result = new TopDocs[hits.length];
|
||||
for (int i = 0; i < hits.length; i++) {
|
||||
SearchHit hit = hits[i];
|
||||
final Query hitQuery;
|
||||
if (isParentHit(hit)) {
|
||||
String field = ParentFieldMapper.joinField(hit.getType());
|
||||
hitQuery = new DocValuesTermsQuery(field, hit.getId());
|
||||
} else if (isChildHit(hit)) {
|
||||
DocumentMapper hitDocumentMapper = mapperService.documentMapper(hit.getType());
|
||||
final String parentType = hitDocumentMapper.parentFieldMapper().type();
|
||||
SearchHitField parentField = hit.field(ParentFieldMapper.NAME);
|
||||
if (parentField == null) {
|
||||
throw new IllegalStateException("All children must have a _parent");
|
||||
}
|
||||
Term uidTerm = context.mapperService().createUidTerm(parentType, parentField.getValue());
|
||||
if (uidTerm == null) {
|
||||
hitQuery = new MatchNoDocsQuery("Missing type: " + parentType);
|
||||
} else {
|
||||
hitQuery = new TermQuery(uidTerm);
|
||||
}
|
||||
} else {
|
||||
hitQuery = new TermQuery(uidTerm);
|
||||
result[i] = Lucene.EMPTY_TOP_DOCS;
|
||||
continue;
|
||||
}
|
||||
} else {
|
||||
return Lucene.EMPTY_TOP_DOCS;
|
||||
}
|
||||
|
||||
BooleanQuery q = new BooleanQuery.Builder()
|
||||
.add(query(), Occur.MUST)
|
||||
// Only include docs that have the current hit as parent
|
||||
.add(hitQuery, Occur.FILTER)
|
||||
// Only include docs that have this inner hits type
|
||||
.add(documentMapper.typeFilter(context.getQueryShardContext()), Occur.FILTER)
|
||||
.build();
|
||||
if (size() == 0) {
|
||||
final int count = context.searcher().count(q);
|
||||
return new TopDocs(count, Lucene.EMPTY_SCORE_DOCS, 0);
|
||||
} else {
|
||||
int topN = Math.min(from() + size(), context.searcher().getIndexReader().maxDoc());
|
||||
TopDocsCollector topDocsCollector;
|
||||
if (sort() != null) {
|
||||
topDocsCollector = TopFieldCollector.create(sort().sort, topN, true, trackScores(), trackScores());
|
||||
BooleanQuery q = new BooleanQuery.Builder()
|
||||
// Only include docs that have the current hit as parent
|
||||
.add(hitQuery, Occur.FILTER)
|
||||
// Only include docs that have this inner hits type
|
||||
.add(documentMapper.typeFilter(context.getQueryShardContext()), Occur.FILTER)
|
||||
.build();
|
||||
Weight weight = context.searcher().createNormalizedWeight(q, false);
|
||||
if (size() == 0) {
|
||||
TotalHitCountCollector totalHitCountCollector = new TotalHitCountCollector();
|
||||
for (LeafReaderContext ctx : context.searcher().getIndexReader().leaves()) {
|
||||
intersect(weight, innerHitQueryWeight, totalHitCountCollector, ctx);
|
||||
}
|
||||
result[i] = new TopDocs(totalHitCountCollector.getTotalHits(), Lucene.EMPTY_SCORE_DOCS, 0);
|
||||
} else {
|
||||
topDocsCollector = TopScoreDocCollector.create(topN);
|
||||
int topN = Math.min(from() + size(), context.searcher().getIndexReader().maxDoc());
|
||||
TopDocsCollector topDocsCollector;
|
||||
if (sort() != null) {
|
||||
topDocsCollector = TopFieldCollector.create(sort().sort, topN, true, trackScores(), trackScores());
|
||||
} else {
|
||||
topDocsCollector = TopScoreDocCollector.create(topN);
|
||||
}
|
||||
try {
|
||||
for (LeafReaderContext ctx : context.searcher().getIndexReader().leaves()) {
|
||||
intersect(weight, innerHitQueryWeight, topDocsCollector, ctx);
|
||||
}
|
||||
} finally {
|
||||
clearReleasables(Lifetime.COLLECTION);
|
||||
}
|
||||
result[i] = topDocsCollector.topDocs(from(), size());
|
||||
}
|
||||
try {
|
||||
context.searcher().search(q, topDocsCollector);
|
||||
} finally {
|
||||
clearReleasables(Lifetime.COLLECTION);
|
||||
}
|
||||
return topDocsCollector.topDocs(from(), size());
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
private boolean isParentHit(SearchHit hit) {
|
||||
|
@ -214,4 +259,42 @@ public final class InnerHitsContext {
|
|||
return documentMapper.type().equals(hitDocumentMapper.parentFieldMapper().type());
|
||||
}
|
||||
}
|
||||
|
||||
static void intersect(Weight weight, Weight innerHitQueryWeight, Collector collector, LeafReaderContext ctx) throws IOException {
|
||||
ScorerSupplier scorerSupplier = weight.scorerSupplier(ctx);
|
||||
if (scorerSupplier == null) {
|
||||
return;
|
||||
}
|
||||
// use random access since this scorer will be consumed on a minority of documents
|
||||
Scorer scorer = scorerSupplier.get(true);
|
||||
|
||||
ScorerSupplier innerHitQueryScorerSupplier = innerHitQueryWeight.scorerSupplier(ctx);
|
||||
if (innerHitQueryScorerSupplier == null) {
|
||||
return;
|
||||
}
|
||||
// use random access since this scorer will be consumed on a minority of documents
|
||||
Scorer innerHitQueryScorer = innerHitQueryScorerSupplier.get(true);
|
||||
|
||||
final LeafCollector leafCollector;
|
||||
try {
|
||||
leafCollector = collector.getLeafCollector(ctx);
|
||||
// Just setting the innerHitQueryScorer is ok, because that is the actual scoring part of the query
|
||||
leafCollector.setScorer(innerHitQueryScorer);
|
||||
} catch (CollectionTerminatedException e) {
|
||||
return;
|
||||
}
|
||||
|
||||
try {
|
||||
Bits acceptDocs = ctx.reader().getLiveDocs();
|
||||
DocIdSetIterator iterator = ConjunctionDISI.intersectIterators(Arrays.asList(innerHitQueryScorer.iterator(),
|
||||
scorer.iterator()));
|
||||
for (int docId = iterator.nextDoc(); docId < DocIdSetIterator.NO_MORE_DOCS; docId = iterator.nextDoc()) {
|
||||
if (acceptDocs == null || acceptDocs.get(docId)) {
|
||||
leafCollector.collect(docId);
|
||||
}
|
||||
}
|
||||
} catch (CollectionTerminatedException e) {
|
||||
// ignore and continue
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,12 +22,11 @@ package org.elasticsearch.search.fetch.subphase;
|
|||
import org.apache.lucene.search.FieldDoc;
|
||||
import org.apache.lucene.search.ScoreDoc;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.search.SearchHit;
|
||||
import org.elasticsearch.search.SearchHits;
|
||||
import org.elasticsearch.search.fetch.FetchPhase;
|
||||
import org.elasticsearch.search.fetch.FetchSearchResult;
|
||||
import org.elasticsearch.search.fetch.FetchSubPhase;
|
||||
import org.elasticsearch.search.SearchHit;
|
||||
import org.elasticsearch.search.SearchHits;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -43,39 +42,42 @@ public final class InnerHitsFetchSubPhase implements FetchSubPhase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void hitExecute(SearchContext context, HitContext hitContext) {
|
||||
public void hitsExecute(SearchContext context, SearchHit[] hits) throws IOException {
|
||||
if ((context.innerHits() != null && context.innerHits().getInnerHits().size() > 0) == false) {
|
||||
return;
|
||||
}
|
||||
Map<String, SearchHits> results = new HashMap<>();
|
||||
|
||||
for (Map.Entry<String, InnerHitsContext.BaseInnerHits> entry : context.innerHits().getInnerHits().entrySet()) {
|
||||
InnerHitsContext.BaseInnerHits innerHits = entry.getValue();
|
||||
TopDocs topDocs;
|
||||
try {
|
||||
topDocs = innerHits.topDocs(context, hitContext);
|
||||
} catch (IOException e) {
|
||||
throw ExceptionsHelper.convertToElastic(e);
|
||||
}
|
||||
innerHits.queryResult().topDocs(topDocs, innerHits.sort() == null ? null : innerHits.sort().formats);
|
||||
int[] docIdsToLoad = new int[topDocs.scoreDocs.length];
|
||||
for (int i = 0; i < topDocs.scoreDocs.length; i++) {
|
||||
docIdsToLoad[i] = topDocs.scoreDocs[i].doc;
|
||||
}
|
||||
innerHits.docIdsToLoad(docIdsToLoad, 0, docIdsToLoad.length);
|
||||
fetchPhase.execute(innerHits);
|
||||
FetchSearchResult fetchResult = innerHits.fetchResult();
|
||||
SearchHit[] internalHits = fetchResult.fetchResult().hits().internalHits();
|
||||
for (int i = 0; i < internalHits.length; i++) {
|
||||
ScoreDoc scoreDoc = topDocs.scoreDocs[i];
|
||||
SearchHit searchHitFields = internalHits[i];
|
||||
searchHitFields.score(scoreDoc.score);
|
||||
if (scoreDoc instanceof FieldDoc) {
|
||||
FieldDoc fieldDoc = (FieldDoc) scoreDoc;
|
||||
searchHitFields.sortValues(fieldDoc.fields, innerHits.sort().formats);
|
||||
TopDocs[] topDocs = innerHits.topDocs(hits);
|
||||
for (int i = 0; i < hits.length; i++) {
|
||||
SearchHit hit = hits[i];
|
||||
TopDocs topDoc = topDocs[i];
|
||||
|
||||
Map<String, SearchHits> results = hit.getInnerHits();
|
||||
if (results == null) {
|
||||
hit.setInnerHits(results = new HashMap<>());
|
||||
}
|
||||
innerHits.queryResult().topDocs(topDoc, innerHits.sort() == null ? null : innerHits.sort().formats);
|
||||
int[] docIdsToLoad = new int[topDoc.scoreDocs.length];
|
||||
for (int j = 0; j < topDoc.scoreDocs.length; j++) {
|
||||
docIdsToLoad[j] = topDoc.scoreDocs[j].doc;
|
||||
}
|
||||
innerHits.docIdsToLoad(docIdsToLoad, 0, docIdsToLoad.length);
|
||||
fetchPhase.execute(innerHits);
|
||||
FetchSearchResult fetchResult = innerHits.fetchResult();
|
||||
SearchHit[] internalHits = fetchResult.fetchResult().hits().internalHits();
|
||||
for (int j = 0; j < internalHits.length; j++) {
|
||||
ScoreDoc scoreDoc = topDoc.scoreDocs[j];
|
||||
SearchHit searchHitFields = internalHits[j];
|
||||
searchHitFields.score(scoreDoc.score);
|
||||
if (scoreDoc instanceof FieldDoc) {
|
||||
FieldDoc fieldDoc = (FieldDoc) scoreDoc;
|
||||
searchHitFields.sortValues(fieldDoc.fields, innerHits.sort().formats);
|
||||
}
|
||||
}
|
||||
results.put(entry.getKey(), fetchResult.hits());
|
||||
}
|
||||
results.put(entry.getKey(), fetchResult.hits());
|
||||
}
|
||||
hitContext.hit().setInnerHits(results);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -58,7 +58,7 @@ public final class DirectCandidateGenerator extends CandidateGenerator {
|
|||
private final TermsEnum termsEnum;
|
||||
private final IndexReader reader;
|
||||
private final long dictSize;
|
||||
private final double logBase = 5;
|
||||
private static final double LOG_BASE = 5;
|
||||
private final long frequencyPlateau;
|
||||
private final Analyzer preFilter;
|
||||
private final Analyzer postFilter;
|
||||
|
@ -189,7 +189,7 @@ public final class DirectCandidateGenerator extends CandidateGenerator {
|
|||
|
||||
protected long thresholdFrequency(long termFrequency, long dictionarySize) {
|
||||
if (termFrequency > 0) {
|
||||
return max(0, round(termFrequency * (log10(termFrequency - frequencyPlateau) * (1.0 / log10(logBase))) + 1));
|
||||
return max(0, round(termFrequency * (log10(termFrequency - frequencyPlateau) * (1.0 / log10(LOG_BASE))) + 1));
|
||||
}
|
||||
return 0;
|
||||
|
||||
|
|
|
@ -0,0 +1,33 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.bootstrap;
|
||||
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
public class BootstrapTests extends ESTestCase {
|
||||
|
||||
public void testConfigDeprecation() {
|
||||
Bootstrap.checkConfigExtension(".json");
|
||||
assertWarnings("elasticsearch.json is deprecated; rename your configuration file to elasticsearch.yaml");
|
||||
Bootstrap.checkConfigExtension(".yml");
|
||||
assertWarnings("elasticsearch.yml is deprecated; rename your configuration file to elasticsearch.yaml");
|
||||
Bootstrap.checkConfigExtension(".yaml"); // no warnings, will be checked in @After
|
||||
}
|
||||
}
|
|
@ -19,6 +19,7 @@
|
|||
package org.elasticsearch.index.query;
|
||||
|
||||
import org.apache.lucene.search.join.ScoreMode;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
@ -311,13 +312,16 @@ public class InnerHitBuilderTests extends ESTestCase {
|
|||
}
|
||||
innerHits.setScriptFields(new HashSet<>(scriptFields.values()));
|
||||
FetchSourceContext randomFetchSourceContext;
|
||||
if (randomBoolean()) {
|
||||
randomFetchSourceContext = new FetchSourceContext(randomBoolean());
|
||||
} else {
|
||||
int randomInt = randomIntBetween(0, 2);
|
||||
if (randomInt == 0) {
|
||||
randomFetchSourceContext = new FetchSourceContext(true, Strings.EMPTY_ARRAY, Strings.EMPTY_ARRAY);
|
||||
} else if (randomInt == 1) {
|
||||
randomFetchSourceContext = new FetchSourceContext(true,
|
||||
generateRandomStringArray(12, 16, false),
|
||||
generateRandomStringArray(12, 16, false)
|
||||
);
|
||||
} else {
|
||||
randomFetchSourceContext = new FetchSourceContext(randomBoolean());
|
||||
}
|
||||
innerHits.setFetchSourceContext(randomFetchSourceContext);
|
||||
if (randomBoolean()) {
|
||||
|
|
|
@ -32,6 +32,7 @@ import org.elasticsearch.action.termvectors.MultiTermVectorsRequest;
|
|||
import org.elasticsearch.action.termvectors.MultiTermVectorsResponse;
|
||||
import org.elasticsearch.action.termvectors.TermVectorsRequest;
|
||||
import org.elasticsearch.action.termvectors.TermVectorsResponse;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.lucene.search.MoreLikeThisQuery;
|
||||
|
@ -64,6 +65,8 @@ import static org.hamcrest.Matchers.instanceOf;
|
|||
|
||||
public class MoreLikeThisQueryBuilderTests extends AbstractQueryTestCase<MoreLikeThisQueryBuilder> {
|
||||
|
||||
private static final String[] SHUFFLE_PROTECTED_FIELDS = new String[]{Item.Field.DOC.getPreferredName()};
|
||||
|
||||
private static String[] randomFields;
|
||||
private static Item[] randomLikeItems;
|
||||
private static Item[] randomUnlikeItems;
|
||||
|
@ -204,6 +207,16 @@ public class MoreLikeThisQueryBuilderTests extends AbstractQueryTestCase<MoreLik
|
|||
return queryBuilder;
|
||||
}
|
||||
|
||||
/**
|
||||
* we don't want to shuffle the "doc" field internally in {@link #testFromXContent()} because even though the
|
||||
* documents would be functionally the same, their {@link BytesReference} representation isn't and thats what we
|
||||
* compare when check for equality of the original and the shuffled builder
|
||||
*/
|
||||
@Override
|
||||
protected String[] shuffleProtectedFields() {
|
||||
return SHUFFLE_PROTECTED_FIELDS;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Set<String> getObjectsHoldingArbitraryContent() {
|
||||
//doc contains arbitrary content, anything can be added to it and no exception will be thrown
|
||||
|
|
|
@ -83,6 +83,9 @@ import static org.hamcrest.Matchers.nullValue;
|
|||
|
||||
public class FunctionScoreQueryBuilderTests extends AbstractQueryTestCase<FunctionScoreQueryBuilder> {
|
||||
|
||||
private static final String[] SHUFFLE_PROTECTED_FIELDS = new String[] {Script.PARAMS_PARSE_FIELD.getPreferredName(),
|
||||
ExponentialDecayFunctionBuilder.NAME, LinearDecayFunctionBuilder.NAME, GaussDecayFunctionBuilder.NAME};
|
||||
|
||||
@Override
|
||||
protected Collection<Class<? extends Plugin>> getPlugins() {
|
||||
return Collections.singleton(TestPlugin.class);
|
||||
|
@ -106,6 +109,12 @@ public class FunctionScoreQueryBuilderTests extends AbstractQueryTestCase<Functi
|
|||
return functionScoreQueryBuilder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String[] shuffleProtectedFields() {
|
||||
// do not shuffle fields that may contain arbitrary content
|
||||
return SHUFFLE_PROTECTED_FIELDS;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Set<String> getObjectsHoldingArbitraryContent() {
|
||||
//script_score.script.params can contain arbitrary parameters. no error is expected when adding additional objects
|
||||
|
@ -218,7 +227,7 @@ public class FunctionScoreQueryBuilderTests extends AbstractQueryTestCase<Functi
|
|||
break;
|
||||
case DATE_FIELD_NAME:
|
||||
origin = new DateTime(System.currentTimeMillis() - randomIntBetween(0, 1000000), DateTimeZone.UTC).toString();
|
||||
scale = randomPositiveTimeValue();
|
||||
scale = randomTimeValue(1, 1000, new String[]{"d", "h", "ms", "s", "m"});
|
||||
offset = randomPositiveTimeValue();
|
||||
break;
|
||||
default:
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.action.bulk.byscroll;
|
||||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.support.ActiveShardCount;
|
|
@ -17,12 +17,11 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.action.bulk.byscroll;
|
||||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.bulk.BulkItemResponse.Failure;
|
||||
import org.elasticsearch.action.bulk.byscroll.ScrollableHitSource.SearchFailure;
|
||||
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
@ -56,7 +55,7 @@ public class BulkByScrollResponseTests extends ESTestCase {
|
|||
randomSimpleString(random()), new IllegalArgumentException("test")));
|
||||
}
|
||||
|
||||
private List<SearchFailure> randomSearchFailures() {
|
||||
private List<ScrollableHitSource.SearchFailure> randomSearchFailures() {
|
||||
if (randomBoolean()) {
|
||||
return emptyList();
|
||||
}
|
||||
|
@ -68,7 +67,7 @@ public class BulkByScrollResponseTests extends ESTestCase {
|
|||
shardId = randomInt();
|
||||
nodeId = usually() ? randomAlphaOfLength(5) : null;
|
||||
}
|
||||
return singletonList(new SearchFailure(new ElasticsearchException("foo"), index, shardId, nodeId));
|
||||
return singletonList(new ScrollableHitSource.SearchFailure(new ElasticsearchException("foo"), index, shardId, nodeId));
|
||||
}
|
||||
|
||||
private void assertResponseEquals(BulkByScrollResponse expected, BulkByScrollResponse actual) {
|
||||
|
@ -86,8 +85,8 @@ public class BulkByScrollResponseTests extends ESTestCase {
|
|||
}
|
||||
assertEquals(expected.getSearchFailures().size(), actual.getSearchFailures().size());
|
||||
for (int i = 0; i < expected.getSearchFailures().size(); i++) {
|
||||
SearchFailure expectedFailure = expected.getSearchFailures().get(i);
|
||||
SearchFailure actualFailure = actual.getSearchFailures().get(i);
|
||||
ScrollableHitSource.SearchFailure expectedFailure = expected.getSearchFailures().get(i);
|
||||
ScrollableHitSource.SearchFailure actualFailure = actual.getSearchFailures().get(i);
|
||||
assertEquals(expectedFailure.getIndex(), actualFailure.getIndex());
|
||||
assertEquals(expectedFailure.getShardId(), actualFailure.getShardId());
|
||||
assertEquals(expectedFailure.getNodeId(), actualFailure.getNodeId());
|
|
@ -17,7 +17,7 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.action.bulk.byscroll;
|
||||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
|
@ -26,6 +26,7 @@ import org.elasticsearch.common.Randomness;
|
|||
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.hamcrest.Matchers;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
@ -75,7 +76,7 @@ public class BulkByScrollTaskStatusTests extends ESTestCase {
|
|||
assertEquals(expected.getReasonCancelled(), actual.getReasonCancelled());
|
||||
assertEquals(expected.getThrottledUntil(), actual.getThrottledUntil());
|
||||
if (version.onOrAfter(Version.V_5_1_1_UNRELEASED)) {
|
||||
assertThat(actual.getSliceStatuses(), hasSize(expected.getSliceStatuses().size()));
|
||||
assertThat(actual.getSliceStatuses(), Matchers.hasSize(expected.getSliceStatuses().size()));
|
||||
for (int i = 0; i < expected.getSliceStatuses().size(); i++) {
|
||||
BulkByScrollTask.StatusOrException sliceStatus = expected.getSliceStatuses().get(i);
|
||||
if (sliceStatus == null) {
|
|
@ -17,7 +17,7 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.action.bulk.byscroll;
|
||||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
|
@ -17,7 +17,7 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.action.bulk.byscroll;
|
||||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.elasticsearch.action.ActionRequestValidationException;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
|
@ -17,7 +17,7 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.action.bulk.byscroll;
|
||||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.test.ESTestCase;
|
|
@ -20,11 +20,9 @@
|
|||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.elasticsearch.action.ActionRequestValidationException;
|
||||
import org.elasticsearch.action.bulk.byscroll.AbstractBulkByScrollRequestTestCase;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.common.bytes.BytesArray;
|
||||
import org.elasticsearch.index.reindex.remote.RemoteInfo;
|
||||
import org.elasticsearch.search.slice.SliceBuilder;
|
||||
|
||||
import static java.util.Collections.emptyMap;
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.elasticsearch.action.bulk.byscroll.AbstractBulkByScrollRequestTestCase;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.support.IndicesOptions;
|
||||
|
|
@ -17,10 +17,8 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.action.bulk.byscroll;
|
||||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.elasticsearch.action.bulk.byscroll.BulkByScrollTask;
|
||||
import org.elasticsearch.action.bulk.byscroll.WorkingBulkByScrollTask;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
||||
import org.elasticsearch.tasks.TaskId;
|
|
@ -158,7 +158,7 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase
|
|||
final ShardRouting shardRouting = shard.routingEntry();
|
||||
assertThat(shardRouting + " local checkpoint mismatch", shardStats.getLocalCheckpoint(), equalTo(numDocs - 1L));
|
||||
/*
|
||||
* After the last indexing operation completes, the primary will advance its global checkpoint. Without an other indexing
|
||||
* After the last indexing operation completes, the primary will advance its global checkpoint. Without another indexing
|
||||
* operation, or a background sync, the primary will not have broadcast this global checkpoint to its replicas. However, a
|
||||
* shard could have recovered from the primary in which case its global checkpoint will be in-sync with the primary.
|
||||
* Therefore, we can only assert that the global checkpoint is number of docs minus one (matching the primary, in case of a
|
||||
|
@ -178,6 +178,7 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase
|
|||
// simulate a background global checkpoint sync at which point we expect the global checkpoint to advance on the replicas
|
||||
shards.syncGlobalCheckpoint();
|
||||
|
||||
final long noOpsPerformed = SequenceNumbersService.NO_OPS_PERFORMED;
|
||||
for (IndexShard shard : shards) {
|
||||
final SeqNoStats shardStats = shard.seqNoStats();
|
||||
final ShardRouting shardRouting = shard.routingEntry();
|
||||
|
@ -185,7 +186,7 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase
|
|||
assertThat(
|
||||
shardRouting + " global checkpoint mismatch",
|
||||
shardStats.getGlobalCheckpoint(),
|
||||
numDocs == 0 ? equalTo(unassignedSeqNo) : equalTo(numDocs - 1L));
|
||||
numDocs == 0 ? equalTo(noOpsPerformed) : equalTo(numDocs - 1L));
|
||||
assertThat(shardRouting + " max seq no mismatch", shardStats.getMaxSeqNo(), equalTo(numDocs - 1L));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -145,17 +145,20 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|||
final Map<String, Long> assigned = new HashMap<>();
|
||||
assigned.putAll(active);
|
||||
assigned.putAll(initializing);
|
||||
final String maxActiveID = active.entrySet().stream().max(Comparator.comparing(Map.Entry::getValue)).get().getKey();
|
||||
tracker.updateAllocationIdsFromMaster(
|
||||
active.entrySet().stream().filter(e -> !e.getKey().equals(maxActiveID)).map(Map.Entry::getKey).collect(Collectors.toSet()),
|
||||
active.keySet(),
|
||||
initializing.keySet());
|
||||
randomSubsetOf(initializing.keySet()).forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, tracker.getGlobalCheckpoint()));
|
||||
assigned.forEach(tracker::updateLocalCheckpoint);
|
||||
final String missingActiveID = randomFrom(active.keySet());
|
||||
assigned
|
||||
.entrySet()
|
||||
.stream()
|
||||
.filter(e -> !e.getKey().equals(missingActiveID))
|
||||
.forEach(e -> tracker.updateLocalCheckpoint(e.getKey(), e.getValue()));
|
||||
|
||||
// now mark all active shards
|
||||
tracker.updateAllocationIdsFromMaster(active.keySet(), initializing.keySet());
|
||||
assertThat(tracker.getGlobalCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
||||
|
||||
// update again
|
||||
// now update all knowledge of all shards
|
||||
assigned.forEach(tracker::updateLocalCheckpoint);
|
||||
assertThat(tracker.getGlobalCheckpoint(), not(equalTo(UNASSIGNED_SEQ_NO)));
|
||||
}
|
||||
|
|
|
@ -69,6 +69,7 @@ import org.elasticsearch.test.IndexSettingsModule;
|
|||
import org.elasticsearch.test.InternalSettingsPlugin;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.UncheckedIOException;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.StandardCopyOption;
|
||||
|
@ -81,6 +82,8 @@ import java.util.concurrent.BrokenBarrierException;
|
|||
import java.util.concurrent.CyclicBarrier;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.function.Predicate;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
import static java.util.Collections.emptyMap;
|
||||
import static java.util.Collections.emptySet;
|
||||
|
@ -168,33 +171,43 @@ public class IndexShardIT extends ESSingleNodeTestCase {
|
|||
IndicesService indicesService = getInstanceFromNode(IndicesService.class);
|
||||
IndexService test = indicesService.indexService(resolveIndex("test"));
|
||||
IndexShard shard = test.getShardOrNull(0);
|
||||
shard.checkIdle(Long.MIN_VALUE);
|
||||
Translog translog = ShardUtilsTests.getShardEngine(shard).getTranslog();
|
||||
Predicate<Translog> needsSync = (tlog) -> {
|
||||
// we can't use tlog.needsSync() here since it also takes the global checkpoint into account
|
||||
// we explicitly want to check here if our durability checks are taken into account so we only
|
||||
// check if we are synced upto the current write location
|
||||
Translog.Location lastWriteLocation = tlog.getLastWriteLocation();
|
||||
try {
|
||||
// the lastWriteLocaltion has a Integer.MAX_VALUE size so we have to create a new one
|
||||
return tlog.ensureSynced(new Translog.Location(lastWriteLocation.generation, lastWriteLocation.translogLocation, 0));
|
||||
} catch (IOException e) {
|
||||
throw new UncheckedIOException(e);
|
||||
}
|
||||
};
|
||||
setDurability(shard, Translog.Durability.REQUEST);
|
||||
assertBusy(() -> assertFalse(ShardUtilsTests.getShardEngine(shard).getTranslog().syncNeeded()));
|
||||
assertFalse(needsSync.test(translog));
|
||||
setDurability(shard, Translog.Durability.ASYNC);
|
||||
client().prepareIndex("test", "bar", "2").setSource("{}", XContentType.JSON).get();
|
||||
assertTrue(ShardUtilsTests.getShardEngine(shard).getTranslog().syncNeeded());
|
||||
assertTrue(needsSync.test(translog));
|
||||
setDurability(shard, Translog.Durability.REQUEST);
|
||||
client().prepareDelete("test", "bar", "1").get();
|
||||
shard.checkIdle(Long.MIN_VALUE);
|
||||
assertBusy(() -> assertFalse(ShardUtilsTests.getShardEngine(shard).getTranslog().syncNeeded()));
|
||||
assertFalse(needsSync.test(translog));
|
||||
|
||||
setDurability(shard, Translog.Durability.ASYNC);
|
||||
client().prepareDelete("test", "bar", "2").get();
|
||||
assertTrue(ShardUtilsTests.getShardEngine(shard).getTranslog().syncNeeded());
|
||||
assertTrue(translog.syncNeeded());
|
||||
setDurability(shard, Translog.Durability.REQUEST);
|
||||
assertNoFailures(client().prepareBulk()
|
||||
.add(client().prepareIndex("test", "bar", "3").setSource("{}", XContentType.JSON))
|
||||
.add(client().prepareDelete("test", "bar", "1")).get());
|
||||
shard.checkIdle(Long.MIN_VALUE);
|
||||
assertBusy(() -> assertFalse(ShardUtilsTests.getShardEngine(shard).getTranslog().syncNeeded()));
|
||||
assertFalse(needsSync.test(translog));
|
||||
|
||||
setDurability(shard, Translog.Durability.ASYNC);
|
||||
assertNoFailures(client().prepareBulk()
|
||||
.add(client().prepareIndex("test", "bar", "4").setSource("{}", XContentType.JSON))
|
||||
.add(client().prepareDelete("test", "bar", "3")).get());
|
||||
setDurability(shard, Translog.Durability.REQUEST);
|
||||
assertTrue(ShardUtilsTests.getShardEngine(shard).getTranslog().syncNeeded());
|
||||
assertTrue(needsSync.test(translog));
|
||||
}
|
||||
|
||||
private void setDurability(IndexShard shard, Translog.Durability durability) {
|
||||
|
|
|
@ -28,17 +28,14 @@ import org.elasticsearch.common.settings.Setting;
|
|||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.settings.SettingsException;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.node.InternalSettingsPreparer;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -155,22 +152,36 @@ public class InternalSettingsPreparerTests extends ESTestCase {
|
|||
|
||||
public void testMultipleSettingsFileNotAllowed() throws IOException {
|
||||
InputStream yaml = getClass().getResourceAsStream("/config/elasticsearch.yaml");
|
||||
InputStream properties = getClass().getResourceAsStream("/config/elasticsearch.properties");
|
||||
Path home = createTempDir();
|
||||
Path config = home.resolve("config");
|
||||
InputStream json = getClass().getResourceAsStream("/config/elasticsearch.json");
|
||||
Path config = homeDir.resolve("config");
|
||||
Files.createDirectory(config);
|
||||
Files.copy(yaml, config.resolve("elasticsearch.yaml"));
|
||||
Files.copy(properties, config.resolve("elasticsearch.properties"));
|
||||
Files.copy(json, config.resolve("elasticsearch.json"));
|
||||
|
||||
try {
|
||||
InternalSettingsPreparer.prepareEnvironment(Settings.builder()
|
||||
.put(baseEnvSettings)
|
||||
.build(), null);
|
||||
} catch (SettingsException e) {
|
||||
assertTrue(e.getMessage(), e.getMessage().contains("multiple settings files found with suffixes"));
|
||||
assertTrue(e.getMessage(), e.getMessage().contains(".yaml"));
|
||||
assertTrue(e.getMessage(), e.getMessage().contains(".properties"));
|
||||
}
|
||||
SettingsException e = expectThrows(SettingsException.class, () ->
|
||||
InternalSettingsPreparer.prepareEnvironment(Settings.builder().put(baseEnvSettings).build(), null)
|
||||
);
|
||||
assertTrue(e.getMessage(), e.getMessage().contains("multiple settings files found with suffixes"));
|
||||
assertTrue(e.getMessage(), e.getMessage().contains(".yaml"));
|
||||
assertTrue(e.getMessage(), e.getMessage().contains(".json"));
|
||||
}
|
||||
|
||||
public void testYmlExtension() throws IOException {
|
||||
InputStream yaml = getClass().getResourceAsStream("/config/elasticsearch.yaml");
|
||||
Path config = homeDir.resolve("config");
|
||||
Files.createDirectory(config);
|
||||
Files.copy(yaml, config.resolve("elasticsearch.yml"));
|
||||
Environment env = InternalSettingsPreparer.prepareEnvironment(Settings.builder().put(baseEnvSettings).build(), null);
|
||||
assertEquals(".yml", env.configExtension());
|
||||
}
|
||||
|
||||
public void testJsonExtension() throws IOException {
|
||||
InputStream yaml = getClass().getResourceAsStream("/config/elasticsearch.json");
|
||||
Path config = homeDir.resolve("config");
|
||||
Files.createDirectory(config);
|
||||
Files.copy(yaml, config.resolve("elasticsearch.json"));
|
||||
Environment env = InternalSettingsPreparer.prepareEnvironment(Settings.builder().put(baseEnvSettings).build(), null);
|
||||
assertEquals(".json", env.configExtension());
|
||||
}
|
||||
|
||||
public void testSecureSettings() {
|
||||
|
|
|
@ -0,0 +1,158 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package org.elasticsearch.search.aggregations;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.Writeable.Reader;
|
||||
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.common.xcontent.XContentParser.Token;
|
||||
import org.elasticsearch.common.xcontent.XContentType;
|
||||
import org.elasticsearch.search.aggregations.InternalOrder.CompoundOrder;
|
||||
import org.elasticsearch.test.AbstractSerializingTestCase;
|
||||
import org.elasticsearch.test.VersionUtils;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
public class InternalOrderTests extends AbstractSerializingTestCase<BucketOrder> {
|
||||
|
||||
@Override
|
||||
protected BucketOrder createTestInstance() {
|
||||
if (randomBoolean()) {
|
||||
return getRandomOrder();
|
||||
} else {
|
||||
List<BucketOrder> orders = new ArrayList<>();
|
||||
for (int i = 0; i < randomInt(3); i++) {
|
||||
orders.add(getRandomOrder());
|
||||
}
|
||||
return BucketOrder.compound(orders);
|
||||
}
|
||||
}
|
||||
|
||||
private BucketOrder getRandomOrder() {
|
||||
switch(randomInt(2)) {
|
||||
case 0: return BucketOrder.key(randomBoolean());
|
||||
case 1: return BucketOrder.count(randomBoolean());
|
||||
default: return BucketOrder.aggregation(randomAlphaOfLength(10), randomBoolean());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Reader<BucketOrder> instanceReader() {
|
||||
return InternalOrder.Streams::readOrder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected BucketOrder doParseInstance(XContentParser parser) throws IOException {
|
||||
Token token = parser.nextToken();
|
||||
if (token == Token.START_OBJECT) {
|
||||
return InternalOrder.Parser.parseOrderParam(parser, null);
|
||||
}
|
||||
if (token == Token.START_ARRAY) {
|
||||
List<BucketOrder> orders = new ArrayList<>();
|
||||
while (parser.nextToken() == Token.START_OBJECT) {
|
||||
orders.add(InternalOrder.Parser.parseOrderParam(parser, null));
|
||||
}
|
||||
return BucketOrder.compound(orders);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected BucketOrder assertSerialization(BucketOrder testInstance) throws IOException {
|
||||
// identical behavior to AbstractWireSerializingTestCase, except assertNotSame is only called for
|
||||
// compound and aggregation order because _key and _count orders are static instances.
|
||||
BucketOrder deserializedInstance = copyInstance(testInstance);
|
||||
assertEquals(testInstance, deserializedInstance);
|
||||
assertEquals(testInstance.hashCode(), deserializedInstance.hashCode());
|
||||
if(testInstance instanceof CompoundOrder || testInstance instanceof InternalOrder.Aggregation) {
|
||||
assertNotSame(testInstance, deserializedInstance);
|
||||
}
|
||||
return deserializedInstance;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void assertParsedInstance(XContentType xContentType, BytesReference instanceAsBytes, BucketOrder expectedInstance)
|
||||
throws IOException {
|
||||
// identical behavior to AbstractSerializingTestCase, except assertNotSame is only called for
|
||||
// compound and aggregation order because _key and _count orders are static instances.
|
||||
XContentParser parser = createParser(XContentFactory.xContent(xContentType), instanceAsBytes);
|
||||
BucketOrder newInstance = parseInstance(parser);
|
||||
assertEquals(expectedInstance, newInstance);
|
||||
assertEquals(expectedInstance.hashCode(), newInstance.hashCode());
|
||||
if(expectedInstance instanceof CompoundOrder || expectedInstance instanceof InternalOrder.Aggregation) {
|
||||
assertNotSame(newInstance, expectedInstance);
|
||||
}
|
||||
}
|
||||
|
||||
public void testHistogramOrderBwc() throws IOException {
|
||||
for (int runs = 0; runs < NUMBER_OF_TEST_RUNS; runs++) {
|
||||
BucketOrder order = createTestInstance();
|
||||
Version bwcVersion = VersionUtils.randomVersionBetween(random(), VersionUtils.getFirstVersion(),
|
||||
VersionUtils.getPreviousVersion(Version.V_6_0_0_alpha2_UNRELEASED));
|
||||
boolean bwcOrderFlag = randomBoolean();
|
||||
try (BytesStreamOutput out = new BytesStreamOutput()) {
|
||||
out.setVersion(bwcVersion);
|
||||
InternalOrder.Streams.writeHistogramOrder(order, out, bwcOrderFlag);
|
||||
try (StreamInput in = out.bytes().streamInput()) {
|
||||
in.setVersion(bwcVersion);
|
||||
BucketOrder actual = InternalOrder.Streams.readHistogramOrder(in, bwcOrderFlag);
|
||||
BucketOrder expected = order;
|
||||
if (order instanceof CompoundOrder) {
|
||||
expected = ((CompoundOrder) order).orderElements.get(0);
|
||||
}
|
||||
assertEquals(expected, actual);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testAggregationOrderEqualsAndHashCode() {
|
||||
String path = randomAlphaOfLength(10);
|
||||
boolean asc = randomBoolean();
|
||||
BucketOrder o1 = BucketOrder.aggregation(path, asc);
|
||||
BucketOrder o2 = BucketOrder.aggregation(path + "test", asc);
|
||||
BucketOrder o3 = BucketOrder.aggregation(path, !asc);
|
||||
BucketOrder o4 = BucketOrder.aggregation(path, asc);
|
||||
assertNotEquals(o1, o2);
|
||||
assertNotEquals(o1.hashCode(), o2.hashCode());
|
||||
assertNotEquals(o1, o3);
|
||||
assertNotEquals(o1.hashCode(), o3.hashCode());
|
||||
assertEquals(o1, o4);
|
||||
assertEquals(o1.hashCode(), o4.hashCode());
|
||||
|
||||
o1 = InternalOrder.compound(o1);
|
||||
o2 = InternalOrder.compound(o2);
|
||||
o3 = InternalOrder.compound(o3);
|
||||
assertNotEquals(o1, o2);
|
||||
assertNotEquals(o1.hashCode(), o2.hashCode());
|
||||
assertNotEquals(o1, o2);
|
||||
assertNotEquals(o1.hashCode(), o2.hashCode());
|
||||
assertNotEquals(o1, o3);
|
||||
assertNotEquals(o1.hashCode(), o3.hashCode());
|
||||
assertNotEquals(o1, o4);
|
||||
assertNotEquals(o1.hashCode(), o4.hashCode());
|
||||
}
|
||||
|
||||
}
|
|
@ -18,7 +18,9 @@
|
|||
*/
|
||||
package org.elasticsearch.search.aggregations.bucket;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchPhaseExecutionException;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.common.joda.DateMathParser;
|
||||
import org.elasticsearch.common.joda.Joda;
|
||||
|
@ -30,13 +32,16 @@ import org.elasticsearch.index.query.QueryBuilders;
|
|||
import org.elasticsearch.plugins.Plugin;
|
||||
import org.elasticsearch.script.Script;
|
||||
import org.elasticsearch.script.ScriptType;
|
||||
import org.elasticsearch.search.aggregations.AggregationExecutionException;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.bucket.DateScriptMocks.DateScriptsMockPlugin;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.ExtendedBounds;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket;
|
||||
import org.elasticsearch.search.aggregations.metrics.avg.Avg;
|
||||
import org.elasticsearch.search.aggregations.metrics.sum.Sum;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
import org.hamcrest.Matchers;
|
||||
import org.joda.time.DateTime;
|
||||
|
@ -57,6 +62,7 @@ import java.util.concurrent.TimeUnit;
|
|||
|
||||
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
|
||||
import static org.elasticsearch.search.aggregations.AggregationBuilders.avg;
|
||||
import static org.elasticsearch.search.aggregations.AggregationBuilders.dateHistogram;
|
||||
import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram;
|
||||
import static org.elasticsearch.search.aggregations.AggregationBuilders.max;
|
||||
|
@ -73,6 +79,8 @@ import static org.hamcrest.core.IsNull.notNullValue;
|
|||
@ESIntegTestCase.SuiteScopeTestCase
|
||||
public class DateHistogramIT extends ESIntegTestCase {
|
||||
|
||||
static Map<DateTime, Map<String, Object>> expectedMultiSortBuckets;
|
||||
|
||||
private DateTime date(int month, int day) {
|
||||
return new DateTime(2012, month, day, 0, 0, DateTimeZone.UTC);
|
||||
}
|
||||
|
@ -98,6 +106,7 @@ public class DateHistogramIT extends ESIntegTestCase {
|
|||
return client().prepareIndex("idx", "type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field("value", value)
|
||||
.field("constant", 1)
|
||||
.field("date", date(month, day))
|
||||
.startArray("dates").value(date(month, day)).value(date(month + 1, day + 1)).endArray()
|
||||
.endObject());
|
||||
|
@ -115,6 +124,9 @@ public class DateHistogramIT extends ESIntegTestCase {
|
|||
.field("value", i * 2)
|
||||
.endObject()));
|
||||
}
|
||||
|
||||
getMultiSortDocs(builders);
|
||||
|
||||
builders.addAll(Arrays.asList(
|
||||
indexDoc(1, 2, 1), // date: Jan 2, dates: Jan 2, Feb 3
|
||||
indexDoc(2, 2, 2), // date: Feb 2, dates: Feb 2, Mar 3
|
||||
|
@ -126,6 +138,50 @@ public class DateHistogramIT extends ESIntegTestCase {
|
|||
ensureSearchable();
|
||||
}
|
||||
|
||||
private void addExpectedBucket(DateTime key, long docCount, double avg, double sum) {
|
||||
Map<String, Object> bucketProps = new HashMap<>();
|
||||
bucketProps.put("_count", docCount);
|
||||
bucketProps.put("avg_l", avg);
|
||||
bucketProps.put("sum_d", sum);
|
||||
expectedMultiSortBuckets.put(key, bucketProps);
|
||||
}
|
||||
|
||||
private void getMultiSortDocs(List<IndexRequestBuilder> builders) throws IOException {
|
||||
expectedMultiSortBuckets = new HashMap<>();
|
||||
addExpectedBucket(date(1, 1), 3, 1, 6);
|
||||
addExpectedBucket(date(1, 2), 3, 2, 6);
|
||||
addExpectedBucket(date(1, 3), 2, 3, 3);
|
||||
addExpectedBucket(date(1, 4), 2, 3, 4);
|
||||
addExpectedBucket(date(1, 5), 2, 5, 3);
|
||||
addExpectedBucket(date(1, 6), 1, 5, 1);
|
||||
addExpectedBucket(date(1, 7), 1, 5, 1);
|
||||
|
||||
assertAcked(client().admin().indices().prepareCreate("sort_idx")
|
||||
.addMapping("type", "date", "type=date").get());
|
||||
for (int i = 1; i <= 3; i++) {
|
||||
builders.add(client().prepareIndex("sort_idx", "type").setSource(
|
||||
jsonBuilder().startObject().field("date", date(1, 1)).field("l", 1).field("d", i).endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "type").setSource(
|
||||
jsonBuilder().startObject().field("date", date(1, 2)).field("l", 2).field("d", i).endObject()));
|
||||
}
|
||||
builders.add(client().prepareIndex("sort_idx", "type").setSource(
|
||||
jsonBuilder().startObject().field("date", date(1, 3)).field("l", 3).field("d", 1).endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "type").setSource(
|
||||
jsonBuilder().startObject().field("date", date(1, 3).plusHours(1)).field("l", 3).field("d", 2).endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "type").setSource(
|
||||
jsonBuilder().startObject().field("date", date(1, 4)).field("l", 3).field("d", 1).endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "type").setSource(
|
||||
jsonBuilder().startObject().field("date", date(1, 4).plusHours(2)).field("l", 3).field("d", 3).endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "type").setSource(
|
||||
jsonBuilder().startObject().field("date", date(1, 5)).field("l", 5).field("d", 1).endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "type").setSource(
|
||||
jsonBuilder().startObject().field("date", date(1, 5).plusHours(12)).field("l", 5).field("d", 2).endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "type").setSource(
|
||||
jsonBuilder().startObject().field("date", date(1, 6)).field("l", 5).field("d", 1).endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "type").setSource(
|
||||
jsonBuilder().startObject().field("date", date(1, 7)).field("l", 5).field("d", 1).endObject()));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Collection<Class<? extends Plugin>> nodePlugins() {
|
||||
return Arrays.asList(
|
||||
|
@ -281,7 +337,7 @@ public class DateHistogramIT extends ESIntegTestCase {
|
|||
.addAggregation(dateHistogram("histo")
|
||||
.field("date")
|
||||
.dateHistogramInterval(DateHistogramInterval.MONTH)
|
||||
.order(Histogram.Order.KEY_ASC))
|
||||
.order(BucketOrder.key(true)))
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -304,7 +360,7 @@ public class DateHistogramIT extends ESIntegTestCase {
|
|||
.addAggregation(dateHistogram("histo")
|
||||
.field("date")
|
||||
.dateHistogramInterval(DateHistogramInterval.MONTH)
|
||||
.order(Histogram.Order.KEY_DESC))
|
||||
.order(BucketOrder.key(false)))
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -326,7 +382,7 @@ public class DateHistogramIT extends ESIntegTestCase {
|
|||
.addAggregation(dateHistogram("histo")
|
||||
.field("date")
|
||||
.dateHistogramInterval(DateHistogramInterval.MONTH)
|
||||
.order(Histogram.Order.COUNT_ASC))
|
||||
.order(BucketOrder.count(true)))
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -348,7 +404,7 @@ public class DateHistogramIT extends ESIntegTestCase {
|
|||
.addAggregation(dateHistogram("histo")
|
||||
.field("date")
|
||||
.dateHistogramInterval(DateHistogramInterval.MONTH)
|
||||
.order(Histogram.Order.COUNT_DESC))
|
||||
.order(BucketOrder.count(false)))
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -428,7 +484,7 @@ public class DateHistogramIT extends ESIntegTestCase {
|
|||
.addAggregation(dateHistogram("histo")
|
||||
.field("date")
|
||||
.dateHistogramInterval(DateHistogramInterval.MONTH)
|
||||
.order(Histogram.Order.aggregation("sum", true))
|
||||
.order(BucketOrder.aggregation("sum", true))
|
||||
.subAggregation(max("sum").field("value")))
|
||||
.execute().actionGet();
|
||||
|
||||
|
@ -451,7 +507,7 @@ public class DateHistogramIT extends ESIntegTestCase {
|
|||
.addAggregation(dateHistogram("histo")
|
||||
.field("date")
|
||||
.dateHistogramInterval(DateHistogramInterval.MONTH)
|
||||
.order(Histogram.Order.aggregation("sum", false))
|
||||
.order(BucketOrder.aggregation("sum", false))
|
||||
.subAggregation(max("sum").field("value")))
|
||||
.execute().actionGet();
|
||||
|
||||
|
@ -474,7 +530,7 @@ public class DateHistogramIT extends ESIntegTestCase {
|
|||
.addAggregation(dateHistogram("histo")
|
||||
.field("date")
|
||||
.dateHistogramInterval(DateHistogramInterval.MONTH)
|
||||
.order(Histogram.Order.aggregation("stats", "sum", false))
|
||||
.order(BucketOrder.aggregation("stats", "sum", false))
|
||||
.subAggregation(stats("stats").field("value")))
|
||||
.execute().actionGet();
|
||||
|
||||
|
@ -492,6 +548,60 @@ public class DateHistogramIT extends ESIntegTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedByTieBreaker() throws Exception {
|
||||
SearchResponse response = client().prepareSearch("idx")
|
||||
.addAggregation(dateHistogram("histo")
|
||||
.field("date")
|
||||
.dateHistogramInterval(DateHistogramInterval.MONTH)
|
||||
.order(BucketOrder.aggregation("max_constant", randomBoolean()))
|
||||
.subAggregation(max("max_constant").field("constant")))
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
||||
Histogram histo = response.getAggregations().get("histo");
|
||||
assertThat(histo, notNullValue());
|
||||
assertThat(histo.getName(), equalTo("histo"));
|
||||
assertThat(histo.getBuckets().size(), equalTo(3));
|
||||
|
||||
int i = 1;
|
||||
for (Histogram.Bucket bucket : histo.getBuckets()) {
|
||||
assertThat(bucket.getKey(), equalTo(date(i, 1)));
|
||||
i++;
|
||||
}
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedByIllegalAgg() throws Exception {
|
||||
boolean asc = true;
|
||||
try {
|
||||
client()
|
||||
.prepareSearch("idx")
|
||||
.addAggregation(
|
||||
dateHistogram("histo").field("date")
|
||||
.dateHistogramInterval(DateHistogramInterval.MONTH)
|
||||
.order(BucketOrder.aggregation("inner_histo>avg", asc))
|
||||
.subAggregation(dateHistogram("inner_histo")
|
||||
.dateHistogramInterval(DateHistogramInterval.MONTH)
|
||||
.field("dates")
|
||||
.subAggregation(avg("avg").field("value"))))
|
||||
.execute().actionGet();
|
||||
fail("Expected an exception");
|
||||
} catch (SearchPhaseExecutionException e) {
|
||||
ElasticsearchException[] rootCauses = e.guessRootCauses();
|
||||
if (rootCauses.length == 1) {
|
||||
ElasticsearchException rootCause = rootCauses[0];
|
||||
if (rootCause instanceof AggregationExecutionException) {
|
||||
AggregationExecutionException aggException = (AggregationExecutionException) rootCause;
|
||||
assertThat(aggException.getMessage(), Matchers.startsWith("Invalid aggregation order path"));
|
||||
} else {
|
||||
throw e;
|
||||
}
|
||||
} else {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldWithValueScript() throws Exception {
|
||||
Map<String, Object> params = new HashMap<>();
|
||||
params.put("fieldname", "date");
|
||||
|
@ -583,12 +693,12 @@ public class DateHistogramIT extends ESIntegTestCase {
|
|||
assertThat(bucket.getDocCount(), equalTo(3L));
|
||||
}
|
||||
|
||||
public void testMultiValuedFieldOrderedByKeyDesc() throws Exception {
|
||||
public void testMultiValuedFieldOrderedByCountDesc() throws Exception {
|
||||
SearchResponse response = client().prepareSearch("idx")
|
||||
.addAggregation(dateHistogram("histo")
|
||||
.field("dates")
|
||||
.dateHistogramInterval(DateHistogramInterval.MONTH)
|
||||
.order(Histogram.Order.COUNT_DESC))
|
||||
.order(BucketOrder.count(false)))
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -598,23 +708,26 @@ public class DateHistogramIT extends ESIntegTestCase {
|
|||
assertThat(histo.getName(), equalTo("histo"));
|
||||
assertThat(histo.getBuckets().size(), equalTo(4));
|
||||
|
||||
// TODO: use diamond once JI-9019884 is fixed
|
||||
List<Histogram.Bucket> buckets = new ArrayList<>(histo.getBuckets());
|
||||
|
||||
Histogram.Bucket bucket = buckets.get(0);
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat(bucket.getKey(), equalTo(date(3, 1)));
|
||||
assertThat(bucket.getDocCount(), equalTo(5L));
|
||||
|
||||
bucket = buckets.get(1);
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat(bucket.getKey(), equalTo(date(2, 1)));
|
||||
assertThat(bucket.getDocCount(), equalTo(3L));
|
||||
|
||||
bucket = buckets.get(2);
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat(bucket.getKey(), equalTo(date(4, 1)));
|
||||
assertThat(bucket.getDocCount(), equalTo(3L));
|
||||
|
||||
bucket = buckets.get(3);
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat(bucket.getKey(), equalTo(date(1, 1)));
|
||||
assertThat(bucket.getDocCount(), equalTo(1L));
|
||||
}
|
||||
|
||||
|
@ -1236,4 +1349,75 @@ public class DateHistogramIT extends ESIntegTestCase {
|
|||
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
|
||||
.getMissCount(), equalTo(1L));
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscAndKeyDesc() throws Exception {
|
||||
int[] expectedDays = new int[] { 1, 2, 4, 3, 7, 6, 5 };
|
||||
assertMultiSortResponse(expectedDays, BucketOrder.aggregation("avg_l", true), BucketOrder.key(false));
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscAndKeyAsc() throws Exception {
|
||||
int[] expectedDays = new int[] { 1, 2, 3, 4, 5, 6, 7 };
|
||||
assertMultiSortResponse(expectedDays, BucketOrder.aggregation("avg_l", true), BucketOrder.key(true));
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedBySingleValueSubAggregationDescAndKeyAsc() throws Exception {
|
||||
int[] expectedDays = new int[] { 5, 6, 7, 3, 4, 2, 1 };
|
||||
assertMultiSortResponse(expectedDays, BucketOrder.aggregation("avg_l", false), BucketOrder.key(true));
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedByCountAscAndSingleValueSubAggregationAsc() throws Exception {
|
||||
int[] expectedDays = new int[] { 6, 7, 3, 4, 5, 1, 2 };
|
||||
assertMultiSortResponse(expectedDays, BucketOrder.count(true), BucketOrder.aggregation("avg_l", true));
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscSingleValueSubAggregationAsc() throws Exception {
|
||||
int[] expectedDays = new int[] { 6, 7, 3, 5, 4, 1, 2 };
|
||||
assertMultiSortResponse(expectedDays, BucketOrder.aggregation("sum_d", true), BucketOrder.aggregation("avg_l", true));
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedByThreeCriteria() throws Exception {
|
||||
int[] expectedDays = new int[] { 2, 1, 4, 5, 3, 6, 7 };
|
||||
assertMultiSortResponse(expectedDays, BucketOrder.count(false), BucketOrder.aggregation("sum_d", false),
|
||||
BucketOrder.aggregation("avg_l", false));
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscAsCompound() throws Exception {
|
||||
int[] expectedDays = new int[] { 1, 2, 3, 4, 5, 6, 7 };
|
||||
assertMultiSortResponse(expectedDays, BucketOrder.aggregation("avg_l", true));
|
||||
}
|
||||
|
||||
private void assertMultiSortResponse(int[] expectedDays, BucketOrder... order) {
|
||||
DateTime[] expectedKeys = Arrays.stream(expectedDays).mapToObj(d -> date(1, d)).toArray(DateTime[]::new);
|
||||
SearchResponse response = client()
|
||||
.prepareSearch("sort_idx")
|
||||
.setTypes("type")
|
||||
.addAggregation(
|
||||
dateHistogram("histo").field("date").dateHistogramInterval(DateHistogramInterval.DAY).order(BucketOrder.compound(order))
|
||||
.subAggregation(avg("avg_l").field("l")).subAggregation(sum("sum_d").field("d"))).execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
||||
Histogram histogram = response.getAggregations().get("histo");
|
||||
assertThat(histogram, notNullValue());
|
||||
assertThat(histogram.getName(), equalTo("histo"));
|
||||
assertThat(histogram.getBuckets().size(), equalTo(expectedKeys.length));
|
||||
|
||||
int i = 0;
|
||||
for (Histogram.Bucket bucket : histogram.getBuckets()) {
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat(key(bucket), equalTo(expectedKeys[i]));
|
||||
assertThat(bucket.getDocCount(), equalTo(expectedMultiSortBuckets.get(expectedKeys[i]).get("_count")));
|
||||
Avg avg = bucket.getAggregations().get("avg_l");
|
||||
assertThat(avg, notNullValue());
|
||||
assertThat(avg.getValue(), equalTo(expectedMultiSortBuckets.get(expectedKeys[i]).get("avg_l")));
|
||||
Sum sum = bucket.getAggregations().get("sum_d");
|
||||
assertThat(sum, notNullValue());
|
||||
assertThat(sum.getValue(), equalTo(expectedMultiSortBuckets.get(expectedKeys[i]).get("sum_d")));
|
||||
i++;
|
||||
}
|
||||
}
|
||||
|
||||
private DateTime key(Histogram.Bucket bucket) {
|
||||
return (DateTime) bucket.getKey();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,10 +20,13 @@
|
|||
package org.elasticsearch.search.aggregations.bucket;
|
||||
|
||||
import org.elasticsearch.search.aggregations.BaseAggregationTestCase;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Order;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.ExtendedBoundsTests;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
public class DateHistogramTests extends BaseAggregationTestCase<DateHistogramAggregationBuilder> {
|
||||
|
||||
|
@ -80,29 +83,41 @@ public class DateHistogramTests extends BaseAggregationTestCase<DateHistogramAgg
|
|||
factory.offset(randomIntBetween(0, 100000));
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
int branch = randomInt(5);
|
||||
switch (branch) {
|
||||
case 0:
|
||||
factory.order(Order.COUNT_ASC);
|
||||
break;
|
||||
case 1:
|
||||
factory.order(Order.COUNT_DESC);
|
||||
break;
|
||||
case 2:
|
||||
factory.order(Order.KEY_ASC);
|
||||
break;
|
||||
case 3:
|
||||
factory.order(Order.KEY_DESC);
|
||||
break;
|
||||
case 4:
|
||||
factory.order(Order.aggregation("foo", true));
|
||||
break;
|
||||
case 5:
|
||||
factory.order(Order.aggregation("foo", false));
|
||||
break;
|
||||
List<BucketOrder> order = randomOrder();
|
||||
if(order.size() == 1 && randomBoolean()) {
|
||||
factory.order(order.get(0));
|
||||
} else {
|
||||
factory.order(order);
|
||||
}
|
||||
}
|
||||
return factory;
|
||||
}
|
||||
|
||||
private List<BucketOrder> randomOrder() {
|
||||
List<BucketOrder> orders = new ArrayList<>();
|
||||
switch (randomInt(4)) {
|
||||
case 0:
|
||||
orders.add(BucketOrder.key(randomBoolean()));
|
||||
break;
|
||||
case 1:
|
||||
orders.add(BucketOrder.count(randomBoolean()));
|
||||
break;
|
||||
case 2:
|
||||
orders.add(BucketOrder.aggregation(randomAlphaOfLengthBetween(3, 20), randomBoolean()));
|
||||
break;
|
||||
case 3:
|
||||
orders.add(BucketOrder.aggregation(randomAlphaOfLengthBetween(3, 20), randomAlphaOfLengthBetween(3, 20), randomBoolean()));
|
||||
break;
|
||||
case 4:
|
||||
int numOrders = randomIntBetween(1, 3);
|
||||
for (int i = 0; i < numOrders; i++) {
|
||||
orders.addAll(randomOrder());
|
||||
}
|
||||
break;
|
||||
default:
|
||||
fail();
|
||||
}
|
||||
return orders;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.elasticsearch.search.aggregations.bucket.terms.Terms;
|
|||
import org.elasticsearch.search.aggregations.bucket.terms.Terms.Bucket;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.max.Max;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
|
||||
import java.util.Collection;
|
||||
|
@ -103,7 +104,7 @@ public class DiversifiedSamplerIT extends ESIntegTestCase {
|
|||
SearchResponse response = client().prepareSearch("test").setTypes("book").setSearchType(SearchType.QUERY_THEN_FETCH)
|
||||
.addAggregation(terms("genres")
|
||||
.field("genre")
|
||||
.order(Terms.Order.aggregation("sample>max_price.value", asc))
|
||||
.order(BucketOrder.aggregation("sample>max_price.value", asc))
|
||||
.subAggregation(sampler("sample").shardSize(100)
|
||||
.subAggregation(max("max_price").field("price")))
|
||||
).execute().actionGet();
|
||||
|
|
|
@ -41,6 +41,7 @@ import org.elasticsearch.search.aggregations.metrics.max.Max;
|
|||
import org.elasticsearch.search.aggregations.metrics.stats.Stats;
|
||||
import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStats;
|
||||
import org.elasticsearch.search.aggregations.metrics.sum.Sum;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
import org.hamcrest.Matchers;
|
||||
|
||||
|
@ -134,6 +135,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
|
|||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, (double) i)
|
||||
.field("num_tag", i < NUM_DOCS/2 + 1 ? 1 : 0) // used to test order by single-bucket sub agg
|
||||
.field("constant", 1)
|
||||
.startArray(MULTI_VALUED_FIELD_NAME).value((double) i).value(i + 1d).endArray()
|
||||
.endObject()));
|
||||
|
||||
|
@ -315,7 +317,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
|
|||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.size(20)
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.term(true))) // we need to sort by terms cause we're checking the first 20 values
|
||||
.order(BucketOrder.key(true))) // we need to sort by terms cause we're checking the first 20 values
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -363,15 +365,15 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
|
|||
assertThat(bucket.getDocCount(), equalTo(1L));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public void testSingleValueFieldWithPartitionedFiltering() throws Exception {
|
||||
runTestFieldWithPartitionedFiltering(SINGLE_VALUED_FIELD_NAME);
|
||||
}
|
||||
|
||||
|
||||
public void testMultiValueFieldWithPartitionedFiltering() throws Exception {
|
||||
runTestFieldWithPartitionedFiltering(MULTI_VALUED_FIELD_NAME);
|
||||
}
|
||||
|
||||
|
||||
private void runTestFieldWithPartitionedFiltering(String field) throws Exception {
|
||||
// Find total number of unique terms
|
||||
SearchResponse allResponse = client().prepareSearch("idx")
|
||||
|
@ -399,14 +401,14 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
|
|||
}
|
||||
}
|
||||
assertEquals(expectedCardinality, foundTerms.size());
|
||||
}
|
||||
}
|
||||
|
||||
public void testSingleValueFieldOrderedByTermAsc() throws Exception {
|
||||
SearchResponse response = client().prepareSearch("idx")
|
||||
.addAggregation(terms("terms")
|
||||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.term(true)))
|
||||
.order(BucketOrder.key(true)))
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -432,7 +434,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
|
|||
.addAggregation(terms("terms")
|
||||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.term(false)))
|
||||
.order(BucketOrder.key(false)))
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -453,6 +455,33 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public void testSingleValueFieldOrderedByTieBreaker() throws Exception {
|
||||
SearchResponse response = client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(BucketOrder.aggregation("max_constant", randomBoolean()))
|
||||
.subAggregation(max("max_constant").field("constant")))
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
||||
|
||||
Terms terms = response.getAggregations().get("terms");
|
||||
assertThat(terms, notNullValue());
|
||||
assertThat(terms.getName(), equalTo("terms"));
|
||||
assertThat(terms.getBuckets().size(), equalTo(5));
|
||||
|
||||
int i = 0;
|
||||
for (Terms.Bucket bucket : terms.getBuckets()) {
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat(key(bucket), equalTo("" + (double)i));
|
||||
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
|
||||
assertThat(bucket.getDocCount(), equalTo(1L));
|
||||
i++;
|
||||
}
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldWithSubAggregation() throws Exception {
|
||||
SearchResponse response = client().prepareSearch("idx")
|
||||
.addAggregation(terms("terms")
|
||||
|
@ -759,7 +788,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
|
|||
.prepareSearch("idx")
|
||||
.addAggregation(
|
||||
terms("terms").field(SINGLE_VALUED_FIELD_NAME).collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.aggregation("avg_i", asc)).subAggregation(avg("avg_i").field(SINGLE_VALUED_FIELD_NAME)))
|
||||
.order(BucketOrder.aggregation("avg_i", asc)).subAggregation(avg("avg_i").field(SINGLE_VALUED_FIELD_NAME)))
|
||||
.execute().actionGet();
|
||||
|
||||
|
||||
|
@ -789,7 +818,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
|
|||
terms("terms")
|
||||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.aggregation("avg_i", asc))
|
||||
.order(BucketOrder.aggregation("avg_i", asc))
|
||||
.subAggregation(avg("avg_i").field(SINGLE_VALUED_FIELD_NAME))
|
||||
.subAggregation(
|
||||
terms("subTerms").field(MULTI_VALUED_FIELD_NAME).collectMode(
|
||||
|
@ -831,7 +860,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
|
|||
.prepareSearch("idx")
|
||||
.addAggregation(
|
||||
terms("num_tags").field("num_tag").collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.aggregation("filter", asc))
|
||||
.order(BucketOrder.aggregation("filter", asc))
|
||||
.subAggregation(filter("filter", QueryBuilders.matchAllQuery()))).execute().actionGet();
|
||||
|
||||
|
||||
|
@ -869,7 +898,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
|
|||
terms("tags")
|
||||
.field("num_tag")
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.aggregation("filter1>filter2>max", asc))
|
||||
.order(BucketOrder.aggregation("filter1>filter2>max", asc))
|
||||
.subAggregation(
|
||||
filter("filter1", QueryBuilders.matchAllQuery()).subAggregation(
|
||||
filter("filter2", QueryBuilders.matchAllQuery()).subAggregation(
|
||||
|
@ -923,7 +952,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
|
|||
client().prepareSearch(index)
|
||||
.addAggregation(
|
||||
terms("terms").field(SINGLE_VALUED_FIELD_NAME).collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.aggregation("avg_i", true))).execute().actionGet();
|
||||
.order(BucketOrder.aggregation("avg_i", true))).execute().actionGet();
|
||||
|
||||
fail("Expected search to fail when trying to sort terms aggregation by sug-aggregation that doesn't exist");
|
||||
|
||||
|
@ -941,7 +970,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
|
|||
terms("terms")
|
||||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.aggregation("num_tags", true))
|
||||
.order(BucketOrder.aggregation("num_tags", true))
|
||||
.subAggregation(
|
||||
terms("num_tags").field("num_tags").collectMode(randomFrom(SubAggCollectionMode.values()))))
|
||||
.execute().actionGet();
|
||||
|
@ -960,7 +989,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
|
|||
client().prepareSearch(index)
|
||||
.addAggregation(
|
||||
terms("terms").field(SINGLE_VALUED_FIELD_NAME + "2").collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.aggregation("stats.foo", true))
|
||||
.order(BucketOrder.aggregation("stats.foo", true))
|
||||
.subAggregation(stats("stats").field(SINGLE_VALUED_FIELD_NAME))).execute().actionGet();
|
||||
|
||||
fail("Expected search to fail when trying to sort terms aggregation by multi-valued sug-aggregation " +
|
||||
|
@ -978,7 +1007,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
|
|||
client().prepareSearch(index)
|
||||
.addAggregation(
|
||||
terms("terms").field(SINGLE_VALUED_FIELD_NAME).collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.aggregation("stats", true))
|
||||
.order(BucketOrder.aggregation("stats", true))
|
||||
.subAggregation(stats("stats").field(SINGLE_VALUED_FIELD_NAME))).execute().actionGet();
|
||||
|
||||
fail("Expected search to fail when trying to sort terms aggregation by multi-valued sug-aggregation " +
|
||||
|
@ -996,7 +1025,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
|
|||
.prepareSearch("idx")
|
||||
.addAggregation(
|
||||
terms("terms").field(SINGLE_VALUED_FIELD_NAME).collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.aggregation("avg_i", asc)).subAggregation(avg("avg_i").field(SINGLE_VALUED_FIELD_NAME)))
|
||||
.order(BucketOrder.aggregation("avg_i", asc)).subAggregation(avg("avg_i").field(SINGLE_VALUED_FIELD_NAME)))
|
||||
.execute().actionGet();
|
||||
|
||||
|
||||
|
@ -1026,7 +1055,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
|
|||
.prepareSearch("idx")
|
||||
.addAggregation(
|
||||
terms("terms").field(SINGLE_VALUED_FIELD_NAME).collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.aggregation("stats.avg", asc))
|
||||
.order(BucketOrder.aggregation("stats.avg", asc))
|
||||
.subAggregation(stats("stats").field(SINGLE_VALUED_FIELD_NAME))).execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -1054,7 +1083,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
|
|||
.prepareSearch("idx")
|
||||
.addAggregation(
|
||||
terms("terms").field(SINGLE_VALUED_FIELD_NAME).collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.aggregation("stats.avg", asc))
|
||||
.order(BucketOrder.aggregation("stats.avg", asc))
|
||||
.subAggregation(stats("stats").field(SINGLE_VALUED_FIELD_NAME))).execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -1082,7 +1111,7 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
|
|||
.prepareSearch("idx")
|
||||
.addAggregation(
|
||||
terms("terms").field(SINGLE_VALUED_FIELD_NAME).collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.aggregation("stats.variance", asc))
|
||||
.order(BucketOrder.aggregation("stats.variance", asc))
|
||||
.subAggregation(extendedStats("stats").field(SINGLE_VALUED_FIELD_NAME))).execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -1139,48 +1168,48 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
|
|||
|
||||
public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscAndTermsDesc() throws Exception {
|
||||
double[] expectedKeys = new double[] { 1, 2, 4, 3, 7, 6, 5 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("avg_l", true), Terms.Order.term(false));
|
||||
assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", true), BucketOrder.key(false));
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscAndTermsAsc() throws Exception {
|
||||
double[] expectedKeys = new double[] { 1, 2, 3, 4, 5, 6, 7 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("avg_l", true), Terms.Order.term(true));
|
||||
assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", true), BucketOrder.key(true));
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedBySingleValueSubAggregationDescAndTermsAsc() throws Exception {
|
||||
double[] expectedKeys = new double[] { 5, 6, 7, 3, 4, 2, 1 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("avg_l", false), Terms.Order.term(true));
|
||||
assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", false), BucketOrder.key(true));
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedByCountAscAndSingleValueSubAggregationAsc() throws Exception {
|
||||
double[] expectedKeys = new double[] { 6, 7, 3, 4, 5, 1, 2 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.count(true), Terms.Order.aggregation("avg_l", true));
|
||||
assertMultiSortResponse(expectedKeys, BucketOrder.count(true), BucketOrder.aggregation("avg_l", true));
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscSingleValueSubAggregationAsc() throws Exception {
|
||||
double[] expectedKeys = new double[] { 6, 7, 3, 5, 4, 1, 2 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("sum_d", true), Terms.Order.aggregation("avg_l", true));
|
||||
assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("sum_d", true), BucketOrder.aggregation("avg_l", true));
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedByThreeCriteria() throws Exception {
|
||||
double[] expectedKeys = new double[] { 2, 1, 4, 5, 3, 6, 7 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.count(false),
|
||||
Terms.Order.aggregation("sum_d", false),
|
||||
Terms.Order.aggregation("avg_l", false));
|
||||
assertMultiSortResponse(expectedKeys, BucketOrder.count(false),
|
||||
BucketOrder.aggregation("sum_d", false),
|
||||
BucketOrder.aggregation("avg_l", false));
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscAsCompound() throws Exception {
|
||||
double[] expectedKeys = new double[] { 1, 2, 3, 4, 5, 6, 7 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("avg_l", true));
|
||||
assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", true));
|
||||
}
|
||||
|
||||
private void assertMultiSortResponse(double[] expectedKeys, Terms.Order... order) {
|
||||
private void assertMultiSortResponse(double[] expectedKeys, BucketOrder... order) {
|
||||
SearchResponse response = client()
|
||||
.prepareSearch("sort_idx")
|
||||
.setTypes("multi_sort_type")
|
||||
.addAggregation(
|
||||
terms("terms").field(SINGLE_VALUED_FIELD_NAME).collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.compound(order)).subAggregation(avg("avg_l").field("l"))
|
||||
.order(BucketOrder.compound(order)).subAggregation(avg("avg_l").field("l"))
|
||||
.subAggregation(sum("sum_d").field("d"))).execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
|
|
@ -19,7 +19,9 @@
|
|||
package org.elasticsearch.search.aggregations.bucket;
|
||||
|
||||
import com.carrotsearch.hppc.LongHashSet;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchPhaseExecutionException;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.query.QueryBuilders;
|
||||
|
@ -27,16 +29,20 @@ import org.elasticsearch.plugins.Plugin;
|
|||
import org.elasticsearch.script.MockScriptPlugin;
|
||||
import org.elasticsearch.script.Script;
|
||||
import org.elasticsearch.script.ScriptType;
|
||||
import org.elasticsearch.search.aggregations.AggregationExecutionException;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||
import org.elasticsearch.search.aggregations.bucket.filter.Filter;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket;
|
||||
import org.elasticsearch.search.aggregations.metrics.avg.Avg;
|
||||
import org.elasticsearch.search.aggregations.metrics.max.Max;
|
||||
import org.elasticsearch.search.aggregations.metrics.stats.Stats;
|
||||
import org.elasticsearch.search.aggregations.metrics.sum.Sum;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
import org.hamcrest.Matchers;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
|
@ -48,6 +54,7 @@ import java.util.function.Function;
|
|||
import static java.util.Collections.emptyMap;
|
||||
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
|
||||
import static org.elasticsearch.search.aggregations.AggregationBuilders.avg;
|
||||
import static org.elasticsearch.search.aggregations.AggregationBuilders.filter;
|
||||
import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram;
|
||||
import static org.elasticsearch.search.aggregations.AggregationBuilders.max;
|
||||
|
@ -72,6 +79,7 @@ public class HistogramIT extends ESIntegTestCase {
|
|||
static int interval;
|
||||
static int numValueBuckets, numValuesBuckets;
|
||||
static long[] valueCounts, valuesCounts;
|
||||
static Map<Long, Map<String, Object>> expectedMultiSortBuckets;
|
||||
|
||||
@Override
|
||||
protected Collection<Class<? extends Plugin>> nodePlugins() {
|
||||
|
@ -130,16 +138,18 @@ public class HistogramIT extends ESIntegTestCase {
|
|||
}
|
||||
List<IndexRequestBuilder> builders = new ArrayList<>();
|
||||
|
||||
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
builders.add(client().prepareIndex("idx", "type").setSource(jsonBuilder()
|
||||
.startObject()
|
||||
.field(SINGLE_VALUED_FIELD_NAME, i + 1)
|
||||
.startArray(MULTI_VALUED_FIELD_NAME).value(i + 1).value(i + 2).endArray()
|
||||
.field("tag", "tag" + i)
|
||||
.field("constant", 1)
|
||||
.endObject()));
|
||||
}
|
||||
|
||||
getMultiSortDocs(builders);
|
||||
|
||||
assertAcked(prepareCreate("empty_bucket_idx").addMapping("type", SINGLE_VALUED_FIELD_NAME, "type=integer"));
|
||||
for (int i = 0; i < 2; i++) {
|
||||
builders.add(client().prepareIndex("empty_bucket_idx", "type", "" + i).setSource(jsonBuilder()
|
||||
|
@ -151,6 +161,51 @@ public class HistogramIT extends ESIntegTestCase {
|
|||
ensureSearchable();
|
||||
}
|
||||
|
||||
private void addExpectedBucket(long key, long docCount, double avg, double sum) {
|
||||
Map<String, Object> bucketProps = new HashMap<>();
|
||||
bucketProps.put("key", key);
|
||||
bucketProps.put("_count", docCount);
|
||||
bucketProps.put("avg_l", avg);
|
||||
bucketProps.put("sum_d", sum);
|
||||
expectedMultiSortBuckets.put(key, bucketProps);
|
||||
}
|
||||
|
||||
private void getMultiSortDocs(List<IndexRequestBuilder> builders) throws IOException {
|
||||
expectedMultiSortBuckets = new HashMap<>();
|
||||
addExpectedBucket(1, 3, 1, 6);
|
||||
addExpectedBucket(2, 3, 2, 6);
|
||||
addExpectedBucket(3, 2, 3, 3);
|
||||
addExpectedBucket(4, 2, 3, 4);
|
||||
addExpectedBucket(5, 2, 5, 3);
|
||||
addExpectedBucket(6, 1, 5, 1);
|
||||
addExpectedBucket(7, 1, 5, 1);
|
||||
|
||||
assertAcked(client().admin().indices().prepareCreate("sort_idx")
|
||||
.addMapping("type", SINGLE_VALUED_FIELD_NAME, "type=double").get());
|
||||
for (int i = 1; i <= 3; i++) {
|
||||
builders.add(client().prepareIndex("sort_idx", "type").setSource(
|
||||
jsonBuilder().startObject().field(SINGLE_VALUED_FIELD_NAME, 1).field("l", 1).field("d", i).endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "type").setSource(
|
||||
jsonBuilder().startObject().field(SINGLE_VALUED_FIELD_NAME, 2).field("l", 2).field("d", i).endObject()));
|
||||
}
|
||||
builders.add(client().prepareIndex("sort_idx", "type").setSource(
|
||||
jsonBuilder().startObject().field(SINGLE_VALUED_FIELD_NAME, 3).field("l", 3).field("d", 1).endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "type").setSource(
|
||||
jsonBuilder().startObject().field(SINGLE_VALUED_FIELD_NAME, 3.8).field("l", 3).field("d", 2).endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "type").setSource(
|
||||
jsonBuilder().startObject().field(SINGLE_VALUED_FIELD_NAME, 4).field("l", 3).field("d", 1).endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "type").setSource(
|
||||
jsonBuilder().startObject().field(SINGLE_VALUED_FIELD_NAME, 4.4).field("l", 3).field("d", 3).endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "type").setSource(
|
||||
jsonBuilder().startObject().field(SINGLE_VALUED_FIELD_NAME, 5).field("l", 5).field("d", 1).endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "type").setSource(
|
||||
jsonBuilder().startObject().field(SINGLE_VALUED_FIELD_NAME, 5.1).field("l", 5).field("d", 2).endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "type").setSource(
|
||||
jsonBuilder().startObject().field(SINGLE_VALUED_FIELD_NAME, 6).field("l", 5).field("d", 1).endObject()));
|
||||
builders.add(client().prepareIndex("sort_idx", "type").setSource(
|
||||
jsonBuilder().startObject().field(SINGLE_VALUED_FIELD_NAME, 7).field("l", 5).field("d", 1).endObject()));
|
||||
}
|
||||
|
||||
public void testSingleValuedField() throws Exception {
|
||||
SearchResponse response = client().prepareSearch("idx")
|
||||
.addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval))
|
||||
|
@ -241,7 +296,7 @@ public class HistogramIT extends ESIntegTestCase {
|
|||
|
||||
public void testSingleValuedFieldOrderedByKeyAsc() throws Exception {
|
||||
SearchResponse response = client().prepareSearch("idx")
|
||||
.addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).order(Histogram.Order.KEY_ASC))
|
||||
.addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).order(BucketOrder.key(true)))
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -252,7 +307,6 @@ public class HistogramIT extends ESIntegTestCase {
|
|||
assertThat(histo.getName(), equalTo("histo"));
|
||||
assertThat(histo.getBuckets().size(), equalTo(numValueBuckets));
|
||||
|
||||
// TODO: use diamond once JI-9019884 is fixed
|
||||
List<Histogram.Bucket> buckets = new ArrayList<>(histo.getBuckets());
|
||||
for (int i = 0; i < numValueBuckets; ++i) {
|
||||
Histogram.Bucket bucket = buckets.get(i);
|
||||
|
@ -264,7 +318,7 @@ public class HistogramIT extends ESIntegTestCase {
|
|||
|
||||
public void testsingleValuedFieldOrderedByKeyDesc() throws Exception {
|
||||
SearchResponse response = client().prepareSearch("idx")
|
||||
.addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).order(Histogram.Order.KEY_DESC))
|
||||
.addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).order(BucketOrder.key(false)))
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -275,7 +329,6 @@ public class HistogramIT extends ESIntegTestCase {
|
|||
assertThat(histo.getName(), equalTo("histo"));
|
||||
assertThat(histo.getBuckets().size(), equalTo(numValueBuckets));
|
||||
|
||||
// TODO: use diamond once JI-9019884 is fixed
|
||||
List<Histogram.Bucket> buckets = new ArrayList<>(histo.getBuckets());
|
||||
for (int i = 0; i < numValueBuckets; ++i) {
|
||||
Histogram.Bucket bucket = buckets.get(numValueBuckets - i - 1);
|
||||
|
@ -287,7 +340,7 @@ public class HistogramIT extends ESIntegTestCase {
|
|||
|
||||
public void testSingleValuedFieldOrderedByCountAsc() throws Exception {
|
||||
SearchResponse response = client().prepareSearch("idx")
|
||||
.addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).order(Histogram.Order.COUNT_ASC))
|
||||
.addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).order(BucketOrder.count(true)))
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -299,7 +352,6 @@ public class HistogramIT extends ESIntegTestCase {
|
|||
assertThat(histo.getBuckets().size(), equalTo(numValueBuckets));
|
||||
|
||||
LongHashSet buckets = new LongHashSet();
|
||||
// TODO: use diamond once JI-9019884 is fixed
|
||||
List<Histogram.Bucket> histoBuckets = new ArrayList<>(histo.getBuckets());
|
||||
long previousCount = Long.MIN_VALUE;
|
||||
for (int i = 0; i < numValueBuckets; ++i) {
|
||||
|
@ -316,7 +368,7 @@ public class HistogramIT extends ESIntegTestCase {
|
|||
|
||||
public void testSingleValuedFieldOrderedByCountDesc() throws Exception {
|
||||
SearchResponse response = client().prepareSearch("idx")
|
||||
.addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).order(Histogram.Order.COUNT_DESC))
|
||||
.addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).order(BucketOrder.count(false)))
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -328,7 +380,6 @@ public class HistogramIT extends ESIntegTestCase {
|
|||
assertThat(histo.getBuckets().size(), equalTo(numValueBuckets));
|
||||
|
||||
LongHashSet buckets = new LongHashSet();
|
||||
// TODO: use diamond once JI-9019884 is fixed
|
||||
List<Histogram.Bucket> histoBuckets = new ArrayList<>(histo.getBuckets());
|
||||
long previousCount = Long.MAX_VALUE;
|
||||
for (int i = 0; i < numValueBuckets; ++i) {
|
||||
|
@ -361,7 +412,6 @@ public class HistogramIT extends ESIntegTestCase {
|
|||
Object[] propertiesDocCounts = (Object[]) ((InternalAggregation)histo).getProperty("_count");
|
||||
Object[] propertiesCounts = (Object[]) ((InternalAggregation)histo).getProperty("sum.value");
|
||||
|
||||
// TODO: use diamond once JI-9019884 is fixed
|
||||
List<Histogram.Bucket> buckets = new ArrayList<>(histo.getBuckets());
|
||||
for (int i = 0; i < numValueBuckets; ++i) {
|
||||
Histogram.Bucket bucket = buckets.get(i);
|
||||
|
@ -390,7 +440,7 @@ public class HistogramIT extends ESIntegTestCase {
|
|||
histogram("histo")
|
||||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.interval(interval)
|
||||
.order(Histogram.Order.aggregation("sum", true))
|
||||
.order(BucketOrder.aggregation("sum", true))
|
||||
.subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME)))
|
||||
.execute().actionGet();
|
||||
|
||||
|
@ -404,7 +454,6 @@ public class HistogramIT extends ESIntegTestCase {
|
|||
|
||||
LongHashSet visited = new LongHashSet();
|
||||
double previousSum = Double.NEGATIVE_INFINITY;
|
||||
// TODO: use diamond once JI-9019884 is fixed
|
||||
List<Histogram.Bucket> buckets = new ArrayList<>(histo.getBuckets());
|
||||
for (int i = 0; i < numValueBuckets; ++i) {
|
||||
Histogram.Bucket bucket = buckets.get(i);
|
||||
|
@ -434,7 +483,7 @@ public class HistogramIT extends ESIntegTestCase {
|
|||
histogram("histo")
|
||||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.interval(interval)
|
||||
.order(Histogram.Order.aggregation("sum", false))
|
||||
.order(BucketOrder.aggregation("sum", false))
|
||||
.subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME)))
|
||||
.execute().actionGet();
|
||||
|
||||
|
@ -448,7 +497,6 @@ public class HistogramIT extends ESIntegTestCase {
|
|||
|
||||
LongHashSet visited = new LongHashSet();
|
||||
double previousSum = Double.POSITIVE_INFINITY;
|
||||
// TODO: use diamond once JI-9019884 is fixed
|
||||
List<Histogram.Bucket> buckets = new ArrayList<>(histo.getBuckets());
|
||||
for (int i = 0; i < numValueBuckets; ++i) {
|
||||
Histogram.Bucket bucket = buckets.get(i);
|
||||
|
@ -478,7 +526,7 @@ public class HistogramIT extends ESIntegTestCase {
|
|||
histogram("histo")
|
||||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.interval(interval)
|
||||
.order(Histogram.Order.aggregation("stats.sum", false))
|
||||
.order(BucketOrder.aggregation("stats.sum", false))
|
||||
.subAggregation(stats("stats").field(SINGLE_VALUED_FIELD_NAME)))
|
||||
.execute().actionGet();
|
||||
|
||||
|
@ -492,7 +540,7 @@ public class HistogramIT extends ESIntegTestCase {
|
|||
|
||||
LongHashSet visited = new LongHashSet();
|
||||
double previousSum = Double.POSITIVE_INFINITY;
|
||||
// TODO: use diamond once JI-9019884 is fixed
|
||||
|
||||
List<Histogram.Bucket> buckets = new ArrayList<>(histo.getBuckets());
|
||||
for (int i = 0; i < numValueBuckets; ++i) {
|
||||
Histogram.Bucket bucket = buckets.get(i);
|
||||
|
@ -523,7 +571,7 @@ public class HistogramIT extends ESIntegTestCase {
|
|||
histogram("histo")
|
||||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.interval(interval)
|
||||
.order(Histogram.Order.aggregation("filter>max", asc))
|
||||
.order(BucketOrder.aggregation("filter>max", asc))
|
||||
.subAggregation(filter("filter", matchAllQuery())
|
||||
.subAggregation(max("max").field(SINGLE_VALUED_FIELD_NAME))))
|
||||
.execute().actionGet();
|
||||
|
@ -538,7 +586,6 @@ public class HistogramIT extends ESIntegTestCase {
|
|||
|
||||
LongHashSet visited = new LongHashSet();
|
||||
double prevMax = asc ? Double.NEGATIVE_INFINITY : Double.POSITIVE_INFINITY;
|
||||
// TODO: use diamond once JI-9019884 is fixed
|
||||
List<Histogram.Bucket> buckets = new ArrayList<>(histo.getBuckets());
|
||||
for (int i = 0; i < numValueBuckets; ++i) {
|
||||
Histogram.Bucket bucket = buckets.get(i);
|
||||
|
@ -558,6 +605,62 @@ public class HistogramIT extends ESIntegTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedByTieBreaker() throws Exception {
|
||||
SearchResponse response = client().prepareSearch("idx")
|
||||
.addAggregation(histogram("histo")
|
||||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.interval(interval)
|
||||
.order(BucketOrder.aggregation("max_constant", randomBoolean()))
|
||||
.subAggregation(max("max_constant").field("constant")))
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
||||
Histogram histo = response.getAggregations().get("histo");
|
||||
assertThat(histo, notNullValue());
|
||||
assertThat(histo.getName(), equalTo("histo"));
|
||||
assertThat(histo.getBuckets().size(), equalTo(numValueBuckets));
|
||||
|
||||
List<Histogram.Bucket> buckets = new ArrayList<>(histo.getBuckets());
|
||||
for (int i = 0; i < numValueBuckets; ++i) {
|
||||
Histogram.Bucket bucket = buckets.get(i);
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat(((Number) bucket.getKey()).longValue(), equalTo((long) i * interval));
|
||||
assertThat(bucket.getDocCount(), equalTo(valueCounts[i]));
|
||||
}
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedByIllegalAgg() throws Exception {
|
||||
boolean asc = true;
|
||||
try {
|
||||
client()
|
||||
.prepareSearch("idx")
|
||||
.addAggregation(
|
||||
histogram("histo").field(SINGLE_VALUED_FIELD_NAME)
|
||||
.interval(interval)
|
||||
.order(BucketOrder.aggregation("inner_histo>avg", asc))
|
||||
.subAggregation(histogram("inner_histo")
|
||||
.interval(interval)
|
||||
.field(MULTI_VALUED_FIELD_NAME)
|
||||
.subAggregation(avg("avg").field("value"))))
|
||||
.execute().actionGet();
|
||||
fail("Expected an exception");
|
||||
} catch (SearchPhaseExecutionException e) {
|
||||
ElasticsearchException[] rootCauses = e.guessRootCauses();
|
||||
if (rootCauses.length == 1) {
|
||||
ElasticsearchException rootCause = rootCauses[0];
|
||||
if (rootCause instanceof AggregationExecutionException) {
|
||||
AggregationExecutionException aggException = (AggregationExecutionException) rootCause;
|
||||
assertThat(aggException.getMessage(), Matchers.startsWith("Invalid aggregation order path"));
|
||||
} else {
|
||||
throw e;
|
||||
}
|
||||
} else {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldWithValueScript() throws Exception {
|
||||
SearchResponse response = client().prepareSearch("idx")
|
||||
.addAggregation(
|
||||
|
@ -614,7 +717,7 @@ public class HistogramIT extends ESIntegTestCase {
|
|||
|
||||
public void testMultiValuedFieldOrderedByKeyDesc() throws Exception {
|
||||
SearchResponse response = client().prepareSearch("idx")
|
||||
.addAggregation(histogram("histo").field(MULTI_VALUED_FIELD_NAME).interval(interval).order(Histogram.Order.KEY_DESC))
|
||||
.addAggregation(histogram("histo").field(MULTI_VALUED_FIELD_NAME).interval(interval).order(BucketOrder.key(false)))
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -625,7 +728,6 @@ public class HistogramIT extends ESIntegTestCase {
|
|||
assertThat(histo.getName(), equalTo("histo"));
|
||||
assertThat(histo.getBuckets().size(), equalTo(numValuesBuckets));
|
||||
|
||||
// TODO: use diamond once JI-9019884 is fixed
|
||||
List<Histogram.Bucket> buckets = new ArrayList<>(histo.getBuckets());
|
||||
for (int i = 0; i < numValuesBuckets; ++i) {
|
||||
Histogram.Bucket bucket = buckets.get(numValuesBuckets - i - 1);
|
||||
|
@ -1036,4 +1138,74 @@ public class HistogramIT extends ESIntegTestCase {
|
|||
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
|
||||
.getMissCount(), equalTo(1L));
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscAndKeyDesc() throws Exception {
|
||||
long[] expectedKeys = new long[] { 1, 2, 4, 3, 7, 6, 5 };
|
||||
assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", true), BucketOrder.key(false));
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscAndKeyAsc() throws Exception {
|
||||
long[] expectedKeys = new long[] { 1, 2, 3, 4, 5, 6, 7 };
|
||||
assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", true), BucketOrder.key(true));
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedBySingleValueSubAggregationDescAndKeyAsc() throws Exception {
|
||||
long[] expectedKeys = new long[] { 5, 6, 7, 3, 4, 2, 1 };
|
||||
assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", false), BucketOrder.key(true));
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedByCountAscAndSingleValueSubAggregationAsc() throws Exception {
|
||||
long[] expectedKeys = new long[] { 6, 7, 3, 4, 5, 1, 2 };
|
||||
assertMultiSortResponse(expectedKeys, BucketOrder.count(true), BucketOrder.aggregation("avg_l", true));
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscSingleValueSubAggregationAsc() throws Exception {
|
||||
long[] expectedKeys = new long[] { 6, 7, 3, 5, 4, 1, 2 };
|
||||
assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("sum_d", true), BucketOrder.aggregation("avg_l", true));
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedByThreeCriteria() throws Exception {
|
||||
long[] expectedKeys = new long[] { 2, 1, 4, 5, 3, 6, 7 };
|
||||
assertMultiSortResponse(expectedKeys, BucketOrder.count(false), BucketOrder.aggregation("sum_d", false),
|
||||
BucketOrder.aggregation("avg_l", false));
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscAsCompound() throws Exception {
|
||||
long[] expectedKeys = new long[] { 1, 2, 3, 4, 5, 6, 7 };
|
||||
assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", true));
|
||||
}
|
||||
|
||||
private void assertMultiSortResponse(long[] expectedKeys, BucketOrder... order) {
|
||||
SearchResponse response = client()
|
||||
.prepareSearch("sort_idx")
|
||||
.setTypes("type")
|
||||
.addAggregation(
|
||||
histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(1).order(BucketOrder.compound(order))
|
||||
.subAggregation(avg("avg_l").field("l")).subAggregation(sum("sum_d").field("d"))).execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
||||
Histogram histogram = response.getAggregations().get("histo");
|
||||
assertThat(histogram, notNullValue());
|
||||
assertThat(histogram.getName(), equalTo("histo"));
|
||||
assertThat(histogram.getBuckets().size(), equalTo(expectedKeys.length));
|
||||
|
||||
int i = 0;
|
||||
for (Histogram.Bucket bucket : histogram.getBuckets()) {
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat(key(bucket), equalTo(expectedKeys[i]));
|
||||
assertThat(bucket.getDocCount(), equalTo(expectedMultiSortBuckets.get(expectedKeys[i]).get("_count")));
|
||||
Avg avg = bucket.getAggregations().get("avg_l");
|
||||
assertThat(avg, notNullValue());
|
||||
assertThat(avg.getValue(), equalTo(expectedMultiSortBuckets.get(expectedKeys[i]).get("avg_l")));
|
||||
Sum sum = bucket.getAggregations().get("sum_d");
|
||||
assertThat(sum, notNullValue());
|
||||
assertThat(sum.getValue(), equalTo(expectedMultiSortBuckets.get(expectedKeys[i]).get("sum_d")));
|
||||
i++;
|
||||
}
|
||||
}
|
||||
|
||||
private long key(Histogram.Bucket bucket) {
|
||||
return ((Number) bucket.getKey()).longValue();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,7 +21,10 @@ package org.elasticsearch.search.aggregations.bucket;
|
|||
|
||||
import org.elasticsearch.search.aggregations.BaseAggregationTestCase;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregationBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Order;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.startsWith;
|
||||
|
@ -54,26 +57,11 @@ public class HistogramTests extends BaseAggregationTestCase<HistogramAggregation
|
|||
factory.offset(randomIntBetween(0, 100000));
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
int branch = randomInt(5);
|
||||
switch (branch) {
|
||||
case 0:
|
||||
factory.order(Order.COUNT_ASC);
|
||||
break;
|
||||
case 1:
|
||||
factory.order(Order.COUNT_DESC);
|
||||
break;
|
||||
case 2:
|
||||
factory.order(Order.KEY_ASC);
|
||||
break;
|
||||
case 3:
|
||||
factory.order(Order.KEY_DESC);
|
||||
break;
|
||||
case 4:
|
||||
factory.order(Order.aggregation("foo", true));
|
||||
break;
|
||||
case 5:
|
||||
factory.order(Order.aggregation("foo", false));
|
||||
break;
|
||||
List<BucketOrder> order = randomOrder();
|
||||
if(order.size() == 1 && randomBoolean()) {
|
||||
factory.order(order.get(0));
|
||||
} else {
|
||||
factory.order(order);
|
||||
}
|
||||
}
|
||||
return factory;
|
||||
|
@ -102,4 +90,31 @@ public class HistogramTests extends BaseAggregationTestCase<HistogramAggregation
|
|||
assertThat(ex.getMessage(), equalTo("maxBound [0.4] must be greater than minBound [0.5]"));
|
||||
}
|
||||
|
||||
private List<BucketOrder> randomOrder() {
|
||||
List<BucketOrder> orders = new ArrayList<>();
|
||||
switch (randomInt(4)) {
|
||||
case 0:
|
||||
orders.add(BucketOrder.key(randomBoolean()));
|
||||
break;
|
||||
case 1:
|
||||
orders.add(BucketOrder.count(randomBoolean()));
|
||||
break;
|
||||
case 2:
|
||||
orders.add(BucketOrder.aggregation(randomAlphaOfLengthBetween(3, 20), randomBoolean()));
|
||||
break;
|
||||
case 3:
|
||||
orders.add(BucketOrder.aggregation(randomAlphaOfLengthBetween(3, 20), randomAlphaOfLengthBetween(3, 20), randomBoolean()));
|
||||
break;
|
||||
case 4:
|
||||
int numOrders = randomIntBetween(1, 3);
|
||||
for (int i = 0; i < numOrders; i++) {
|
||||
orders.addAll(randomOrder());
|
||||
}
|
||||
break;
|
||||
default:
|
||||
fail();
|
||||
}
|
||||
return orders;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -40,6 +40,7 @@ import org.elasticsearch.search.aggregations.metrics.max.Max;
|
|||
import org.elasticsearch.search.aggregations.metrics.stats.Stats;
|
||||
import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStats;
|
||||
import org.elasticsearch.search.aggregations.metrics.sum.Sum;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
import org.hamcrest.Matchers;
|
||||
|
||||
|
@ -121,6 +122,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
|
|||
.field(SINGLE_VALUED_FIELD_NAME, i)
|
||||
.startArray(MULTI_VALUED_FIELD_NAME).value(i).value(i + 1).endArray()
|
||||
.field("num_tag", i < lowCardBuilders.length / 2 + 1 ? 1 : 0) // used to test order by single-bucket sub agg
|
||||
.field("constant", 1)
|
||||
.endObject());
|
||||
}
|
||||
indexRandom(true, lowCardBuilders);
|
||||
|
@ -392,7 +394,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
|
|||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.size(20)
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.term(true))) // we need to sort by terms cause we're checking the first 20 values
|
||||
.order(BucketOrder.key(true))) // we need to sort by terms cause we're checking the first 20 values
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -417,7 +419,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
|
|||
.addAggregation(terms("terms")
|
||||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.term(true)))
|
||||
.order(BucketOrder.key(true)))
|
||||
.execute().actionGet();
|
||||
assertSearchResponse(response);
|
||||
|
||||
|
@ -441,7 +443,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
|
|||
.addAggregation(terms("terms")
|
||||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.term(false)))
|
||||
.order(BucketOrder.key(false)))
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -462,6 +464,31 @@ public class LongTermsIT extends AbstractTermsTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public void testSingleValueFieldOrderedByTieBreaker() throws Exception {
|
||||
SearchResponse response = client().prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(terms("terms")
|
||||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(BucketOrder.aggregation("max_constant", randomBoolean()))
|
||||
.subAggregation(max("max_constant").field("constant")))
|
||||
.execute().actionGet();
|
||||
assertSearchResponse(response);
|
||||
|
||||
Terms terms = response.getAggregations().get("terms");
|
||||
assertThat(terms, notNullValue());
|
||||
assertThat(terms.getName(), equalTo("terms"));
|
||||
assertThat(terms.getBuckets().size(), equalTo(5));
|
||||
|
||||
int i = 0;
|
||||
for (Terms.Bucket bucket : terms.getBuckets()) {
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat(key(bucket), equalTo("" + i));
|
||||
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
|
||||
assertThat(bucket.getDocCount(), equalTo(1L));
|
||||
i++;
|
||||
}
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldWithSubAggregation() throws Exception {
|
||||
SearchResponse response = client().prepareSearch("idx")
|
||||
.addAggregation(terms("terms")
|
||||
|
@ -769,7 +796,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
|
|||
.addAggregation(terms("terms")
|
||||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.aggregation("avg_i", asc))
|
||||
.order(BucketOrder.aggregation("avg_i", asc))
|
||||
.subAggregation(avg("avg_i").field(SINGLE_VALUED_FIELD_NAME))
|
||||
).execute().actionGet();
|
||||
|
||||
|
@ -798,7 +825,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
|
|||
.addAggregation(terms("terms")
|
||||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.aggregation("avg_i", asc))
|
||||
.order(BucketOrder.aggregation("avg_i", asc))
|
||||
.subAggregation(
|
||||
avg("avg_i").field(SINGLE_VALUED_FIELD_NAME))
|
||||
.subAggregation(
|
||||
|
@ -842,7 +869,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
|
|||
.addAggregation(terms("num_tags")
|
||||
.field("num_tag")
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.aggregation("filter", asc))
|
||||
.order(BucketOrder.aggregation("filter", asc))
|
||||
.subAggregation(filter("filter", QueryBuilders.matchAllQuery()))
|
||||
).get();
|
||||
|
||||
|
@ -879,7 +906,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
|
|||
.addAggregation(terms("tags")
|
||||
.field("num_tag")
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.aggregation("filter1>filter2>max", asc))
|
||||
.order(BucketOrder.aggregation("filter1>filter2>max", asc))
|
||||
.subAggregation(filter("filter1", QueryBuilders.matchAllQuery()).subAggregation(
|
||||
filter("filter2", QueryBuilders.matchAllQuery())
|
||||
.subAggregation(max("max").field(SINGLE_VALUED_FIELD_NAME))))
|
||||
|
@ -934,7 +961,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
|
|||
.addAggregation(terms("terms")
|
||||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.aggregation("avg_i", true))
|
||||
.order(BucketOrder.aggregation("avg_i", true))
|
||||
).execute().actionGet();
|
||||
|
||||
fail("Expected search to fail when trying to sort terms aggregation by sug-aggregation that doesn't exist");
|
||||
|
@ -952,7 +979,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
|
|||
.addAggregation(terms("terms")
|
||||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.aggregation("num_tags", true))
|
||||
.order(BucketOrder.aggregation("num_tags", true))
|
||||
.subAggregation(terms("num_tags").field("num_tags")
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values())))
|
||||
).execute().actionGet();
|
||||
|
@ -972,7 +999,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
|
|||
.addAggregation(terms("terms")
|
||||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.aggregation("stats.foo", true))
|
||||
.order(BucketOrder.aggregation("stats.foo", true))
|
||||
.subAggregation(stats("stats").field(SINGLE_VALUED_FIELD_NAME))
|
||||
).execute().actionGet();
|
||||
|
||||
|
@ -992,7 +1019,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
|
|||
.addAggregation(terms("terms")
|
||||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.aggregation("stats", true))
|
||||
.order(BucketOrder.aggregation("stats", true))
|
||||
.subAggregation(stats("stats").field(SINGLE_VALUED_FIELD_NAME))
|
||||
).execute().actionGet();
|
||||
|
||||
|
@ -1011,7 +1038,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
|
|||
.addAggregation(terms("terms")
|
||||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.aggregation("avg_i", asc))
|
||||
.order(BucketOrder.aggregation("avg_i", asc))
|
||||
.subAggregation(avg("avg_i").field(SINGLE_VALUED_FIELD_NAME))
|
||||
).execute().actionGet();
|
||||
|
||||
|
@ -1043,7 +1070,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
|
|||
.addAggregation(terms("terms")
|
||||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.aggregation("stats.avg", asc))
|
||||
.order(BucketOrder.aggregation("stats.avg", asc))
|
||||
.subAggregation(stats("stats").field(SINGLE_VALUED_FIELD_NAME))
|
||||
).execute().actionGet();
|
||||
|
||||
|
@ -1073,7 +1100,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
|
|||
.addAggregation(terms("terms")
|
||||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.aggregation("stats.avg", asc))
|
||||
.order(BucketOrder.aggregation("stats.avg", asc))
|
||||
.subAggregation(stats("stats").field(SINGLE_VALUED_FIELD_NAME))
|
||||
).execute().actionGet();
|
||||
|
||||
|
@ -1103,7 +1130,7 @@ public class LongTermsIT extends AbstractTermsTestCase {
|
|||
.addAggregation(terms("terms")
|
||||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.aggregation("stats.variance", asc))
|
||||
.order(BucketOrder.aggregation("stats.variance", asc))
|
||||
.subAggregation(extendedStats("stats").field(SINGLE_VALUED_FIELD_NAME))
|
||||
).execute().actionGet();
|
||||
|
||||
|
@ -1129,47 +1156,47 @@ public class LongTermsIT extends AbstractTermsTestCase {
|
|||
|
||||
public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscAndTermsDesc() throws Exception {
|
||||
long[] expectedKeys = new long[] { 1, 2, 4, 3, 7, 6, 5 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("avg_l", true), Terms.Order.term(false));
|
||||
assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", true), BucketOrder.key(false));
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscAndTermsAsc() throws Exception {
|
||||
long[] expectedKeys = new long[] { 1, 2, 3, 4, 5, 6, 7 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("avg_l", true), Terms.Order.term(true));
|
||||
assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", true), BucketOrder.key(true));
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedBySingleValueSubAggregationDescAndTermsAsc() throws Exception {
|
||||
long[] expectedKeys = new long[] { 5, 6, 7, 3, 4, 2, 1 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("avg_l", false), Terms.Order.term(true));
|
||||
assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", false), BucketOrder.key(true));
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedByCountAscAndSingleValueSubAggregationAsc() throws Exception {
|
||||
long[] expectedKeys = new long[] { 6, 7, 3, 4, 5, 1, 2 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.count(true), Terms.Order.aggregation("avg_l", true));
|
||||
assertMultiSortResponse(expectedKeys, BucketOrder.count(true), BucketOrder.aggregation("avg_l", true));
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscSingleValueSubAggregationAsc() throws Exception {
|
||||
long[] expectedKeys = new long[] { 6, 7, 3, 5, 4, 1, 2 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("sum_d", true), Terms.Order.aggregation("avg_l", true));
|
||||
assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("sum_d", true), BucketOrder.aggregation("avg_l", true));
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedByThreeCriteria() throws Exception {
|
||||
long[] expectedKeys = new long[] { 2, 1, 4, 5, 3, 6, 7 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.count(false),
|
||||
Terms.Order.aggregation("sum_d", false),
|
||||
Terms.Order.aggregation("avg_l", false));
|
||||
assertMultiSortResponse(expectedKeys, BucketOrder.count(false),
|
||||
BucketOrder.aggregation("sum_d", false),
|
||||
BucketOrder.aggregation("avg_l", false));
|
||||
}
|
||||
|
||||
public void testSingleValuedFieldOrderedBySingleValueSubAggregationAscAsCompound() throws Exception {
|
||||
long[] expectedKeys = new long[] { 1, 2, 3, 4, 5, 6, 7 };
|
||||
assertMultiSortResponse(expectedKeys, Terms.Order.aggregation("avg_l", true));
|
||||
assertMultiSortResponse(expectedKeys, BucketOrder.aggregation("avg_l", true));
|
||||
}
|
||||
|
||||
private void assertMultiSortResponse(long[] expectedKeys, Terms.Order... order) {
|
||||
private void assertMultiSortResponse(long[] expectedKeys, BucketOrder... order) {
|
||||
SearchResponse response = client().prepareSearch("sort_idx").setTypes("multi_sort_type")
|
||||
.addAggregation(terms("terms")
|
||||
.field(SINGLE_VALUED_FIELD_NAME)
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.order(Terms.Order.compound(order))
|
||||
.order(BucketOrder.compound(order))
|
||||
.subAggregation(avg("avg_l").field("l"))
|
||||
.subAggregation(sum("sum_d").field("d"))
|
||||
).execute().actionGet();
|
||||
|
|
|
@ -37,6 +37,7 @@ import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
|
|||
import org.elasticsearch.search.aggregations.bucket.terms.Terms;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.DateTimeZone;
|
||||
|
@ -190,122 +191,122 @@ public class MinDocCountIT extends AbstractTermsTestCase {
|
|||
}
|
||||
|
||||
public void testStringTermAsc() throws Exception {
|
||||
testMinDocCountOnTerms("s", Script.NO, Terms.Order.term(true));
|
||||
testMinDocCountOnTerms("s", Script.NO, BucketOrder.key(true));
|
||||
}
|
||||
|
||||
public void testStringScriptTermAsc() throws Exception {
|
||||
testMinDocCountOnTerms("s", Script.YES, Terms.Order.term(true));
|
||||
testMinDocCountOnTerms("s", Script.YES, BucketOrder.key(true));
|
||||
}
|
||||
|
||||
public void testStringTermDesc() throws Exception {
|
||||
testMinDocCountOnTerms("s", Script.NO, Terms.Order.term(false));
|
||||
testMinDocCountOnTerms("s", Script.NO, BucketOrder.key(false));
|
||||
}
|
||||
|
||||
public void testStringScriptTermDesc() throws Exception {
|
||||
testMinDocCountOnTerms("s", Script.YES, Terms.Order.term(false));
|
||||
testMinDocCountOnTerms("s", Script.YES, BucketOrder.key(false));
|
||||
}
|
||||
|
||||
public void testStringCountAsc() throws Exception {
|
||||
testMinDocCountOnTerms("s", Script.NO, Terms.Order.count(true));
|
||||
testMinDocCountOnTerms("s", Script.NO, BucketOrder.count(true));
|
||||
}
|
||||
|
||||
public void testStringScriptCountAsc() throws Exception {
|
||||
testMinDocCountOnTerms("s", Script.YES, Terms.Order.count(true));
|
||||
testMinDocCountOnTerms("s", Script.YES, BucketOrder.count(true));
|
||||
}
|
||||
|
||||
public void testStringCountDesc() throws Exception {
|
||||
testMinDocCountOnTerms("s", Script.NO, Terms.Order.count(false));
|
||||
testMinDocCountOnTerms("s", Script.NO, BucketOrder.count(false));
|
||||
}
|
||||
|
||||
public void testStringScriptCountDesc() throws Exception {
|
||||
testMinDocCountOnTerms("s", Script.YES, Terms.Order.count(false));
|
||||
testMinDocCountOnTerms("s", Script.YES, BucketOrder.count(false));
|
||||
}
|
||||
|
||||
public void testStringCountAscWithInclude() throws Exception {
|
||||
testMinDocCountOnTerms("s", Script.NO, Terms.Order.count(true), ".*a.*", true);
|
||||
testMinDocCountOnTerms("s", Script.NO, BucketOrder.count(true), ".*a.*", true);
|
||||
}
|
||||
|
||||
public void testStringScriptCountAscWithInclude() throws Exception {
|
||||
testMinDocCountOnTerms("s", Script.YES, Terms.Order.count(true), ".*a.*", true);
|
||||
testMinDocCountOnTerms("s", Script.YES, BucketOrder.count(true), ".*a.*", true);
|
||||
}
|
||||
|
||||
public void testStringCountDescWithInclude() throws Exception {
|
||||
testMinDocCountOnTerms("s", Script.NO, Terms.Order.count(false), ".*a.*", true);
|
||||
testMinDocCountOnTerms("s", Script.NO, BucketOrder.count(false), ".*a.*", true);
|
||||
}
|
||||
|
||||
public void testStringScriptCountDescWithInclude() throws Exception {
|
||||
testMinDocCountOnTerms("s", Script.YES, Terms.Order.count(false), ".*a.*", true);
|
||||
testMinDocCountOnTerms("s", Script.YES, BucketOrder.count(false), ".*a.*", true);
|
||||
}
|
||||
|
||||
public void testLongTermAsc() throws Exception {
|
||||
testMinDocCountOnTerms("l", Script.NO, Terms.Order.term(true));
|
||||
testMinDocCountOnTerms("l", Script.NO, BucketOrder.key(true));
|
||||
}
|
||||
|
||||
public void testLongScriptTermAsc() throws Exception {
|
||||
testMinDocCountOnTerms("l", Script.YES, Terms.Order.term(true));
|
||||
testMinDocCountOnTerms("l", Script.YES, BucketOrder.key(true));
|
||||
}
|
||||
|
||||
public void testLongTermDesc() throws Exception {
|
||||
testMinDocCountOnTerms("l", Script.NO, Terms.Order.term(false));
|
||||
testMinDocCountOnTerms("l", Script.NO, BucketOrder.key(false));
|
||||
}
|
||||
|
||||
public void testLongScriptTermDesc() throws Exception {
|
||||
testMinDocCountOnTerms("l", Script.YES, Terms.Order.term(false));
|
||||
testMinDocCountOnTerms("l", Script.YES, BucketOrder.key(false));
|
||||
}
|
||||
|
||||
public void testLongCountAsc() throws Exception {
|
||||
testMinDocCountOnTerms("l", Script.NO, Terms.Order.count(true));
|
||||
testMinDocCountOnTerms("l", Script.NO, BucketOrder.count(true));
|
||||
}
|
||||
|
||||
public void testLongScriptCountAsc() throws Exception {
|
||||
testMinDocCountOnTerms("l", Script.YES, Terms.Order.count(true));
|
||||
testMinDocCountOnTerms("l", Script.YES, BucketOrder.count(true));
|
||||
}
|
||||
|
||||
public void testLongCountDesc() throws Exception {
|
||||
testMinDocCountOnTerms("l", Script.NO, Terms.Order.count(false));
|
||||
testMinDocCountOnTerms("l", Script.NO, BucketOrder.count(false));
|
||||
}
|
||||
|
||||
public void testLongScriptCountDesc() throws Exception {
|
||||
testMinDocCountOnTerms("l", Script.YES, Terms.Order.count(false));
|
||||
testMinDocCountOnTerms("l", Script.YES, BucketOrder.count(false));
|
||||
}
|
||||
|
||||
public void testDoubleTermAsc() throws Exception {
|
||||
testMinDocCountOnTerms("d", Script.NO, Terms.Order.term(true));
|
||||
testMinDocCountOnTerms("d", Script.NO, BucketOrder.key(true));
|
||||
}
|
||||
|
||||
public void testDoubleScriptTermAsc() throws Exception {
|
||||
testMinDocCountOnTerms("d", Script.YES, Terms.Order.term(true));
|
||||
testMinDocCountOnTerms("d", Script.YES, BucketOrder.key(true));
|
||||
}
|
||||
|
||||
public void testDoubleTermDesc() throws Exception {
|
||||
testMinDocCountOnTerms("d", Script.NO, Terms.Order.term(false));
|
||||
testMinDocCountOnTerms("d", Script.NO, BucketOrder.key(false));
|
||||
}
|
||||
|
||||
public void testDoubleScriptTermDesc() throws Exception {
|
||||
testMinDocCountOnTerms("d", Script.YES, Terms.Order.term(false));
|
||||
testMinDocCountOnTerms("d", Script.YES, BucketOrder.key(false));
|
||||
}
|
||||
|
||||
public void testDoubleCountAsc() throws Exception {
|
||||
testMinDocCountOnTerms("d", Script.NO, Terms.Order.count(true));
|
||||
testMinDocCountOnTerms("d", Script.NO, BucketOrder.count(true));
|
||||
}
|
||||
|
||||
public void testDoubleScriptCountAsc() throws Exception {
|
||||
testMinDocCountOnTerms("d", Script.YES, Terms.Order.count(true));
|
||||
testMinDocCountOnTerms("d", Script.YES, BucketOrder.count(true));
|
||||
}
|
||||
|
||||
public void testDoubleCountDesc() throws Exception {
|
||||
testMinDocCountOnTerms("d", Script.NO, Terms.Order.count(false));
|
||||
testMinDocCountOnTerms("d", Script.NO, BucketOrder.count(false));
|
||||
}
|
||||
|
||||
public void testDoubleScriptCountDesc() throws Exception {
|
||||
testMinDocCountOnTerms("d", Script.YES, Terms.Order.count(false));
|
||||
testMinDocCountOnTerms("d", Script.YES, BucketOrder.count(false));
|
||||
}
|
||||
|
||||
private void testMinDocCountOnTerms(String field, Script script, Terms.Order order) throws Exception {
|
||||
private void testMinDocCountOnTerms(String field, Script script, BucketOrder order) throws Exception {
|
||||
testMinDocCountOnTerms(field, script, order, null, true);
|
||||
}
|
||||
|
||||
private void testMinDocCountOnTerms(String field, Script script, Terms.Order order, String include, boolean retry) throws Exception {
|
||||
private void testMinDocCountOnTerms(String field, Script script, BucketOrder order, String include, boolean retry) throws Exception {
|
||||
// all terms
|
||||
final SearchResponse allTermsResponse = client().prepareSearch("idx").setTypes("type")
|
||||
.setSize(0)
|
||||
|
@ -342,38 +343,38 @@ public class MinDocCountIT extends AbstractTermsTestCase {
|
|||
}
|
||||
|
||||
public void testHistogramCountAsc() throws Exception {
|
||||
testMinDocCountOnHistogram(Histogram.Order.COUNT_ASC);
|
||||
testMinDocCountOnHistogram(BucketOrder.count(true));
|
||||
}
|
||||
|
||||
public void testHistogramCountDesc() throws Exception {
|
||||
testMinDocCountOnHistogram(Histogram.Order.COUNT_DESC);
|
||||
testMinDocCountOnHistogram(BucketOrder.count(false));
|
||||
}
|
||||
|
||||
public void testHistogramKeyAsc() throws Exception {
|
||||
testMinDocCountOnHistogram(Histogram.Order.KEY_ASC);
|
||||
testMinDocCountOnHistogram(BucketOrder.key(true));
|
||||
}
|
||||
|
||||
public void testHistogramKeyDesc() throws Exception {
|
||||
testMinDocCountOnHistogram(Histogram.Order.KEY_DESC);
|
||||
testMinDocCountOnHistogram(BucketOrder.key(false));
|
||||
}
|
||||
|
||||
public void testDateHistogramCountAsc() throws Exception {
|
||||
testMinDocCountOnDateHistogram(Histogram.Order.COUNT_ASC);
|
||||
testMinDocCountOnDateHistogram(BucketOrder.count(true));
|
||||
}
|
||||
|
||||
public void testDateHistogramCountDesc() throws Exception {
|
||||
testMinDocCountOnDateHistogram(Histogram.Order.COUNT_DESC);
|
||||
testMinDocCountOnDateHistogram(BucketOrder.count(false));
|
||||
}
|
||||
|
||||
public void testDateHistogramKeyAsc() throws Exception {
|
||||
testMinDocCountOnDateHistogram(Histogram.Order.KEY_ASC);
|
||||
testMinDocCountOnDateHistogram(BucketOrder.key(true));
|
||||
}
|
||||
|
||||
public void testDateHistogramKeyDesc() throws Exception {
|
||||
testMinDocCountOnDateHistogram(Histogram.Order.KEY_DESC);
|
||||
testMinDocCountOnDateHistogram(BucketOrder.key(false));
|
||||
}
|
||||
|
||||
private void testMinDocCountOnHistogram(Histogram.Order order) throws Exception {
|
||||
private void testMinDocCountOnHistogram(BucketOrder order) throws Exception {
|
||||
final int interval = randomIntBetween(1, 3);
|
||||
final SearchResponse allResponse = client().prepareSearch("idx").setTypes("type")
|
||||
.setSize(0)
|
||||
|
@ -393,7 +394,7 @@ public class MinDocCountIT extends AbstractTermsTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
private void testMinDocCountOnDateHistogram(Histogram.Order order) throws Exception {
|
||||
private void testMinDocCountOnDateHistogram(BucketOrder order) throws Exception {
|
||||
final SearchResponse allResponse = client().prepareSearch("idx").setTypes("type")
|
||||
.setSize(0)
|
||||
.setQuery(QUERY)
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.elasticsearch.search.aggregations.metrics.avg.Avg;
|
|||
import org.elasticsearch.search.aggregations.metrics.avg.AvgAggregationBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStats;
|
||||
import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStatsAggregationBuilder;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSource;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder;
|
||||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
|
@ -150,7 +151,7 @@ public class NaNSortingIT extends ESIntegTestCase {
|
|||
final boolean asc = randomBoolean();
|
||||
SubAggregation agg = randomFrom(SubAggregation.values());
|
||||
SearchResponse response = client().prepareSearch("idx")
|
||||
.addAggregation(terms("terms").field(fieldName).collectMode(randomFrom(SubAggCollectionMode.values())).subAggregation(agg.builder()).order(Terms.Order.aggregation(agg.sortKey(), asc)))
|
||||
.addAggregation(terms("terms").field(fieldName).collectMode(randomFrom(SubAggCollectionMode.values())).subAggregation(agg.builder()).order(BucketOrder.aggregation(agg.sortKey(), asc)))
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -175,7 +176,7 @@ public class NaNSortingIT extends ESIntegTestCase {
|
|||
SubAggregation agg = randomFrom(SubAggregation.values());
|
||||
SearchResponse response = client().prepareSearch("idx")
|
||||
.addAggregation(histogram("histo")
|
||||
.field("long_value").interval(randomIntBetween(1, 2)).subAggregation(agg.builder()).order(Histogram.Order.aggregation(agg.sortKey(), asc)))
|
||||
.field("long_value").interval(randomIntBetween(1, 2)).subAggregation(agg.builder()).order(BucketOrder.aggregation(agg.sortKey(), asc)))
|
||||
.execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.elasticsearch.search.aggregations.bucket.nested.Nested;
|
|||
import org.elasticsearch.search.aggregations.bucket.nested.ReverseNested;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.Terms;
|
||||
import org.elasticsearch.search.aggregations.metrics.valuecount.ValueCount;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
|
||||
import java.util.ArrayList;
|
||||
|
@ -349,13 +350,13 @@ public class ReverseNestedIT extends ESIntegTestCase {
|
|||
SearchResponse response = client().prepareSearch("idx2")
|
||||
.addAggregation(nested("nested1", "nested1.nested2")
|
||||
.subAggregation(
|
||||
terms("field2").field("nested1.nested2.field2").order(Terms.Order.term(true))
|
||||
terms("field2").field("nested1.nested2.field2").order(BucketOrder.key(true))
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
.size(10000)
|
||||
.subAggregation(
|
||||
reverseNested("nested1_to_field1").path("nested1")
|
||||
.subAggregation(
|
||||
terms("field1").field("nested1.field1").order(Terms.Order.term(true))
|
||||
terms("field1").field("nested1.field1").order(BucketOrder.key(true))
|
||||
.collectMode(randomFrom(SubAggCollectionMode.values()))
|
||||
)
|
||||
)
|
||||
|
|
|
@ -28,6 +28,7 @@ import org.elasticsearch.search.aggregations.bucket.sampler.SamplerAggregationBu
|
|||
import org.elasticsearch.search.aggregations.bucket.terms.Terms;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.Terms.Bucket;
|
||||
import org.elasticsearch.search.aggregations.metrics.max.Max;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
|
||||
import java.util.Collection;
|
||||
|
@ -99,7 +100,7 @@ public class SamplerIT extends ESIntegTestCase {
|
|||
SearchResponse response = client().prepareSearch("test").setTypes("book").setSearchType(SearchType.QUERY_THEN_FETCH)
|
||||
.addAggregation(terms("genres")
|
||||
.field("genre")
|
||||
.order(Terms.Order.aggregation("sample>max_price.value", asc))
|
||||
.order(BucketOrder.aggregation("sample>max_price.value", asc))
|
||||
.subAggregation(sampler("sample").shardSize(100)
|
||||
.subAggregation(max("max_price").field("price")))
|
||||
).execute().actionGet();
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue