Merge branch 'master' into enhancement/use_shard_bulk_for_single_ops

This commit is contained in:
Areek Zillur 2016-12-19 10:07:02 -05:00
commit eb5cc1e241
179 changed files with 2916 additions and 1777 deletions

5
.gitignore vendored
View File

@ -38,11 +38,14 @@ dependency-reduced-pom.xml
# osx stuff
.DS_Store
# default folders in which the create_bwc_index.py expects to find old es versions in
/backwards
/dev-tools/backwards
# needed in case docs build is run...maybe we can configure doc build to generate files under build?
html_docs
# random old stuff that we should look at the necessity of...
/tmp/
backwards/
eclipse-build

View File

@ -24,7 +24,7 @@ buildscript {
}
}
dependencies {
classpath 'com.github.jengelman.gradle.plugins:shadow:1.2.3'
classpath 'com.github.jengelman.gradle.plugins:shadow:1.2.4'
}
}
@ -44,9 +44,8 @@ task test(type: Test, overwrite: true)
dependencies {
compile("org.elasticsearch:elasticsearch:${version}") {
// JMH ships with the conflicting version 4.6 (JMH will not update this dependency as it is Java 6 compatible and joptsimple is one
// of the most recent compatible version). This prevents us from using jopt-simple in benchmarks (which should be ok) but allows us
// to invoke the JMH uberjar as usual.
// JMH ships with the conflicting version 4.6. This prevents us from using jopt-simple in benchmarks (which should be ok) but allows
// us to invoke the JMH uberjar as usual.
exclude group: 'net.sf.jopt-simple', module: 'jopt-simple'
}
compile "org.openjdk.jmh:jmh-core:$versions.jmh"

View File

@ -268,6 +268,7 @@ class ClusterFormationTasks {
static Task configureWriteConfigTask(String name, Project project, Task setup, NodeInfo node, NodeInfo seedNode) {
Map esConfig = [
'cluster.name' : node.clusterName,
'node.name' : "node-" + node.nodeNum,
'pidfile' : node.pidFile,
'path.repo' : "${node.sharedDir}/repo",
'path.shared_data' : "${node.sharedDir}/",

View File

@ -216,7 +216,6 @@
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]bootstrap[/\\]JNANatives.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]bootstrap[/\\]JVMCheck.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]bootstrap[/\\]JarHell.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]bootstrap[/\\]Seccomp.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]bootstrap[/\\]Security.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]client[/\\]ElasticsearchClient.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]client[/\\]FilterClient.java" checks="LineLength" />

View File

@ -21,4 +21,4 @@ commonscodec = 1.10
hamcrest = 1.3
securemock = 1.2
# benchmark dependencies
jmh = 1.15
jmh = 1.17.3

View File

@ -310,7 +310,6 @@ import org.elasticsearch.rest.action.search.RestExplainAction;
import org.elasticsearch.rest.action.search.RestMultiSearchAction;
import org.elasticsearch.rest.action.search.RestSearchAction;
import org.elasticsearch.rest.action.search.RestSearchScrollAction;
import org.elasticsearch.rest.action.search.RestSuggestAction;
import org.elasticsearch.threadpool.ThreadPool;
import static java.util.Collections.unmodifiableList;
@ -550,7 +549,6 @@ public class ActionModule extends AbstractModule {
registerRestHandler(handlers, RestMultiGetAction.class);
registerRestHandler(handlers, RestDeleteAction.class);
registerRestHandler(handlers, org.elasticsearch.rest.action.document.RestCountAction.class);
registerRestHandler(handlers, RestSuggestAction.class);
registerRestHandler(handlers, RestTermVectorsAction.class);
registerRestHandler(handlers, RestMultiTermVectorsAction.class);
registerRestHandler(handlers, RestBulkAction.class);

View File

@ -18,6 +18,7 @@
*/
package org.elasticsearch.action;
import org.elasticsearch.Version;
import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.action.support.WriteResponse;
@ -214,7 +215,11 @@ public abstract class DocWriteResponse extends ReplicationResponse implements Wr
type = in.readString();
id = in.readString();
version = in.readZLong();
seqNo = in.readZLong();
if (in.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) {
seqNo = in.readZLong();
} else {
seqNo = SequenceNumbersService.UNASSIGNED_SEQ_NO;
}
forcedRefresh = in.readBoolean();
result = Result.readFrom(in);
}
@ -226,7 +231,9 @@ public abstract class DocWriteResponse extends ReplicationResponse implements Wr
out.writeString(type);
out.writeString(id);
out.writeZLong(version);
out.writeZLong(seqNo);
if (out.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) {
out.writeZLong(seqNo);
}
out.writeBoolean(forcedRefresh);
result.writeTo(out);
}

View File

@ -58,6 +58,6 @@ public class ShardFlushRequest extends ReplicationRequest<ShardFlushRequest> {
@Override
public String toString() {
return "flush {" + super.toString() + "}";
return "flush {" + shardId + "}";
}
}

View File

@ -64,16 +64,6 @@ public class TransportShardFlushAction extends TransportReplicationAction<ShardF
return new ReplicaResult();
}
@Override
protected ClusterBlockLevel globalBlockLevel() {
return ClusterBlockLevel.METADATA_WRITE;
}
@Override
protected ClusterBlockLevel indexBlockLevel() {
return ClusterBlockLevel.METADATA_WRITE;
}
@Override
protected boolean shouldExecuteReplication(Settings settings) {
return true;

View File

@ -67,16 +67,6 @@ public class TransportShardRefreshAction
return new ReplicaResult();
}
@Override
protected ClusterBlockLevel globalBlockLevel() {
return ClusterBlockLevel.METADATA_WRITE;
}
@Override
protected ClusterBlockLevel indexBlockLevel() {
return ClusterBlockLevel.METADATA_WRITE;
}
@Override
protected boolean shouldExecuteReplication(Settings settings) {
return true;

View File

@ -46,6 +46,9 @@ import org.elasticsearch.indices.IndicesQueryCache;
import org.elasticsearch.search.suggest.completion.CompletionStats;
import java.io.IOException;
import java.util.Arrays;
import java.util.Objects;
import java.util.stream.Stream;
public class CommonStats implements Writeable, ToXContent {
@ -225,45 +228,19 @@ public class CommonStats implements Writeable, ToXContent {
}
public CommonStats(StreamInput in) throws IOException {
if (in.readBoolean()) {
docs = DocsStats.readDocStats(in);
}
if (in.readBoolean()) {
store = StoreStats.readStoreStats(in);
}
if (in.readBoolean()) {
indexing = IndexingStats.readIndexingStats(in);
}
if (in.readBoolean()) {
get = GetStats.readGetStats(in);
}
if (in.readBoolean()) {
search = SearchStats.readSearchStats(in);
}
if (in.readBoolean()) {
merge = MergeStats.readMergeStats(in);
}
if (in.readBoolean()) {
refresh = RefreshStats.readRefreshStats(in);
}
if (in.readBoolean()) {
flush = FlushStats.readFlushStats(in);
}
if (in.readBoolean()) {
warmer = WarmerStats.readWarmerStats(in);
}
if (in.readBoolean()) {
queryCache = QueryCacheStats.readQueryCacheStats(in);
}
if (in.readBoolean()) {
fieldData = FieldDataStats.readFieldDataStats(in);
}
if (in.readBoolean()) {
completion = CompletionStats.readCompletionStats(in);
}
if (in.readBoolean()) {
segments = SegmentsStats.readSegmentsStats(in);
}
docs = in.readOptionalStreamable(DocsStats::new);
store = in.readOptionalStreamable(StoreStats::new);
indexing = in.readOptionalStreamable(IndexingStats::new);
get = in.readOptionalStreamable(GetStats::new);
search = in.readOptionalStreamable(SearchStats::new);
merge = in.readOptionalStreamable(MergeStats::new);
refresh = in.readOptionalStreamable(RefreshStats::new);
flush = in.readOptionalStreamable(FlushStats::new);
warmer = in.readOptionalStreamable(WarmerStats::new);
queryCache = in.readOptionalStreamable(QueryCacheStats::new);
fieldData = in.readOptionalStreamable(FieldDataStats::new);
completion = in.readOptionalStreamable(CompletionStats::new);
segments = in.readOptionalStreamable(SegmentsStats::new);
translog = in.readOptionalStreamable(TranslogStats::new);
requestCache = in.readOptionalStreamable(RequestCacheStats::new);
recoveryStats = in.readOptionalStreamable(RecoveryStats::new);
@ -271,84 +248,19 @@ public class CommonStats implements Writeable, ToXContent {
@Override
public void writeTo(StreamOutput out) throws IOException {
if (docs == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
docs.writeTo(out);
}
if (store == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
store.writeTo(out);
}
if (indexing == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
indexing.writeTo(out);
}
if (get == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
get.writeTo(out);
}
if (search == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
search.writeTo(out);
}
if (merge == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
merge.writeTo(out);
}
if (refresh == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
refresh.writeTo(out);
}
if (flush == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
flush.writeTo(out);
}
if (warmer == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
warmer.writeTo(out);
}
if (queryCache == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
queryCache.writeTo(out);
}
if (fieldData == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
fieldData.writeTo(out);
}
if (completion == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
completion.writeTo(out);
}
if (segments == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
segments.writeTo(out);
}
out.writeOptionalStreamable(docs);
out.writeOptionalStreamable(store);
out.writeOptionalStreamable(indexing);
out.writeOptionalStreamable(get);
out.writeOptionalStreamable(search);
out.writeOptionalStreamable(merge);
out.writeOptionalStreamable(refresh);
out.writeOptionalStreamable(flush);
out.writeOptionalStreamable(warmer);
out.writeOptionalStreamable(queryCache);
out.writeOptionalStreamable(fieldData);
out.writeOptionalStreamable(completion);
out.writeOptionalStreamable(segments);
out.writeOptionalStreamable(translog);
out.writeOptionalStreamable(requestCache);
out.writeOptionalStreamable(recoveryStats);
@ -590,53 +502,12 @@ public class CommonStats implements Writeable, ToXContent {
// note, requires a wrapping object
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
if (docs != null) {
docs.toXContent(builder, params);
}
if (store != null) {
store.toXContent(builder, params);
}
if (indexing != null) {
indexing.toXContent(builder, params);
}
if (get != null) {
get.toXContent(builder, params);
}
if (search != null) {
search.toXContent(builder, params);
}
if (merge != null) {
merge.toXContent(builder, params);
}
if (refresh != null) {
refresh.toXContent(builder, params);
}
if (flush != null) {
flush.toXContent(builder, params);
}
if (warmer != null) {
warmer.toXContent(builder, params);
}
if (queryCache != null) {
queryCache.toXContent(builder, params);
}
if (fieldData != null) {
fieldData.toXContent(builder, params);
}
if (completion != null) {
completion.toXContent(builder, params);
}
if (segments != null) {
segments.toXContent(builder, params);
}
if (translog != null) {
translog.toXContent(builder, params);
}
if (requestCache != null) {
requestCache.toXContent(builder, params);
}
if (recoveryStats != null) {
recoveryStats.toXContent(builder, params);
final Stream<ToXContent> stream = Arrays.stream(new ToXContent[] {
docs, store, indexing, get, search, merge, refresh, flush, warmer, queryCache,
fieldData, completion, segments, translog, requestCache, recoveryStats})
.filter(Objects::nonNull);
for (ToXContent toXContent : ((Iterable<ToXContent>)stream::iterator)) {
toXContent.toXContent(builder, params);
}
return builder;
}

View File

@ -135,19 +135,13 @@ public class IndicesStatsResponse extends BroadcastResponse implements ToXConten
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
shards = new ShardStats[in.readVInt()];
for (int i = 0; i < shards.length; i++) {
shards[i] = ShardStats.readShardStats(in);
}
shards = in.readArray(ShardStats::readShardStats, (size) -> new ShardStats[size]);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(shards.length);
for (ShardStats shard : shards) {
shard.writeTo(out);
}
out.writeArray(shards);
}
@Override

View File

@ -19,11 +19,13 @@
package org.elasticsearch.action.admin.indices.stats;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.routing.ShardRouting;
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.io.stream.Writeable;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.engine.CommitStats;
@ -32,7 +34,7 @@ import org.elasticsearch.index.shard.ShardPath;
import java.io.IOException;
public class ShardStats implements Streamable, ToXContent {
public class ShardStats implements Streamable, Writeable, ToXContent {
private ShardRouting shardRouting;
private CommonStats commonStats;
@Nullable
@ -102,7 +104,9 @@ public class ShardStats implements Streamable, ToXContent {
statePath = in.readString();
dataPath = in.readString();
isCustomDataPath = in.readBoolean();
seqNoStats = in.readOptionalWriteable(SeqNoStats::new);
if (in.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) {
seqNoStats = in.readOptionalWriteable(SeqNoStats::new);
}
}
@Override
@ -113,7 +117,9 @@ public class ShardStats implements Streamable, ToXContent {
out.writeString(statePath);
out.writeString(dataPath);
out.writeBoolean(isCustomDataPath);
out.writeOptionalWriteable(seqNoStats);
if (out.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) {
out.writeOptionalWriteable(seqNoStats);
}
}
@Override

View File

@ -155,7 +155,7 @@ public class TransportShardBulkAction extends TransportWriteAction<BulkShardRequ
final long version = indexResult.getVersion();
indexRequest.version(version);
indexRequest.versionType(indexRequest.versionType().versionTypeForReplicationAndRecovery());
indexRequest.seqNo(indexResult.getSeqNo());
indexRequest.setSeqNo(indexResult.getSeqNo());
assert indexRequest.versionType().validateVersionForWrites(indexRequest.version());
response = new IndexResponse(primary.shardId(), indexRequest.type(), indexRequest.id(), indexResult.getSeqNo(),
indexResult.getVersion(), indexResult.isCreated());
@ -179,7 +179,7 @@ public class TransportShardBulkAction extends TransportWriteAction<BulkShardRequ
// update the request with the version so it will go to the replicas
deleteRequest.versionType(deleteRequest.versionType().versionTypeForReplicationAndRecovery());
deleteRequest.version(deleteResult.getVersion());
deleteRequest.seqNo(deleteResult.getSeqNo());
deleteRequest.setSeqNo(deleteResult.getSeqNo());
assert deleteRequest.versionType().validateVersionForWrites(deleteRequest.version());
response = new DeleteResponse(request.shardId(), deleteRequest.type(), deleteRequest.id(), deleteResult.getSeqNo(),
deleteResult.getVersion(), deleteResult.isFound());
@ -189,6 +189,7 @@ public class TransportShardBulkAction extends TransportWriteAction<BulkShardRequ
break;
default: throw new IllegalStateException("unexpected opType [" + itemRequest.opType() + "] found");
}
// update the bulk item request because update request execution can mutate the bulk item request
request.items()[requestIndex] = replicaRequest;
if (operationResult == null) { // in case of noop update operation
@ -289,6 +290,7 @@ public class TransportShardBulkAction extends TransportWriteAction<BulkShardRequ
final long version = updateOperationResult.getVersion();
indexRequest.version(version);
indexRequest.versionType(indexRequest.versionType().versionTypeForReplicationAndRecovery());
indexRequest.setSeqNo(updateOperationResult.getSeqNo());
assert indexRequest.versionType().validateVersionForWrites(indexRequest.version());
}
break;
@ -299,6 +301,7 @@ public class TransportShardBulkAction extends TransportWriteAction<BulkShardRequ
// update the request with the version so it will go to the replicas
deleteRequest.versionType(deleteRequest.versionType().versionTypeForReplicationAndRecovery());
deleteRequest.version(updateOperationResult.getVersion());
deleteRequest.setSeqNo(updateOperationResult.getSeqNo());
assert deleteRequest.versionType().validateVersionForWrites(deleteRequest.version());
}
break;
@ -349,6 +352,10 @@ public class TransportShardBulkAction extends TransportWriteAction<BulkShardRequ
replicaRequest = new BulkItemRequest(request.items()[requestIndex].id(), updateDeleteRequest);
break;
}
assert (replicaRequest.request() instanceof IndexRequest
&& ((IndexRequest) replicaRequest.request()).getSeqNo() != SequenceNumbersService.UNASSIGNED_SEQ_NO) ||
(replicaRequest.request() instanceof DeleteRequest
&& ((DeleteRequest) replicaRequest.request()).getSeqNo() != SequenceNumbersService.UNASSIGNED_SEQ_NO);
// successful operation
break; // out of retry loop
} else if (updateOperationResult.getFailure() instanceof VersionConflictEngineException == false) {
@ -371,10 +378,10 @@ public class TransportShardBulkAction extends TransportWriteAction<BulkShardRequ
switch (docWriteRequest.opType()) {
case CREATE:
case INDEX:
operationResult = executeIndexRequestOnReplica(((IndexRequest) docWriteRequest), replica);
operationResult = executeIndexRequestOnReplica((IndexRequest) docWriteRequest, replica);
break;
case DELETE:
operationResult = executeDeleteRequestOnReplica(((DeleteRequest) docWriteRequest), replica);
operationResult = executeDeleteRequestOnReplica((DeleteRequest) docWriteRequest, replica);
break;
default:
throw new IllegalStateException("Unexpected request operation type on replica: "
@ -465,9 +472,9 @@ public class TransportShardBulkAction extends TransportWriteAction<BulkShardRequ
final Engine.Index operation;
try {
operation = replica.prepareIndexOnReplica(sourceToParse, request.seqNo(), request.version(), request.versionType(), request.getAutoGeneratedTimestamp(), request.isRetry());
operation = replica.prepareIndexOnReplica(sourceToParse, request.getSeqNo(), request.version(), request.versionType(), request.getAutoGeneratedTimestamp(), request.isRetry());
} catch (MapperParsingException e) {
return new Engine.IndexResult(e, request.version(), request.seqNo());
return new Engine.IndexResult(e, request.version(), request.getSeqNo());
}
Mapping update = operation.parsedDoc().dynamicMappingsUpdate();
if (update != null) {
@ -490,7 +497,7 @@ public class TransportShardBulkAction extends TransportWriteAction<BulkShardRequ
try {
operation = prepareIndexOperationOnPrimary(request, primary);
} catch (MapperParsingException | IllegalArgumentException e) {
return new Engine.IndexResult(e, request.version(), request.seqNo());
return new Engine.IndexResult(e, request.version(), request.getSeqNo());
}
Mapping update = operation.parsedDoc().dynamicMappingsUpdate();
final ShardId shardId = primary.shardId();
@ -501,12 +508,12 @@ public class TransportShardBulkAction extends TransportWriteAction<BulkShardRequ
mappingUpdatedAction.updateMappingOnMaster(shardId.getIndex(), request.type(), update);
} catch (IllegalArgumentException e) {
// throws IAE on conflicts merging dynamic mappings
return new Engine.IndexResult(e, request.version(), request.seqNo());
return new Engine.IndexResult(e, request.version(), request.getSeqNo());
}
try {
operation = prepareIndexOperationOnPrimary(request, primary);
} catch (MapperParsingException | IllegalArgumentException e) {
return new Engine.IndexResult(e, request.version(), request.seqNo());
return new Engine.IndexResult(e, request.version(), request.getSeqNo());
}
update = operation.parsedDoc().dynamicMappingsUpdate();
if (update != null) {
@ -524,7 +531,7 @@ public class TransportShardBulkAction extends TransportWriteAction<BulkShardRequ
public static Engine.DeleteResult executeDeleteRequestOnReplica(DeleteRequest request, IndexShard replica) {
final Engine.Delete delete = replica.prepareDeleteOnReplica(request.type(), request.id(),
request.seqNo(), request.primaryTerm(), request.version(), request.versionType());
request.getSeqNo(), request.primaryTerm(), request.version(), request.versionType());
return replica.delete(delete);
}
}

View File

@ -524,7 +524,10 @@ public class IndexRequest extends ReplicatedWriteRequest<IndexRequest> implement
out.writeOptionalString(routing);
out.writeOptionalString(parent);
if (out.getVersion().before(Version.V_6_0_0_alpha1_UNRELEASED)) {
out.writeOptionalString(null);
// Serialize a fake timestamp. 5.x expect this value to be set by the #process method so we can't use null.
// On the other hand, indices created on 5.x do not index the timestamp field. Therefore passing a 0 (or any value) for
// the transport layer OK as it will be ignored.
out.writeOptionalString("0");
out.writeOptionalWriteable(null);
}
out.writeBytesReference(source);

View File

@ -505,11 +505,7 @@ public abstract class TransportBroadcastByNodeAction<Request extends BroadcastRe
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
indicesLevelRequest = readRequestFrom(in);
int size = in.readVInt();
shards = new ArrayList<>(size);
for (int i = 0; i < size; i++) {
shards.add(new ShardRouting(in));
}
shards = in.readList(ShardRouting::new);
nodeId = in.readString();
}
@ -517,11 +513,7 @@ public abstract class TransportBroadcastByNodeAction<Request extends BroadcastRe
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
indicesLevelRequest.writeTo(out);
int size = shards.size();
out.writeVInt(size);
for (int i = 0; i < size; i++) {
shards.get(i).writeTo(out);
}
out.writeList(shards);
out.writeString(nodeId);
}
}
@ -566,18 +558,9 @@ public abstract class TransportBroadcastByNodeAction<Request extends BroadcastRe
super.readFrom(in);
nodeId = in.readString();
totalShards = in.readVInt();
int resultsSize = in.readVInt();
results = new ArrayList<>(resultsSize);
for (; resultsSize > 0; resultsSize--) {
final ShardOperationResult result = in.readBoolean() ? readShardResult(in) : null;
results.add(result);
}
results = in.readList((stream) -> stream.readBoolean() ? readShardResult(stream) : null);
if (in.readBoolean()) {
int failureShards = in.readVInt();
exceptions = new ArrayList<>(failureShards);
for (int i = 0; i < failureShards; i++) {
exceptions.add(new BroadcastShardOperationFailedException(in));
}
exceptions = in.readList(BroadcastShardOperationFailedException::new);
} else {
exceptions = null;
}
@ -594,11 +577,7 @@ public abstract class TransportBroadcastByNodeAction<Request extends BroadcastRe
}
out.writeBoolean(exceptions != null);
if (exceptions != null) {
int failureShards = exceptions.size();
out.writeVInt(failureShards);
for (int i = 0; i < failureShards; i++) {
exceptions.get(i).writeTo(out);
}
out.writeList(exceptions);
}
}
}

View File

@ -37,4 +37,9 @@ public class BasicReplicationRequest extends ReplicationRequest<BasicReplication
public BasicReplicationRequest(ShardId shardId) {
super(shardId);
}
@Override
public String toString() {
return "BasicReplicationRequest{" + shardId + "}";
}
}

View File

@ -19,12 +19,14 @@
package org.elasticsearch.action.support.replication;
import org.elasticsearch.Version;
import org.elasticsearch.action.bulk.BulkShardRequest;
import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.index.seqno.SequenceNumbersService;
import org.elasticsearch.index.shard.ShardId;
import java.io.IOException;
@ -36,6 +38,8 @@ import java.io.IOException;
public abstract class ReplicatedWriteRequest<R extends ReplicatedWriteRequest<R>> extends ReplicationRequest<R> implements WriteRequest<R> {
private RefreshPolicy refreshPolicy = RefreshPolicy.NONE;
private long seqNo = SequenceNumbersService.UNASSIGNED_SEQ_NO;
/**
* Constructor for deserialization.
*/
@ -62,11 +66,32 @@ public abstract class ReplicatedWriteRequest<R extends ReplicatedWriteRequest<R>
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
refreshPolicy = RefreshPolicy.readFrom(in);
if (in.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) {
seqNo = in.readZLong();
} else {
seqNo = SequenceNumbersService.UNASSIGNED_SEQ_NO;
}
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
refreshPolicy.writeTo(out);
if (out.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) {
out.writeZLong(seqNo);
}
}
/**
* Returns the sequence number for this operation. The sequence number is assigned while the operation
* is performed on the primary shard.
*/
public long getSeqNo() {
return seqNo;
}
/** sets the sequence number for this operation. should only be called on the primary shard */
public void setSeqNo(long seqNo) {
this.seqNo = seqNo;
}
}

View File

@ -283,7 +283,7 @@ public class ReplicationOperation<
}
private void decPendingAndFinishIfNeeded() {
assert pendingActions.get() > 0;
assert pendingActions.get() > 0 : "pending action count goes below 0 for request [" + request + "]";
if (pendingActions.decrementAndGet() == 0) {
finish();
}

View File

@ -55,7 +55,6 @@ public abstract class ReplicationRequest<Request extends ReplicationRequest<Requ
*/
protected ShardId shardId;
long seqNo;
long primaryTerm;
protected TimeValue timeout = DEFAULT_TIMEOUT;
@ -171,19 +170,6 @@ public abstract class ReplicationRequest<Request extends ReplicationRequest<Requ
return routedBasedOnClusterVersion;
}
/**
* Returns the sequence number for this operation. The sequence number is assigned while the operation
* is performed on the primary shard.
*/
public long seqNo() {
return seqNo;
}
/** sets the sequence number for this operation. should only be called on the primary shard */
public void seqNo(long seqNo) {
this.seqNo = seqNo;
}
/** returns the primary term active at the time the operation was performed on the primary shard */
public long primaryTerm() {
return primaryTerm;
@ -215,7 +201,6 @@ public abstract class ReplicationRequest<Request extends ReplicationRequest<Requ
timeout = new TimeValue(in);
index = in.readString();
routedBasedOnClusterVersion = in.readVLong();
seqNo = in.readVLong();
primaryTerm = in.readVLong();
}
@ -232,7 +217,6 @@ public abstract class ReplicationRequest<Request extends ReplicationRequest<Requ
timeout.writeTo(out);
out.writeString(index);
out.writeVLong(routedBasedOnClusterVersion);
out.writeVLong(seqNo);
out.writeVLong(primaryTerm);
}
@ -252,13 +236,7 @@ public abstract class ReplicationRequest<Request extends ReplicationRequest<Requ
}
@Override
public String toString() {
if (shardId != null) {
return shardId.toString();
} else {
return index;
}
}
public abstract String toString(); // force a proper to string to ease debugging
@Override
public String getDescription() {

View File

@ -21,6 +21,7 @@ package org.elasticsearch.action.support.replication;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionListenerResponseHandler;
import org.elasticsearch.action.ActionResponse;
@ -43,6 +44,7 @@ import org.elasticsearch.cluster.routing.AllocationId;
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lease.Releasable;
@ -57,6 +59,7 @@ import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.IndexShardState;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardNotFoundException;
import org.elasticsearch.indices.IndexClosedException;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.node.NodeClosedException;
import org.elasticsearch.tasks.Task;
@ -183,17 +186,19 @@ public abstract class TransportReplicationAction<
protected abstract ReplicaResult shardOperationOnReplica(ReplicaRequest shardRequest, IndexShard replica) throws Exception;
/**
* Cluster level block to check before request execution
* Cluster level block to check before request execution. Returning null means that no blocks need to be checked.
*/
@Nullable
protected ClusterBlockLevel globalBlockLevel() {
return ClusterBlockLevel.WRITE;
return null;
}
/**
* Index level block to check before request execution
* Index level block to check before request execution. Returning null means that no blocks need to be checked.
*/
@Nullable
protected ClusterBlockLevel indexBlockLevel() {
return ClusterBlockLevel.WRITE;
return null;
}
/**
@ -645,6 +650,9 @@ public abstract class TransportReplicationAction<
retry(new IndexNotFoundException(concreteIndex));
return;
}
if (indexMetaData.getState() == IndexMetaData.State.CLOSE) {
throw new IndexClosedException(indexMetaData.getIndex());
}
// resolve all derived request fields, so we can route and apply it
resolveRequest(state.metaData(), indexMetaData, request);
@ -667,7 +675,7 @@ public abstract class TransportReplicationAction<
private void performLocalAction(ClusterState state, ShardRouting primary, DiscoveryNode node) {
setPhase(task, "waiting_on_primary");
if (logger.isTraceEnabled()) {
logger.trace("send action [{}] on primary [{}] for request [{}] with cluster state version [{}] to [{}] ",
logger.trace("send action [{}] to local primary [{}] for request [{}] with cluster state version [{}] to [{}] ",
transportPrimaryAction, request.shardId(), request, state.version(), primary.currentNodeId());
}
performAction(node, transportPrimaryAction, true, new ConcreteShardRequest<>(request, primary.allocationId().getId()));
@ -721,15 +729,21 @@ public abstract class TransportReplicationAction<
}
private boolean handleBlockExceptions(ClusterState state) {
ClusterBlockException blockException = state.blocks().globalBlockedException(globalBlockLevel());
if (blockException != null) {
handleBlockException(blockException);
return true;
ClusterBlockLevel globalBlockLevel = globalBlockLevel();
if (globalBlockLevel != null) {
ClusterBlockException blockException = state.blocks().globalBlockedException(globalBlockLevel);
if (blockException != null) {
handleBlockException(blockException);
return true;
}
}
blockException = state.blocks().indexBlockedException(indexBlockLevel(), concreteIndex(state));
if (blockException != null) {
handleBlockException(blockException);
return true;
ClusterBlockLevel indexBlockLevel = indexBlockLevel();
if (indexBlockLevel != null) {
ClusterBlockException blockException = state.blocks().indexBlockedException(indexBlockLevel, concreteIndex(state));
if (blockException != null) {
handleBlockException(blockException);
return true;
}
}
return false;
}
@ -954,6 +968,8 @@ public abstract class TransportReplicationAction<
public PrimaryResult perform(Request request) throws Exception {
PrimaryResult result = shardOperationOnPrimary(request, indexShard);
if (result.replicaRequest() != null) {
assert result.finalFailure == null : "a replica request [" + result.replicaRequest()
+ "] with a primary failure [" + result.finalFailure + "]";
result.replicaRequest().primaryTerm(indexShard.getPrimaryTerm());
}
return result;
@ -987,16 +1003,25 @@ public abstract class TransportReplicationAction<
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
localCheckpoint = in.readZLong();
allocationId = in.readString();
if (in.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) {
super.readFrom(in);
localCheckpoint = in.readZLong();
allocationId = in.readString();
} else {
// 5.x used to read empty responses, which don't really read anything off the stream, so just do nothing.
}
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeZLong(localCheckpoint);
out.writeString(allocationId);
if (out.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) {
super.writeTo(out);
out.writeZLong(localCheckpoint);
out.writeString(allocationId);
} else {
// we use to write empty responses
Empty.INSTANCE.writeTo(out);
}
}
@Override
@ -1020,10 +1045,9 @@ public abstract class TransportReplicationAction<
listener.onFailure(new NoNodeAvailableException("unknown node [" + nodeId + "]"));
return;
}
transportService.sendRequest(node, transportReplicaAction,
new ConcreteShardRequest<>(request, replica.allocationId().getId()), transportOptions,
// Eclipse can't handle when this is <> so we specify the type here.
new ActionListenerResponseHandler<ReplicaResponse>(listener, ReplicaResponse::new));
final ConcreteShardRequest<ReplicaRequest> concreteShardRequest =
new ConcreteShardRequest<>(request, replica.allocationId().getId());
sendReplicaRequest(concreteShardRequest, node, listener);
}
@Override
@ -1064,6 +1088,14 @@ public abstract class TransportReplicationAction<
}
}
/** sends the given replica request to the supplied nodes */
protected void sendReplicaRequest(ConcreteShardRequest<ReplicaRequest> concreteShardRequest, DiscoveryNode node,
ActionListener<ReplicationOperation.ReplicaResponse> listener) {
transportService.sendRequest(node, transportReplicaAction, concreteShardRequest, transportOptions,
// Eclipse can't handle when this is <> so we specify the type here.
new ActionListenerResponseHandler<ReplicaResponse>(listener, ReplicaResponse::new));
}
/** a wrapper class to encapsulate a request when being sent to a specific allocation id **/
public static final class ConcreteShardRequest<R extends TransportRequest> extends TransportRequest {

View File

@ -25,6 +25,7 @@ import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.action.support.WriteResponse;
import org.elasticsearch.cluster.action.shard.ShardStateAction;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Nullable;
@ -193,6 +194,16 @@ public abstract class TransportWriteAction<
}
}
@Override
protected ClusterBlockLevel globalBlockLevel() {
return ClusterBlockLevel.WRITE;
}
@Override
protected ClusterBlockLevel indexBlockLevel() {
return ClusterBlockLevel.WRITE;
}
/**
* callback used by {@link AsyncAfterWriteAction} to notify that all post
* process actions have been executed

View File

@ -30,11 +30,13 @@ import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.StringHelper;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.Version;
import org.elasticsearch.cli.ExitCodes;
import org.elasticsearch.cli.Terminal;
import org.elasticsearch.cli.UserException;
import org.elasticsearch.common.PidFile;
import org.elasticsearch.common.SuppressForbidden;
import org.elasticsearch.common.inject.CreationException;
import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.common.logging.ESLoggerFactory;
import org.elasticsearch.common.logging.LogConfigurator;
import org.elasticsearch.common.logging.Loggers;
@ -56,7 +58,9 @@ 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.concurrent.CountDownLatch;
/**
@ -93,7 +97,7 @@ final class Bootstrap {
}
/** initialize native resources */
public static void initializeNatives(Path tmpFile, boolean mlockAll, boolean seccomp, boolean ctrlHandler) {
public static void initializeNatives(Path tmpFile, boolean mlockAll, boolean systemCallFilter, boolean ctrlHandler) {
final Logger logger = Loggers.getLogger(Bootstrap.class);
// check if the user is running as root, and bail
@ -101,9 +105,9 @@ final class Bootstrap {
throw new RuntimeException("can not run elasticsearch as root");
}
// enable secure computing mode
if (seccomp) {
Natives.trySeccomp(tmpFile);
// enable system call filter
if (systemCallFilter) {
Natives.tryInstallSystemCallFilter(tmpFile);
}
// mlockall if requested
@ -177,7 +181,7 @@ final class Bootstrap {
initializeNatives(
environment.tmpFile(),
BootstrapSettings.MEMORY_LOCK_SETTING.get(settings),
BootstrapSettings.SECCOMP_SETTING.get(settings),
BootstrapSettings.SYSTEM_CALL_FILTER_SETTING.get(settings),
BootstrapSettings.CTRLHANDLER_SETTING.get(settings));
// initialize probes before the security manager is installed

View File

@ -166,7 +166,7 @@ final class BootstrapChecks {
}
checks.add(new ClientJvmCheck());
checks.add(new UseSerialGCCheck());
checks.add(new SystemCallFilterCheck(BootstrapSettings.SECCOMP_SETTING.get(settings)));
checks.add(new SystemCallFilterCheck(BootstrapSettings.SYSTEM_CALL_FILTER_SETTING.get(settings)));
checks.add(new OnErrorCheck());
checks.add(new OnOutOfMemoryErrorCheck());
checks.add(new G1GCCheck());
@ -463,12 +463,12 @@ final class BootstrapChecks {
@Override
public boolean check() {
return areSystemCallFiltersEnabled && !isSeccompInstalled();
return areSystemCallFiltersEnabled && !isSystemCallFilterInstalled();
}
// visible for testing
boolean isSeccompInstalled() {
return Natives.isSeccompInstalled();
boolean isSystemCallFilterInstalled() {
return Natives.isSystemCallFilterInstalled();
}
@Override
@ -483,12 +483,12 @@ final class BootstrapChecks {
@Override
public boolean check() {
return isSeccompInstalled() && mightFork();
return isSystemCallFilterInstalled() && mightFork();
}
// visible for testing
boolean isSeccompInstalled() {
return Natives.isSeccompInstalled();
boolean isSystemCallFilterInstalled() {
return Natives.isSystemCallFilterInstalled();
}
// visible for testing
@ -521,7 +521,7 @@ final class BootstrapChecks {
"OnError [%s] requires forking but is prevented by system call filters ([%s=true]);" +
" upgrade to at least Java 8u92 and use ExitOnOutOfMemoryError",
onError(),
BootstrapSettings.SECCOMP_SETTING.getKey());
BootstrapSettings.SYSTEM_CALL_FILTER_SETTING.getKey());
}
}
@ -546,7 +546,7 @@ final class BootstrapChecks {
"OnOutOfMemoryError [%s] requires forking but is prevented by system call filters ([%s=true]);" +
" upgrade to at least Java 8u92 and use ExitOnOutOfMemoryError",
onOutOfMemoryError(),
BootstrapSettings.SECCOMP_SETTING.getKey());
BootstrapSettings.SYSTEM_CALL_FILTER_SETTING.getKey());
}
}

View File

@ -24,16 +24,16 @@ import org.elasticsearch.common.SuppressForbidden;
import java.util.Dictionary;
import java.util.Enumeration;
/**
* Exposes system startup information
/**
* Exposes system startup information
*/
@SuppressForbidden(reason = "exposes read-only view of system properties")
public final class BootstrapInfo {
/** no instantiation */
private BootstrapInfo() {}
/**
/**
* Returns true if we successfully loaded native libraries.
* <p>
* If this returns false, then native operations such as locking
@ -42,19 +42,19 @@ public final class BootstrapInfo {
public static boolean isNativesAvailable() {
return Natives.JNA_AVAILABLE;
}
/**
/**
* Returns true if we were able to lock the process's address space.
*/
public static boolean isMemoryLocked() {
return Natives.isMemoryLocked();
}
/**
* Returns true if secure computing mode is enabled (supported systems only)
* Returns true if system call filter is installed (supported systems only)
*/
public static boolean isSeccompInstalled() {
return Natives.isSeccompInstalled();
public static boolean isSystemCallFilterInstalled() {
return Natives.isSystemCallFilterInstalled();
}
/**

View File

@ -33,8 +33,8 @@ public final class BootstrapSettings {
public static final Setting<Boolean> MEMORY_LOCK_SETTING =
Setting.boolSetting("bootstrap.memory_lock", false, Property.NodeScope);
public static final Setting<Boolean> SECCOMP_SETTING =
Setting.boolSetting("bootstrap.seccomp", true, Property.NodeScope);
public static final Setting<Boolean> SYSTEM_CALL_FILTER_SETTING =
Setting.boolSetting("bootstrap.system_call_filter", true, Property.NodeScope);
public static final Setting<Boolean> CTRLHANDLER_SETTING =
Setting.boolSetting("bootstrap.ctrlhandler", true, Property.NodeScope);

View File

@ -43,11 +43,11 @@ class JNANatives {
// Set to true, in case native mlockall call was successful
static boolean LOCAL_MLOCKALL = false;
// Set to true, in case native seccomp call was successful
static boolean LOCAL_SECCOMP = false;
// Set to true, in case native system call filter install was successful
static boolean LOCAL_SYSTEM_CALL_FILTER = false;
// Set to true, in case policy can be applied to all threads of the process (even existing ones)
// otherwise they are only inherited for new threads (ES app threads)
static boolean LOCAL_SECCOMP_ALL = false;
static boolean LOCAL_SYSTEM_CALL_FILTER_ALL = false;
// set to the maximum number of threads that can be created for
// the user ID that owns the running Elasticsearch process
static long MAX_NUMBER_OF_THREADS = -1;
@ -210,12 +210,12 @@ class JNANatives {
}
}
static void trySeccomp(Path tmpFile) {
static void tryInstallSystemCallFilter(Path tmpFile) {
try {
int ret = Seccomp.init(tmpFile);
LOCAL_SECCOMP = true;
int ret = SystemCallFilter.init(tmpFile);
LOCAL_SYSTEM_CALL_FILTER = true;
if (ret == 1) {
LOCAL_SECCOMP_ALL = true;
LOCAL_SYSTEM_CALL_FILTER_ALL = true;
}
} catch (Exception e) {
// this is likely to happen unless the kernel is newish, its a best effort at the moment

View File

@ -91,12 +91,12 @@ final class Natives {
return JNANatives.LOCAL_MLOCKALL;
}
static void trySeccomp(Path tmpFile) {
static void tryInstallSystemCallFilter(Path tmpFile) {
if (!JNA_AVAILABLE) {
logger.warn("cannot install syscall filters because JNA is not available");
logger.warn("cannot install system call filter because JNA is not available");
return;
}
JNANatives.trySeccomp(tmpFile);
JNANatives.tryInstallSystemCallFilter(tmpFile);
}
static void trySetMaxNumberOfThreads() {
@ -115,10 +115,10 @@ final class Natives {
JNANatives.trySetMaxSizeVirtualMemory();
}
static boolean isSeccompInstalled() {
static boolean isSystemCallFilterInstalled() {
if (!JNA_AVAILABLE) {
return false;
}
return JNANatives.LOCAL_SECCOMP;
return JNANatives.LOCAL_SYSTEM_CALL_FILTER;
}
}

View File

@ -34,7 +34,7 @@ import java.util.List;
import java.util.Locale;
/**
* Spawns native plugin controller processes if present. Will only work prior to seccomp being set up.
* Spawns native plugin controller processes if present. Will only work prior to a system call filter being installed.
*/
final class Spawner implements Closeable {

View File

@ -43,8 +43,7 @@ import java.util.List;
import java.util.Map;
/**
* Installs a limited form of secure computing mode,
* to filters system calls to block process execution.
* Installs a system call filter to block process execution.
* <p>
* This is supported on Linux, Solaris, FreeBSD, OpenBSD, Mac OS X, and Windows.
* <p>
@ -91,8 +90,8 @@ import java.util.Map;
* https://docs.oracle.com/cd/E23824_01/html/821-1456/prbac-2.html</a>
*/
// not an example of how to write code!!!
final class Seccomp {
private static final Logger logger = Loggers.getLogger(Seccomp.class);
final class SystemCallFilter {
private static final Logger logger = Loggers.getLogger(SystemCallFilter.class);
// Linux implementation, based on seccomp(2) or prctl(2) with bpf filtering
@ -269,7 +268,8 @@ final class Seccomp {
// we couldn't link methods, could be some really ancient kernel (e.g. < 2.1.57) or some bug
if (linux_libc == null) {
throw new UnsupportedOperationException("seccomp unavailable: could not link methods. requires kernel 3.5+ with CONFIG_SECCOMP and CONFIG_SECCOMP_FILTER compiled in");
throw new UnsupportedOperationException("seccomp unavailable: could not link methods. requires kernel 3.5+ " +
"with CONFIG_SECCOMP and CONFIG_SECCOMP_FILTER compiled in");
}
// pure paranoia:
@ -319,7 +319,8 @@ final class Seccomp {
switch (errno) {
case ENOSYS: break; // ok
case EINVAL: break; // ok
default: throw new UnsupportedOperationException("seccomp(SECCOMP_SET_MODE_FILTER, BOGUS_FLAG): " + JNACLibrary.strerror(errno));
default: throw new UnsupportedOperationException("seccomp(SECCOMP_SET_MODE_FILTER, BOGUS_FLAG): "
+ JNACLibrary.strerror(errno));
}
}
@ -346,7 +347,8 @@ final class Seccomp {
int errno = Native.getLastError();
if (errno == EINVAL) {
// friendly error, this will be the typical case for an old kernel
throw new UnsupportedOperationException("seccomp unavailable: requires kernel 3.5+ with CONFIG_SECCOMP and CONFIG_SECCOMP_FILTER compiled in");
throw new UnsupportedOperationException("seccomp unavailable: requires kernel 3.5+ with" +
" CONFIG_SECCOMP and CONFIG_SECCOMP_FILTER compiled in");
} else {
throw new UnsupportedOperationException("prctl(PR_GET_NO_NEW_PRIVS): " + JNACLibrary.strerror(errno));
}
@ -358,7 +360,8 @@ final class Seccomp {
default:
int errno = Native.getLastError();
if (errno == EINVAL) {
throw new UnsupportedOperationException("seccomp unavailable: CONFIG_SECCOMP not compiled into kernel, CONFIG_SECCOMP and CONFIG_SECCOMP_FILTER are needed");
throw new UnsupportedOperationException("seccomp unavailable: CONFIG_SECCOMP not compiled into kernel," +
" CONFIG_SECCOMP and CONFIG_SECCOMP_FILTER are needed");
} else {
throw new UnsupportedOperationException("prctl(PR_GET_SECCOMP): " + JNACLibrary.strerror(errno));
}
@ -368,7 +371,8 @@ final class Seccomp {
int errno = Native.getLastError();
switch (errno) {
case EFAULT: break; // available
case EINVAL: throw new UnsupportedOperationException("seccomp unavailable: CONFIG_SECCOMP_FILTER not compiled into kernel, CONFIG_SECCOMP and CONFIG_SECCOMP_FILTER are needed");
case EINVAL: throw new UnsupportedOperationException("seccomp unavailable: CONFIG_SECCOMP_FILTER not" +
" compiled into kernel, CONFIG_SECCOMP and CONFIG_SECCOMP_FILTER are needed");
default: throw new UnsupportedOperationException("prctl(PR_SET_SECCOMP): " + JNACLibrary.strerror(errno));
}
}
@ -380,10 +384,12 @@ final class Seccomp {
// check it worked
if (linux_prctl(PR_GET_NO_NEW_PRIVS, 0, 0, 0, 0) != 1) {
throw new UnsupportedOperationException("seccomp filter did not really succeed: prctl(PR_GET_NO_NEW_PRIVS): " + JNACLibrary.strerror(Native.getLastError()));
throw new UnsupportedOperationException("seccomp filter did not really succeed: prctl(PR_GET_NO_NEW_PRIVS): " +
JNACLibrary.strerror(Native.getLastError()));
}
// BPF installed to check arch, limit, then syscall. See https://www.kernel.org/doc/Documentation/prctl/seccomp_filter.txt for details.
// BPF installed to check arch, limit, then syscall.
// See https://www.kernel.org/doc/Documentation/prctl/seccomp_filter.txt for details.
SockFilter insns[] = {
/* 1 */ BPF_STMT(BPF_LD + BPF_W + BPF_ABS, SECCOMP_DATA_ARCH_OFFSET), //
/* 2 */ BPF_JUMP(BPF_JMP + BPF_JEQ + BPF_K, arch.audit, 0, 7), // if (arch != audit) goto fail;
@ -408,7 +414,8 @@ final class Seccomp {
method = 0;
int errno1 = Native.getLastError();
if (logger.isDebugEnabled()) {
logger.debug("seccomp(SECCOMP_SET_MODE_FILTER): {}, falling back to prctl(PR_SET_SECCOMP)...", JNACLibrary.strerror(errno1));
logger.debug("seccomp(SECCOMP_SET_MODE_FILTER): {}, falling back to prctl(PR_SET_SECCOMP)...",
JNACLibrary.strerror(errno1));
}
if (linux_prctl(PR_SET_SECCOMP, SECCOMP_MODE_FILTER, pointer, 0, 0) != 0) {
int errno2 = Native.getLastError();
@ -419,7 +426,8 @@ final class Seccomp {
// now check that the filter was really installed, we should be in filter mode.
if (linux_prctl(PR_GET_SECCOMP, 0, 0, 0, 0) != 2) {
throw new UnsupportedOperationException("seccomp filter installation did not really succeed. seccomp(PR_GET_SECCOMP): " + JNACLibrary.strerror(Native.getLastError()));
throw new UnsupportedOperationException("seccomp filter installation did not really succeed. seccomp(PR_GET_SECCOMP): "
+ JNACLibrary.strerror(Native.getLastError()));
}
logger.debug("Linux seccomp filter installation successful, threads: [{}]", method == 1 ? "all" : "app" );

View File

@ -204,7 +204,7 @@ public class MappingMetaData extends AbstractDiffable<MappingMetaData> {
// timestamp
out.writeBoolean(false); // enabled
out.writeString(DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.format());
out.writeOptionalString(null);
out.writeOptionalString("now"); // 5.x default
out.writeOptionalBoolean(null);
}
out.writeBoolean(hasParentField());

View File

@ -67,6 +67,7 @@ import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.MapperService.MergeReason;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.indices.IndexCreationException;
import org.elasticsearch.indices.IndicesService;
@ -356,10 +357,10 @@ public class MetaDataCreateIndexService extends AbstractComponent {
// now add the mappings
MapperService mapperService = indexService.mapperService();
try {
mapperService.merge(mappings, request.updateAllTypes());
} catch (MapperParsingException mpe) {
mapperService.merge(mappings, MergeReason.MAPPING_UPDATE, request.updateAllTypes());
} catch (Exception e) {
removalExtraInfo = "failed on parsing default mapping/mappings on index creation";
throw mpe;
throw e;
}
// the context is only used for validation so it's fine to pass fake values for the shard id and the current

View File

@ -141,15 +141,11 @@ public class MetaDataIndexAliasesService extends AbstractComponent {
// temporarily create the index and add mappings so we can parse the filter
try {
indexService = indicesService.createIndex(index, emptyList(), shardId -> {});
indicesToClose.add(index.getIndex());
} catch (IOException e) {
throw new ElasticsearchException("Failed to create temporary index for parsing the alias", e);
}
for (ObjectCursor<MappingMetaData> cursor : index.getMappings().values()) {
MappingMetaData mappingMetaData = cursor.value;
indexService.mapperService().merge(mappingMetaData.type(), mappingMetaData.source(),
MapperService.MergeReason.MAPPING_RECOVERY, false);
}
indicesToClose.add(index.getIndex());
indexService.mapperService().merge(index, MapperService.MergeReason.MAPPING_RECOVERY, false);
}
indices.put(action.getIndex(), indexService);
}

View File

@ -39,6 +39,7 @@ import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.MapperService.MergeReason;
import org.elasticsearch.indices.IndexTemplateMissingException;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.InvalidIndexTemplateException;
@ -222,7 +223,7 @@ public class MetaDataIndexTemplateService extends AbstractComponent {
mappingsForValidation.put(entry.getKey(), MapperService.parseMapping(entry.getValue()));
}
dummyIndexService.mapperService().merge(mappingsForValidation, false);
dummyIndexService.mapperService().merge(mappingsForValidation, MergeReason.MAPPING_UPDATE, false);
} finally {
if (createdIndex != null) {

View File

@ -147,10 +147,7 @@ public class MetaDataIndexUpgradeService extends AbstractComponent {
};
try (IndexAnalyzers fakeIndexAnalzyers = new IndexAnalyzers(indexSettings, fakeDefault, fakeDefault, fakeDefault, analyzerMap)) {
MapperService mapperService = new MapperService(indexSettings, fakeIndexAnalzyers, similarityService, mapperRegistry, () -> null);
for (ObjectCursor<MappingMetaData> cursor : indexMetaData.getMappings().values()) {
MappingMetaData mappingMetaData = cursor.value;
mapperService.merge(mappingMetaData.type(), mappingMetaData.source(), MapperService.MergeReason.MAPPING_RECOVERY, false);
}
mapperService.merge(indexMetaData, MapperService.MergeReason.MAPPING_RECOVERY, false);
}
} catch (Exception ex) {
// Wrap the inner exception so we have the index name in the exception message

View File

@ -43,6 +43,7 @@ import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.MapperService.MergeReason;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.InvalidTypeNameException;
@ -146,10 +147,7 @@ public class MetaDataMappingService extends AbstractComponent {
// we need to create the index here, and add the current mapping to it, so we can merge
indexService = indicesService.createIndex(indexMetaData, Collections.emptyList(), shardId -> {});
removeIndex = true;
for (ObjectCursor<MappingMetaData> metaData : indexMetaData.getMappings().values()) {
// don't apply the default mapping, it has been applied when the mapping was created
indexService.mapperService().merge(metaData.value.type(), metaData.value.source(), MapperService.MergeReason.MAPPING_RECOVERY, true);
}
indexService.mapperService().merge(indexMetaData, MergeReason.MAPPING_RECOVERY, true);
}
IndexMetaData.Builder builder = IndexMetaData.builder(indexMetaData);
@ -226,10 +224,7 @@ public class MetaDataMappingService extends AbstractComponent {
MapperService mapperService = indicesService.createIndexMapperService(indexMetaData);
indexMapperServices.put(index, mapperService);
// add mappings for all types, we need them for cross-type validation
for (ObjectCursor<MappingMetaData> mapping : indexMetaData.getMappings().values()) {
mapperService.merge(mapping.value.type(), mapping.value.source(),
MapperService.MergeReason.MAPPING_RECOVERY, request.updateAllTypes());
}
mapperService.merge(indexMetaData, MergeReason.MAPPING_RECOVERY, request.updateAllTypes());
}
}
currentState = applyRequest(currentState, request, indexMapperServices);
@ -313,7 +308,7 @@ public class MetaDataMappingService extends AbstractComponent {
if (existingMapper != null) {
existingSource = existingMapper.mappingSource();
}
DocumentMapper mergedMapper = mapperService.merge(mappingType, mappingUpdateSource, MapperService.MergeReason.MAPPING_UPDATE, request.updateAllTypes());
DocumentMapper mergedMapper = mapperService.merge(mappingType, mappingUpdateSource, MergeReason.MAPPING_UPDATE, request.updateAllTypes());
CompressedXContent updatedSource = mergedMapper.mappingSource();
if (existingSource != null) {

View File

@ -0,0 +1,132 @@
/*
* 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;
import com.carrotsearch.hppc.ObjectLongHashMap;
import com.carrotsearch.hppc.cursors.ObjectLongCursor;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.xcontent.XContentBuilder;
import java.io.IOException;
import java.util.Iterator;
import java.util.Objects;
/**
* A reusable class to encode <tt>field -&gt; memory size</tt> mappings
*/
public final class FieldMemoryStats implements Writeable, Iterable<ObjectLongCursor<String>>{
private final ObjectLongHashMap<String> stats;
/**
* Creates a new FieldMemoryStats instance
*/
public FieldMemoryStats(ObjectLongHashMap<String> stats) {
this.stats = Objects.requireNonNull(stats, "status must be non-null");
assert !stats.containsKey(null);
}
/**
* Creates a new FieldMemoryStats instance from a stream
*/
public FieldMemoryStats(StreamInput input) throws IOException {
int size = input.readVInt();
stats = new ObjectLongHashMap<>(size);
for (int i = 0; i < size; i++) {
stats.put(input.readString(), input.readVLong());
}
}
/**
* Adds / merges the given field memory stats into this stats instance
*/
public void add(FieldMemoryStats fieldMemoryStats) {
for (ObjectLongCursor<String> entry : fieldMemoryStats.stats) {
stats.addTo(entry.key, entry.value);
}
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeVInt(stats.size());
for (ObjectLongCursor<String> entry : stats) {
out.writeString(entry.key);
out.writeVLong(entry.value);
}
}
/**
* Generates x-content into the given builder for each of the fields in this stats instance
* @param builder the builder to generated on
* @param key the top level key for this stats object
* @param rawKey the raw byte key for each of the fields byte sizes
* @param readableKey the readable key for each of the fields byte sizes
*/
public void toXContent(XContentBuilder builder, String key, String rawKey, String readableKey) throws IOException {
builder.startObject(key);
for (ObjectLongCursor<String> entry : stats) {
builder.startObject(entry.key);
builder.byteSizeField(rawKey, readableKey, entry.value);
builder.endObject();
}
builder.endObject();
}
/**
* Creates a deep copy of this stats instance
*/
public FieldMemoryStats copy() {
return new FieldMemoryStats(stats.clone());
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
FieldMemoryStats that = (FieldMemoryStats) o;
return Objects.equals(stats, that.stats);
}
@Override
public int hashCode() {
return Objects.hash(stats);
}
@Override
public Iterator<ObjectLongCursor<String>> iterator() {
return stats.iterator();
}
/**
* Returns the fields value in bytes or <code>0</code> if it's not present in the stats
*/
public long get(String field) {
return stats.get(field);
}
/**
* Returns <code>true</code> iff the given field is in the stats
*/
public boolean containsField(String field) {
return stats.containsKey(field);
}
}

View File

@ -467,16 +467,32 @@ public abstract class StreamOutput extends OutputStream {
* @param keyWriter The key writer
* @param valueWriter The value writer
*/
public <K, V> void writeMapOfLists(final Map<K, List<V>> map, final Writer<K> keyWriter, final Writer<V> valueWriter)
public final <K, V> void writeMapOfLists(final Map<K, List<V>> map, final Writer<K> keyWriter, final Writer<V> valueWriter)
throws IOException {
writeVInt(map.size());
for (final Map.Entry<K, List<V>> entry : map.entrySet()) {
keyWriter.write(this, entry.getKey());
writeVInt(entry.getValue().size());
for (final V value : entry.getValue()) {
writeMap(map, keyWriter, (stream, list) -> {
writeVInt(list.size());
for (final V value : list) {
valueWriter.write(this, value);
}
});
}
/**
* Write a {@link Map} of {@code K}-type keys to {@code V}-type.
* <pre><code>
* Map&lt;String, String&gt; map = ...;
* out.writeMap(map, StreamOutput::writeString, StreamOutput::writeString);
* </code></pre>
*
* @param keyWriter The key writer
* @param valueWriter The value writer
*/
public final <K, V> void writeMap(final Map<K, V> map, final Writer<K> keyWriter, final Writer<V> valueWriter)
throws IOException {
writeVInt(map.size());
for (final Map.Entry<K, V> entry : map.entrySet()) {
keyWriter.write(this, entry.getKey());
valueWriter.write(this, entry.getValue());
}
}

View File

@ -239,11 +239,9 @@ public abstract class AbstractScopedSettings extends AbstractComponent {
*/
public final void validate(Settings settings) {
List<RuntimeException> exceptions = new ArrayList<>();
// we want them sorted for deterministic error messages
SortedMap<String, String> sortedSettings = new TreeMap<>(settings.getAsMap());
for (Map.Entry<String, String> entry : sortedSettings.entrySet()) {
for (String key : settings.getAsMap().keySet()) { // settings iterate in deterministic fashion
try {
validate(entry.getKey(), settings);
validate(key, settings);
} catch (RuntimeException ex) {
exceptions.add(ex);
}

View File

@ -390,7 +390,7 @@ public final class ClusterSettings extends AbstractScopedSettings {
PluginsService.MANDATORY_SETTING,
BootstrapSettings.SECURITY_FILTER_BAD_DEFAULTS_SETTING,
BootstrapSettings.MEMORY_LOCK_SETTING,
BootstrapSettings.SECCOMP_SETTING,
BootstrapSettings.SYSTEM_CALL_FILTER_SETTING,
BootstrapSettings.CTRLHANDLER_SETTING,
IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING,
IndexingMemoryController.MIN_INDEX_BUFFER_SIZE_SETTING,

View File

@ -817,7 +817,9 @@ public class Setting<T> extends ToXContentToBytes {
@Override
public void apply(Settings value, Settings current, Settings previous) {
logger.info("updating [{}] from [{}] to [{}]", key, getRaw(previous), getRaw(current));
if (logger.isInfoEnabled()) { // getRaw can create quite some objects
logger.info("updating [{}] from [{}] to [{}]", key, getRaw(previous), getRaw(current));
}
consumer.accept(value);
}

View File

@ -42,6 +42,8 @@ import java.io.InputStreamReader;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.AbstractMap;
import java.util.AbstractSet;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
@ -52,16 +54,15 @@ import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.Objects;
import java.util.Set;
import java.util.SortedMap;
import java.util.TreeMap;
import java.util.concurrent.TimeUnit;
import java.util.function.Function;
import java.util.function.Predicate;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import static org.elasticsearch.common.unit.ByteSizeValue.parseBytesSizeValue;
import static org.elasticsearch.common.unit.SizeValue.parseSizeValue;
@ -75,11 +76,10 @@ public final class Settings implements ToXContent {
public static final Settings EMPTY = new Builder().build();
private static final Pattern ARRAY_PATTERN = Pattern.compile("(.*)\\.\\d+$");
private SortedMap<String, String> settings;
private Map<String, String> settings;
Settings(Map<String, String> settings) {
// we use a sorted map for consistent serialization when using getAsMap()
this.settings = Collections.unmodifiableSortedMap(new TreeMap<>(settings));
this.settings = Collections.unmodifiableMap(settings);
}
/**
@ -87,7 +87,8 @@ public final class Settings implements ToXContent {
* @return an unmodifiable map of settings
*/
public Map<String, String> getAsMap() {
return Collections.unmodifiableMap(this.settings);
// settings is always unmodifiable
return this.settings;
}
/**
@ -186,30 +187,14 @@ public final class Settings implements ToXContent {
* A settings that are filtered (and key is removed) with the specified prefix.
*/
public Settings getByPrefix(String prefix) {
Builder builder = new Builder();
for (Map.Entry<String, String> entry : getAsMap().entrySet()) {
if (entry.getKey().startsWith(prefix)) {
if (entry.getKey().length() < prefix.length()) {
// ignore this. one
continue;
}
builder.put(entry.getKey().substring(prefix.length()), entry.getValue());
}
}
return builder.build();
return new Settings(new FilteredMap(this.settings, (k) -> k.startsWith(prefix), prefix));
}
/**
* Returns a new settings object that contains all setting of the current one filtered by the given settings key predicate.
*/
public Settings filter(Predicate<String> predicate) {
Builder builder = new Builder();
for (Map.Entry<String, String> entry : getAsMap().entrySet()) {
if (predicate.test(entry.getKey())) {
builder.put(entry.getKey(), entry.getValue());
}
}
return builder.build();
return new Settings(new FilteredMap(this.settings, predicate, null));
}
/**
@ -443,6 +428,7 @@ public final class Settings implements ToXContent {
}
return getGroupsInternal(settingPrefix, ignoreNonGrouped);
}
private Map<String, Settings> getGroupsInternal(String settingPrefix, boolean ignoreNonGrouped) throws SettingsException {
// we don't really care that it might happen twice
Map<String, Map<String, String>> map = new LinkedHashMap<>();
@ -602,7 +588,8 @@ public final class Settings implements ToXContent {
public static final Settings EMPTY_SETTINGS = new Builder().build();
private final Map<String, String> map = new LinkedHashMap<>();
// we use a sorted map for consistent serialization when using getAsMap()
private final Map<String, String> map = new TreeMap<>();
private Builder() {
@ -1032,7 +1019,124 @@ public final class Settings implements ToXContent {
* set on this builder.
*/
public Settings build() {
return new Settings(Collections.unmodifiableMap(map));
return new Settings(map);
}
}
// TODO We could use an FST internally to make things even faster and more compact
private static final class FilteredMap extends AbstractMap<String, String> {
private final Map<String, String> delegate;
private final Predicate<String> filter;
private final String prefix;
// we cache that size since we have to iterate the entire set
// this is safe to do since this map is only used with unmodifiable maps
private int size = -1;
@Override
public Set<Entry<String, String>> entrySet() {
Set<Entry<String, String>> delegateSet = delegate.entrySet();
AbstractSet<Entry<String, String>> filterSet = new AbstractSet<Entry<String, String>>() {
@Override
public Iterator<Entry<String, String>> iterator() {
Iterator<Entry<String, String>> iter = delegateSet.iterator();
return new Iterator<Entry<String, String>>() {
private int numIterated;
private Entry<String, String> currentElement;
@Override
public boolean hasNext() {
if (currentElement != null) {
return true; // protect against calling hasNext twice
} else {
if (numIterated == size) { // early terminate
assert size != -1 : "size was never set: " + numIterated + " vs. " + size;
return false;
}
while (iter.hasNext()) {
if (filter.test((currentElement = iter.next()).getKey())) {
numIterated++;
return true;
}
}
// we didn't find anything
currentElement = null;
return false;
}
}
@Override
public Entry<String, String> next() {
if (currentElement == null && hasNext() == false) { // protect against no #hasNext call or not respecting it
throw new NoSuchElementException("make sure to call hasNext first");
}
final Entry<String, String> current = this.currentElement;
this.currentElement = null;
if (prefix == null) {
return current;
}
return new Entry<String, String>() {
@Override
public String getKey() {
return current.getKey().substring(prefix.length());
}
@Override
public String getValue() {
return current.getValue();
}
@Override
public String setValue(String value) {
throw new UnsupportedOperationException();
}
};
}
};
}
@Override
public int size() {
return FilteredMap.this.size();
}
};
return filterSet;
}
private FilteredMap(Map<String, String> delegate, Predicate<String> filter, String prefix) {
this.delegate = delegate;
this.filter = filter;
this.prefix = prefix;
}
@Override
public String get(Object key) {
if (key instanceof String) {
final String theKey = prefix == null ? (String)key : prefix + key;
if (filter.test(theKey)) {
return delegate.get(theKey);
}
}
return null;
}
@Override
public boolean containsKey(Object key) {
if (key instanceof String) {
final String theKey = prefix == null ? (String) key : prefix + key;
if (filter.test(theKey)) {
return delegate.containsKey(theKey);
}
}
return false;
}
@Override
public int size() {
if (size == -1) {
size = Math.toIntExact(delegate.keySet().stream().filter((e) -> filter.test(e)).count());
}
return size;
}
}
}

View File

@ -19,6 +19,7 @@
package org.elasticsearch.common.xcontent;
import org.elasticsearch.common.Booleans;
import org.elasticsearch.common.bytes.BytesReference;
import java.io.IOException;
@ -33,6 +34,27 @@ import java.util.Set;
*/
public interface XContent {
/*
* NOTE: This comment is only meant for maintainers of the Elasticsearch code base and is intentionally not a Javadoc comment as it
* describes an undocumented system property.
*
*
* Determines whether the XContent parser will always check for duplicate keys. This behavior is enabled by default but
* can be disabled by setting the otherwise undocumented system property "es.xcontent.strict_duplicate_detection to "false".
*
* Before we've enabled this mode, we had custom duplicate checks in various parts of the code base. As the user can still disable this
* mode and fall back to the legacy duplicate checks, we still need to keep the custom duplicate checks around and we also need to keep
* the tests around.
*
* If this fallback via system property is removed one day in the future you can remove all tests that call this method and also remove
* the corresponding custom duplicate check code.
*
*/
static boolean isStrictDuplicateDetectionEnabled() {
// Don't allow duplicate keys in JSON content by default but let the user opt out
return Booleans.parseBooleanExact(System.getProperty("es.xcontent.strict_duplicate_detection", "true"));
}
/**
* The type this content handles and produces.
*/

View File

@ -141,7 +141,12 @@ public class XContentFactory {
/**
* Guesses the content type based on the provided char sequence.
*
* @deprecated the content type should not be guessed except for few cases where we effectively don't know the content type.
* The REST layer should move to reading the Content-Type header instead. There are other places where auto-detection may be needed.
* This method is deprecated to prevent usages of it from spreading further without specific reasons.
*/
@Deprecated
public static XContentType xContentType(CharSequence content) {
int length = content.length() < GUESS_HEADER_LENGTH ? content.length() : GUESS_HEADER_LENGTH;
if (length == 0) {
@ -174,8 +179,13 @@ public class XContentFactory {
}
/**
* Guesses the content (type) based on the provided char sequence.
* Guesses the content (type) based on the provided char sequence and returns the corresponding {@link XContent}
*
* @deprecated the content type should not be guessed except for few cases where we effectively don't know the content type.
* The REST layer should move to reading the Content-Type header instead. There are other places where auto-detection may be needed.
* This method is deprecated to prevent usages of it from spreading further without specific reasons.
*/
@Deprecated
public static XContent xContent(CharSequence content) {
XContentType type = xContentType(content);
if (type == null) {
@ -185,15 +195,24 @@ public class XContentFactory {
}
/**
* Guesses the content type based on the provided bytes.
* Guesses the content type based on the provided bytes and returns the corresponding {@link XContent}
*
* @deprecated the content type should not be guessed except for few cases where we effectively don't know the content type.
* The REST layer should move to reading the Content-Type header instead. There are other places where auto-detection may be needed.
* This method is deprecated to prevent usages of it from spreading further without specific reasons.
*/
@Deprecated
public static XContent xContent(byte[] data) {
return xContent(data, 0, data.length);
}
/**
* Guesses the content type based on the provided bytes.
* Guesses the content type based on the provided bytes and returns the corresponding {@link XContent}
*
* @deprecated guessing the content type should not be needed ideally. We should rather know the content type upfront or read it
* from headers. Till we fixed the REST layer to read the Content-Type header, that should be the only place where guessing is needed.
*/
@Deprecated
public static XContent xContent(byte[] data, int offset, int length) {
XContentType type = xContentType(data, offset, length);
if (type == null) {
@ -204,14 +223,24 @@ public class XContentFactory {
/**
* Guesses the content type based on the provided bytes.
*
* @deprecated the content type should not be guessed except for few cases where we effectively don't know the content type.
* The REST layer should move to reading the Content-Type header instead. There are other places where auto-detection may be needed.
* This method is deprecated to prevent usages of it from spreading further without specific reasons.
*/
@Deprecated
public static XContentType xContentType(byte[] data) {
return xContentType(data, 0, data.length);
}
/**
* Guesses the content type based on the provided input stream without consuming it.
*
* @deprecated the content type should not be guessed except for few cases where we effectively don't know the content type.
* The REST layer should move to reading the Content-Type header instead. There are other places where auto-detection may be needed.
* This method is deprecated to prevent usages of it from spreading further without specific reasons.
*/
@Deprecated
public static XContentType xContentType(InputStream si) throws IOException {
if (si.markSupported() == false) {
throw new IllegalArgumentException("Cannot guess the xcontent type without mark/reset support on " + si.getClass());
@ -228,11 +257,24 @@ public class XContentFactory {
/**
* Guesses the content type based on the provided bytes.
*
* @deprecated the content type should not be guessed except for few cases where we effectively don't know the content type.
* The REST layer should move to reading the Content-Type header instead. There are other places where auto-detection may be needed.
* This method is deprecated to prevent usages of it from spreading further without specific reasons.
*/
@Deprecated
public static XContentType xContentType(byte[] data, int offset, int length) {
return xContentType(new BytesArray(data, offset, length));
}
/**
* Guesses the content type based on the provided bytes and returns the corresponding {@link XContent}
*
* @deprecated the content type should not be guessed except for few cases where we effectively don't know the content type.
* The REST layer should move to reading the Content-Type header instead. There are other places where auto-detection may be needed.
* This method is deprecated to prevent usages of it from spreading further without specific reasons.
*/
@Deprecated
public static XContent xContent(BytesReference bytes) {
XContentType type = xContentType(bytes);
if (type == null) {
@ -243,7 +285,12 @@ public class XContentFactory {
/**
* Guesses the content type based on the provided bytes.
*
* @deprecated the content type should not be guessed except for few cases where we effectively don't know the content type.
* The REST layer should move to reading the Content-Type header instead. There are other places where auto-detection may be needed.
* This method is deprecated to prevent usages of it from spreading further without specific reasons.
*/
@Deprecated
public static XContentType xContentType(BytesReference bytes) {
int length = bytes.length();
if (length == 0) {

View File

@ -21,6 +21,7 @@ package org.elasticsearch.common.xcontent.cbor;
import com.fasterxml.jackson.core.JsonEncoding;
import com.fasterxml.jackson.core.JsonGenerator;
import com.fasterxml.jackson.core.JsonParser;
import com.fasterxml.jackson.dataformat.cbor.CBORFactory;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.common.bytes.BytesReference;
@ -54,6 +55,7 @@ public class CborXContent implements XContent {
cborFactory.configure(CBORFactory.Feature.FAIL_ON_SYMBOL_HASH_OVERFLOW, false); // this trips on many mappings now...
// Do not automatically close unclosed objects/arrays in com.fasterxml.jackson.dataformat.cbor.CBORGenerator#close() method
cborFactory.configure(JsonGenerator.Feature.AUTO_CLOSE_JSON_CONTENT, false);
cborFactory.configure(JsonParser.Feature.STRICT_DUPLICATE_DETECTION, XContent.isStrictDuplicateDetectionEnabled());
cborXContent = new CborXContent();
}

View File

@ -23,7 +23,6 @@ import com.fasterxml.jackson.core.JsonEncoding;
import com.fasterxml.jackson.core.JsonFactory;
import com.fasterxml.jackson.core.JsonGenerator;
import com.fasterxml.jackson.core.JsonParser;
import org.elasticsearch.common.Booleans;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.FastStringReader;
import org.elasticsearch.common.xcontent.XContent;
@ -50,27 +49,6 @@ public class JsonXContent implements XContent {
public static final JsonXContent jsonXContent;
/*
* NOTE: This comment is only meant for maintainers of the Elasticsearch code base and is intentionally not a Javadoc comment as it
* describes an undocumented system property.
*
*
* Determines whether the JSON parser will always check for duplicate keys in JSON content. This behavior is enabled by default but
* can be disabled by setting the otherwise undocumented system property "es.json.strict_duplicate_detection" to "false".
*
* Before we've enabled this mode, we had custom duplicate checks in various parts of the code base. As the user can still disable this
* mode and fall back to the legacy duplicate checks, we still need to keep the custom duplicate checks around and we also need to keep
* the tests around.
*
* If this fallback via system property is removed one day in the future you can remove all tests that call this method and also remove
* the corresponding custom duplicate check code.
*
*/
public static boolean isStrictDuplicateDetectionEnabled() {
// Don't allow duplicate keys in JSON content by default but let the user opt out
return Booleans.parseBooleanExact(System.getProperty("es.json.strict_duplicate_detection", "true"));
}
static {
jsonFactory = new JsonFactory();
jsonFactory.configure(JsonGenerator.Feature.QUOTE_FIELD_NAMES, true);
@ -78,7 +56,7 @@ public class JsonXContent implements XContent {
jsonFactory.configure(JsonFactory.Feature.FAIL_ON_SYMBOL_HASH_OVERFLOW, false); // this trips on many mappings now...
// Do not automatically close unclosed objects/arrays in com.fasterxml.jackson.core.json.UTF8JsonGenerator#close() method
jsonFactory.configure(JsonGenerator.Feature.AUTO_CLOSE_JSON_CONTENT, false);
jsonFactory.configure(JsonParser.Feature.STRICT_DUPLICATE_DETECTION, isStrictDuplicateDetectionEnabled());
jsonFactory.configure(JsonParser.Feature.STRICT_DUPLICATE_DETECTION, XContent.isStrictDuplicateDetectionEnabled());
jsonXContent = new JsonXContent();
}

View File

@ -21,6 +21,7 @@ package org.elasticsearch.common.xcontent.smile;
import com.fasterxml.jackson.core.JsonEncoding;
import com.fasterxml.jackson.core.JsonGenerator;
import com.fasterxml.jackson.core.JsonParser;
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
import com.fasterxml.jackson.dataformat.smile.SmileGenerator;
import org.elasticsearch.common.bytes.BytesReference;
@ -55,6 +56,7 @@ public class SmileXContent implements XContent {
smileFactory.configure(SmileFactory.Feature.FAIL_ON_SYMBOL_HASH_OVERFLOW, false); // this trips on many mappings now...
// Do not automatically close unclosed objects/arrays in com.fasterxml.jackson.dataformat.smile.SmileGenerator#close() method
smileFactory.configure(JsonGenerator.Feature.AUTO_CLOSE_JSON_CONTENT, false);
smileFactory.configure(JsonParser.Feature.STRICT_DUPLICATE_DETECTION, XContent.isStrictDuplicateDetectionEnabled());
smileXContent = new SmileXContent();
}

View File

@ -20,6 +20,7 @@
package org.elasticsearch.common.xcontent.yaml;
import com.fasterxml.jackson.core.JsonEncoding;
import com.fasterxml.jackson.core.JsonParser;
import com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.common.bytes.BytesReference;
@ -50,6 +51,7 @@ public class YamlXContent implements XContent {
static {
yamlFactory = new YAMLFactory();
yamlFactory.configure(JsonParser.Feature.STRICT_DUPLICATE_DETECTION, XContent.isStrictDuplicateDetectionEnabled());
yamlXContent = new YamlXContent();
}

View File

@ -106,13 +106,6 @@ public class QueryCacheStats implements Streamable, ToXContent {
return cacheCount - cacheSize;
}
public static QueryCacheStats readQueryCacheStats(StreamInput in) throws IOException {
QueryCacheStats stats = new QueryCacheStats();
stats.readFrom(in);
return stats;
}
@Override
public void readFrom(StreamInput in) throws IOException {
ramBytesUsed = in.readLong();

View File

@ -49,13 +49,6 @@ public final class CommitStats implements Streamable, ToXContent {
}
private CommitStats() {
}
public static CommitStats readCommitStatsFrom(StreamInput in) throws IOException {
CommitStats commitStats = new CommitStats();
commitStats.readFrom(in);
return commitStats;
}
public static CommitStats readOptionalCommitStatsFrom(StreamInput in) throws IOException {

View File

@ -78,6 +78,7 @@ import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
@ -86,6 +87,7 @@ import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import java.util.function.Function;
import java.util.function.LongSupplier;
import java.util.function.Supplier;
public class InternalEngine extends Engine {
@ -175,9 +177,18 @@ public class InternalEngine extends Engine {
throw new IllegalArgumentException(openMode.toString());
}
logger.trace("recovered [{}]", seqNoStats);
indexWriter = writer;
seqNoService = sequenceNumberService(shardId, engineConfig.getIndexSettings(), seqNoStats);
translog = openTranslog(engineConfig, writer, seqNoService::getGlobalCheckpoint);
// norelease
/*
* We have no guarantees that all operations above the local checkpoint are in the Lucene commit or the translog. This means
* that we there might be operations greater than the local checkpoint that will not be replayed. Here we force the local
* checkpoint to the maximum sequence number in the commit (at the potential expense of correctness).
*/
while (seqNoService().getLocalCheckpoint() < seqNoService().getMaxSeqNo()) {
seqNoService().markSeqNoAsCompleted(seqNoService().getLocalCheckpoint() + 1);
}
indexWriter = writer;
translog = openTranslog(engineConfig, writer, () -> seqNoService().getGlobalCheckpoint());
assert translog.getGeneration() != null;
} catch (IOException | TranslogCorruptedException e) {
throw new EngineCreationFailureException(shardId, "failed to create engine", e);
@ -412,7 +423,7 @@ public class InternalEngine extends Engine {
@Override
public GetResult get(Get get, Function<String, Searcher> searcherFactory) throws EngineException {
try (ReleasableLock lock = readLock.acquire()) {
try (ReleasableLock ignored = readLock.acquire()) {
ensureOpen();
if (get.realtime()) {
VersionValue versionValue = versionMap.getUnderLock(get.uid());
@ -434,11 +445,28 @@ public class InternalEngine extends Engine {
}
}
private boolean checkVersionConflict(
final Operation op,
final long currentVersion,
final long expectedVersion,
final boolean deleted) {
/**
* Checks for version conflicts. If a version conflict exists, the optional return value represents the operation result. Otherwise, if
* no conflicts are found, the optional return value is not present.
*
* @param <T> the result type
* @param op the operation
* @param currentVersion the current version
* @param expectedVersion the expected version
* @param deleted {@code true} if the current version is not found or represents a delete
* @param onSuccess if there is a version conflict that can be ignored, the result of the operation
* @param onFailure if there is a version conflict that can not be ignored, the result of the operation
* @return if there is a version conflict, the optional value is present and represents the operation result, otherwise the return value
* is not present
*/
private <T extends Result> Optional<T> checkVersionConflict(
final Operation op,
final long currentVersion,
final long expectedVersion,
final boolean deleted,
final Supplier<T> onSuccess,
final Function<VersionConflictEngineException, T> onFailure) {
final T result;
if (op.versionType() == VersionType.FORCE) {
if (engineConfig.getIndexSettings().getIndexVersionCreated().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) {
// If index was created in 5.0 or later, 'force' is not allowed at all
@ -452,14 +480,22 @@ public class InternalEngine extends Engine {
if (op.versionType().isVersionConflictForWrites(currentVersion, expectedVersion, deleted)) {
if (op.origin().isRecovery()) {
// version conflict, but okay
return true;
result = onSuccess.get();
} else {
// fatal version conflict
throw new VersionConflictEngineException(shardId, op.type(), op.id(),
final VersionConflictEngineException e =
new VersionConflictEngineException(
shardId,
op.type(),
op.id(),
op.versionType().explainConflictForWrites(currentVersion, expectedVersion, deleted));
result = onFailure.apply(e);
}
return Optional.of(result);
} else {
return Optional.empty();
}
return false;
}
private long checkDeletedAndGCed(VersionValue versionValue) {
@ -475,7 +511,7 @@ public class InternalEngine extends Engine {
@Override
public IndexResult index(Index index) {
IndexResult result;
try (ReleasableLock lock = readLock.acquire()) {
try (ReleasableLock ignored = readLock.acquire()) {
ensureOpen();
if (index.origin().isRecovery()) {
// Don't throttle recovery operations
@ -573,7 +609,7 @@ public class InternalEngine extends Engine {
assert assertSequenceNumber(index.origin(), index.seqNo());
final Translog.Location location;
final long updatedVersion;
IndexResult indexResult = null;
long seqNo = index.seqNo();
try (Releasable ignored = acquireLock(index.uid())) {
lastWriteNanos = index.startTime();
/* if we have an autoGeneratedID that comes into the engine we can potentially optimize
@ -638,28 +674,33 @@ public class InternalEngine extends Engine {
}
}
final long expectedVersion = index.version();
if (checkVersionConflict(index, currentVersion, expectedVersion, deleted)) {
// skip index operation because of version conflict on recovery
indexResult = new IndexResult(expectedVersion, SequenceNumbersService.UNASSIGNED_SEQ_NO, false);
final Optional<IndexResult> checkVersionConflictResult =
checkVersionConflict(
index,
currentVersion,
expectedVersion,
deleted,
() -> new IndexResult(currentVersion, index.seqNo(), false),
e -> new IndexResult(e, currentVersion, index.seqNo()));
final IndexResult indexResult;
if (checkVersionConflictResult.isPresent()) {
indexResult = checkVersionConflictResult.get();
} else {
final long seqNo;
// no version conflict
if (index.origin() == Operation.Origin.PRIMARY) {
seqNo = seqNoService.generateSeqNo();
} else {
seqNo = index.seqNo();
seqNo = seqNoService().generateSeqNo();
}
/**
* Update the document's sequence number and primary term; the sequence number here is derived here from either the sequence
* number service if this is on the primary, or the existing document's sequence number if this is on the replica. The
* primary term here has already been set, see IndexShard#prepareIndex where the Engine$Index operation is created.
*/
index.parsedDoc().updateSeqID(seqNo, index.primaryTerm());
updatedVersion = index.versionType().updateVersion(currentVersion, expectedVersion);
index.parsedDoc().version().setLongValue(updatedVersion);
// Update the document's sequence number and primary term, the
// sequence number here is derived here from either the sequence
// number service if this is on the primary, or the existing
// document's sequence number if this is on the replica. The
// primary term here has already been set, see
// IndexShard.prepareIndex where the Engine.Index operation is
// created
index.parsedDoc().updateSeqID(seqNo, index.primaryTerm());
if (currentVersion == Versions.NOT_FOUND && forceUpdateDocument == false) {
// document does not exists, we can optimize for create, but double check if assertions are running
assert assertDocDoesNotExist(index, canOptimizeAddDocument == false);
@ -669,8 +710,8 @@ public class InternalEngine extends Engine {
}
indexResult = new IndexResult(updatedVersion, seqNo, deleted);
location = index.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY
? translog.add(new Translog.Index(index, indexResult))
: null;
? translog.add(new Translog.Index(index, indexResult))
: null;
versionMap.putUnderLock(index.uid().bytes(), new VersionValue(updatedVersion));
indexResult.setTranslogLocation(location);
}
@ -678,8 +719,8 @@ public class InternalEngine extends Engine {
indexResult.freeze();
return indexResult;
} finally {
if (indexResult != null && indexResult.getSeqNo() != SequenceNumbersService.UNASSIGNED_SEQ_NO) {
seqNoService.markSeqNoAsCompleted(indexResult.getSeqNo());
if (seqNo != SequenceNumbersService.UNASSIGNED_SEQ_NO) {
seqNoService().markSeqNoAsCompleted(seqNo);
}
}
@ -724,7 +765,7 @@ public class InternalEngine extends Engine {
@Override
public DeleteResult delete(Delete delete) {
DeleteResult result;
try (ReleasableLock lock = readLock.acquire()) {
try (ReleasableLock ignored = readLock.acquire()) {
ensureOpen();
// NOTE: we don't throttle this when merges fall behind because delete-by-id does not create new segments:
result = innerDelete(delete);
@ -748,7 +789,7 @@ public class InternalEngine extends Engine {
final Translog.Location location;
final long updatedVersion;
final boolean found;
DeleteResult deleteResult = null;
long seqNo = delete.seqNo();
try (Releasable ignored = acquireLock(delete.uid())) {
lastWriteNanos = delete.startTime();
final long currentVersion;
@ -764,32 +805,40 @@ public class InternalEngine extends Engine {
}
final long expectedVersion = delete.version();
if (checkVersionConflict(delete, currentVersion, expectedVersion, deleted)) {
// skip executing delete because of version conflict on recovery
deleteResult = new DeleteResult(expectedVersion, SequenceNumbersService.UNASSIGNED_SEQ_NO, true);
final Optional<DeleteResult> result =
checkVersionConflict(
delete,
currentVersion,
expectedVersion,
deleted,
() -> new DeleteResult(expectedVersion, delete.seqNo(), true),
e -> new DeleteResult(e, expectedVersion, delete.seqNo()));
final DeleteResult deleteResult;
if (result.isPresent()) {
deleteResult = result.get();
} else {
final long seqNo;
if (delete.origin() == Operation.Origin.PRIMARY) {
seqNo = seqNoService.generateSeqNo();
} else {
seqNo = delete.seqNo();
seqNo = seqNoService().generateSeqNo();
}
updatedVersion = delete.versionType().updateVersion(currentVersion, expectedVersion);
found = deleteIfFound(delete.uid(), currentVersion, deleted, versionValue);
deleteResult = new DeleteResult(updatedVersion, seqNo, found);
location = delete.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY
? translog.add(new Translog.Delete(delete, deleteResult))
: null;
? translog.add(new Translog.Delete(delete, deleteResult))
: null;
versionMap.putUnderLock(delete.uid().bytes(),
new DeleteVersionValue(updatedVersion, engineConfig.getThreadPool().estimatedTimeInMillis()));
new DeleteVersionValue(updatedVersion, engineConfig.getThreadPool().estimatedTimeInMillis()));
deleteResult.setTranslogLocation(location);
}
deleteResult.setTook(System.nanoTime() - delete.startTime());
deleteResult.freeze();
return deleteResult;
} finally {
if (deleteResult != null && deleteResult.getSeqNo() != SequenceNumbersService.UNASSIGNED_SEQ_NO) {
seqNoService.markSeqNoAsCompleted(deleteResult.getSeqNo());
if (seqNo != SequenceNumbersService.UNASSIGNED_SEQ_NO) {
seqNoService().markSeqNoAsCompleted(seqNo);
}
}
}

View File

@ -286,12 +286,6 @@ public class SegmentsStats implements Streamable, ToXContent {
return maxUnsafeAutoIdTimestamp;
}
public static SegmentsStats readSegmentsStats(StreamInput in) throws IOException {
SegmentsStats stats = new SegmentsStats();
stats.readFrom(in);
return stats;
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(Fields.SEGMENTS);
@ -391,10 +385,9 @@ public class SegmentsStats implements Streamable, ToXContent {
out.writeLong(maxUnsafeAutoIdTimestamp);
out.writeVInt(fileSizes.size());
for (Iterator<ObjectObjectCursor<String, Long>> it = fileSizes.iterator(); it.hasNext();) {
ObjectObjectCursor<String, Long> entry = it.next();
for (ObjectObjectCursor<String, Long> entry : fileSizes) {
out.writeString(entry.key);
out.writeLong(entry.value);
out.writeLong(entry.value.longValue());
}
}
}

View File

@ -19,7 +19,7 @@
package org.elasticsearch.index.fielddata;
import com.carrotsearch.hppc.ObjectLongHashMap;
import org.elasticsearch.common.FieldMemoryStats;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
@ -29,19 +29,25 @@ import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import java.io.IOException;
import java.util.Objects;
public class FieldDataStats implements Streamable, ToXContent {
private static final String FIELDDATA = "fielddata";
private static final String MEMORY_SIZE = "memory_size";
private static final String MEMORY_SIZE_IN_BYTES = "memory_size_in_bytes";
private static final String EVICTIONS = "evictions";
private static final String FIELDS = "fields";
long memorySize;
long evictions;
@Nullable
ObjectLongHashMap<String> fields;
FieldMemoryStats fields;
public FieldDataStats() {
}
public FieldDataStats(long memorySize, long evictions, @Nullable ObjectLongHashMap<String> fields) {
public FieldDataStats(long memorySize, long evictions, @Nullable FieldMemoryStats fields) {
this.memorySize = memorySize;
this.evictions = evictions;
this.fields = fields;
@ -52,16 +58,9 @@ public class FieldDataStats implements Streamable, ToXContent {
this.evictions += stats.evictions;
if (stats.fields != null) {
if (fields == null) {
fields = stats.fields.clone();
fields = stats.fields.copy();
} else {
assert !stats.fields.containsKey(null);
final Object[] keys = stats.fields.keys;
final long[] values = stats.fields.values;
for (int i = 0; i < keys.length; i++) {
if (keys[i] != null) {
fields.addTo((String) keys[i], values[i]);
}
}
fields.add(stats.fields);
}
}
}
@ -79,78 +78,48 @@ public class FieldDataStats implements Streamable, ToXContent {
}
@Nullable
public ObjectLongHashMap<String> getFields() {
public FieldMemoryStats getFields() {
return fields;
}
public static FieldDataStats readFieldDataStats(StreamInput in) throws IOException {
FieldDataStats stats = new FieldDataStats();
stats.readFrom(in);
return stats;
}
@Override
public void readFrom(StreamInput in) throws IOException {
memorySize = in.readVLong();
evictions = in.readVLong();
if (in.readBoolean()) {
int size = in.readVInt();
fields = new ObjectLongHashMap<>(size);
for (int i = 0; i < size; i++) {
fields.put(in.readString(), in.readVLong());
}
}
fields = in.readOptionalWriteable(FieldMemoryStats::new);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeVLong(memorySize);
out.writeVLong(evictions);
if (fields == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
out.writeVInt(fields.size());
assert !fields.containsKey(null);
final Object[] keys = fields.keys;
final long[] values = fields.values;
for (int i = 0; i < keys.length; i++) {
if (keys[i] != null) {
out.writeString((String) keys[i]);
out.writeVLong(values[i]);
}
}
}
out.writeOptionalWriteable(fields);
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(Fields.FIELDDATA);
builder.byteSizeField(Fields.MEMORY_SIZE_IN_BYTES, Fields.MEMORY_SIZE, memorySize);
builder.field(Fields.EVICTIONS, getEvictions());
builder.startObject(FIELDDATA);
builder.byteSizeField(MEMORY_SIZE_IN_BYTES, MEMORY_SIZE, memorySize);
builder.field(EVICTIONS, getEvictions());
if (fields != null) {
builder.startObject(Fields.FIELDS);
assert !fields.containsKey(null);
final Object[] keys = fields.keys;
final long[] values = fields.values;
for (int i = 0; i < keys.length; i++) {
if (keys[i] != null) {
builder.startObject((String) keys[i]);
builder.byteSizeField(Fields.MEMORY_SIZE_IN_BYTES, Fields.MEMORY_SIZE, values[i]);
builder.endObject();
}
}
builder.endObject();
fields.toXContent(builder, FIELDS, MEMORY_SIZE_IN_BYTES, MEMORY_SIZE);
}
builder.endObject();
return builder;
}
static final class Fields {
static final String FIELDDATA = "fielddata";
static final String MEMORY_SIZE = "memory_size";
static final String MEMORY_SIZE_IN_BYTES = "memory_size_in_bytes";
static final String EVICTIONS = "evictions";
static final String FIELDS = "fields";
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
FieldDataStats that = (FieldDataStats) o;
return memorySize == that.memorySize &&
evictions == that.evictions &&
Objects.equals(fields, that.fields);
}
@Override
public int hashCode() {
return Objects.hash(memorySize, evictions, fields);
}
}

View File

@ -21,6 +21,7 @@ package org.elasticsearch.index.fielddata;
import com.carrotsearch.hppc.ObjectLongHashMap;
import org.apache.lucene.util.Accountable;
import org.elasticsearch.common.FieldMemoryStats;
import org.elasticsearch.common.metrics.CounterMetric;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
@ -45,7 +46,8 @@ public class ShardFieldData implements IndexFieldDataCache.Listener {
}
}
}
return new FieldDataStats(totalMetric.count(), evictionsMetric.count(), fieldTotals);
return new FieldDataStats(totalMetric.count(), evictionsMetric.count(), fieldTotals == null ? null :
new FieldMemoryStats(fieldTotals));
}
@Override

View File

@ -81,12 +81,6 @@ public class FlushStats implements Streamable, ToXContent {
return new TimeValue(totalTimeInMillis);
}
public static FlushStats readFlushStats(StreamInput in) throws IOException {
FlushStats flushStats = new FlushStats();
flushStats.readFrom(in);
return flushStats;
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(Fields.FLUSH);

View File

@ -134,12 +134,6 @@ public class GetStats implements Streamable, ToXContent {
static final String CURRENT = "current";
}
public static GetStats readGetStats(StreamInput in) throws IOException {
GetStats stats = new GetStats();
stats.readFrom(in);
return stats;
}
@Override
public void readFrom(StreamInput in) throws IOException {
existsCount = in.readVLong();

View File

@ -34,6 +34,7 @@ import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.similarity.SimilarityService;
import java.io.IOException;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@ -100,7 +101,7 @@ public class AllFieldMapper extends MetadataFieldMapper {
public static class TypeParser implements MetadataFieldMapper.TypeParser {
@Override
public MetadataFieldMapper.Builder parse(String name, Map<String, Object> node,
public MetadataFieldMapper.Builder<?,?> parse(String name, Map<String, Object> node,
ParserContext parserContext) throws MapperParsingException {
Builder builder = new Builder(parserContext.mapperService().fullName(NAME));
builder.fieldType().setIndexAnalyzer(parserContext.getIndexAnalyzers().getDefaultIndexAnalyzer());
@ -141,8 +142,14 @@ public class AllFieldMapper extends MetadataFieldMapper {
}
@Override
public MetadataFieldMapper getDefault(Settings indexSettings, MappedFieldType fieldType, String typeName) {
return new AllFieldMapper(indexSettings, fieldType);
public MetadataFieldMapper getDefault(MappedFieldType fieldType, ParserContext context) {
final Settings indexSettings = context.mapperService().getIndexSettings().getSettings();
if (fieldType != null) {
return new AllFieldMapper(indexSettings, fieldType);
} else {
return parse(NAME, Collections.emptyMap(), context)
.build(new BuilderContext(indexSettings, new ContentPath(1)));
}
}
}
@ -179,7 +186,7 @@ public class AllFieldMapper extends MetadataFieldMapper {
private EnabledAttributeMapper enabledState;
private AllFieldMapper(Settings indexSettings, MappedFieldType existing) {
this(existing == null ? Defaults.FIELD_TYPE.clone() : existing.clone(), Defaults.ENABLED, indexSettings);
this(existing.clone(), Defaults.ENABLED, indexSettings);
}
private AllFieldMapper(MappedFieldType fieldType, EnabledAttributeMapper enabled, Settings indexSettings) {

View File

@ -74,7 +74,8 @@ public class DocumentMapper implements ToXContent {
final MetadataFieldMapper metadataMapper;
if (existingMetadataMapper == null) {
final TypeParser parser = entry.getValue();
metadataMapper = parser.getDefault(indexSettings, mapperService.fullName(name), builder.name());
metadataMapper = parser.getDefault(mapperService.fullName(name),
mapperService.documentMapperParser().parserContext(builder.name()));
} else {
metadataMapper = existingMetadataMapper;
}

View File

@ -30,6 +30,7 @@ import org.elasticsearch.index.query.QueryShardContext;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@ -98,7 +99,7 @@ public class FieldNamesFieldMapper extends MetadataFieldMapper {
public static class TypeParser implements MetadataFieldMapper.TypeParser {
@Override
public MetadataFieldMapper.Builder parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
public MetadataFieldMapper.Builder<?,?> parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
Builder builder = new Builder(parserContext.mapperService().fullName(NAME));
for (Iterator<Map.Entry<String, Object>> iterator = node.entrySet().iterator(); iterator.hasNext();) {
@ -114,8 +115,14 @@ public class FieldNamesFieldMapper extends MetadataFieldMapper {
}
@Override
public MetadataFieldMapper getDefault(Settings indexSettings, MappedFieldType fieldType, String typeName) {
return new FieldNamesFieldMapper(indexSettings, fieldType);
public MetadataFieldMapper getDefault(MappedFieldType fieldType, ParserContext context) {
final Settings indexSettings = context.mapperService().getIndexSettings().getSettings();
if (fieldType != null) {
return new FieldNamesFieldMapper(indexSettings, fieldType);
} else {
return parse(NAME, Collections.emptyMap(), context)
.build(new BuilderContext(indexSettings, new ContentPath(1)));
}
}
}
@ -183,7 +190,7 @@ public class FieldNamesFieldMapper extends MetadataFieldMapper {
}
private FieldNamesFieldMapper(Settings indexSettings, MappedFieldType existing) {
this(existing == null ? Defaults.FIELD_TYPE.clone() : existing.clone(), indexSettings);
this(existing.clone(), indexSettings);
}
private FieldNamesFieldMapper(MappedFieldType fieldType, Settings indexSettings) {

View File

@ -37,6 +37,7 @@ import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.iterable.Iterables;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.mapper.Mapper.TypeParser.ParserContext;
import org.elasticsearch.index.query.QueryShardContext;
import java.io.IOException;
@ -79,7 +80,8 @@ public class IdFieldMapper extends MetadataFieldMapper {
}
@Override
public MetadataFieldMapper getDefault(Settings indexSettings, MappedFieldType fieldType, String typeName) {
public MetadataFieldMapper getDefault(MappedFieldType fieldType, ParserContext context) {
final Settings indexSettings = context.mapperService().getIndexSettings().getSettings();
return new IdFieldMapper(indexSettings, fieldType);
}
}

View File

@ -19,7 +19,6 @@
package org.elasticsearch.index.mapper;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.search.Query;
@ -85,7 +84,8 @@ public class IndexFieldMapper extends MetadataFieldMapper {
}
@Override
public MetadataFieldMapper getDefault(Settings indexSettings, MappedFieldType fieldType, String typeName) {
public MetadataFieldMapper getDefault(MappedFieldType fieldType, ParserContext context) {
final Settings indexSettings = context.mapperService().getIndexSettings().getSettings();
return new IndexFieldMapper(indexSettings, fieldType);
}
}

View File

@ -28,6 +28,7 @@ import org.elasticsearch.ElasticsearchGenerationException;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MappingMetaData;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.settings.Setting;
@ -51,6 +52,7 @@ import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -61,7 +63,6 @@ import java.util.stream.Collectors;
import static java.util.Collections.emptyMap;
import static java.util.Collections.emptySet;
import static java.util.Collections.unmodifiableMap;
import static org.elasticsearch.common.collect.MapBuilder.newMapBuilder;
public class MapperService extends AbstractIndexComponent implements Closeable {
@ -153,7 +154,7 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
}
/**
* Returns true if the "_all" field is enabled for the type
* Returns true if the "_all" field is enabled on any type.
*/
public boolean allEnabled() {
return this.allEnabled;
@ -191,153 +192,235 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
}
}
/**
* Update mapping by only merging the metadata that is different between received and stored entries
*/
public boolean updateMapping(IndexMetaData indexMetaData) throws IOException {
assert indexMetaData.getIndex().equals(index()) : "index mismatch: expected " + index() + " but was " + indexMetaData.getIndex();
// go over and add the relevant mappings (or update them)
final Set<String> existingMappers = new HashSet<>(mappers.keySet());
final Map<String, DocumentMapper> updatedEntries;
try {
// only update entries if needed
updatedEntries = internalMerge(indexMetaData, MergeReason.MAPPING_RECOVERY, true, true);
} catch (Exception e) {
logger.warn((org.apache.logging.log4j.util.Supplier<?>) () -> new ParameterizedMessage("[{}] failed to apply mappings", index()), e);
throw e;
}
boolean requireRefresh = false;
for (ObjectCursor<MappingMetaData> cursor : indexMetaData.getMappings().values()) {
MappingMetaData mappingMd = cursor.value;
String mappingType = mappingMd.type();
CompressedXContent mappingSource = mappingMd.source();
for (DocumentMapper documentMapper : updatedEntries.values()) {
String mappingType = documentMapper.type();
CompressedXContent incomingMappingSource = indexMetaData.mapping(mappingType).source();
String op = existingMappers.contains(mappingType) ? "updated" : "added";
if (logger.isDebugEnabled() && incomingMappingSource.compressed().length < 512) {
logger.debug("[{}] {} mapping [{}], source [{}]", index(), op, mappingType, incomingMappingSource.string());
} else if (logger.isTraceEnabled()) {
logger.trace("[{}] {} mapping [{}], source [{}]", index(), op, mappingType, incomingMappingSource.string());
} else {
logger.debug("[{}] {} mapping [{}] (source suppressed due to length, use TRACE level if needed)", index(), op, mappingType);
}
// refresh mapping can happen when the parsing/merging of the mapping from the metadata doesn't result in the same
// mapping, in this case, we send to the master to refresh its own version of the mappings (to conform with the
// merge version of it, which it does when refreshing the mappings), and warn log it.
try {
DocumentMapper existingMapper = documentMapper(mappingType);
if (documentMapper(mappingType).mappingSource().equals(incomingMappingSource) == false) {
logger.debug("[{}] parsed mapping [{}], and got different sources\noriginal:\n{}\nparsed:\n{}", index(), mappingType,
incomingMappingSource, documentMapper(mappingType).mappingSource());
if (existingMapper == null || mappingSource.equals(existingMapper.mappingSource()) == false) {
String op = existingMapper == null ? "adding" : "updating";
if (logger.isDebugEnabled() && mappingSource.compressed().length < 512) {
logger.debug("[{}] {} mapping [{}], source [{}]", index(), op, mappingType, mappingSource.string());
} else if (logger.isTraceEnabled()) {
logger.trace("[{}] {} mapping [{}], source [{}]", index(), op, mappingType, mappingSource.string());
} else {
logger.debug("[{}] {} mapping [{}] (source suppressed due to length, use TRACE level if needed)", index(), op,
mappingType);
}
merge(mappingType, mappingSource, MergeReason.MAPPING_RECOVERY, true);
if (!documentMapper(mappingType).mappingSource().equals(mappingSource)) {
logger.debug("[{}] parsed mapping [{}], and got different sources\noriginal:\n{}\nparsed:\n{}", index(),
mappingType, mappingSource, documentMapper(mappingType).mappingSource());
requireRefresh = true;
}
}
} catch (Exception e) {
logger.warn(
(org.apache.logging.log4j.util.Supplier<?>)
() -> new ParameterizedMessage("[{}] failed to add mapping [{}], source [{}]", index(), mappingType, mappingSource),
e);
throw e;
requireRefresh = true;
}
}
return requireRefresh;
}
//TODO: make this atomic
public void merge(Map<String, Map<String, Object>> mappings, boolean updateAllTypes) throws MapperParsingException {
// first, add the default mapping
if (mappings.containsKey(DEFAULT_MAPPING)) {
try {
this.merge(DEFAULT_MAPPING, new CompressedXContent(XContentFactory.jsonBuilder().map(mappings.get(DEFAULT_MAPPING)).string()), MergeReason.MAPPING_UPDATE, updateAllTypes);
} catch (Exception e) {
throw new MapperParsingException("Failed to parse mapping [{}]: {}", e, DEFAULT_MAPPING, e.getMessage());
}
}
public void merge(Map<String, Map<String, Object>> mappings, MergeReason reason, boolean updateAllTypes) {
Map<String, CompressedXContent> mappingSourcesCompressed = new LinkedHashMap<>(mappings.size());
for (Map.Entry<String, Map<String, Object>> entry : mappings.entrySet()) {
if (entry.getKey().equals(DEFAULT_MAPPING)) {
continue;
}
try {
// apply the default here, its the first time we parse it
this.merge(entry.getKey(), new CompressedXContent(XContentFactory.jsonBuilder().map(entry.getValue()).string()), MergeReason.MAPPING_UPDATE, updateAllTypes);
mappingSourcesCompressed.put(entry.getKey(), new CompressedXContent(XContentFactory.jsonBuilder().map(entry.getValue()).string()));
} catch (Exception e) {
throw new MapperParsingException("Failed to parse mapping [{}]: {}", e, entry.getKey(), e.getMessage());
}
}
internalMerge(mappingSourcesCompressed, reason, updateAllTypes);
}
public void merge(IndexMetaData indexMetaData, MergeReason reason, boolean updateAllTypes) {
internalMerge(indexMetaData, reason, updateAllTypes, false);
}
public DocumentMapper merge(String type, CompressedXContent mappingSource, MergeReason reason, boolean updateAllTypes) {
if (DEFAULT_MAPPING.equals(type)) {
return internalMerge(Collections.singletonMap(type, mappingSource), reason, updateAllTypes).get(type);
}
private synchronized Map<String, DocumentMapper> internalMerge(IndexMetaData indexMetaData, MergeReason reason, boolean updateAllTypes,
boolean onlyUpdateIfNeeded) {
Map<String, CompressedXContent> map = new LinkedHashMap<>();
for (ObjectCursor<MappingMetaData> cursor : indexMetaData.getMappings().values()) {
MappingMetaData mappingMetaData = cursor.value;
if (onlyUpdateIfNeeded) {
DocumentMapper existingMapper = documentMapper(mappingMetaData.type());
if (existingMapper == null || mappingMetaData.source().equals(existingMapper.mappingSource()) == false) {
map.put(mappingMetaData.type(), mappingMetaData.source());
}
} else {
map.put(mappingMetaData.type(), mappingMetaData.source());
}
}
return internalMerge(map, reason, updateAllTypes);
}
private synchronized Map<String, DocumentMapper> internalMerge(Map<String, CompressedXContent> mappings, MergeReason reason, boolean updateAllTypes) {
DocumentMapper defaultMapper = null;
String defaultMappingSource = null;
if (mappings.containsKey(DEFAULT_MAPPING)) {
// verify we can parse it
// NOTE: never apply the default here
DocumentMapper mapper = documentParser.parse(type, mappingSource);
// still add it as a document mapper so we have it registered and, for example, persisted back into
// the cluster meta data if needed, or checked for existence
synchronized (this) {
mappers = newMapBuilder(mappers).put(type, mapper).map();
try {
defaultMapper = documentParser.parse(DEFAULT_MAPPING, mappings.get(DEFAULT_MAPPING));
} catch (Exception e) {
throw new MapperParsingException("Failed to parse mapping [{}]: {}", e, DEFAULT_MAPPING, e.getMessage());
}
try {
defaultMappingSource = mappingSource.string();
defaultMappingSource = mappings.get(DEFAULT_MAPPING).string();
} catch (IOException e) {
throw new ElasticsearchGenerationException("failed to un-compress", e);
}
return mapper;
}
final String defaultMappingSourceOrLastStored;
if (defaultMappingSource != null) {
defaultMappingSourceOrLastStored = defaultMappingSource;
} else {
synchronized (this) {
final boolean applyDefault =
// the default was already applied if we are recovering
reason != MergeReason.MAPPING_RECOVERY
// only apply the default mapping if we don't have the type yet
&& mappers.containsKey(type) == false;
DocumentMapper mergeWith = parse(type, mappingSource, applyDefault);
return merge(mergeWith, reason, updateAllTypes);
defaultMappingSourceOrLastStored = this.defaultMappingSource;
}
List<DocumentMapper> documentMappers = new ArrayList<>();
for (Map.Entry<String, CompressedXContent> entry : mappings.entrySet()) {
String type = entry.getKey();
if (type.equals(DEFAULT_MAPPING)) {
continue;
}
final boolean applyDefault =
// the default was already applied if we are recovering
reason != MergeReason.MAPPING_RECOVERY
// only apply the default mapping if we don't have the type yet
&& mappers.containsKey(type) == false;
try {
DocumentMapper documentMapper = documentParser.parse(type, entry.getValue(), applyDefault ? defaultMappingSourceOrLastStored : null);
documentMappers.add(documentMapper);
} catch (Exception e) {
throw new MapperParsingException("Failed to parse mapping [{}]: {}", e, entry.getKey(), e.getMessage());
}
}
return internalMerge(defaultMapper, defaultMappingSource, documentMappers, reason, updateAllTypes);
}
private synchronized DocumentMapper merge(DocumentMapper mapper, MergeReason reason, boolean updateAllTypes) {
if (mapper.type().length() == 0) {
throw new InvalidTypeNameException("mapping type name is empty");
}
if (mapper.type().length() > 255) {
throw new InvalidTypeNameException("mapping type name [" + mapper.type() + "] is too long; limit is length 255 but was [" + mapper.type().length() + "]");
}
if (mapper.type().charAt(0) == '_') {
throw new InvalidTypeNameException("mapping type name [" + mapper.type() + "] can't start with '_'");
}
if (mapper.type().contains("#")) {
throw new InvalidTypeNameException("mapping type name [" + mapper.type() + "] should not include '#' in it");
}
if (mapper.type().contains(",")) {
throw new InvalidTypeNameException("mapping type name [" + mapper.type() + "] should not include ',' in it");
}
if (mapper.type().equals(mapper.parentFieldMapper().type())) {
throw new IllegalArgumentException("The [_parent.type] option can't point to the same type");
}
if (typeNameStartsWithIllegalDot(mapper)) {
throw new IllegalArgumentException("mapping type name [" + mapper.type() + "] must not start with a '.'");
}
// 1. compute the merged DocumentMapper
DocumentMapper oldMapper = mappers.get(mapper.type());
DocumentMapper newMapper;
if (oldMapper != null) {
newMapper = oldMapper.merge(mapper.mapping(), updateAllTypes);
} else {
newMapper = mapper;
}
// 2. check basic sanity of the new mapping
List<ObjectMapper> objectMappers = new ArrayList<>();
List<FieldMapper> fieldMappers = new ArrayList<>();
Collections.addAll(fieldMappers, newMapper.mapping().metadataMappers);
MapperUtils.collect(newMapper.mapping().root(), objectMappers, fieldMappers);
checkFieldUniqueness(newMapper.type(), objectMappers, fieldMappers);
checkObjectsCompatibility(objectMappers, updateAllTypes);
// 3. update lookup data-structures
// this will in particular make sure that the merged fields are compatible with other types
FieldTypeLookup fieldTypes = this.fieldTypes.copyAndAddAll(newMapper.type(), fieldMappers, updateAllTypes);
private synchronized Map<String, DocumentMapper> internalMerge(@Nullable DocumentMapper defaultMapper, @Nullable String defaultMappingSource,
List<DocumentMapper> documentMappers, MergeReason reason, boolean updateAllTypes) {
boolean hasNested = this.hasNested;
Map<String, ObjectMapper> fullPathObjectMappers = new HashMap<>(this.fullPathObjectMappers);
for (ObjectMapper objectMapper : objectMappers) {
fullPathObjectMappers.put(objectMapper.fullPath(), objectMapper);
if (objectMapper.nested().isNested()) {
hasNested = true;
}
boolean allEnabled = this.allEnabled;
Map<String, ObjectMapper> fullPathObjectMappers = this.fullPathObjectMappers;
FieldTypeLookup fieldTypes = this.fieldTypes;
Set<String> parentTypes = this.parentTypes;
Map<String, DocumentMapper> mappers = new HashMap<>(this.mappers);
Map<String, DocumentMapper> results = new LinkedHashMap<>(documentMappers.size() + 1);
if (defaultMapper != null) {
assert defaultMapper.type().equals(DEFAULT_MAPPING);
mappers.put(DEFAULT_MAPPING, defaultMapper);
results.put(DEFAULT_MAPPING, defaultMapper);
}
for (DocumentMapper mapper : documentMappers) {
// check naming
if (mapper.type().length() == 0) {
throw new InvalidTypeNameException("mapping type name is empty");
}
if (mapper.type().length() > 255) {
throw new InvalidTypeNameException("mapping type name [" + mapper.type() + "] is too long; limit is length 255 but was [" + mapper.type().length() + "]");
}
if (mapper.type().charAt(0) == '_') {
throw new InvalidTypeNameException("mapping type name [" + mapper.type() + "] can't start with '_'");
}
if (mapper.type().contains("#")) {
throw new InvalidTypeNameException("mapping type name [" + mapper.type() + "] should not include '#' in it");
}
if (mapper.type().contains(",")) {
throw new InvalidTypeNameException("mapping type name [" + mapper.type() + "] should not include ',' in it");
}
if (mapper.type().equals(mapper.parentFieldMapper().type())) {
throw new IllegalArgumentException("The [_parent.type] option can't point to the same type");
}
if (typeNameStartsWithIllegalDot(mapper)) {
throw new IllegalArgumentException("mapping type name [" + mapper.type() + "] must not start with a '.'");
}
// compute the merged DocumentMapper
DocumentMapper oldMapper = mappers.get(mapper.type());
DocumentMapper newMapper;
if (oldMapper != null) {
newMapper = oldMapper.merge(mapper.mapping(), updateAllTypes);
} else {
newMapper = mapper;
}
// check basic sanity of the new mapping
List<ObjectMapper> objectMappers = new ArrayList<>();
List<FieldMapper> fieldMappers = new ArrayList<>();
Collections.addAll(fieldMappers, newMapper.mapping().metadataMappers);
MapperUtils.collect(newMapper.mapping().root(), objectMappers, fieldMappers);
checkFieldUniqueness(newMapper.type(), objectMappers, fieldMappers, fullPathObjectMappers, fieldTypes);
checkObjectsCompatibility(objectMappers, updateAllTypes, fullPathObjectMappers);
// update lookup data-structures
// this will in particular make sure that the merged fields are compatible with other types
fieldTypes = fieldTypes.copyAndAddAll(newMapper.type(), fieldMappers, updateAllTypes);
for (ObjectMapper objectMapper : objectMappers) {
if (fullPathObjectMappers == this.fullPathObjectMappers) {
fullPathObjectMappers = new HashMap<>(this.fullPathObjectMappers);
}
fullPathObjectMappers.put(objectMapper.fullPath(), objectMapper);
if (objectMapper.nested().isNested()) {
hasNested = true;
}
}
if (reason == MergeReason.MAPPING_UPDATE) {
// this check will only be performed on the master node when there is
// a call to the update mapping API. For all other cases like
// the master node restoring mappings from disk or data nodes
// deserializing cluster state that was sent by the master node,
// this check will be skipped.
checkTotalFieldsLimit(objectMappers.size() + fieldMappers.size());
}
if (oldMapper == null && newMapper.parentFieldMapper().active()) {
if (parentTypes == this.parentTypes) {
parentTypes = new HashSet<>(this.parentTypes);
}
parentTypes.add(mapper.parentFieldMapper().type());
}
// this is only correct because types cannot be removed and we do not
// allow to disable an existing _all field
allEnabled |= mapper.allFieldMapper().enabled();
results.put(newMapper.type(), newMapper);
mappers.put(newMapper.type(), newMapper);
}
fullPathObjectMappers = Collections.unmodifiableMap(fullPathObjectMappers);
if (reason == MergeReason.MAPPING_UPDATE) {
// this check will only be performed on the master node when there is
@ -346,43 +429,46 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
// deserializing cluster state that was sent by the master node,
// this check will be skipped.
checkNestedFieldsLimit(fullPathObjectMappers);
checkTotalFieldsLimit(objectMappers.size() + fieldMappers.size());
checkDepthLimit(fullPathObjectMappers.keySet());
}
Set<String> parentTypes = this.parentTypes;
if (oldMapper == null && newMapper.parentFieldMapper().active()) {
parentTypes = new HashSet<>(parentTypes.size() + 1);
parentTypes.addAll(this.parentTypes);
parentTypes.add(mapper.parentFieldMapper().type());
parentTypes = Collections.unmodifiableSet(parentTypes);
}
Map<String, DocumentMapper> mappers = new HashMap<>(this.mappers);
mappers.put(newMapper.type(), newMapper);
for (Map.Entry<String, DocumentMapper> entry : mappers.entrySet()) {
if (entry.getKey().equals(DEFAULT_MAPPING)) {
continue;
}
DocumentMapper m = entry.getValue();
DocumentMapper documentMapper = entry.getValue();
// apply changes to the field types back
m = m.updateFieldType(fieldTypes.fullNameToFieldType);
entry.setValue(m);
DocumentMapper updatedDocumentMapper = documentMapper.updateFieldType(fieldTypes.fullNameToFieldType);
if (updatedDocumentMapper != documentMapper) {
// update both mappers and result
entry.setValue(updatedDocumentMapper);
if (results.containsKey(updatedDocumentMapper.type())) {
results.put(updatedDocumentMapper.type(), updatedDocumentMapper);
}
}
}
mappers = Collections.unmodifiableMap(mappers);
// 4. commit the change
// make structures immutable
mappers = Collections.unmodifiableMap(mappers);
results = Collections.unmodifiableMap(results);
parentTypes = Collections.unmodifiableSet(parentTypes);
fullPathObjectMappers = Collections.unmodifiableMap(fullPathObjectMappers);
// commit the change
if (defaultMappingSource != null) {
this.defaultMappingSource = defaultMappingSource;
}
this.mappers = mappers;
this.fieldTypes = fieldTypes;
this.hasNested = hasNested;
this.fullPathObjectMappers = fullPathObjectMappers;
this.parentTypes = parentTypes;
this.allEnabled = mapper.allFieldMapper().enabled();
this.allEnabled = allEnabled;
assert assertSerialization(newMapper);
assert assertMappersShareSameFieldType();
assert results.values().stream().allMatch(this::assertSerialization);
return newMapper;
return results;
}
private boolean assertMappersShareSameFieldType() {
@ -419,8 +505,8 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
return true;
}
private void checkFieldUniqueness(String type, Collection<ObjectMapper> objectMappers, Collection<FieldMapper> fieldMappers) {
assert Thread.holdsLock(this);
private static void checkFieldUniqueness(String type, Collection<ObjectMapper> objectMappers, Collection<FieldMapper> fieldMappers,
Map<String, ObjectMapper> fullPathObjectMappers, FieldTypeLookup fieldTypes) {
// first check within mapping
final Set<String> objectFullNames = new HashSet<>();
@ -457,9 +543,8 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
}
}
private void checkObjectsCompatibility(Collection<ObjectMapper> objectMappers, boolean updateAllTypes) {
assert Thread.holdsLock(this);
private static void checkObjectsCompatibility(Collection<ObjectMapper> objectMappers, boolean updateAllTypes,
Map<String, ObjectMapper> fullPathObjectMappers) {
for (ObjectMapper newObjectMapper : objectMappers) {
ObjectMapper existingObjectMapper = fullPathObjectMappers.get(newObjectMapper.fullPath());
if (existingObjectMapper != null) {

View File

@ -39,14 +39,13 @@ public abstract class MetadataFieldMapper extends FieldMapper {
* Get the default {@link MetadataFieldMapper} to use, if nothing had to be parsed.
* @param fieldType null if this is the first root mapper on this index, the existing
* fieldType for this index otherwise
* @param indexSettings the index-level settings
* @param fieldType the existing field type for this meta mapper on the current index
* or null if this is the first type being introduced
* @param typeName the name of the type that this mapper will be used on
* @param parserContext context that may be useful to build the field like analyzers
*/
// TODO: remove the fieldType parameter which is only used for bw compat with pre-2.0
// since settings could be modified
MetadataFieldMapper getDefault(Settings indexSettings, MappedFieldType fieldType, String typeName);
MetadataFieldMapper getDefault(MappedFieldType fieldType, ParserContext parserContext);
}
public abstract static class Builder<T extends Builder, Y extends MetadataFieldMapper> extends FieldMapper.Builder<T, Y> {

View File

@ -18,7 +18,6 @@
*/
package org.elasticsearch.index.mapper;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.SortedDocValuesField;
import org.apache.lucene.index.DocValuesType;
import org.apache.lucene.index.IndexOptions;
@ -131,7 +130,9 @@ public class ParentFieldMapper extends MetadataFieldMapper {
}
@Override
public MetadataFieldMapper getDefault(Settings indexSettings, MappedFieldType fieldType, String typeName) {
public MetadataFieldMapper getDefault(MappedFieldType fieldType, ParserContext context) {
final Settings indexSettings = context.mapperService().getIndexSettings().getSettings();
final String typeName = context.type();
KeywordFieldMapper parentJoinField = createParentJoinFieldMapper(typeName, new BuilderContext(indexSettings, new ContentPath(0)));
MappedFieldType childJoinFieldType = new ParentFieldType(Defaults.FIELD_TYPE, typeName);
childJoinFieldType.setName(ParentFieldMapper.NAME);

View File

@ -27,6 +27,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import java.io.IOException;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@ -78,7 +79,7 @@ public class RoutingFieldMapper extends MetadataFieldMapper {
public static class TypeParser implements MetadataFieldMapper.TypeParser {
@Override
public MetadataFieldMapper.Builder parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
public MetadataFieldMapper.Builder<?,?> parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
Builder builder = new Builder(parserContext.mapperService().fullName(NAME));
for (Iterator<Map.Entry<String, Object>> iterator = node.entrySet().iterator(); iterator.hasNext();) {
Map.Entry<String, Object> entry = iterator.next();
@ -93,8 +94,14 @@ public class RoutingFieldMapper extends MetadataFieldMapper {
}
@Override
public MetadataFieldMapper getDefault(Settings indexSettings, MappedFieldType fieldType, String typeName) {
return new RoutingFieldMapper(indexSettings, fieldType);
public MetadataFieldMapper getDefault(MappedFieldType fieldType, ParserContext context) {
final Settings indexSettings = context.mapperService().getIndexSettings().getSettings();
if (fieldType != null) {
return new RoutingFieldMapper(indexSettings, fieldType);
} else {
return parse(NAME, Collections.emptyMap(), context)
.build(new BuilderContext(indexSettings, new ContentPath(1)));
}
}
}
@ -121,7 +128,7 @@ public class RoutingFieldMapper extends MetadataFieldMapper {
private boolean required;
private RoutingFieldMapper(Settings indexSettings, MappedFieldType existing) {
this(existing == null ? Defaults.FIELD_TYPE.clone() : existing.clone(), Defaults.REQUIRED, indexSettings);
this(existing.clone(), Defaults.REQUIRED, indexSettings);
}
private RoutingFieldMapper(MappedFieldType fieldType, boolean required, Settings indexSettings) {

View File

@ -48,6 +48,7 @@ import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.Mapper.TypeParser.ParserContext;
import org.elasticsearch.index.mapper.ParseContext.Document;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.query.QueryShardException;
@ -136,7 +137,8 @@ public class SeqNoFieldMapper extends MetadataFieldMapper {
}
@Override
public MetadataFieldMapper getDefault(Settings indexSettings, MappedFieldType fieldType, String typeName) {
public MetadataFieldMapper getDefault(MappedFieldType fieldType, ParserContext context) {
final Settings indexSettings = context.mapperService().getIndexSettings().getSettings();
return new SeqNoFieldMapper(indexSettings);
}
}

View File

@ -19,7 +19,6 @@
package org.elasticsearch.index.mapper;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexableField;
@ -109,7 +108,7 @@ public class SourceFieldMapper extends MetadataFieldMapper {
public static class TypeParser implements MetadataFieldMapper.TypeParser {
@Override
public MetadataFieldMapper.Builder parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
public MetadataFieldMapper.Builder<?,?> parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
Builder builder = new Builder();
for (Iterator<Map.Entry<String, Object>> iterator = node.entrySet().iterator(); iterator.hasNext();) {
@ -144,7 +143,8 @@ public class SourceFieldMapper extends MetadataFieldMapper {
}
@Override
public MetadataFieldMapper getDefault(Settings indexSettings, MappedFieldType fieldType, String typeName) {
public MetadataFieldMapper getDefault(MappedFieldType fieldType, ParserContext context) {
final Settings indexSettings = context.mapperService().getIndexSettings().getSettings();
return new SourceFieldMapper(indexSettings);
}
}

View File

@ -76,12 +76,13 @@ public class TypeFieldMapper extends MetadataFieldMapper {
public static class TypeParser implements MetadataFieldMapper.TypeParser {
@Override
public MetadataFieldMapper.Builder parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
public MetadataFieldMapper.Builder<?,?> parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
throw new MapperParsingException(NAME + " is not configurable");
}
@Override
public MetadataFieldMapper getDefault(Settings indexSettings, MappedFieldType fieldType, String typeName) {
public MetadataFieldMapper getDefault(MappedFieldType fieldType, ParserContext context) {
final Settings indexSettings = context.mapperService().getIndexSettings().getSettings();
return new TypeFieldMapper(indexSettings, fieldType);
}
}

View File

@ -69,7 +69,8 @@ public class UidFieldMapper extends MetadataFieldMapper {
}
@Override
public MetadataFieldMapper getDefault(Settings indexSettings, MappedFieldType fieldType, String typeName) {
public MetadataFieldMapper getDefault(MappedFieldType fieldType, ParserContext context) {
final Settings indexSettings = context.mapperService().getIndexSettings().getSettings();
return new UidFieldMapper(indexSettings, fieldType);
}
}

View File

@ -62,7 +62,8 @@ public class VersionFieldMapper extends MetadataFieldMapper {
}
@Override
public MetadataFieldMapper getDefault(Settings indexSettings, MappedFieldType fieldType, String typeName) {
public MetadataFieldMapper getDefault(MappedFieldType fieldType, ParserContext context) {
final Settings indexSettings = context.mapperService().getIndexSettings().getSettings();
return new VersionFieldMapper(indexSettings);
}
}

View File

@ -182,12 +182,6 @@ public class MergeStats implements Streamable, ToXContent {
return new ByteSizeValue(currentSizeInBytes);
}
public static MergeStats readMergeStats(StreamInput in) throws IOException {
MergeStats stats = new MergeStats();
stats.readFrom(in);
return stats;
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(Fields.MERGES);

View File

@ -81,12 +81,6 @@ public class RefreshStats implements Streamable, ToXContent {
return new TimeValue(totalTimeInMillis);
}
public static RefreshStats readRefreshStats(StreamInput in) throws IOException {
RefreshStats refreshStats = new RefreshStats();
refreshStats.readFrom(in);
return refreshStats;
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(Fields.REFRESH);

View File

@ -19,6 +19,7 @@
package org.elasticsearch.index.search.stats;
import org.elasticsearch.action.support.ToXContentToBytes;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
@ -32,7 +33,7 @@ import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
public class SearchStats implements Streamable, ToXContent {
public class SearchStats extends ToXContentToBytes implements Streamable {
public static class Stats implements Streamable, ToXContent {
@ -338,22 +339,12 @@ public class SearchStats implements Streamable, ToXContent {
static final String SUGGEST_CURRENT = "suggest_current";
}
public static SearchStats readSearchStats(StreamInput in) throws IOException {
SearchStats searchStats = new SearchStats();
searchStats.readFrom(in);
return searchStats;
}
@Override
public void readFrom(StreamInput in) throws IOException {
totalStats = Stats.readStats(in);
openContexts = in.readVLong();
if (in.readBoolean()) {
int size = in.readVInt();
groupStats = new HashMap<>(size);
for (int i = 0; i < size; i++) {
groupStats.put(in.readString(), Stats.readStats(in));
}
groupStats = in.readMap(StreamInput::readString, Stats::readStats);
}
}
@ -365,24 +356,7 @@ public class SearchStats implements Streamable, ToXContent {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
out.writeVInt(groupStats.size());
for (Map.Entry<String, Stats> entry : groupStats.entrySet()) {
out.writeString(entry.getKey());
entry.getValue().writeTo(out);
}
}
}
@Override
public String toString() {
try {
XContentBuilder builder = XContentFactory.jsonBuilder().prettyPrint();
builder.startObject();
toXContent(builder, EMPTY_PARAMS);
builder.endObject();
return builder.string();
} catch (IOException e) {
return "{ \"error\" : \"" + e.getMessage() + "\"}";
out.writeMap(groupStats, StreamOutput::writeString, (stream, stats) -> stats.writeTo(stream));
}
}
}

View File

@ -149,12 +149,14 @@ public class GlobalCheckpointService extends AbstractIndexShardComponent {
* updates the global checkpoint on a replica shard (after it has been updated by the primary).
*/
synchronized void updateCheckpointOnReplica(long globalCheckpoint) {
/*
* The global checkpoint here is a local knowledge which is updated under the mandate of the primary. It can happen that the primary
* information is lagging compared to a replica (e.g., if a replica is promoted to primary but has stale info relative to other
* replica shards). In these cases, the local knowledge of the global checkpoint could be higher than sync from the lagging primary.
*/
if (this.globalCheckpoint <= globalCheckpoint) {
this.globalCheckpoint = globalCheckpoint;
logger.trace("global checkpoint updated from primary to [{}]", globalCheckpoint);
} else {
throw new IllegalArgumentException("global checkpoint from primary should never decrease. current [" +
this.globalCheckpoint + "], got [" + globalCheckpoint + "]");
}
}

View File

@ -20,20 +20,21 @@ package org.elasticsearch.index.seqno;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.logging.log4j.util.Supplier;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.replication.ReplicationOperation;
import org.elasticsearch.action.support.replication.ReplicationRequest;
import org.elasticsearch.action.support.replication.ReplicationResponse;
import org.elasticsearch.action.support.replication.TransportReplicationAction;
import org.elasticsearch.cluster.action.shard.ShardStateAction;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.IndicesService;
@ -41,8 +42,6 @@ import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.io.UnsupportedEncodingException;
public class GlobalCheckpointSyncAction extends TransportReplicationAction<GlobalCheckpointSyncAction.PrimaryRequest,
GlobalCheckpointSyncAction.ReplicaRequest, ReplicationResponse> {
@ -65,6 +64,17 @@ public class GlobalCheckpointSyncAction extends TransportReplicationAction<Globa
return new ReplicationResponse();
}
@Override
protected void sendReplicaRequest(ConcreteShardRequest<ReplicaRequest> concreteShardRequest, DiscoveryNode node,
ActionListener<ReplicationOperation.ReplicaResponse> listener) {
if (node.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) {
super.sendReplicaRequest(concreteShardRequest, node, listener);
} else {
listener.onResponse(
new ReplicaResponse(concreteShardRequest.getTargetAllocationID(), SequenceNumbersService.UNASSIGNED_SEQ_NO));
}
}
@Override
protected PrimaryResult shardOperationOnPrimary(PrimaryRequest request, IndexShard indexShard) throws Exception {
long checkpoint = indexShard.getGlobalCheckpoint();
@ -105,6 +115,11 @@ public class GlobalCheckpointSyncAction extends TransportReplicationAction<Globa
public PrimaryRequest(ShardId shardId) {
super(shardId);
}
@Override
public String toString() {
return "GlobalCkpSyncPrimary{" + shardId + "}";
}
}
public static final class ReplicaRequest extends ReplicationRequest<GlobalCheckpointSyncAction.ReplicaRequest> {
@ -134,6 +149,14 @@ public class GlobalCheckpointSyncAction extends TransportReplicationAction<Globa
public long getCheckpoint() {
return checkpoint;
}
@Override
public String toString() {
return "GlobalCkpSyncReplica{" +
"checkpoint=" + checkpoint +
", shardId=" + shardId +
'}';
}
}
}

View File

@ -57,12 +57,6 @@ public class DocsStats implements Streamable, ToXContent {
return this.deleted;
}
public static DocsStats readDocStats(StreamInput in) throws IOException {
DocsStats docsStats = new DocsStats();
docsStats.readFrom(in);
return docsStats;
}
@Override
public void readFrom(StreamInput in) throws IOException {
count = in.readVLong();

View File

@ -143,11 +143,7 @@ public class IndexingStats implements Streamable, ToXContent {
indexCount = in.readVLong();
indexTimeInMillis = in.readVLong();
indexCurrent = in.readVLong();
if(in.getVersion().onOrAfter(Version.V_2_1_0)){
indexFailedCount = in.readVLong();
}
indexFailedCount = in.readVLong();
deleteCount = in.readVLong();
deleteTimeInMillis = in.readVLong();
deleteCurrent = in.readVLong();
@ -161,11 +157,7 @@ public class IndexingStats implements Streamable, ToXContent {
out.writeVLong(indexCount);
out.writeVLong(indexTimeInMillis);
out.writeVLong(indexCurrent);
if(out.getVersion().onOrAfter(Version.V_2_1_0)) {
out.writeVLong(indexFailedCount);
}
out.writeVLong(indexFailedCount);
out.writeVLong(deleteCount);
out.writeVLong(deleteTimeInMillis);
out.writeVLong(deleteCurrent);
@ -283,21 +275,11 @@ public class IndexingStats implements Streamable, ToXContent {
static final String THROTTLED_TIME = "throttle_time";
}
public static IndexingStats readIndexingStats(StreamInput in) throws IOException {
IndexingStats indexingStats = new IndexingStats();
indexingStats.readFrom(in);
return indexingStats;
}
@Override
public void readFrom(StreamInput in) throws IOException {
totalStats = Stats.readStats(in);
if (in.readBoolean()) {
int size = in.readVInt();
typeStats = new HashMap<>(size);
for (int i = 0; i < size; i++) {
typeStats.put(in.readString(), Stats.readStats(in));
}
typeStats = in.readMap(StreamInput::readString, Stats::readStats);
}
}
@ -308,11 +290,7 @@ public class IndexingStats implements Streamable, ToXContent {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
out.writeVInt(typeStats.size());
for (Map.Entry<String, Stats> entry : typeStats.entrySet()) {
out.writeString(entry.getKey());
entry.getValue().writeTo(out);
}
out.writeMap(typeStats, StreamOutput::writeString, (stream, stats) -> stats.writeTo(stream));
}
}
}

View File

@ -26,8 +26,7 @@ import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.Lock;
import org.apache.lucene.store.NoLockFactory;
import org.elasticsearch.cluster.metadata.MappingMetaData;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.store.Store;
@ -123,8 +122,8 @@ final class LocalShardSnapshot implements Closeable {
}
}
ImmutableOpenMap<String, MappingMetaData> getMappings() {
return shard.indexSettings.getIndexMetaData().getMappings();
IndexMetaData getIndexMetaData() {
return shard.indexSettings.getIndexMetaData();
}
@Override

View File

@ -104,12 +104,11 @@ final class StoreRecovery {
if (indices.size() > 1) {
throw new IllegalArgumentException("can't add shards from more than one index");
}
for (ObjectObjectCursor<String, MappingMetaData> mapping : shards.get(0).getMappings()) {
IndexMetaData indexMetaData = shards.get(0).getIndexMetaData();
for (ObjectObjectCursor<String, MappingMetaData> mapping : indexMetaData.getMappings()) {
mappingUpdateConsumer.accept(mapping.key, mapping.value);
}
for (ObjectObjectCursor<String, MappingMetaData> mapping : shards.get(0).getMappings()) {
indexShard.mapperService().merge(mapping.key,mapping.value.source(), MapperService.MergeReason.MAPPING_RECOVERY, true);
}
indexShard.mapperService().merge(indexMetaData, MapperService.MergeReason.MAPPING_RECOVERY, true);
return executeRecovery(indexShard, () -> {
logger.debug("starting recovery from local shards {}", shards);
try {

View File

@ -65,12 +65,6 @@ public class StoreStats implements Streamable, ToXContent {
return size();
}
public static StoreStats readStoreStats(StreamInput in) throws IOException {
StoreStats store = new StoreStats();
store.readFrom(in);
return store;
}
@Override
public void readFrom(StreamInput in) throws IOException {
sizeInBytes = in.readVLong();

View File

@ -86,12 +86,6 @@ public class WarmerStats implements Streamable, ToXContent {
return new TimeValue(totalTimeInMillis);
}
public static WarmerStats readWarmerStats(StreamInput in) throws IOException {
WarmerStats refreshStats = new WarmerStats();
refreshStats.readFrom(in);
return refreshStats;
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(Fields.WARMER);

View File

@ -485,11 +485,7 @@ public class IndicesService extends AbstractLifecycleComponent
final IndexService service =
createIndexService("metadata verification", metaData, indicesQueryCache, indicesFieldDataCache, emptyList(), s -> {});
closeables.add(() -> service.close("metadata verification", false));
for (ObjectCursor<MappingMetaData> typeMapping : metaData.getMappings().values()) {
// don't apply the default mapping, it has been applied when the mapping was created
service.mapperService().merge(typeMapping.value.type(), typeMapping.value.source(),
MapperService.MergeReason.MAPPING_RECOVERY, true);
}
service.mapperService().merge(metaData, MapperService.MergeReason.MAPPING_RECOVERY, true);
if (metaData.equals(metaDataUpdate) == false) {
service.updateMetaData(metaDataUpdate);
}

View File

@ -24,6 +24,7 @@ import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.logging.log4j.util.Supplier;
import org.apache.lucene.store.LockObtainFailedException;
import org.elasticsearch.ResourceAlreadyExistsException;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateApplier;
@ -561,9 +562,15 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple
shard.updateRoutingEntry(shardRouting);
if (shardRouting.primary()) {
IndexShardRoutingTable indexShardRoutingTable = routingTable.shardRoutingTable(shardRouting.shardId());
Set<String> activeIds = indexShardRoutingTable.activeShards().stream().map(r -> r.allocationId().getId())
Set<String> activeIds = indexShardRoutingTable.activeShards().stream()
// filter to shards that track seq# and should be taken into consideration for checkpoint tracking
// shards on old nodes will go through a file based recovery which will also transfer seq# information.
.filter(sr -> nodes.get(sr.currentNodeId()).getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED))
.map(r -> r.allocationId().getId())
.collect(Collectors.toSet());
Set<String> initializingIds = indexShardRoutingTable.getAllInitializingShards().stream().map(r -> r.allocationId().getId())
Set<String> initializingIds = indexShardRoutingTable.getAllInitializingShards().stream()
.filter(sr -> nodes.get(sr.currentNodeId()).getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED))
.map(r -> r.allocationId().getId())
.collect(Collectors.toSet());
shard.updateAllocationIdsFromMaster(activeIds, initializingIds);
}

View File

@ -312,9 +312,9 @@ public class PeerRecoveryTargetService extends AbstractComponent implements Inde
@Override
public void messageReceived(RecoveryFinalizeRecoveryRequest request, TransportChannel channel) throws Exception {
try (RecoveriesCollection.RecoveryRef recoveryRef = onGoingRecoveries.getRecoverySafe(request.recoveryId(), request.shardId()))
{
recoveryRef.status().finalizeRecovery();
try (RecoveriesCollection.RecoveryRef recoveryRef =
onGoingRecoveries.getRecoverySafe(request.recoveryId(), request.shardId())) {
recoveryRef.status().finalizeRecovery(request.globalCheckpoint());
}
channel.sendResponse(TransportResponse.Empty.INSTANCE);
}

View File

@ -19,8 +19,10 @@
package org.elasticsearch.indices.recovery;
import org.elasticsearch.Version;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.index.seqno.SequenceNumbersService;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.transport.TransportRequest;
@ -29,15 +31,16 @@ import java.io.IOException;
public class RecoveryFinalizeRecoveryRequest extends TransportRequest {
private long recoveryId;
private ShardId shardId;
private long globalCheckpoint;
public RecoveryFinalizeRecoveryRequest() {
}
RecoveryFinalizeRecoveryRequest(long recoveryId, ShardId shardId) {
RecoveryFinalizeRecoveryRequest(final long recoveryId, final ShardId shardId, final long globalCheckpoint) {
this.recoveryId = recoveryId;
this.shardId = shardId;
this.globalCheckpoint = globalCheckpoint;
}
public long recoveryId() {
@ -48,11 +51,20 @@ public class RecoveryFinalizeRecoveryRequest extends TransportRequest {
return shardId;
}
public long globalCheckpoint() {
return globalCheckpoint;
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
recoveryId = in.readLong();
shardId = ShardId.readShardId(in);
if (in.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) {
globalCheckpoint = in.readZLong();
} else {
globalCheckpoint = SequenceNumbersService.UNASSIGNED_SEQ_NO;
}
}
@Override
@ -60,5 +72,9 @@ public class RecoveryFinalizeRecoveryRequest extends TransportRequest {
super.writeTo(out);
out.writeLong(recoveryId);
shardId.writeTo(out);
if (out.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) {
out.writeZLong(globalCheckpoint);
}
}
}

View File

@ -391,8 +391,8 @@ public class RecoverySourceHandler {
StopWatch stopWatch = new StopWatch().start();
logger.trace("[{}][{}] finalizing recovery to {}", indexName, shardId, request.targetNode());
cancellableThreads.execute(() -> {
recoveryTarget.finalizeRecovery();
shard.markAllocationIdAsInSync(recoveryTarget.getTargetAllocationId());
recoveryTarget.finalizeRecovery(shard.getGlobalCheckpoint());
});
if (request.isPrimaryRelocation()) {

View File

@ -333,7 +333,8 @@ public class RecoveryTarget extends AbstractRefCounted implements RecoveryTarget
}
@Override
public void finalizeRecovery() {
public void finalizeRecovery(final long globalCheckpoint) {
indexShard().updateGlobalCheckpointOnReplica(globalCheckpoint);
final IndexShard indexShard = indexShard();
indexShard.finalizeRecovery();
}

View File

@ -39,11 +39,12 @@ public interface RecoveryTargetHandler {
void prepareForTranslogOperations(int totalTranslogOps, long maxUnsafeAutoIdTimestamp) throws IOException;
/**
* The finalize request clears unreferenced translog files, refreshes the engine now that
* new segments are available, and enables garbage collection of
* tombstone files.
**/
void finalizeRecovery();
* The finalize request refreshes the engine now that new segments are available, enables garbage collection of tombstone files, and
* updates the global checkpoint.
*
* @param globalCheckpoint the global checkpoint on the recovery source
*/
void finalizeRecovery(long globalCheckpoint);
/**
* Blockingly waits for cluster state with at least clusterStateVersion to be available
@ -82,4 +83,5 @@ public interface RecoveryTargetHandler {
* @return the allocation id of the target shard.
*/
String getTargetAllocationId();
}

View File

@ -86,9 +86,9 @@ public class RemoteRecoveryTargetHandler implements RecoveryTargetHandler {
}
@Override
public void finalizeRecovery() {
public void finalizeRecovery(final long globalCheckpoint) {
transportService.submitRequest(targetNode, PeerRecoveryTargetService.Actions.FINALIZE,
new RecoveryFinalizeRecoveryRequest(recoveryId, shardId),
new RecoveryFinalizeRecoveryRequest(recoveryId, shardId, globalCheckpoint),
TransportRequestOptions.builder().withTimeout(recoverySettings.internalActionLongTimeout()).build(),
EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
}

View File

@ -27,18 +27,14 @@ import org.elasticsearch.index.mapper.RoutingFieldMapper;
import org.elasticsearch.index.mapper.SourceFieldMapper;
import org.elasticsearch.index.mapper.TypeFieldMapper;
import java.text.DateFormat;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Base64;
import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Objects;
import java.util.TimeZone;
/**
* Represents a single document being captured before indexing and holds the source and metadata (like id, type and index).
@ -68,9 +64,7 @@ public final class IngestDocument {
}
this.ingestMetadata = new HashMap<>();
DateFormat df = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSZZ", Locale.ROOT);
df.setTimeZone(TimeZone.getTimeZone("UTC"));
this.ingestMetadata.put(TIMESTAMP, df.format(new Date()));
this.ingestMetadata.put(TIMESTAMP, new Date());
}
/**
@ -595,6 +589,8 @@ public final class IngestDocument {
value instanceof Long || value instanceof Float ||
value instanceof Double || value instanceof Boolean) {
return value;
} else if (value instanceof Date) {
return ((Date) value).clone();
} else {
throw new IllegalArgumentException("unexpected value type [" + value.getClass() + "]");
}

View File

@ -54,7 +54,7 @@ public class IngestStats implements Writeable, ToXContent {
@Override
public void writeTo(StreamOutput out) throws IOException {
totalStats.writeTo(out);
out.writeVLong(statsPerPipeline.size());
out.writeVInt(statsPerPipeline.size());
for (Map.Entry<String, Stats> entry : statsPerPipeline.entrySet()) {
out.writeString(entry.getKey());
entry.getValue().writeTo(out);

View File

@ -1,97 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.rest.action.search;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.client.node.NodeClient;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.rest.BaseRestHandler;
import org.elasticsearch.rest.BytesRestResponse;
import org.elasticsearch.rest.RestController;
import org.elasticsearch.rest.RestRequest;
import org.elasticsearch.rest.RestResponse;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.rest.action.RestBuilderListener;
import org.elasticsearch.search.SearchRequestParsers;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.suggest.Suggest;
import org.elasticsearch.search.suggest.SuggestBuilder;
import java.io.IOException;
import static org.elasticsearch.rest.RestRequest.Method.GET;
import static org.elasticsearch.rest.RestRequest.Method.POST;
import static org.elasticsearch.rest.action.RestActions.buildBroadcastShardsHeader;
public class RestSuggestAction extends BaseRestHandler {
private final SearchRequestParsers searchRequestParsers;
@Inject
public RestSuggestAction(Settings settings, RestController controller,
SearchRequestParsers searchRequestParsers) {
super(settings);
this.searchRequestParsers = searchRequestParsers;
controller.registerAsDeprecatedHandler(POST, "/_suggest", this,
"[POST /_suggest] is deprecated! Use [POST /_search] instead.", deprecationLogger);
controller.registerAsDeprecatedHandler(GET, "/_suggest", this,
"[GET /_suggest] is deprecated! Use [GET /_search] instead.", deprecationLogger);
controller.registerAsDeprecatedHandler(POST, "/{index}/_suggest", this,
"[POST /{index}/_suggest] is deprecated! Use [POST /{index}/_search] instead.", deprecationLogger);
controller.registerAsDeprecatedHandler(GET, "/{index}/_suggest", this,
"[GET /{index}/_suggest] is deprecated! Use [GET /{index}/_search] instead.", deprecationLogger);
}
@Override
public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException {
final SearchRequest searchRequest = new SearchRequest(
Strings.splitStringByCommaToArray(request.param("index")), new SearchSourceBuilder());
searchRequest.indicesOptions(IndicesOptions.fromRequest(request, searchRequest.indicesOptions()));
try (XContentParser parser = request.contentOrSourceParamParser()) {
final QueryParseContext context = new QueryParseContext(searchRequestParsers.queryParsers, parser, parseFieldMatcher);
searchRequest.source().suggest(SuggestBuilder.fromXContent(context, searchRequestParsers.suggesters));
}
searchRequest.routing(request.param("routing"));
searchRequest.preference(request.param("preference"));
return channel -> client.search(searchRequest, new RestBuilderListener<SearchResponse>(channel) {
@Override
public RestResponse buildResponse(SearchResponse response, XContentBuilder builder) throws Exception {
RestStatus restStatus = RestStatus.status(response.getSuccessfulShards(),
response.getTotalShards(), response.getShardFailures());
builder.startObject();
buildBroadcastShardsHeader(builder, request, response.getTotalShards(),
response.getSuccessfulShards(), response.getFailedShards(), response.getShardFailures());
Suggest suggest = response.getSuggest();
if (suggest != null) {
suggest.toInnerXContent(builder, request);
}
builder.endObject();
return new BytesRestResponse(restStatus, builder);
}
});
}
}

View File

@ -32,6 +32,7 @@ import org.elasticsearch.search.profile.aggregation.ProfilingAggregator;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
@ -255,7 +256,7 @@ public class AggregatorFactories {
} else {
// Check the non-pipeline sub-aggregator
// factories
AggregationBuilder[] subBuilders = aggBuilder.factoriesBuilder.getAggregatorFactories();
List<AggregationBuilder> subBuilders = aggBuilder.factoriesBuilder.aggregationBuilders;
boolean foundSubBuilder = false;
for (AggregationBuilder subBuilder : subBuilders) {
if (aggName.equals(subBuilder.name)) {
@ -297,12 +298,12 @@ public class AggregatorFactories {
}
}
AggregationBuilder[] getAggregatorFactories() {
return this.aggregationBuilders.toArray(new AggregationBuilder[this.aggregationBuilders.size()]);
public List<AggregationBuilder> getAggregatorFactories() {
return Collections.unmodifiableList(aggregationBuilders);
}
List<PipelineAggregationBuilder> getPipelineAggregatorFactories() {
return this.pipelineAggregatorBuilders;
public List<PipelineAggregationBuilder> getPipelineAggregatorFactories() {
return Collections.unmodifiableList(pipelineAggregatorBuilders);
}
public int count() {

View File

@ -150,18 +150,10 @@ public class Suggest implements Iterable<Suggest.Suggestion<? extends Entry<? ex
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(NAME);
toInnerXContent(builder, params);
builder.endObject();
return builder;
}
/**
* use to write suggestion entries without <code>NAME</code> object
*/
public XContentBuilder toInnerXContent(XContentBuilder builder, Params params) throws IOException {
for (Suggestion<?> suggestion : suggestions) {
suggestion.toXContent(builder, params);
}
builder.endObject();
return builder;
}

View File

@ -27,6 +27,7 @@ import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.Terms;
import org.apache.lucene.search.suggest.document.CompletionTerms;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.FieldMemoryStats;
import org.elasticsearch.common.regex.Regex;
import java.io.IOException;
@ -64,6 +65,6 @@ public class CompletionFieldStats {
throw new ElasticsearchException(ioe);
}
}
return new CompletionStats(sizeInBytes, completionFields);
return new CompletionStats(sizeInBytes, completionFields == null ? null : new FieldMemoryStats(completionFields));
}
}

View File

@ -18,7 +18,7 @@
*/
package org.elasticsearch.search.suggest.completion;
import com.carrotsearch.hppc.ObjectLongHashMap;
import org.elasticsearch.common.FieldMemoryStats;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
@ -31,15 +31,19 @@ import java.io.IOException;
public class CompletionStats implements Streamable, ToXContent {
private long sizeInBytes;
private static final String COMPLETION = "completion";
private static final String SIZE_IN_BYTES = "size_in_bytes";
private static final String SIZE = "size";
private static final String FIELDS = "fields";
private long sizeInBytes;
@Nullable
private ObjectLongHashMap<String> fields;
private FieldMemoryStats fields;
public CompletionStats() {
}
public CompletionStats(long size, @Nullable ObjectLongHashMap<String> fields) {
public CompletionStats(long size, @Nullable FieldMemoryStats fields) {
this.sizeInBytes = size;
this.fields = fields;
}
@ -52,98 +56,43 @@ public class CompletionStats implements Streamable, ToXContent {
return new ByteSizeValue(sizeInBytes);
}
public ObjectLongHashMap<String> getFields() {
public FieldMemoryStats getFields() {
return fields;
}
@Override
public void readFrom(StreamInput in) throws IOException {
sizeInBytes = in.readVLong();
if (in.readBoolean()) {
int size = in.readVInt();
fields = new ObjectLongHashMap<>(size);
for (int i = 0; i < size; i++) {
fields.put(in.readString(), in.readVLong());
}
}
fields = in.readOptionalWriteable(FieldMemoryStats::new);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeVLong(sizeInBytes);
if (fields == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
out.writeVInt(fields.size());
assert !fields.containsKey(null);
final Object[] keys = fields.keys;
final long[] values = fields.values;
for (int i = 0; i < keys.length; i++) {
if (keys[i] != null) {
out.writeString((String) keys[i]);
out.writeVLong(values[i]);
}
}
}
out.writeOptionalWriteable(fields);
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(Fields.COMPLETION);
builder.byteSizeField(Fields.SIZE_IN_BYTES, Fields.SIZE, sizeInBytes);
builder.startObject(COMPLETION);
builder.byteSizeField(SIZE_IN_BYTES, SIZE, sizeInBytes);
if (fields != null) {
builder.startObject(Fields.FIELDS);
assert !fields.containsKey(null);
final Object[] keys = fields.keys;
final long[] values = fields.values;
for (int i = 0; i < keys.length; i++) {
if (keys[i] != null) {
builder.startObject((String) keys[i]);
builder.byteSizeField(Fields.SIZE_IN_BYTES, Fields.SIZE, values[i]);
builder.endObject();
}
}
builder.endObject();
fields.toXContent(builder, FIELDS, SIZE_IN_BYTES, SIZE);
}
builder.endObject();
return builder;
}
public static CompletionStats readCompletionStats(StreamInput in) throws IOException {
CompletionStats stats = new CompletionStats();
stats.readFrom(in);
return stats;
}
static final class Fields {
static final String COMPLETION = "completion";
static final String SIZE_IN_BYTES = "size_in_bytes";
static final String SIZE = "size";
static final String FIELDS = "fields";
}
public void add(CompletionStats completion) {
if (completion == null) {
return;
}
sizeInBytes += completion.getSizeInBytes();
if (completion.fields != null) {
if (fields == null) {
fields = completion.fields.clone();
fields = completion.fields.copy();
} else {
assert !completion.fields.containsKey(null);
final Object[] keys = completion.fields.keys;
final long[] values = completion.fields.values;
for (int i = 0; i < keys.length; i++) {
if (keys[i] != null) {
fields.addTo((String) keys[i], values[i]);
}
}
fields.add(completion.fields);
}
}
}

View File

@ -148,9 +148,6 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
Setting.byteSizeSetting("transport.tcp.receive_buffer_size", NetworkService.TcpSettings.TCP_RECEIVE_BUFFER_SIZE,
Setting.Property.NodeScope);
// test-setting only
static final Setting<Boolean> CONNECTION_HANDSHAKE = Setting.boolSetting("transport.tcp.handshake", true);
private static final long NINETY_PER_HEAP_SIZE = (long) (JvmInfo.jvmInfo().getMem().getHeapMax().getBytes() * 0.9);
private static final int PING_DATA_SIZE = -1;
protected final boolean blockingClient;
@ -161,7 +158,6 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
protected final ThreadPool threadPool;
private final BigArrays bigArrays;
protected final NetworkService networkService;
private final boolean doHandshakes;
protected volatile TransportServiceAdapter transportServiceAdapter;
// node id to actual channel
@ -200,7 +196,6 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
this.transportName = transportName;
this.blockingClient = TCP_BLOCKING_CLIENT.get(settings);
defaultConnectionProfile = buildDefaultConnectionProfile(settings);
this.doHandshakes = CONNECTION_HANDSHAKE.get(settings);
}
static ConnectionProfile buildDefaultConnectionProfile(Settings settings) {
@ -463,21 +458,13 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
"failed to connect to [{}], cleaning dangling connections", node), e);
throw e;
}
if (doHandshakes) { // some tests need to disable this
Channel channel = nodeChannels.channel(TransportRequestOptions.Type.PING);
final TimeValue connectTimeout = connectionProfile.getConnectTimeout() == null ?
defaultConnectionProfile.getConnectTimeout():
connectionProfile.getConnectTimeout();
final TimeValue handshakeTimeout = connectionProfile.getHandshakeTimeout() == null ?
connectTimeout : connectionProfile.getHandshakeTimeout();
Version version = executeHandshake(node, channel, handshakeTimeout);
if (version != null) {
// this is a BWC layer, if we talk to a pre 5.2 node then the handshake is not supported
// this will go away in master once it's all ported to 5.2 but for now we keep this to make
// the backport straight forward
nodeChannels = new NodeChannels(nodeChannels, version);
}
}
Channel channel = nodeChannels.channel(TransportRequestOptions.Type.PING);
final TimeValue connectTimeout = connectionProfile.getConnectTimeout() == null ?
defaultConnectionProfile.getConnectTimeout() :
connectionProfile.getConnectTimeout();
final TimeValue handshakeTimeout = connectionProfile.getHandshakeTimeout() == null ?
connectTimeout : connectionProfile.getHandshakeTimeout();
Version version = executeHandshake(node, channel, handshakeTimeout);
// we acquire a connection lock, so no way there is an existing connection
connectedNodes.put(node, nodeChannels);
if (logger.isDebugEnabled()) {
@ -1130,7 +1117,7 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
* @param length the payload length in bytes
* @see TcpHeader
*/
private BytesReference buildHeader(long requestId, byte status, Version protocolVersion, int length) throws IOException {
final BytesReference buildHeader(long requestId, byte status, Version protocolVersion, int length) throws IOException {
try (BytesStreamOutput headerOutput = new BytesStreamOutput(TcpHeader.HEADER_SIZE)) {
headerOutput.setVersion(protocolVersion);
TcpHeader.writeHeader(headerOutput, requestId, status, protocolVersion, length);
@ -1306,7 +1293,7 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
handleRequest(channel, profileName, streamIn, requestId, messageLengthBytes, version, remoteAddress, status);
} else {
final TransportResponseHandler<?> handler;
if (TransportStatus.isHandshake(status) && doHandshakes) {
if (TransportStatus.isHandshake(status)) {
handler = pendingHandshakes.remove(requestId);
} else {
TransportResponseHandler theHandler = transportServiceAdapter.onResponseReceived(requestId);
@ -1398,7 +1385,7 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
transportServiceAdapter.onRequestReceived(requestId, action);
TransportChannel transportChannel = null;
try {
if (TransportStatus.isHandshake(status) && doHandshakes) {
if (TransportStatus.isHandshake(status)) {
final VersionHandshakeResponse response = new VersionHandshakeResponse(getCurrentVersion());
sendResponse(version, channel, response, requestId, HANDSHAKE_ACTION_NAME, TransportResponseOptions.EMPTY,
TransportStatus.setHandshake((byte)0));
@ -1509,8 +1496,7 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
}
}
// pkg private for testing
final Version executeHandshake(DiscoveryNode node, Channel channel, TimeValue timeout) throws IOException, InterruptedException {
protected Version executeHandshake(DiscoveryNode node, Channel channel, TimeValue timeout) throws IOException, InterruptedException {
numHandshakes.inc();
final long requestId = newRequestId();
final HandshakeResponseHandler handler = new HandshakeResponseHandler(channel);
@ -1520,7 +1506,7 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
boolean success = false;
try {
if (isOpen(channel) == false) {
// we have to protect ourself here since sendRequestToChannel won't barf if the channel is closed.
// we have to protect us here since sendRequestToChannel won't barf if the channel is closed.
// it's weird but to change it will cause a lot of impact on the exception handling code all over the codebase.
// yet, if we don't check the state here we might have registered a pending handshake handler but the close
// listener calling #onChannelClosed might have already run and we are waiting on the latch below unitl we time out.

View File

@ -290,6 +290,9 @@ public class TransportService extends AbstractLifecycleComponent {
return transport.getLocalAddresses();
}
/**
* Returns <code>true</code> iff the given node is already connected.
*/
public boolean nodeConnected(DiscoveryNode node) {
return node.equals(localNode) || transport.nodeConnected(node);
}
@ -311,6 +314,20 @@ public class TransportService extends AbstractLifecycleComponent {
transport.connectToNode(node, connectionProfile);
}
/**
* Establishes and returns a new connection to the given node. The connection is NOT maintained by this service, it's the callers
* responsibility to close the connection once it goes out of scope.
* @param node the node to connect to
* @param profile the connection profile to use
*/
public Transport.Connection openConnection(final DiscoveryNode node, ConnectionProfile profile) throws IOException {
if (node.equals(localNode)) {
return localNodeConnection;
} else {
return transport.openConnection(node, profile);
}
}
/**
* Lightly connect to the specified node, returning updated node
* information. The handshake will fail if the cluster name on the
@ -337,7 +354,19 @@ public class TransportService extends AbstractLifecycleComponent {
return handshakeNode;
}
private DiscoveryNode handshake(
/**
* Executes a high-level handshake using the given connection
* and returns the discovery node of the node the connection
* was established with. The handshake will fail if the cluster
* name on the target node mismatches the local cluster name.
*
* @param connection the connection to a specific node
* @param handshakeTimeout handshake timeout
* @return the connected node
* @throws ConnectTransportException if the connection failed
* @throws IllegalStateException if the handshake failed
*/
public DiscoveryNode handshake(
final Transport.Connection connection,
final long handshakeTimeout) throws ConnectTransportException {
final HandshakeResponse response;
@ -465,7 +494,7 @@ public class TransportService extends AbstractLifecycleComponent {
}
}
final <T extends TransportResponse> void sendRequest(final Transport.Connection connection, final String action,
public final <T extends TransportResponse> void sendRequest(final Transport.Connection connection, final String action,
final TransportRequest request,
final TransportRequestOptions options,
TransportResponseHandler<T> handler) {
@ -477,7 +506,7 @@ public class TransportService extends AbstractLifecycleComponent {
* Returns either a real transport connection or a local node connection if we are using the local node optimization.
* @throws NodeNotConnectedException if the given node is not connected
*/
private Transport.Connection getConnection(DiscoveryNode node) {
public Transport.Connection getConnection(DiscoveryNode node) {
if (Objects.requireNonNull(node, "node must be non-null").equals(localNode)) {
return localNodeConnection;
} else {

View File

@ -39,7 +39,6 @@ import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.query.RangeQueryBuilder;
import org.elasticsearch.index.query.TermsQueryBuilder;
import org.elasticsearch.test.ESIntegTestCase;
@ -277,15 +276,8 @@ public class CreateIndexIT extends ESIntegTestCase {
.startObject("text")
.field("type", "text")
.endObject().endObject().endObject());
try {
b.get();
} catch (MapperParsingException e) {
StringBuilder messages = new StringBuilder();
for (Exception rootCause: e.guessRootCauses()) {
messages.append(rootCause.getMessage());
}
assertThat(messages.toString(), containsString("mapper [text] is used by multiple types"));
}
IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> b.get());
assertThat(e.getMessage(), containsString("mapper [text] is used by multiple types"));
}
public void testRestartIndexCreationAfterFullClusterRestart() throws Exception {

View File

@ -46,7 +46,7 @@ public class FlushBlocksIT extends ESIntegTestCase {
}
// Request is not blocked
for (String blockSetting : Arrays.asList(SETTING_BLOCKS_READ, SETTING_BLOCKS_WRITE)) {
for (String blockSetting : Arrays.asList(SETTING_BLOCKS_READ, SETTING_BLOCKS_WRITE, SETTING_READ_ONLY, SETTING_BLOCKS_METADATA)) {
try {
enableIndexBlock("test", blockSetting);
FlushResponse response = client().admin().indices().prepareFlush("test").execute().actionGet();
@ -56,28 +56,5 @@ public class FlushBlocksIT extends ESIntegTestCase {
disableIndexBlock("test", blockSetting);
}
}
// Request is blocked
for (String blockSetting : Arrays.asList(SETTING_READ_ONLY, SETTING_BLOCKS_METADATA)) {
try {
enableIndexBlock("test", blockSetting);
FlushResponse flushResponse = client().admin().indices().prepareFlush("test").get();
assertBlocked(flushResponse);
} finally {
disableIndexBlock("test", blockSetting);
}
}
// Flushing all indices is blocked when the cluster is read-only
try {
FlushResponse response = client().admin().indices().prepareFlush().execute().actionGet();
assertNoFailures(response);
assertThat(response.getSuccessfulShards(), equalTo(numShards.totalNumShards));
setClusterReadOnly(true);
assertBlocked(client().admin().indices().prepareFlush().get());
} finally {
setClusterReadOnly(false);
}
}
}

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