Merge branch 'master' into enhancement/use_shard_bulk_for_single_ops

This commit is contained in:
Areek Zillur 2016-12-21 00:27:41 -05:00
commit 180ceef134
273 changed files with 3878 additions and 1468 deletions

View File

@ -349,7 +349,6 @@
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]index[/\\]fielddata[/\\]plain[/\\]ParentChildIndexFieldData.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]index[/\\]fielddata[/\\]plain[/\\]SortedNumericDVIndexFieldData.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]index[/\\]fielddata[/\\]plain[/\\]SortedSetDVOrdinalsIndexFieldData.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]index[/\\]get[/\\]GetResult.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]index[/\\]get[/\\]ShardGetService.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]index[/\\]mapper[/\\]DocumentFieldMappers.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]index[/\\]mapper[/\\]DocumentMapper.java" checks="LineLength" />

View File

@ -44,7 +44,7 @@ import java.util.stream.Collectors;
import static java.util.Collections.unmodifiableMap;
import static org.elasticsearch.cluster.metadata.IndexMetaData.INDEX_UUID_NA_VALUE;
import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureFieldName;
import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken;
import static org.elasticsearch.common.xcontent.XContentParserUtils.throwUnknownField;
/**
@ -357,7 +357,8 @@ public class ElasticsearchException extends RuntimeException implements ToXConte
* instances.
*/
public static ElasticsearchException fromXContent(XContentParser parser) throws IOException {
XContentParser.Token token = ensureFieldName(parser.nextToken(), parser::getTokenLocation);
XContentParser.Token token = parser.nextToken();
ensureExpectedToken(XContentParser.Token.FIELD_NAME, token, parser::getTokenLocation);
String type = null, reason = null, stack = null;
ElasticsearchException cause = null;
@ -786,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;

View File

@ -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) {

View File

@ -35,6 +35,8 @@ import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.rest.RestStatus;
import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.Locale;
/**
@ -186,8 +188,9 @@ public abstract class DocWriteResponse extends ReplicationResponse implements Wr
/**
* Gets the location of the written document as a string suitable for a {@code Location} header.
* @param routing any routing used in the request. If null the location doesn't include routing information.
*
*/
public String getLocation(@Nullable String routing) {
public String getLocation(@Nullable String routing) throws URISyntaxException {
// Absolute path for the location of the document. This should be allowed as of HTTP/1.1:
// https://tools.ietf.org/html/rfc7231#section-7.1.2
String index = getIndex();
@ -205,7 +208,9 @@ public abstract class DocWriteResponse extends ReplicationResponse implements Wr
if (routing != null) {
location.append(routingStart).append(routing);
}
return location.toString();
URI uri = new URI(location.toString());
return uri.toASCIIString();
}
@Override

View File

@ -141,8 +141,8 @@ public class TransportClusterHealthAction extends TransportMasterNodeReadAction<
}
assert waitFor >= 0;
final ClusterStateObserver observer = new ClusterStateObserver(clusterService, logger, threadPool.getThreadContext());
final ClusterState state = observer.observedState();
final ClusterState state = clusterService.state();
final ClusterStateObserver observer = new ClusterStateObserver(state, clusterService, null, logger, threadPool.getThreadContext());
if (request.timeout().millis() == 0) {
listener.onResponse(getResponse(request, state, waitFor, request.timeout().millis() == 0));
return;
@ -163,8 +163,7 @@ public class TransportClusterHealthAction extends TransportMasterNodeReadAction<
@Override
public void onTimeout(TimeValue timeout) {
final ClusterState clusterState = clusterService.state();
final ClusterHealthResponse response = getResponse(request, clusterState, concreteWaitFor, true);
final ClusterHealthResponse response = getResponse(request, observer.setAndGetObservedState(), concreteWaitFor, true);
listener.onResponse(response);
}
};

View File

@ -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));
}

View File

@ -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());
}

View File

@ -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) {

View File

@ -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) {

View File

@ -240,7 +240,7 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
@Override
protected void doRun() throws Exception {
final ClusterState clusterState = observer.observedState();
final ClusterState clusterState = observer.setAndGetObservedState();
if (handleBlockExceptions(clusterState)) {
return;
}

View File

@ -109,7 +109,6 @@ public class TransportShardBulkAction extends TransportWriteAction<BulkShardRequ
BulkShardRequest request, IndexShard primary) throws Exception {
final IndexMetaData metaData = primary.indexSettings().getIndexMetaData();
logger.info("TTRACE: in bulk shardOperationPrimary for [{}]", request);
long[] preVersions = new long[request.items().length];
VersionType[] preVersionTypes = new VersionType[request.items().length];
Translog.Location location = null;
@ -365,7 +364,6 @@ public class TransportShardBulkAction extends TransportWriteAction<BulkShardRequ
@Override
public WriteReplicaResult<BulkShardRequest> shardOperationOnReplica(BulkShardRequest request, IndexShard replica) throws Exception {
logger.info("TTRACE: in bulk shardOperationReplica for [{}]", request);
Translog.Location location = null;
for (int i = 0; i < request.items().length; i++) {
BulkItemRequest item = request.items()[i];

View File

@ -27,12 +27,14 @@ 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.XContentParser;
import org.elasticsearch.index.get.GetField;
import org.elasticsearch.index.get.GetResult;
import java.io.IOException;
import java.util.Iterator;
import java.util.Map;
import java.util.Objects;
/**
* The response of a get action.
@ -42,7 +44,7 @@ import java.util.Map;
*/
public class GetResponse extends ActionResponse implements Iterable<GetField>, ToXContent {
private GetResult getResult;
GetResult getResult;
GetResponse() {
}
@ -156,6 +158,11 @@ public class GetResponse extends ActionResponse implements Iterable<GetField>, T
return getResult.toXContent(builder, params);
}
public static GetResponse fromXContent(XContentParser parser) throws IOException {
GetResult getResult = GetResult.fromXContent(parser);
return new GetResponse(getResult);
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
@ -168,6 +175,23 @@ public class GetResponse extends ActionResponse implements Iterable<GetField>, T
getResult.writeTo(out);
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
GetResponse getResponse = (GetResponse) o;
return Objects.equals(getResult, getResponse.getResult);
}
@Override
public int hashCode() {
return Objects.hash(getResult);
}
@Override
public String toString() {
return Strings.toString(this, true);

View File

@ -140,9 +140,7 @@ public class MultiGetResponse extends ActionResponse implements Iterable<MultiGe
builder.endObject();
} else {
GetResponse getResponse = response.getResponse();
builder.startObject();
getResponse.toXContent(builder, params);
builder.endObject();
}
}
builder.endArray();
@ -154,9 +152,6 @@ public class MultiGetResponse extends ActionResponse implements Iterable<MultiGe
static final String _INDEX = "_index";
static final String _TYPE = "_type";
static final String _ID = "_id";
static final String ERROR = "error";
static final String ROOT_CAUSE = "root_cause";
}
@Override

View File

@ -69,9 +69,10 @@ public class ActiveShardsObserver extends AbstractComponent {
return;
}
final ClusterStateObserver observer = new ClusterStateObserver(clusterService, logger, threadPool.getThreadContext());
if (activeShardCount.enoughShardsActive(observer.observedState(), indexName)) {
onResult.accept(true);
final ClusterState state = clusterService.state();
final ClusterStateObserver observer = new ClusterStateObserver(state, clusterService, null, logger, threadPool.getThreadContext());
if (activeShardCount.enoughShardsActive(state, indexName)) {
onResult.accept(true);
} else {
final Predicate<ClusterState> shardsAllocatedPredicate = newState -> activeShardCount.enoughShardsActive(newState, indexName);

View File

@ -121,12 +121,12 @@ public abstract class TransportMasterNodeAction<Request extends MasterNodeReques
}
public void start() {
this.observer = new ClusterStateObserver(clusterService, request.masterNodeTimeout(), logger, threadPool.getThreadContext());
doStart();
ClusterState state = clusterService.state();
this.observer = new ClusterStateObserver(state, clusterService, request.masterNodeTimeout(), logger, threadPool.getThreadContext());
doStart(state);
}
protected void doStart() {
final ClusterState clusterState = observer.observedState();
protected void doStart(ClusterState clusterState) {
final Predicate<ClusterState> masterChangePredicate = MasterNodeChangePredicate.build(clusterState);
final DiscoveryNodes nodes = clusterState.nodes();
if (nodes.isLocalNodeElectedMaster() || localExecute(request)) {
@ -197,7 +197,7 @@ public abstract class TransportMasterNodeAction<Request extends MasterNodeReques
new ClusterStateObserver.Listener() {
@Override
public void onNewClusterState(ClusterState state) {
doStart();
doStart(state);
}
@Override

View File

@ -23,19 +23,26 @@ import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.rest.RestStatus;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Objects;
import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken;
import static org.elasticsearch.common.xcontent.XContentParserUtils.throwUnknownField;
/**
* Base class for write action responses.
*/
@ -186,6 +193,48 @@ public class ReplicationResponse extends ActionResponse {
return builder;
}
public static ShardInfo fromXContent(XContentParser parser) throws IOException {
XContentParser.Token token = parser.nextToken();
ensureExpectedToken(XContentParser.Token.FIELD_NAME, token, parser::getTokenLocation);
String currentFieldName = parser.currentName();
if (_SHARDS.equals(currentFieldName) == false) {
throwUnknownField(currentFieldName, parser.getTokenLocation());
}
token = parser.nextToken();
ensureExpectedToken(XContentParser.Token.START_OBJECT, token, parser::getTokenLocation);
int total = 0, successful = 0;
List<Failure> failuresList = null;
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (token.isValue()) {
if (TOTAL.equals(currentFieldName)) {
total = parser.intValue();
} else if (SUCCESSFUL.equals(currentFieldName)) {
successful = parser.intValue();
} else if (FAILED.equals(currentFieldName) == false) {
throwUnknownField(currentFieldName, parser.getTokenLocation());
}
} else if (token == XContentParser.Token.START_ARRAY) {
if (FAILURES.equals(currentFieldName)) {
failuresList = new ArrayList<>();
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
failuresList.add(Failure.fromXContent(parser));
}
} else {
throwUnknownField(currentFieldName, parser.getTokenLocation());
}
}
}
Failure[] failures = EMPTY;
if (failuresList != null) {
failures = failuresList.toArray(new Failure[failuresList.size()]);
}
return new ShardInfo(total, successful, failures);
}
@Override
public String toString() {
return "ShardInfo{" +
@ -338,6 +387,45 @@ public class ReplicationResponse extends ActionResponse {
builder.endObject();
return builder;
}
public static Failure fromXContent(XContentParser parser) throws IOException {
XContentParser.Token token = parser.currentToken();
ensureExpectedToken(XContentParser.Token.START_OBJECT, token, parser::getTokenLocation);
String shardIndex = null, nodeId = null;
int shardId = -1;
boolean primary = false;
RestStatus status = null;
ElasticsearchException reason = null;
String currentFieldName = null;
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (token.isValue()) {
if (_INDEX.equals(currentFieldName)) {
shardIndex = parser.text();
} else if (_SHARD.equals(currentFieldName)) {
shardId = parser.intValue();
} else if (_NODE.equals(currentFieldName)) {
nodeId = parser.text();
} else if (STATUS.equals(currentFieldName)) {
status = RestStatus.valueOf(parser.text());
} else if (PRIMARY.equals(currentFieldName)) {
primary = parser.booleanValue();
} else {
throwUnknownField(currentFieldName, parser.getTokenLocation());
}
} else if (token == XContentParser.Token.START_OBJECT) {
if (REASON.equals(currentFieldName)) {
reason = ElasticsearchException.fromXContent(parser);
} else {
throwUnknownField(currentFieldName, parser.getTokenLocation());
}
}
}
return new Failure(new ShardId(shardIndex, IndexMetaData.INDEX_UUID_NA_VALUE, shardId), nodeId, reason, status, primary);
}
}
}
}

View File

@ -638,7 +638,7 @@ public abstract class TransportReplicationAction<
@Override
protected void doRun() {
setPhase(task, "routing");
final ClusterState state = observer.observedState();
final ClusterState state = observer.setAndGetObservedState();
if (handleBlockExceptions(state)) {
return;
}

View File

@ -111,7 +111,6 @@ public abstract class TransportInstanceSingleOperationAction<Request extends Ins
private final Request request;
private volatile ClusterStateObserver observer;
private ShardIterator shardIt;
private DiscoveryNodes nodes;
AsyncSingleAction(Request request, ActionListener<Response> listener) {
this.request = request;
@ -119,13 +118,12 @@ public abstract class TransportInstanceSingleOperationAction<Request extends Ins
}
public void start() {
this.observer = new ClusterStateObserver(clusterService, request.timeout(), logger, threadPool.getThreadContext());
doStart();
ClusterState state = clusterService.state();
this.observer = new ClusterStateObserver(state, clusterService, request.timeout(), logger, threadPool.getThreadContext());
doStart(state);
}
protected void doStart() {
final ClusterState clusterState = observer.observedState();
nodes = clusterState.nodes();
protected void doStart(ClusterState clusterState) {
try {
ClusterBlockException blockException = checkGlobalBlock(clusterState);
if (blockException != null) {
@ -171,7 +169,7 @@ public abstract class TransportInstanceSingleOperationAction<Request extends Ins
}
request.shardId = shardIt.shardId();
DiscoveryNode node = nodes.get(shard.currentNodeId());
DiscoveryNode node = clusterState.nodes().get(shard.currentNodeId());
transportService.sendRequest(node, shardActionName, request, transportOptions(), new TransportResponseHandler<Response>() {
@Override
@ -221,18 +219,18 @@ public abstract class TransportInstanceSingleOperationAction<Request extends Ins
observer.waitForNextChange(new ClusterStateObserver.Listener() {
@Override
public void onNewClusterState(ClusterState state) {
doStart();
doStart(state);
}
@Override
public void onClusterServiceClose() {
listener.onFailure(new NodeClosedException(nodes.getLocalNode()));
listener.onFailure(new NodeClosedException(clusterService.localNode()));
}
@Override
public void onTimeout(TimeValue timeout) {
// just to be on the safe side, see if we can start it now?
doStart();
doStart(observer.setAndGetObservedState());
}
}, request.timeout());
}

View File

@ -58,9 +58,7 @@ import java.net.URISyntaxException;
import java.nio.file.Path;
import java.security.NoSuchAlgorithmException;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Objects;
import java.util.Collections;
import java.util.concurrent.CountDownLatch;
/**
@ -226,13 +224,14 @@ final class Bootstrap {
};
}
private static Environment initialEnvironment(boolean foreground, Path pidFile, Map<String, String> esSettings) {
private static Environment initialEnvironment(boolean foreground, Path pidFile, Settings initialSettings) {
Terminal terminal = foreground ? Terminal.DEFAULT : null;
Settings.Builder builder = Settings.builder();
if (pidFile != null) {
builder.put(Environment.PIDFILE_SETTING.getKey(), pidFile);
}
return InternalSettingsPreparer.prepareEnvironment(builder.build(), terminal, esSettings);
builder.put(initialSettings);
return InternalSettingsPreparer.prepareEnvironment(builder.build(), terminal, Collections.emptyMap());
}
private void start() throws NodeValidationException {
@ -262,7 +261,7 @@ final class Bootstrap {
final boolean foreground,
final Path pidFile,
final boolean quiet,
final Map<String, String> esSettings) throws BootstrapException, NodeValidationException, UserException {
final Settings initialSettings) throws BootstrapException, NodeValidationException, UserException {
// Set the system property before anything has a chance to trigger its use
initLoggerPrefix();
@ -272,7 +271,7 @@ final class Bootstrap {
INSTANCE = new Bootstrap();
Environment environment = initialEnvironment(foreground, pidFile, esSettings);
Environment environment = initialEnvironment(foreground, pidFile, initialSettings);
try {
LogConfigurator.configure(environment);
} catch (IOException e) {

View File

@ -20,14 +20,14 @@
package org.elasticsearch.bootstrap;
import java.nio.file.Path;
import java.util.Map;
/**
* Wrapper exception for checked exceptions thrown during the bootstrap process. Methods invoked
* during bootstrap should explicitly declare the checked exceptions that they can throw, rather
* than declaring the top-level checked exception {@link Exception}. This exception exists to wrap
* these checked exceptions so that {@link Bootstrap#init(boolean, Path, boolean, Map)} does not have to
* declare all of these checked exceptions.
* these checked exceptions so that
* {@link Bootstrap#init(boolean, Path, boolean, org.elasticsearch.common.settings.Settings)}
* does not have to declare all of these checked exceptions.
*/
class BootstrapException extends Exception {

View File

@ -25,9 +25,11 @@ import joptsimple.OptionSpecBuilder;
import joptsimple.util.PathConverter;
import org.elasticsearch.Build;
import org.elasticsearch.cli.ExitCodes;
import org.elasticsearch.cli.SettingCommand;
import org.elasticsearch.cli.EnvironmentAwareCommand;
import org.elasticsearch.cli.Terminal;
import org.elasticsearch.cli.UserException;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.monitor.jvm.JvmInfo;
import org.elasticsearch.node.NodeValidationException;
@ -40,7 +42,7 @@ import java.util.Map;
/**
* This class starts elasticsearch.
*/
class Elasticsearch extends SettingCommand {
class Elasticsearch extends EnvironmentAwareCommand {
private final OptionSpecBuilder versionOption;
private final OptionSpecBuilder daemonizeOption;
@ -90,7 +92,7 @@ class Elasticsearch extends SettingCommand {
}
@Override
protected void execute(Terminal terminal, OptionSet options, Map<String, String> settings) throws UserException {
protected void execute(Terminal terminal, OptionSet options, Environment env) throws UserException {
if (options.nonOptionArguments().isEmpty() == false) {
throw new UserException(ExitCodes.USAGE, "Positional arguments not allowed, found " + options.nonOptionArguments());
}
@ -109,16 +111,16 @@ class Elasticsearch extends SettingCommand {
final boolean quiet = options.has(quietOption);
try {
init(daemonize, pidFile, quiet, settings);
init(daemonize, pidFile, quiet, env.settings());
} catch (NodeValidationException e) {
throw new UserException(ExitCodes.CONFIG, e.getMessage());
}
}
void init(final boolean daemonize, final Path pidFile, final boolean quiet, final Map<String, String> esSettings)
void init(final boolean daemonize, final Path pidFile, final boolean quiet, Settings initialSettings)
throws NodeValidationException, UserException {
try {
Bootstrap.init(!daemonize, pidFile, quiet, esSettings);
Bootstrap.init(!daemonize, pidFile, quiet, initialSettings);
} catch (BootstrapException | RuntimeException e) {
// format exceptions to the console in a special way
// to avoid 2MB stacktraces from guice, etc.

View File

@ -22,16 +22,20 @@ package org.elasticsearch.cli;
import joptsimple.OptionSet;
import joptsimple.OptionSpec;
import joptsimple.util.KeyValuePair;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.node.internal.InternalSettingsPreparer;
import java.util.HashMap;
import java.util.Locale;
import java.util.Map;
public abstract class SettingCommand extends Command {
/** A cli command which requires an {@link org.elasticsearch.env.Environment} to use current paths and settings. */
public abstract class EnvironmentAwareCommand extends Command {
private final OptionSpec<KeyValuePair> settingOption;
public SettingCommand(String description) {
public EnvironmentAwareCommand(String description) {
super(description);
this.settingOption = parser.accepts("E", "Configure a setting").withRequiredArg().ofType(KeyValuePair.class);
}
@ -51,9 +55,15 @@ public abstract class SettingCommand extends Command {
putSystemPropertyIfSettingIsMissing(settings, "path.home", "es.path.home");
putSystemPropertyIfSettingIsMissing(settings, "path.logs", "es.path.logs");
execute(terminal, options, settings);
execute(terminal, options, createEnv(terminal, settings));
}
/** Create an {@link Environment} for the command to use. Overrideable for tests. */
protected Environment createEnv(Terminal terminal, Map<String, String> settings) {
return InternalSettingsPreparer.prepareEnvironment(Settings.EMPTY, terminal, settings);
}
/** Ensure the given setting exists, reading it from system properties if not already set. */
protected static void putSystemPropertyIfSettingIsMissing(final Map<String, String> settings, final String setting, final String key) {
final String value = System.getProperty(key);
if (value != null) {
@ -72,6 +82,7 @@ public abstract class SettingCommand extends Command {
}
}
protected abstract void execute(Terminal terminal, OptionSet options, Map<String, String> settings) throws Exception;
/** Execute the command with the initialized {@link Environment}. */
protected abstract void execute(Terminal terminal, OptionSet options, Environment env) throws Exception;
}

View File

@ -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);

View File

@ -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);

View File

@ -26,6 +26,7 @@ import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.ThreadContext;
import java.util.Objects;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Predicate;
@ -45,7 +46,7 @@ public class ClusterStateObserver {
volatile TimeValue timeOutValue;
final AtomicReference<ClusterState> lastObservedState;
final AtomicReference<StoredState> lastObservedState;
final TimeoutClusterStateListener clusterStateListener = new ObserverClusterStateListener();
// observingContext is not null when waiting on cluster state changes
final AtomicReference<ObservingContext> observingContext = new AtomicReference<>(null);
@ -73,7 +74,7 @@ public class ClusterStateObserver {
public ClusterStateObserver(ClusterState initialState, ClusterService clusterService, @Nullable TimeValue timeout, Logger logger,
ThreadContext contextHolder) {
this.clusterService = clusterService;
this.lastObservedState = new AtomicReference<>(initialState);
this.lastObservedState = new AtomicReference<>(new StoredState(initialState));
this.timeOutValue = timeout;
if (timeOutValue != null) {
this.startTimeNS = System.nanoTime();
@ -82,11 +83,14 @@ public class ClusterStateObserver {
this.contextHolder = contextHolder;
}
/** last cluster state and status observed by this observer. Note that this may not be the current one */
public ClusterState observedState() {
ClusterState state = lastObservedState.get();
assert state != null;
return state;
/** sets the last observed state to the currently applied cluster state and returns it */
public ClusterState setAndGetObservedState() {
if (observingContext.get() != null) {
throw new ElasticsearchException("cannot set current cluster state while waiting for a cluster state change");
}
ClusterState clusterState = clusterService.state();
lastObservedState.set(new StoredState(clusterState));
return clusterState;
}
/** indicates whether this observer has timedout */
@ -130,7 +134,7 @@ public class ClusterStateObserver {
logger.trace("observer timed out. notifying listener. timeout setting [{}], time since start [{}]", timeOutValue, new TimeValue(timeSinceStartMS));
// update to latest, in case people want to retry
timedOut = true;
lastObservedState.set(clusterService.state());
lastObservedState.set(new StoredState(clusterService.state()));
listener.onTimeout(timeOutValue);
return;
}
@ -146,11 +150,10 @@ public class ClusterStateObserver {
// sample a new state
ClusterState newState = clusterService.state();
ClusterState lastState = lastObservedState.get();
if (newState != lastState && statePredicate.test(newState)) {
if (lastObservedState.get().sameState(newState) == false && statePredicate.test(newState)) {
// good enough, let's go.
logger.trace("observer: sampled state accepted by predicate ({})", newState);
lastObservedState.set(newState);
lastObservedState.set(new StoredState(newState));
listener.onNewClusterState(newState);
} else {
logger.trace("observer: sampled state rejected by predicate ({}). adding listener to ClusterService", newState);
@ -177,7 +180,7 @@ public class ClusterStateObserver {
if (observingContext.compareAndSet(context, null)) {
clusterService.removeTimeoutListener(this);
logger.trace("observer: accepting cluster state change ({})", state);
lastObservedState.set(state);
lastObservedState.set(new StoredState(state));
context.listener.onNewClusterState(state);
} else {
logger.trace("observer: predicate approved change but observing context has changed - ignoring (new cluster state version [{}])", state.version());
@ -195,13 +198,12 @@ public class ClusterStateObserver {
return;
}
ClusterState newState = clusterService.state();
ClusterState lastState = lastObservedState.get();
if (newState != lastState && context.statePredicate.test(newState)) {
if (lastObservedState.get().sameState(newState) == false && context.statePredicate.test(newState)) {
// double check we're still listening
if (observingContext.compareAndSet(context, null)) {
logger.trace("observer: post adding listener: accepting current cluster state ({})", newState);
clusterService.removeTimeoutListener(this);
lastObservedState.set(newState);
lastObservedState.set(new StoredState(newState));
context.listener.onNewClusterState(newState);
} else {
logger.trace("observer: postAdded - predicate approved state but observing context has changed - ignoring ({})", newState);
@ -230,13 +232,30 @@ public class ClusterStateObserver {
long timeSinceStartMS = TimeValue.nsecToMSec(System.nanoTime() - startTimeNS);
logger.trace("observer: timeout notification from cluster service. timeout setting [{}], time since start [{}]", timeOutValue, new TimeValue(timeSinceStartMS));
// update to latest, in case people want to retry
lastObservedState.set(clusterService.state());
lastObservedState.set(new StoredState(clusterService.state()));
timedOut = true;
context.listener.onTimeout(timeOutValue);
}
}
}
/**
* The observer considers two cluster states to be the same if they have the same version and master node id (i.e. null or set)
*/
private static class StoredState {
private final String masterNodeId;
private final long version;
public StoredState(ClusterState clusterState) {
this.masterNodeId = clusterState.nodes().getMasterNodeId();
this.version = clusterState.version();
}
public boolean sameState(ClusterState clusterState) {
return version == clusterState.version() && Objects.equals(masterNodeId, clusterState.nodes().getMasterNodeId());
}
}
public interface Listener {
/** called when a new state is observed */

View File

@ -69,6 +69,7 @@ import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Set;
import java.util.function.Predicate;
public class ShardStateAction extends AbstractComponent {
@ -91,11 +92,13 @@ public class ShardStateAction extends AbstractComponent {
transportService.registerRequestHandler(SHARD_FAILED_ACTION_NAME, ShardEntry::new, ThreadPool.Names.SAME, new ShardFailedTransportHandler(clusterService, new ShardFailedClusterStateTaskExecutor(allocationService, routingService, logger), logger));
}
private void sendShardAction(final String actionName, final ClusterStateObserver observer, final ShardEntry shardEntry, final Listener listener) {
DiscoveryNode masterNode = observer.observedState().nodes().getMasterNode();
private void sendShardAction(final String actionName, final ClusterState currentState, final ShardEntry shardEntry, final Listener listener) {
ClusterStateObserver observer = new ClusterStateObserver(currentState, clusterService, null, logger, threadPool.getThreadContext());
DiscoveryNode masterNode = currentState.nodes().getMasterNode();
Predicate<ClusterState> changePredicate = MasterNodeChangePredicate.build(currentState);
if (masterNode == null) {
logger.warn("{} no master known for action [{}] for shard entry [{}]", shardEntry.shardId, actionName, shardEntry);
waitForNewMasterAndRetry(actionName, observer, shardEntry, listener);
waitForNewMasterAndRetry(actionName, observer, shardEntry, listener, changePredicate);
} else {
logger.debug("{} sending [{}] to [{}] for shard entry [{}]", shardEntry.shardId, actionName, masterNode.getId(), shardEntry);
transportService.sendRequest(masterNode,
@ -108,7 +111,7 @@ public class ShardStateAction extends AbstractComponent {
@Override
public void handleException(TransportException exp) {
if (isMasterChannelException(exp)) {
waitForNewMasterAndRetry(actionName, observer, shardEntry, listener);
waitForNewMasterAndRetry(actionName, observer, shardEntry, listener, changePredicate);
} else {
logger.warn((Supplier<?>) () -> new ParameterizedMessage("{} unexpected failure while sending request [{}] to [{}] for shard entry [{}]", shardEntry.shardId, actionName, masterNode, shardEntry), exp);
listener.onFailure(exp instanceof RemoteTransportException ? (Exception) (exp.getCause() instanceof Exception ? exp.getCause() : new ElasticsearchException(exp.getCause())) : exp);
@ -162,20 +165,19 @@ public class ShardStateAction extends AbstractComponent {
private void shardFailed(final ShardId shardId, String allocationId, long primaryTerm, final String message,
@Nullable final Exception failure, Listener listener, ClusterState currentState) {
ClusterStateObserver observer = new ClusterStateObserver(currentState, clusterService, null, logger, threadPool.getThreadContext());
ShardEntry shardEntry = new ShardEntry(shardId, allocationId, primaryTerm, message, failure);
sendShardAction(SHARD_FAILED_ACTION_NAME, observer, shardEntry, listener);
sendShardAction(SHARD_FAILED_ACTION_NAME, currentState, shardEntry, listener);
}
// visible for testing
protected void waitForNewMasterAndRetry(String actionName, ClusterStateObserver observer, ShardEntry shardEntry, Listener listener) {
protected void waitForNewMasterAndRetry(String actionName, ClusterStateObserver observer, ShardEntry shardEntry, Listener listener, Predicate<ClusterState> changePredicate) {
observer.waitForNextChange(new ClusterStateObserver.Listener() {
@Override
public void onNewClusterState(ClusterState state) {
if (logger.isTraceEnabled()) {
logger.trace("new cluster state [{}] after waiting for master election to fail shard entry [{}]", state, shardEntry);
}
sendShardAction(actionName, observer, shardEntry, listener);
sendShardAction(actionName, state, shardEntry, listener);
}
@Override
@ -189,7 +191,7 @@ public class ShardStateAction extends AbstractComponent {
// we wait indefinitely for a new master
assert false;
}
}, MasterNodeChangePredicate.build(observer.observedState()));
}, changePredicate);
}
private static class ShardFailedTransportHandler implements TransportRequestHandler<ShardEntry> {
@ -354,9 +356,8 @@ public class ShardStateAction extends AbstractComponent {
shardStarted(shardRouting, message, listener, clusterService.state());
}
public void shardStarted(final ShardRouting shardRouting, final String message, Listener listener, ClusterState currentState) {
ClusterStateObserver observer = new ClusterStateObserver(currentState, clusterService, null, logger, threadPool.getThreadContext());
ShardEntry shardEntry = new ShardEntry(shardRouting.shardId(), shardRouting.allocationId().getId(), 0L, message, null);
sendShardAction(SHARD_STARTED_ACTION_NAME, observer, shardEntry, listener);
sendShardAction(SHARD_STARTED_ACTION_NAME, currentState, shardEntry, listener);
}
private static class ShardStartedTransportHandler implements TransportRequestHandler<ShardEntry> {

View File

@ -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) {

View File

@ -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);

View File

@ -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();

View File

@ -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();
}

View File

@ -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());
}

View File

@ -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);
}
}

View File

@ -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);

View File

@ -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);

View File

@ -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) {

View File

@ -125,7 +125,7 @@ public class AwarenessAllocationDecider extends AllocationDecider {
private Decision underCapacity(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation, boolean moveToNode) {
if (awarenessAttributes.length == 0) {
return allocation.decision(Decision.YES, NAME,
"allocation awareness is not enabled, set [%s] to enable it",
"allocation awareness is not enabled, set cluster setting [%s] to enable it",
CLUSTER_ROUTING_ALLOCATION_AWARENESS_ATTRIBUTE_SETTING.getKey());
}
@ -135,7 +135,7 @@ public class AwarenessAllocationDecider extends AllocationDecider {
// the node the shard exists on must be associated with an awareness attribute
if (!node.node().getAttributes().containsKey(awarenessAttribute)) {
return allocation.decision(Decision.NO, NAME,
"node does not contain the awareness attribute [%s]; required attributes [%s=%s]",
"node does not contain the awareness attribute [%s]; required attributes cluster setting [%s=%s]",
awarenessAttribute, CLUSTER_ROUTING_ALLOCATION_AWARENESS_ATTRIBUTE_SETTING.getKey(),
allocation.debugDecision() ? Strings.arrayToCommaDelimitedString(awarenessAttributes) : null);
}

View File

@ -120,13 +120,13 @@ public class ClusterRebalanceAllocationDecider extends AllocationDecider {
// check if there are unassigned primaries.
if ( allocation.routingNodes().hasUnassignedPrimaries() ) {
return allocation.decision(Decision.NO, NAME,
"the cluster has unassigned primary shards and [%s] is set to [%s]",
"the cluster has unassigned primary shards and cluster setting [%s] is set to [%s]",
CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, type);
}
// check if there are initializing primaries that don't have a relocatingNodeId entry.
if ( allocation.routingNodes().hasInactivePrimaries() ) {
return allocation.decision(Decision.NO, NAME,
"the cluster has inactive primary shards and [%s] is set to [%s]",
"the cluster has inactive primary shards and cluster setting [%s] is set to [%s]",
CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, type);
}
@ -136,14 +136,14 @@ public class ClusterRebalanceAllocationDecider extends AllocationDecider {
// check if there are unassigned shards.
if (allocation.routingNodes().hasUnassignedShards() ) {
return allocation.decision(Decision.NO, NAME,
"the cluster has unassigned shards and [%s] is set to [%s]",
"the cluster has unassigned shards and cluster setting [%s] is set to [%s]",
CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, type);
}
// in case all indices are assigned, are there initializing shards which
// are not relocating?
if ( allocation.routingNodes().hasInactiveShards() ) {
return allocation.decision(Decision.NO, NAME,
"the cluster has inactive shards and [%s] is set to [%s]",
"the cluster has inactive shards and cluster setting [%s] is set to [%s]",
CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, type);
}
}

View File

@ -67,7 +67,7 @@ public class ConcurrentRebalanceAllocationDecider extends AllocationDecider {
int relocatingShards = allocation.routingNodes().getRelocatingShardCount();
if (relocatingShards >= clusterConcurrentRebalance) {
return allocation.decision(Decision.THROTTLE, NAME,
"reached the limit of concurrently rebalancing shards [%d], [%s=%d]",
"reached the limit of concurrently rebalancing shards [%d], cluster setting [%s=%d]",
relocatingShards,
CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE_SETTING.getKey(),
clusterConcurrentRebalance);

View File

@ -138,7 +138,8 @@ public class DiskThresholdDecider extends AllocationDecider {
diskThresholdSettings.getFreeBytesThresholdLow(), freeBytes, node.nodeId());
}
return allocation.decision(Decision.NO, NAME,
"the node is above the low watermark [%s=%s], having less than the minimum required [%s] free space, actual free: [%s]",
"the node is above the low watermark cluster setting [%s=%s], having less than the minimum required [%s] free " +
"space, actual free: [%s]",
CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(),
diskThresholdSettings.getLowWatermarkRaw(),
diskThresholdSettings.getFreeBytesThresholdLow(), new ByteSizeValue(freeBytes));
@ -162,8 +163,8 @@ public class DiskThresholdDecider extends AllocationDecider {
diskThresholdSettings.getFreeBytesThresholdHigh(), freeBytes, node.nodeId());
}
return allocation.decision(Decision.NO, NAME,
"the node is above the high watermark [%s=%s], having less than the minimum required [%s] free space, " +
"actual free: [%s]",
"the node is above the high watermark cluster setting [%s=%s], having less than the minimum required [%s] free " +
"space, actual free: [%s]",
CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(),
diskThresholdSettings.getHighWatermarkRaw(),
diskThresholdSettings.getFreeBytesThresholdHigh(), new ByteSizeValue(freeBytes));
@ -180,8 +181,8 @@ public class DiskThresholdDecider extends AllocationDecider {
Strings.format1Decimals(usedDiskPercentage, "%"), node.nodeId());
}
return allocation.decision(Decision.NO, NAME,
"the node is above the low watermark [%s=%s], using more disk space than the maximum allowed [%s%%], " +
"actual free: [%s%%]",
"the node is above the low watermark cluster setting [%s=%s], using more disk space than the maximum allowed " +
"[%s%%], actual free: [%s%%]",
CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(),
diskThresholdSettings.getLowWatermarkRaw(), usedDiskThresholdLow, freeDiskPercentage);
} else if (freeDiskPercentage > diskThresholdSettings.getFreeDiskThresholdHigh()) {
@ -206,8 +207,8 @@ public class DiskThresholdDecider extends AllocationDecider {
Strings.format1Decimals(freeDiskPercentage, "%"), node.nodeId());
}
return allocation.decision(Decision.NO, NAME,
"the node is above the high watermark [%s=%s], using more disk space than the maximum allowed [%s%%], " +
"actual free: [%s%%]",
"the node is above the high watermark cluster setting [%s=%s], using more disk space than the maximum allowed " +
"[%s%%], actual free: [%s%%]",
CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(),
diskThresholdSettings.getHighWatermarkRaw(), usedDiskThresholdHigh, freeDiskPercentage);
}
@ -222,7 +223,7 @@ public class DiskThresholdDecider extends AllocationDecider {
"{} free bytes threshold ({} bytes free), preventing allocation",
node.nodeId(), diskThresholdSettings.getFreeBytesThresholdHigh(), freeBytesAfterShard);
return allocation.decision(Decision.NO, NAME,
"allocating the shard to this node will bring the node above the high watermark [%s=%s] " +
"allocating the shard to this node will bring the node above the high watermark cluster setting [%s=%s] " +
"and cause it to have less than the minimum required [%s] of free space (free bytes after shard added: [%s])",
CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(),
diskThresholdSettings.getHighWatermarkRaw(),
@ -234,7 +235,7 @@ public class DiskThresholdDecider extends AllocationDecider {
node.nodeId(), Strings.format1Decimals(diskThresholdSettings.getFreeDiskThresholdHigh(), "%"),
Strings.format1Decimals(freeSpaceAfterShard, "%"));
return allocation.decision(Decision.NO, NAME,
"allocating the shard to this node will bring the node above the high watermark [%s=%s] " +
"allocating the shard to this node will bring the node above the high watermark cluster setting [%s=%s] " +
"and cause it to use more disk space than the maximum allowed [%s%%] (free space after shard added: [%s%%])",
CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(),
diskThresholdSettings.getHighWatermarkRaw(), usedDiskThresholdHigh, freeSpaceAfterShard);
@ -279,7 +280,7 @@ public class DiskThresholdDecider extends AllocationDecider {
diskThresholdSettings.getFreeBytesThresholdHigh(), freeBytes, node.nodeId());
}
return allocation.decision(Decision.NO, NAME,
"the shard cannot remain on this node because it is above the high watermark [%s=%s] " +
"the shard cannot remain on this node because it is above the high watermark cluster setting [%s=%s] " +
"and there is less than the required [%s] free space on node, actual free: [%s]",
CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(),
diskThresholdSettings.getHighWatermarkRaw(),
@ -291,7 +292,7 @@ public class DiskThresholdDecider extends AllocationDecider {
diskThresholdSettings.getFreeDiskThresholdHigh(), freeDiskPercentage, node.nodeId());
}
return allocation.decision(Decision.NO, NAME,
"the shard cannot remain on this node because it is above the high watermark [%s=%s] " +
"the shard cannot remain on this node because it is above the high watermark cluster setting [%s=%s] " +
"and there is less than the required [%s%%] free disk on node, actual free: [%s%%]",
CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(),
diskThresholdSettings.getHighWatermarkRaw(),

View File

@ -189,10 +189,12 @@ public class EnableAllocationDecider extends AllocationDecider {
}
private static String setting(Rebalance rebalance, boolean usedIndexSetting) {
StringBuilder buf = new StringBuilder("[");
StringBuilder buf = new StringBuilder();
if (usedIndexSetting) {
buf.append("index setting [");
buf.append(INDEX_ROUTING_REBALANCE_ENABLE_SETTING.getKey());
} else {
buf.append("cluster setting [");
buf.append(CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey());
}
buf.append("=").append(rebalance.toString().toLowerCase(Locale.ROOT)).append("]");

View File

@ -141,19 +141,19 @@ public class FilterAllocationDecider extends AllocationDecider {
private Decision shouldIndexFilter(IndexMetaData indexMd, RoutingNode node, RoutingAllocation allocation) {
if (indexMd.requireFilters() != null) {
if (!indexMd.requireFilters().match(node.node())) {
return allocation.decision(Decision.NO, NAME, "node does not match [%s] filters [%s]",
return allocation.decision(Decision.NO, NAME, "node does not match index setting [%s] filters [%s]",
IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_PREFIX, indexMd.requireFilters());
}
}
if (indexMd.includeFilters() != null) {
if (!indexMd.includeFilters().match(node.node())) {
return allocation.decision(Decision.NO, NAME, "node does not match [%s] filters [%s]",
return allocation.decision(Decision.NO, NAME, "node does not match index setting [%s] filters [%s]",
IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_PREFIX, indexMd.includeFilters());
}
}
if (indexMd.excludeFilters() != null) {
if (indexMd.excludeFilters().match(node.node())) {
return allocation.decision(Decision.NO, NAME, "node matches [%s] filters [%s]",
return allocation.decision(Decision.NO, NAME, "node matches index setting [%s] filters [%s]",
IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey(), indexMd.excludeFilters());
}
}
@ -163,19 +163,19 @@ public class FilterAllocationDecider extends AllocationDecider {
private Decision shouldClusterFilter(RoutingNode node, RoutingAllocation allocation) {
if (clusterRequireFilters != null) {
if (!clusterRequireFilters.match(node.node())) {
return allocation.decision(Decision.NO, NAME, "node does not match [%s] filters [%s]",
return allocation.decision(Decision.NO, NAME, "node does not match cluster setting [%s] filters [%s]",
CLUSTER_ROUTING_REQUIRE_GROUP_PREFIX, clusterRequireFilters);
}
}
if (clusterIncludeFilters != null) {
if (!clusterIncludeFilters.match(node.node())) {
return allocation.decision(Decision.NO, NAME, "node does not [%s] filters [%s]",
return allocation.decision(Decision.NO, NAME, "node does not cluster setting [%s] filters [%s]",
CLUSTER_ROUTING_INCLUDE_GROUP_PREFIX, clusterIncludeFilters);
}
}
if (clusterExcludeFilters != null) {
if (clusterExcludeFilters.match(node.node())) {
return allocation.decision(Decision.NO, NAME, "node matches [%s] filters [%s]",
return allocation.decision(Decision.NO, NAME, "node matches cluster setting [%s] filters [%s]",
CLUSTER_ROUTING_EXCLUDE_GROUP_PREFIX, clusterExcludeFilters);
}
}

View File

@ -88,7 +88,8 @@ public class SameShardAllocationDecider extends AllocationDecider {
String host = checkNodeOnSameHostAddress ? node.node().getHostAddress() : node.node().getHostName();
return allocation.decision(Decision.NO, NAME,
"the shard cannot be allocated on host %s [%s], where it already exists on node [%s]; " +
"set [%s] to false to allow multiple nodes on the same host to hold the same shard copies",
"set cluster setting [%s] to false to allow multiple nodes on the same host to hold the same " +
"shard copies",
hostType, host, node.nodeId(), CLUSTER_ROUTING_ALLOCATION_SAME_HOST_SETTING.getKey());
}
}

View File

@ -122,12 +122,12 @@ public class ShardsLimitAllocationDecider extends AllocationDecider {
if (clusterShardLimit > 0 && decider.test(nodeShardCount, clusterShardLimit)) {
return allocation.decision(Decision.NO, NAME,
"too many shards [%d] allocated to this node, [%s=%d]",
"too many shards [%d] allocated to this node, cluster setting [%s=%d]",
nodeShardCount, CLUSTER_TOTAL_SHARDS_PER_NODE_SETTING.getKey(), clusterShardLimit);
}
if (indexShardLimit > 0 && decider.test(indexShardCount, indexShardLimit)) {
return allocation.decision(Decision.NO, NAME,
"too many shards [%d] allocated to this node for index [%s], [%s=%d]",
"too many shards [%d] allocated to this node for index [%s], index setting [%s=%d]",
indexShardCount, shardRouting.getIndexName(), INDEX_TOTAL_SHARDS_PER_NODE_SETTING.getKey(), indexShardLimit);
}
return allocation.decision(Decision.YES, NAME,
@ -157,7 +157,7 @@ public class ShardsLimitAllocationDecider extends AllocationDecider {
}
if (clusterShardLimit >= 0 && nodeShardCount >= clusterShardLimit) {
return allocation.decision(Decision.NO, NAME,
"too many shards [%d] allocated to this node, [%s=%d]",
"too many shards [%d] allocated to this node, cluster setting [%s=%d]",
nodeShardCount, CLUSTER_TOTAL_SHARDS_PER_NODE_SETTING.getKey(), clusterShardLimit);
}
return allocation.decision(Decision.YES, NAME,

View File

@ -126,7 +126,8 @@ public class ThrottlingAllocationDecider extends AllocationDecider {
}
if (primariesInRecovery >= primariesInitialRecoveries) {
// TODO: Should index creation not be throttled for primary shards?
return allocation.decision(THROTTLE, NAME, "reached the limit of ongoing initial primary recoveries [%d], [%s=%d]",
return allocation.decision(THROTTLE, NAME,
"reached the limit of ongoing initial primary recoveries [%d], cluster setting [%s=%d]",
primariesInRecovery, CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES_SETTING.getKey(),
primariesInitialRecoveries);
} else {
@ -140,7 +141,7 @@ public class ThrottlingAllocationDecider extends AllocationDecider {
int currentInRecoveries = allocation.routingNodes().getIncomingRecoveries(node.nodeId());
if (currentInRecoveries >= concurrentIncomingRecoveries) {
return allocation.decision(THROTTLE, NAME,
"reached the limit of incoming shard recoveries [%d], [%s=%d] (can also be set via [%s])",
"reached the limit of incoming shard recoveries [%d], cluster setting [%s=%d] (can also be set via [%s])",
currentInRecoveries, CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_INCOMING_RECOVERIES_SETTING.getKey(),
concurrentIncomingRecoveries,
CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES_SETTING.getKey());
@ -154,7 +155,7 @@ public class ThrottlingAllocationDecider extends AllocationDecider {
if (primaryNodeOutRecoveries >= concurrentOutgoingRecoveries) {
return allocation.decision(THROTTLE, NAME,
"reached the limit of outgoing shard recoveries [%d] on the node [%s] which holds the primary, " +
"[%s=%d] (can also be set via [%s])",
"cluster setting [%s=%d] (can also be set via [%s])",
primaryNodeOutRecoveries, node.nodeId(),
CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_OUTGOING_RECOVERIES_SETTING.getKey(),
concurrentOutgoingRecoveries,

View File

@ -101,14 +101,10 @@ public class ParseField {
/**
* @param fieldName
* the field name to match against this {@link ParseField}
* @param strict
* if true an exception will be thrown if a deprecated field name
* is given. If false the deprecated name will be matched but a
* message will also be logged to the {@link DeprecationLogger}
* @return true if <code>fieldName</code> matches any of the acceptable
* names for this {@link ParseField}.
*/
boolean match(String fieldName, boolean strict) {
public boolean match(String fieldName) {
Objects.requireNonNull(fieldName, "fieldName cannot be null");
// if this parse field has not been completely deprecated then try to
// match the preferred name
@ -128,11 +124,7 @@ public class ParseField {
// message to indicate what should be used instead
msg = "Deprecated field [" + fieldName + "] used, replaced by [" + allReplacedWith + "]";
}
if (strict) {
throw new IllegalArgumentException(msg);
} else {
DEPRECATION_LOGGER.deprecated(msg);
}
DEPRECATION_LOGGER.deprecated(msg);
return true;
}
}

View File

@ -22,38 +22,29 @@ package org.elasticsearch.common;
import org.elasticsearch.common.settings.Settings;
/**
* Matcher to use in combination with {@link ParseField} while parsing requests. Matches a {@link ParseField}
* against a field name and throw deprecation exception depending on the current value of the {@link #PARSE_STRICT} setting.
* Matcher to use in combination with {@link ParseField} while parsing requests.
*
* @deprecated This class used to be useful to parse in strict mode and emit errors rather than deprecation warnings. Now that we return
* warnings as response headers all the time, it is no longer useful and will soon be removed. The removal is in progress and there is
* already no strict mode in fact. Use {@link ParseField} directly.
*/
@Deprecated
public class ParseFieldMatcher {
public static final String PARSE_STRICT = "index.query.parse.strict";
public static final ParseFieldMatcher EMPTY = new ParseFieldMatcher(false);
public static final ParseFieldMatcher STRICT = new ParseFieldMatcher(true);
private final boolean strict;
public static final ParseFieldMatcher EMPTY = new ParseFieldMatcher(Settings.EMPTY);
public static final ParseFieldMatcher STRICT = new ParseFieldMatcher(Settings.EMPTY);
public ParseFieldMatcher(Settings settings) {
this(settings.getAsBoolean(PARSE_STRICT, false));
}
public ParseFieldMatcher(boolean strict) {
this.strict = strict;
}
/** Should deprecated settings be rejected? */
public boolean isStrict() {
return strict;
//we don't do anything with the settings argument, this whole class will be soon removed
}
/**
* Matches a {@link ParseField} against a field name, and throws deprecation exception depending on the current
* value of the {@link #PARSE_STRICT} setting.
* Matches a {@link ParseField} against a field name,
* @param fieldName the field name found in the request while parsing
* @param parseField the parse field that we are looking for
* @throws IllegalArgumentException whenever we are in strict mode and the request contained a deprecated field
* @return true whenever the parse field that we are looking for was found, false otherwise
*/
public boolean match(String fieldName, ParseField parseField) {
return parseField.match(fieldName, strict);
return parseField.match(fieldName);
}
}

View File

@ -21,7 +21,6 @@ package org.elasticsearch.common.logging;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.elasticsearch.common.SuppressLoggerChecks;
import org.elasticsearch.common.util.concurrent.ThreadContext;
@ -42,7 +41,7 @@ public class DeprecationLogger {
*
* https://tools.ietf.org/html/rfc7234#section-5.5
*/
public static final String DEPRECATION_HEADER = "Warning";
public static final String WARNING_HEADER = "Warning";
/**
* This is set once by the {@code Node} constructor, but it uses {@link CopyOnWriteArraySet} to ensure that tests can run in parallel.
@ -128,7 +127,7 @@ public class DeprecationLogger {
while (iterator.hasNext()) {
try {
iterator.next().addResponseHeader(DEPRECATION_HEADER, formattedMessage);
iterator.next().addResponseHeader(WARNING_HEADER, formattedMessage);
} catch (IllegalStateException e) {
// ignored; it should be removed shortly
}

View File

@ -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());
}

View File

@ -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);

View File

@ -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);
}
}

View File

@ -100,7 +100,10 @@ public final class Text implements Comparable<Text> {
@Override
public boolean equals(Object obj) {
if (obj == null) {
if (this == obj) {
return true;
}
if (obj == null || getClass() != obj.getClass()) {
return false;
}
return bytes().equals(((Text) obj).bytes());

View File

@ -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;
}
}
}

View File

@ -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;
}

View File

@ -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);
}
}
@ -349,4 +375,22 @@ public class XContentHelper {
builder.rawField(field, source);
}
}
/**
* Returns the bytes that represent the XContent output of the provided {@link ToXContent} object, using the provided
* {@link XContentType}. Wraps the output into a new anonymous object depending on the value of the wrapInObject argument.
*/
public static BytesReference toXContent(ToXContent toXContent, XContentType xContentType, boolean wrapInObject) throws IOException {
try (XContentBuilder builder = XContentBuilder.builder(xContentType.xContent())) {
if (wrapInObject) {
builder.startObject();
}
toXContent.toXContent(builder, ToXContent.EMPTY_PARAMS);
if (wrapInObject) {
builder.endObject();
}
return builder.bytes();
}
}
}

View File

@ -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();
}

View File

@ -22,7 +22,6 @@ package org.elasticsearch.common.xcontent;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import java.io.IOException;
import java.util.Locale;
import java.util.function.Supplier;
@ -35,34 +34,6 @@ public final class XContentParserUtils {
private XContentParserUtils() {
}
/**
* Makes sure that current token is of type {@link XContentParser.Token#FIELD_NAME}
*
* @return the token
* @throws ParsingException if the token is not of type {@link XContentParser.Token#FIELD_NAME}
*/
public static Token ensureFieldName(Token token, Supplier<XContentLocation> location) throws IOException {
return ensureType(Token.FIELD_NAME, token, location);
}
/**
* Makes sure that current token is of type {@link XContentParser.Token#FIELD_NAME} and the the field name is equal to the provided one
*
* @return the token
* @throws ParsingException if the token is not of type {@link XContentParser.Token#FIELD_NAME} or is not equal to the given
* field name
*/
public static Token ensureFieldName(XContentParser parser, Token token, String fieldName) throws IOException {
Token t = ensureType(Token.FIELD_NAME, token, parser::getTokenLocation);
String current = parser.currentName() != null ? parser.currentName() : "<null>";
if (current.equals(fieldName) == false) {
String message = "Failed to parse object: expecting field with name [%s] but found [%s]";
throw new ParsingException(parser.getTokenLocation(), String.format(Locale.ROOT, message, fieldName, current));
}
return t;
}
/**
* @throws ParsingException with a "unknown field found" reason
*/
@ -72,16 +43,14 @@ public final class XContentParserUtils {
}
/**
* Makes sure that current token is of the expected type
* Makes sure that provided token is of the expected type
*
* @return the token
* @throws ParsingException if the token is not equal to the expected type
*/
private static Token ensureType(Token expected, Token current, Supplier<XContentLocation> location) {
if (current != expected) {
public static void ensureExpectedToken(Token expected, Token actual, Supplier<XContentLocation> location) {
if (actual != expected) {
String message = "Failed to parse object: expecting token of type [%s] but found [%s]";
throw new ParsingException(location.get(), String.format(Locale.ROOT, message, expected, current));
throw new ParsingException(location.get(), String.format(Locale.ROOT, message, expected, actual));
}
return current;
}
}

View File

@ -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));
}
}

View File

@ -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

View File

@ -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));
}
}

View File

@ -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);
}
}

View File

@ -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;
}

View File

@ -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));
}
}

View File

@ -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

View File

@ -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()
@ -356,6 +361,16 @@ public abstract class AbstractXContentParser implements XContentParser {
return null;
}
@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();
}

View File

@ -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));
}
}

View File

@ -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

View File

@ -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) {

View File

@ -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"); });
}

View File

@ -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);
}

View File

@ -19,17 +19,25 @@
package org.elasticsearch.index.get;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.mapper.MapperService;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Objects;
public class GetField implements Streamable, Iterable<Object> {
import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken;
public class GetField implements Streamable, ToXContent, Iterable<Object> {
private String name;
private List<Object> values;
@ -38,8 +46,8 @@ public class GetField implements Streamable, Iterable<Object> {
}
public GetField(String name, List<Object> values) {
this.name = name;
this.values = values;
this.name = Objects.requireNonNull(name, "name must not be null");
this.values = Objects.requireNonNull(values, "values must not be null");
}
public String getName() {
@ -90,4 +98,69 @@ public class GetField implements Streamable, Iterable<Object> {
out.writeGenericValue(obj);
}
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startArray(name);
for (Object value : values) {
//this call doesn't really need to support writing any kind of object.
//Stored fields values are converted using MappedFieldType#valueForDisplay.
//As a result they can either be Strings, Numbers, Booleans, or BytesReference, that's all.
builder.value(value);
}
builder.endArray();
return builder;
}
public static GetField fromXContent(XContentParser parser) throws IOException {
ensureExpectedToken(XContentParser.Token.FIELD_NAME, parser.currentToken(), parser::getTokenLocation);
String fieldName = parser.currentName();
XContentParser.Token token = parser.nextToken();
ensureExpectedToken(XContentParser.Token.START_ARRAY, token, parser::getTokenLocation);
List<Object> values = new ArrayList<>();
while((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
Object value;
if (token == XContentParser.Token.VALUE_STRING) {
//binary values will be parsed back and returned as base64 strings when reading from json and yaml
value = parser.text();
} else if (token == XContentParser.Token.VALUE_NUMBER) {
value = parser.numberValue();
} else if (token == XContentParser.Token.VALUE_BOOLEAN) {
value = parser.booleanValue();
} else if (token == XContentParser.Token.VALUE_EMBEDDED_OBJECT) {
//binary values will be parsed back and returned as BytesArray when reading from cbor and smile
value = new BytesArray(parser.binaryValue());
} else {
throw new ParsingException(parser.getTokenLocation(), "Failed to parse object: unsupported token found [" + token + "]");
}
values.add(value);
}
return new GetField(fieldName, values);
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
GetField objects = (GetField) o;
return Objects.equals(name, objects.name) &&
Objects.equals(values, objects.values);
}
@Override
public int hashCode() {
return Objects.hash(name, values);
}
@Override
public String toString() {
return "GetField{" +
"name='" + name + '\'' +
", values=" + values +
'}';
}
}

View File

@ -28,6 +28,7 @@ import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.mapper.SourceFieldMapper;
import org.elasticsearch.search.lookup.SourceLookup;
@ -38,12 +39,22 @@ import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import static java.util.Collections.emptyMap;
import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken;
import static org.elasticsearch.common.xcontent.XContentParserUtils.throwUnknownField;
import static org.elasticsearch.index.get.GetField.readGetField;
public class GetResult implements Streamable, Iterable<GetField>, ToXContent {
private static final String _INDEX = "_index";
private static final String _TYPE = "_type";
private static final String _ID = "_id";
private static final String _VERSION = "_version";
private static final String FOUND = "found";
private static final String FIELDS = "fields";
private String index;
private String type;
private String id;
@ -57,7 +68,8 @@ public class GetResult implements Streamable, Iterable<GetField>, ToXContent {
GetResult() {
}
public GetResult(String index, String type, String id, long version, boolean exists, BytesReference source, Map<String, GetField> fields) {
public GetResult(String index, String type, String id, long version, boolean exists, BytesReference source,
Map<String, GetField> fields) {
this.index = index;
this.type = type;
this.id = id;
@ -196,15 +208,6 @@ public class GetResult implements Streamable, Iterable<GetField>, ToXContent {
return fields.values().iterator();
}
static final class Fields {
static final String _INDEX = "_index";
static final String _TYPE = "_type";
static final String _ID = "_id";
static final String _VERSION = "_version";
static final String FOUND = "found";
static final String FIELDS = "fields";
}
public XContentBuilder toXContentEmbedded(XContentBuilder builder, Params params) throws IOException {
List<GetField> metaFields = new ArrayList<>();
List<GetField> otherFields = new ArrayList<>();
@ -225,20 +228,16 @@ public class GetResult implements Streamable, Iterable<GetField>, ToXContent {
builder.field(field.getName(), field.getValue());
}
builder.field(Fields.FOUND, exists);
builder.field(FOUND, exists);
if (source != null) {
XContentHelper.writeRawField(SourceFieldMapper.NAME, source, builder, params);
}
if (!otherFields.isEmpty()) {
builder.startObject(Fields.FIELDS);
builder.startObject(FIELDS);
for (GetField field : otherFields) {
builder.startArray(field.getName());
for (Object value : field.getValues()) {
builder.value(value);
}
builder.endArray();
field.toXContent(builder, params);
}
builder.endObject();
}
@ -247,23 +246,69 @@ public class GetResult implements Streamable, Iterable<GetField>, ToXContent {
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
if (!isExists()) {
builder.field(Fields._INDEX, index);
builder.field(Fields._TYPE, type);
builder.field(Fields._ID, id);
builder.field(Fields.FOUND, false);
} else {
builder.field(Fields._INDEX, index);
builder.field(Fields._TYPE, type);
builder.field(Fields._ID, id);
builder.startObject();
builder.field(_INDEX, index);
builder.field(_TYPE, type);
builder.field(_ID, id);
if (isExists()) {
if (version != -1) {
builder.field(Fields._VERSION, version);
builder.field(_VERSION, version);
}
toXContentEmbedded(builder, params);
} else {
builder.field(FOUND, false);
}
builder.endObject();
return builder;
}
public static GetResult fromXContent(XContentParser parser) throws IOException {
XContentParser.Token token = parser.nextToken();
ensureExpectedToken(XContentParser.Token.START_OBJECT, token, parser::getTokenLocation);
String currentFieldName = null;
String index = null, type = null, id = null;
long version = -1;
boolean found = false;
BytesReference source = null;
Map<String, GetField> fields = new HashMap<>();
while((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (token.isValue()) {
if (_INDEX.equals(currentFieldName)) {
index = parser.text();
} else if (_TYPE.equals(currentFieldName)) {
type = parser.text();
} else if (_ID.equals(currentFieldName)) {
id = parser.text();
} else if (_VERSION.equals(currentFieldName)) {
version = parser.longValue();
} else if (FOUND.equals(currentFieldName)) {
found = parser.booleanValue();
} else {
fields.put(currentFieldName, new GetField(currentFieldName, Collections.singletonList(parser.objectText())));
}
} else if (token == XContentParser.Token.START_OBJECT) {
if (SourceFieldMapper.NAME.equals(currentFieldName)) {
try (XContentBuilder builder = XContentBuilder.builder(parser.contentType().xContent())) {
//the original document gets slightly modified: whitespaces or pretty printing are not preserved,
//it all depends on the current builder settings
builder.copyCurrentStructure(parser);
source = builder.bytes();
}
} else if (FIELDS.equals(currentFieldName)) {
while(parser.nextToken() != XContentParser.Token.END_OBJECT) {
GetField getField = GetField.fromXContent(parser);
fields.put(getField.getName(), getField);
}
} else {
throwUnknownField(currentFieldName, parser.getTokenLocation());
}
}
}
return new GetResult(index, type, id, version, found, source, fields);
}
public static GetResult readGetResult(StreamInput in) throws IOException {
GetResult result = new GetResult();
result.readFrom(in);
@ -314,5 +359,28 @@ public class GetResult implements Streamable, Iterable<GetField>, ToXContent {
}
}
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
GetResult getResult = (GetResult) o;
return version == getResult.version &&
exists == getResult.exists &&
Objects.equals(index, getResult.index) &&
Objects.equals(type, getResult.type) &&
Objects.equals(id, getResult.id) &&
Objects.equals(fields, getResult.fields) &&
Objects.equals(sourceAsMap(), getResult.sourceAsMap());
}
@Override
public int hashCode() {
return Objects.hash(index, type, id, version, exists, fields, sourceAsMap());
}
}

View File

@ -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;
}
}

View File

@ -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);

View File

@ -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();
}
}

View File

@ -25,7 +25,6 @@ import org.elasticsearch.Version;
import org.elasticsearch.common.joda.FormatDateTimeFormatter;
import org.elasticsearch.common.joda.Joda;
import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.common.logging.ESLoggerFactory;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.xcontent.support.XContentMapValues;
import org.elasticsearch.index.analysis.NamedAnalyzer;
@ -43,7 +42,6 @@ import java.util.Set;
import static org.elasticsearch.common.xcontent.support.XContentMapValues.isArray;
import static org.elasticsearch.common.xcontent.support.XContentMapValues.lenientNodeBooleanValue;
import static org.elasticsearch.common.xcontent.support.XContentMapValues.nodeFloatValue;
import static org.elasticsearch.common.xcontent.support.XContentMapValues.nodeIntegerValue;
import static org.elasticsearch.common.xcontent.support.XContentMapValues.nodeMapValue;
import static org.elasticsearch.common.xcontent.support.XContentMapValues.nodeStringValue;
@ -59,16 +57,11 @@ public class TypeParsers {
private static final Set<String> BOOLEAN_STRINGS = new HashSet<>(Arrays.asList("true", "false"));
public static boolean nodeBooleanValue(String name, Object node, Mapper.TypeParser.ParserContext parserContext) {
// Hook onto ParseFieldMatcher so that parsing becomes strict when setting index.query.parse.strict
if (parserContext.parseFieldMatcher().isStrict()) {
return XContentMapValues.nodeBooleanValue(node);
} else {
// TODO: remove this leniency in 6.0
if (BOOLEAN_STRINGS.contains(node.toString()) == false) {
DEPRECATION_LOGGER.deprecated("Expected a boolean for property [{}] but got [{}]", name, node);
}
return XContentMapValues.lenientNodeBooleanValue(node);
// TODO: remove this leniency in 6.0
if (BOOLEAN_STRINGS.contains(node.toString()) == false) {
DEPRECATION_LOGGER.deprecated("Expected a boolean for property [{}] but got [{}]", name, node);
}
return XContentMapValues.lenientNodeBooleanValue(node);
}
private static void parseAnalyzersAndTermVectors(FieldMapper.Builder builder, String name, Map<String, Object> fieldNode, Mapper.TypeParser.ParserContext parserContext) {
@ -211,10 +204,10 @@ public class TypeParsers {
throw new MapperParsingException("[" + propName + "] must not have a [null] value");
}
if (propName.equals("store")) {
builder.store(parseStore(name, propNode.toString(), parserContext));
builder.store(parseStore(propNode.toString()));
iterator.remove();
} else if (propName.equals("index")) {
builder.index(parseIndex(name, propNode.toString(), parserContext));
builder.index(parseIndex(name, propNode.toString()));
iterator.remove();
} else if (propName.equals(DOC_VALUES)) {
builder.docValues(nodeBooleanValue(DOC_VALUES, propNode, parserContext));
@ -229,7 +222,11 @@ public class TypeParsers {
builder.indexOptions(nodeIndexOptionValue(propNode));
iterator.remove();
} else if (propName.equals("include_in_all")) {
builder.includeInAll(nodeBooleanValue("include_in_all", propNode, parserContext));
if (parserContext.isWithinMultiField()) {
throw new MapperParsingException("include_in_all in multi fields is not allowed. Found the include_in_all in field [" + name + "] which is within a multi field.");
} else {
builder.includeInAll(nodeBooleanValue("include_in_all", propNode, parserContext));
}
iterator.remove();
} else if (propName.equals("similarity")) {
SimilarityProvider similarityProvider = resolveSimilarity(parserContext, name, propNode.toString());
@ -346,7 +343,7 @@ public class TypeParsers {
}
}
public static boolean parseIndex(String fieldName, String index, Mapper.TypeParser.ParserContext parserContext) throws MapperParsingException {
private static boolean parseIndex(String fieldName, String index) throws MapperParsingException {
switch (index) {
case "true":
return true;
@ -355,31 +352,23 @@ public class TypeParsers {
case "not_analyzed":
case "analyzed":
case "no":
if (parserContext.parseFieldMatcher().isStrict() == false) {
DEPRECATION_LOGGER.deprecated("Expected a boolean for property [index] but got [{}]", index);
return "no".equals(index) == false;
} else {
throw new IllegalArgumentException("Can't parse [index] value [" + index + "] for field [" + fieldName + "], expected [true] or [false]");
}
DEPRECATION_LOGGER.deprecated("Expected a boolean for property [index] but got [{}]", index);
return "no".equals(index) == false;
default:
throw new IllegalArgumentException("Can't parse [index] value [" + index + "] for field [" + fieldName + "], expected [true] or [false]");
}
}
public static boolean parseStore(String fieldName, String store, Mapper.TypeParser.ParserContext parserContext) throws MapperParsingException {
if (parserContext.parseFieldMatcher().isStrict()) {
return XContentMapValues.nodeBooleanValue(store);
private static boolean parseStore(String store) throws MapperParsingException {
if (BOOLEAN_STRINGS.contains(store) == false) {
DEPRECATION_LOGGER.deprecated("Expected a boolean for property [store] but got [{}]", store);
}
if ("no".equals(store)) {
return false;
} else if ("yes".equals(store)) {
return true;
} else {
if (BOOLEAN_STRINGS.contains(store) == false) {
DEPRECATION_LOGGER.deprecated("Expected a boolean for property [store] but got [{}]", store);
}
if ("no".equals(store)) {
return false;
} else if ("yes".equals(store)) {
return true;
} else {
return lenientNodeBooleanValue(store);
}
return lenientNodeBooleanValue(store);
}
}

View File

@ -62,9 +62,9 @@ public class GeoBoundingBoxQueryBuilder extends AbstractQueryBuilder<GeoBounding
private static final ParseField TYPE_FIELD = new ParseField("type");
private static final ParseField VALIDATION_METHOD_FIELD = new ParseField("validation_method");
private static final ParseField COERCE_FIELD =new ParseField("coerce", "normalize")
.withAllDeprecated("use field validation_method instead");
.withAllDeprecated("validation_method");
private static final ParseField IGNORE_MALFORMED_FIELD = new ParseField("ignore_malformed")
.withAllDeprecated("use field validation_method instead");
.withAllDeprecated("validation_method");
private static final ParseField FIELD_FIELD = new ParseField("field");
private static final ParseField TOP_FIELD = new ParseField("top");
private static final ParseField BOTTOM_FIELD = new ParseField("bottom");

View File

@ -66,10 +66,8 @@ public class GeoDistanceQueryBuilder extends AbstractQueryBuilder<GeoDistanceQue
public static final boolean DEFAULT_IGNORE_UNMAPPED = false;
private static final ParseField VALIDATION_METHOD_FIELD = new ParseField("validation_method");
private static final ParseField IGNORE_MALFORMED_FIELD = new ParseField("ignore_malformed")
.withAllDeprecated("use validation_method instead");
private static final ParseField COERCE_FIELD = new ParseField("coerce", "normalize")
.withAllDeprecated("use validation_method instead");
private static final ParseField IGNORE_MALFORMED_FIELD = new ParseField("ignore_malformed").withAllDeprecated("validation_method");
private static final ParseField COERCE_FIELD = new ParseField("coerce", "normalize").withAllDeprecated("validation_method");
@Deprecated
private static final ParseField OPTIMIZE_BBOX_FIELD = new ParseField("optimize_bbox")
.withAllDeprecated("no replacement: `optimize_bbox` is no longer supported due to recent improvements");

View File

@ -49,10 +49,8 @@ public class GeoPolygonQueryBuilder extends AbstractQueryBuilder<GeoPolygonQuery
*/
public static final boolean DEFAULT_IGNORE_UNMAPPED = false;
private static final ParseField COERCE_FIELD = new ParseField("coerce", "normalize")
.withAllDeprecated("use validation_method instead");
private static final ParseField IGNORE_MALFORMED_FIELD = new ParseField("ignore_malformed")
.withAllDeprecated("use validation_method instead");
private static final ParseField COERCE_FIELD = new ParseField("coerce", "normalize").withAllDeprecated("validation_method");
private static final ParseField IGNORE_MALFORMED_FIELD = new ParseField("ignore_malformed").withAllDeprecated("validation_method");
private static final ParseField VALIDATION_METHOD = new ParseField("validation_method");
private static final ParseField POINTS_FIELD = new ParseField("points");
private static final ParseField IGNORE_UNMAPPED_FIELD = new ParseField("ignore_unmapped");

View File

@ -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) {

View File

@ -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) {

View File

@ -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;
@ -92,6 +95,13 @@ public class QueryRewriteContext implements ParseFieldMatcherSupplier {
return this.indexSettings.getParseFieldMatcher();
}
/**
* 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.

View File

@ -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();
}

View File

@ -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();

View File

@ -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;

View File

@ -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)) {

View File

@ -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) {

View File

@ -34,10 +34,11 @@ import org.apache.lucene.store.OutputStreamDataOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.cli.SettingCommand;
import org.elasticsearch.cli.EnvironmentAwareCommand;
import org.elasticsearch.cli.Terminal;
import org.elasticsearch.common.SuppressForbidden;
import org.elasticsearch.common.io.PathUtils;
import org.elasticsearch.env.Environment;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.index.seqno.SequenceNumbersService;
@ -55,7 +56,7 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
public class TruncateTranslogCommand extends SettingCommand {
public class TruncateTranslogCommand extends EnvironmentAwareCommand {
private final OptionSpec<String> translogFolder;
private final OptionSpec<Void> batchMode;
@ -87,7 +88,7 @@ public class TruncateTranslogCommand extends SettingCommand {
}
@Override
protected void execute(Terminal terminal, OptionSet options, Map<String, String> settings) throws Exception {
protected void execute(Terminal terminal, OptionSet options, Environment env) throws Exception {
boolean batch = options.has(batchMode);
Path translogPath = getTranslogPath(options);

View File

@ -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);
}
}

View File

@ -397,21 +397,21 @@ public class PeerRecoveryTargetService extends AbstractComponent implements Inde
}
private void waitForClusterState(long clusterStateVersion) {
ClusterStateObserver observer = new ClusterStateObserver(clusterService, TimeValue.timeValueMinutes(5), logger,
final ClusterState clusterState = clusterService.state();
ClusterStateObserver observer = new ClusterStateObserver(clusterState, clusterService, TimeValue.timeValueMinutes(5), logger,
threadPool.getThreadContext());
final ClusterState clusterState = observer.observedState();
if (clusterState.getVersion() >= clusterStateVersion) {
logger.trace("node has cluster state with version higher than {} (current: {})", clusterStateVersion,
clusterState.getVersion());
return;
} else {
logger.trace("waiting for cluster state version {} (current: {})", clusterStateVersion, clusterState.getVersion());
final PlainActionFuture<Void> future = new PlainActionFuture<>();
final PlainActionFuture<Long> future = new PlainActionFuture<>();
observer.waitForNextChange(new ClusterStateObserver.Listener() {
@Override
public void onNewClusterState(ClusterState state) {
future.onResponse(null);
future.onResponse(state.getVersion());
}
@Override
@ -425,15 +425,14 @@ public class PeerRecoveryTargetService extends AbstractComponent implements Inde
}
}, newState -> newState.getVersion() >= clusterStateVersion);
try {
future.get();
logger.trace("successfully waited for cluster state with version {} (current: {})", clusterStateVersion,
observer.observedState().getVersion());
long currentVersion = future.get();
logger.trace("successfully waited for cluster state with version {} (current: {})", clusterStateVersion, currentVersion);
} catch (Exception e) {
logger.debug(
(Supplier<?>) () -> new ParameterizedMessage(
"failed waiting for cluster state with version {} (current: {})",
clusterStateVersion,
observer.observedState().getVersion()),
clusterService.state().getVersion()),
e);
throw ExceptionsHelper.convertToRuntime(e);
}

View File

@ -145,8 +145,24 @@ public final class IngestDocument {
* or if the field that is found at the provided path is not of the expected type.
*/
public byte[] getFieldValueAsBytes(String path) {
Object object = getFieldValue(path, Object.class);
if (object instanceof byte[]) {
return getFieldValueAsBytes(path, false);
}
/**
* Returns the value contained in the document for the provided path as a byte array.
* If the path value is a string, a base64 decode operation will happen.
* If the path value is a byte array, it is just returned
* @param path The path within the document in dot-notation
* @param ignoreMissing The flag to determine whether to throw an exception when `path` is not found in the document.
* @return the byte array for the provided path if existing
* @throws IllegalArgumentException if the path is null, empty, invalid, if the field doesn't exist
* or if the field that is found at the provided path is not of the expected type.
*/
public byte[] getFieldValueAsBytes(String path, boolean ignoreMissing) {
Object object = getFieldValue(path, Object.class, ignoreMissing);
if (object == null) {
return null;
} else if (object instanceof byte[]) {
return (byte[]) object;
} else if (object instanceof String) {
return Base64.getDecoder().decode(object.toString());

View File

@ -41,9 +41,11 @@ import org.elasticsearch.common.xcontent.XContentHelper;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
public class PipelineStore extends AbstractComponent implements ClusterStateApplier {
@ -111,17 +113,26 @@ public class PipelineStore extends AbstractComponent implements ClusterStateAppl
return currentState;
}
Map<String, PipelineConfiguration> pipelines = currentIngestMetadata.getPipelines();
if (pipelines.containsKey(request.getId()) == false) {
throw new ResourceNotFoundException("pipeline [{}] is missing", request.getId());
} else {
pipelines = new HashMap<>(pipelines);
pipelines.remove(request.getId());
ClusterState.Builder newState = ClusterState.builder(currentState);
newState.metaData(MetaData.builder(currentState.getMetaData())
.putCustom(IngestMetadata.TYPE, new IngestMetadata(pipelines))
.build());
return newState.build();
Set<String> toRemove = new HashSet<>();
for (String pipelineKey : pipelines.keySet()) {
if (Regex.simpleMatch(request.getId(), pipelineKey)) {
toRemove.add(pipelineKey);
}
}
if (toRemove.isEmpty() && Regex.isMatchAllPattern(request.getId()) == false) {
throw new ResourceNotFoundException("pipeline [{}] is missing", request.getId());
} else if (toRemove.isEmpty()) {
return currentState;
}
final Map<String, PipelineConfiguration> pipelinesCopy = new HashMap<>(pipelines);
for (String key : toRemove) {
pipelinesCopy.remove(key);
}
ClusterState.Builder newState = ClusterState.builder(currentState);
newState.metaData(MetaData.builder(currentState.getMetaData())
.putCustom(IngestMetadata.TYPE, new IngestMetadata(pipelinesCopy))
.build());
return newState.build();
}
/**

View File

@ -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());
{
@ -589,8 +599,9 @@ public class Node implements Closeable {
final TimeValue initialStateTimeout = DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.get(settings);
if (initialStateTimeout.millis() > 0) {
final ThreadPool thread = injector.getInstance(ThreadPool.class);
ClusterStateObserver observer = new ClusterStateObserver(clusterService, null, logger, thread.getThreadContext());
if (observer.observedState().nodes().getMasterNodeId() == null) {
ClusterState clusterState = clusterService.state();
ClusterStateObserver observer = new ClusterStateObserver(clusterState, clusterService, null, logger, thread.getThreadContext());
if (clusterState.nodes().getMasterNodeId() == null) {
logger.debug("waiting to join the cluster. timeout [{}]", initialStateTimeout);
final CountDownLatch latch = new CountDownLatch(1);
observer.waitForNextChange(new ClusterStateObserver.Listener() {

View File

@ -27,7 +27,7 @@ import org.apache.lucene.util.IOUtils;
import org.elasticsearch.Version;
import org.elasticsearch.bootstrap.JarHell;
import org.elasticsearch.cli.ExitCodes;
import org.elasticsearch.cli.SettingCommand;
import org.elasticsearch.cli.EnvironmentAwareCommand;
import org.elasticsearch.cli.Terminal;
import org.elasticsearch.cli.UserException;
import org.elasticsearch.common.collect.Tuple;
@ -103,7 +103,7 @@ import static org.elasticsearch.cli.Terminal.Verbosity.VERBOSE;
* elasticsearch config directory, using the name of the plugin. If any files to be installed
* already exist, they will be skipped.
*/
class InstallPluginCommand extends SettingCommand {
class InstallPluginCommand extends EnvironmentAwareCommand {
private static final String PROPERTY_STAGING_ID = "es.plugins.staging";
@ -189,18 +189,17 @@ class InstallPluginCommand extends SettingCommand {
}
@Override
protected void execute(Terminal terminal, OptionSet options, Map<String, String> settings) throws Exception {
protected void execute(Terminal terminal, OptionSet options, Environment env) throws Exception {
String pluginId = arguments.value(options);
boolean isBatch = options.has(batchOption) || System.console() == null;
execute(terminal, pluginId, isBatch, settings);
execute(terminal, pluginId, isBatch, env);
}
// pkg private for testing
void execute(Terminal terminal, String pluginId, boolean isBatch, Map<String, String> settings) throws Exception {
void execute(Terminal terminal, String pluginId, boolean isBatch, Environment env) throws Exception {
if (pluginId == null) {
throw new UserException(ExitCodes.USAGE, "plugin id is required");
}
final Environment env = InternalSettingsPreparer.prepareEnvironment(Settings.EMPTY, terminal, settings);
// TODO: remove this leniency!! is it needed anymore?
if (Files.exists(env.pluginsFile()) == false) {
terminal.println("Plugins directory [" + env.pluginsFile() + "] does not exist. Creating...");

View File

@ -20,7 +20,7 @@
package org.elasticsearch.plugins;
import joptsimple.OptionSet;
import org.elasticsearch.cli.SettingCommand;
import org.elasticsearch.cli.EnvironmentAwareCommand;
import org.elasticsearch.cli.Terminal;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
@ -38,15 +38,14 @@ import java.util.Map;
/**
* A command for the plugin cli to list plugins installed in elasticsearch.
*/
class ListPluginsCommand extends SettingCommand {
class ListPluginsCommand extends EnvironmentAwareCommand {
ListPluginsCommand() {
super("Lists installed elasticsearch plugins");
}
@Override
protected void execute(Terminal terminal, OptionSet options, Map<String, String> settings) throws Exception {
final Environment env = InternalSettingsPreparer.prepareEnvironment(Settings.EMPTY, terminal, settings);
protected void execute(Terminal terminal, OptionSet options, Environment env) throws Exception {
if (Files.exists(env.pluginsFile()) == false) {
throw new IOException("Plugins directory missing: " + env.pluginsFile());
}

View File

@ -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();
}
}

View File

@ -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();
}
@ -126,6 +127,14 @@ public abstract class Plugin implements Closeable {
return Collections.emptyList();
}
/**
* 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.

View File

@ -19,32 +19,28 @@
package org.elasticsearch.plugins;
import joptsimple.OptionSet;
import joptsimple.OptionSpec;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.cli.ExitCodes;
import org.elasticsearch.cli.SettingCommand;
import org.elasticsearch.cli.Terminal;
import org.elasticsearch.cli.UserException;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.node.internal.InternalSettingsPreparer;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.StandardCopyOption;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import joptsimple.OptionSet;
import joptsimple.OptionSpec;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.cli.EnvironmentAwareCommand;
import org.elasticsearch.cli.ExitCodes;
import org.elasticsearch.cli.Terminal;
import org.elasticsearch.cli.UserException;
import org.elasticsearch.common.Strings;
import org.elasticsearch.env.Environment;
import static org.elasticsearch.cli.Terminal.Verbosity.VERBOSE;
/**
* A command for the plugin cli to remove a plugin from elasticsearch.
*/
class RemovePluginCommand extends SettingCommand {
class RemovePluginCommand extends EnvironmentAwareCommand {
private final OptionSpec<String> arguments;
@ -54,15 +50,13 @@ class RemovePluginCommand extends SettingCommand {
}
@Override
protected void execute(Terminal terminal, OptionSet options, Map<String, String> settings) throws Exception {
protected void execute(Terminal terminal, OptionSet options, Environment env) throws Exception {
String arg = arguments.value(options);
execute(terminal, arg, settings);
execute(terminal, arg, env);
}
// pkg private for testing
void execute(Terminal terminal, String pluginName, Map<String, String> settings) throws Exception {
final Environment env = InternalSettingsPreparer.prepareEnvironment(Settings.EMPTY, terminal, settings);
void execute(Terminal terminal, String pluginName, Environment env) throws Exception {
terminal.println("-> Removing " + Strings.coalesceToEmpty(pluginName) + "...");
final Path pluginDir = env.pluginsFile().resolve(pluginName);

View File

@ -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;
}
}

View File

@ -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);
}
}

View File

@ -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;
}
@ -228,6 +232,13 @@ public abstract class RestRequest implements ToXContent.Params {
return 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
@ -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 {

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