mirror of
https://github.com/elastic/elasticsearch.git
synced 2025-06-29 09:54:06 -04:00
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:
commit
93826e4d56
109 changed files with 1355 additions and 742 deletions
|
@ -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]
|
||||
|
|
|
@ -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[]
|
||||
|
|
2
pom.xml
2
pom.xml
|
@ -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>
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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,9 +1359,18 @@ 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";
|
||||
}
|
||||
|
|
|
@ -185,51 +185,55 @@ public class MulticastZenPing extends AbstractLifecycleComponent<ZenPing> implem
|
|||
return;
|
||||
}
|
||||
final int id = pingIdGenerator.incrementAndGet();
|
||||
receivedResponses.put(id, new PingCollection());
|
||||
sendPingRequest(id);
|
||||
// try and send another ping request halfway through (just in case someone woke up during it...)
|
||||
// this can be a good trade-off to nailing the initial lookup or un-delivered messages
|
||||
threadPool.schedule(TimeValue.timeValueMillis(timeout.millis() / 2), ThreadPool.Names.GENERIC, new AbstractRunnable() {
|
||||
@Override
|
||||
public void onFailure(Throwable t) {
|
||||
logger.warn("[{}] failed to send second ping request", t, id);
|
||||
finalizePingCycle(id, listener);
|
||||
}
|
||||
try {
|
||||
receivedResponses.put(id, new PingCollection());
|
||||
sendPingRequest(id);
|
||||
// try and send another ping request halfway through (just in case someone woke up during it...)
|
||||
// this can be a good trade-off to nailing the initial lookup or un-delivered messages
|
||||
threadPool.schedule(TimeValue.timeValueMillis(timeout.millis() / 2), ThreadPool.Names.GENERIC, new AbstractRunnable() {
|
||||
@Override
|
||||
public void onFailure(Throwable t) {
|
||||
logger.warn("[{}] failed to send second ping request", t, id);
|
||||
finalizePingCycle(id, listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doRun() {
|
||||
sendPingRequest(id);
|
||||
threadPool.schedule(TimeValue.timeValueMillis(timeout.millis() / 2), ThreadPool.Names.GENERIC, new AbstractRunnable() {
|
||||
@Override
|
||||
public void onFailure(Throwable t) {
|
||||
logger.warn("[{}] failed to send third ping request", t, id);
|
||||
finalizePingCycle(id, listener);
|
||||
}
|
||||
@Override
|
||||
public void doRun() {
|
||||
sendPingRequest(id);
|
||||
threadPool.schedule(TimeValue.timeValueMillis(timeout.millis() / 2), ThreadPool.Names.GENERIC, new AbstractRunnable() {
|
||||
@Override
|
||||
public void onFailure(Throwable t) {
|
||||
logger.warn("[{}] failed to send third ping request", t, id);
|
||||
finalizePingCycle(id, listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doRun() {
|
||||
// make one last ping, but finalize as soon as all nodes have responded or a timeout has past
|
||||
PingCollection collection = receivedResponses.get(id);
|
||||
FinalizingPingCollection finalizingPingCollection = new FinalizingPingCollection(id, collection, collection.size(), listener);
|
||||
receivedResponses.put(id, finalizingPingCollection);
|
||||
logger.trace("[{}] sending last pings", id);
|
||||
sendPingRequest(id);
|
||||
threadPool.schedule(TimeValue.timeValueMillis(timeout.millis() / 4), ThreadPool.Names.GENERIC, new AbstractRunnable() {
|
||||
@Override
|
||||
public void onFailure(Throwable t) {
|
||||
logger.warn("[{}] failed to finalize ping", t, id);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doRun() throws Exception {
|
||||
finalizePingCycle(id, listener);
|
||||
}
|
||||
});
|
||||
}
|
||||
});
|
||||
}
|
||||
});
|
||||
@Override
|
||||
public void doRun() {
|
||||
// make one last ping, but finalize as soon as all nodes have responded or a timeout has past
|
||||
PingCollection collection = receivedResponses.get(id);
|
||||
FinalizingPingCollection finalizingPingCollection = new FinalizingPingCollection(id, collection, collection.size(), listener);
|
||||
receivedResponses.put(id, finalizingPingCollection);
|
||||
logger.trace("[{}] sending last pings", id);
|
||||
sendPingRequest(id);
|
||||
threadPool.schedule(TimeValue.timeValueMillis(timeout.millis() / 4), ThreadPool.Names.GENERIC, new AbstractRunnable() {
|
||||
@Override
|
||||
public void onFailure(Throwable t) {
|
||||
logger.warn("[{}] failed to finalize ping", t, id);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doRun() throws Exception {
|
||||
finalizePingCycle(id, listener);
|
||||
}
|
||||
});
|
||||
}
|
||||
});
|
||||
}
|
||||
});
|
||||
} catch (Exception e) {
|
||||
logger.warn("failed to ping", e);
|
||||
finalizePingCycle(id, listener);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -201,44 +201,49 @@ 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());
|
||||
receivedResponses.put(sendPingsHandler.id(), sendPingsHandler);
|
||||
try {
|
||||
sendPings(timeout, null, sendPingsHandler);
|
||||
} catch (RejectedExecutionException e) {
|
||||
logger.debug("Ping execution rejected", e);
|
||||
// The RejectedExecutionException can come from the fact unicastConnectExecutor is at its max down in sendPings
|
||||
// But don't bail here, we can retry later on after the send ping has been scheduled.
|
||||
}
|
||||
threadPool.schedule(TimeValue.timeValueMillis(timeout.millis() / 2), ThreadPool.Names.GENERIC, new AbstractRunnable() {
|
||||
@Override
|
||||
protected void doRun() {
|
||||
receivedResponses.put(sendPingsHandler.id(), sendPingsHandler);
|
||||
try {
|
||||
sendPings(timeout, null, sendPingsHandler);
|
||||
threadPool.schedule(TimeValue.timeValueMillis(timeout.millis() / 2), ThreadPool.Names.GENERIC, new AbstractRunnable() {
|
||||
@Override
|
||||
protected void doRun() throws Exception {
|
||||
sendPings(timeout, TimeValue.timeValueMillis(timeout.millis() / 2), sendPingsHandler);
|
||||
sendPingsHandler.close();
|
||||
for (DiscoveryNode node : sendPingsHandler.nodeToDisconnect) {
|
||||
logger.trace("[{}] disconnecting from {}", sendPingsHandler.id(), node);
|
||||
transportService.disconnectFromNode(node);
|
||||
} catch (RejectedExecutionException e) {
|
||||
logger.debug("Ping execution rejected", e);
|
||||
// The RejectedExecutionException can come from the fact unicastConnectExecutor is at its max down in sendPings
|
||||
// But don't bail here, we can retry later on after the send ping has been scheduled.
|
||||
}
|
||||
threadPool.schedule(TimeValue.timeValueMillis(timeout.millis() / 2), ThreadPool.Names.GENERIC, new AbstractRunnable() {
|
||||
@Override
|
||||
protected void doRun() {
|
||||
sendPings(timeout, null, sendPingsHandler);
|
||||
threadPool.schedule(TimeValue.timeValueMillis(timeout.millis() / 2), ThreadPool.Names.GENERIC, new AbstractRunnable() {
|
||||
@Override
|
||||
protected void doRun() throws Exception {
|
||||
sendPings(timeout, TimeValue.timeValueMillis(timeout.millis() / 2), sendPingsHandler);
|
||||
sendPingsHandler.close();
|
||||
for (DiscoveryNode node : sendPingsHandler.nodeToDisconnect) {
|
||||
logger.trace("[{}] disconnecting from {}", sendPingsHandler.id(), node);
|
||||
transportService.disconnectFromNode(node);
|
||||
}
|
||||
listener.onPing(sendPingsHandler.pingCollection().toArray());
|
||||
}
|
||||
listener.onPing(sendPingsHandler.pingCollection().toArray());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Throwable t) {
|
||||
logger.debug("Ping execution failed", t);
|
||||
sendPingsHandler.close();
|
||||
}
|
||||
});
|
||||
}
|
||||
@Override
|
||||
public void onFailure(Throwable t) {
|
||||
logger.debug("Ping execution failed", t);
|
||||
sendPingsHandler.close();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Throwable t) {
|
||||
logger.debug("Ping execution failed", t);
|
||||
sendPingsHandler.close();
|
||||
}
|
||||
});
|
||||
@Override
|
||||
public void onFailure(Throwable t) {
|
||||
logger.debug("Ping execution failed", t);
|
||||
sendPingsHandler.close();
|
||||
}
|
||||
});
|
||||
} catch (Exception e) {
|
||||
sendPingsHandler.close();
|
||||
throw new ElasticsearchException("Ping execution failed", e);
|
||||
}
|
||||
}
|
||||
|
||||
class SendPingsHandler implements Closeable {
|
||||
|
|
|
@ -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){
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
if (metadata.hasLegacyChecksum() || metadata.checksum() == null) {
|
||||
logger.debug("create legacy output for {}", metadata.name());
|
||||
return output;
|
||||
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 {}", fileName);
|
||||
} else {
|
||||
assert metadata.writtenBy() != null;
|
||||
assert metadata.writtenBy().onOrAfter(Version.LUCENE_4_8_0);
|
||||
output = new VerifyingIndexOutput(metadata, output);
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (success == false) {
|
||||
IOUtils.closeWhileHandlingException(output);
|
||||
}
|
||||
}
|
||||
assert metadata.writtenBy() != null;
|
||||
assert metadata.writtenBy().onOrAfter(Version.LUCENE_4_8_0);
|
||||
return new VerifyingIndexOutput(metadata, 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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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() +
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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() +
|
||||
|
|
|
@ -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?");
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
/*
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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" })
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
Loading…
Add table
Add a link
Reference in a new issue