Merge branch 'master' into enhancement/lucene_5_0_upgrade

Conflicts:
	src/main/java/org/elasticsearch/index/store/DistributorDirectory.java
	src/main/java/org/elasticsearch/index/store/Store.java
	src/main/java/org/elasticsearch/indices/recovery/RecoveryStatus.java
	src/test/java/org/elasticsearch/index/store/DistributorDirectoryTest.java
	src/test/java/org/elasticsearch/index/store/StoreTest.java
	src/test/java/org/elasticsearch/indices/recovery/RecoveryStatusTests.java
This commit is contained in:
Simon Willnauer 2014-11-04 12:54:27 +01:00
commit 93826e4d56
109 changed files with 1355 additions and 742 deletions

View file

@ -24,7 +24,7 @@ time as optimize.
The `upgrade` API accepts the following request parameters:
[horizontal]
`wait_for_completion`:: Should the request wait the upgrade to complete. Defaults
`wait_for_completion`:: Should the request wait for the upgrade to complete. Defaults
to `false`.
[float]

View file

@ -42,6 +42,7 @@ The following snippet captures the basic structure of aggregations:
"<aggregation_type>" : {
<aggregation_body>
}
[,"meta" : { [<meta_data_body>] } ]?
[,"aggregations" : { [<sub_aggregation>]+ } ]?
}
[,"<aggregation_name_2>" : { ... } ]*
@ -148,6 +149,49 @@ $ curl -XGET 'http://localhost:9200/twitter/tweet/_search?search_type=count' -d
Setting `search_type` to `count` avoids executing the fetch phase of the search making the request more efficient. See
<<search-request-search-type>> for more information on the `search_type` parameter.
[float]
=== Metadata
You can associate a piece of metadata with individual aggregations at request time that will be returned in place
at response time.
Consider this example where we want to associate the color blue with our `terms` aggregation.
[source,js]
--------------------------------------------------
{
...
aggs": {
"titles": {
"terms": {
"field": "title"
},
"meta": {
"color": "blue"
},
}
}
}
--------------------------------------------------
Then that piece of metadata will be returned in place for our `titles` terms aggregation
[source,js]
--------------------------------------------------
{
...
"aggregations": {
"titles": {
"meta": {
"color" : "blue"
},
"buckets": [
]
}
}
}
--------------------------------------------------
include::aggregations/metrics.asciidoc[]
include::aggregations/bucket.asciidoc[]

View file

@ -1299,6 +1299,8 @@
<excludes>
<!-- unit tests for yaml suite parser & rest spec parser need to be excluded -->
<exclude>org/elasticsearch/test/rest/test/**/*</exclude>
<!-- unit tests for test framework classes-->
<exclude>org/elasticsearch/test/test/**/*</exclude>
</excludes>
</configuration>
</execution>

View file

@ -19,6 +19,7 @@
package org.elasticsearch;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.rest.RestStatus;
@ -184,4 +185,18 @@ public final class ExceptionsHelper {
)
);
}
/**
* Throws the specified exception. If null if specified then <code>true</code> is returned.
*/
public static boolean reThrowIfNotNull(@Nullable Throwable e) {
if (e != null) {
if (e instanceof RuntimeException) {
throw (RuntimeException) e;
} else {
throw new RuntimeException(e);
}
}
return true;
}
}

View file

@ -231,8 +231,8 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
@Override
protected void doStart() throws ElasticsearchException {
nodesFD.setLocalNode(clusterService.localNode());
joinThreadControl.start();
pingService.start();
// start the join thread from a cluster state update. See {@link JoinThreadControl} for details.
@ -249,7 +249,6 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
logger.warn("failed to start initial join process", t);
}
});
}
@Override
@ -344,8 +343,8 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
}
/**
* returns true if there is a currently a background thread active for (re)joining the cluster
* used for testing.
* returns true if zen discovery is started and there is a currently a background thread active for (re)joining
* the cluster used for testing.
*/
public boolean joiningCluster() {
return joinThreadControl.joinThreadActive();
@ -1278,21 +1277,22 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
private class JoinThreadControl {
private final ThreadPool threadPool;
private final AtomicBoolean running = new AtomicBoolean(false);
private final AtomicReference<Thread> currentJoinThread = new AtomicReference<>();
public JoinThreadControl(ThreadPool threadPool) {
this.threadPool = threadPool;
}
/** returns true if there is currently an active join thread */
/** returns true if join thread control is started and there is currently an active join thread */
public boolean joinThreadActive() {
Thread currentThread = currentJoinThread.get();
return currentThread != null && currentThread.isAlive();
return running.get() && currentThread != null && currentThread.isAlive();
}
/** returns true if the supplied thread is the currently active joinThread */
/** returns true if join thread control is started and the supplied thread is the currently active joinThread */
public boolean joinThreadActive(Thread joinThread) {
return joinThread.equals(currentJoinThread.get());
return running.get() && joinThread.equals(currentJoinThread.get());
}
/** cleans any running joining thread and calls {@link #rejoin} */
@ -1302,7 +1302,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
return rejoin(clusterState, reason);
}
/** starts a new joining thread if there is no currently active one */
/** starts a new joining thread if there is no currently active one and join thread controlling is started */
public void startNewThreadIfNotRunning() {
assertClusterStateThread();
if (joinThreadActive()) {
@ -1315,15 +1315,18 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
if (!currentJoinThread.compareAndSet(null, currentThread)) {
return;
}
while (joinThreadActive(currentThread)) {
while (running.get() && joinThreadActive(currentThread)) {
try {
innerJoinCluster();
return;
} catch (Throwable t) {
logger.error("unexpected error while joining cluster, trying again", t);
} catch (Exception e) {
logger.error("unexpected error while joining cluster, trying again", e);
// Because we catch any exception here, we want to know in
// tests if an uncaught exception got to this point and the test infra uncaught exception
// leak detection can catch this. In practise no uncaught exception should leak
assert ExceptionsHelper.reThrowIfNotNull(e);
}
}
// cleaning the current thread from currentJoinThread is done by explicit calls.
}
});
@ -1348,6 +1351,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
}
public void stop() {
running.set(false);
Thread joinThread = currentJoinThread.getAndSet(null);
if (joinThread != null) {
try {
@ -1355,8 +1359,17 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
} catch (Exception e) {
// ignore
}
try {
joinThread.join(10000);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
}
}
}
public void start() {
running.set(true);
}
private void assertClusterStateThread() {
assert Thread.currentThread().getName().contains(InternalClusterService.UPDATE_THREAD_NAME) : "not called from the cluster state update thread";

View file

@ -185,6 +185,7 @@ public class MulticastZenPing extends AbstractLifecycleComponent<ZenPing> implem
return;
}
final int id = pingIdGenerator.incrementAndGet();
try {
receivedResponses.put(id, new PingCollection());
sendPingRequest(id);
// try and send another ping request halfway through (just in case someone woke up during it...)
@ -229,7 +230,10 @@ public class MulticastZenPing extends AbstractLifecycleComponent<ZenPing> implem
});
}
});
} catch (Exception e) {
logger.warn("failed to ping", e);
finalizePingCycle(id, listener);
}
}
/**

View file

@ -201,6 +201,7 @@ public class UnicastZenPing extends AbstractLifecycleComponent<ZenPing> implemen
@Override
public void ping(final PingListener listener, final TimeValue timeout) throws ElasticsearchException {
final SendPingsHandler sendPingsHandler = new SendPingsHandler(pingHandlerIdGenerator.incrementAndGet());
try {
receivedResponses.put(sendPingsHandler.id(), sendPingsHandler);
try {
sendPings(timeout, null, sendPingsHandler);
@ -239,6 +240,10 @@ public class UnicastZenPing extends AbstractLifecycleComponent<ZenPing> implemen
sendPingsHandler.close();
}
});
} catch (Exception e) {
sendPingsHandler.close();
throw new ElasticsearchException("Ping execution failed", e);
}
}
class SendPingsHandler implements Closeable {

View file

@ -829,7 +829,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
boolean success = false;
RecoveryState.File file = recoveryState.getIndex().file(fileInfo.name());
try (InputStream stream = new PartSliceStream(blobContainer, fileInfo)) {
try (final IndexOutput indexOutput = store.createVerifyingOutput(fileInfo.physicalName(), IOContext.DEFAULT, fileInfo.metadata())) {
try (final IndexOutput indexOutput = store.createVerifyingOutput(fileInfo.physicalName(), fileInfo.metadata(), IOContext.DEFAULT)) {
final byte[] buffer = new byte[BUFFER_SIZE];
int length;
while((length=stream.read(buffer))>0){

View file

@ -18,18 +18,15 @@
*/
package org.elasticsearch.index.store;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.store.*;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ElasticsearchIllegalStateException;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.math.MathUtils;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.index.store.distributor.Distributor;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicInteger;
@ -76,29 +73,22 @@ public final class DistributorDirectory extends BaseDirectory {
this.distributor = distributor;
for (Directory dir : distributor.all()) {
for (String file : dir.listAll()) {
if (!usePrimary(file)) {
nameDirMapping.put(file, dir);
}
}
}
lockFactory = new DistributorLockFactoryWrapper(distributor.primary());
}
@Override
public final String[] listAll() throws IOException {
final ArrayList<String> files = new ArrayList<>();
for (Directory dir : distributor.all()) {
for (String file : dir.listAll()) {
files.add(file);
}
}
return files.toArray(new String[files.size()]);
return nameDirMapping.keySet().toArray(new String[0]);
}
@Override
public void deleteFile(String name) throws IOException {
getDirectory(name, true).deleteFile(name);
Directory remove = nameDirMapping.remove(name);
assert usePrimary(name) || remove != null : "Tried to delete file " + name + " but couldn't";
assert remove != null : "Tried to delete file " + name + " but couldn't";
}
@Override
@ -125,10 +115,6 @@ public final class DistributorDirectory extends BaseDirectory {
throw new IOException("Can't rename file from " + source
+ " to: " + dest + ": target file already exists");
}
if (usePrimary(dest) && directory != distributor.primary()) {
throw new ElasticsearchIllegalStateException("Can not rename: " + source + " to " + dest
+ " destination file must be placed in a primary directory but source directory is not a primary");
}
boolean success = false;
try {
directory.renameFile(source, dest);
@ -160,31 +146,16 @@ public final class DistributorDirectory extends BaseDirectory {
return getDirectory(name, true);
}
/**
* Returns true if the primary directory should be used for the given file.
*/
private boolean usePrimary(String name) {
return Store.isChecksum(name) ||
IndexFileNames.OLD_SEGMENTS_GEN.equals(name) ||
IndexWriter.WRITE_LOCK_NAME.equals(name) ||
name.startsWith(IndexFileNames.SEGMENTS) ||
name.startsWith(IndexFileNames.PENDING_SEGMENTS);
}
/**
* Returns the directory that has previously been associated with this file name or associates the name with a directory
* if failIfNotAssociated is set to false.
*/
private Directory getDirectory(String name, boolean failIfNotAssociated) throws IOException {
if (usePrimary(name)) {
return distributor.primary();
}
Directory directory = nameDirMapping.get(name);
if (directory == null) {
if (failIfNotAssociated) {
throw new FileNotFoundException("No such file [" + name + "]");
}
// Pick a directory and associate this new file with it:
final Directory dir = distributor.any();
directory = nameDirMapping.putIfAbsent(name, dir);
@ -197,24 +168,10 @@ public final class DistributorDirectory extends BaseDirectory {
return directory;
}
@Override
public Lock makeLock(String name) {
return distributor.primary().makeLock(name);
}
@Override
public void clearLock(String name) throws IOException {
distributor.primary().clearLock(name);
}
@Override
public LockFactory getLockFactory() {
return distributor.primary().getLockFactory();
}
@Override
public void setLockFactory(LockFactory lockFactory) throws IOException {
distributor.primary().setLockFactory(lockFactory);
super.setLockFactory(new DistributorLockFactoryWrapper(distributor.primary()));
}
@Override
@ -227,21 +184,118 @@ public final class DistributorDirectory extends BaseDirectory {
return distributor.toString();
}
Distributor getDistributor() {
return distributor;
}
/**
* Creates an IndexOutput for the given temporary file name and ensures that files that need to be placed in a primary
* directory can be successfully renamed.
* @param tempFilename the temporary file name
* @param origFileName the name of the file used to place it in a directory.
* @param context the IOContext used to open the index output
* Basic checks to ensure the internal mapping is consistent - should only be used in assertions
*/
public IndexOutput createTempOutput(String tempFilename, String origFileName, IOContext context) throws IOException {
if (usePrimary(origFileName)) {
Directory directory = nameDirMapping.putIfAbsent(tempFilename, distributor.primary());
if (directory != null && directory != distributor.primary()) {
throw new ElasticsearchIllegalStateException("temporary file [" + tempFilename + "] already exists in a non-primary directory");
static boolean assertConsistency(ESLogger logger, DistributorDirectory dir) throws IOException {
boolean consistent = true;
StringBuilder builder = new StringBuilder();
Directory[] all = dir.distributor.all();
for (Directory d : all) {
for (String file : d.listAll()) {
final Directory directory = dir.nameDirMapping.get(file);
if (directory == null) {
consistent = false;
builder.append("File ").append(file)
.append(" was not mapped to a directory but exists in one of the distributors directories")
.append(System.lineSeparator());
}
if (directory != d) {
consistent = false;
builder.append("File ").append(file).append(" was mapped to a directory ").append(directory)
.append(" but exists in another distributor directory").append(d)
.append(System.lineSeparator());
}
}
}
if (consistent == false) {
logger.info(builder.toString());
}
assert consistent: builder.toString();
return consistent; // return boolean so it can be easily be used in asserts
}
/**
* This inner class is a simple wrapper around the original
* lock factory to track files written / created through the
* lock factory. For instance {@link NativeFSLockFactory} creates real
* files that we should expose for consistency reasons.
*/
private class DistributorLockFactoryWrapper extends LockFactory {
private final Directory dir;
private final LockFactory delegate;
private final boolean writesFiles;
public DistributorLockFactoryWrapper(Directory dir) {
this.dir = dir;
final FSDirectory leaf = DirectoryUtils.getLeaf(dir, FSDirectory.class);
if (leaf != null) {
writesFiles = leaf.getLockFactory() instanceof FSLockFactory;
} else {
writesFiles = false;
}
this.delegate = dir.getLockFactory();
}
@Override
public void setLockPrefix(String lockPrefix) {
delegate.setLockPrefix(lockPrefix);
}
@Override
public String getLockPrefix() {
return delegate.getLockPrefix();
}
@Override
public Lock makeLock(String lockName) {
return new DistributorLock(delegate.makeLock(lockName), lockName);
}
@Override
public void clearLock(String lockName) throws IOException {
delegate.clearLock(lockName);
}
@Override
public String toString() {
return "DistributorLockFactoryWrapper(" + delegate.toString() + ")";
}
private class DistributorLock extends Lock {
private final Lock delegateLock;
private final String name;
DistributorLock(Lock delegate, String name) {
this.delegateLock = delegate;
this.name = name;
}
@Override
public boolean obtain() throws IOException {
if (delegateLock.obtain()) {
if (writesFiles) {
assert (nameDirMapping.containsKey(name) == false || nameDirMapping.get(name) == dir);
nameDirMapping.putIfAbsent(name, dir);
}
return true;
} else {
return false;
}
}
@Override
public void close() throws IOException { delegateLock.close(); }
@Override
public boolean isLocked() throws IOException {
return delegateLock.isLocked();
}
return distributor.primary().createOutput(tempFilename, context);
}
return createOutput(tempFilename, context);
}
}

View file

@ -196,14 +196,6 @@ public class Store extends AbstractIndexShardComponent implements CloseableIndex
}
}
/**
* Creates an IndexOutput for the given temporary file name and ensures that files that need to be placed in a primary
* directory can be successfully renamed.
*/
public IndexOutput createTempOutput(String tempFilename, String origFileName, IOContext context) throws IOException {
return distributorDirectory.createTempOutput(tempFilename, origFileName, context);
}
/**
* Deletes the content of a shard store. Be careful calling this!.
*/
@ -334,26 +326,24 @@ public class Store extends AbstractIndexShardComponent implements CloseableIndex
* Note: Checksums are calculated nevertheless since lucene does it by default sicne version 4.8.0. This method only adds the
* verification against the checksum in the given metadata and does not add any significant overhead.
*/
public IndexOutput createVerifyingOutput(final String filename, final IOContext context, final StoreFileMetaData metadata) throws IOException {
return createVerifyingOutput(directory().createOutput(filename, context), metadata);
}
/**
* The returned IndexOutput might validate the files checksum if the file has been written with a newer lucene version
* and the metadata holds the necessary information to detect that it was been written by Lucene 4.8 or newer. If it has only
* a legacy checksum, returned IndexOutput will not verify the checksum.
*
* Note: Checksums are calculated nevertheless since lucene does it by default sicne version 4.8.0. This method only adds the
* verification against the checksum in the given metadata and does not add any significant overhead.
*/
public IndexOutput createVerifyingOutput(IndexOutput output, final StoreFileMetaData metadata) throws IOException {
public IndexOutput createVerifyingOutput(String fileName, final StoreFileMetaData metadata, final IOContext context) throws IOException {
IndexOutput output = directory().createOutput(fileName, context);
boolean success = false;
try {
if (metadata.hasLegacyChecksum() || metadata.checksum() == null) {
logger.debug("create legacy output for {}", metadata.name());
return output;
}
logger.debug("create legacy output for {}", fileName);
} else {
assert metadata.writtenBy() != null;
assert metadata.writtenBy().onOrAfter(Version.LUCENE_4_8_0);
return new VerifyingIndexOutput(metadata, output);
output = new VerifyingIndexOutput(metadata, output);
}
success = true;
} finally {
if (success == false) {
IOUtils.closeWhileHandlingException(output);
}
}
return output;
}
public static void verify(IndexOutput output) throws IOException {
@ -437,7 +427,7 @@ public class Store extends AbstractIndexShardComponent implements CloseableIndex
/**
* This exists so {@link org.elasticsearch.index.codec.postingsformat.BloomFilterPostingsFormat} can load its boolean setting; can we find a more straightforward way?
*/
public class StoreDirectory extends FilterDirectory {
public final class StoreDirectory extends FilterDirectory {
StoreDirectory(Directory delegateDirectory) throws IOException {
super(delegateDirectory);
@ -481,6 +471,7 @@ public class Store extends AbstractIndexShardComponent implements CloseableIndex
}
private void innerClose() throws IOException {
assert DistributorDirectory.assertConsistency(logger, distributorDirectory);
super.close();
}

View file

@ -252,7 +252,7 @@ public class RecoveryStatus extends AbstractRefCounted {
String tempFileName = getTempNameForFile(fileName);
// add first, before it's created
tempFileNames.add(tempFileName);
IndexOutput indexOutput = store.createVerifyingOutput(store.createTempOutput(tempFileName, fileName, IOContext.DEFAULT), metaData);
IndexOutput indexOutput = store.createVerifyingOutput(tempFileName, metaData, IOContext.DEFAULT);
openIndexOutputs.put(fileName, indexOutput);
return indexOutput;
}

View file

@ -18,6 +18,8 @@
*/
package org.elasticsearch.search.aggregations;
import java.util.Map;
/**
* An aggregation
*/
@ -28,4 +30,8 @@ public interface Aggregation {
*/
String getName();
/**
* Get the optional byte array metadata that was set on the aggregation
*/
Map<String, Object> getMetaData();
}

View file

@ -38,6 +38,7 @@ public abstract class AggregationBuilder<B extends AggregationBuilder<B>> extend
private List<AbstractAggregationBuilder> aggregations;
private BytesReference aggregationsBinary;
private Map<String, Object> metaData;
/**
* Sole constructor, typically used by sub-classes.
@ -101,10 +102,21 @@ public abstract class AggregationBuilder<B extends AggregationBuilder<B>> extend
}
}
/**
* Sets the meta data to be included in the aggregation response
*/
public B setMetaData(Map<String, Object> metaData) {
this.metaData = metaData;
return (B)this;
}
@Override
public final XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(getName());
if (this.metaData != null) {
builder.field("meta", this.metaData);
}
builder.field(type);
internalXContent(builder, params);

View file

@ -44,6 +44,7 @@ public abstract class Aggregator extends BucketCollector implements Releasable {
return aggregator.shouldCollect();
}
};
private final Map<String, Object> metaData;
/**
* Returns whether any of the parent aggregators has {@link BucketAggregationMode#PER_BUCKET} as a bucket aggregation mode.
@ -60,6 +61,10 @@ public abstract class Aggregator extends BucketCollector implements Releasable {
public static final ParseField COLLECT_MODE = new ParseField("collect_mode");
public Map<String, Object> getMetaData() {
return this.metaData;
}
/**
* Defines the nature of the aggregator's aggregation execution when nested in other aggregators and the buckets they create.
*/
@ -177,9 +182,11 @@ public abstract class Aggregator extends BucketCollector implements Releasable {
* @param estimatedBucketsCount When served as a sub-aggregator, indicate how many buckets the parent aggregator will generate.
* @param context The aggregation context
* @param parent The parent aggregator (may be {@code null} for top level aggregators)
* @param metaData The metaData associated with this aggregator
*/
protected Aggregator(String name, BucketAggregationMode bucketAggregationMode, AggregatorFactories factories, long estimatedBucketsCount, AggregationContext context, Aggregator parent) {
protected Aggregator(String name, BucketAggregationMode bucketAggregationMode, AggregatorFactories factories, long estimatedBucketsCount, AggregationContext context, Aggregator parent, Map<String, Object> metaData) {
this.name = name;
this.metaData = metaData;
this.parent = parent;
this.estimatedBucketCount = estimatedBucketsCount;
this.context = context;
@ -217,6 +224,7 @@ public abstract class Aggregator extends BucketCollector implements Releasable {
}
};
}
protected void preCollection() {
Iterable<Aggregator> collectables = Iterables.filter(Arrays.asList(subAggregators), COLLECTABLE_AGGREGATOR);
List<BucketCollector> nextPassCollectors = new ArrayList<>();
@ -362,9 +370,6 @@ public abstract class Aggregator extends BucketCollector implements Releasable {
results.add(buildAggregation(bucketOrdinal));
}
public abstract InternalAggregation buildEmptyAggregation();
protected final InternalAggregations buildEmptySubAggregations() {

View file

@ -75,7 +75,7 @@ public class AggregatorFactories {
continue;
}
// the aggregator doesn't support multiple ordinals, let's wrap it so that it does.
aggregators[i] = new Aggregator(first.name(), BucketAggregationMode.MULTI_BUCKETS, AggregatorFactories.EMPTY, 1, first.context(), first.parent()) {
aggregators[i] = new Aggregator(first.name(), BucketAggregationMode.MULTI_BUCKETS, AggregatorFactories.EMPTY, 1, first.context(), first.parent(), first.getMetaData()) {
ObjectArray<Aggregator> aggregators;

View file

@ -20,6 +20,8 @@ package org.elasticsearch.search.aggregations;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import java.util.Map;
/**
* A factory that knows how to create an {@link Aggregator} of a specific type.
*/
@ -29,6 +31,7 @@ public abstract class AggregatorFactory {
protected String type;
protected AggregatorFactory parent;
protected AggregatorFactories factories = AggregatorFactories.EMPTY;
protected Map<String, Object> metaData;
/**
* Constructs a new aggregator factory.
@ -79,9 +82,17 @@ public abstract class AggregatorFactory {
*
* @return The created aggregator
*/
public abstract Aggregator create(AggregationContext context, Aggregator parent, long expectedBucketsCount);
protected abstract Aggregator createInternal(AggregationContext context, Aggregator parent, long expectedBucketsCount, Map<String, Object> metaData);
public Aggregator create(AggregationContext context, Aggregator parent, long expectedBucketsCount) {
Aggregator aggregator = createInternal(context, parent, expectedBucketsCount, this.metaData);
return aggregator;
}
public void doValidate() {
}
public void setMetaData(Map<String, Object> metaData) {
this.metaData = metaData;
}
}

View file

@ -26,6 +26,7 @@ import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Map;
import java.util.Set;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
@ -100,6 +101,8 @@ public class AggregatorParsers {
AggregatorFactory factory = null;
AggregatorFactories subFactories = null;
Map<String, Object> metaData = null;
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token != XContentParser.Token.FIELD_NAME) {
throw new SearchParseException(context, "Expected [" + XContentParser.Token.FIELD_NAME + "] under a [" + XContentParser.Token.START_OBJECT + "], but got a [" + token + "] in [" + aggregationName + "]");
@ -112,6 +115,9 @@ public class AggregatorParsers {
}
switch (fieldName) {
case "meta":
metaData = parser.map();
break;
case "aggregations":
case "aggs":
if (subFactories != null) {
@ -135,6 +141,10 @@ public class AggregatorParsers {
throw new SearchParseException(context, "Missing definition for aggregation [" + aggregationName + "]");
}
if (metaData != null) {
factory.setMetaData(metaData);
}
if (subFactories != null) {
factory.subFactories(subFactories);
}

View file

@ -18,6 +18,7 @@
*/
package org.elasticsearch.search.aggregations;
import org.elasticsearch.Version;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.StreamInput;
@ -31,12 +32,14 @@ import org.elasticsearch.script.ScriptService;
import java.io.IOException;
import java.util.List;
import java.util.Map;
/**
* An internal implementation of {@link Aggregation}. Serves as a base class for all aggregation implementations.
*/
public abstract class InternalAggregation implements Aggregation, ToXContent, Streamable {
/**
* The aggregation type that holds all the string types that are associated with an aggregation:
* <ul>
@ -111,6 +114,8 @@ public abstract class InternalAggregation implements Aggregation, ToXContent, St
protected String name;
protected Map<String, Object> metaData;
/** Constructs an un initialized addAggregation (used for serialization) **/
protected InternalAggregation() {}
@ -119,8 +124,9 @@ public abstract class InternalAggregation implements Aggregation, ToXContent, St
*
* @param name The name of the get.
*/
protected InternalAggregation(String name) {
protected InternalAggregation(String name, Map<String, Object> metaData) {
this.name = name;
this.metaData = metaData;
}
@Override
@ -159,9 +165,17 @@ public abstract class InternalAggregation implements Aggregation, ToXContent, St
out.writeVInt(size);
}
public Map<String, Object> getMetaData() {
return metaData;
}
@Override
public final XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(name);
if (this.metaData != null) {
builder.field(CommonFields.META);
builder.map(this.metaData);
}
doXContentBody(builder, params);
builder.endObject();
return builder;
@ -169,10 +183,31 @@ public abstract class InternalAggregation implements Aggregation, ToXContent, St
public abstract XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException;
public final void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
if (out.getVersion().onOrAfter(Version.V_1_5_0)) {
out.writeGenericValue(metaData);
}
doWriteTo(out);
}
protected abstract void doWriteTo(StreamOutput out) throws IOException;
public final void readFrom(StreamInput in) throws IOException {
name = in.readString();
if (in.getVersion().onOrAfter(Version.V_1_5_0)) {
metaData = in.readMap();
}
doReadFrom(in);
}
protected abstract void doReadFrom(StreamInput in) throws IOException;
/**
* Common xcontent fields that are shared among addAggregation
*/
public static final class CommonFields {
public static final XContentBuilderString META = new XContentBuilderString("meta");
public static final XContentBuilderString BUCKETS = new XContentBuilderString("buckets");
public static final XContentBuilderString VALUE = new XContentBuilderString("value");
public static final XContentBuilderString VALUES = new XContentBuilderString("values");

View file

@ -23,6 +23,7 @@ import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import java.io.IOException;
import java.util.Map;
/**
* An aggregator that is not collected, this can typically be used when running an aggregation over a field that doesn't have
@ -30,8 +31,8 @@ import java.io.IOException;
*/
public abstract class NonCollectingAggregator extends Aggregator {
protected NonCollectingAggregator(String name, AggregationContext context, Aggregator parent) {
super(name, BucketAggregationMode.MULTI_BUCKETS, AggregatorFactories.EMPTY, 0, context, parent);
protected NonCollectingAggregator(String name, AggregationContext context, Aggregator parent, Map<String, Object> metaData) {
super(name, BucketAggregationMode.MULTI_BUCKETS, AggregatorFactories.EMPTY, 0, context, parent, metaData);
}
private void fail() {

View file

@ -26,6 +26,7 @@ import org.elasticsearch.search.aggregations.support.AggregationContext;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Map;
/**
*
@ -35,8 +36,8 @@ public abstract class BucketsAggregator extends Aggregator {
private IntArray docCounts;
public BucketsAggregator(String name, BucketAggregationMode bucketAggregationMode, AggregatorFactories factories,
long estimatedBucketsCount, AggregationContext context, Aggregator parent) {
super(name, bucketAggregationMode, factories, estimatedBucketsCount, context, parent);
long estimatedBucketsCount, AggregationContext context, Aggregator parent, Map<String, Object> metaData) {
super(name, bucketAggregationMode, factories, estimatedBucketsCount, context, parent, metaData);
docCounts = bigArrays.newIntArray(estimatedBucketsCount, true);
}

View file

@ -18,6 +18,7 @@
*/
package org.elasticsearch.search.aggregations.bucket;
import org.elasticsearch.common.collect.HppcMaps;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -27,6 +28,7 @@ import org.elasticsearch.search.aggregations.InternalAggregations;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
/**
* A base class for all the single bucket aggregations.
@ -45,8 +47,8 @@ public abstract class InternalSingleBucketAggregation extends InternalAggregatio
* @param docCount The document count in the single bucket.
* @param aggregations The already built sub-aggregations that are associated with the bucket.
*/
protected InternalSingleBucketAggregation(String name, long docCount, InternalAggregations aggregations) {
super(name);
protected InternalSingleBucketAggregation(String name, long docCount, InternalAggregations aggregations, Map<String, Object> metaData) {
super(name, metaData);
this.docCount = docCount;
this.aggregations = aggregations;
}
@ -81,15 +83,13 @@ public abstract class InternalSingleBucketAggregation extends InternalAggregatio
}
@Override
public void readFrom(StreamInput in) throws IOException {
name = in.readString();
protected void doReadFrom(StreamInput in) throws IOException {
docCount = in.readVLong();
aggregations = InternalAggregations.readAggregations(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
protected void doWriteTo(StreamOutput out) throws IOException {
out.writeVLong(docCount);
aggregations.writeTo(out);
}

View file

@ -22,14 +22,16 @@ import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import java.util.Map;
/**
* A bucket aggregator that doesn't create new buckets.
*/
public abstract class SingleBucketAggregator extends BucketsAggregator {
protected SingleBucketAggregator(String name, AggregatorFactories factories,
AggregationContext aggregationContext, Aggregator parent) {
super(name, BucketAggregationMode.MULTI_BUCKETS, factories, parent == null ? 1 : parent.estimatedBucketCount(), aggregationContext, parent);
AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
super(name, BucketAggregationMode.MULTI_BUCKETS, factories, parent == null ? 1 : parent.estimatedBucketCount(), aggregationContext, parent, metaData);
}
@Override

View file

@ -25,6 +25,7 @@ import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.bucket.InternalSingleBucketAggregation;
import java.io.IOException;
import java.util.Map;
/**
*/
@ -48,8 +49,8 @@ public class InternalChildren extends InternalSingleBucketAggregation implements
public InternalChildren() {
}
public InternalChildren(String name, long docCount, InternalAggregations aggregations) {
super(name, docCount, aggregations);
public InternalChildren(String name, long docCount, InternalAggregations aggregations, Map<String, Object> metaData) {
super(name, docCount, aggregations, metaData);
}
@Override
@ -59,6 +60,6 @@ public class InternalChildren extends InternalSingleBucketAggregation implements
@Override
protected InternalSingleBucketAggregation newAggregation(String name, long docCount, InternalAggregations subAggregations) {
return new InternalChildren(name, docCount, subAggregations);
return new InternalChildren(name, docCount, subAggregations, getMetaData());
}
}

View file

@ -46,6 +46,7 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
// The RecordingPerReaderBucketCollector assumes per segment recording which isn't the case for this
// aggregation, for this reason that collector can't be used
@ -72,8 +73,8 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator implement
public ParentToChildrenAggregator(String name, AggregatorFactories factories, AggregationContext aggregationContext,
Aggregator parent, String parentType, Filter childFilter, Filter parentFilter,
ValuesSource.Bytes.WithOrdinals.ParentChild valuesSource, long maxOrd) {
super(name, factories, aggregationContext, parent);
ValuesSource.Bytes.WithOrdinals.ParentChild valuesSource, long maxOrd, Map<String, Object> metaData) {
super(name, factories, aggregationContext, parent, metaData);
this.parentType = parentType;
// The child filter doesn't rely on random access it just used to iterate over all docs with a specific type,
// so use the filter cache instead. When the filter cache is smarter with what filter impl to pick we can benefit
@ -88,12 +89,12 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator implement
@Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
return new InternalChildren(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal));
return new InternalChildren(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), getMetaData());
}
@Override
public InternalAggregation buildEmptyAggregation() {
return new InternalChildren(name, 0, buildEmptySubAggregations());
return new InternalChildren(name, 0, buildEmptySubAggregations(), getMetaData());
}
@Override
@ -188,7 +189,7 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator implement
Releasables.close(parentOrdToBuckets, parentOrdToOtherBuckets);
}
public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource.Bytes.WithOrdinals.ParentChild> {
public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource.Bytes.WithOrdinals.ParentChild, Map<String, Object>> {
private final String parentType;
private final Filter parentFilter;
@ -202,14 +203,14 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator implement
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent) {
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
throw new ElasticsearchIllegalStateException("[children] aggregation doesn't support unmapped");
}
@Override
protected Aggregator create(ValuesSource.Bytes.WithOrdinals.ParentChild valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent) {
protected Aggregator create(ValuesSource.Bytes.WithOrdinals.ParentChild valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
long maxOrd = valuesSource.globalMaxOrd(aggregationContext.searchContext().searcher(), parentType);
return new ParentToChildrenAggregator(name, factories, aggregationContext, parent, parentType, childFilter, parentFilter, valuesSource, maxOrd);
return new ParentToChildrenAggregator(name, factories, aggregationContext, parent, parentType, childFilter, parentFilter, valuesSource, maxOrd, metaData);
}
}

View file

@ -27,6 +27,7 @@ import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import java.io.IOException;
import java.util.Map;
/**
* Aggregate all docs that match a filter.
@ -41,8 +42,9 @@ public class FilterAggregator extends SingleBucketAggregator {
org.apache.lucene.search.Filter filter,
AggregatorFactories factories,
AggregationContext aggregationContext,
Aggregator parent) {
super(name, factories, aggregationContext, parent);
Aggregator parent,
Map<String, Object> metaData) {
super(name, factories, aggregationContext, parent, metaData);
this.filter = filter;
}
@ -64,12 +66,12 @@ public class FilterAggregator extends SingleBucketAggregator {
@Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
return new InternalFilter(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal));
return new InternalFilter(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), getMetaData());
}
@Override
public InternalAggregation buildEmptyAggregation() {
return new InternalFilter(name, 0, buildEmptySubAggregations());
return new InternalFilter(name, 0, buildEmptySubAggregations(), getMetaData());
}
public static class Factory extends AggregatorFactory {
@ -82,8 +84,8 @@ public class FilterAggregator extends SingleBucketAggregator {
}
@Override
public Aggregator create(AggregationContext context, Aggregator parent, long expectedBucketsCount) {
return new FilterAggregator(name, filter, factories, context, parent);
public Aggregator createInternal(AggregationContext context, Aggregator parent, long expectedBucketsCount, Map<String, Object> metaData) {
return new FilterAggregator(name, filter, factories, context, parent, metaData);
}
}

View file

@ -24,6 +24,7 @@ import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.bucket.InternalSingleBucketAggregation;
import java.io.IOException;
import java.util.Map;
/**
*
@ -47,8 +48,8 @@ public class InternalFilter extends InternalSingleBucketAggregation implements F
InternalFilter() {} // for serialization
InternalFilter(String name, long docCount, InternalAggregations subAggregations) {
super(name, docCount, subAggregations);
InternalFilter(String name, long docCount, InternalAggregations subAggregations, Map<String, Object> metaData) {
super(name, docCount, subAggregations, metaData);
}
@Override
@ -58,6 +59,6 @@ public class InternalFilter extends InternalSingleBucketAggregation implements F
@Override
protected InternalSingleBucketAggregation newAggregation(String name, long docCount, InternalAggregations subAggregations) {
return new InternalFilter(name, docCount, subAggregations);
return new InternalFilter(name, docCount, subAggregations, getMetaData());
}
}

View file

@ -30,6 +30,7 @@ import org.elasticsearch.search.aggregations.support.AggregationContext;
import java.io.IOException;
import java.util.List;
import java.util.Map;
/**
*
@ -52,9 +53,9 @@ public class FiltersAggregator extends BucketsAggregator {
private boolean keyed;
public FiltersAggregator(String name, AggregatorFactories factories, List<KeyedFilter> filters, boolean keyed, AggregationContext aggregationContext,
Aggregator parent) {
Aggregator parent, Map<String, Object> metaData) {
super(name, BucketAggregationMode.MULTI_BUCKETS, factories, filters.size() * (parent == null ? 1 : parent.estimatedBucketCount()),
aggregationContext, parent);
aggregationContext, parent, metaData);
this.keyed = keyed;
this.filters = filters.toArray(new KeyedFilter[filters.size()]);
this.bits = new Bits[this.filters.length];
@ -94,7 +95,7 @@ public class FiltersAggregator extends BucketsAggregator {
InternalFilters.Bucket bucket = new InternalFilters.Bucket(filter.key, bucketDocCount(bucketOrd), bucketAggregations(bucketOrd), keyed);
buckets.add(bucket);
}
return new InternalFilters(name, buckets, keyed);
return new InternalFilters(name, buckets, keyed, getMetaData());
}
@Override
@ -105,7 +106,7 @@ public class FiltersAggregator extends BucketsAggregator {
InternalFilters.Bucket bucket = new InternalFilters.Bucket(filters[i].key, 0, subAggs, keyed);
buckets.add(bucket);
}
return new InternalFilters(name, buckets, keyed);
return new InternalFilters(name, buckets, keyed, getMetaData());
}
private final long bucketOrd(long owningBucketOrdinal, int filterOrd) {
@ -124,8 +125,8 @@ public class FiltersAggregator extends BucketsAggregator {
}
@Override
public Aggregator create(AggregationContext context, Aggregator parent, long expectedBucketsCount) {
return new FiltersAggregator(name, factories, filters, keyed, context, parent);
public Aggregator createInternal(AggregationContext context, Aggregator parent, long expectedBucketsCount, Map<String, Object> metaData) {
return new FiltersAggregator(name, factories, filters, keyed, context, parent, metaData);
}
}

View file

@ -162,8 +162,8 @@ public class InternalFilters extends InternalAggregation implements Filters {
public InternalFilters() {} // for serialization
public InternalFilters(String name, List<Bucket> buckets, boolean keyed) {
super(name);
public InternalFilters(String name, List<Bucket> buckets, boolean keyed, Map<String, Object> metaData) {
super(name, metaData);
this.buckets = buckets;
this.keyed = keyed;
}
@ -210,7 +210,7 @@ public class InternalFilters extends InternalAggregation implements Filters {
}
}
InternalFilters reduced = new InternalFilters(name, new ArrayList<Bucket>(bucketsList.size()), keyed);
InternalFilters reduced = new InternalFilters(name, new ArrayList<Bucket>(bucketsList.size()), keyed, getMetaData());
for (List<Bucket> sameRangeList : bucketsList) {
reduced.buckets.add((sameRangeList.get(0)).reduce(sameRangeList, reduceContext));
}
@ -218,8 +218,7 @@ public class InternalFilters extends InternalAggregation implements Filters {
}
@Override
public void readFrom(StreamInput in) throws IOException {
name = in.readString();
protected void doReadFrom(StreamInput in) throws IOException {
keyed = in.readBoolean();
int size = in.readVInt();
List<Bucket> buckets = Lists.newArrayListWithCapacity(size);
@ -233,8 +232,7 @@ public class InternalFilters extends InternalAggregation implements Filters {
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
protected void doWriteTo(StreamOutput out) throws IOException {
out.writeBoolean(keyed);
out.writeVInt(buckets.size());
for (Bucket bucket : buckets) {

View file

@ -32,6 +32,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSource;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.Map;
/**
* Aggregates data expressed as GeoHash longs (for efficiency's sake) but formats results as Geohash strings.
@ -49,8 +50,8 @@ public class GeoHashGridAggregator extends BucketsAggregator {
private SortedNumericDocValues values;
public GeoHashGridAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource,
int requiredSize, int shardSize, AggregationContext aggregationContext, Aggregator parent) {
super(name, BucketAggregationMode.PER_BUCKET, factories, INITIAL_CAPACITY, aggregationContext, parent);
int requiredSize, int shardSize, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
super(name, BucketAggregationMode.PER_BUCKET, factories, INITIAL_CAPACITY, aggregationContext, parent, metaData);
this.valuesSource = valuesSource;
this.requiredSize = requiredSize;
this.shardSize = shardSize;
@ -120,12 +121,12 @@ public class GeoHashGridAggregator extends BucketsAggregator {
bucket.aggregations = bucketAggregations(bucket.bucketOrd);
list[i] = bucket;
}
return new InternalGeoHashGrid(name, requiredSize, Arrays.asList(list));
return new InternalGeoHashGrid(name, requiredSize, Arrays.asList(list), getMetaData());
}
@Override
public InternalGeoHashGrid buildEmptyAggregation() {
return new InternalGeoHashGrid(name, requiredSize, Collections.<InternalGeoHashGrid.Bucket>emptyList());
return new InternalGeoHashGrid(name, requiredSize, Collections.<InternalGeoHashGrid.Bucket>emptyList(), getMetaData());
}

View file

@ -37,6 +37,7 @@ import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Collections;
import java.util.Map;
/**
* Aggregates Geo information into cells determined by geohashes of a given precision.
@ -102,7 +103,7 @@ public class GeoHashGridParser implements Aggregator.Parser {
}
private static class GeoGridFactory extends ValuesSourceAggregatorFactory<ValuesSource.GeoPoint> {
private static class GeoGridFactory extends ValuesSourceAggregatorFactory<ValuesSource.GeoPoint, Map<String, Object>> {
private int precision;
private int requiredSize;
@ -116,9 +117,9 @@ public class GeoHashGridParser implements Aggregator.Parser {
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent) {
final InternalAggregation aggregation = new InternalGeoHashGrid(name, requiredSize, Collections.<InternalGeoHashGrid.Bucket>emptyList());
return new NonCollectingAggregator(name, aggregationContext, parent) {
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
final InternalAggregation aggregation = new InternalGeoHashGrid(name, requiredSize, Collections.<InternalGeoHashGrid.Bucket>emptyList(), metaData);
return new NonCollectingAggregator(name, aggregationContext, parent, metaData) {
public InternalAggregation buildEmptyAggregation() {
return aggregation;
}
@ -126,10 +127,10 @@ public class GeoHashGridParser implements Aggregator.Parser {
}
@Override
protected Aggregator create(final ValuesSource.GeoPoint valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent) {
protected Aggregator create(final ValuesSource.GeoPoint valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
final CellValues cellIdValues = new CellValues(valuesSource, precision);
ValuesSource.Numeric cellIdSource = new CellIdSource(cellIdValues, valuesSource.metaData());
return new GeoHashGridAggregator(name, factories, cellIdSource, requiredSize, shardSize, aggregationContext, parent);
return new GeoHashGridAggregator(name, factories, cellIdSource, requiredSize, shardSize, aggregationContext, parent, metaData);
}

View file

@ -175,8 +175,8 @@ public class InternalGeoHashGrid extends InternalAggregation implements GeoHashG
InternalGeoHashGrid() {
} // for serialization
public InternalGeoHashGrid(String name, int requiredSize, Collection<Bucket> buckets) {
super(name);
public InternalGeoHashGrid(String name, int requiredSize, Collection<Bucket> buckets, Map<String, Object> metaData) {
super(name, metaData);
this.requiredSize = requiredSize;
this.buckets = buckets;
}
@ -244,12 +244,11 @@ public class InternalGeoHashGrid extends InternalAggregation implements GeoHashG
for (int i = ordered.size() - 1; i >= 0; i--) {
list[i] = ordered.pop();
}
return new InternalGeoHashGrid(getName(), requiredSize, Arrays.asList(list));
return new InternalGeoHashGrid(getName(), requiredSize, Arrays.asList(list), getMetaData());
}
@Override
public void readFrom(StreamInput in) throws IOException {
this.name = in.readString();
protected void doReadFrom(StreamInput in) throws IOException {
this.requiredSize = readSize(in);
int size = in.readVInt();
List<Bucket> buckets = new ArrayList<>(size);
@ -263,8 +262,7 @@ public class InternalGeoHashGrid extends InternalAggregation implements GeoHashG
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
protected void doWriteTo(StreamOutput out) throws IOException {
writeSize(requiredSize, out);
out.writeVInt(buckets.size());
for (Bucket bucket : buckets) {

View file

@ -24,14 +24,15 @@ import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import java.io.IOException;
import java.util.Map;
/**
*
*/
public class GlobalAggregator extends SingleBucketAggregator {
public GlobalAggregator(String name, AggregatorFactories subFactories, AggregationContext aggregationContext) {
super(name, subFactories, aggregationContext, null);
public GlobalAggregator(String name, AggregatorFactories subFactories, AggregationContext aggregationContext, Map<String, Object> metaData) {
super(name, subFactories, aggregationContext, null, metaData);
}
@Override
@ -47,7 +48,7 @@ public class GlobalAggregator extends SingleBucketAggregator {
@Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
assert owningBucketOrdinal == 0 : "global aggregator can only be a top level aggregator";
return new InternalGlobal(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal));
return new InternalGlobal(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), getMetaData());
}
@Override
@ -62,12 +63,12 @@ public class GlobalAggregator extends SingleBucketAggregator {
}
@Override
public Aggregator create(AggregationContext context, Aggregator parent, long expectedBucketsCount) {
public Aggregator createInternal(AggregationContext context, Aggregator parent, long expectedBucketsCount, Map<String, Object> metaData) {
if (parent != null) {
throw new AggregationExecutionException("Aggregation [" + parent.name() + "] cannot have a global " +
"sub-aggregation [" + name + "]. Global aggregations can only be defined as top level aggregations");
}
return new GlobalAggregator(name, factories, context);
return new GlobalAggregator(name, factories, context, metaData);
}
}

View file

@ -24,6 +24,7 @@ import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.bucket.InternalSingleBucketAggregation;
import java.io.IOException;
import java.util.Map;
/**
* A global scope get (the document set on which we aggregate is all documents in the search context (ie. index + type)
@ -48,8 +49,8 @@ public class InternalGlobal extends InternalSingleBucketAggregation implements G
InternalGlobal() {} // for serialization
InternalGlobal(String name, long docCount, InternalAggregations aggregations) {
super(name, docCount, aggregations);
InternalGlobal(String name, long docCount, InternalAggregations aggregations, Map<String, Object> metaData) {
super(name, docCount, aggregations, metaData);
}
@Override
@ -59,6 +60,6 @@ public class InternalGlobal extends InternalSingleBucketAggregation implements G
@Override
protected InternalSingleBucketAggregation newAggregation(String name, long docCount, InternalAggregations subAggregations) {
return new InternalGlobal(name, docCount, subAggregations);
return new InternalGlobal(name, docCount, subAggregations, getMetaData());
}
}

View file

@ -39,6 +39,7 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
public class HistogramAggregator extends BucketsAggregator {
@ -59,9 +60,9 @@ public class HistogramAggregator extends BucketsAggregator {
boolean keyed, long minDocCount, @Nullable ExtendedBounds extendedBounds,
@Nullable ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter,
long initialCapacity, InternalHistogram.Factory<?> histogramFactory,
AggregationContext aggregationContext, Aggregator parent) {
AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
super(name, BucketAggregationMode.PER_BUCKET, factories, initialCapacity, aggregationContext, parent);
super(name, BucketAggregationMode.PER_BUCKET, factories, initialCapacity, aggregationContext, parent, metaData);
this.rounding = rounding;
this.order = order;
this.keyed = keyed;
@ -121,13 +122,13 @@ public class HistogramAggregator extends BucketsAggregator {
// value source will be null for unmapped fields
InternalHistogram.EmptyBucketInfo emptyBucketInfo = minDocCount == 0 ? new InternalHistogram.EmptyBucketInfo(rounding, buildEmptySubAggregations(), extendedBounds) : null;
return histogramFactory.create(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed);
return histogramFactory.create(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed, getMetaData());
}
@Override
public InternalAggregation buildEmptyAggregation() {
InternalHistogram.EmptyBucketInfo emptyBucketInfo = minDocCount == 0 ? new InternalHistogram.EmptyBucketInfo(rounding, buildEmptySubAggregations(), extendedBounds) : null;
return histogramFactory.create(name, Collections.emptyList(), order, minDocCount, emptyBucketInfo, formatter, keyed);
return histogramFactory.create(name, Collections.emptyList(), order, minDocCount, emptyBucketInfo, formatter, keyed, getMetaData());
}
@Override
@ -135,7 +136,7 @@ public class HistogramAggregator extends BucketsAggregator {
Releasables.close(bucketOrds);
}
public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource.Numeric> {
public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource.Numeric, Map<String, Object>> {
private final Rounding rounding;
private final InternalOrder order;
@ -158,12 +159,12 @@ public class HistogramAggregator extends BucketsAggregator {
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent) {
return new HistogramAggregator(name, factories, rounding, order, keyed, minDocCount, null, null, config.formatter(), 0, histogramFactory, aggregationContext, parent);
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
return new HistogramAggregator(name, factories, rounding, order, keyed, minDocCount, null, null, config.formatter(), 0, histogramFactory, aggregationContext, parent, metaData);
}
@Override
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent) {
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
// todo if we'll keep track of min/max values in IndexFieldData, we could use the max here to come up with a better estimation for the buckets count
long estimatedBucketCount = 50;
if (hasParentBucketAggregator(parent)) {
@ -179,7 +180,7 @@ public class HistogramAggregator extends BucketsAggregator {
extendedBounds.processAndValidate(name, aggregationContext.searchContext(), config.parser());
roundedBounds = extendedBounds.round(rounding);
}
return new HistogramAggregator(name, factories, rounding, order, keyed, minDocCount, roundedBounds, valuesSource, config.formatter(), estimatedBucketCount, histogramFactory, aggregationContext, parent);
return new HistogramAggregator(name, factories, rounding, order, keyed, minDocCount, roundedBounds, valuesSource, config.formatter(), estimatedBucketCount, histogramFactory, aggregationContext, parent, metaData);
}
}

View file

@ -31,6 +31,7 @@ import org.joda.time.DateTimeZone;
import java.io.IOException;
import java.util.List;
import java.util.Map;
/**
*
@ -113,8 +114,8 @@ public class InternalDateHistogram extends InternalHistogram<InternalDateHistogr
@Override
public InternalDateHistogram create(String name, List<InternalDateHistogram.Bucket> buckets, InternalOrder order,
long minDocCount, EmptyBucketInfo emptyBucketInfo, @Nullable ValueFormatter formatter, boolean keyed) {
return new InternalDateHistogram(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed);
long minDocCount, EmptyBucketInfo emptyBucketInfo, @Nullable ValueFormatter formatter, boolean keyed, Map<String, Object> metaData) {
return new InternalDateHistogram(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed, metaData);
}
@Override
@ -128,8 +129,8 @@ public class InternalDateHistogram extends InternalHistogram<InternalDateHistogr
InternalDateHistogram() {} // for serialization
InternalDateHistogram(String name, List<InternalDateHistogram.Bucket> buckets, InternalOrder order, long minDocCount,
EmptyBucketInfo emptyBucketInfo, @Nullable ValueFormatter formatter, boolean keyed) {
super(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed);
EmptyBucketInfo emptyBucketInfo, @Nullable ValueFormatter formatter, boolean keyed, Map<String, Object> metaData) {
super(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed, metaData);
}
@Override
@ -175,8 +176,8 @@ public class InternalDateHistogram extends InternalHistogram<InternalDateHistogr
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
protected void doReadFrom(StreamInput in) throws IOException {
super.doReadFrom(in);
bucketsMap = null; // we need to reset this on read (as it's lazily created on demand)
}

View file

@ -43,6 +43,7 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.ListIterator;
import java.util.Map;
/**
* TODO should be renamed to InternalNumericHistogram (see comment on {@link Histogram})?
@ -233,8 +234,8 @@ public class InternalHistogram<B extends InternalHistogram.Bucket> extends Inter
}
public InternalHistogram<B> create(String name, List<B> buckets, InternalOrder order, long minDocCount,
EmptyBucketInfo emptyBucketInfo, @Nullable ValueFormatter formatter, boolean keyed) {
return new InternalHistogram<>(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed);
EmptyBucketInfo emptyBucketInfo, @Nullable ValueFormatter formatter, boolean keyed, Map<String, Object> metaData) {
return new InternalHistogram<>(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed, metaData);
}
public B createBucket(long key, long docCount, InternalAggregations aggregations, boolean keyed, @Nullable ValueFormatter formatter) {
@ -254,8 +255,8 @@ public class InternalHistogram<B extends InternalHistogram.Bucket> extends Inter
InternalHistogram() {} // for serialization
InternalHistogram(String name, List<B> buckets, InternalOrder order, long minDocCount,
EmptyBucketInfo emptyBucketInfo, @Nullable ValueFormatter formatter, boolean keyed) {
super(name);
EmptyBucketInfo emptyBucketInfo, @Nullable ValueFormatter formatter, boolean keyed, Map<String, Object> metaData) {
super(name, metaData);
this.buckets = buckets;
this.order = order;
assert (minDocCount == 0) == (emptyBucketInfo != null);
@ -388,7 +389,7 @@ public class InternalHistogram<B extends InternalHistogram.Bucket> extends Inter
CollectionUtil.introSort(reducedBuckets, order.comparator());
}
return getFactory().create(getName(), reducedBuckets, order, minDocCount, emptyBucketInfo, formatter, keyed);
return getFactory().create(getName(), reducedBuckets, order, minDocCount, emptyBucketInfo, formatter, keyed, getMetaData());
}
protected B createBucket(long key, long docCount, InternalAggregations aggregations, boolean keyed, @Nullable ValueFormatter formatter) {
@ -400,8 +401,7 @@ public class InternalHistogram<B extends InternalHistogram.Bucket> extends Inter
}
@Override
public void readFrom(StreamInput in) throws IOException {
name = in.readString();
protected void doReadFrom(StreamInput in) throws IOException {
order = InternalOrder.Streams.readOrder(in);
minDocCount = in.readVLong();
if (minDocCount == 0) {
@ -421,8 +421,7 @@ public class InternalHistogram<B extends InternalHistogram.Bucket> extends Inter
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
protected void doWriteTo(StreamOutput out) throws IOException {
InternalOrder.Streams.writeOrder(order, out);
out.writeVLong(minDocCount);
if (minDocCount == 0) {

View file

@ -24,6 +24,7 @@ import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.bucket.InternalSingleBucketAggregation;
import java.io.IOException;
import java.util.Map;
/**
*
@ -49,8 +50,8 @@ public class InternalMissing extends InternalSingleBucketAggregation implements
InternalMissing() {
}
InternalMissing(String name, long docCount, InternalAggregations aggregations) {
super(name, docCount, aggregations);
InternalMissing(String name, long docCount, InternalAggregations aggregations, Map<String, Object> metaData) {
super(name, docCount, aggregations, metaData);
}
@Override
@ -60,6 +61,6 @@ public class InternalMissing extends InternalSingleBucketAggregation implements
@Override
protected InternalSingleBucketAggregation newAggregation(String name, long docCount, InternalAggregations subAggregations) {
return new InternalMissing(name, docCount, subAggregations);
return new InternalMissing(name, docCount, subAggregations, getMetaData());
}
}

View file

@ -30,6 +30,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFacto
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import java.io.IOException;
import java.util.Map;
/**
*
@ -40,8 +41,8 @@ public class MissingAggregator extends SingleBucketAggregator {
private Bits docsWithValue;
public MissingAggregator(String name, AggregatorFactories factories, ValuesSource valuesSource,
AggregationContext aggregationContext, Aggregator parent) {
super(name, factories, aggregationContext, parent);
AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
super(name, factories, aggregationContext, parent, metaData);
this.valuesSource = valuesSource;
}
@ -63,28 +64,28 @@ public class MissingAggregator extends SingleBucketAggregator {
@Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
return new InternalMissing(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal));
return new InternalMissing(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), getMetaData());
}
@Override
public InternalAggregation buildEmptyAggregation() {
return new InternalMissing(name, 0, buildEmptySubAggregations());
return new InternalMissing(name, 0, buildEmptySubAggregations(), getMetaData());
}
public static class Factory extends ValuesSourceAggregatorFactory {
public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource, Map<String, Object>> {
public Factory(String name, ValuesSourceConfig valueSourceConfig) {
super(name, InternalMissing.TYPE.name(), valueSourceConfig);
}
@Override
protected MissingAggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent) {
return new MissingAggregator(name, factories, null, aggregationContext, parent);
protected MissingAggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
return new MissingAggregator(name, factories, null, aggregationContext, parent, metaData);
}
@Override
protected MissingAggregator create(ValuesSource valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent) {
return new MissingAggregator(name, factories, valuesSource, aggregationContext, parent);
protected MissingAggregator create(ValuesSource valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
return new MissingAggregator(name, factories, valuesSource, aggregationContext, parent, metaData);
}
}

View file

@ -24,6 +24,7 @@ import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.bucket.InternalSingleBucketAggregation;
import java.io.IOException;
import java.util.Map;
/**
*
@ -48,8 +49,8 @@ public class InternalNested extends InternalSingleBucketAggregation implements N
public InternalNested() {
}
public InternalNested(String name, long docCount, InternalAggregations aggregations) {
super(name, docCount, aggregations);
public InternalNested(String name, long docCount, InternalAggregations aggregations, Map<String, Object> metaData) {
super(name, docCount, aggregations, metaData);
}
@Override
@ -59,6 +60,6 @@ public class InternalNested extends InternalSingleBucketAggregation implements N
@Override
protected InternalSingleBucketAggregation newAggregation(String name, long docCount, InternalAggregations subAggregations) {
return new InternalNested(name, docCount, subAggregations);
return new InternalNested(name, docCount, subAggregations, getMetaData());
}
}

View file

@ -24,6 +24,7 @@ import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.bucket.InternalSingleBucketAggregation;
import java.io.IOException;
import java.util.Map;
/**
*
@ -48,8 +49,8 @@ public class InternalReverseNested extends InternalSingleBucketAggregation imple
public InternalReverseNested() {
}
public InternalReverseNested(String name, long docCount, InternalAggregations aggregations) {
super(name, docCount, aggregations);
public InternalReverseNested(String name, long docCount, InternalAggregations aggregations, Map<String, Object> metaData) {
super(name, docCount, aggregations, metaData);
}
@Override
@ -59,6 +60,6 @@ public class InternalReverseNested extends InternalSingleBucketAggregation imple
@Override
protected InternalSingleBucketAggregation newAggregation(String name, long docCount, InternalAggregations subAggregations) {
return new InternalReverseNested(name, docCount, subAggregations);
return new InternalReverseNested(name, docCount, subAggregations, getMetaData());
}
}

View file

@ -35,6 +35,7 @@ import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Map;
/**
*
@ -49,8 +50,8 @@ public class NestedAggregator extends SingleBucketAggregator implements ReaderCo
private Bits childDocs;
private FixedBitSet parentDocs;
public NestedAggregator(String name, AggregatorFactories factories, String nestedPath, AggregationContext aggregationContext, Aggregator parentAggregator) {
super(name, factories, aggregationContext, parentAggregator);
public NestedAggregator(String name, AggregatorFactories factories, String nestedPath, AggregationContext aggregationContext, Aggregator parentAggregator, Map<String, Object> metaData) {
super(name, factories, aggregationContext, parentAggregator, metaData);
this.nestedPath = nestedPath;
this.parentAggregator = parentAggregator;
MapperService.SmartNameObjectMapper mapper = aggregationContext.searchContext().smartNameObjectMapper(nestedPath);
@ -117,12 +118,12 @@ public class NestedAggregator extends SingleBucketAggregator implements ReaderCo
@Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
return new InternalNested(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal));
return new InternalNested(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), getMetaData());
}
@Override
public InternalAggregation buildEmptyAggregation() {
return new InternalNested(name, 0, buildEmptySubAggregations());
return new InternalNested(name, 0, buildEmptySubAggregations(), getMetaData());
}
public String getNestedPath() {
@ -150,8 +151,8 @@ public class NestedAggregator extends SingleBucketAggregator implements ReaderCo
}
@Override
public Aggregator create(AggregationContext context, Aggregator parent, long expectedBucketsCount) {
return new NestedAggregator(name, factories, path, context, parent);
public Aggregator createInternal(AggregationContext context, Aggregator parent, long expectedBucketsCount, Map<String, Object> metaData) {
return new NestedAggregator(name, factories, path, context, parent, metaData);
}
}
}

View file

@ -36,6 +36,7 @@ import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Map;
/**
*
@ -48,8 +49,8 @@ public class ReverseNestedAggregator extends SingleBucketAggregator implements R
// TODO: Add LongIntPagedHashMap?
private final LongIntOpenHashMap bucketOrdToLastCollectedParentDoc;
public ReverseNestedAggregator(String name, AggregatorFactories factories, String nestedPath, AggregationContext aggregationContext, Aggregator parent) {
super(name, factories, aggregationContext, parent);
public ReverseNestedAggregator(String name, AggregatorFactories factories, String nestedPath, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
super(name, factories, aggregationContext, parent, metaData);
// Early validation
NestedAggregator closestNestedAggregator = findClosestNestedAggregator(parent);
@ -134,12 +135,12 @@ public class ReverseNestedAggregator extends SingleBucketAggregator implements R
@Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
return new InternalReverseNested(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal));
return new InternalReverseNested(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), getMetaData());
}
@Override
public InternalAggregation buildEmptyAggregation() {
return new InternalReverseNested(name, 0, buildEmptySubAggregations());
return new InternalReverseNested(name, 0, buildEmptySubAggregations(), getMetaData());
}
Filter getParentFilter() {
@ -156,8 +157,8 @@ public class ReverseNestedAggregator extends SingleBucketAggregator implements R
}
@Override
public Aggregator create(AggregationContext context, Aggregator parent, long expectedBucketsCount) {
return new ReverseNestedAggregator(name, factories, path, context, parent);
public Aggregator createInternal(AggregationContext context, Aggregator parent, long expectedBucketsCount, Map<String, Object> metaData) {
return new ReverseNestedAggregator(name, factories, path, context, parent, metaData);
}
}
}

View file

@ -199,8 +199,8 @@ public class InternalRange<B extends InternalRange.Bucket> extends InternalAggre
return TYPE.name();
}
public R create(String name, List<B> ranges, @Nullable ValueFormatter formatter, boolean keyed) {
return (R) new InternalRange<>(name, ranges, formatter, keyed);
public R create(String name, List<B> ranges, @Nullable ValueFormatter formatter, boolean keyed, Map<String, Object> metaData) {
return (R) new InternalRange<>(name, ranges, formatter, keyed, metaData);
}
@ -216,8 +216,8 @@ public class InternalRange<B extends InternalRange.Bucket> extends InternalAggre
public InternalRange() {} // for serialization
public InternalRange(String name, List<B> ranges, @Nullable ValueFormatter formatter, boolean keyed) {
super(name);
public InternalRange(String name, List<B> ranges, @Nullable ValueFormatter formatter, boolean keyed, Map<String, Object> metaData) {
super(name, metaData);
this.ranges = ranges;
this.formatter = formatter;
this.keyed = keyed;
@ -268,12 +268,11 @@ public class InternalRange<B extends InternalRange.Bucket> extends InternalAggre
for (int i = 0; i < this.ranges.size(); ++i) {
ranges.add((B) rangeList[i].get(0).reduce(rangeList[i], reduceContext));
}
return getFactory().create(name, ranges, formatter, keyed);
return getFactory().create(name, ranges, formatter, keyed, getMetaData());
}
@Override
public void readFrom(StreamInput in) throws IOException {
name = in.readString();
protected void doReadFrom(StreamInput in) throws IOException {
formatter = ValueFormatterStreams.readOptional(in);
keyed = in.readBoolean();
int size = in.readVInt();
@ -287,8 +286,7 @@ public class InternalRange<B extends InternalRange.Bucket> extends InternalAggre
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
protected void doWriteTo(StreamOutput out) throws IOException {
ValueFormatterStreams.writeOptional(formatter, out);
out.writeBoolean(keyed);
out.writeVInt(ranges.size());

View file

@ -37,6 +37,7 @@ import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
/**
*
@ -96,9 +97,10 @@ public class RangeAggregator extends BucketsAggregator {
List<Range> ranges,
boolean keyed,
AggregationContext aggregationContext,
Aggregator parent) {
Aggregator parent,
Map<String, Object> metaData) {
super(name, BucketAggregationMode.MULTI_BUCKETS, factories, ranges.size() * (parent == null ? 1 : parent.estimatedBucketCount()), aggregationContext, parent);
super(name, BucketAggregationMode.MULTI_BUCKETS, factories, ranges.size() * (parent == null ? 1 : parent.estimatedBucketCount()), aggregationContext, parent, metaData);
assert valuesSource != null;
this.valuesSource = valuesSource;
this.formatter = format != null ? format.formatter() : null;
@ -204,7 +206,7 @@ public class RangeAggregator extends BucketsAggregator {
buckets.add(bucket);
}
// value source can be null in the case of unmapped fields
return rangeFactory.create(name, buckets, formatter, keyed);
return rangeFactory.create(name, buckets, formatter, keyed, getMetaData());
}
@Override
@ -218,7 +220,7 @@ public class RangeAggregator extends BucketsAggregator {
buckets.add(bucket);
}
// value source can be null in the case of unmapped fields
return rangeFactory.create(name, buckets, formatter, keyed);
return rangeFactory.create(name, buckets, formatter, keyed, getMetaData());
}
private static final void sortRanges(final Range[] ranges) {
@ -255,9 +257,10 @@ public class RangeAggregator extends BucketsAggregator {
ValueFormat format,
AggregationContext context,
Aggregator parent,
InternalRange.Factory factory) {
InternalRange.Factory factory,
Map<String, Object> metaData) {
super(name, context, parent);
super(name, context, parent, metaData);
this.ranges = ranges;
ValueParser parser = format != null ? format.parser() : ValueParser.RAW;
for (Range range : this.ranges) {
@ -275,11 +278,11 @@ public class RangeAggregator extends BucketsAggregator {
for (RangeAggregator.Range range : ranges) {
buckets.add(factory.createBucket(range.key, range.from, range.to, 0, subAggs, keyed, formatter));
}
return factory.create(name, buckets, formatter, keyed);
return factory.create(name, buckets, formatter, keyed, getMetaData());
}
}
public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource.Numeric> {
public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource.Numeric, Map<String, Object>> {
private final InternalRange.Factory rangeFactory;
private final List<Range> ranges;
@ -293,13 +296,13 @@ public class RangeAggregator extends BucketsAggregator {
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent) {
return new Unmapped(name, ranges, keyed, config.format(), aggregationContext, parent, rangeFactory);
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
return new Unmapped(name, ranges, keyed, config.format(), aggregationContext, parent, rangeFactory, metaData);
}
@Override
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent) {
return new RangeAggregator(name, factories, valuesSource, config.format(), rangeFactory, ranges, keyed, aggregationContext, parent);
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
return new RangeAggregator(name, factories, valuesSource, config.format(), rangeFactory, ranges, keyed, aggregationContext, parent, metaData);
}
}

View file

@ -32,6 +32,7 @@ import org.joda.time.DateTimeZone;
import java.io.IOException;
import java.util.List;
import java.util.Map;
/**
*
@ -119,8 +120,8 @@ public class InternalDateRange extends InternalRange<InternalDateRange.Bucket> i
}
@Override
public InternalDateRange create(String name, List<InternalDateRange.Bucket> ranges, ValueFormatter formatter, boolean keyed) {
return new InternalDateRange(name, ranges, formatter, keyed);
public InternalDateRange create(String name, List<InternalDateRange.Bucket> ranges, ValueFormatter formatter, boolean keyed, Map<String, Object> metaData) {
return new InternalDateRange(name, ranges, formatter, keyed, metaData);
}
@Override
@ -131,8 +132,8 @@ public class InternalDateRange extends InternalRange<InternalDateRange.Bucket> i
InternalDateRange() {} // for serialization
InternalDateRange(String name, List<InternalDateRange.Bucket> ranges, @Nullable ValueFormatter formatter, boolean keyed) {
super(name, ranges, formatter, keyed);
InternalDateRange(String name, List<InternalDateRange.Bucket> ranges, @Nullable ValueFormatter formatter, boolean keyed, Map<String, Object> metaData) {
super(name, ranges, formatter, keyed, metaData);
}
@Override

View file

@ -42,6 +42,7 @@ import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
/**
*
@ -152,7 +153,7 @@ public class GeoDistanceParser implements Aggregator.Parser {
return new GeoDistanceFactory(aggregationName, vsParser.config(), InternalGeoDistance.FACTORY, origin, unit, distanceType, ranges, keyed);
}
private static class GeoDistanceFactory extends ValuesSourceAggregatorFactory<ValuesSource.GeoPoint> {
private static class GeoDistanceFactory extends ValuesSourceAggregatorFactory<ValuesSource.GeoPoint, Map<String, Object>> {
private final GeoPoint origin;
private final DistanceUnit unit;
@ -174,15 +175,15 @@ public class GeoDistanceParser implements Aggregator.Parser {
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent) {
return new Unmapped(name, ranges, keyed, null, aggregationContext, parent, rangeFactory);
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
return new Unmapped(name, ranges, keyed, null, aggregationContext, parent, rangeFactory, metaData);
}
@Override
protected Aggregator create(final ValuesSource.GeoPoint valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent) {
protected Aggregator create(final ValuesSource.GeoPoint valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
DistanceSource distanceSource = new DistanceSource(valuesSource, distanceType, origin, unit);
aggregationContext.registerReaderContextAware(distanceSource);
return new RangeAggregator(name, factories, distanceSource, null, rangeFactory, ranges, keyed, aggregationContext, parent);
return new RangeAggregator(name, factories, distanceSource, null, rangeFactory, ranges, keyed, aggregationContext, parent, metaData);
}
private static class DistanceSource extends ValuesSource.Numeric implements ReaderContextAware {

View file

@ -30,6 +30,7 @@ import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
import java.io.IOException;
import java.util.List;
import java.util.Map;
/**
*
@ -107,8 +108,8 @@ public class InternalGeoDistance extends InternalRange<InternalGeoDistance.Bucke
}
@Override
public InternalGeoDistance create(String name, List<Bucket> ranges, @Nullable ValueFormatter formatter, boolean keyed) {
return new InternalGeoDistance(name, ranges, formatter, keyed);
public InternalGeoDistance create(String name, List<Bucket> ranges, @Nullable ValueFormatter formatter, boolean keyed, Map<String, Object> metaData) {
return new InternalGeoDistance(name, ranges, formatter, keyed, metaData);
}
@Override
@ -119,8 +120,8 @@ public class InternalGeoDistance extends InternalRange<InternalGeoDistance.Bucke
InternalGeoDistance() {} // for serialization
public InternalGeoDistance(String name, List<Bucket> ranges, @Nullable ValueFormatter formatter, boolean keyed) {
super(name, ranges, formatter, keyed);
public InternalGeoDistance(String name, List<Bucket> ranges, @Nullable ValueFormatter formatter, boolean keyed, Map<String, Object> metaData) {
super(name, ranges, formatter, keyed, metaData);
}
@Override

View file

@ -30,6 +30,7 @@ import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
import java.io.IOException;
import java.util.List;
import java.util.Map;
/**
*
@ -116,8 +117,8 @@ public class InternalIPv4Range extends InternalRange<InternalIPv4Range.Bucket> i
}
@Override
public InternalIPv4Range create(String name, List<Bucket> ranges, @Nullable ValueFormatter formatter, boolean keyed) {
return new InternalIPv4Range(name, ranges, keyed);
public InternalIPv4Range create(String name, List<Bucket> ranges, @Nullable ValueFormatter formatter, boolean keyed, Map<String, Object> metaData) {
return new InternalIPv4Range(name, ranges, keyed, metaData);
}
@Override
@ -128,8 +129,8 @@ public class InternalIPv4Range extends InternalRange<InternalIPv4Range.Bucket> i
public InternalIPv4Range() {} // for serialization
public InternalIPv4Range(String name, List<InternalIPv4Range.Bucket> ranges, boolean keyed) {
super(name, ranges, ValueFormatter.IPv4, keyed);
public InternalIPv4Range(String name, List<InternalIPv4Range.Bucket> ranges, boolean keyed, Map<String, Object> metaData) {
super(name, ranges, ValueFormatter.IPv4, keyed, metaData);
}
@Override

View file

@ -33,6 +33,7 @@ import org.elasticsearch.search.internal.ContextIndexSearcher;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.Map;
/**
* An global ordinal based implementation of significant terms, based on {@link SignificantStringTermsAggregator}.
@ -45,9 +46,9 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri
public GlobalOrdinalsSignificantTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource,
long estimatedBucketCount, long maxOrd, BucketCountThresholds bucketCountThresholds,
IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent,
SignificantTermsAggregatorFactory termsAggFactory) {
SignificantTermsAggregatorFactory termsAggFactory, Map<String, Object> metaData) {
super(name, factories, valuesSource, estimatedBucketCount, maxOrd, null, bucketCountThresholds, includeExclude, aggregationContext, parent, SubAggCollectionMode.DEPTH_FIRST, false);
super(name, factories, valuesSource, estimatedBucketCount, maxOrd, null, bucketCountThresholds, includeExclude, aggregationContext, parent, SubAggCollectionMode.DEPTH_FIRST, false, metaData);
this.termsAggFactory = termsAggFactory;
}
@ -113,7 +114,7 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri
list[i] = bucket;
}
return new SignificantStringTerms(subsetSize, supersetSize, name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Arrays.asList(list));
return new SignificantStringTerms(subsetSize, supersetSize, name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Arrays.asList(list), getMetaData());
}
@Override
@ -122,7 +123,7 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri
ContextIndexSearcher searcher = context.searchContext().searcher();
IndexReader topReader = searcher.getIndexReader();
int supersetSize = topReader.numDocs();
return new SignificantStringTerms(0, supersetSize, name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Collections.<InternalSignificantTerms.Bucket>emptyList());
return new SignificantStringTerms(0, supersetSize, name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Collections.<InternalSignificantTerms.Bucket>emptyList(), getMetaData());
}
@Override
@ -134,8 +135,8 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri
private final LongHash bucketOrds;
public WithHash(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, long estimatedBucketCount, BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggFactory) {
super(name, factories, valuesSource, estimatedBucketCount, estimatedBucketCount, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggFactory);
public WithHash(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, long estimatedBucketCount, BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggFactory, Map<String, Object> metaData) {
super(name, factories, valuesSource, estimatedBucketCount, estimatedBucketCount, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggFactory, metaData);
bucketOrds = new LongHash(estimatedBucketCount, aggregationContext.bigArrays());
}

View file

@ -115,8 +115,8 @@ public abstract class InternalSignificantTerms extends InternalAggregation imple
}
}
protected InternalSignificantTerms(long subsetSize, long supersetSize, String name, int requiredSize, long minDocCount, SignificanceHeuristic significanceHeuristic, List<Bucket> buckets) {
super(name);
protected InternalSignificantTerms(long subsetSize, long supersetSize, String name, int requiredSize, long minDocCount, SignificanceHeuristic significanceHeuristic, List<Bucket> buckets, Map<String, Object> metaData) {
super(name, metaData);
this.requiredSize = requiredSize;
this.minDocCount = minDocCount;
this.buckets = buckets;

View file

@ -162,9 +162,9 @@ public class SignificantLongTerms extends InternalSignificantTerms {
} // for serialization
public SignificantLongTerms(long subsetSize, long supersetSize, String name, @Nullable ValueFormatter formatter,
int requiredSize, long minDocCount, SignificanceHeuristic significanceHeuristic, List<InternalSignificantTerms.Bucket> buckets) {
int requiredSize, long minDocCount, SignificanceHeuristic significanceHeuristic, List<InternalSignificantTerms.Bucket> buckets, Map<String, Object> metaData) {
super(subsetSize, supersetSize, name, requiredSize, minDocCount, significanceHeuristic, buckets);
super(subsetSize, supersetSize, name, requiredSize, minDocCount, significanceHeuristic, buckets, metaData);
this.formatter = formatter;
}
@ -176,12 +176,11 @@ public class SignificantLongTerms extends InternalSignificantTerms {
@Override
InternalSignificantTerms newAggregation(long subsetSize, long supersetSize,
List<InternalSignificantTerms.Bucket> buckets) {
return new SignificantLongTerms(subsetSize, supersetSize, getName(), formatter, requiredSize, minDocCount, significanceHeuristic, buckets);
return new SignificantLongTerms(subsetSize, supersetSize, getName(), formatter, requiredSize, minDocCount, significanceHeuristic, buckets, getMetaData());
}
@Override
public void readFrom(StreamInput in) throws IOException {
this.name = in.readString();
protected void doReadFrom(StreamInput in) throws IOException {
this.formatter = ValueFormatterStreams.readOptional(in);
this.requiredSize = readSize(in);
this.minDocCount = in.readVLong();
@ -202,8 +201,7 @@ public class SignificantLongTerms extends InternalSignificantTerms {
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
protected void doWriteTo(StreamOutput out) throws IOException {
ValueFormatterStreams.writeOptional(formatter, out);
writeSize(requiredSize, out);
out.writeVLong(minDocCount);

View file

@ -33,6 +33,7 @@ import org.elasticsearch.search.internal.ContextIndexSearcher;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.Map;
/**
*
@ -41,9 +42,9 @@ public class SignificantLongTermsAggregator extends LongTermsAggregator {
public SignificantLongTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, @Nullable ValueFormat format,
long estimatedBucketCount, BucketCountThresholds bucketCountThresholds,
AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggFactory, IncludeExclude.LongFilter includeExclude) {
AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggFactory, IncludeExclude.LongFilter includeExclude, Map<String, Object> metaData) {
super(name, factories, valuesSource, format, estimatedBucketCount, null, bucketCountThresholds, aggregationContext, parent, SubAggCollectionMode.DEPTH_FIRST, false, includeExclude);
super(name, factories, valuesSource, format, estimatedBucketCount, null, bucketCountThresholds, aggregationContext, parent, SubAggCollectionMode.DEPTH_FIRST, false, includeExclude, metaData);
this.termsAggFactory = termsAggFactory;
}
@ -92,7 +93,7 @@ public class SignificantLongTermsAggregator extends LongTermsAggregator {
bucket.aggregations = bucketAggregations(bucket.bucketOrd);
list[i] = bucket;
}
return new SignificantLongTerms(subsetSize, supersetSize, name, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Arrays.asList(list));
return new SignificantLongTerms(subsetSize, supersetSize, name, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Arrays.asList(list), getMetaData());
}
@Override
@ -101,7 +102,7 @@ public class SignificantLongTermsAggregator extends LongTermsAggregator {
ContextIndexSearcher searcher = context.searchContext().searcher();
IndexReader topReader = searcher.getIndexReader();
int supersetSize = topReader.numDocs();
return new SignificantLongTerms(0, supersetSize, name, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Collections.<InternalSignificantTerms.Bucket>emptyList());
return new SignificantLongTerms(0, supersetSize, name, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Collections.<InternalSignificantTerms.Bucket>emptyList(), getMetaData());
}
@Override

View file

@ -156,8 +156,8 @@ public class SignificantStringTerms extends InternalSignificantTerms {
SignificantStringTerms() {} // for serialization
public SignificantStringTerms(long subsetSize, long supersetSize, String name, int requiredSize,
long minDocCount, SignificanceHeuristic significanceHeuristic, List<InternalSignificantTerms.Bucket> buckets) {
super(subsetSize, supersetSize, name, requiredSize, minDocCount, significanceHeuristic, buckets);
long minDocCount, SignificanceHeuristic significanceHeuristic, List<InternalSignificantTerms.Bucket> buckets, Map<String, Object> metaData) {
super(subsetSize, supersetSize, name, requiredSize, minDocCount, significanceHeuristic, buckets, metaData);
}
@Override
@ -168,12 +168,11 @@ public class SignificantStringTerms extends InternalSignificantTerms {
@Override
InternalSignificantTerms newAggregation(long subsetSize, long supersetSize,
List<InternalSignificantTerms.Bucket> buckets) {
return new SignificantStringTerms(subsetSize, supersetSize, getName(), requiredSize, minDocCount, significanceHeuristic, buckets);
return new SignificantStringTerms(subsetSize, supersetSize, getName(), requiredSize, minDocCount, significanceHeuristic, buckets, getMetaData());
}
@Override
public void readFrom(StreamInput in) throws IOException {
this.name = in.readString();
protected void doReadFrom(StreamInput in) throws IOException {
this.requiredSize = readSize(in);
this.minDocCount = in.readVLong();
this.subsetSize = in.readVLong();
@ -192,8 +191,7 @@ public class SignificantStringTerms extends InternalSignificantTerms {
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
protected void doWriteTo(StreamOutput out) throws IOException {
writeSize(requiredSize, out);
out.writeVLong(minDocCount);
out.writeVLong(subsetSize);

View file

@ -32,6 +32,7 @@ import org.elasticsearch.search.internal.ContextIndexSearcher;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.Map;
/**
* An aggregator of significant string values.
@ -44,9 +45,9 @@ public class SignificantStringTermsAggregator extends StringTermsAggregator {
public SignificantStringTermsAggregator(String name, AggregatorFactories factories, ValuesSource valuesSource,
long estimatedBucketCount, BucketCountThresholds bucketCountThresholds,
IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent,
SignificantTermsAggregatorFactory termsAggFactory) {
SignificantTermsAggregatorFactory termsAggFactory, Map<String, Object> metaData) {
super(name, factories, valuesSource, estimatedBucketCount, null, bucketCountThresholds, includeExclude, aggregationContext, parent, SubAggCollectionMode.DEPTH_FIRST, false);
super(name, factories, valuesSource, estimatedBucketCount, null, bucketCountThresholds, includeExclude, aggregationContext, parent, SubAggCollectionMode.DEPTH_FIRST, false, metaData);
this.termsAggFactory = termsAggFactory;
}
@ -97,7 +98,7 @@ public class SignificantStringTermsAggregator extends StringTermsAggregator {
list[i] = bucket;
}
return new SignificantStringTerms(subsetSize, supersetSize, name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Arrays.asList(list));
return new SignificantStringTerms(subsetSize, supersetSize, name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Arrays.asList(list), getMetaData());
}
@Override
@ -106,7 +107,7 @@ public class SignificantStringTermsAggregator extends StringTermsAggregator {
ContextIndexSearcher searcher = context.searchContext().searcher();
IndexReader topReader = searcher.getIndexReader();
int supersetSize = topReader.numDocs();
return new SignificantStringTerms(0, supersetSize, name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Collections.<InternalSignificantTerms.Bucket>emptyList());
return new SignificantStringTerms(0, supersetSize, name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Collections.<InternalSignificantTerms.Bucket>emptyList(), getMetaData());
}
@Override

View file

@ -42,11 +42,12 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Map;
/**
*
*/
public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFactory implements Releasable {
public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource, Map<String, Object>> implements Releasable {
public SignificanceHeuristic getSignificanceHeuristic() {
return significanceHeuristic;
@ -59,8 +60,8 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
@Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount,
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory) {
return new SignificantStringTermsAggregator(name, factories, valuesSource, estimatedBucketCount, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggregatorFactory);
AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map<String, Object> metaData) {
return new SignificantStringTermsAggregator(name, factories, valuesSource, estimatedBucketCount, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggregatorFactory, metaData);
}
@Override
@ -74,11 +75,11 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
@Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount,
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory) {
AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map<String, Object> metaData) {
ValuesSource.Bytes.WithOrdinals valueSourceWithOrdinals = (ValuesSource.Bytes.WithOrdinals) valuesSource;
IndexSearcher indexSearcher = aggregationContext.searchContext().searcher();
long maxOrd = valueSourceWithOrdinals.globalMaxOrd(indexSearcher);
return new GlobalOrdinalsSignificantTermsAggregator(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, estimatedBucketCount, maxOrd, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggregatorFactory);
return new GlobalOrdinalsSignificantTermsAggregator(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, estimatedBucketCount, maxOrd, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggregatorFactory, metaData);
}
@Override
@ -92,8 +93,8 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
@Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount,
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory) {
return new GlobalOrdinalsSignificantTermsAggregator.WithHash(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, estimatedBucketCount, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggregatorFactory);
AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map<String, Object> metaData) {
return new GlobalOrdinalsSignificantTermsAggregator.WithHash(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, estimatedBucketCount, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggregatorFactory, metaData);
}
@Override
@ -119,7 +120,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
abstract Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount,
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory);
AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map<String, Object> metaData);
abstract boolean needsGlobalOrdinals();
@ -158,9 +159,9 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent) {
final InternalAggregation aggregation = new UnmappedSignificantTerms(name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount());
return new NonCollectingAggregator(name, aggregationContext, parent) {
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
final InternalAggregation aggregation = new UnmappedSignificantTerms(name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), metaData);
return new NonCollectingAggregator(name, aggregationContext, parent, metaData) {
@Override
public InternalAggregation buildEmptyAggregation() {
return aggregation;
@ -169,7 +170,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
}
@Override
protected Aggregator create(ValuesSource valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent) {
protected Aggregator create(ValuesSource valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
numberOfAggregatorsCreated++;
long estimatedBucketCount = TermsAggregatorFactory.estimatedBucketCount(valuesSource, parent);
@ -191,7 +192,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
}
assert execution != null;
valuesSource.setNeedsGlobalOrdinals(execution.needsGlobalOrdinals());
return execution.create(name, factories, valuesSource, estimatedBucketCount, bucketCountThresholds, includeExclude, aggregationContext, parent, this);
return execution.create(name, factories, valuesSource, estimatedBucketCount, bucketCountThresholds, includeExclude, aggregationContext, parent, this, metaData);
}
@ -209,7 +210,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
if (includeExclude != null) {
longFilter = includeExclude.convertToLongFilter();
}
return new SignificantLongTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(), estimatedBucketCount, bucketCountThresholds, aggregationContext, parent, this, longFilter);
return new SignificantLongTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(), estimatedBucketCount, bucketCountThresholds, aggregationContext, parent, this, longFilter, metaData);
}
throw new AggregationExecutionException("sigfnificant_terms aggregation cannot be applied to field [" + config.fieldContext().field() +

View file

@ -56,10 +56,10 @@ public class UnmappedSignificantTerms extends InternalSignificantTerms {
UnmappedSignificantTerms() {} // for serialization
public UnmappedSignificantTerms(String name, int requiredSize, long minDocCount) {
public UnmappedSignificantTerms(String name, int requiredSize, long minDocCount, Map<String, Object> metaData) {
//We pass zero for index/subset sizes because for the purpose of significant term analysis
// we assume an unmapped index's size is irrelevant to the proceedings.
super(0, 0, name, requiredSize, minDocCount, JLHScore.INSTANCE, BUCKETS);
super(0, 0, name, requiredSize, minDocCount, JLHScore.INSTANCE, BUCKETS, metaData);
}
@Override
@ -83,8 +83,7 @@ public class UnmappedSignificantTerms extends InternalSignificantTerms {
}
@Override
public void readFrom(StreamInput in) throws IOException {
this.name = in.readString();
protected void doReadFrom(StreamInput in) throws IOException {
this.requiredSize = readSize(in);
this.minDocCount = in.readVLong();
this.buckets = BUCKETS;
@ -92,8 +91,7 @@ public class UnmappedSignificantTerms extends InternalSignificantTerms {
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
protected void doWriteTo(StreamOutput out) throws IOException {
writeSize(requiredSize, out);
out.writeVLong(minDocCount);
}

View file

@ -25,6 +25,7 @@ import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import java.util.Collections;
import java.util.Map;
abstract class AbstractStringTermsAggregator extends TermsAggregator {
@ -33,8 +34,8 @@ abstract class AbstractStringTermsAggregator extends TermsAggregator {
public AbstractStringTermsAggregator(String name, AggregatorFactories factories,
long estimatedBucketsCount, AggregationContext context, Aggregator parent,
Terms.Order order, BucketCountThresholds bucketCountThresholds, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError) {
super(name, BucketAggregationMode.PER_BUCKET, factories, estimatedBucketsCount, context, parent, bucketCountThresholds, order, subAggCollectMode);
Terms.Order order, BucketCountThresholds bucketCountThresholds, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData) {
super(name, BucketAggregationMode.PER_BUCKET, factories, estimatedBucketsCount, context, parent, bucketCountThresholds, order, subAggCollectMode, metaData);
this.showTermDocCountError = showTermDocCountError;
}
@ -45,7 +46,7 @@ abstract class AbstractStringTermsAggregator extends TermsAggregator {
@Override
public InternalAggregation buildEmptyAggregation() {
return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Collections.<InternalTerms.Bucket>emptyList(), showTermDocCountError, 0, 0);
return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Collections.<InternalTerms.Bucket>emptyList(), showTermDocCountError, 0, 0, getMetaData());
}
}

View file

@ -163,8 +163,8 @@ public class DoubleTerms extends InternalTerms {
DoubleTerms() {} // for serialization
public DoubleTerms(String name, Terms.Order order, @Nullable ValueFormatter formatter, int requiredSize, int shardSize, long minDocCount, List<InternalTerms.Bucket> buckets, boolean showTermDocCountError, long docCountError, long otherDocCount) {
super(name, order, requiredSize, shardSize, minDocCount, buckets, showTermDocCountError, docCountError, otherDocCount);
public DoubleTerms(String name, Terms.Order order, @Nullable ValueFormatter formatter, int requiredSize, int shardSize, long minDocCount, List<InternalTerms.Bucket> buckets, boolean showTermDocCountError, long docCountError, long otherDocCount, Map<String, Object> metaData) {
super(name, order, requiredSize, shardSize, minDocCount, buckets, showTermDocCountError, docCountError, otherDocCount, metaData);
this.formatter = formatter;
}
@ -174,13 +174,12 @@ public class DoubleTerms extends InternalTerms {
}
@Override
protected InternalTerms newAggregation(String name, List<InternalTerms.Bucket> buckets, boolean showTermDocCountError, long docCountError, long otherDocCount) {
return new DoubleTerms(name, order, formatter, requiredSize, shardSize, minDocCount, buckets, showTermDocCountError, docCountError, otherDocCount);
protected InternalTerms newAggregation(String name, List<InternalTerms.Bucket> buckets, boolean showTermDocCountError, long docCountError, long otherDocCount, Map<String, Object> metaData) {
return new DoubleTerms(name, order, formatter, requiredSize, shardSize, minDocCount, buckets, showTermDocCountError, docCountError, otherDocCount, metaData);
}
@Override
public void readFrom(StreamInput in) throws IOException {
this.name = in.readString();
protected void doReadFrom(StreamInput in) throws IOException {
if (in.getVersion().onOrAfter(Version.V_1_4_0_Beta1)) {
this.docCountError = in.readLong();
} else {
@ -212,8 +211,7 @@ public class DoubleTerms extends InternalTerms {
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
protected void doWriteTo(StreamOutput out) throws IOException {
if (out.getVersion().onOrAfter(Version.V_1_4_0_Beta1)) {
out.writeLong(docCountError);
}

View file

@ -31,6 +31,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.format.ValueFormat;
import java.util.Arrays;
import java.util.Map;
/**
*
@ -38,8 +39,8 @@ import java.util.Arrays;
public class DoubleTermsAggregator extends LongTermsAggregator {
public DoubleTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, @Nullable ValueFormat format, long estimatedBucketCount,
Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, IncludeExclude.LongFilter longFilter) {
super(name, factories, valuesSource, format, estimatedBucketCount, order, bucketCountThresholds, aggregationContext, parent, collectionMode, showTermDocCountError, longFilter);
Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, IncludeExclude.LongFilter longFilter, Map<String, Object> metaData) {
super(name, factories, valuesSource, format, estimatedBucketCount, order, bucketCountThresholds, aggregationContext, parent, collectionMode, showTermDocCountError, longFilter, metaData);
}
@Override
@ -70,7 +71,7 @@ public class DoubleTermsAggregator extends LongTermsAggregator {
for (int i = 0; i < buckets.length; ++i) {
buckets[i] = convertToDouble(buckets[i]);
}
return new DoubleTerms(terms.getName(), terms.order, terms.formatter, terms.requiredSize, terms.shardSize, terms.minDocCount, Arrays.asList(buckets), terms.showTermDocCountError, terms.docCountError, terms.otherDocCount);
return new DoubleTerms(terms.getName(), terms.order, terms.formatter, terms.requiredSize, terms.shardSize, terms.minDocCount, Arrays.asList(buckets), terms.showTermDocCountError, terms.docCountError, terms.otherDocCount, terms.getMetaData());
}
}

View file

@ -50,6 +50,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSource;
import java.io.IOException;
import java.util.Arrays;
import java.util.Map;
/**
* An aggregator of string values that relies on global ordinals in order to build buckets.
@ -73,8 +74,8 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
public GlobalOrdinalsStringTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, long estimatedBucketCount,
long maxOrd, Terms.Order order, BucketCountThresholds bucketCountThresholds,
IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError) {
super(name, factories, maxOrd, aggregationContext, parent, order, bucketCountThresholds, collectionMode, showTermDocCountError);
IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map<String, Object> metaData) {
super(name, factories, maxOrd, aggregationContext, parent, order, bucketCountThresholds, collectionMode, showTermDocCountError, metaData);
this.valuesSource = valuesSource;
this.includeExclude = includeExclude;
}
@ -200,7 +201,7 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
bucket.docCountError = 0;
}
return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Arrays.asList(list), showTermDocCountError, 0, otherDocCount);
return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Arrays.asList(list), showTermDocCountError, 0, otherDocCount, getMetaData());
}
/** This is used internally only, just for compare using global ordinal instead of term bytes in the PQ */
@ -272,9 +273,9 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
public WithHash(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, long estimatedBucketCount,
long maxOrd, Terms.Order order, BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, AggregationContext aggregationContext,
Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError) {
Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map<String, Object> metaData) {
// Set maxOrd to estimatedBucketCount! To be conservative with memory.
super(name, factories, valuesSource, estimatedBucketCount, estimatedBucketCount, order, bucketCountThresholds, includeExclude, aggregationContext, parent, collectionMode, showTermDocCountError);
super(name, factories, valuesSource, estimatedBucketCount, estimatedBucketCount, order, bucketCountThresholds, includeExclude, aggregationContext, parent, collectionMode, showTermDocCountError, metaData);
bucketOrds = new LongHash(estimatedBucketCount, aggregationContext.bigArrays());
}
@ -340,8 +341,8 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
private RandomAccessOrds segmentOrds;
public LowCardinality(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, long estimatedBucketCount,
long maxOrd, Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError) {
super(name, factories, valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, null, aggregationContext, parent, collectionMode, showTermDocCountError);
long maxOrd, Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map<String, Object> metaData) {
super(name, factories, valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, null, aggregationContext, parent, collectionMode, showTermDocCountError, metaData);
assert factories == null || factories.count() == 0;
this.segmentDocCounts = bigArrays.newIntArray(maxOrd + 1, true);
}

View file

@ -117,8 +117,8 @@ public abstract class InternalTerms extends InternalAggregation implements Terms
protected InternalTerms() {} // for serialization
protected InternalTerms(String name, Terms.Order order, int requiredSize, int shardSize, long minDocCount, List<Bucket> buckets, boolean showTermDocCountError, long docCountError, long otherDocCount) {
super(name);
protected InternalTerms(String name, Terms.Order order, int requiredSize, int shardSize, long minDocCount, List<Bucket> buckets, boolean showTermDocCountError, long docCountError, long otherDocCount, Map<String, Object> metaData) {
super(name, metaData);
this.order = order;
this.requiredSize = requiredSize;
this.shardSize = shardSize;
@ -216,9 +216,9 @@ public abstract class InternalTerms extends InternalAggregation implements Terms
} else {
docCountError = aggregations.size() == 1 ? 0 : sumDocCountError;
}
return newAggregation(name, Arrays.asList(list), showTermDocCountError, docCountError, otherDocCount);
return newAggregation(name, Arrays.asList(list), showTermDocCountError, docCountError, otherDocCount, getMetaData());
}
protected abstract InternalTerms newAggregation(String name, List<Bucket> buckets, boolean showTermDocCountError, long docCountError, long otherDocCount);
protected abstract InternalTerms newAggregation(String name, List<Bucket> buckets, boolean showTermDocCountError, long docCountError, long otherDocCount, Map<String, Object> metaData);
}

View file

@ -163,8 +163,8 @@ public class LongTerms extends InternalTerms {
LongTerms() {} // for serialization
public LongTerms(String name, Terms.Order order, @Nullable ValueFormatter formatter, int requiredSize, int shardSize, long minDocCount, List<InternalTerms.Bucket> buckets, boolean showTermDocCountError, long docCountError, long otherDocCount) {
super(name, order, requiredSize, shardSize, minDocCount, buckets, showTermDocCountError, docCountError, otherDocCount);
public LongTerms(String name, Terms.Order order, @Nullable ValueFormatter formatter, int requiredSize, int shardSize, long minDocCount, List<InternalTerms.Bucket> buckets, boolean showTermDocCountError, long docCountError, long otherDocCount, Map<String, Object> metaData) {
super(name, order, requiredSize, shardSize, minDocCount, buckets, showTermDocCountError, docCountError, otherDocCount, metaData);
this.formatter = formatter;
}
@ -174,13 +174,12 @@ public class LongTerms extends InternalTerms {
}
@Override
protected InternalTerms newAggregation(String name, List<InternalTerms.Bucket> buckets, boolean showTermDocCountError, long docCountError, long otherDocCount) {
return new LongTerms(name, order, formatter, requiredSize, shardSize, minDocCount, buckets, showTermDocCountError, docCountError, otherDocCount);
protected InternalTerms newAggregation(String name, List<InternalTerms.Bucket> buckets, boolean showTermDocCountError, long docCountError, long otherDocCount, Map<String, Object> metaData) {
return new LongTerms(name, order, formatter, requiredSize, shardSize, minDocCount, buckets, showTermDocCountError, docCountError, otherDocCount, metaData);
}
@Override
public void readFrom(StreamInput in) throws IOException {
this.name = in.readString();
protected void doReadFrom(StreamInput in) throws IOException {
if (in.getVersion().onOrAfter(Version.V_1_4_0_Beta1)) {
this.docCountError = in.readLong();
} else {
@ -212,8 +211,7 @@ public class LongTerms extends InternalTerms {
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
protected void doWriteTo(StreamOutput out) throws IOException {
if (out.getVersion().onOrAfter(Version.V_1_4_0_Beta1)) {
out.writeLong(docCountError);
}

View file

@ -37,6 +37,7 @@ import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.Map;
/**
*
@ -51,8 +52,8 @@ public class LongTermsAggregator extends TermsAggregator {
private LongFilter longFilter;
public LongTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, @Nullable ValueFormat format, long estimatedBucketCount,
Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, IncludeExclude.LongFilter longFilter) {
super(name, BucketAggregationMode.PER_BUCKET, factories, estimatedBucketCount, aggregationContext, parent, bucketCountThresholds, order, subAggCollectMode);
Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, IncludeExclude.LongFilter longFilter, Map<String, Object> metaData) {
super(name, BucketAggregationMode.PER_BUCKET, factories, estimatedBucketCount, aggregationContext, parent, bucketCountThresholds, order, subAggCollectMode, metaData);
this.valuesSource = valuesSource;
this.showTermDocCountError = showTermDocCountError;
this.formatter = format != null ? format.formatter() : null;
@ -156,13 +157,13 @@ public class LongTermsAggregator extends TermsAggregator {
list[i].docCountError = 0;
}
return new LongTerms(name, order, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Arrays.asList(list), showTermDocCountError, 0, otherDocCount);
return new LongTerms(name, order, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Arrays.asList(list), showTermDocCountError, 0, otherDocCount, getMetaData());
}
@Override
public InternalAggregation buildEmptyAggregation() {
return new LongTerms(name, order, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Collections.<InternalTerms.Bucket>emptyList(), showTermDocCountError, 0, 0);
return new LongTerms(name, order, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Collections.<InternalTerms.Bucket>emptyList(), showTermDocCountError, 0, 0, getMetaData());
}
@Override

View file

@ -159,8 +159,8 @@ public class StringTerms extends InternalTerms {
StringTerms() {} // for serialization
public StringTerms(String name, Terms.Order order, int requiredSize, int shardSize, long minDocCount, List<InternalTerms.Bucket> buckets, boolean showTermDocCountError, long docCountError, long otherDocCount) {
super(name, order, requiredSize, shardSize, minDocCount, buckets, showTermDocCountError, docCountError, otherDocCount);
public StringTerms(String name, Terms.Order order, int requiredSize, int shardSize, long minDocCount, List<InternalTerms.Bucket> buckets, boolean showTermDocCountError, long docCountError, long otherDocCount, Map<String, Object> metaData) {
super(name, order, requiredSize, shardSize, minDocCount, buckets, showTermDocCountError, docCountError, otherDocCount, metaData);
}
@Override
@ -169,13 +169,12 @@ public class StringTerms extends InternalTerms {
}
@Override
protected InternalTerms newAggregation(String name, List<InternalTerms.Bucket> buckets, boolean showTermDocCountError, long docCountError, long otherDocCount) {
return new StringTerms(name, order, requiredSize, shardSize, minDocCount, buckets, showTermDocCountError, docCountError, otherDocCount);
protected InternalTerms newAggregation(String name, List<InternalTerms.Bucket> buckets, boolean showTermDocCountError, long docCountError, long otherDocCount, Map<String, Object> metaData) {
return new StringTerms(name, order, requiredSize, shardSize, minDocCount, buckets, showTermDocCountError, docCountError, otherDocCount, metaData);
}
@Override
public void readFrom(StreamInput in) throws IOException {
this.name = in.readString();
protected void doReadFrom(StreamInput in) throws IOException {
if (in.getVersion().onOrAfter(Version.V_1_4_0_Beta1)) {
this.docCountError = in.readLong();
} else {
@ -206,8 +205,7 @@ public class StringTerms extends InternalTerms {
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
protected void doWriteTo(StreamOutput out) throws IOException {
if (out.getVersion().onOrAfter(Version.V_1_4_0_Beta1)) {
out.writeLong(docCountError);
}

View file

@ -35,6 +35,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSource;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.Map;
/**
* An aggregator of string values.
@ -49,9 +50,9 @@ public class StringTermsAggregator extends AbstractStringTermsAggregator {
public StringTermsAggregator(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount,
Terms.Order order, BucketCountThresholds bucketCountThresholds,
IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError) {
IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map<String, Object> metaData) {
super(name, factories, estimatedBucketCount, aggregationContext, parent, order, bucketCountThresholds, collectionMode, showTermDocCountError);
super(name, factories, estimatedBucketCount, aggregationContext, parent, order, bucketCountThresholds, collectionMode, showTermDocCountError, metaData);
this.valuesSource = valuesSource;
this.includeExclude = includeExclude;
bucketOrds = new BytesRefHash(estimatedBucketCount, aggregationContext.bigArrays());
@ -156,7 +157,7 @@ public class StringTermsAggregator extends AbstractStringTermsAggregator {
bucket.docCountError = 0;
}
return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Arrays.asList(list), showTermDocCountError, 0, otherDocCount);
return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Arrays.asList(list), showTermDocCountError, 0, otherDocCount, getMetaData());
}
@Override

View file

@ -33,6 +33,7 @@ import org.elasticsearch.search.aggregations.support.OrderPath;
import java.io.IOException;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
public abstract class TermsAggregator extends BucketsAggregator {
@ -134,8 +135,8 @@ public abstract class TermsAggregator extends BucketsAggregator {
protected Set<Aggregator> aggsUsedForSorting = new HashSet<>();
protected SubAggCollectionMode subAggCollectMode;
public TermsAggregator(String name, BucketAggregationMode bucketAggregationMode, AggregatorFactories factories, long estimatedBucketsCount, AggregationContext context, Aggregator parent, BucketCountThresholds bucketCountThresholds, Terms.Order order, SubAggCollectionMode subAggCollectMode) {
super(name, bucketAggregationMode, factories, estimatedBucketsCount, context, parent);
public TermsAggregator(String name, BucketAggregationMode bucketAggregationMode, AggregatorFactories factories, long estimatedBucketsCount, AggregationContext context, Aggregator parent, BucketCountThresholds bucketCountThresholds, Terms.Order order, SubAggCollectionMode subAggCollectMode, Map<String, Object> metaData) {
super(name, bucketAggregationMode, factories, estimatedBucketsCount, context, parent, metaData);
this.bucketCountThresholds = bucketCountThresholds;
this.order = InternalOrder.validate(order, this);
this.subAggCollectMode = subAggCollectMode;

View file

@ -29,10 +29,12 @@ import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import java.util.Map;
/**
*
*/
public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource, Map<String, Object>> {
public enum ExecutionMode {
@ -41,8 +43,8 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
@Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount,
long maxOrd, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError) {
return new StringTermsAggregator(name, factories, valuesSource, estimatedBucketCount, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError);
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData) {
return new StringTermsAggregator(name, factories, valuesSource, estimatedBucketCount, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData);
}
@Override
@ -56,8 +58,8 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
@Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount,
long maxOrd, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError) {
return new GlobalOrdinalsStringTermsAggregator(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError);
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData) {
return new GlobalOrdinalsStringTermsAggregator(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData);
}
@Override
@ -71,8 +73,8 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
@Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount,
long maxOrd, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError) {
return new GlobalOrdinalsStringTermsAggregator.WithHash(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError);
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData) {
return new GlobalOrdinalsStringTermsAggregator.WithHash(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData);
}
@Override
@ -85,11 +87,11 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
@Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount,
long maxOrd, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError) {
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData) {
if (includeExclude != null || factories.count() > 0) {
return GLOBAL_ORDINALS.create(name, factories, valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError);
return GLOBAL_ORDINALS.create(name, factories, valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData);
}
return new GlobalOrdinalsStringTermsAggregator.LowCardinality(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, aggregationContext, parent, subAggCollectMode, showTermDocCountError);
return new GlobalOrdinalsStringTermsAggregator.LowCardinality(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData);
}
@Override
@ -115,7 +117,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
abstract Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount,
long maxOrd, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds,
IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError);
IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData);
abstract boolean needsGlobalOrdinals();
@ -143,9 +145,9 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent) {
final InternalAggregation aggregation = new UnmappedTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount());
return new NonCollectingAggregator(name, aggregationContext, parent) {
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
final InternalAggregation aggregation = new UnmappedTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), metaData);
return new NonCollectingAggregator(name, aggregationContext, parent, metaData) {
@Override
public InternalAggregation buildEmptyAggregation() {
return aggregation;
@ -178,7 +180,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
}
@Override
protected Aggregator create(ValuesSource valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent) {
protected Aggregator create(ValuesSource valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
long estimatedBucketCount = estimatedBucketCount(valuesSource, parent);
if (valuesSource instanceof ValuesSource.Bytes) {
@ -228,7 +230,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
assert execution != null;
valuesSource.setNeedsGlobalOrdinals(execution.needsGlobalOrdinals());
return execution.create(name, factories, valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError);
return execution.create(name, factories, valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData);
}
if ((includeExclude != null) && (includeExclude.isRegexBased())) {
@ -244,13 +246,13 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
}
return new DoubleTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(),
estimatedBucketCount, order, bucketCountThresholds, aggregationContext, parent, subAggCollectMode,
showTermDocCountError, longFilter);
showTermDocCountError, longFilter, metaData);
}
if (includeExclude != null) {
longFilter = includeExclude.convertToLongFilter();
}
return new LongTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(), estimatedBucketCount,
order, bucketCountThresholds, aggregationContext, parent, subAggCollectMode, showTermDocCountError, longFilter);
order, bucketCountThresholds, aggregationContext, parent, subAggCollectMode, showTermDocCountError, longFilter, metaData);
}
throw new AggregationExecutionException("terms aggregation cannot be applied to field [" + config.fieldContext().field() +

View file

@ -54,8 +54,8 @@ public class UnmappedTerms extends InternalTerms {
UnmappedTerms() {} // for serialization
public UnmappedTerms(String name, Terms.Order order, int requiredSize, int shardSize, long minDocCount) {
super(name, order, requiredSize, shardSize, minDocCount, BUCKETS, false, 0, 0);
public UnmappedTerms(String name, Terms.Order order, int requiredSize, int shardSize, long minDocCount, Map<String, Object> metaData) {
super(name, order, requiredSize, shardSize, minDocCount, BUCKETS, false, 0, 0, metaData);
}
@Override
@ -64,8 +64,7 @@ public class UnmappedTerms extends InternalTerms {
}
@Override
public void readFrom(StreamInput in) throws IOException {
this.name = in.readString();
protected void doReadFrom(StreamInput in) throws IOException {
this.docCountError = 0;
this.order = InternalOrder.Streams.readOrder(in);
this.requiredSize = readSize(in);
@ -75,8 +74,7 @@ public class UnmappedTerms extends InternalTerms {
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
protected void doWriteTo(StreamOutput out) throws IOException {
InternalOrder.Streams.writeOrder(order, out);
writeSize(requiredSize, out);
out.writeVLong(minDocCount);
@ -93,7 +91,7 @@ public class UnmappedTerms extends InternalTerms {
}
@Override
protected InternalTerms newAggregation(String name, List<Bucket> buckets, boolean showTermDocCountError, long docCountError, long otherDocCount) {
protected InternalTerms newAggregation(String name, List<Bucket> buckets, boolean showTermDocCountError, long docCountError, long otherDocCount, Map<String, Object> metaData) {
throw new UnsupportedOperationException("How did you get there?");
}

View file

@ -21,11 +21,13 @@ package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.search.aggregations.InternalAggregation;
import java.util.Map;
public abstract class InternalMetricsAggregation extends InternalAggregation {
protected InternalMetricsAggregation() {} // for serialization
protected InternalMetricsAggregation(String name) {
super(name);
protected InternalMetricsAggregation(String name, Map<String, Object> metaData) {
super(name, metaData);
}
}

View file

@ -20,6 +20,8 @@ package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
import java.util.Map;
/**
*
*/
@ -31,8 +33,8 @@ public abstract class InternalNumericMetricsAggregation extends InternalMetricsA
protected SingleValue() {}
protected SingleValue(String name) {
super(name);
protected SingleValue(String name, Map<String, Object> metaData) {
super(name, metaData);
}
public abstract double value();
@ -42,8 +44,8 @@ public abstract class InternalNumericMetricsAggregation extends InternalMetricsA
protected MultiValue() {}
protected MultiValue(String name) {
super(name);
protected MultiValue(String name, Map<String, Object> metaData) {
super(name, metaData);
}
public abstract double value(String name);
@ -52,8 +54,8 @@ public abstract class InternalNumericMetricsAggregation extends InternalMetricsA
private InternalNumericMetricsAggregation() {} // for serialization
private InternalNumericMetricsAggregation(String name) {
super(name);
private InternalNumericMetricsAggregation(String name, Map<String, Object> metaData) {
super(name, metaData);
}
}

View file

@ -23,13 +23,15 @@ import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import java.util.Map;
public abstract class MetricsAggregator extends Aggregator {
protected MetricsAggregator(String name, long estimatedBucketsCount, AggregationContext context, Aggregator parent) {
this(name, estimatedBucketsCount, BucketAggregationMode.MULTI_BUCKETS, context, parent);
protected MetricsAggregator(String name, long estimatedBucketsCount, AggregationContext context, Aggregator parent, Map<String, Object> metaData) {
this(name, estimatedBucketsCount, BucketAggregationMode.MULTI_BUCKETS, context, parent, metaData);
}
protected MetricsAggregator(String name, long estimatedBucketsCount, BucketAggregationMode bucketAggregationMode, AggregationContext context, Aggregator parent) {
super(name, bucketAggregationMode, AggregatorFactories.EMPTY, estimatedBucketsCount, context, parent);
protected MetricsAggregator(String name, long estimatedBucketsCount, BucketAggregationMode bucketAggregationMode, AggregationContext context, Aggregator parent, Map<String, Object> metaData) {
super(name, bucketAggregationMode, AggregatorFactories.EMPTY, estimatedBucketsCount, context, parent, metaData);
}
}

View file

@ -21,19 +21,21 @@ package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import java.util.Map;
/**
*
*/
public abstract class NumericMetricsAggregator extends MetricsAggregator {
private NumericMetricsAggregator(String name, long estimatedBucketsCount, AggregationContext context, Aggregator parent) {
super(name, estimatedBucketsCount, context, parent);
private NumericMetricsAggregator(String name, long estimatedBucketsCount, AggregationContext context, Aggregator parent, Map<String, Object> metaData) {
super(name, estimatedBucketsCount, context, parent, metaData);
}
public static abstract class SingleValue extends NumericMetricsAggregator {
protected SingleValue(String name, long estimatedBucketsCount, AggregationContext context, Aggregator parent) {
super(name, estimatedBucketsCount, context, parent);
protected SingleValue(String name, long estimatedBucketsCount, AggregationContext context, Aggregator parent, Map<String, Object> metaData) {
super(name, estimatedBucketsCount, context, parent, metaData);
}
public abstract double metric(long owningBucketOrd);
@ -41,8 +43,8 @@ public abstract class NumericMetricsAggregator extends MetricsAggregator {
public static abstract class MultiValue extends NumericMetricsAggregator {
protected MultiValue(String name, long estimatedBucketsCount, AggregationContext context, Aggregator parent) {
super(name, estimatedBucketsCount, context, parent);
protected MultiValue(String name, long estimatedBucketsCount, AggregationContext context, Aggregator parent, Map<String, Object> metaData) {
super(name, estimatedBucketsCount, context, parent, metaData);
}
public abstract boolean hasMetric(String name);

View file

@ -32,6 +32,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFacto
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import java.io.IOException;
import java.util.Map;
/**
*
@ -44,8 +45,8 @@ public class AvgAggregator extends NumericMetricsAggregator.SingleValue {
private LongArray counts;
private DoubleArray sums;
public AvgAggregator(String name, long estimatedBucketsCount, ValuesSource.Numeric valuesSource, AggregationContext context, Aggregator parent) {
super(name, estimatedBucketsCount, context, parent);
public AvgAggregator(String name, long estimatedBucketsCount, ValuesSource.Numeric valuesSource, AggregationContext context, Aggregator parent, Map<String, Object> metaData) {
super(name, estimatedBucketsCount, context, parent, metaData);
this.valuesSource = valuesSource;
if (valuesSource != null) {
final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount;
@ -87,30 +88,30 @@ public class AvgAggregator extends NumericMetricsAggregator.SingleValue {
@Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
if (valuesSource == null || owningBucketOrdinal >= counts.size()) {
return new InternalAvg(name, 0l, 0);
return new InternalAvg(name, 0l, 0, getMetaData());
}
return new InternalAvg(name, sums.get(owningBucketOrdinal), counts.get(owningBucketOrdinal));
return new InternalAvg(name, sums.get(owningBucketOrdinal), counts.get(owningBucketOrdinal), getMetaData());
}
@Override
public InternalAggregation buildEmptyAggregation() {
return new InternalAvg(name, 0.0, 0l);
return new InternalAvg(name, 0.0, 0l, getMetaData());
}
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric, Map<String, Object>> {
public Factory(String name, String type, ValuesSourceConfig<ValuesSource.Numeric> valuesSourceConfig) {
super(name, type, valuesSourceConfig);
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent) {
return new AvgAggregator(name, 0, null, aggregationContext, parent);
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
return new AvgAggregator(name, 0, null, aggregationContext, parent, metaData);
}
@Override
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent) {
return new AvgAggregator(name, expectedBucketsCount, valuesSource, aggregationContext, parent);
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
return new AvgAggregator(name, expectedBucketsCount, valuesSource, aggregationContext, parent, metaData);
}
}

View file

@ -27,6 +27,7 @@ import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggre
import org.elasticsearch.search.aggregations.support.format.ValueFormatterStreams;
import java.io.IOException;
import java.util.Map;
/**
*
@ -53,8 +54,8 @@ public class InternalAvg extends InternalNumericMetricsAggregation.SingleValue i
InternalAvg() {} // for serialization
public InternalAvg(String name, double sum, long count) {
super(name);
public InternalAvg(String name, double sum, long count, Map<String, Object> metaData) {
super(name, metaData);
this.sum = sum;
this.count = count;
}
@ -81,20 +82,18 @@ public class InternalAvg extends InternalNumericMetricsAggregation.SingleValue i
count += ((InternalAvg) aggregation).count;
sum += ((InternalAvg) aggregation).sum;
}
return new InternalAvg(getName(), sum, count);
return new InternalAvg(getName(), sum, count, getMetaData());
}
@Override
public void readFrom(StreamInput in) throws IOException {
name = in.readString();
protected void doReadFrom(StreamInput in) throws IOException {
valueFormatter = ValueFormatterStreams.readOptional(in);
sum = in.readDouble();
count = in.readVLong();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
protected void doWriteTo(StreamOutput out) throws IOException {
ValueFormatterStreams.writeOptional(valueFormatter, out);
out.writeDouble(sum);
out.writeVLong(count);

View file

@ -44,6 +44,7 @@ import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import java.io.IOException;
import java.util.Map;
/**
* An aggregator that computes approximate counts of unique values.
@ -61,8 +62,8 @@ public class CardinalityAggregator extends NumericMetricsAggregator.SingleValue
private Collector collector;
public CardinalityAggregator(String name, long estimatedBucketsCount, ValuesSource valuesSource, boolean rehash,
int precision, AggregationContext context, Aggregator parent) {
super(name, estimatedBucketsCount, context, parent);
int precision, AggregationContext context, Aggregator parent, Map<String, Object> metaData) {
super(name, estimatedBucketsCount, context, parent, metaData);
this.valuesSource = valuesSource;
this.rehash = rehash;
this.precision = precision;
@ -151,12 +152,12 @@ public class CardinalityAggregator extends NumericMetricsAggregator.SingleValue
// this Aggregator (and its HLL++ counters) is released.
HyperLogLogPlusPlus copy = new HyperLogLogPlusPlus(precision, BigArrays.NON_RECYCLING_INSTANCE, 1);
copy.merge(0, counts, owningBucketOrdinal);
return new InternalCardinality(name, copy);
return new InternalCardinality(name, copy, getMetaData());
}
@Override
public InternalAggregation buildEmptyAggregation() {
return new InternalCardinality(name, null);
return new InternalCardinality(name, null, getMetaData());
}
@Override

View file

@ -27,7 +27,9 @@ import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
final class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource> {
import java.util.Map;
final class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource, Map<String, Object>> {
private final long precisionThreshold;
private final boolean rehash;
@ -43,16 +45,16 @@ final class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory<V
}
@Override
protected Aggregator createUnmapped(AggregationContext context, Aggregator parent) {
return new CardinalityAggregator(name, parent == null ? 1 : parent.estimatedBucketCount(), null, true, precision(parent), context, parent);
protected Aggregator createUnmapped(AggregationContext context, Aggregator parent, Map<String, Object> metaData) {
return new CardinalityAggregator(name, parent == null ? 1 : parent.estimatedBucketCount(), null, true, precision(parent), context, parent, metaData);
}
@Override
protected Aggregator create(ValuesSource valuesSource, long expectedBucketsCount, AggregationContext context, Aggregator parent) {
protected Aggregator create(ValuesSource valuesSource, long expectedBucketsCount, AggregationContext context, Aggregator parent, Map<String, Object> metaData) {
if (!(valuesSource instanceof ValuesSource.Numeric) && !rehash) {
throw new AggregationExecutionException("Turning off rehashing for cardinality aggregation [" + name + "] on non-numeric values in not allowed");
}
return new CardinalityAggregator(name, parent == null ? 1 : parent.estimatedBucketCount(), valuesSource, rehash, precision(parent), context, parent);
return new CardinalityAggregator(name, parent == null ? 1 : parent.estimatedBucketCount(), valuesSource, rehash, precision(parent), context, parent, metaData);
}
/*

View file

@ -30,6 +30,7 @@ import org.elasticsearch.search.aggregations.support.format.ValueFormatterStream
import java.io.IOException;
import java.util.List;
import java.util.Map;
public final class InternalCardinality extends InternalNumericMetricsAggregation.SingleValue implements Cardinality {
@ -50,8 +51,8 @@ public final class InternalCardinality extends InternalNumericMetricsAggregation
private HyperLogLogPlusPlus counts;
InternalCardinality(String name, HyperLogLogPlusPlus counts) {
super(name);
InternalCardinality(String name, HyperLogLogPlusPlus counts, Map<String, Object> metaData) {
super(name, metaData);
this.counts = counts;
}
@ -74,8 +75,7 @@ public final class InternalCardinality extends InternalNumericMetricsAggregation
}
@Override
public void readFrom(StreamInput in) throws IOException {
name = in.readString();
protected void doReadFrom(StreamInput in) throws IOException {
valueFormatter = ValueFormatterStreams.readOptional(in);
if (in.readBoolean()) {
counts = HyperLogLogPlusPlus.readFrom(in, BigArrays.NON_RECYCLING_INSTANCE);
@ -85,8 +85,7 @@ public final class InternalCardinality extends InternalNumericMetricsAggregation
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
protected void doWriteTo(StreamOutput out) throws IOException {
ValueFormatterStreams.writeOptional(valueFormatter, out);
if (counts != null) {
out.writeBoolean(true);
@ -104,7 +103,7 @@ public final class InternalCardinality extends InternalNumericMetricsAggregation
final InternalCardinality cardinality = (InternalCardinality) aggregation;
if (cardinality.counts != null) {
if (reduced == null) {
reduced = new InternalCardinality(name, new HyperLogLogPlusPlus(cardinality.counts.precision(), BigArrays.NON_RECYCLING_INSTANCE, 1));
reduced = new InternalCardinality(name, new HyperLogLogPlusPlus(cardinality.counts.precision(), BigArrays.NON_RECYCLING_INSTANCE, 1), getMetaData());
}
reduced.merge(cardinality);
}

View file

@ -33,6 +33,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFacto
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import java.io.IOException;
import java.util.Map;
public final class GeoBoundsAggregator extends MetricsAggregator {
@ -47,8 +48,8 @@ public final class GeoBoundsAggregator extends MetricsAggregator {
private MultiGeoPointValues values;
protected GeoBoundsAggregator(String name, long estimatedBucketsCount,
AggregationContext aggregationContext, Aggregator parent, ValuesSource.GeoPoint valuesSource, boolean wrapLongitude) {
super(name, estimatedBucketsCount, aggregationContext, parent);
AggregationContext aggregationContext, Aggregator parent, ValuesSource.GeoPoint valuesSource, boolean wrapLongitude, Map<String, Object> metaData) {
super(name, estimatedBucketsCount, aggregationContext, parent, metaData);
this.valuesSource = valuesSource;
this.wrapLongitude = wrapLongitude;
if (valuesSource != null) {
@ -89,13 +90,13 @@ public final class GeoBoundsAggregator extends MetricsAggregator {
double posRight = posRights.get(owningBucketOrdinal);
double negLeft = negLefts.get(owningBucketOrdinal);
double negRight = negRights.get(owningBucketOrdinal);
return new InternalGeoBounds(name, top, bottom, posLeft, posRight, negLeft, negRight, wrapLongitude);
return new InternalGeoBounds(name, top, bottom, posLeft, posRight, negLeft, negRight, wrapLongitude, getMetaData());
}
@Override
public InternalAggregation buildEmptyAggregation() {
return new InternalGeoBounds(name, Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY, Double.POSITIVE_INFINITY,
Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, wrapLongitude);
Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, wrapLongitude, getMetaData());
}
@Override
@ -159,7 +160,7 @@ public final class GeoBoundsAggregator extends MetricsAggregator {
Releasables.close(tops, bottoms, posLefts, posRights, negLefts, negRights);
}
public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource.GeoPoint> {
public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource.GeoPoint, Map<String, Object>> {
private final boolean wrapLongitude;
@ -169,14 +170,14 @@ public final class GeoBoundsAggregator extends MetricsAggregator {
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent) {
return new GeoBoundsAggregator(name, 0, aggregationContext, parent, null, wrapLongitude);
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
return new GeoBoundsAggregator(name, 0, aggregationContext, parent, null, wrapLongitude, metaData);
}
@Override
protected Aggregator create(ValuesSource.GeoPoint valuesSource, long expectedBucketsCount, AggregationContext aggregationContext,
Aggregator parent) {
return new GeoBoundsAggregator(name, expectedBucketsCount, aggregationContext, parent, valuesSource, wrapLongitude);
Aggregator parent, Map<String, Object> metaData) {
return new GeoBoundsAggregator(name, expectedBucketsCount, aggregationContext, parent, valuesSource, wrapLongitude, metaData);
}
}

View file

@ -28,6 +28,7 @@ import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.metrics.InternalMetricsAggregation;
import java.io.IOException;
import java.util.Map;
public class InternalGeoBounds extends InternalMetricsAggregation implements GeoBounds {
@ -53,8 +54,8 @@ public class InternalGeoBounds extends InternalMetricsAggregation implements Geo
}
InternalGeoBounds(String name, double top, double bottom, double posLeft, double posRight,
double negLeft, double negRight, boolean wrapLongitude) {
super(name);
double negLeft, double negRight, boolean wrapLongitude, Map<String, Object> metaData) {
super(name, metaData);
this.top = top;
this.bottom = bottom;
this.posLeft = posLeft;
@ -100,7 +101,7 @@ public class InternalGeoBounds extends InternalMetricsAggregation implements Geo
negRight = bounds.negRight;
}
}
return new InternalGeoBounds(name, top, bottom, posLeft, posRight, negLeft, negRight, wrapLongitude);
return new InternalGeoBounds(name, top, bottom, posLeft, posRight, negLeft, negRight, wrapLongitude, getMetaData());
}
@Override
@ -123,8 +124,7 @@ public class InternalGeoBounds extends InternalMetricsAggregation implements Geo
}
@Override
public void readFrom(StreamInput in) throws IOException {
name = in.readString();
protected void doReadFrom(StreamInput in) throws IOException {
top = in.readDouble();
bottom = in.readDouble();
posLeft = in.readDouble();
@ -135,8 +135,7 @@ public class InternalGeoBounds extends InternalMetricsAggregation implements Geo
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
protected void doWriteTo(StreamOutput out) throws IOException {
out.writeDouble(top);
out.writeDouble(bottom);
out.writeDouble(posLeft);

View file

@ -27,6 +27,7 @@ import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggre
import org.elasticsearch.search.aggregations.support.format.ValueFormatterStreams;
import java.io.IOException;
import java.util.Map;
/**
*
@ -52,8 +53,8 @@ public class InternalMax extends InternalNumericMetricsAggregation.SingleValue i
InternalMax() {} // for serialization
public InternalMax(String name, double max) {
super(name);
public InternalMax(String name, double max, Map<String, Object> metaData) {
super(name, metaData);
this.max = max;
}
@ -77,19 +78,17 @@ public class InternalMax extends InternalNumericMetricsAggregation.SingleValue i
for (InternalAggregation aggregation : reduceContext.aggregations()) {
max = Math.max(max, ((InternalMax) aggregation).max);
}
return new InternalMax(name, max);
return new InternalMax(name, max, getMetaData());
}
@Override
public void readFrom(StreamInput in) throws IOException {
name = in.readString();
protected void doReadFrom(StreamInput in) throws IOException {
valueFormatter = ValueFormatterStreams.readOptional(in);
max = in.readDouble();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
protected void doWriteTo(StreamOutput out) throws IOException {
ValueFormatterStreams.writeOptional(valueFormatter, out);
out.writeDouble(max);
}

View file

@ -33,6 +33,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFacto
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import java.io.IOException;
import java.util.Map;
/**
*
@ -44,8 +45,8 @@ public class MaxAggregator extends NumericMetricsAggregator.SingleValue {
private DoubleArray maxes;
public MaxAggregator(String name, long estimatedBucketsCount, ValuesSource.Numeric valuesSource, AggregationContext context, Aggregator parent) {
super(name, estimatedBucketsCount, context, parent);
public MaxAggregator(String name, long estimatedBucketsCount, ValuesSource.Numeric valuesSource, AggregationContext context, Aggregator parent, Map<String, Object> metaData) {
super(name, estimatedBucketsCount, context, parent, metaData);
this.valuesSource = valuesSource;
if (valuesSource != null) {
final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount;
@ -86,31 +87,31 @@ public class MaxAggregator extends NumericMetricsAggregator.SingleValue {
@Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
if (valuesSource == null) {
return new InternalMax(name, Double.NEGATIVE_INFINITY);
return new InternalMax(name, Double.NEGATIVE_INFINITY, getMetaData());
}
assert owningBucketOrdinal < maxes.size();
return new InternalMax(name, maxes.get(owningBucketOrdinal));
return new InternalMax(name, maxes.get(owningBucketOrdinal), getMetaData());
}
@Override
public InternalAggregation buildEmptyAggregation() {
return new InternalMax(name, Double.NEGATIVE_INFINITY);
return new InternalMax(name, Double.NEGATIVE_INFINITY, getMetaData());
}
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric, Map<String, Object>> {
public Factory(String name, ValuesSourceConfig<ValuesSource.Numeric> valuesSourceConfig) {
super(name, InternalMax.TYPE.name(), valuesSourceConfig);
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent) {
return new MaxAggregator(name, 0, null, aggregationContext, parent);
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
return new MaxAggregator(name, 0, null, aggregationContext, parent, metaData);
}
@Override
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent) {
return new MaxAggregator(name, expectedBucketsCount, valuesSource, aggregationContext, parent);
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
return new MaxAggregator(name, expectedBucketsCount, valuesSource, aggregationContext, parent, metaData);
}
}

View file

@ -27,6 +27,7 @@ import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggre
import org.elasticsearch.search.aggregations.support.format.ValueFormatterStreams;
import java.io.IOException;
import java.util.Map;
/**
*
@ -53,8 +54,8 @@ public class InternalMin extends InternalNumericMetricsAggregation.SingleValue i
InternalMin() {} // for serialization
public InternalMin(String name, double min) {
super(name);
public InternalMin(String name, double min, Map<String, Object> metaData) {
super(name, metaData);
this.min = min;
}
@ -78,19 +79,17 @@ public class InternalMin extends InternalNumericMetricsAggregation.SingleValue i
for (InternalAggregation aggregation : reduceContext.aggregations()) {
min = Math.min(min, ((InternalMin) aggregation).min);
}
return new InternalMin(getName(), min);
return new InternalMin(getName(), min, getMetaData());
}
@Override
public void readFrom(StreamInput in) throws IOException {
name = in.readString();
protected void doReadFrom(StreamInput in) throws IOException {
valueFormatter = ValueFormatterStreams.readOptional(in);
min = in.readDouble();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
protected void doWriteTo(StreamOutput out) throws IOException {
ValueFormatterStreams.writeOptional(valueFormatter, out);
out.writeDouble(min);
}

View file

@ -33,6 +33,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFacto
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import java.io.IOException;
import java.util.Map;
/**
*
@ -44,8 +45,8 @@ public class MinAggregator extends NumericMetricsAggregator.SingleValue {
private DoubleArray mins;
public MinAggregator(String name, long estimatedBucketsCount, ValuesSource.Numeric valuesSource, AggregationContext context, Aggregator parent) {
super(name, estimatedBucketsCount, context, parent);
public MinAggregator(String name, long estimatedBucketsCount, ValuesSource.Numeric valuesSource, AggregationContext context, Aggregator parent, Map<String, Object> metaData) {
super(name, estimatedBucketsCount, context, parent, metaData);
this.valuesSource = valuesSource;
if (valuesSource != null) {
final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount;
@ -86,31 +87,31 @@ public class MinAggregator extends NumericMetricsAggregator.SingleValue {
@Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
if (valuesSource == null) {
return new InternalMin(name, Double.POSITIVE_INFINITY);
return new InternalMin(name, Double.POSITIVE_INFINITY, getMetaData());
}
assert owningBucketOrdinal < mins.size();
return new InternalMin(name, mins.get(owningBucketOrdinal));
return new InternalMin(name, mins.get(owningBucketOrdinal), getMetaData());
}
@Override
public InternalAggregation buildEmptyAggregation() {
return new InternalMin(name, Double.POSITIVE_INFINITY);
return new InternalMin(name, Double.POSITIVE_INFINITY, getMetaData());
}
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric, Map<String,Object>> {
public Factory(String name, ValuesSourceConfig<ValuesSource.Numeric> valuesSourceConfig) {
super(name, InternalMin.TYPE.name(), valuesSourceConfig);
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent) {
return new MinAggregator(name, 0, null, aggregationContext, parent);
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
return new MinAggregator(name, 0, null, aggregationContext, parent, metaData);
}
@Override
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent) {
return new MinAggregator(name, expectedBucketsCount, valuesSource, aggregationContext, parent);
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
return new MinAggregator(name, expectedBucketsCount, valuesSource, aggregationContext, parent, metaData);
}
}

View file

@ -31,6 +31,7 @@ import org.elasticsearch.search.aggregations.support.format.ValueFormatterStream
import java.io.IOException;
import java.util.List;
import java.util.Map;
abstract class AbstractInternalPercentiles extends InternalNumericMetricsAggregation.MultiValue {
@ -40,8 +41,8 @@ abstract class AbstractInternalPercentiles extends InternalNumericMetricsAggrega
AbstractInternalPercentiles() {} // for serialization
public AbstractInternalPercentiles(String name, double[] keys, TDigestState state, boolean keyed) {
super(name);
public AbstractInternalPercentiles(String name, double[] keys, TDigestState state, boolean keyed, Map<String, Object> metaData) {
super(name, metaData);
this.keys = keys;
this.state = state;
this.keyed = keyed;
@ -65,14 +66,13 @@ abstract class AbstractInternalPercentiles extends InternalNumericMetricsAggrega
}
merged.add(percentiles.state);
}
return createReduced(getName(), keys, merged, keyed);
return createReduced(getName(), keys, merged, keyed, getMetaData());
}
protected abstract AbstractInternalPercentiles createReduced(String name, double[] keys, TDigestState merged, boolean keyed);
protected abstract AbstractInternalPercentiles createReduced(String name, double[] keys, TDigestState merged, boolean keyed, Map<String, Object> metaData);
@Override
public void readFrom(StreamInput in) throws IOException {
name = in.readString();
protected void doReadFrom(StreamInput in) throws IOException {
valueFormatter = ValueFormatterStreams.readOptional(in);
if (in.getVersion().before(Version.V_1_2_0)) {
final byte id = in.readByte();
@ -89,8 +89,7 @@ abstract class AbstractInternalPercentiles extends InternalNumericMetricsAggrega
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
protected void doWriteTo(StreamOutput out) throws IOException {
ValueFormatterStreams.writeOptional(valueFormatter, out);
if (out.getVersion().before(Version.V_1_2_0)) {
out.writeByte((byte) 0);

View file

@ -31,6 +31,7 @@ import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import java.io.IOException;
import java.util.Map;
public abstract class AbstractPercentilesAggregator extends NumericMetricsAggregator.MultiValue {
@ -46,8 +47,8 @@ public abstract class AbstractPercentilesAggregator extends NumericMetricsAggreg
protected final boolean keyed;
public AbstractPercentilesAggregator(String name, long estimatedBucketsCount, ValuesSource.Numeric valuesSource, AggregationContext context,
Aggregator parent, double[] keys, double compression, boolean keyed) {
super(name, estimatedBucketsCount, context, parent);
Aggregator parent, double[] keys, double compression, boolean keyed, Map<String, Object> metaData) {
super(name, estimatedBucketsCount, context, parent, metaData);
this.valuesSource = valuesSource;
this.keyed = keyed;
this.states = bigArrays.newObjectArray(estimatedBucketsCount);

View file

@ -25,6 +25,7 @@ import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.TDigest
import java.io.IOException;
import java.util.Iterator;
import java.util.Map;
/**
*
@ -48,8 +49,8 @@ public class InternalPercentileRanks extends AbstractInternalPercentiles impleme
InternalPercentileRanks() {} // for serialization
public InternalPercentileRanks(String name, double[] cdfValues, TDigestState state, boolean keyed) {
super(name, cdfValues, state, keyed);
public InternalPercentileRanks(String name, double[] cdfValues, TDigestState state, boolean keyed, Map<String, Object> metaData) {
super(name, cdfValues, state, keyed, metaData);
}
@Override
@ -67,8 +68,8 @@ public class InternalPercentileRanks extends AbstractInternalPercentiles impleme
return percent(key);
}
protected AbstractInternalPercentiles createReduced(String name, double[] keys, TDigestState merged, boolean keyed) {
return new InternalPercentileRanks(name, keys, merged, keyed);
protected AbstractInternalPercentiles createReduced(String name, double[] keys, TDigestState merged, boolean keyed, Map<String, Object> metaData) {
return new InternalPercentileRanks(name, keys, merged, keyed, metaData);
}
@Override

View file

@ -25,6 +25,7 @@ import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.TDigest
import java.io.IOException;
import java.util.Iterator;
import java.util.Map;
/**
*
@ -48,8 +49,8 @@ public class InternalPercentiles extends AbstractInternalPercentiles implements
InternalPercentiles() {} // for serialization
public InternalPercentiles(String name, double[] percents, TDigestState state, boolean keyed) {
super(name, percents, state, keyed);
public InternalPercentiles(String name, double[] percents, TDigestState state, boolean keyed, Map<String, Object> metaData) {
super(name, percents, state, keyed, metaData);
}
@Override
@ -67,8 +68,8 @@ public class InternalPercentiles extends AbstractInternalPercentiles implements
return percentile(key);
}
protected AbstractInternalPercentiles createReduced(String name, double[] keys, TDigestState merged, boolean keyed) {
return new InternalPercentiles(name, keys, merged, keyed);
protected AbstractInternalPercentiles createReduced(String name, double[] keys, TDigestState merged, boolean keyed, Map<String, Object> metaData) {
return new InternalPercentiles(name, keys, merged, keyed, metaData);
}
@Override

View file

@ -24,14 +24,16 @@ import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.TDigest
import org.elasticsearch.search.aggregations.support.*;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import java.util.Map;
/**
*
*/
public class PercentileRanksAggregator extends AbstractPercentilesAggregator {
public PercentileRanksAggregator(String name, long estimatedBucketsCount, Numeric valuesSource, AggregationContext context,
Aggregator parent, double[] percents, double compression, boolean keyed) {
super(name, estimatedBucketsCount, valuesSource, context, parent, percents, compression, keyed);
Aggregator parent, double[] percents, double compression, boolean keyed, Map<String, Object> metaData) {
super(name, estimatedBucketsCount, valuesSource, context, parent, percents, compression, keyed, metaData);
}
@Override
@ -40,13 +42,13 @@ public class PercentileRanksAggregator extends AbstractPercentilesAggregator {
if (state == null) {
return buildEmptyAggregation();
} else {
return new InternalPercentileRanks(name, keys, state, keyed);
return new InternalPercentileRanks(name, keys, state, keyed, getMetaData());
}
}
@Override
public InternalAggregation buildEmptyAggregation() {
return new InternalPercentileRanks(name, keys, new TDigestState(compression), keyed);
return new InternalPercentileRanks(name, keys, new TDigestState(compression), keyed, getMetaData());
}
@Override
@ -59,7 +61,7 @@ public class PercentileRanksAggregator extends AbstractPercentilesAggregator {
}
}
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric, Map<String, Object>> {
private final double[] values;
private final double compression;
@ -74,13 +76,13 @@ public class PercentileRanksAggregator extends AbstractPercentilesAggregator {
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent) {
return new PercentileRanksAggregator(name, 0, null, aggregationContext, parent, values, compression, keyed);
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
return new PercentileRanksAggregator(name, 0, null, aggregationContext, parent, values, compression, keyed, metaData);
}
@Override
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent) {
return new PercentileRanksAggregator(name, expectedBucketsCount, valuesSource, aggregationContext, parent, values, compression, keyed);
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
return new PercentileRanksAggregator(name, expectedBucketsCount, valuesSource, aggregationContext, parent, values, compression, keyed, metaData);
}
}
}

View file

@ -24,14 +24,16 @@ import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.TDigest
import org.elasticsearch.search.aggregations.support.*;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import java.util.Map;
/**
*
*/
public class PercentilesAggregator extends AbstractPercentilesAggregator {
public PercentilesAggregator(String name, long estimatedBucketsCount, Numeric valuesSource, AggregationContext context,
Aggregator parent, double[] percents, double compression, boolean keyed) {
super(name, estimatedBucketsCount, valuesSource, context, parent, percents, compression, keyed);
Aggregator parent, double[] percents, double compression, boolean keyed, Map<String, Object> metaData) {
super(name, estimatedBucketsCount, valuesSource, context, parent, percents, compression, keyed, metaData);
}
@Override
@ -40,7 +42,7 @@ public class PercentilesAggregator extends AbstractPercentilesAggregator {
if (state == null) {
return buildEmptyAggregation();
} else {
return new InternalPercentiles(name, keys, state, keyed);
return new InternalPercentiles(name, keys, state, keyed, getMetaData());
}
}
@ -56,10 +58,10 @@ public class PercentilesAggregator extends AbstractPercentilesAggregator {
@Override
public InternalAggregation buildEmptyAggregation() {
return new InternalPercentiles(name, keys, new TDigestState(compression), keyed);
return new InternalPercentiles(name, keys, new TDigestState(compression), keyed, getMetaData());
}
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric, Map<String, Object>> {
private final double[] percents;
private final double compression;
@ -74,13 +76,13 @@ public class PercentilesAggregator extends AbstractPercentilesAggregator {
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent) {
return new PercentilesAggregator(name, 0, null, aggregationContext, parent, percents, compression, keyed);
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
return new PercentilesAggregator(name, 0, null, aggregationContext, parent, percents, compression, keyed, metaData);
}
@Override
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent) {
return new PercentilesAggregator(name, expectedBucketsCount, valuesSource, aggregationContext, parent, percents, compression, keyed);
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
return new PercentilesAggregator(name, expectedBucketsCount, valuesSource, aggregationContext, parent, percents, compression, keyed, metaData);
}
}
}

View file

@ -60,13 +60,13 @@ public class InternalScriptedMetric extends InternalMetricsAggregation implement
private InternalScriptedMetric() {
}
private InternalScriptedMetric(String name) {
super(name);
private InternalScriptedMetric(String name, Map<String, Object> metaData) {
super(name, metaData);
}
public InternalScriptedMetric(String name, Object aggregation, String scriptLang, ScriptType scriptType, String reduceScript,
Map<String, Object> reduceParams) {
this(name);
Map<String, Object> reduceParams, Map<String, Object> metaData) {
this(name, metaData);
this.aggregation = aggregation;
this.scriptType = scriptType;
this.reduceScript = reduceScript;
@ -103,7 +103,7 @@ public class InternalScriptedMetric extends InternalMetricsAggregation implement
aggregation = aggregationObjects;
}
return new InternalScriptedMetric(firstAggregation.getName(), aggregation, firstAggregation.scriptLang, firstAggregation.scriptType,
firstAggregation.reduceScript, firstAggregation.reduceParams);
firstAggregation.reduceScript, firstAggregation.reduceParams, getMetaData());
}
@ -113,8 +113,7 @@ public class InternalScriptedMetric extends InternalMetricsAggregation implement
}
@Override
public void readFrom(StreamInput in) throws IOException {
name = in.readString();
protected void doReadFrom(StreamInput in) throws IOException {
scriptLang = in.readOptionalString();
scriptType = ScriptType.readFrom(in);
reduceScript = in.readOptionalString();
@ -123,8 +122,7 @@ public class InternalScriptedMetric extends InternalMetricsAggregation implement
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
protected void doWriteTo(StreamOutput out) throws IOException {
out.writeOptionalString(scriptLang);
ScriptType.writeTo(scriptType, out);
out.writeOptionalString(reduceScript);

View file

@ -52,8 +52,8 @@ public class ScriptedMetricAggregator extends MetricsAggregator {
protected ScriptedMetricAggregator(String name, String scriptLang, ScriptType initScriptType, String initScript,
ScriptType mapScriptType, String mapScript, ScriptType combineScriptType, String combineScript, ScriptType reduceScriptType,
String reduceScript, Map<String, Object> params, Map<String, Object> reduceParams, AggregationContext context, Aggregator parent) {
super(name, 1, BucketAggregationMode.PER_BUCKET, context, parent);
String reduceScript, Map<String, Object> params, Map<String, Object> reduceParams, AggregationContext context, Aggregator parent, Map<String, Object> metaData) {
super(name, 1, BucketAggregationMode.PER_BUCKET, context, parent, metaData);
this.scriptService = context.searchContext().scriptService();
this.scriptLang = scriptLang;
this.reduceScriptType = reduceScriptType;
@ -104,12 +104,12 @@ public class ScriptedMetricAggregator extends MetricsAggregator {
} else {
aggregation = params.get("_agg");
}
return new InternalScriptedMetric(name, aggregation, scriptLang, reduceScriptType, reduceScript, reduceParams);
return new InternalScriptedMetric(name, aggregation, scriptLang, reduceScriptType, reduceScript, reduceParams, getMetaData());
}
@Override
public InternalAggregation buildEmptyAggregation() {
return new InternalScriptedMetric(name, null, scriptLang, reduceScriptType, reduceScript, reduceParams);
return new InternalScriptedMetric(name, null, scriptLang, reduceScriptType, reduceScript, reduceParams, getMetaData());
}
public static class Factory extends AggregatorFactory {
@ -143,7 +143,7 @@ public class ScriptedMetricAggregator extends MetricsAggregator {
}
@Override
public Aggregator create(AggregationContext context, Aggregator parent, long expectedBucketsCount) {
public Aggregator createInternal(AggregationContext context, Aggregator parent, long expectedBucketsCount, Map<String, Object> metaData) {
Map<String, Object> params = null;
if (this.params != null) {
params = deepCopyParams(this.params, context.searchContext());
@ -153,7 +153,7 @@ public class ScriptedMetricAggregator extends MetricsAggregator {
reduceParams = deepCopyParams(this.reduceParams, context.searchContext());
}
return new ScriptedMetricAggregator(name, scriptLang, initScriptType, initScript, mapScriptType, mapScript, combineScriptType,
combineScript, reduceScriptType, reduceScript, params, reduceParams, context, parent);
combineScript, reduceScriptType, reduceScript, params, reduceParams, context, parent, metaData);
}
@SuppressWarnings({ "unchecked" })

View file

@ -28,6 +28,7 @@ import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggre
import org.elasticsearch.search.aggregations.support.format.ValueFormatterStreams;
import java.io.IOException;
import java.util.Map;
/**
*
@ -65,8 +66,8 @@ public class InternalStats extends InternalNumericMetricsAggregation.MultiValue
protected InternalStats() {} // for serialization
public InternalStats(String name, long count, double sum, double min, double max) {
super(name);
public InternalStats(String name, long count, double sum, double min, double max, Map<String, Object> metaData) {
super(name, metaData);
this.count = count;
this.sum = sum;
this.min = min;
@ -130,12 +131,11 @@ public class InternalStats extends InternalNumericMetricsAggregation.MultiValue
max = Math.max(max, stats.getMax());
sum += stats.getSum();
}
return new InternalStats(name, count, sum, min, max);
return new InternalStats(name, count, sum, min, max, getMetaData());
}
@Override
public void readFrom(StreamInput in) throws IOException {
name = in.readString();
protected void doReadFrom(StreamInput in) throws IOException {
valueFormatter = ValueFormatterStreams.readOptional(in);
count = in.readVLong();
min = in.readDouble();
@ -148,8 +148,7 @@ public class InternalStats extends InternalNumericMetricsAggregation.MultiValue
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
protected void doWriteTo(StreamOutput out) throws IOException {
ValueFormatterStreams.writeOptional(valueFormatter, out);
out.writeVLong(count);
out.writeDouble(min);

View file

@ -34,6 +34,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFacto
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import java.io.IOException;
import java.util.Map;
/**
*
@ -48,8 +49,8 @@ public class StatsAggegator extends NumericMetricsAggregator.MultiValue {
private DoubleArray mins;
private DoubleArray maxes;
public StatsAggegator(String name, long estimatedBucketsCount, ValuesSource.Numeric valuesSource, AggregationContext context, Aggregator parent) {
super(name, estimatedBucketsCount, context, parent);
public StatsAggegator(String name, long estimatedBucketsCount, ValuesSource.Numeric valuesSource, AggregationContext context, Aggregator parent, Map<String, Object> metaData) {
super(name, estimatedBucketsCount, context, parent, metaData);
this.valuesSource = valuesSource;
if (valuesSource != null) {
final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount;
@ -128,31 +129,31 @@ public class StatsAggegator extends NumericMetricsAggregator.MultiValue {
@Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
if (valuesSource == null) {
return new InternalStats(name, 0, 0, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY);
return new InternalStats(name, 0, 0, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, getMetaData());
}
assert owningBucketOrdinal < counts.size();
return new InternalStats(name, counts.get(owningBucketOrdinal), sums.get(owningBucketOrdinal), mins.get(owningBucketOrdinal), maxes.get(owningBucketOrdinal));
return new InternalStats(name, counts.get(owningBucketOrdinal), sums.get(owningBucketOrdinal), mins.get(owningBucketOrdinal), maxes.get(owningBucketOrdinal), getMetaData());
}
@Override
public InternalAggregation buildEmptyAggregation() {
return new InternalStats(name, 0, 0, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY);
return new InternalStats(name, 0, 0, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, getMetaData());
}
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric, Map<String, Object>> {
public Factory(String name, ValuesSourceConfig<ValuesSource.Numeric> valuesSourceConfig) {
super(name, InternalStats.TYPE.name(), valuesSourceConfig);
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent) {
return new StatsAggegator(name, 0, null, aggregationContext, parent);
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
return new StatsAggegator(name, 0, null, aggregationContext, parent, metaData);
}
@Override
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent) {
return new StatsAggegator(name, expectedBucketsCount, valuesSource, aggregationContext, parent);
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
return new StatsAggegator(name, expectedBucketsCount, valuesSource, aggregationContext, parent, metaData);
}
}

View file

@ -34,6 +34,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFacto
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import java.io.IOException;
import java.util.Map;
/**
*
@ -49,8 +50,8 @@ public class ExtendedStatsAggregator extends NumericMetricsAggregator.MultiValue
private DoubleArray maxes;
private DoubleArray sumOfSqrs;
public ExtendedStatsAggregator(String name, long estimatedBucketsCount, ValuesSource.Numeric valuesSource, AggregationContext context, Aggregator parent) {
super(name, estimatedBucketsCount, context, parent);
public ExtendedStatsAggregator(String name, long estimatedBucketsCount, ValuesSource.Numeric valuesSource, AggregationContext context, Aggregator parent, Map<String, Object> metaData) {
super(name, estimatedBucketsCount, context, parent, metaData);
this.valuesSource = valuesSource;
if (valuesSource != null) {
final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount;
@ -143,16 +144,16 @@ public class ExtendedStatsAggregator extends NumericMetricsAggregator.MultiValue
@Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
if (valuesSource == null) {
return new InternalExtendedStats(name, 0, 0d, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, 0d);
return new InternalExtendedStats(name, 0, 0d, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, 0d, getMetaData());
}
assert owningBucketOrdinal < counts.size();
return new InternalExtendedStats(name, counts.get(owningBucketOrdinal), sums.get(owningBucketOrdinal), mins.get(owningBucketOrdinal),
maxes.get(owningBucketOrdinal), sumOfSqrs.get(owningBucketOrdinal));
maxes.get(owningBucketOrdinal), sumOfSqrs.get(owningBucketOrdinal), getMetaData());
}
@Override
public InternalAggregation buildEmptyAggregation() {
return new InternalExtendedStats(name, 0, 0d, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, 0d);
return new InternalExtendedStats(name, 0, 0d, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, 0d, getMetaData());
}
@Override
@ -160,20 +161,20 @@ public class ExtendedStatsAggregator extends NumericMetricsAggregator.MultiValue
Releasables.close(counts, maxes, mins, sumOfSqrs, sums);
}
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric, Map<String, Object>> {
public Factory(String name, ValuesSourceConfig<ValuesSource.Numeric> valuesSourceConfig) {
super(name, InternalExtendedStats.TYPE.name(), valuesSourceConfig);
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent) {
return new ExtendedStatsAggregator(name, 0, null, aggregationContext, parent);
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
return new ExtendedStatsAggregator(name, 0, null, aggregationContext, parent, metaData);
}
@Override
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent) {
return new ExtendedStatsAggregator(name, expectedBucketsCount, valuesSource, aggregationContext, parent);
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
return new ExtendedStatsAggregator(name, expectedBucketsCount, valuesSource, aggregationContext, parent, metaData);
}
}
}

View file

@ -27,6 +27,7 @@ import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.metrics.stats.InternalStats;
import java.io.IOException;
import java.util.Map;
/**
*
@ -61,8 +62,8 @@ public class InternalExtendedStats extends InternalStats implements ExtendedStat
InternalExtendedStats() {} // for serialization
public InternalExtendedStats(String name, long count, double sum, double min, double max, double sumOfSqrs) {
super(name, count, sum, min, max);
public InternalExtendedStats(String name, long count, double sum, double min, double max, double sumOfSqrs, Map<String, Object> metaData) {
super(name, count, sum, min, max, metaData);
this.sumOfSqrs = sumOfSqrs;
}
@ -108,7 +109,7 @@ public class InternalExtendedStats extends InternalStats implements ExtendedStat
sumOfSqrs += stats.getSumOfSquares();
}
final InternalStats stats = super.reduce(reduceContext);
return new InternalExtendedStats(name, stats.getCount(), stats.getSum(), stats.getMin(), stats.getMax(), sumOfSqrs);
return new InternalExtendedStats(name, stats.getCount(), stats.getSum(), stats.getMin(), stats.getMax(), sumOfSqrs, getMetaData());
}
@Override

View file

@ -27,6 +27,7 @@ import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggre
import org.elasticsearch.search.aggregations.support.format.ValueFormatterStreams;
import java.io.IOException;
import java.util.Map;
/**
*
@ -52,8 +53,8 @@ public class InternalSum extends InternalNumericMetricsAggregation.SingleValue i
InternalSum() {} // for serialization
InternalSum(String name, double sum) {
super(name);
InternalSum(String name, double sum, Map<String, Object> metaData){
super(name, metaData);
this.sum = sum;
}
@ -77,19 +78,17 @@ public class InternalSum extends InternalNumericMetricsAggregation.SingleValue i
for (InternalAggregation aggregation : reduceContext.aggregations()) {
sum += ((InternalSum) aggregation).sum;
}
return new InternalSum(name, sum);
return new InternalSum(name, sum, getMetaData());
}
@Override
public void readFrom(StreamInput in) throws IOException {
name = in.readString();
protected void doReadFrom(StreamInput in) throws IOException {
valueFormatter = ValueFormatterStreams.readOptional(in);
sum = in.readDouble();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
protected void doWriteTo(StreamOutput out) throws IOException {
ValueFormatterStreams.writeOptional(valueFormatter, out);
out.writeDouble(sum);
}

View file

@ -31,6 +31,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFacto
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import java.io.IOException;
import java.util.Map;
/**
*
@ -42,8 +43,8 @@ public class SumAggregator extends NumericMetricsAggregator.SingleValue {
private DoubleArray sums;
public SumAggregator(String name, long estimatedBucketsCount, ValuesSource.Numeric valuesSource, AggregationContext context, Aggregator parent) {
super(name, estimatedBucketsCount, context, parent);
public SumAggregator(String name, long estimatedBucketsCount, ValuesSource.Numeric valuesSource, AggregationContext context, Aggregator parent, Map<String, Object> metaData) {
super(name, estimatedBucketsCount, context, parent, metaData);
this.valuesSource = valuesSource;
if (valuesSource != null) {
final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount;
@ -81,30 +82,30 @@ public class SumAggregator extends NumericMetricsAggregator.SingleValue {
@Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
if (valuesSource == null) {
return new InternalSum(name, 0);
return new InternalSum(name, 0, getMetaData());
}
return new InternalSum(name, sums.get(owningBucketOrdinal));
return new InternalSum(name, sums.get(owningBucketOrdinal), getMetaData());
}
@Override
public InternalAggregation buildEmptyAggregation() {
return new InternalSum(name, 0.0);
return new InternalSum(name, 0.0, getMetaData());
}
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric, Map<String, Object>> {
public Factory(String name, ValuesSourceConfig<ValuesSource.Numeric> valuesSourceConfig) {
super(name, InternalSum.TYPE.name(), valuesSourceConfig);
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent) {
return new SumAggregator(name, 0, null, aggregationContext, parent);
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
return new SumAggregator(name, 0, null, aggregationContext, parent, metaData);
}
@Override
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent) {
return new SumAggregator(name, expectedBucketsCount, valuesSource, aggregationContext, parent);
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) {
return new SumAggregator(name, expectedBucketsCount, valuesSource, aggregationContext, parent, metaData);
}
}

View file

@ -36,6 +36,7 @@ import org.elasticsearch.search.internal.InternalSearchHits;
import java.io.IOException;
import java.util.List;
import java.util.Map;
/**
*/
@ -129,8 +130,7 @@ public class InternalTopHits extends InternalMetricsAggregation implements TopHi
}
@Override
public void readFrom(StreamInput in) throws IOException {
name = in.readString();
protected void doReadFrom(StreamInput in) throws IOException {
from = in.readVInt();
size = in.readVInt();
topDocs = Lucene.readTopDocs(in);
@ -138,8 +138,7 @@ public class InternalTopHits extends InternalMetricsAggregation implements TopHi
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
protected void doWriteTo(StreamOutput out) throws IOException {
out.writeVInt(from);
out.writeVInt(size);
Lucene.writeTopDocs(out, topDocs, 0);

View file

@ -35,6 +35,7 @@ import org.elasticsearch.search.internal.InternalSearchHit;
import org.elasticsearch.search.internal.InternalSearchHits;
import java.io.IOException;
import java.util.Map;
/**
*/
@ -57,8 +58,8 @@ public class TopHitsAggregator extends MetricsAggregator implements ScorerAware
private Scorer currentScorer;
private LeafReaderContext currentContext;
public TopHitsAggregator(FetchPhase fetchPhase, TopHitsContext topHitsContext, String name, long estimatedBucketsCount, AggregationContext context, Aggregator parent) {
super(name, estimatedBucketsCount, context, parent);
public TopHitsAggregator(FetchPhase fetchPhase, TopHitsContext topHitsContext, String name, long estimatedBucketsCount, AggregationContext context, Aggregator parent, Map<String, Object> metaData) {
super(name, estimatedBucketsCount, context, parent, metaData);
this.fetchPhase = fetchPhase;
topDocsCollectors = new LongObjectPagedHashMap<>(estimatedBucketsCount, context.bigArrays());
this.topHitsContext = topHitsContext;
@ -165,8 +166,8 @@ public class TopHitsAggregator extends MetricsAggregator implements ScorerAware
}
@Override
public Aggregator create(AggregationContext aggregationContext, Aggregator parent, long expectedBucketsCount) {
return new TopHitsAggregator(fetchPhase, topHitsContext, name, expectedBucketsCount, aggregationContext, parent);
public Aggregator createInternal(AggregationContext aggregationContext, Aggregator parent, long expectedBucketsCount, Map<String, Object> metaData) {
return new TopHitsAggregator(fetchPhase, topHitsContext, name, expectedBucketsCount, aggregationContext, parent, metaData);
}
@Override

View file

@ -26,6 +26,7 @@ import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation;
import java.io.IOException;
import java.util.Map;
/**
* An internal implementation of {@link ValueCount}.
@ -51,8 +52,8 @@ public class InternalValueCount extends InternalNumericMetricsAggregation.Single
InternalValueCount() {} // for serialization
public InternalValueCount(String name, long value) {
super(name);
public InternalValueCount(String name, long value, Map<String, Object> metaData) {
super(name, metaData);
this.value = value;
}
@ -77,18 +78,16 @@ public class InternalValueCount extends InternalNumericMetricsAggregation.Single
for (InternalAggregation aggregation : reduceContext.aggregations()) {
valueCount += ((InternalValueCount) aggregation).value;
}
return new InternalValueCount(name, valueCount);
return new InternalValueCount(name, valueCount, getMetaData());
}
@Override
public void readFrom(StreamInput in) throws IOException {
name = in.readString();
protected void doReadFrom(StreamInput in) throws IOException {
value = in.readVLong();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
protected void doWriteTo(StreamOutput out) throws IOException {
out.writeVLong(value);
}

Some files were not shown because too many files have changed in this diff Show more