summaryrefslogtreecommitdiff
path: root/core/src/main/java/org/elasticsearch
diff options
context:
space:
mode:
authorNik Everett <nik9000@gmail.com>2016-12-20 11:05:24 -0500
committerGitHub <noreply@github.com>2016-12-20 11:05:24 -0500
commita04dcfb95b1defb954c7be5016ad5865e9099c97 (patch)
tree4b6b4414013c422ef5543311a4844a47aba116fe /core/src/main/java/org/elasticsearch
parent73320566c1e6c6cfd6b3f40c1a70b23ff2cbdf29 (diff)
Introduce XContentParser#namedObject (#22003)
Introduces `XContentParser#namedObject which works a little like `StreamInput#readNamedWriteable`: on startup components register parsers under names and a superclass. At runtime we look up the parser and call it to parse the object. Right now the parsers take a context object they use to help with the parsing but I hope to be able to eliminate the need for this context as most what it is used for at this point is to move around parser registries which should be replaced by this method eventually. I make no effort to do so in this PR because it is big enough already. This is meant to the a start down a road that allows us to remove classes like `QueryParseContext`, `AggregatorParsers`, `IndicesQueriesRegistry`, and `ParseFieldRegistry`. The goal here is to reduce the amount of plumbing required to allow parsing pluggable things. With this you don't have to pass registries all over the place. Instead you must pass a super registry to fewer places and use it to wrap the reader. This is the same tradeoff that we use for NamedWriteable and it allows much, much simpler binary serialization. We think we want that same thing for xcontent serialization. The only parsing actually converted to this method is parsing `ScoreFunctions` inside of `FunctionScoreQuery`. I chose this because it is relatively self contained.
Diffstat (limited to 'core/src/main/java/org/elasticsearch')
-rw-r--r--core/src/main/java/org/elasticsearch/ElasticsearchException.java4
-rw-r--r--core/src/main/java/org/elasticsearch/ElasticsearchParseException.java3
-rw-r--r--core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/get/TransportGetTaskAction.java8
-rw-r--r--core/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexRequest.java10
-rw-r--r--core/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java22
-rw-r--r--core/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java7
-rw-r--r--core/src/main/java/org/elasticsearch/client/transport/TransportClient.java11
-rw-r--r--core/src/main/java/org/elasticsearch/cluster/ClusterState.java18
-rw-r--r--core/src/main/java/org/elasticsearch/cluster/metadata/AliasMetaData.java17
-rw-r--r--core/src/main/java/org/elasticsearch/cluster/metadata/AliasValidator.java16
-rw-r--r--core/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java13
-rw-r--r--core/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java12
-rw-r--r--core/src/main/java/org/elasticsearch/cluster/metadata/MappingMetaData.java6
-rw-r--r--core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java18
-rw-r--r--core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java15
-rw-r--r--core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexTemplateService.java11
-rw-r--r--core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java9
-rw-r--r--core/src/main/java/org/elasticsearch/common/network/NetworkModule.java4
-rw-r--r--core/src/main/java/org/elasticsearch/common/settings/Setting.java4
-rw-r--r--core/src/main/java/org/elasticsearch/common/settings/loader/XContentSettingsLoader.java7
-rw-r--r--core/src/main/java/org/elasticsearch/common/xcontent/NamedXContentRegistry.java211
-rw-r--r--core/src/main/java/org/elasticsearch/common/xcontent/XContent.java12
-rw-r--r--core/src/main/java/org/elasticsearch/common/xcontent/XContentHelper.java70
-rw-r--r--core/src/main/java/org/elasticsearch/common/xcontent/XContentParser.java11
-rw-r--r--core/src/main/java/org/elasticsearch/common/xcontent/cbor/CborXContent.java25
-rw-r--r--core/src/main/java/org/elasticsearch/common/xcontent/cbor/CborXContentParser.java6
-rw-r--r--core/src/main/java/org/elasticsearch/common/xcontent/json/JsonXContent.java25
-rw-r--r--core/src/main/java/org/elasticsearch/common/xcontent/json/JsonXContentGenerator.java7
-rw-r--r--core/src/main/java/org/elasticsearch/common/xcontent/json/JsonXContentParser.java5
-rw-r--r--core/src/main/java/org/elasticsearch/common/xcontent/smile/SmileXContent.java25
-rw-r--r--core/src/main/java/org/elasticsearch/common/xcontent/smile/SmileXContentParser.java6
-rw-r--r--core/src/main/java/org/elasticsearch/common/xcontent/support/AbstractXContentParser.java15
-rw-r--r--core/src/main/java/org/elasticsearch/common/xcontent/yaml/YamlXContent.java25
-rw-r--r--core/src/main/java/org/elasticsearch/common/xcontent/yaml/YamlXContentParser.java6
-rw-r--r--core/src/main/java/org/elasticsearch/gateway/MetaDataStateFormat.java9
-rw-r--r--core/src/main/java/org/elasticsearch/index/IndexModule.java14
-rw-r--r--core/src/main/java/org/elasticsearch/index/IndexService.java13
-rw-r--r--core/src/main/java/org/elasticsearch/index/mapper/DocumentMapperParser.java11
-rw-r--r--core/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java2
-rwxr-xr-xcore/src/main/java/org/elasticsearch/index/mapper/MapperService.java10
-rw-r--r--core/src/main/java/org/elasticsearch/index/query/GeoShapeQueryBuilder.java4
-rw-r--r--core/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java10
-rw-r--r--core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java14
-rw-r--r--core/src/main/java/org/elasticsearch/index/query/QueryShardContext.java11
-rw-r--r--core/src/main/java/org/elasticsearch/index/query/WrapperQueryBuilder.java2
-rw-r--r--core/src/main/java/org/elasticsearch/index/query/functionscore/DecayFunctionBuilder.java9
-rw-r--r--core/src/main/java/org/elasticsearch/index/query/functionscore/FunctionScoreQueryBuilder.java21
-rw-r--r--core/src/main/java/org/elasticsearch/index/shard/CommitPoints.java4
-rw-r--r--core/src/main/java/org/elasticsearch/indices/IndicesService.java22
-rw-r--r--core/src/main/java/org/elasticsearch/node/Node.java20
-rw-r--r--core/src/main/java/org/elasticsearch/plugins/NetworkPlugin.java6
-rw-r--r--core/src/main/java/org/elasticsearch/plugins/Plugin.java23
-rw-r--r--core/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreFormat.java5
-rw-r--r--core/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java4
-rw-r--r--core/src/main/java/org/elasticsearch/rest/RestRequest.java21
-rw-r--r--core/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java7
-rw-r--r--core/src/main/java/org/elasticsearch/script/ScriptMetaData.java4
-rw-r--r--core/src/main/java/org/elasticsearch/search/SearchModule.java25
-rw-r--r--core/src/main/java/org/elasticsearch/search/internal/AliasFilter.java11
-rw-r--r--core/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java40
-rw-r--r--core/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggestionBuilder.java9
-rw-r--r--core/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggester.java3
62 files changed, 686 insertions, 312 deletions
diff --git a/core/src/main/java/org/elasticsearch/ElasticsearchException.java b/core/src/main/java/org/elasticsearch/ElasticsearchException.java
index 7698790a78..bd3ea6797d 100644
--- a/core/src/main/java/org/elasticsearch/ElasticsearchException.java
+++ b/core/src/main/java/org/elasticsearch/ElasticsearchException.java
@@ -787,7 +787,9 @@ public class ElasticsearchException extends RuntimeException implements ToXConte
TASK_CANCELLED_EXCEPTION(org.elasticsearch.tasks.TaskCancelledException.class,
org.elasticsearch.tasks.TaskCancelledException::new, 146, Version.V_5_1_1_UNRELEASED),
SHARD_LOCK_OBTAIN_FAILED_EXCEPTION(org.elasticsearch.env.ShardLockObtainFailedException.class,
- org.elasticsearch.env.ShardLockObtainFailedException::new, 147, Version.V_5_0_2);
+ org.elasticsearch.env.ShardLockObtainFailedException::new, 147, Version.V_5_0_2),
+ UNKNOWN_NAMED_OBJECT_EXCEPTION(org.elasticsearch.common.xcontent.NamedXContentRegistry.UnknownNamedObjectException.class,
+ org.elasticsearch.common.xcontent.NamedXContentRegistry.UnknownNamedObjectException::new, 148, Version.V_5_2_0_UNRELEASED);
final Class<? extends ElasticsearchException> exceptionClass;
final FunctionThatThrowsIOException<StreamInput, ? extends ElasticsearchException> constructor;
diff --git a/core/src/main/java/org/elasticsearch/ElasticsearchParseException.java b/core/src/main/java/org/elasticsearch/ElasticsearchParseException.java
index cdccdb8da9..1711e9a3aa 100644
--- a/core/src/main/java/org/elasticsearch/ElasticsearchParseException.java
+++ b/core/src/main/java/org/elasticsearch/ElasticsearchParseException.java
@@ -24,6 +24,9 @@ import org.elasticsearch.rest.RestStatus;
import java.io.IOException;
+/**
+ * Unchecked exception that is translated into a {@code 400 BAD REQUEST} error when it bubbles out over HTTP.
+ */
public class ElasticsearchParseException extends ElasticsearchException {
public ElasticsearchParseException(String msg, Object... args) {
diff --git a/core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/get/TransportGetTaskAction.java b/core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/get/TransportGetTaskAction.java
index b73adbf811..a6c8941358 100644
--- a/core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/get/TransportGetTaskAction.java
+++ b/core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/get/TransportGetTaskAction.java
@@ -35,6 +35,7 @@ import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.IndexNotFoundException;
@@ -67,14 +68,17 @@ public class TransportGetTaskAction extends HandledTransportAction<GetTaskReques
private final ClusterService clusterService;
private final TransportService transportService;
private final Client client;
+ private final NamedXContentRegistry xContentRegistry;
@Inject
public TransportGetTaskAction(Settings settings, ThreadPool threadPool, TransportService transportService, ActionFilters actionFilters,
- IndexNameExpressionResolver indexNameExpressionResolver, ClusterService clusterService, Client client) {
+ IndexNameExpressionResolver indexNameExpressionResolver, ClusterService clusterService, Client client,
+ NamedXContentRegistry xContentRegistry) {
super(settings, GetTaskAction.NAME, threadPool, transportService, actionFilters, indexNameExpressionResolver, GetTaskRequest::new);
this.clusterService = clusterService;
this.transportService = transportService;
this.client = client;
+ this.xContentRegistry = xContentRegistry;
}
@Override
@@ -246,7 +250,7 @@ public class TransportGetTaskAction extends HandledTransportAction<GetTaskReques
listener.onFailure(new ElasticsearchException("Stored task status for [{}] didn't contain any source!", response.getId()));
return;
}
- try (XContentParser parser = XContentHelper.createParser(response.getSourceAsBytesRef())) {
+ try (XContentParser parser = XContentHelper.createParser(xContentRegistry, response.getSourceAsBytesRef())) {
TaskResult result = TaskResult.PARSER.apply(parser, () -> ParseFieldMatcher.STRICT);
listener.onResponse(new GetTaskResponse(result));
}
diff --git a/core/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexRequest.java b/core/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexRequest.java
index 17df06dbf4..203483d89b 100644
--- a/core/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexRequest.java
+++ b/core/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexRequest.java
@@ -35,6 +35,7 @@ import org.elasticsearch.common.collect.MapBuilder;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
@@ -307,7 +308,8 @@ public class CreateIndexRequest extends AcknowledgedRequest<CreateIndexRequest>
* Sets the aliases that will be associated with the index when it gets created
*/
public CreateIndexRequest aliases(BytesReference source) {
- try (XContentParser parser = XContentHelper.createParser(source)) {
+ // EMPTY is safe here because we never call namedObject
+ try (XContentParser parser = XContentHelper.createParser(NamedXContentRegistry.EMPTY, source)) {
//move to the first alias
parser.nextToken();
while ((parser.nextToken()) != XContentParser.Token.END_OBJECT) {
@@ -361,11 +363,7 @@ public class CreateIndexRequest extends AcknowledgedRequest<CreateIndexRequest>
public CreateIndexRequest source(BytesReference source) {
XContentType xContentType = XContentFactory.xContentType(source);
if (xContentType != null) {
- try (XContentParser parser = XContentFactory.xContent(xContentType).createParser(source)) {
- source(parser.map());
- } catch (IOException e) {
- throw new ElasticsearchParseException("failed to parse source for create index", e);
- }
+ source(XContentHelper.convertToMap(source, false).v2());
} else {
settings(source.utf8ToString());
}
diff --git a/core/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java b/core/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java
index 77c3a1d4c2..a7d6241d31 100644
--- a/core/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java
+++ b/core/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java
@@ -36,6 +36,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
@@ -351,11 +352,7 @@ public class PutIndexTemplateRequest extends MasterNodeRequest<PutIndexTemplateR
* The template source definition.
*/
public PutIndexTemplateRequest source(String templateSource) {
- try (XContentParser parser = XContentFactory.xContent(templateSource).createParser(templateSource)) {
- return source(parser.mapOrdered());
- } catch (Exception e) {
- throw new IllegalArgumentException("failed to parse template source [" + templateSource + "]", e);
- }
+ return source(XContentHelper.convertToMap(XContentFactory.xContent(templateSource), templateSource, true));
}
/**
@@ -369,22 +366,14 @@ public class PutIndexTemplateRequest extends MasterNodeRequest<PutIndexTemplateR
* The template source definition.
*/
public PutIndexTemplateRequest source(byte[] source, int offset, int length) {
- try (XContentParser parser = XContentFactory.xContent(source, offset, length).createParser(source, offset, length)) {
- return source(parser.mapOrdered());
- } catch (IOException e) {
- throw new IllegalArgumentException("failed to parse template source", e);
- }
+ return source(new BytesArray(source, offset, length));
}
/**
* The template source definition.
*/
public PutIndexTemplateRequest source(BytesReference source) {
- try (XContentParser parser = XContentFactory.xContent(source).createParser(source)) {
- return source(parser.mapOrdered());
- } catch (IOException e) {
- throw new IllegalArgumentException("failed to parse template source", e);
- }
+ return source(XContentHelper.convertToMap(source, true).v2());
}
public PutIndexTemplateRequest custom(IndexMetaData.Custom custom) {
@@ -432,7 +421,8 @@ public class PutIndexTemplateRequest extends MasterNodeRequest<PutIndexTemplateR
* Sets the aliases that will be associated with the index when it gets created
*/
public PutIndexTemplateRequest aliases(BytesReference source) {
- try (XContentParser parser = XContentHelper.createParser(source)) {
+ // EMPTY is safe here because we never call namedObject
+ try (XContentParser parser = XContentHelper.createParser(NamedXContentRegistry.EMPTY, source)) {
//move to the first alias
parser.nextToken();
while ((parser.nextToken()) != XContentParser.Token.END_OBJECT) {
diff --git a/core/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java b/core/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java
index 5c3e8d9295..2ce9f4d47c 100644
--- a/core/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java
+++ b/core/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java
@@ -39,6 +39,7 @@ import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.lucene.uid.Versions;
import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContent;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
@@ -283,7 +284,8 @@ public class BulkRequest extends ActionRequest implements CompositeIndicesReques
line++;
// now parse the action
- try (XContentParser parser = xContent.createParser(data.slice(from, nextMarker - from))) {
+ // EMPTY is safe here because we never call namedObject
+ try (XContentParser parser = xContent.createParser(NamedXContentRegistry.EMPTY, data.slice(from, nextMarker - from))) {
// move pointers
from = nextMarker + 1;
@@ -400,7 +402,8 @@ public class BulkRequest extends ActionRequest implements CompositeIndicesReques
.version(version).versionType(versionType)
.routing(routing)
.parent(parent);
- try (XContentParser sliceParser = xContent.createParser(data.slice(from, nextMarker - from))) {
+ // EMPTY is safe here because we never call namedObject
+ try (XContentParser sliceParser = xContent.createParser(NamedXContentRegistry.EMPTY, data.slice(from, nextMarker - from))) {
updateRequest.fromXContent(sliceParser);
}
if (fetchSourceContext != null) {
diff --git a/core/src/main/java/org/elasticsearch/client/transport/TransportClient.java b/core/src/main/java/org/elasticsearch/client/transport/TransportClient.java
index 4b9e4b1eac..79f39d143f 100644
--- a/core/src/main/java/org/elasticsearch/client/transport/TransportClient.java
+++ b/core/src/main/java/org/elasticsearch/client/transport/TransportClient.java
@@ -41,6 +41,7 @@ import org.elasticsearch.common.settings.SettingsModule;
import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.BigArrays;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.node.Node;
import org.elasticsearch.node.internal.InternalSettingsPreparer;
@@ -63,8 +64,11 @@ import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import static java.util.stream.Collectors.toList;
import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds;
/**
@@ -140,6 +144,11 @@ public abstract class TransportClient extends AbstractClient {
.flatMap(p -> p.getNamedWriteables().stream())
.collect(Collectors.toList()));
NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(entries);
+ NamedXContentRegistry xContentRegistry = new NamedXContentRegistry(Stream.of(
+ searchModule.getNamedXContents().stream(),
+ pluginsService.filterPlugins(Plugin.class).stream()
+ .flatMap(p -> p.getNamedXContent().stream())
+ ).flatMap(Function.identity()).collect(toList()));
ModulesBuilder modules = new ModulesBuilder();
// plugin modules must be added here, before others or we can get crazy injection errors...
@@ -158,7 +167,7 @@ public abstract class TransportClient extends AbstractClient {
resourcesToClose.add(bigArrays);
modules.add(settingsModule);
NetworkModule networkModule = new NetworkModule(settings, true, pluginsService.filterPlugins(NetworkPlugin.class), threadPool,
- bigArrays, circuitBreakerService, namedWriteableRegistry, networkService);
+ bigArrays, circuitBreakerService, namedWriteableRegistry, xContentRegistry, networkService);
final Transport transport = networkModule.getTransportSupplier().get();
final TransportService transportService = new TransportService(settings, transport, threadPool,
networkModule.getTransportInterceptor(), null);
diff --git a/core/src/main/java/org/elasticsearch/cluster/ClusterState.java b/core/src/main/java/org/elasticsearch/cluster/ClusterState.java
index 7b6f2b55aa..34c9b64256 100644
--- a/core/src/main/java/org/elasticsearch/cluster/ClusterState.java
+++ b/core/src/main/java/org/elasticsearch/cluster/ClusterState.java
@@ -22,6 +22,7 @@ package org.elasticsearch.cluster;
import com.carrotsearch.hppc.cursors.IntObjectCursor;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
+
import org.elasticsearch.cluster.block.ClusterBlock;
import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.metadata.IndexMetaData;
@@ -40,6 +41,7 @@ import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.UUIDs;
+import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.compress.CompressedXContent;
@@ -49,8 +51,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.common.xcontent.XContentFactory;
-import org.elasticsearch.common.xcontent.XContentParser;
+import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.discovery.zen.PublishClusterStateAction;
@@ -403,11 +404,7 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
builder.startObject("mappings");
for (ObjectObjectCursor<String, CompressedXContent> cursor1 : templateMetaData.mappings()) {
- byte[] mappingSource = cursor1.value.uncompressed();
- Map<String, Object> mapping;
- try (XContentParser parser = XContentFactory.xContent(mappingSource).createParser(mappingSource)) {
- mapping = parser.map();
- }
+ Map<String, Object> mapping = XContentHelper.convertToMap(new BytesArray(cursor1.value.uncompressed()), false).v2();
if (mapping.size() == 1 && mapping.containsKey(cursor1.key)) {
// the type name is the root value, reduce it
mapping = (Map<String, Object>) mapping.get(cursor1.key);
@@ -435,11 +432,8 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
builder.startObject("mappings");
for (ObjectObjectCursor<String, MappingMetaData> cursor : indexMetaData.getMappings()) {
- byte[] mappingSource = cursor.value.source().uncompressed();
- Map<String, Object> mapping;
- try (XContentParser parser = XContentFactory.xContent(mappingSource).createParser(mappingSource)) {
- mapping = parser.map();
- }
+ Map<String, Object> mapping = XContentHelper
+ .convertToMap(new BytesArray(cursor.value.source().uncompressed()), false).v2();
if (mapping.size() == 1 && mapping.containsKey(cursor.key)) {
// the type name is the root value, reduce it
mapping = (Map<String, Object>) mapping.get(cursor.key);
diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/AliasMetaData.java b/core/src/main/java/org/elasticsearch/cluster/metadata/AliasMetaData.java
index e62a3935ad..647b355cc0 100644
--- a/core/src/main/java/org/elasticsearch/cluster/metadata/AliasMetaData.java
+++ b/core/src/main/java/org/elasticsearch/cluster/metadata/AliasMetaData.java
@@ -22,12 +22,14 @@ package org.elasticsearch.cluster.metadata;
import org.elasticsearch.ElasticsearchGenerationException;
import org.elasticsearch.cluster.AbstractDiffable;
import org.elasticsearch.common.Strings;
+import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
+import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import java.io.IOException;
@@ -225,14 +227,7 @@ public class AliasMetaData extends AbstractDiffable<AliasMetaData> {
this.filter = null;
return this;
}
- try {
- try (XContentParser parser = XContentFactory.xContent(filter).createParser(filter)) {
- filter(parser.mapOrdered());
- }
- return this;
- } catch (IOException e) {
- throw new ElasticsearchGenerationException("Failed to generate [" + filter + "]", e);
- }
+ return filter(XContentHelper.convertToMap(XContentFactory.xContent(filter), filter, true));
}
public Builder filter(Map<String, Object> filter) {
@@ -286,11 +281,7 @@ public class AliasMetaData extends AbstractDiffable<AliasMetaData> {
if (binary) {
builder.field("filter", aliasMetaData.filter.compressed());
} else {
- byte[] data = aliasMetaData.filter().uncompressed();
- try (XContentParser parser = XContentFactory.xContent(data).createParser(data)) {
- Map<String, Object> filter = parser.mapOrdered();
- builder.field("filter", filter);
- }
+ builder.field("filter", XContentHelper.convertToMap(new BytesArray(aliasMetaData.filter().uncompressed()), true).v2());
}
}
if (aliasMetaData.indexRouting() != null) {
diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/AliasValidator.java b/core/src/main/java/org/elasticsearch/cluster/metadata/AliasValidator.java
index 29faa0f795..bce6e45c79 100644
--- a/core/src/main/java/org/elasticsearch/cluster/metadata/AliasValidator.java
+++ b/core/src/main/java/org/elasticsearch/cluster/metadata/AliasValidator.java
@@ -25,7 +25,9 @@ import org.elasticsearch.common.Strings;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentFactory;
+import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryParseContext;
@@ -74,8 +76,8 @@ public class AliasValidator extends AbstractComponent {
public void validateAliasStandalone(Alias alias) {
validateAliasStandalone(alias.name(), alias.indexRouting());
if (Strings.hasLength(alias.filter())) {
- try (XContentParser parser = XContentFactory.xContent(alias.filter()).createParser(alias.filter())) {
- parser.map();
+ try {
+ XContentHelper.convertToMap(XContentFactory.xContent(alias.filter()), alias.filter(), false);
} catch (Exception e) {
throw new IllegalArgumentException("failed to parse filter for alias [" + alias.name() + "]", e);
}
@@ -113,9 +115,10 @@ public class AliasValidator extends AbstractComponent {
* provided {@link org.elasticsearch.index.query.QueryShardContext}
* @throws IllegalArgumentException if the filter is not valid
*/
- public void validateAliasFilter(String alias, String filter, QueryShardContext queryShardContext) {
+ public void validateAliasFilter(String alias, String filter, QueryShardContext queryShardContext,
+ NamedXContentRegistry xContentRegistry) {
assert queryShardContext != null;
- try (XContentParser parser = XContentFactory.xContent(filter).createParser(filter)) {
+ try (XContentParser parser = XContentFactory.xContent(filter).createParser(xContentRegistry, filter)) {
validateAliasFilter(parser, queryShardContext);
} catch (Exception e) {
throw new IllegalArgumentException("failed to parse filter for alias [" + alias + "]", e);
@@ -127,9 +130,10 @@ public class AliasValidator extends AbstractComponent {
* provided {@link org.elasticsearch.index.query.QueryShardContext}
* @throws IllegalArgumentException if the filter is not valid
*/
- public void validateAliasFilter(String alias, byte[] filter, QueryShardContext queryShardContext) {
+ public void validateAliasFilter(String alias, byte[] filter, QueryShardContext queryShardContext,
+ NamedXContentRegistry xContentRegistry) {
assert queryShardContext != null;
- try (XContentParser parser = XContentFactory.xContent(filter).createParser(filter)) {
+ try (XContentParser parser = XContentFactory.xContent(filter).createParser(xContentRegistry, filter)) {
validateAliasFilter(parser, queryShardContext);
} catch (Exception e) {
throw new IllegalArgumentException("failed to parse filter for alias [" + alias + "]", e);
diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java b/core/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java
index a1f217e137..b28ec4a0c8 100644
--- a/core/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java
+++ b/core/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java
@@ -23,6 +23,7 @@ import com.carrotsearch.hppc.LongArrayList;
import com.carrotsearch.hppc.cursors.IntObjectCursor;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
+
import org.elasticsearch.Version;
import org.elasticsearch.action.support.ActiveShardCount;
import org.elasticsearch.cluster.Diff;
@@ -34,6 +35,7 @@ import org.elasticsearch.cluster.node.DiscoveryNodeFilters;
import org.elasticsearch.cluster.routing.allocation.IndexMetaDataUpdater;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.ParseFieldMatcher;
+import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.collect.ImmutableOpenIntMap;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.collect.MapBuilder;
@@ -48,6 +50,7 @@ import org.elasticsearch.common.xcontent.FromXContentBuilder;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
+import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.gateway.MetaDataStateFormat;
@@ -831,9 +834,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
}
public Builder putMapping(String type, String source) throws IOException {
- try (XContentParser parser = XContentFactory.xContent(source).createParser(source)) {
- putMapping(new MappingMetaData(type, parser.mapOrdered()));
- }
+ putMapping(new MappingMetaData(type, XContentHelper.convertToMap(XContentFactory.xContent(source), source, true)));
return this;
}
@@ -1047,11 +1048,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
if (binary) {
builder.value(cursor.value.source().compressed());
} else {
- byte[] data = cursor.value.source().uncompressed();
- try (XContentParser parser = XContentFactory.xContent(data).createParser(data)) {
- Map<String, Object> mapping = parser.mapOrdered();
- builder.map(mapping);
- }
+ builder.map(XContentHelper.convertToMap(new BytesArray(cursor.value.source().uncompressed()), true).v2());
}
}
builder.endArray();
diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java b/core/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java
index feabc380c4..7ffa4878fe 100644
--- a/core/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java
+++ b/core/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java
@@ -24,6 +24,7 @@ import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.AbstractDiffable;
import org.elasticsearch.common.Nullable;
+import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.collect.MapBuilder;
import org.elasticsearch.common.compress.CompressedXContent;
@@ -37,6 +38,7 @@ import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
+import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import java.io.IOException;
@@ -395,10 +397,7 @@ public class IndexTemplateMetaData extends AbstractDiffable<IndexTemplateMetaDat
builder.startObject("mappings");
for (ObjectObjectCursor<String, CompressedXContent> cursor : indexTemplateMetaData.mappings()) {
byte[] mappingSource = cursor.value.uncompressed();
- Map<String, Object> mapping;
- try (XContentParser parser = XContentFactory.xContent(mappingSource).createParser(mappingSource)) {;
- mapping = parser.map();
- }
+ Map<String, Object> mapping = XContentHelper.convertToMap(new BytesArray(mappingSource), false).v2();
if (mapping.size() == 1 && mapping.containsKey(cursor.key)) {
// the type name is the root value, reduce it
mapping = (Map<String, Object>) mapping.get(cursor.key);
@@ -411,10 +410,7 @@ public class IndexTemplateMetaData extends AbstractDiffable<IndexTemplateMetaDat
builder.startArray("mappings");
for (ObjectObjectCursor<String, CompressedXContent> cursor : indexTemplateMetaData.mappings()) {
byte[] data = cursor.value.uncompressed();
- try (XContentParser parser = XContentFactory.xContent(data).createParser(data)) {
- Map<String, Object> mapping = parser.mapOrdered();
- builder.map(mapping);
- }
+ builder.map(XContentHelper.convertToMap(new BytesArray(data), true).v2());
}
builder.endArray();
}
diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MappingMetaData.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MappingMetaData.java
index 0f9db99326..39fb3e381f 100644
--- a/core/src/main/java/org/elasticsearch/cluster/metadata/MappingMetaData.java
+++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MappingMetaData.java
@@ -27,7 +27,6 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
-import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.mapper.DateFieldMapper;
import org.elasticsearch.index.mapper.DocumentMapper;
@@ -89,10 +88,7 @@ public class MappingMetaData extends AbstractDiffable<MappingMetaData> {
public MappingMetaData(CompressedXContent mapping) throws IOException {
this.source = mapping;
- Map<String, Object> mappingMap;
- try (XContentParser parser = XContentHelper.createParser(mapping.compressedReference())) {
- mappingMap = parser.mapOrdered();
- }
+ Map<String, Object> mappingMap = XContentHelper.convertToMap(mapping.compressedReference(), true).v2();
if (mappingMap.size() != 1) {
throw new IllegalStateException("Can't derive type from mapping, no root type: " + mapping.string());
}
diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java
index 9d81939995..1cbc81cb26 100644
--- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java
+++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java
@@ -59,6 +59,7 @@ import org.elasticsearch.common.io.PathUtils;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.settings.IndexScopedSettings;
import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.env.Environment;
import org.elasticsearch.index.Index;
@@ -113,12 +114,14 @@ public class MetaDataCreateIndexService extends AbstractComponent {
private final Environment env;
private final IndexScopedSettings indexScopedSettings;
private final ActiveShardsObserver activeShardsObserver;
+ private final NamedXContentRegistry xContentRegistry;
@Inject
public MetaDataCreateIndexService(Settings settings, ClusterService clusterService,
IndicesService indicesService, AllocationService allocationService,
AliasValidator aliasValidator, Environment env,
- IndexScopedSettings indexScopedSettings, ThreadPool threadPool) {
+ IndexScopedSettings indexScopedSettings, ThreadPool threadPool,
+ NamedXContentRegistry xContentRegistry) {
super(settings);
this.clusterService = clusterService;
this.indicesService = indicesService;
@@ -127,6 +130,7 @@ public class MetaDataCreateIndexService extends AbstractComponent {
this.env = env;
this.indexScopedSettings = indexScopedSettings;
this.activeShardsObserver = new ActiveShardsObserver(settings, clusterService, threadPool);
+ this.xContentRegistry = xContentRegistry;
}
/**
@@ -248,7 +252,7 @@ public class MetaDataCreateIndexService extends AbstractComponent {
List<String> templateNames = new ArrayList<>();
for (Map.Entry<String, String> entry : request.mappings().entrySet()) {
- mappings.put(entry.getKey(), MapperService.parseMapping(entry.getValue()));
+ mappings.put(entry.getKey(), MapperService.parseMapping(xContentRegistry, entry.getValue()));
}
for (Map.Entry<String, Custom> entry : request.customs().entrySet()) {
@@ -260,9 +264,10 @@ public class MetaDataCreateIndexService extends AbstractComponent {
templateNames.add(template.getName());
for (ObjectObjectCursor<String, CompressedXContent> cursor : template.mappings()) {
if (mappings.containsKey(cursor.key)) {
- XContentHelper.mergeDefaults(mappings.get(cursor.key), MapperService.parseMapping(cursor.value.string()));
+ XContentHelper.mergeDefaults(mappings.get(cursor.key),
+ MapperService.parseMapping(xContentRegistry, cursor.value.string()));
} else {
- mappings.put(cursor.key, MapperService.parseMapping(cursor.value.string()));
+ mappings.put(cursor.key, MapperService.parseMapping(xContentRegistry, cursor.value.string()));
}
}
// handle custom
@@ -368,12 +373,13 @@ public class MetaDataCreateIndexService extends AbstractComponent {
final QueryShardContext queryShardContext = indexService.newQueryShardContext(0, null, () -> 0L);
for (Alias alias : request.aliases()) {
if (Strings.hasLength(alias.filter())) {
- aliasValidator.validateAliasFilter(alias.name(), alias.filter(), queryShardContext);
+ aliasValidator.validateAliasFilter(alias.name(), alias.filter(), queryShardContext, xContentRegistry);
}
}
for (AliasMetaData aliasMetaData : templatesAliases.values()) {
if (aliasMetaData.filter() != null) {
- aliasValidator.validateAliasFilter(aliasMetaData.alias(), aliasMetaData.filter().uncompressed(), queryShardContext);
+ aliasValidator.validateAliasFilter(aliasMetaData.alias(), aliasMetaData.filter().uncompressed(),
+ queryShardContext, xContentRegistry);
}
}
diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java
index f1584ee325..81fbddce46 100644
--- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java
+++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java
@@ -33,6 +33,7 @@ import org.elasticsearch.common.Strings;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.index.IndexService;
@@ -64,18 +65,17 @@ public class MetaDataIndexAliasesService extends AbstractComponent {
private final MetaDataDeleteIndexService deleteIndexService;
+ private final NamedXContentRegistry xContentRegistry;
+
@Inject
- public MetaDataIndexAliasesService(
- Settings settings,
- ClusterService clusterService,
- IndicesService indicesService,
- AliasValidator aliasValidator,
- MetaDataDeleteIndexService deleteIndexService) {
+ public MetaDataIndexAliasesService(Settings settings, ClusterService clusterService, IndicesService indicesService,
+ AliasValidator aliasValidator, MetaDataDeleteIndexService deleteIndexService, NamedXContentRegistry xContentRegistry) {
super(settings);
this.clusterService = clusterService;
this.indicesService = indicesService;
this.aliasValidator = aliasValidator;
this.deleteIndexService = deleteIndexService;
+ this.xContentRegistry = xContentRegistry;
}
public void indicesAliases(final IndicesAliasesClusterStateUpdateRequest request,
@@ -151,7 +151,8 @@ public class MetaDataIndexAliasesService extends AbstractComponent {
}
// the context is only used for validation so it's fine to pass fake values for the shard id and the current
// timestamp
- aliasValidator.validateAliasFilter(alias, filter, indexService.newQueryShardContext(0, null, () -> 0L));
+ aliasValidator.validateAliasFilter(alias, filter, indexService.newQueryShardContext(0, null, () -> 0L),
+ xContentRegistry);
}
};
changed |= action.apply(newAliasValidator, metadata, index);
diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexTemplateService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexTemplateService.java
index 2e11f1e7f4..b8c19fbeb4 100644
--- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexTemplateService.java
+++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexTemplateService.java
@@ -35,6 +35,7 @@ import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.settings.IndexScopedSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.mapper.MapperParsingException;
@@ -65,18 +66,20 @@ public class MetaDataIndexTemplateService extends AbstractComponent {
private final IndicesService indicesService;
private final MetaDataCreateIndexService metaDataCreateIndexService;
private final IndexScopedSettings indexScopedSettings;
+ private final NamedXContentRegistry xContentRegistry;
@Inject
public MetaDataIndexTemplateService(Settings settings, ClusterService clusterService,
MetaDataCreateIndexService metaDataCreateIndexService,
AliasValidator aliasValidator, IndicesService indicesService,
- IndexScopedSettings indexScopedSettings) {
+ IndexScopedSettings indexScopedSettings, NamedXContentRegistry xContentRegistry) {
super(settings);
this.clusterService = clusterService;
this.aliasValidator = aliasValidator;
this.indicesService = indicesService;
this.metaDataCreateIndexService = metaDataCreateIndexService;
this.indexScopedSettings = indexScopedSettings;
+ this.xContentRegistry = xContentRegistry;
}
public void removeTemplates(final RemoveRequest request, final RemoveListener listener) {
@@ -165,7 +168,7 @@ public class MetaDataIndexTemplateService extends AbstractComponent {
throw new IllegalArgumentException("index_template [" + request.name + "] already exists");
}
- validateAndAddTemplate(request, templateBuilder, indicesService);
+ validateAndAddTemplate(request, templateBuilder, indicesService, xContentRegistry);
for (Alias alias : request.aliases) {
AliasMetaData aliasMetaData = AliasMetaData.builder(alias.name()).filter(alias.filter())
@@ -190,7 +193,7 @@ public class MetaDataIndexTemplateService extends AbstractComponent {
}
private static void validateAndAddTemplate(final PutRequest request, IndexTemplateMetaData.Builder templateBuilder,
- IndicesService indicesService) throws Exception {
+ IndicesService indicesService, NamedXContentRegistry xContentRegistry) throws Exception {
Index createdIndex = null;
final String temporaryIndexName = UUIDs.randomBase64UUID();
try {
@@ -220,7 +223,7 @@ public class MetaDataIndexTemplateService extends AbstractComponent {
} catch (Exception e) {
throw new MapperParsingException("Failed to parse mapping [{}]: {}", e, entry.getKey(), e.getMessage());
}
- mappingsForValidation.put(entry.getKey(), MapperService.parseMapping(entry.getValue()));
+ mappingsForValidation.put(entry.getKey(), MapperService.parseMapping(xContentRegistry, entry.getValue()));
}
dummyIndexService.mapperService().merge(mappingsForValidation, MergeReason.MAPPING_UPDATE, false);
diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java
index e299874990..1779699d44 100644
--- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java
+++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java
@@ -27,6 +27,7 @@ import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.IndexScopedSettings;
import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.analysis.AnalyzerScope;
import org.elasticsearch.index.analysis.IndexAnalyzers;
@@ -50,12 +51,15 @@ import java.util.Set;
*/
public class MetaDataIndexUpgradeService extends AbstractComponent {
+ private final NamedXContentRegistry xContentRegistry;
private final MapperRegistry mapperRegistry;
private final IndexScopedSettings indexScopedSettings;
@Inject
- public MetaDataIndexUpgradeService(Settings settings, MapperRegistry mapperRegistry, IndexScopedSettings indexScopedSettings) {
+ public MetaDataIndexUpgradeService(Settings settings, NamedXContentRegistry xContentRegistry, MapperRegistry mapperRegistry,
+ IndexScopedSettings indexScopedSettings) {
super(settings);
+ this.xContentRegistry = xContentRegistry;
this.mapperRegistry = mapperRegistry;
this.indexScopedSettings = indexScopedSettings;
}
@@ -146,7 +150,8 @@ public class MetaDataIndexUpgradeService extends AbstractComponent {
}
};
try (IndexAnalyzers fakeIndexAnalzyers = new IndexAnalyzers(indexSettings, fakeDefault, fakeDefault, fakeDefault, analyzerMap)) {
- MapperService mapperService = new MapperService(indexSettings, fakeIndexAnalzyers, similarityService, mapperRegistry, () -> null);
+ MapperService mapperService = new MapperService(indexSettings, fakeIndexAnalzyers, xContentRegistry, similarityService,
+ mapperRegistry, () -> null);
mapperService.merge(indexMetaData, MapperService.MergeReason.MAPPING_RECOVERY, false);
}
} catch (Exception ex) {
diff --git a/core/src/main/java/org/elasticsearch/common/network/NetworkModule.java b/core/src/main/java/org/elasticsearch/common/network/NetworkModule.java
index 530ecefd4c..679a271dea 100644
--- a/core/src/main/java/org/elasticsearch/common/network/NetworkModule.java
+++ b/core/src/main/java/org/elasticsearch/common/network/NetworkModule.java
@@ -35,6 +35,7 @@ import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.http.HttpServerTransport;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.plugins.NetworkPlugin;
@@ -107,13 +108,14 @@ public final class NetworkModule {
BigArrays bigArrays,
CircuitBreakerService circuitBreakerService,
NamedWriteableRegistry namedWriteableRegistry,
+ NamedXContentRegistry xContentRegistry,
NetworkService networkService) {
this.settings = settings;
this.transportClient = transportClient;
for (NetworkPlugin plugin : plugins) {
if (transportClient == false && HTTP_ENABLED.get(settings)) {
Map<String, Supplier<HttpServerTransport>> httpTransportFactory = plugin.getHttpTransports(settings, threadPool, bigArrays,
- circuitBreakerService, namedWriteableRegistry, networkService);
+ circuitBreakerService, namedWriteableRegistry, xContentRegistry, networkService);
for (Map.Entry<String, Supplier<HttpServerTransport>> entry : httpTransportFactory.entrySet()) {
registerHttpTransport(entry.getKey(), entry.getValue());
}
diff --git a/core/src/main/java/org/elasticsearch/common/settings/Setting.java b/core/src/main/java/org/elasticsearch/common/settings/Setting.java
index 5d9adbc34c..6f52ea1609 100644
--- a/core/src/main/java/org/elasticsearch/common/settings/Setting.java
+++ b/core/src/main/java/org/elasticsearch/common/settings/Setting.java
@@ -32,6 +32,7 @@ import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.MemorySizeValue;
import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
@@ -702,7 +703,8 @@ public class Setting<T> extends ToXContentToBytes {
}
private static List<String> parseableStringToList(String parsableString) {
- try (XContentParser xContentParser = XContentType.JSON.xContent().createParser(parsableString)) {
+ // EMPTY is safe here because we never call namedObject
+ try (XContentParser xContentParser = XContentType.JSON.xContent().createParser(NamedXContentRegistry.EMPTY, parsableString)) {
XContentParser.Token token = xContentParser.nextToken();
if (token != XContentParser.Token.START_ARRAY) {
throw new IllegalArgumentException("expected START_ARRAY but got " + token);
diff --git a/core/src/main/java/org/elasticsearch/common/settings/loader/XContentSettingsLoader.java b/core/src/main/java/org/elasticsearch/common/settings/loader/XContentSettingsLoader.java
index 30c62b91c7..d7eaa627a2 100644
--- a/core/src/main/java/org/elasticsearch/common/settings/loader/XContentSettingsLoader.java
+++ b/core/src/main/java/org/elasticsearch/common/settings/loader/XContentSettingsLoader.java
@@ -20,6 +20,7 @@
package org.elasticsearch.common.settings.loader;
import org.elasticsearch.ElasticsearchParseException;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
@@ -46,14 +47,16 @@ public abstract class XContentSettingsLoader implements SettingsLoader {
@Override
public Map<String, String> load(String source) throws IOException {
- try (XContentParser parser = XContentFactory.xContent(contentType()).createParser(source)) {
+ // It is safe to use EMPTY here because this never uses namedObject
+ try (XContentParser parser = XContentFactory.xContent(contentType()).createParser(NamedXContentRegistry.EMPTY, source)) {
return load(parser);
}
}
@Override
public Map<String, String> load(byte[] source) throws IOException {
- try (XContentParser parser = XContentFactory.xContent(contentType()).createParser(source)) {
+ // It is safe to use EMPTY here because this never uses namedObject
+ try (XContentParser parser = XContentFactory.xContent(contentType()).createParser(NamedXContentRegistry.EMPTY, source)) {
return load(parser);
}
}
diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/NamedXContentRegistry.java b/core/src/main/java/org/elasticsearch/common/xcontent/NamedXContentRegistry.java
new file mode 100644
index 0000000000..046908d654
--- /dev/null
+++ b/core/src/main/java/org/elasticsearch/common/xcontent/NamedXContentRegistry.java
@@ -0,0 +1,211 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.common.xcontent;
+
+import org.elasticsearch.ElasticsearchException;
+import org.elasticsearch.common.ParseField;
+import org.elasticsearch.common.ParsingException;
+import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.io.stream.StreamOutput;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import static java.util.Collections.emptyList;
+import static java.util.Collections.emptyMap;
+import static java.util.Collections.unmodifiableMap;
+import static java.util.Objects.requireNonNull;
+
+public class NamedXContentRegistry {
+ /**
+ * The empty {@link NamedXContentRegistry} for use when you are sure that you aren't going to call
+ * {@link XContentParser#namedObject(Class, String, Object)}. Be *very* careful with this singleton because a parser using it will fail
+ * every call to {@linkplain XContentParser#namedObject(Class, String, Object)}. Every non-test usage really should be checked thorowly
+ * and marked with a comment about how it was checked. That way anyone that sees code that uses it knows that it is potentially
+ * dangerous.
+ */
+ public static final NamedXContentRegistry EMPTY = new NamedXContentRegistry(emptyList());
+
+ /**
+ * Parses an object with the type T from parser.
+ */
+ public interface FromXContent<T> {
+ /**
+ * Parses an object with the type T from parser.
+ */
+ T fromXContent(XContentParser parser) throws IOException;
+ }
+
+ /**
+ * Parses an object with the type T from parser.
+ * @deprecated prefer {@link FromXContent} if possible
+ */
+ @Deprecated
+ public interface FromXContentWithContext<T> {
+ T fromXContent(XContentParser parser, Object context) throws IOException;
+ }
+
+ /**
+ * An entry in the {@linkplain NamedXContentRegistry} containing the name of the object and the parser that can parse it.
+ */
+ public static class Entry {
+ /** The class that this entry can read. */
+ public final Class<?> categoryClass;
+
+ /** A name for the entry which is unique within the {@link #categoryClass}. */
+ public final ParseField name;
+
+ /** A parser capability of parser the entry's class. */
+ private final FromXContentWithContext<?> parser;
+
+ /** Creates a new entry which can be stored by the registry. */
+ public <T> Entry(Class<T> categoryClass, ParseField name, FromXContent<? extends T> parser) {
+ this.categoryClass = Objects.requireNonNull(categoryClass);
+ this.name = Objects.requireNonNull(name);
+ this.parser = Objects.requireNonNull((p, c) -> parser.fromXContent(p));
+ }
+ /**
+ * Creates a new entry which can be stored by the registry.
+ * @deprecated prefer {@link Entry#Entry(Class, ParseField, FromXContent)}. Contexts will be removed when possible
+ */
+ @Deprecated
+ public <T> Entry(Class<T> categoryClass, ParseField name, FromXContentWithContext<? extends T> parser) {
+ this.categoryClass = Objects.requireNonNull(categoryClass);
+ this.name = Objects.requireNonNull(name);
+ this.parser = Objects.requireNonNull(parser);
+ }
+ }
+
+ private final Map<Class<?>, Map<String, Entry>> registry;
+
+ public NamedXContentRegistry(List<Entry> entries) {
+ if (entries.isEmpty()) {
+ registry = emptyMap();
+ return;
+ }
+ entries = new ArrayList<>(entries);
+ entries.sort((e1, e2) -> e1.categoryClass.getName().compareTo(e2.categoryClass.getName()));
+
+ Map<Class<?>, Map<String, Entry>> registry = new HashMap<>();
+ Map<String, Entry> parsers = null;
+ Class<?> currentCategory = null;
+ for (Entry entry : entries) {
+ if (currentCategory != entry.categoryClass) {
+ if (currentCategory != null) {
+ // we've seen the last of this category, put it into the big map
+ registry.put(currentCategory, unmodifiableMap(parsers));
+ }
+ parsers = new HashMap<>();
+ currentCategory = entry.categoryClass;
+ }
+
+ for (String name : entry.name.getAllNamesIncludedDeprecated()) {
+ Object old = parsers.put(name, entry);
+ if (old != null) {
+ throw new IllegalArgumentException("NamedXContent [" + currentCategory.getName() + "][" + entry.name + "]" +
+ " is already registered for [" + old.getClass().getName() + "]," +
+ " cannot register [" + entry.parser.getClass().getName() + "]");
+ }
+ }
+ }
+ // handle the last category
+ registry.put(currentCategory, unmodifiableMap(parsers));
+
+ this.registry = unmodifiableMap(registry);
+ }
+
+ /**
+ * Parse a named object, throwing an exception if the parser isn't found. Throws an {@link ElasticsearchException} if the
+ * {@code categoryClass} isn't registered because this is almost always a bug. Throws a {@link UnknownNamedObjectException} if the
+ * {@code categoryClass} is registered but the {@code name} isn't.
+ */
+ public <T, C> T parseNamedObject(Class<T> categoryClass, String name, XContentParser parser, C context) throws IOException {
+ Map<String, Entry> parsers = registry.get(categoryClass);
+ if (parsers == null) {
+ if (registry.isEmpty()) {
+ // The "empty" registry will never work so we throw a better exception as a hint.
+ throw new ElasticsearchException("namedObject is not supported for this parser");
+ }
+ throw new ElasticsearchException("Unknown namedObject category [" + categoryClass.getName() + "]");
+ }
+ Entry entry = parsers.get(name);
+ if (entry == null) {
+ throw new UnknownNamedObjectException(parser.getTokenLocation(), categoryClass, name);
+ }
+ if (false == entry.name.match(name)) {
+ /* Note that this shouldn't happen because we already looked up the entry using the names but we need to call `match` anyway
+ * because it is responsible for logging deprecation warnings. */
+ throw new ParsingException(parser.getTokenLocation(),
+ "Unknown " + categoryClass.getSimpleName() + " [" + name + "]: Parser didn't match");
+ }
+ return categoryClass.cast(entry.parser.fromXContent(parser, context));
+ }
+
+ /**
+ * Thrown when {@link NamedXContentRegistry#parseNamedObject(Class, String, XContentParser, Object)} is called with an unregistered
+ * name. When this bubbles up to the rest layer it is converted into a response with {@code 400 BAD REQUEST} status.
+ */
+ public static class UnknownNamedObjectException extends ParsingException {
+ private final String categoryClass;
+ private final String name;
+
+ public UnknownNamedObjectException(XContentLocation contentLocation, Class<?> categoryClass,
+ String name) {
+ super(contentLocation, "Unknown " + categoryClass.getSimpleName() + " [" + name + "]");
+ this.categoryClass = requireNonNull(categoryClass, "categoryClass is required").getName();
+ this.name = requireNonNull(name, "name is required");
+ }
+
+ /**
+ * Read from a stream.
+ */
+ public UnknownNamedObjectException(StreamInput in) throws IOException {
+ super(in);
+ categoryClass = in.readString();
+ name = in.readString();
+ }
+
+ @Override
+ public void writeTo(StreamOutput out) throws IOException {
+ super.writeTo(out);
+ out.writeString(categoryClass);
+ out.writeString(name);
+ }
+
+ /**
+ * Category class that was missing a parser. This is a String instead of a class because the class might not be on the classpath
+ * of all nodes or it might be exclusive to a plugin or something.
+ */
+ public String getCategoryClass() {
+ return categoryClass;
+ }
+
+ /**
+ * Name of the missing parser.
+ */
+ public String getName() {
+ return name;
+ }
+ }
+}
diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/XContent.java b/core/src/main/java/org/elasticsearch/common/xcontent/XContent.java
index 72210f09d9..e7dbeafe5d 100644
--- a/core/src/main/java/org/elasticsearch/common/xcontent/XContent.java
+++ b/core/src/main/java/org/elasticsearch/common/xcontent/XContent.java
@@ -83,31 +83,31 @@ public interface XContent {
/**
* Creates a parser over the provided string content.
*/
- XContentParser createParser(String content) throws IOException;
+ XContentParser createParser(NamedXContentRegistry xContentRegistry, String content) throws IOException;
/**
* Creates a parser over the provided input stream.
*/
- XContentParser createParser(InputStream is) throws IOException;
+ XContentParser createParser(NamedXContentRegistry xContentRegistry, InputStream is) throws IOException;
/**
* Creates a parser over the provided bytes.
*/
- XContentParser createParser(byte[] data) throws IOException;
+ XContentParser createParser(NamedXContentRegistry xContentRegistry, byte[] data) throws IOException;
/**
* Creates a parser over the provided bytes.
*/
- XContentParser createParser(byte[] data, int offset, int length) throws IOException;
+ XContentParser createParser(NamedXContentRegistry xContentRegistry, byte[] data, int offset, int length) throws IOException;
/**
* Creates a parser over the provided bytes.
*/
- XContentParser createParser(BytesReference bytes) throws IOException;
+ XContentParser createParser(NamedXContentRegistry xContentRegistry, BytesReference bytes) throws IOException;
/**
* Creates a parser over the provided reader.
*/
- XContentParser createParser(Reader reader) throws IOException;
+ XContentParser createParser(NamedXContentRegistry xContentRegistry, Reader reader) throws IOException;
}
diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/XContentHelper.java b/core/src/main/java/org/elasticsearch/common/xcontent/XContentHelper.java
index b288a314ee..e68a58a750 100644
--- a/core/src/main/java/org/elasticsearch/common/xcontent/XContentHelper.java
+++ b/core/src/main/java/org/elasticsearch/common/xcontent/XContentHelper.java
@@ -41,7 +41,7 @@ import static org.elasticsearch.common.xcontent.ToXContent.EMPTY_PARAMS;
@SuppressWarnings("unchecked")
public class XContentHelper {
- public static XContentParser createParser(BytesReference bytes) throws IOException {
+ public static XContentParser createParser(NamedXContentRegistry xContentRegistry, BytesReference bytes) throws IOException {
Compressor compressor = CompressorFactory.compressor(bytes);
if (compressor != null) {
InputStream compressedInput = compressor.streamInput(bytes.streamInput());
@@ -49,13 +49,14 @@ public class XContentHelper {
compressedInput = new BufferedInputStream(compressedInput);
}
XContentType contentType = XContentFactory.xContentType(compressedInput);
- return XContentFactory.xContent(contentType).createParser(compressedInput);
+ return XContentFactory.xContent(contentType).createParser(xContentRegistry, compressedInput);
} else {
- return XContentFactory.xContent(bytes).createParser(bytes.streamInput());
+ return XContentFactory.xContent(bytes).createParser(xContentRegistry, bytes.streamInput());
}
}
- public static Tuple<XContentType, Map<String, Object>> convertToMap(BytesReference bytes, boolean ordered) throws ElasticsearchParseException {
+ public static Tuple<XContentType, Map<String, Object>> convertToMap(BytesReference bytes, boolean ordered)
+ throws ElasticsearchParseException {
try {
XContentType contentType;
InputStream input;
@@ -71,13 +72,34 @@ public class XContentHelper {
contentType = XContentFactory.xContentType(bytes);
input = bytes.streamInput();
}
- try (XContentParser parser = XContentFactory.xContent(contentType).createParser(input)) {
- if (ordered) {
- return Tuple.tuple(contentType, parser.mapOrdered());
- } else {
- return Tuple.tuple(contentType, parser.map());
- }
- }
+ return new Tuple<>(contentType, convertToMap(XContentFactory.xContent(contentType), input, ordered));
+ } catch (IOException e) {
+ throw new ElasticsearchParseException("Failed to parse content to map", e);
+ }
+ }
+
+ /**
+ * Convert a string in some {@link XContent} format to a {@link Map}. Throws an {@link ElasticsearchParseException} if there is any
+ * error.
+ */
+ public static Map<String, Object> convertToMap(XContent xContent, String string, boolean ordered) throws ElasticsearchParseException {
+ // It is safe to use EMPTY here because this never uses namedObject
+ try (XContentParser parser = xContent.createParser(NamedXContentRegistry.EMPTY, string)) {
+ return ordered ? parser.mapOrdered() : parser.map();
+ } catch (IOException e) {
+ throw new ElasticsearchParseException("Failed to parse content to map", e);
+ }
+ }
+
+ /**
+ * Convert a string in some {@link XContent} format to a {@link Map}. Throws an {@link ElasticsearchParseException} if there is any
+ * error. Note that unlike {@link #convertToMap(BytesReference, boolean)}, this doesn't automatically uncompress the input.
+ */
+ public static Map<String, Object> convertToMap(XContent xContent, InputStream input, boolean ordered)
+ throws ElasticsearchParseException {
+ // It is safe to use EMPTY here because this never uses namedObject
+ try (XContentParser parser = xContent.createParser(NamedXContentRegistry.EMPTY, input)) {
+ return ordered ? parser.mapOrdered() : parser.map();
} catch (IOException e) {
throw new ElasticsearchParseException("Failed to parse content to map", e);
}
@@ -92,7 +114,9 @@ public class XContentHelper {
if (xContentType == XContentType.JSON && !reformatJson) {
return bytes.utf8ToString();
}
- try (XContentParser parser = XContentFactory.xContent(xContentType).createParser(bytes.streamInput())) {
+ // It is safe to use EMPTY here because this never uses namedObject
+ try (XContentParser parser = XContentFactory.xContent(xContentType).createParser(NamedXContentRegistry.EMPTY,
+ bytes.streamInput())) {
parser.nextToken();
XContentBuilder builder = XContentFactory.jsonBuilder();
if (prettyPrint) {
@@ -191,7 +215,6 @@ public class XContentHelper {
* Merges the defaults provided as the second parameter into the content of the first. Only does recursive merge
* for inner maps.
*/
- @SuppressWarnings({"unchecked"})
public static void mergeDefaults(Map<String, Object> content, Map<String, Object> defaults) {
for (Map.Entry<String, Object> defaultEntry : defaults.entrySet()) {
if (!content.containsKey(defaultEntry.getKey())) {
@@ -255,33 +278,36 @@ public class XContentHelper {
return true;
}
- public static void copyCurrentStructure(XContentGenerator generator, XContentParser parser) throws IOException {
+ /**
+ * Low level implementation detail of {@link XContentGenerator#copyCurrentStructure(XContentParser)}.
+ */
+ public static void copyCurrentStructure(XContentGenerator destination, XContentParser parser) throws IOException {
XContentParser.Token token = parser.currentToken();
// Let's handle field-name separately first
if (token == XContentParser.Token.FIELD_NAME) {
- generator.writeFieldName(parser.currentName());
+ destination.writeFieldName(parser.currentName());
token = parser.nextToken();
// fall-through to copy the associated value
}
switch (token) {
case START_ARRAY:
- generator.writeStartArray();
+ destination.writeStartArray();
while (parser.nextToken() != XContentParser.Token.END_ARRAY) {
- copyCurrentStructure(generator, parser);
+ copyCurrentStructure(destination, parser);
}
- generator.writeEndArray();
+ destination.writeEndArray();
break;
case START_OBJECT:
- generator.writeStartObject();
+ destination.writeStartObject();
while (parser.nextToken() != XContentParser.Token.END_OBJECT) {
- copyCurrentStructure(generator, parser);
+ copyCurrentStructure(destination, parser);
}
- generator.writeEndObject();
+ destination.writeEndObject();
break;
default: // others are simple:
- copyCurrentEvent(generator, parser);
+ copyCurrentEvent(destination, parser);
}
}
diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/XContentParser.java b/core/src/main/java/org/elasticsearch/common/xcontent/XContentParser.java
index e5ab2a9f4c..7ca7744226 100644
--- a/core/src/main/java/org/elasticsearch/common/xcontent/XContentParser.java
+++ b/core/src/main/java/org/elasticsearch/common/xcontent/XContentParser.java
@@ -249,5 +249,16 @@ public interface XContentParser extends Releasable {
*/
XContentLocation getTokenLocation();
+ // TODO remove context entirely when it isn't needed
+ /**
+ * Parse an object by name.
+ */
+ <T> T namedObject(Class<T> categoryClass, String name, Object context) throws IOException;
+
+ /**
+ * The registry used to resolve {@link #namedObject(Class, String, Object)}. Use this when building a sub-parser from this parser.
+ */
+ NamedXContentRegistry getXContentRegistry();
+
boolean isClosed();
}
diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/cbor/CborXContent.java b/core/src/main/java/org/elasticsearch/common/xcontent/cbor/CborXContent.java
index d79173cfc2..56435fd364 100644
--- a/core/src/main/java/org/elasticsearch/common/xcontent/cbor/CborXContent.java
+++ b/core/src/main/java/org/elasticsearch/common/xcontent/cbor/CborXContent.java
@@ -26,6 +26,7 @@ import com.fasterxml.jackson.dataformat.cbor.CBORFactory;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.FastStringReader;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentGenerator;
@@ -78,33 +79,33 @@ public class CborXContent implements XContent {
}
@Override
- public XContentParser createParser(String content) throws IOException {
- return new CborXContentParser(cborFactory.createParser(new FastStringReader(content)));
+ public XContentParser createParser(NamedXContentRegistry xContentRegistry, String content) throws IOException {
+ return new CborXContentParser(xContentRegistry, cborFactory.createParser(new FastStringReader(content)));
}
@Override
- public XContentParser createParser(InputStream is) throws IOException {
- return new CborXContentParser(cborFactory.createParser(is));
+ public XContentParser createParser(NamedXContentRegistry xContentRegistry, InputStream is) throws IOException {
+ return new CborXContentParser(xContentRegistry, cborFactory.createParser(is));
}
@Override
- public XContentParser createParser(byte[] data) throws IOException {
- return new CborXContentParser(cborFactory.createParser(data));
+ public XContentParser createParser(NamedXContentRegistry xContentRegistry, byte[] data) throws IOException {
+ return new CborXContentParser(xContentRegistry, cborFactory.createParser(data));
}
@Override
- public XContentParser createParser(byte[] data, int offset, int length) throws IOException {
- return new CborXContentParser(cborFactory.createParser(data, offset, length));
+ public XContentParser createParser(NamedXContentRegistry xContentRegistry, byte[] data, int offset, int length) throws IOException {
+ return new CborXContentParser(xContentRegistry, cborFactory.createParser(data, offset, length));
}
@Override
- public XContentParser createParser(BytesReference bytes) throws IOException {
- return createParser(bytes.streamInput());
+ public XContentParser createParser(NamedXContentRegistry xContentRegistry, BytesReference bytes) throws IOException {
+ return createParser(xContentRegistry, bytes.streamInput());
}
@Override
- public XContentParser createParser(Reader reader) throws IOException {
- return new CborXContentParser(cborFactory.createParser(reader));
+ public XContentParser createParser(NamedXContentRegistry xContentRegistry, Reader reader) throws IOException {
+ return new CborXContentParser(xContentRegistry, cborFactory.createParser(reader));
}
}
diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/cbor/CborXContentParser.java b/core/src/main/java/org/elasticsearch/common/xcontent/cbor/CborXContentParser.java
index 772a5322cc..61b4886420 100644
--- a/core/src/main/java/org/elasticsearch/common/xcontent/cbor/CborXContentParser.java
+++ b/core/src/main/java/org/elasticsearch/common/xcontent/cbor/CborXContentParser.java
@@ -20,13 +20,15 @@
package org.elasticsearch.common.xcontent.cbor;
import com.fasterxml.jackson.core.JsonParser;
+
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.common.xcontent.json.JsonXContentParser;
public class CborXContentParser extends JsonXContentParser {
- public CborXContentParser(JsonParser parser) {
- super(parser);
+ public CborXContentParser(NamedXContentRegistry xContentRegistry, JsonParser parser) {
+ super(xContentRegistry, parser);
}
@Override
diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/json/JsonXContent.java b/core/src/main/java/org/elasticsearch/common/xcontent/json/JsonXContent.java
index 1b0b351e6e..2e4393723e 100644
--- a/core/src/main/java/org/elasticsearch/common/xcontent/json/JsonXContent.java
+++ b/core/src/main/java/org/elasticsearch/common/xcontent/json/JsonXContent.java
@@ -25,6 +25,7 @@ import com.fasterxml.jackson.core.JsonGenerator;
import com.fasterxml.jackson.core.JsonParser;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.FastStringReader;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentGenerator;
@@ -79,32 +80,32 @@ public class JsonXContent implements XContent {
}
@Override
- public XContentParser createParser(String content) throws IOException {
- return new JsonXContentParser(jsonFactory.createParser(new FastStringReader(content)));
+ public XContentParser createParser(NamedXContentRegistry xContentRegistry, String content) throws IOException {
+ return new JsonXContentParser(xContentRegistry, jsonFactory.createParser(new FastStringReader(content)));
}
@Override
- public XContentParser createParser(InputStream is) throws IOException {
- return new JsonXContentParser(jsonFactory.createParser(is));
+ public XContentParser createParser(NamedXContentRegistry xContentRegistry, InputStream is) throws IOException {
+ return new JsonXContentParser(xContentRegistry, jsonFactory.createParser(is));
}
@Override
- public XContentParser createParser(byte[] data) throws IOException {
- return new JsonXContentParser(jsonFactory.createParser(data));
+ public XContentParser createParser(NamedXContentRegistry xContentRegistry, byte[] data) throws IOException {
+ return new JsonXContentParser(xContentRegistry, jsonFactory.createParser(data));
}
@Override
- public XContentParser createParser(byte[] data, int offset, int length) throws IOException {
- return new JsonXContentParser(jsonFactory.createParser(data, offset, length));
+ public XContentParser createParser(NamedXContentRegistry xContentRegistry, byte[] data, int offset, int length) throws IOException {
+ return new JsonXContentParser(xContentRegistry, jsonFactory.createParser(data, offset, length));
}
@Override
- public XContentParser createParser(BytesReference bytes) throws IOException {
- return createParser(bytes.streamInput());
+ public XContentParser createParser(NamedXContentRegistry xContentRegistry, BytesReference bytes) throws IOException {
+ return createParser(xContentRegistry, bytes.streamInput());
}
@Override
- public XContentParser createParser(Reader reader) throws IOException {
- return new JsonXContentParser(jsonFactory.createParser(reader));
+ public XContentParser createParser(NamedXContentRegistry xContentRegistry, Reader reader) throws IOException {
+ return new JsonXContentParser(xContentRegistry, jsonFactory.createParser(reader));
}
}
diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/json/JsonXContentGenerator.java b/core/src/main/java/org/elasticsearch/common/xcontent/json/JsonXContentGenerator.java
index 763fac4c6a..0742e4a716 100644
--- a/core/src/main/java/org/elasticsearch/common/xcontent/json/JsonXContentGenerator.java
+++ b/core/src/main/java/org/elasticsearch/common/xcontent/json/JsonXContentGenerator.java
@@ -31,6 +31,7 @@ import com.fasterxml.jackson.core.util.JsonGeneratorDelegate;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.Streams;
import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContent;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentGenerator;
@@ -312,7 +313,8 @@ public class JsonXContentGenerator implements XContentGenerator {
throw new IllegalArgumentException("Can't write raw bytes whose xcontent-type can't be guessed");
}
if (mayWriteRawData(contentType) == false) {
- try (XContentParser parser = XContentFactory.xContent(contentType).createParser(content)) {
+ // EMPTY is safe here because we never call namedObject when writing raw data
+ try (XContentParser parser = XContentFactory.xContent(contentType).createParser(NamedXContentRegistry.EMPTY, content)) {
parser.nextToken();
writeFieldName(name);
copyCurrentStructure(parser);
@@ -378,8 +380,9 @@ public class JsonXContentGenerator implements XContentGenerator {
}
protected void copyRawValue(BytesReference content, XContent xContent) throws IOException {
+ // EMPTY is safe here because we never call namedObject
try (StreamInput input = content.streamInput();
- XContentParser parser = xContent.createParser(input)) {
+ XContentParser parser = xContent.createParser(NamedXContentRegistry.EMPTY, input)) {
copyCurrentStructure(parser);
}
}
diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/json/JsonXContentParser.java b/core/src/main/java/org/elasticsearch/common/xcontent/json/JsonXContentParser.java
index f7ed46a649..e5c30208ed 100644
--- a/core/src/main/java/org/elasticsearch/common/xcontent/json/JsonXContentParser.java
+++ b/core/src/main/java/org/elasticsearch/common/xcontent/json/JsonXContentParser.java
@@ -22,8 +22,10 @@ package org.elasticsearch.common.xcontent.json;
import com.fasterxml.jackson.core.JsonLocation;
import com.fasterxml.jackson.core.JsonParser;
import com.fasterxml.jackson.core.JsonToken;
+
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentLocation;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.common.xcontent.support.AbstractXContentParser;
@@ -35,7 +37,8 @@ public class JsonXContentParser extends AbstractXContentParser {
final JsonParser parser;
- public JsonXContentParser(JsonParser parser) {
+ public JsonXContentParser(NamedXContentRegistry xContentRegistry, JsonParser parser) {
+ super(xContentRegistry);
this.parser = parser;
}
diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/smile/SmileXContent.java b/core/src/main/java/org/elasticsearch/common/xcontent/smile/SmileXContent.java
index 643326cd82..b43a13a919 100644
--- a/core/src/main/java/org/elasticsearch/common/xcontent/smile/SmileXContent.java
+++ b/core/src/main/java/org/elasticsearch/common/xcontent/smile/SmileXContent.java
@@ -26,6 +26,7 @@ import com.fasterxml.jackson.dataformat.smile.SmileFactory;
import com.fasterxml.jackson.dataformat.smile.SmileGenerator;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.FastStringReader;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentGenerator;
@@ -79,32 +80,32 @@ public class SmileXContent implements XContent {
}
@Override
- public XContentParser createParser(String content) throws IOException {
- return new SmileXContentParser(smileFactory.createParser(new FastStringReader(content)));
+ public XContentParser createParser(NamedXContentRegistry xContentRegistry, String content) throws IOException {
+ return new SmileXContentParser(xContentRegistry, smileFactory.createParser(new FastStringReader(content)));
}
@Override
- public XContentParser createParser(InputStream is) throws IOException {
- return new SmileXContentParser(smileFactory.createParser(is));
+ public XContentParser createParser(NamedXContentRegistry xContentRegistry, InputStream is) throws IOException {
+ return new SmileXContentParser(xContentRegistry, smileFactory.createParser(is));
}
@Override
- public XContentParser createParser(byte[] data) throws IOException {
- return new SmileXContentParser(smileFactory.createParser(data));
+ public XContentParser createParser(NamedXContentRegistry xContentRegistry, byte[] data) throws IOException {
+ return new SmileXContentParser(xContentRegistry, smileFactory.createParser(data));
}
@Override
- public XContentParser createParser(byte[] data, int offset, int length) throws IOException {
- return new SmileXContentParser(smileFactory.createParser(data, offset, length));
+ public XContentParser createParser(NamedXContentRegistry xContentRegistry, byte[] data, int offset, int length) throws IOException {
+ return new SmileXContentParser(xContentRegistry, smileFactory.createParser(data, offset, length));
}
@Override
- public XContentParser createParser(BytesReference bytes) throws IOException {
- return createParser(bytes.streamInput());
+ public XContentParser createParser(NamedXContentRegistry xContentRegistry, BytesReference bytes) throws IOException {
+ return createParser(xContentRegistry, bytes.streamInput());
}
@Override
- public XContentParser createParser(Reader reader) throws IOException {
- return new SmileXContentParser(smileFactory.createParser(reader));
+ public XContentParser createParser(NamedXContentRegistry xContentRegistry, Reader reader) throws IOException {
+ return new SmileXContentParser(xContentRegistry, smileFactory.createParser(reader));
}
}
diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/smile/SmileXContentParser.java b/core/src/main/java/org/elasticsearch/common/xcontent/smile/SmileXContentParser.java
index ad8e12e70b..c7b4b8c000 100644
--- a/core/src/main/java/org/elasticsearch/common/xcontent/smile/SmileXContentParser.java
+++ b/core/src/main/java/org/elasticsearch/common/xcontent/smile/SmileXContentParser.java
@@ -20,13 +20,15 @@
package org.elasticsearch.common.xcontent.smile;
import com.fasterxml.jackson.core.JsonParser;
+
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.common.xcontent.json.JsonXContentParser;
public class SmileXContentParser extends JsonXContentParser {
- public SmileXContentParser(JsonParser parser) {
- super(parser);
+ public SmileXContentParser(NamedXContentRegistry xContentRegistry, JsonParser parser) {
+ super(xContentRegistry, parser);
}
@Override
diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/support/AbstractXContentParser.java b/core/src/main/java/org/elasticsearch/common/xcontent/support/AbstractXContentParser.java
index d13dcbd9c9..162e5f7fb7 100644
--- a/core/src/main/java/org/elasticsearch/common/xcontent/support/AbstractXContentParser.java
+++ b/core/src/main/java/org/elasticsearch/common/xcontent/support/AbstractXContentParser.java
@@ -22,6 +22,7 @@ package org.elasticsearch.common.xcontent.support;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.common.Booleans;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentParser;
import java.io.IOException;
@@ -49,7 +50,11 @@ public abstract class AbstractXContentParser implements XContentParser {
}
}
+ private final NamedXContentRegistry xContentRegistry;
+ public AbstractXContentParser(NamedXContentRegistry xContentRegistry) {
+ this.xContentRegistry = xContentRegistry;
+ }
// The 3rd party parsers we rely on are known to silently truncate fractions: see
// http://fasterxml.github.io/jackson-core/javadoc/2.3.0/com/fasterxml/jackson/core/JsonParser.html#getShortValue()
@@ -357,5 +362,15 @@ public abstract class AbstractXContentParser implements XContentParser {
}
@Override
+ public <T> T namedObject(Class<T> categoryClass, String name, Object context) throws IOException {
+ return xContentRegistry.parseNamedObject(categoryClass, name, this, context);
+ }
+
+ @Override
+ public NamedXContentRegistry getXContentRegistry() {
+ return xContentRegistry;
+ }
+
+ @Override
public abstract boolean isClosed();
}
diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/yaml/YamlXContent.java b/core/src/main/java/org/elasticsearch/common/xcontent/yaml/YamlXContent.java
index 7413f05f58..56dda843c4 100644
--- a/core/src/main/java/org/elasticsearch/common/xcontent/yaml/YamlXContent.java
+++ b/core/src/main/java/org/elasticsearch/common/xcontent/yaml/YamlXContent.java
@@ -25,6 +25,7 @@ import com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.FastStringReader;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentGenerator;
@@ -74,32 +75,32 @@ public class YamlXContent implements XContent {
}
@Override
- public XContentParser createParser(String content) throws IOException {
- return new YamlXContentParser(yamlFactory.createParser(new FastStringReader(content)));
+ public XContentParser createParser(NamedXContentRegistry xContentRegistry, String content) throws IOException {
+ return new YamlXContentParser(xContentRegistry, yamlFactory.createParser(new FastStringReader(content)));
}
@Override
- public XContentParser createParser(InputStream is) throws IOException {
- return new YamlXContentParser(yamlFactory.createParser(is));
+ public XContentParser createParser(NamedXContentRegistry xContentRegistry, InputStream is) throws IOException {
+ return new YamlXContentParser(xContentRegistry, yamlFactory.createParser(is));
}
@Override
- public XContentParser createParser(byte[] data) throws IOException {
- return new YamlXContentParser(yamlFactory.createParser(data));
+ public XContentParser createParser(NamedXContentRegistry xContentRegistry, byte[] data) throws IOException {
+ return new YamlXContentParser(xContentRegistry, yamlFactory.createParser(data));
}
@Override
- public XContentParser createParser(byte[] data, int offset, int length) throws IOException {
- return new YamlXContentParser(yamlFactory.createParser(data, offset, length));
+ public XContentParser createParser(NamedXContentRegistry xContentRegistry, byte[] data, int offset, int length) throws IOException {
+ return new YamlXContentParser(xContentRegistry, yamlFactory.createParser(data, offset, length));
}
@Override
- public XContentParser createParser(BytesReference bytes) throws IOException {
- return createParser(bytes.streamInput());
+ public XContentParser createParser(NamedXContentRegistry xContentRegistry, BytesReference bytes) throws IOException {
+ return createParser(xContentRegistry, bytes.streamInput());
}
@Override
- public XContentParser createParser(Reader reader) throws IOException {
- return new YamlXContentParser(yamlFactory.createParser(reader));
+ public XContentParser createParser(NamedXContentRegistry xContentRegistry, Reader reader) throws IOException {
+ return new YamlXContentParser(xContentRegistry, yamlFactory.createParser(reader));
}
}
diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/yaml/YamlXContentParser.java b/core/src/main/java/org/elasticsearch/common/xcontent/yaml/YamlXContentParser.java
index 5efceac7dc..c2fdcfa740 100644
--- a/core/src/main/java/org/elasticsearch/common/xcontent/yaml/YamlXContentParser.java
+++ b/core/src/main/java/org/elasticsearch/common/xcontent/yaml/YamlXContentParser.java
@@ -20,13 +20,15 @@
package org.elasticsearch.common.xcontent.yaml;
import com.fasterxml.jackson.core.JsonParser;
+
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.common.xcontent.json.JsonXContentParser;
public class YamlXContentParser extends JsonXContentParser {
- public YamlXContentParser(JsonParser parser) {
- super(parser);
+ public YamlXContentParser(NamedXContentRegistry xContentRegistry, JsonParser parser) {
+ super(xContentRegistry, parser);
}
@Override
diff --git a/core/src/main/java/org/elasticsearch/gateway/MetaDataStateFormat.java b/core/src/main/java/org/elasticsearch/gateway/MetaDataStateFormat.java
index 71c3190e2e..a4509315f1 100644
--- a/core/src/main/java/org/elasticsearch/gateway/MetaDataStateFormat.java
+++ b/core/src/main/java/org/elasticsearch/gateway/MetaDataStateFormat.java
@@ -35,6 +35,7 @@ import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.lucene.store.IndexOutputOutputStream;
import org.elasticsearch.common.lucene.store.InputStreamIndexInput;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
@@ -196,8 +197,9 @@ public abstract class MetaDataStateFormat<T> {
long filePointer = indexInput.getFilePointer();
long contentSize = indexInput.length() - CodecUtil.footerLength() - filePointer;
try (IndexInput slice = indexInput.slice("state_xcontent", filePointer, contentSize)) {
- try (XContentParser parser = XContentFactory.xContent(xContentType).createParser(new InputStreamIndexInput(slice,
- contentSize))) {
+ // It is safe to use EMPTY here because this never uses namedObject
+ try (XContentParser parser = XContentFactory.xContent(xContentType).createParser(NamedXContentRegistry.EMPTY,
+ new InputStreamIndexInput(slice, contentSize))) {
return fromXContent(parser);
}
}
@@ -311,7 +313,8 @@ public abstract class MetaDataStateFormat<T> {
logger.debug("{}: no data for [{}], ignoring...", prefix, stateFile.toAbsolutePath());
continue;
}
- try (final XContentParser parser = XContentHelper.createParser(new BytesArray(data))) {
+ // EMPTY is safe here because no parser uses namedObject
+ try (XContentParser parser = XContentHelper.createParser(NamedXContentRegistry.EMPTY, new BytesArray(data))) {
state = fromXContent(parser);
}
if (state == null) {
diff --git a/core/src/main/java/org/elasticsearch/index/IndexModule.java b/core/src/main/java/org/elasticsearch/index/IndexModule.java
index 8389335d88..3734a4eab5 100644
--- a/core/src/main/java/org/elasticsearch/index/IndexModule.java
+++ b/core/src/main/java/org/elasticsearch/index/IndexModule.java
@@ -27,6 +27,7 @@ import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.index.analysis.AnalysisRegistry;
import org.elasticsearch.index.cache.query.DisabledQueryCache;
@@ -320,6 +321,7 @@ public final class IndexModule {
public IndexService newIndexService(
NodeEnvironment environment,
+ NamedXContentRegistry xContentRegistry,
IndexService.ShardStoreDeleter shardStoreDeleter,
CircuitBreakerService circuitBreakerService,
BigArrays bigArrays,
@@ -362,18 +364,18 @@ public final class IndexModule {
} else {
queryCache = new DisabledQueryCache(indexSettings);
}
- return new IndexService(indexSettings, environment, new SimilarityService(indexSettings, similarities), shardStoreDeleter,
- analysisRegistry, engineFactory.get(), circuitBreakerService, bigArrays, threadPool, scriptService, indicesQueriesRegistry,
- clusterService, client, queryCache, store, eventListener, searcherWrapperFactory, mapperRegistry, indicesFieldDataCache,
- globalCheckpointSyncer, searchOperationListeners, indexOperationListeners);
+ return new IndexService(indexSettings, environment, xContentRegistry, new SimilarityService(indexSettings, similarities),
+ shardStoreDeleter, analysisRegistry, engineFactory.get(), circuitBreakerService, bigArrays, threadPool, scriptService,
+ indicesQueriesRegistry, clusterService, client, queryCache, store, eventListener, searcherWrapperFactory, mapperRegistry,
+ indicesFieldDataCache, globalCheckpointSyncer, searchOperationListeners, indexOperationListeners);
}
/**
* creates a new mapper service to do administrative work like mapping updates. This *should not* be used for document parsing.
* doing so will result in an exception.
*/
- public MapperService newIndexMapperService(MapperRegistry mapperRegistry) throws IOException {
- return new MapperService(indexSettings, analysisRegistry.build(indexSettings),
+ public MapperService newIndexMapperService(NamedXContentRegistry xContentRegistry, MapperRegistry mapperRegistry) throws IOException {
+ return new MapperService(indexSettings, analysisRegistry.build(indexSettings), xContentRegistry,
new SimilarityService(indexSettings, similarities), mapperRegistry,
() -> { throw new UnsupportedOperationException("no index query shard context available"); });
}
diff --git a/core/src/main/java/org/elasticsearch/index/IndexService.java b/core/src/main/java/org/elasticsearch/index/IndexService.java
index 5ba2889f4b..11018d75b3 100644
--- a/core/src/main/java/org/elasticsearch/index/IndexService.java
+++ b/core/src/main/java/org/elasticsearch/index/IndexService.java
@@ -34,6 +34,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.concurrent.FutureUtils;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.env.ShardLock;
import org.elasticsearch.env.ShardLockObtainFailedException;
@@ -102,6 +103,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
private final IndexSearcherWrapper searcherWrapper;
private final IndexCache indexCache;
private final MapperService mapperService;
+ private final NamedXContentRegistry xContentRegistry;
private final SimilarityService similarityService;
private final EngineFactory engineFactory;
private final IndexWarmer warmer;
@@ -123,6 +125,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
private final Client client;
public IndexService(IndexSettings indexSettings, NodeEnvironment nodeEnv,
+ NamedXContentRegistry xContentRegistry,
SimilarityService similarityService,
ShardStoreDeleter shardStoreDeleter,
AnalysisRegistry registry,
@@ -146,8 +149,10 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
super(indexSettings);
this.indexSettings = indexSettings;
this.globalCheckpointSyncer = globalCheckpointSyncer;
+ this.xContentRegistry = xContentRegistry;
this.similarityService = similarityService;
- this.mapperService = new MapperService(indexSettings, registry.build(indexSettings), similarityService, mapperRegistry,
+ this.mapperService = new MapperService(indexSettings, registry.build(indexSettings), xContentRegistry, similarityService,
+ mapperRegistry,
// we parse all percolator queries as they would be parsed on shard 0
() -> newQueryShardContext(0, null, () -> {
throw new IllegalArgumentException("Percolator queries are not allowed to use the current timestamp");
@@ -236,6 +241,10 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
return mapperService;
}
+ public NamedXContentRegistry xContentRegistry() {
+ return xContentRegistry;
+ }
+
public SimilarityService similarityService() {
return similarityService;
}
@@ -469,7 +478,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
public QueryShardContext newQueryShardContext(int shardId, IndexReader indexReader, LongSupplier nowInMillis) {
return new QueryShardContext(
shardId, indexSettings, indexCache.bitsetFilterCache(), indexFieldData, mapperService(),
- similarityService(), scriptService, queryRegistry,
+ similarityService(), scriptService, xContentRegistry, queryRegistry,
client, indexReader,
nowInMillis);
}
diff --git a/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapperParser.java b/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapperParser.java
index b3e967bdad..d05cec27b2 100644
--- a/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapperParser.java
+++ b/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapperParser.java
@@ -24,6 +24,7 @@ import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.compress.CompressedXContent;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
@@ -44,6 +45,7 @@ public class DocumentMapperParser {
final MapperService mapperService;
final IndexAnalyzers indexAnalyzers;
+ private final NamedXContentRegistry xContentRegistry;
private final SimilarityService similarityService;
private final Supplier<QueryShardContext> queryShardContextSupplier;
@@ -56,11 +58,12 @@ public class DocumentMapperParser {
private final Map<String, MetadataFieldMapper.TypeParser> rootTypeParsers;
public DocumentMapperParser(IndexSettings indexSettings, MapperService mapperService, IndexAnalyzers indexAnalyzers,
- SimilarityService similarityService, MapperRegistry mapperRegistry,
+ NamedXContentRegistry xContentRegistry, SimilarityService similarityService, MapperRegistry mapperRegistry,
Supplier<QueryShardContext> queryShardContextSupplier) {
this.parseFieldMatcher = new ParseFieldMatcher(indexSettings.getSettings());
this.mapperService = mapperService;
this.indexAnalyzers = indexAnalyzers;
+ this.xContentRegistry = xContentRegistry;
this.similarityService = similarityService;
this.queryShardContextSupplier = queryShardContextSupplier;
this.typeParsers = mapperRegistry.getMapperParsers();
@@ -159,7 +162,7 @@ public class DocumentMapperParser {
private Tuple<String, Map<String, Object>> extractMapping(String type, String source) throws MapperParsingException {
Map<String, Object> root;
- try (XContentParser parser = XContentFactory.xContent(source).createParser(source)) {
+ try (XContentParser parser = XContentFactory.xContent(source).createParser(xContentRegistry, source)) {
root = parser.mapOrdered();
} catch (Exception e) {
throw new MapperParsingException("failed to parse mapping definition", e);
@@ -182,4 +185,8 @@ public class DocumentMapperParser {
}
return mapping;
}
+
+ NamedXContentRegistry getXContentRegistry() {
+ return xContentRegistry;
+ }
}
diff --git a/core/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java b/core/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java
index 72d021c0e3..d88ef2c7f4 100644
--- a/core/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java
+++ b/core/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java
@@ -58,7 +58,7 @@ final class DocumentParser {
final Mapping mapping = docMapper.mapping();
final ParseContext.InternalParseContext context;
- try (XContentParser parser = XContentHelper.createParser(source.source())) {
+ try (XContentParser parser = XContentHelper.createParser(docMapperParser.getXContentRegistry(), source.source())) {
context = new ParseContext.InternalParseContext(indexSettings.getSettings(),
docMapperParser, docMapper, source, parser);
validateStart(parser);
diff --git a/core/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/core/src/main/java/org/elasticsearch/index/mapper/MapperService.java
index 74e9712028..1d34d570a6 100755
--- a/core/src/main/java/org/elasticsearch/index/mapper/MapperService.java
+++ b/core/src/main/java/org/elasticsearch/index/mapper/MapperService.java
@@ -33,6 +33,7 @@ import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.AbstractIndexComponent;
@@ -127,13 +128,14 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
final MapperRegistry mapperRegistry;
- public MapperService(IndexSettings indexSettings, IndexAnalyzers indexAnalyzers,
+ public MapperService(IndexSettings indexSettings, IndexAnalyzers indexAnalyzers, NamedXContentRegistry xContentRegistry,
SimilarityService similarityService, MapperRegistry mapperRegistry,
Supplier<QueryShardContext> queryShardContextSupplier) {
super(indexSettings);
this.indexAnalyzers = indexAnalyzers;
this.fieldTypes = new FieldTypeLookup();
- this.documentParser = new DocumentMapperParser(indexSettings, this, indexAnalyzers, similarityService, mapperRegistry, queryShardContextSupplier);
+ this.documentParser = new DocumentMapperParser(indexSettings, this, indexAnalyzers, xContentRegistry, similarityService,
+ mapperRegistry, queryShardContextSupplier);
this.indexAnalyzer = new MapperAnalyzerWrapper(indexAnalyzers.getDefaultIndexAnalyzer(), p -> p.indexAnalyzer());
this.searchAnalyzer = new MapperAnalyzerWrapper(indexAnalyzers.getDefaultSearchAnalyzer(), p -> p.searchAnalyzer());
this.searchQuoteAnalyzer = new MapperAnalyzerWrapper(indexAnalyzers.getDefaultSearchQuoteAnalyzer(), p -> p.searchQuoteAnalyzer());
@@ -186,8 +188,8 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
return this.documentParser;
}
- public static Map<String, Object> parseMapping(String mappingSource) throws Exception {
- try (XContentParser parser = XContentFactory.xContent(mappingSource).createParser(mappingSource)) {
+ public static Map<String, Object> parseMapping(NamedXContentRegistry xContentRegistry, String mappingSource) throws Exception {
+ try (XContentParser parser = XContentFactory.xContent(mappingSource).createParser(xContentRegistry, mappingSource)) {
return parser.map();
}
}
diff --git a/core/src/main/java/org/elasticsearch/index/query/GeoShapeQueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/GeoShapeQueryBuilder.java
index c54cf2af07..3caf1ea460 100644
--- a/core/src/main/java/org/elasticsearch/index/query/GeoShapeQueryBuilder.java
+++ b/core/src/main/java/org/elasticsearch/index/query/GeoShapeQueryBuilder.java
@@ -39,6 +39,7 @@ import org.elasticsearch.common.geo.SpatialStrategy;
import org.elasticsearch.common.geo.builders.ShapeBuilder;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
@@ -383,7 +384,8 @@ public class GeoShapeQueryBuilder extends AbstractQueryBuilder<GeoShapeQueryBuil
String[] pathElements = path.split("\\.");
int currentPathSlot = 0;
- try (XContentParser parser = XContentHelper.createParser(response.getSourceAsBytesRef())) {
+ // It is safe to use EMPTY here because this never uses namedObject
+ try (XContentParser parser = XContentHelper.createParser(NamedXContentRegistry.EMPTY, response.getSourceAsBytesRef())) {
XContentParser.Token currentToken;
while ((currentToken = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (currentToken == XContentParser.Token.FIELD_NAME) {
diff --git a/core/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java
index cad4f11214..e776f4953d 100644
--- a/core/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java
+++ b/core/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java
@@ -52,10 +52,10 @@ import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.VersionType;
-import org.elasticsearch.index.mapper.MappedFieldType;
-import org.elasticsearch.index.mapper.UidFieldMapper;
import org.elasticsearch.index.mapper.KeywordFieldMapper.KeywordFieldType;
+import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.TextFieldMapper.TextFieldType;
+import org.elasticsearch.index.mapper.UidFieldMapper;
import java.io.IOException;
import java.util.ArrayList;
@@ -421,11 +421,7 @@ public class MoreLikeThisQueryBuilder extends AbstractQueryBuilder<MoreLikeThisQ
if (contentType == builder.contentType()) {
builder.rawField(Field.DOC.getPreferredName(), this.doc);
} else {
- try (XContentParser parser = XContentFactory.xContent(contentType).createParser(this.doc)) {
- parser.nextToken();
- builder.field(Field.DOC.getPreferredName());
- builder.copyCurrentStructure(parser);
- }
+ builder.rawField(Field.DOC.getPreferredName(), doc);
}
}
if (this.fields != null) {
diff --git a/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java b/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java
index 6521369453..e98af78cf2 100644
--- a/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java
+++ b/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java
@@ -23,6 +23,7 @@ import org.elasticsearch.client.Client;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.ParseFieldMatcherSupplier;
import org.elasticsearch.common.bytes.BytesReference;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.mapper.MapperService;
@@ -41,17 +42,19 @@ public class QueryRewriteContext implements ParseFieldMatcherSupplier {
protected final MapperService mapperService;
protected final ScriptService scriptService;
protected final IndexSettings indexSettings;
+ private final NamedXContentRegistry xContentRegistry;
protected final IndicesQueriesRegistry indicesQueriesRegistry;
protected final Client client;
protected final IndexReader reader;
protected final LongSupplier nowInMillis;
public QueryRewriteContext(IndexSettings indexSettings, MapperService mapperService, ScriptService scriptService,
- IndicesQueriesRegistry indicesQueriesRegistry, Client client, IndexReader reader,
- LongSupplier nowInMillis) {
+ NamedXContentRegistry xContentRegistry, IndicesQueriesRegistry indicesQueriesRegistry, Client client, IndexReader reader,
+ LongSupplier nowInMillis) {
this.mapperService = mapperService;
this.scriptService = scriptService;
this.indexSettings = indexSettings;
+ this.xContentRegistry = xContentRegistry;
this.indicesQueriesRegistry = indicesQueriesRegistry;
this.client = client;
this.reader = reader;
@@ -93,6 +96,13 @@ public class QueryRewriteContext implements ParseFieldMatcherSupplier {
}
/**
+ * The registry used to build new {@link XContentParser}s. Contains registered named parsers needed to parse the query.
+ */
+ public NamedXContentRegistry getXContentRegistry() {
+ return xContentRegistry;
+ }
+
+ /**
* Returns a new {@link QueryParseContext} that wraps the provided parser, using the ParseFieldMatcher settings that
* are configured in the index settings. The default script language will always default to Painless.
*/
diff --git a/core/src/main/java/org/elasticsearch/index/query/QueryShardContext.java b/core/src/main/java/org/elasticsearch/index/query/QueryShardContext.java
index 1d5fc1bc9a..a4a1193063 100644
--- a/core/src/main/java/org/elasticsearch/index/query/QueryShardContext.java
+++ b/core/src/main/java/org/elasticsearch/index/query/QueryShardContext.java
@@ -33,6 +33,7 @@ import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.lucene.search.Queries;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.analysis.IndexAnalyzers;
@@ -98,10 +99,10 @@ public class QueryShardContext extends QueryRewriteContext {
private boolean isFilter;
public QueryShardContext(int shardId, IndexSettings indexSettings, BitsetFilterCache bitsetFilterCache,
- IndexFieldDataService indexFieldDataService, MapperService mapperService, SimilarityService similarityService,
- ScriptService scriptService, final IndicesQueriesRegistry indicesQueriesRegistry, Client client,
- IndexReader reader, LongSupplier nowInMillis) {
- super(indexSettings, mapperService, scriptService, indicesQueriesRegistry, client, reader, nowInMillis);
+ IndexFieldDataService indexFieldDataService, MapperService mapperService, SimilarityService similarityService,
+ ScriptService scriptService, NamedXContentRegistry xContentRegistry, IndicesQueriesRegistry indicesQueriesRegistry,
+ Client client, IndexReader reader, LongSupplier nowInMillis) {
+ super(indexSettings, mapperService, scriptService, xContentRegistry, indicesQueriesRegistry, client, reader, nowInMillis);
this.shardId = shardId;
this.indexSettings = indexSettings;
this.similarityService = similarityService;
@@ -116,7 +117,7 @@ public class QueryShardContext extends QueryRewriteContext {
public QueryShardContext(QueryShardContext source) {
this(source.shardId, source.indexSettings, source.bitsetFilterCache, source.indexFieldDataService, source.mapperService,
- source.similarityService, source.scriptService, source.indicesQueriesRegistry, source.client,
+ source.similarityService, source.scriptService, source.getXContentRegistry(), source.indicesQueriesRegistry, source.client,
source.reader, source.nowInMillis);
this.types = source.getTypes();
}
diff --git a/core/src/main/java/org/elasticsearch/index/query/WrapperQueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/WrapperQueryBuilder.java
index 1702679e93..42fc830c88 100644
--- a/core/src/main/java/org/elasticsearch/index/query/WrapperQueryBuilder.java
+++ b/core/src/main/java/org/elasticsearch/index/query/WrapperQueryBuilder.java
@@ -160,7 +160,7 @@ public class WrapperQueryBuilder extends AbstractQueryBuilder<WrapperQueryBuilde
@Override
protected QueryBuilder doRewrite(QueryRewriteContext context) throws IOException {
- try (XContentParser qSourceParser = XContentFactory.xContent(source).createParser(source)) {
+ try (XContentParser qSourceParser = XContentFactory.xContent(source).createParser(context.getXContentRegistry(), source)) {
QueryParseContext parseContext = context.newParseContext(qSourceParser);
final QueryBuilder queryBuilder = parseContext.parseInnerQueryBuilder();
diff --git a/core/src/main/java/org/elasticsearch/index/query/functionscore/DecayFunctionBuilder.java b/core/src/main/java/org/elasticsearch/index/query/functionscore/DecayFunctionBuilder.java
index cf36ea9dff..9faf8dc38c 100644
--- a/core/src/main/java/org/elasticsearch/index/query/functionscore/DecayFunctionBuilder.java
+++ b/core/src/main/java/org/elasticsearch/index/query/functionscore/DecayFunctionBuilder.java
@@ -34,6 +34,7 @@ import org.elasticsearch.common.lucene.search.function.LeafScoreFunction;
import org.elasticsearch.common.lucene.search.function.ScoreFunction;
import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
@@ -145,10 +146,7 @@ public abstract class DecayFunctionBuilder<DFB extends DecayFunctionBuilder<DFB>
@Override
public void doXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(getName());
- builder.field(fieldName);
- try (XContentParser parser = XContentFactory.xContent(functionBytes).createParser(functionBytes)) {
- builder.copyCurrentStructure(parser);
- }
+ builder.rawField(fieldName, functionBytes);
builder.field(DecayFunctionParser.MULTI_VALUE_MODE.getPreferredName(), multiValueMode.name());
builder.endObject();
}
@@ -181,7 +179,8 @@ public abstract class DecayFunctionBuilder<DFB extends DecayFunctionBuilder<DFB>
@Override
protected ScoreFunction doToFunction(QueryShardContext context) throws IOException {
AbstractDistanceScoreFunction scoreFunction;
- try (XContentParser parser = XContentFactory.xContent(functionBytes).createParser(functionBytes)) {
+ // EMPTY is safe because parseVariable doesn't use namedObject
+ try (XContentParser parser = XContentFactory.xContent(functionBytes).createParser(NamedXContentRegistry.EMPTY, functionBytes)) {
scoreFunction = parseVariable(fieldName, parser, context, multiValueMode);
}
return scoreFunction;
diff --git a/core/src/main/java/org/elasticsearch/index/query/functionscore/FunctionScoreQueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/functionscore/FunctionScoreQueryBuilder.java
index b037261d1f..93e2b03e5f 100644
--- a/core/src/main/java/org/elasticsearch/index/query/functionscore/FunctionScoreQueryBuilder.java
+++ b/core/src/main/java/org/elasticsearch/index/query/functionscore/FunctionScoreQueryBuilder.java
@@ -31,7 +31,6 @@ import org.elasticsearch.common.lucene.search.function.FiltersFunctionScoreQuery
import org.elasticsearch.common.lucene.search.function.FiltersFunctionScoreQuery.FilterFunction;
import org.elasticsearch.common.lucene.search.function.FunctionScoreQuery;
import org.elasticsearch.common.lucene.search.function.ScoreFunction;
-import org.elasticsearch.common.xcontent.ParseFieldRegistry;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentLocation;
@@ -436,8 +435,7 @@ public class FunctionScoreQueryBuilder extends AbstractQueryBuilder<FunctionScor
InnerHitBuilder.extractInnerHits(query(), innerHits);
}
- public static FunctionScoreQueryBuilder fromXContent(ParseFieldRegistry<ScoreFunctionParser<?>> scoreFunctionsRegistry,
- QueryParseContext parseContext) throws IOException {
+ public static FunctionScoreQueryBuilder fromXContent(QueryParseContext parseContext) throws IOException {
XContentParser parser = parseContext.parser();
QueryBuilder query = null;
@@ -481,11 +479,8 @@ public class FunctionScoreQueryBuilder extends AbstractQueryBuilder<FunctionScor
singleFunctionFound = true;
singleFunctionName = currentFieldName;
- // we try to parse a score function. If there is no score function for the current field name,
- // getScoreFunction will throw.
- ScoreFunctionBuilder<?> scoreFunction = scoreFunctionsRegistry
- .lookup(currentFieldName, parseContext.getParseFieldMatcher(), parser.getTokenLocation())
- .fromXContent(parseContext);
+ ScoreFunctionBuilder<?> scoreFunction = parser.namedObject(ScoreFunctionBuilder.class, currentFieldName,
+ parseContext);
filterFunctionBuilders.add(new FunctionScoreQueryBuilder.FilterFunctionBuilder(scoreFunction));
}
} else if (token == XContentParser.Token.START_ARRAY) {
@@ -495,7 +490,7 @@ public class FunctionScoreQueryBuilder extends AbstractQueryBuilder<FunctionScor
handleMisplacedFunctionsDeclaration(parser.getTokenLocation(), errorString);
}
functionArrayFound = true;
- currentFieldName = parseFiltersAndFunctions(scoreFunctionsRegistry, parseContext, filterFunctionBuilders);
+ currentFieldName = parseFiltersAndFunctions(parseContext, filterFunctionBuilders);
} else {
throw new ParsingException(parser.getTokenLocation(), "failed to parse [{}] query. array [{}] is not supported",
NAME, currentFieldName);
@@ -562,9 +557,8 @@ public class FunctionScoreQueryBuilder extends AbstractQueryBuilder<FunctionScor
MISPLACED_FUNCTION_MESSAGE_PREFIX + errorString);
}
- private static String parseFiltersAndFunctions(ParseFieldRegistry<ScoreFunctionParser<?>> scoreFunctionsRegistry,
- QueryParseContext parseContext, List<FunctionScoreQueryBuilder.FilterFunctionBuilder> filterFunctionBuilders)
- throws IOException {
+ private static String parseFiltersAndFunctions(QueryParseContext parseContext,
+ List<FunctionScoreQueryBuilder.FilterFunctionBuilder> filterFunctionBuilders) throws IOException {
String currentFieldName = null;
XContentParser.Token token;
XContentParser parser = parseContext.parser();
@@ -589,8 +583,7 @@ public class FunctionScoreQueryBuilder extends AbstractQueryBuilder<FunctionScor
"failed to parse function_score functions. already found [{}], now encountering [{}].",
scoreFunction.getName(), currentFieldName);
}
- scoreFunction = scoreFunctionsRegistry.lookup(currentFieldName, parseContext.getParseFieldMatcher(),
- parser.getTokenLocation()).fromXContent(parseContext);
+ scoreFunction = parser.namedObject(ScoreFunctionBuilder.class, currentFieldName, parseContext);
}
} else if (token.isValue()) {
if (parseContext.getParseFieldMatcher().match(currentFieldName, WEIGHT_FIELD)) {
diff --git a/core/src/main/java/org/elasticsearch/index/shard/CommitPoints.java b/core/src/main/java/org/elasticsearch/index/shard/CommitPoints.java
index e3b6817f13..12a3107622 100644
--- a/core/src/main/java/org/elasticsearch/index/shard/CommitPoints.java
+++ b/core/src/main/java/org/elasticsearch/index/shard/CommitPoints.java
@@ -21,6 +21,7 @@ package org.elasticsearch.index.shard;
import org.apache.lucene.util.CollectionUtil;
import org.elasticsearch.common.bytes.BytesReference;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
@@ -118,7 +119,8 @@ public class CommitPoints implements Iterable<CommitPoint> {
}
public static CommitPoint fromXContent(byte[] data) throws Exception {
- try (XContentParser parser = XContentFactory.xContent(XContentType.JSON).createParser(data)) {
+ // EMPTY is safe here because we never call namedObject
+ try (XContentParser parser = XContentFactory.xContent(XContentType.JSON).createParser(NamedXContentRegistry.EMPTY, data)) {
String currentFieldName = null;
XContentParser.Token token = parser.nextToken();
if (token == null) {
diff --git a/core/src/main/java/org/elasticsearch/indices/IndicesService.java b/core/src/main/java/org/elasticsearch/indices/IndicesService.java
index f4c586abc2..413af6466d 100644
--- a/core/src/main/java/org/elasticsearch/indices/IndicesService.java
+++ b/core/src/main/java/org/elasticsearch/indices/IndicesService.java
@@ -20,6 +20,7 @@
package org.elasticsearch.indices;
import com.carrotsearch.hppc.cursors.ObjectCursor;
+
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.lucene.index.DirectoryReader;
@@ -68,6 +69,8 @@ import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.Callback;
import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.common.util.iterable.Iterables;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
+import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.env.ShardLock;
@@ -133,7 +136,6 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Consumer;
-import java.util.function.Function;
import java.util.function.Supplier;
import java.util.stream.Collectors;
@@ -151,6 +153,7 @@ public class IndicesService extends AbstractLifecycleComponent
Setting.positiveTimeSetting("indices.cache.cleanup_interval", TimeValue.timeValueMinutes(1), Property.NodeScope);
private final PluginsService pluginsService;
private final NodeEnvironment nodeEnv;
+ private final NamedXContentRegistry xContentRegistry;
private final TimeValue shardsClosedTimeout;
private final AnalysisRegistry analysisRegistry;
private final IndicesQueriesRegistry indicesQueriesRegistry;
@@ -182,7 +185,7 @@ public class IndicesService extends AbstractLifecycleComponent
threadPool.schedule(this.cleanInterval, ThreadPool.Names.SAME, this.cacheCleaner);
}
- public IndicesService(Settings settings, PluginsService pluginsService, NodeEnvironment nodeEnv,
+ public IndicesService(Settings settings, PluginsService pluginsService, NodeEnvironment nodeEnv, NamedXContentRegistry xContentRegistry,
ClusterSettings clusterSettings, AnalysisRegistry analysisRegistry,
IndicesQueriesRegistry indicesQueriesRegistry, IndexNameExpressionResolver indexNameExpressionResolver,
MapperRegistry mapperRegistry, NamedWriteableRegistry namedWriteableRegistry,
@@ -193,6 +196,7 @@ public class IndicesService extends AbstractLifecycleComponent
this.threadPool = threadPool;
this.pluginsService = pluginsService;
this.nodeEnv = nodeEnv;
+ this.xContentRegistry = xContentRegistry;
this.shardsClosedTimeout = settings.getAsTime(INDICES_SHARDS_CLOSED_TIMEOUT, new TimeValue(1, TimeUnit.DAYS));
this.analysisRegistry = analysisRegistry;
this.indicesQueriesRegistry = indicesQueriesRegistry;
@@ -441,6 +445,7 @@ public class IndicesService extends AbstractLifecycleComponent
}
return indexModule.newIndexService(
nodeEnv,
+ xContentRegistry,
this,
circuitBreakerService,
bigArrays,
@@ -465,7 +470,7 @@ public class IndicesService extends AbstractLifecycleComponent
final IndexSettings idxSettings = new IndexSettings(indexMetaData, this.settings, indexScopeSetting);
final IndexModule indexModule = new IndexModule(idxSettings, analysisRegistry);
pluginsService.onIndexModule(indexModule);
- return indexModule.newIndexMapperService(mapperRegistry);
+ return indexModule.newIndexMapperService(xContentRegistry, mapperRegistry);
}
/**
@@ -1259,11 +1264,16 @@ public class IndicesService extends AbstractLifecycleComponent
private final IndexDeletionAllowedPredicate ALWAYS_TRUE = (Index index, IndexSettings indexSettings) -> true;
public AliasFilter buildAliasFilter(ClusterState state, String index, String... expressions) {
- Function<XContentParser, QueryParseContext> factory =
- (parser) -> new QueryParseContext(indicesQueriesRegistry, parser, new ParseFieldMatcher(settings));
+ /* Being static, parseAliasFilter doesn't have access to whatever guts it needs to parse a query. Instead of passing in a bunch
+ * of dependencies we pass in a function that can perform the parsing. */
+ ShardSearchRequest.FilterParser filterParser = bytes -> {
+ try (XContentParser parser = XContentFactory.xContent(bytes).createParser(xContentRegistry, bytes)) {
+ return new QueryParseContext(indicesQueriesRegistry, parser, new ParseFieldMatcher(settings)).parseInnerQueryBuilder();
+ }
+ };
String[] aliases = indexNameExpressionResolver.filteringAliases(state, index, expressions);
IndexMetaData indexMetaData = state.metaData().index(index);
- return new AliasFilter(ShardSearchRequest.parseAliasFilter(factory, indexMetaData, aliases), aliases);
+ return new AliasFilter(ShardSearchRequest.parseAliasFilter(filterParser, indexMetaData, aliases), aliases);
}
}
diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java
index a0d50b665d..bfe5ad453b 100644
--- a/core/src/main/java/org/elasticsearch/node/Node.java
+++ b/core/src/main/java/org/elasticsearch/node/Node.java
@@ -74,6 +74,7 @@ import org.elasticsearch.common.transport.BoundTransportAddress;
import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.BigArrays;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.discovery.DiscoveryModule;
import org.elasticsearch.discovery.DiscoverySettings;
@@ -159,6 +160,8 @@ import java.util.function.UnaryOperator;
import java.util.stream.Collectors;
import java.util.stream.Stream;
+import static java.util.stream.Collectors.toList;
+
/**
* A node represent a node within a cluster (<tt>cluster.name</tt>). The {@link #client()} can be used
* in order to use a {@link Client} to perform actions/operations against the cluster.
@@ -362,8 +365,13 @@ public class Node implements Closeable {
.flatMap(p -> p.getNamedWriteables().stream()))
.flatMap(Function.identity()).collect(Collectors.toList());
final NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(namedWriteables);
+ NamedXContentRegistry xContentRegistry = new NamedXContentRegistry(Stream.of(
+ searchModule.getNamedXContents().stream(),
+ pluginsService.filterPlugins(Plugin.class).stream()
+ .flatMap(p -> p.getNamedXContent().stream())
+ ).flatMap(Function.identity()).collect(toList()));
final MetaStateService metaStateService = new MetaStateService(settings, nodeEnvironment);
- final IndicesService indicesService = new IndicesService(settings, pluginsService, nodeEnvironment,
+ final IndicesService indicesService = new IndicesService(settings, pluginsService, nodeEnvironment, xContentRegistry,
settingsModule.getClusterSettings(), analysisModule.getAnalysisRegistry(), searchModule.getQueryParserRegistry(),
clusterModule.getIndexNameExpressionResolver(), indicesModule.getMapperRegistry(), namedWriteableRegistry,
threadPool, settingsModule.getIndexScopedSettings(), circuitBreakerService, bigArrays, scriptModule.getScriptService(),
@@ -371,14 +379,15 @@ public class Node implements Closeable {
Collection<Object> pluginComponents = pluginsService.filterPlugins(Plugin.class).stream()
.flatMap(p -> p.createComponents(client, clusterService, threadPool, resourceWatcherService,
- scriptModule.getScriptService(), searchModule.getSearchRequestParsers()).stream())
+ scriptModule.getScriptService(), searchModule.getSearchRequestParsers(),
+ xContentRegistry).stream())
.collect(Collectors.toList());
Collection<UnaryOperator<Map<String, MetaData.Custom>>> customMetaDataUpgraders =
pluginsService.filterPlugins(Plugin.class).stream()
.map(Plugin::getCustomMetaDataUpgrader)
.collect(Collectors.toList());
- final NetworkModule networkModule = new NetworkModule(settings, false, pluginsService.filterPlugins(NetworkPlugin.class), threadPool,
- bigArrays, circuitBreakerService, namedWriteableRegistry, networkService);
+ final NetworkModule networkModule = new NetworkModule(settings, false, pluginsService.filterPlugins(NetworkPlugin.class),
+ threadPool, bigArrays, circuitBreakerService, namedWriteableRegistry, xContentRegistry, networkService);
final MetaDataUpgrader metaDataUpgrader = new MetaDataUpgrader(customMetaDataUpgraders);
final Transport transport = networkModule.getTransportSupplier().get();
final TransportService transportService = newTransportService(settings, transport, threadPool,
@@ -404,6 +413,7 @@ public class Node implements Closeable {
b.bind(IndicesQueriesRegistry.class).toInstance(searchModule.getQueryParserRegistry());
b.bind(SearchRequestParsers.class).toInstance(searchModule.getSearchRequestParsers());
b.bind(SearchExtRegistry.class).toInstance(searchModule.getSearchExtRegistry());
+ b.bind(NamedXContentRegistry.class).toInstance(xContentRegistry);
b.bind(PluginsService.class).toInstance(pluginsService);
b.bind(Client.class).toInstance(client);
b.bind(NodeClient.class).toInstance(client);
@@ -432,7 +442,7 @@ public class Node implements Closeable {
b.bind(AllocationCommandRegistry.class).toInstance(NetworkModule.getAllocationCommandRegistry());
b.bind(UpdateHelper.class).toInstance(new UpdateHelper(settings, scriptModule.getScriptService()));
b.bind(MetaDataIndexUpgradeService.class).toInstance(new MetaDataIndexUpgradeService(settings,
- indicesModule.getMapperRegistry(), settingsModule.getIndexScopedSettings()));
+ xContentRegistry, indicesModule.getMapperRegistry(), settingsModule.getIndexScopedSettings()));
b.bind(ClusterInfoService.class).toInstance(clusterInfoService);
b.bind(Discovery.class).toInstance(discoveryModule.getDiscovery());
{
diff --git a/core/src/main/java/org/elasticsearch/plugins/NetworkPlugin.java b/core/src/main/java/org/elasticsearch/plugins/NetworkPlugin.java
index 32d0e6058e..991a21f1b3 100644
--- a/core/src/main/java/org/elasticsearch/plugins/NetworkPlugin.java
+++ b/core/src/main/java/org/elasticsearch/plugins/NetworkPlugin.java
@@ -27,6 +27,7 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.network.NetworkService;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.http.HttpServerTransport;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.threadpool.ThreadPool;
@@ -62,9 +63,8 @@ public interface NetworkPlugin {
* See {@link org.elasticsearch.common.network.NetworkModule#HTTP_TYPE_SETTING} to configure a specific implementation.
*/
default Map<String, Supplier<HttpServerTransport>> getHttpTransports(Settings settings, ThreadPool threadPool, BigArrays bigArrays,
- CircuitBreakerService circuitBreakerService,
- NamedWriteableRegistry namedWriteableRegistry,
- NetworkService networkService) {
+ CircuitBreakerService circuitBreakerService, NamedWriteableRegistry namedWriteableRegistry,
+ NamedXContentRegistry xContentRegistry, NetworkService networkService) {
return Collections.emptyMap();
}
}
diff --git a/core/src/main/java/org/elasticsearch/plugins/Plugin.java b/core/src/main/java/org/elasticsearch/plugins/Plugin.java
index 32d902248a..e7d97b0724 100644
--- a/core/src/main/java/org/elasticsearch/plugins/Plugin.java
+++ b/core/src/main/java/org/elasticsearch/plugins/Plugin.java
@@ -19,12 +19,6 @@
package org.elasticsearch.plugins;
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-
import org.elasticsearch.action.ActionModule;
import org.elasticsearch.bootstrap.BootstrapCheck;
import org.elasticsearch.client.Client;
@@ -39,6 +33,8 @@ import org.elasticsearch.common.network.NetworkModule;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.SettingsModule;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
+import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.discovery.DiscoveryModule;
import org.elasticsearch.index.IndexModule;
import org.elasticsearch.indices.analysis.AnalysisModule;
@@ -51,6 +47,11 @@ import org.elasticsearch.threadpool.ExecutorBuilder;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.watcher.ResourceWatcherService;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
import java.util.Map;
import java.util.function.UnaryOperator;
@@ -106,7 +107,7 @@ public abstract class Plugin implements Closeable {
*/
public Collection<Object> createComponents(Client client, ClusterService clusterService, ThreadPool threadPool,
ResourceWatcherService resourceWatcherService, ScriptService scriptService,
- SearchRequestParsers searchRequestParsers) {
+ SearchRequestParsers searchRequestParsers, NamedXContentRegistry xContentRegistry) {
return Collections.emptyList();
}
@@ -127,6 +128,14 @@ public abstract class Plugin implements Closeable {
}
/**
+ * Returns parsers for named objects this plugin will parse from {@link XContentParser#namedObject(Class, String, Object)}.
+ * @see NamedWriteableRegistry
+ */
+ public List<NamedXContentRegistry.Entry> getNamedXContent() {
+ return Collections.emptyList();
+ }
+
+ /**
* Called before a new index is created on a node. The given module can be used to register index-level
* extensions.
*/
diff --git a/core/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreFormat.java b/core/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreFormat.java
index 04900705e0..aadf871c09 100644
--- a/core/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreFormat.java
+++ b/core/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreFormat.java
@@ -23,6 +23,7 @@ import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.blobstore.BlobContainer;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.xcontent.FromXContentBuilder;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
@@ -109,10 +110,10 @@ public abstract class BlobStoreFormat<T extends ToXContent> {
}
protected T read(BytesReference bytes) throws IOException {
- try (XContentParser parser = XContentHelper.createParser(bytes)) {
+ // EMPTY is safe here because no reader calls namedObject
+ try (XContentParser parser = XContentHelper.createParser(NamedXContentRegistry.EMPTY, bytes)) {
T obj = reader.fromXContent(parser, parseFieldMatcher);
return obj;
-
}
}
diff --git a/core/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/core/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
index aa487056cc..c7abded5e0 100644
--- a/core/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
+++ b/core/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
@@ -68,6 +68,7 @@ import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.util.iterable.Iterables;
import org.elasticsearch.common.util.set.Sets;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
@@ -625,7 +626,8 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
try (InputStream blob = snapshotsBlobContainer.readBlob(snapshotsIndexBlobName)) {
BytesStreamOutput out = new BytesStreamOutput();
Streams.copy(blob, out);
- try (XContentParser parser = XContentHelper.createParser(out.bytes())) {
+ // EMPTY is safe here because RepositoryData#fromXContent calls namedObject
+ try (XContentParser parser = XContentHelper.createParser(NamedXContentRegistry.EMPTY, out.bytes())) {
repositoryData = RepositoryData.fromXContent(parser);
}
}
diff --git a/core/src/main/java/org/elasticsearch/rest/RestRequest.java b/core/src/main/java/org/elasticsearch/rest/RestRequest.java
index ad6a367a7b..8c05a2b3ae 100644
--- a/core/src/main/java/org/elasticsearch/rest/RestRequest.java
+++ b/core/src/main/java/org/elasticsearch/rest/RestRequest.java
@@ -28,6 +28,7 @@ import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
@@ -46,11 +47,13 @@ import static org.elasticsearch.common.unit.TimeValue.parseTimeValue;
public abstract class RestRequest implements ToXContent.Params {
+ private final NamedXContentRegistry xContentRegistry;
private final Map<String, String> params;
private final String rawPath;
private final Set<String> consumedParams = new HashSet<>();
- public RestRequest(String uri) {
+ public RestRequest(NamedXContentRegistry xContentRegistry, String uri) {
+ this.xContentRegistry = xContentRegistry;
final Map<String, String> params = new HashMap<>();
int pathEndPos = uri.indexOf('?');
if (pathEndPos < 0) {
@@ -62,7 +65,8 @@ public abstract class RestRequest implements ToXContent.Params {
this.params = params;
}
- public RestRequest(Map<String, String> params, String path) {
+ public RestRequest(NamedXContentRegistry xContentRegistry, Map<String, String> params, String path) {
+ this.xContentRegistry = xContentRegistry;
this.params = params;
this.rawPath = path;
}
@@ -229,6 +233,13 @@ public abstract class RestRequest implements ToXContent.Params {
}
/**
+ * Get the {@link NamedXContentRegistry} that should be used to create parsers from this request.
+ */
+ public NamedXContentRegistry getXContentRegistry() {
+ return xContentRegistry;
+ }
+
+ /**
* A parser for the contents of this request if there is a body, otherwise throws an {@link ElasticsearchParseException}. Use
* {@link #applyContentParser(CheckedConsumer)} if you want to gracefully handle when the request doesn't have any contents. Use
* {@link #contentOrSourceParamParser()} for requests that support specifying the request body in the {@code source} param.
@@ -238,7 +249,7 @@ public abstract class RestRequest implements ToXContent.Params {
if (content.length() == 0) {
throw new ElasticsearchParseException("Body required");
}
- return XContentFactory.xContent(content).createParser(content);
+ return XContentFactory.xContent(content).createParser(xContentRegistry, content);
}
/**
@@ -270,7 +281,7 @@ public abstract class RestRequest implements ToXContent.Params {
if (content.length() == 0) {
throw new ElasticsearchParseException("Body required");
}
- return XContentFactory.xContent(content).createParser(content);
+ return XContentFactory.xContent(content).createParser(xContentRegistry, content);
}
/**
@@ -281,7 +292,7 @@ public abstract class RestRequest implements ToXContent.Params {
public final void withContentOrSourceParamParserOrNull(CheckedConsumer<XContentParser, IOException> withParser) throws IOException {
BytesReference content = contentOrSourceParam();
if (content.length() > 0) {
- try (XContentParser parser = XContentFactory.xContent(content).createParser(content)) {
+ try (XContentParser parser = XContentFactory.xContent(content).createParser(xContentRegistry, content)) {
withParser.accept(parser);
}
} else {
diff --git a/core/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java b/core/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java
index 6dfd587f84..502d427050 100644
--- a/core/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java
+++ b/core/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java
@@ -82,7 +82,6 @@ public class RestMultiSearchAction extends BaseRestHandler {
public static MultiSearchRequest parseRequest(RestRequest restRequest, boolean allowExplicitIndex,
SearchRequestParsers searchRequestParsers,
ParseFieldMatcher parseFieldMatcher) throws IOException {
-
MultiSearchRequest multiRequest = new MultiSearchRequest();
if (restRequest.hasParam("max_concurrent_searches")) {
multiRequest.maxConcurrentSearchRequests(restRequest.paramAsInt("max_concurrent_searches", 0));
@@ -107,7 +106,7 @@ public class RestMultiSearchAction extends BaseRestHandler {
* Parses a multi-line {@link RestRequest} body, instanciating a {@link SearchRequest} for each line and applying the given consumer.
*/
public static void parseMultiLineRequest(RestRequest request, IndicesOptions indicesOptions, boolean allowExplicitIndex,
- BiConsumer<SearchRequest, XContentParser> consumer) throws IOException {
+ BiConsumer<SearchRequest, XContentParser> consumer) throws IOException {
String[] indices = Strings.splitStringByCommaToArray(request.param("index"));
String[] types = Strings.splitStringByCommaToArray(request.param("type"));
@@ -153,7 +152,7 @@ public class RestMultiSearchAction extends BaseRestHandler {
// now parse the action
if (nextMarker - from > 0) {
- try (XContentParser parser = xContent.createParser(data.slice(from, nextMarker - from))) {
+ try (XContentParser parser = xContent.createParser(request.getXContentRegistry(), data.slice(from, nextMarker - from))) {
Map<String, Object> source = parser.map();
for (Map.Entry<String, Object> entry : source.entrySet()) {
Object value = entry.getValue();
@@ -187,7 +186,7 @@ public class RestMultiSearchAction extends BaseRestHandler {
break;
}
BytesReference bytes = data.slice(from, nextMarker - from);
- try (XContentParser parser = XContentFactory.xContent(bytes).createParser(bytes)) {
+ try (XContentParser parser = XContentFactory.xContent(bytes).createParser(request.getXContentRegistry(), bytes)) {
consumer.accept(searchRequest, parser);
}
// move pointers
diff --git a/core/src/main/java/org/elasticsearch/script/ScriptMetaData.java b/core/src/main/java/org/elasticsearch/script/ScriptMetaData.java
index 84855da2f9..44456dbec2 100644
--- a/core/src/main/java/org/elasticsearch/script/ScriptMetaData.java
+++ b/core/src/main/java/org/elasticsearch/script/ScriptMetaData.java
@@ -28,6 +28,7 @@ import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
@@ -75,7 +76,8 @@ public final class ScriptMetaData implements MetaData.Custom {
// 2) wrapped into a 'template' json object or field
// 3) just as is
// In order to fetch the actual script in consistent manner this parsing logic is needed:
- try (XContentParser parser = XContentHelper.createParser(scriptAsBytes);
+ // EMPTY is ok here because we never call namedObject, we're just copying structure.
+ try (XContentParser parser = XContentHelper.createParser(NamedXContentRegistry.EMPTY, scriptAsBytes);
XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON)) {
parser.nextToken();
parser.nextToken();
diff --git a/core/src/main/java/org/elasticsearch/search/SearchModule.java b/core/src/main/java/org/elasticsearch/search/SearchModule.java
index cdfcaeab90..a5ea15c670 100644
--- a/core/src/main/java/org/elasticsearch/search/SearchModule.java
+++ b/core/src/main/java/org/elasticsearch/search/SearchModule.java
@@ -23,11 +23,14 @@ import org.apache.lucene.search.BooleanQuery;
import org.elasticsearch.common.NamedRegistry;
import org.elasticsearch.common.geo.ShapesAvailability;
import org.elasticsearch.common.geo.builders.ShapeBuilders;
+import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry.Entry;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.ParseFieldRegistry;
+import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.BoolQueryBuilder;
import org.elasticsearch.index.query.BoostingQueryBuilder;
import org.elasticsearch.index.query.CommonTermsQueryBuilder;
@@ -54,6 +57,7 @@ import org.elasticsearch.index.query.NestedQueryBuilder;
import org.elasticsearch.index.query.ParentIdQueryBuilder;
import org.elasticsearch.index.query.PrefixQueryBuilder;
import org.elasticsearch.index.query.QueryBuilder;
+import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.index.query.QueryStringQueryBuilder;
import org.elasticsearch.index.query.RangeQueryBuilder;
import org.elasticsearch.index.query.RegexpQueryBuilder;
@@ -79,7 +83,6 @@ import org.elasticsearch.index.query.functionscore.GaussDecayFunctionBuilder;
import org.elasticsearch.index.query.functionscore.LinearDecayFunctionBuilder;
import org.elasticsearch.index.query.functionscore.RandomScoreFunctionBuilder;
import org.elasticsearch.index.query.functionscore.ScoreFunctionBuilder;
-import org.elasticsearch.index.query.functionscore.ScoreFunctionParser;
import org.elasticsearch.index.query.functionscore.ScriptScoreFunctionBuilder;
import org.elasticsearch.index.query.functionscore.WeightBuilder;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
@@ -266,7 +269,6 @@ public class SearchModule {
private final boolean transportClient;
private final Map<String, Highlighter> highlighters;
private final Map<String, Suggester<?>> suggesters;
- private final ParseFieldRegistry<ScoreFunctionParser<?>> scoreFunctionParserRegistry = new ParseFieldRegistry<>("score_function");
private final IndicesQueriesRegistry queryParserRegistry = new IndicesQueriesRegistry();
private final ParseFieldRegistry<Aggregator.Parser> aggregationParserRegistry = new ParseFieldRegistry<>("aggregation");
private final ParseFieldRegistry<PipelineAggregator.Parser> pipelineAggregationParserRegistry = new ParseFieldRegistry<>(
@@ -281,7 +283,8 @@ public class SearchModule {
private final SearchExtRegistry searchExtParserRegistry = new SearchExtRegistry();
private final Settings settings;
- private final List<Entry> namedWriteables = new ArrayList<>();
+ private final List<NamedWriteableRegistry.Entry> namedWriteables = new ArrayList<>();
+ private final List<NamedXContentRegistry.Entry> namedXContents = new ArrayList<>();
private final SearchRequestParsers searchRequestParsers;
public SearchModule(Settings settings, boolean transportClient, List<SearchPlugin> plugins) {
@@ -304,10 +307,14 @@ public class SearchModule {
searchRequestParsers = new SearchRequestParsers(queryParserRegistry, aggregatorParsers, getSuggesters(), searchExtParserRegistry);
}
- public List<Entry> getNamedWriteables() {
+ public List<NamedWriteableRegistry.Entry> getNamedWriteables() {
return namedWriteables;
}
+ public List<NamedXContentRegistry.Entry> getNamedXContents() {
+ return namedXContents;
+ }
+
public Suggesters getSuggesters() {
return new Suggesters(suggesters);
}
@@ -618,8 +625,12 @@ public class SearchModule {
}
private void registerScoreFunction(ScoreFunctionSpec<?> scoreFunction) {
- scoreFunctionParserRegistry.register(scoreFunction.getParser(), scoreFunction.getName());
- namedWriteables.add(new Entry(ScoreFunctionBuilder.class, scoreFunction.getName().getPreferredName(), scoreFunction.getReader()));
+ namedWriteables.add(new NamedWriteableRegistry.Entry(
+ ScoreFunctionBuilder.class, scoreFunction.getName().getPreferredName(), scoreFunction.getReader()));
+ // TODO remove funky contexts
+ namedXContents.add(new NamedXContentRegistry.Entry(
+ ScoreFunctionBuilder.class, scoreFunction.getName(),
+ (XContentParser p, Object c) -> scoreFunction.getParser().fromXContent((QueryParseContext) c)));
}
private void registerValueFormats() {
@@ -742,7 +753,7 @@ public class SearchModule {
registerQuery(
new QuerySpec<>(SpanMultiTermQueryBuilder.NAME, SpanMultiTermQueryBuilder::new, SpanMultiTermQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(FunctionScoreQueryBuilder.NAME, FunctionScoreQueryBuilder::new,
- c -> FunctionScoreQueryBuilder.fromXContent(scoreFunctionParserRegistry, c)));
+ FunctionScoreQueryBuilder::fromXContent));
registerQuery(
new QuerySpec<>(SimpleQueryStringBuilder.NAME, SimpleQueryStringBuilder::new, SimpleQueryStringBuilder::fromXContent));
registerQuery(new QuerySpec<>(TypeQueryBuilder.NAME, TypeQueryBuilder::new, TypeQueryBuilder::fromXContent));
diff --git a/core/src/main/java/org/elasticsearch/search/internal/AliasFilter.java b/core/src/main/java/org/elasticsearch/search/internal/AliasFilter.java
index fd68438052..8d55dfbab0 100644
--- a/core/src/main/java/org/elasticsearch/search/internal/AliasFilter.java
+++ b/core/src/main/java/org/elasticsearch/search/internal/AliasFilter.java
@@ -25,6 +25,8 @@ import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
+import org.elasticsearch.common.xcontent.XContentFactory;
+import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryRewriteContext;
@@ -62,7 +64,14 @@ public final class AliasFilter implements Writeable {
if (reparseAliases) {
// we are processing a filter received from a 5.0 node - we need to reparse this on the executing node
final IndexMetaData indexMetaData = context.getIndexSettings().getIndexMetaData();
- return ShardSearchRequest.parseAliasFilter(context::newParseContext, indexMetaData, aliases);
+ /* Being static, parseAliasFilter doesn't have access to whatever guts it needs to parse a query. Instead of passing in a bunch
+ * of dependencies we pass in a function that can perform the parsing. */
+ ShardSearchRequest.FilterParser filterParser = bytes -> {
+ try (XContentParser parser = XContentFactory.xContent(bytes).createParser(context.getXContentRegistry(), bytes)) {
+ return context.newParseContext(parser).parseInnerQueryBuilder();
+ }
+ };
+ return ShardSearchRequest.parseAliasFilter(filterParser, indexMetaData, aliases);
}
return filter;
}
diff --git a/core/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java b/core/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java
index cc7dc3eb5f..f021d7730c 100644
--- a/core/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java
+++ b/core/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java
@@ -24,12 +24,9 @@ import org.elasticsearch.cluster.metadata.AliasMetaData;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.collect.ImmutableOpenMap;
-import org.elasticsearch.common.xcontent.XContentFactory;
-import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.query.BoolQueryBuilder;
import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.AliasFilterParsingException;
@@ -91,13 +88,17 @@ public interface ShardSearchRequest {
*/
void rewrite(QueryShardContext context) throws IOException;
+ @FunctionalInterface
+ public interface FilterParser {
+ QueryBuilder parse(byte[] bytes) throws IOException;
+ }
/**
* Returns the filter associated with listed filtering aliases.
* <p>
* The list of filtering aliases should be obtained by calling MetaData.filteringAliases.
* Returns <tt>null</tt> if no filtering is required.</p>
*/
- static QueryBuilder parseAliasFilter(Function<XContentParser, QueryParseContext> contextFactory,
+ static QueryBuilder parseAliasFilter(FilterParser filterParser,
IndexMetaData metaData, String... aliasNames) {
if (aliasNames == null || aliasNames.length == 0) {
return null;
@@ -109,10 +110,7 @@ public interface ShardSearchRequest {
return null;
}
try {
- byte[] filterSource = alias.filter().uncompressed();
- try (XContentParser parser = XContentFactory.xContent(filterSource).createParser(filterSource)) {
- return contextFactory.apply(parser).parseInnerQueryBuilder();
- }
+ return filterParser.parse(alias.filter().uncompressed());
} catch (IOException ex) {
throw new AliasFilterParsingException(index, alias.getAlias(), "Invalid alias filter", ex);
}
@@ -128,19 +126,19 @@ public interface ShardSearchRequest {
// we need to bench here a bit, to see maybe it makes sense to use OrFilter
BoolQueryBuilder combined = new BoolQueryBuilder();
for (String aliasName : aliasNames) {
- AliasMetaData alias = aliases.get(aliasName);
- if (alias == null) {
- // This shouldn't happen unless alias disappeared after filteringAliases was called.
- throw new InvalidAliasNameException(index, aliasNames[0],
- "Unknown alias name was passed to alias Filter");
- }
- QueryBuilder parsedFilter = parserFunction.apply(alias);
- if (parsedFilter != null) {
- combined.should(parsedFilter);
- } else {
- // The filter might be null only if filter was removed after filteringAliases was called
- return null;
- }
+ AliasMetaData alias = aliases.get(aliasName);
+ if (alias == null) {
+ // This shouldn't happen unless alias disappeared after filteringAliases was called.
+ throw new InvalidAliasNameException(index, aliasNames[0],
+ "Unknown alias name was passed to alias Filter");
+ }
+ QueryBuilder parsedFilter = parserFunction.apply(alias);
+ if (parsedFilter != null) {
+ combined.should(parsedFilter);
+ } else {
+ // The filter might be null only if filter was removed after filteringAliases was called
+ return null;
+ }
}
return combined;
}
diff --git a/core/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggestionBuilder.java b/core/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggestionBuilder.java
index 09382d9aaf..0fd3726384 100644
--- a/core/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggestionBuilder.java
+++ b/core/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggestionBuilder.java
@@ -42,7 +42,6 @@ import org.elasticsearch.search.suggest.completion.context.ContextMapping;
import org.elasticsearch.search.suggest.completion.context.ContextMappings;
import java.io.IOException;
-import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -233,10 +232,7 @@ public class CompletionSuggestionBuilder extends SuggestionBuilder<CompletionSug
regexOptions.toXContent(builder, params);
}
if (contextBytes != null) {
- try (XContentParser contextParser = XContentFactory.xContent(XContentType.JSON).createParser(contextBytes)) {
- builder.field(CONTEXTS_FIELD.getPreferredName());
- builder.copyCurrentStructure(contextParser);
- }
+ builder.rawField(CONTEXTS_FIELD.getPreferredName(), contextBytes);
}
return builder;
}
@@ -270,7 +266,8 @@ public class CompletionSuggestionBuilder extends SuggestionBuilder<CompletionSug
CompletionFieldMapper.CompletionFieldType type = (CompletionFieldMapper.CompletionFieldType) mappedFieldType;
suggestionContext.setFieldType(type);
if (type.hasContextMappings() && contextBytes != null) {
- try (XContentParser contextParser = XContentFactory.xContent(contextBytes).createParser(contextBytes)) {
+ try (XContentParser contextParser = XContentFactory.xContent(contextBytes).createParser(context.getXContentRegistry(),
+ contextBytes)) {
if (type.hasContextMappings() && contextParser != null) {
ContextMappings contextMappings = type.getContextMappings();
contextParser.nextToken();
diff --git a/core/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggester.java b/core/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggester.java
index 8eb0aa0529..88648395e6 100644
--- a/core/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggester.java
+++ b/core/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggester.java
@@ -120,7 +120,8 @@ public final class PhraseSuggester extends Suggester<PhraseSuggestionContext> {
QueryShardContext shardContext = suggestion.getShardContext();
final ExecutableScript executable = collateScript.apply(vars);
final BytesReference querySource = (BytesReference) executable.run();
- try (XContentParser parser = XContentFactory.xContent(querySource).createParser(querySource)) {
+ try (XContentParser parser = XContentFactory.xContent(querySource).createParser(shardContext.getXContentRegistry(),
+ querySource)) {
QueryBuilder innerQueryBuilder = shardContext.newParseContext(parser).parseInnerQueryBuilder();
final ParsedQuery parsedQuery = shardContext.toQuery(innerQueryBuilder);
collateMatch = Lucene.exists(searcher, parsedQuery.query());