Merge branch 'master' into immediate_shard_active

Conflicts:
	core/src/main/java/org/elasticsearch/index/shard/IndexShard.java
	core/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java
	core/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java
This commit is contained in:
Michael McCandless 2015-10-06 04:19:56 -04:00 committed by mikemccand
commit a082135538
136 changed files with 1694 additions and 1514 deletions

View File

@ -274,7 +274,7 @@
<include>org/elasticsearch/common/cli/CliToolTestCase$*.class</include>
<include>org/elasticsearch/cluster/MockInternalClusterInfoService.class</include>
<include>org/elasticsearch/cluster/MockInternalClusterInfoService$*.class</include>
<include>org/elasticsearch/index/shard/MockEngineFactoryPlugin.class</include>
<include>org/elasticsearch/index/MockEngineFactoryPlugin.class</include>
<include>org/elasticsearch/search/MockSearchService.class</include>
<include>org/elasticsearch/search/MockSearchService$*.class</include>
<include>org/elasticsearch/search/aggregations/bucket/AbstractTermsTestCase.class</include>

View File

@ -31,7 +31,7 @@ import org.elasticsearch.common.xcontent.XContentBuilderString;
import org.elasticsearch.index.cache.query.QueryCacheStats;
import org.elasticsearch.index.engine.SegmentsStats;
import org.elasticsearch.index.fielddata.FieldDataStats;
import org.elasticsearch.index.percolator.stats.PercolateStats;
import org.elasticsearch.index.percolator.PercolateStats;
import org.elasticsearch.index.shard.DocsStats;
import org.elasticsearch.index.store.StoreStats;
import org.elasticsearch.search.suggest.completion.CompletionStats;

View File

@ -83,7 +83,7 @@ public class TransportClearIndicesCacheAction extends TransportBroadcastByNodeAc
protected EmptyResult shardOperation(ClearIndicesCacheRequest request, ShardRouting shardRouting) {
IndexService service = indicesService.indexService(shardRouting.getIndex());
if (service != null) {
IndexShard shard = service.shard(shardRouting.id());
IndexShard shard = service.getShardOrNull(shardRouting.id());
boolean clearedAtLeastOne = false;
if (request.queryCache()) {
clearedAtLeastOne = true;

View File

@ -62,7 +62,7 @@ public class TransportShardFlushAction extends TransportReplicationAction<ShardF
@Override
protected Tuple<ActionWriteResponse, ShardFlushRequest> shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) throws Throwable {
IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()).shardSafe(shardRequest.shardId.id());
IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()).getShard(shardRequest.shardId.id());
indexShard.flush(shardRequest.request.getRequest());
logger.trace("{} flush request executed on primary", indexShard.shardId());
return new Tuple<>(new ActionWriteResponse(), shardRequest.request);
@ -70,7 +70,7 @@ public class TransportShardFlushAction extends TransportReplicationAction<ShardF
@Override
protected void shardOperationOnReplica(ShardId shardId, ShardFlushRequest request) {
IndexShard indexShard = indicesService.indexServiceSafe(request.shardId().getIndex()).shardSafe(request.shardId().id());
IndexShard indexShard = indicesService.indexServiceSafe(request.shardId().getIndex()).getShard(request.shardId().id());
indexShard.flush(request.getRequest());
logger.trace("{} flush request executed on replica", indexShard.shardId());
}

View File

@ -75,7 +75,7 @@ public class TransportOptimizeAction extends TransportBroadcastByNodeAction<Opti
@Override
protected EmptyResult shardOperation(OptimizeRequest request, ShardRouting shardRouting) throws IOException {
IndexShard indexShard = indicesService.indexServiceSafe(shardRouting.shardId().getIndex()).shardSafe(shardRouting.shardId().id());
IndexShard indexShard = indicesService.indexServiceSafe(shardRouting.shardId().getIndex()).getShard(shardRouting.shardId().id());
indexShard.optimize(request);
return EmptyResult.INSTANCE;
}

View File

@ -100,7 +100,7 @@ public class TransportRecoveryAction extends TransportBroadcastByNodeAction<Reco
@Override
protected RecoveryState shardOperation(RecoveryRequest request, ShardRouting shardRouting) {
IndexService indexService = indicesService.indexServiceSafe(shardRouting.shardId().getIndex());
IndexShard indexShard = indexService.shardSafe(shardRouting.shardId().id());
IndexShard indexShard = indexService.getShard(shardRouting.shardId().id());
return indexShard.recoveryState();
}

View File

@ -63,7 +63,7 @@ public class TransportShardRefreshAction extends TransportReplicationAction<Repl
@Override
protected Tuple<ActionWriteResponse, ReplicationRequest> shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) throws Throwable {
IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()).shardSafe(shardRequest.shardId.id());
IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()).getShard(shardRequest.shardId.id());
indexShard.refresh("api");
logger.trace("{} refresh request executed on primary", indexShard.shardId());
return new Tuple<>(new ActionWriteResponse(), shardRequest.request);
@ -71,7 +71,7 @@ public class TransportShardRefreshAction extends TransportReplicationAction<Repl
@Override
protected void shardOperationOnReplica(ShardId shardId, ReplicationRequest request) {
IndexShard indexShard = indicesService.indexServiceSafe(shardId.getIndex()).shardSafe(shardId.id());
IndexShard indexShard = indicesService.indexServiceSafe(shardId.getIndex()).getShard(shardId.id());
indexShard.refresh("api");
logger.trace("{} refresh request executed on replica", indexShard.shardId());
}

View File

@ -94,7 +94,7 @@ public class TransportIndicesSegmentsAction extends TransportBroadcastByNodeActi
@Override
protected ShardSegments shardOperation(IndicesSegmentsRequest request, ShardRouting shardRouting) {
IndexService indexService = indicesService.indexServiceSafe(shardRouting.getIndex());
IndexShard indexShard = indexService.shardSafe(shardRouting.id());
return new ShardSegments(indexShard.routingEntry(), indexShard.engine().segments(request.verbose()));
IndexShard indexShard = indexService.getShard(shardRouting.id());
return new ShardSegments(indexShard.routingEntry(), indexShard.segments(request.verbose()));
}
}

View File

@ -34,7 +34,7 @@ import org.elasticsearch.index.flush.FlushStats;
import org.elasticsearch.index.get.GetStats;
import org.elasticsearch.index.indexing.IndexingStats;
import org.elasticsearch.index.merge.MergeStats;
import org.elasticsearch.index.percolator.stats.PercolateStats;
import org.elasticsearch.index.percolator.PercolateStats;
import org.elasticsearch.index.recovery.RecoveryStats;
import org.elasticsearch.index.refresh.RefreshStats;
import org.elasticsearch.index.search.stats.SearchStats;
@ -167,7 +167,7 @@ public class CommonStats implements Streamable, ToXContent {
segments = indexShard.segmentStats();
break;
case Percolate:
percolate = indexShard.shardPercolateService().stats();
percolate = indexShard.percolateStats();
break;
case Translog:
translog = indexShard.translogStats();

View File

@ -95,7 +95,7 @@ public class TransportIndicesStatsAction extends TransportBroadcastByNodeAction<
@Override
protected ShardStats shardOperation(IndicesStatsRequest request, ShardRouting shardRouting) {
IndexService indexService = indicesService.indexServiceSafe(shardRouting.shardId().getIndex());
IndexShard indexShard = indexService.shardSafe(shardRouting.shardId().id());
IndexShard indexShard = indexService.getShard(shardRouting.shardId().id());
// if we don't have the routing entry yet, we need it stats wise, we treat it as if the shard is not ready yet
if (indexShard.routingEntry() == null) {
throw new ShardNotFoundException(indexShard.shardId());

View File

@ -96,8 +96,8 @@ public class TransportUpgradeStatusAction extends TransportBroadcastByNodeAction
@Override
protected ShardUpgradeStatus shardOperation(UpgradeStatusRequest request, ShardRouting shardRouting) {
IndexService indexService = indicesService.indexServiceSafe(shardRouting.shardId().getIndex());
IndexShard indexShard = indexService.shardSafe(shardRouting.shardId().id());
List<Segment> segments = indexShard.engine().segments(false);
IndexShard indexShard = indexService.getShard(shardRouting.shardId().id());
List<Segment> segments = indexShard.segments(false);
long total_bytes = 0;
long to_upgrade_bytes = 0;
long to_upgrade_bytes_ancient = 0;

View File

@ -119,7 +119,7 @@ public class TransportUpgradeAction extends TransportBroadcastByNodeAction<Upgra
@Override
protected ShardUpgradeResult shardOperation(UpgradeRequest request, ShardRouting shardRouting) throws IOException {
IndexShard indexShard = indicesService.indexServiceSafe(shardRouting.shardId().getIndex()).shardSafe(shardRouting.shardId().id());
IndexShard indexShard = indicesService.indexServiceSafe(shardRouting.shardId().getIndex()).getShard(shardRouting.shardId().id());
org.apache.lucene.util.Version oldestLuceneSegment = indexShard.upgrade(request);
// We are using the current version of Elasticsearch as upgrade version since we update mapping to match the current version
return new ShardUpgradeResult(shardRouting.shardId(), indexShard.routingEntry().primary(), Version.CURRENT, oldestLuceneSegment);

View File

@ -163,7 +163,7 @@ public class TransportValidateQueryAction extends TransportBroadcastAction<Valid
protected ShardValidateQueryResponse shardOperation(ShardValidateQueryRequest request) {
IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
IndexQueryParserService queryParserService = indexService.queryParserService();
IndexShard indexShard = indexService.shardSafe(request.shardId().id());
IndexShard indexShard = indexService.getShard(request.shardId().id());
boolean valid;
String explanation = null;

View File

@ -116,7 +116,7 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
protected Tuple<BulkShardResponse, BulkShardRequest> shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) {
final BulkShardRequest request = shardRequest.request;
final IndexService indexService = indicesService.indexServiceSafe(request.index());
final IndexShard indexShard = indexService.shardSafe(shardRequest.shardId.id());
final IndexShard indexShard = indexService.getShard(shardRequest.shardId.id());
long[] preVersions = new long[request.items().length];
VersionType[] preVersionTypes = new VersionType[request.items().length];
@ -447,7 +447,7 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
@Override
protected void shardOperationOnReplica(ShardId shardId, BulkShardRequest request) {
IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
IndexShard indexShard = indexService.shardSafe(shardId.id());
IndexShard indexShard = indexService.getShard(shardId.id());
Translog.Location location = null;
for (int i = 0; i < request.items().length; i++) {
BulkItemRequest item = request.items()[i];

View File

@ -42,7 +42,6 @@ import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.indices.IndexAlreadyExistsException;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.threadpool.ThreadPool;
@ -130,7 +129,7 @@ public class TransportDeleteAction extends TransportReplicationAction<DeleteRequ
@Override
protected Tuple<DeleteResponse, DeleteRequest> shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) {
DeleteRequest request = shardRequest.request;
IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()).shardSafe(shardRequest.shardId.id());
IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()).getShard(shardRequest.shardId.id());
Engine.Delete delete = indexShard.prepareDelete(request.type(), request.id(), request.version(), request.versionType(), Engine.Operation.Origin.PRIMARY);
indexShard.delete(delete);
// update the request with teh version so it will go to the replicas
@ -146,7 +145,7 @@ public class TransportDeleteAction extends TransportReplicationAction<DeleteRequ
@Override
protected void shardOperationOnReplica(ShardId shardId, DeleteRequest request) {
IndexShard indexShard = indicesService.indexServiceSafe(shardId.getIndex()).shardSafe(shardId.id());
IndexShard indexShard = indicesService.indexServiceSafe(shardId.getIndex()).getShard(shardId.id());
Engine.Delete delete = indexShard.prepareDelete(request.type(), request.id(), request.version(), request.versionType(), Engine.Operation.Origin.REPLICA);
indexShard.delete(delete);

View File

@ -148,7 +148,7 @@ public class TransportExistsAction extends TransportBroadcastAction<ExistsReques
@Override
protected ShardExistsResponse shardOperation(ShardExistsRequest request) {
IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
IndexShard indexShard = indexService.shardSafe(request.shardId().id());
IndexShard indexShard = indexService.getShard(request.shardId().id());
SearchShardTarget shardTarget = new SearchShardTarget(clusterService.localNode().id(), request.shardId().getIndex(), request.shardId().id());
SearchContext context = new DefaultSearchContext(0,

View File

@ -104,7 +104,7 @@ public class TransportExplainAction extends TransportSingleShardAction<ExplainRe
@Override
protected ExplainResponse shardOperation(ExplainRequest request, ShardId shardId) {
IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
IndexShard indexShard = indexService.shardSafe(shardId.id());
IndexShard indexShard = indexService.getShard(shardId.id());
Term uidTerm = new Term(UidFieldMapper.NAME, Uid.createUidAsBytes(request.type(), request.id()));
Engine.GetResult result = indexShard.get(new Engine.Get(false, uidTerm));
if (!result.exists()) {

View File

@ -152,7 +152,7 @@ public class TransportFieldStatsTransportAction extends TransportBroadcastAction
Map<String, FieldStats> fieldStats = new HashMap<>();
IndexService indexServices = indicesService.indexServiceSafe(shardId.getIndex());
MapperService mapperService = indexServices.mapperService();
IndexShard shard = indexServices.shardSafe(shardId.id());
IndexShard shard = indexServices.getShard(shardId.id());
try (Engine.Searcher searcher = shard.acquireSearcher("fieldstats")) {
for (String field : request.getFields()) {
MappedFieldType fieldType = mapperService.fullName(field);

View File

@ -92,7 +92,7 @@ public class TransportGetAction extends TransportSingleShardAction<GetRequest, G
@Override
protected GetResponse shardOperation(GetRequest request, ShardId shardId) {
IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
IndexShard indexShard = indexService.shardSafe(shardId.id());
IndexShard indexShard = indexService.getShard(shardId.id());
if (request.refresh() && !request.realtime()) {
indexShard.refresh("refresh_flag_get");

View File

@ -87,7 +87,7 @@ public class TransportShardMultiGetAction extends TransportSingleShardAction<Mul
@Override
protected MultiGetShardResponse shardOperation(MultiGetShardRequest request, ShardId shardId) {
IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
IndexShard indexShard = indexService.shardSafe(shardId.id());
IndexShard indexShard = indexService.getShard(shardId.id());
if (request.refresh() && !request.realtime()) {
indexShard.refresh("refresh_flag_mget");

View File

@ -164,7 +164,7 @@ public class TransportIndexAction extends TransportReplicationAction<IndexReques
}
IndexService indexService = indicesService.indexServiceSafe(shardRequest.shardId.getIndex());
IndexShard indexShard = indexService.shardSafe(shardRequest.shardId.id());
IndexShard indexShard = indexService.getShard(shardRequest.shardId.id());
final WriteResult<IndexResponse> result = executeIndexRequestOnPrimary(null, request, indexShard);
final IndexResponse response = result.response;
@ -176,7 +176,7 @@ public class TransportIndexAction extends TransportReplicationAction<IndexReques
@Override
protected void shardOperationOnReplica(ShardId shardId, IndexRequest request) {
IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
IndexShard indexShard = indexService.shardSafe(shardId.id());
IndexShard indexShard = indexService.getShard(shardId.id());
SourceToParse sourceToParse = SourceToParse.source(SourceToParse.Origin.REPLICA, request.source()).index(shardId.getIndex()).type(request.type()).id(request.id())
.routing(request.routing()).parent(request.parent()).timestamp(request.timestamp()).ttl(request.ttl());

View File

@ -130,7 +130,7 @@ public class TransportSuggestAction extends TransportBroadcastAction<SuggestRequ
@Override
protected ShardSuggestResponse shardOperation(ShardSuggestRequest request) {
IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
IndexShard indexShard = indexService.shardSafe(request.shardId().id());
IndexShard indexShard = indexService.getShard(request.shardId().id());
ShardSuggestMetric suggestMetric = indexShard.getSuggestMetric();
suggestMetric.preSuggest();
long startTime = System.nanoTime();

View File

@ -666,7 +666,7 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
protected Releasable getIndexShardOperationsCounter(ShardId shardId) {
IndexService indexService = indicesService.indexServiceSafe(shardId.index().getName());
IndexShard indexShard = indexService.shardSafe(shardId.id());
IndexShard indexShard = indexService.getShard(shardId.id());
return new IndexShardReference(indexShard);
}
@ -678,7 +678,7 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
logger.debug("ignoring failed replica [{}][{}] because index was already removed.", index, shardId);
return;
}
IndexShard indexShard = indexService.shard(shardId);
IndexShard indexShard = indexService.getShardOrNull(shardId);
if (indexShard == null) {
logger.debug("ignoring failed replica [{}][{}] because index was already removed.", index, shardId);
return;

View File

@ -79,7 +79,7 @@ public class TransportShardMultiTermsVectorAction extends TransportSingleShardAc
TermVectorsRequest termVectorsRequest = request.requests.get(i);
try {
IndexService indexService = indicesService.indexServiceSafe(request.index());
IndexShard indexShard = indexService.shardSafe(shardId.id());
IndexShard indexShard = indexService.getShard(shardId.id());
TermVectorsResponse termVectorsResponse = indexShard.getTermVectors(termVectorsRequest);
termVectorsResponse.updateTookInMillis(termVectorsRequest.startTime());
response.add(request.locations.get(i), termVectorsResponse);

View File

@ -82,7 +82,7 @@ public class TransportTermVectorsAction extends TransportSingleShardAction<TermV
@Override
protected TermVectorsResponse shardOperation(TermVectorsRequest request, ShardId shardId) {
IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
IndexShard indexShard = indexService.shardSafe(shardId.id());
IndexShard indexShard = indexService.getShard(shardId.id());
TermVectorsResponse response = indexShard.getTermVectors(request);
response.updateTookInMillis(request.startTime());
return response;

View File

@ -166,7 +166,7 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
protected void shardOperation(final UpdateRequest request, final ActionListener<UpdateResponse> listener, final int retryCount) {
IndexService indexService = indicesService.indexServiceSafe(request.concreteIndex());
IndexShard indexShard = indexService.shardSafe(request.shardId());
IndexShard indexShard = indexService.getShard(request.shardId());
final UpdateHelper.Result result = updateHelper.prepare(request, indexShard);
switch (result.operation()) {
case UPSERT:
@ -266,7 +266,7 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
UpdateResponse update = result.action();
IndexService indexServiceOrNull = indicesService.indexService(request.concreteIndex());
if (indexServiceOrNull != null) {
IndexShard shard = indexService.shard(request.shardId());
IndexShard shard = indexService.getShardOrNull(request.shardId());
if (shard != null) {
shard.indexingService().noopUpdate(request.type());
}

View File

@ -45,9 +45,16 @@ public final class BootstrapInfo {
}
/**
* Returns true if secure computing mode is enabled (linux/amd64 only)
* Returns true if secure computing mode is enabled (linux/amd64, OS X only)
*/
public static boolean isSeccompInstalled() {
return Natives.isSeccompInstalled();
}
/**
* codebase location for untrusted scripts (provide some additional safety)
* <p>
* This is not a full URL, just a path.
*/
public static final String UNTRUSTED_CODEBASE = "/untrusted";
}

View File

@ -26,29 +26,27 @@ import java.net.URL;
import java.security.CodeSource;
import java.security.Permission;
import java.security.PermissionCollection;
import java.security.Permissions;
import java.security.Policy;
import java.security.ProtectionDomain;
import java.security.URIParameter;
import java.util.PropertyPermission;
/** custom policy for union of static and dynamic permissions */
final class ESPolicy extends Policy {
/** template policy file, the one used in tests */
static final String POLICY_RESOURCE = "security.policy";
/** limited policy for groovy scripts */
static final String GROOVY_RESOURCE = "groovy.policy";
/** limited policy for scripts */
static final String UNTRUSTED_RESOURCE = "untrusted.policy";
final Policy template;
final Policy groovy;
final Policy untrusted;
final PermissionCollection dynamic;
public ESPolicy(PermissionCollection dynamic) throws Exception {
URI policyUri = getClass().getResource(POLICY_RESOURCE).toURI();
URI groovyUri = getClass().getResource(GROOVY_RESOURCE).toURI();
URI untrustedUri = getClass().getResource(UNTRUSTED_RESOURCE).toURI();
this.template = Policy.getInstance("JavaPolicy", new URIParameter(policyUri));
this.groovy = Policy.getInstance("JavaPolicy", new URIParameter(groovyUri));
this.untrusted = Policy.getInstance("JavaPolicy", new URIParameter(untrustedUri));
this.dynamic = dynamic;
}
@ -56,15 +54,17 @@ final class ESPolicy extends Policy {
public boolean implies(ProtectionDomain domain, Permission permission) {
CodeSource codeSource = domain.getCodeSource();
// codesource can be null when reducing privileges via doPrivileged()
if (codeSource != null) {
URL location = codeSource.getLocation();
// location can be null... ??? nobody knows
// https://bugs.openjdk.java.net/browse/JDK-8129972
if (location != null) {
// run groovy scripts with no permissions (except logging property)
if ("/groovy/script".equals(location.getFile())) {
return groovy.implies(domain, permission);
}
if (codeSource == null) {
return false;
}
URL location = codeSource.getLocation();
// location can be null... ??? nobody knows
// https://bugs.openjdk.java.net/browse/JDK-8129972
if (location != null) {
// run scripts with limited permissions
if (BootstrapInfo.UNTRUSTED_CODEBASE.equals(location.getFile())) {
return untrusted.implies(domain, permission);
}
}

View File

@ -203,6 +203,9 @@ public class MetaDataCreateIndexService extends AbstractComponent {
if (state.metaData().hasAlias(index)) {
throw new InvalidIndexNameException(new Index(index), index, "already exists as alias");
}
if (index.equals(".") || index.equals("..")) {
throw new InvalidIndexNameException(new Index(index), index, "must not be '.' or '..'");
}
}
private void createIndex(final CreateIndexClusterStateUpdateRequest request, final ActionListener<ClusterStateUpdateResponse> listener, final Semaphore mdLock) {

View File

@ -598,12 +598,12 @@ public class DiskThresholdDecider extends AllocationDecider {
return allocation.decision(Decision.YES, NAME, "disk threshold decider disabled");
}
// Allow allocation regardless if only a single node is available
if (allocation.nodes().size() <= 1) {
// Allow allocation regardless if only a single data node is available
if (allocation.nodes().dataNodes().size() <= 1) {
if (logger.isTraceEnabled()) {
logger.trace("only a single node is present, allowing allocation");
logger.trace("only a single data node is present, allowing allocation");
}
return allocation.decision(Decision.YES, NAME, "only a single node is present");
return allocation.decision(Decision.YES, NAME, "only a single data node is present");
}
// Fail open there is no info available

View File

@ -0,0 +1,77 @@
/*
* 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.hash;
import org.elasticsearch.ElasticsearchException;
import java.security.MessageDigest;
import java.security.NoSuchAlgorithmException;
public class MessageDigests {
private static final MessageDigest MD5_DIGEST;
private static final MessageDigest SHA_1_DIGEST;
private static final MessageDigest SHA_256_DIGEST;
static {
try {
MD5_DIGEST = MessageDigest.getInstance("MD5");
SHA_1_DIGEST = MessageDigest.getInstance("SHA-1");
SHA_256_DIGEST = MessageDigest.getInstance("SHA-256");
} catch (NoSuchAlgorithmException e) {
throw new ElasticsearchException("Unexpected exception creating MessageDigest instance", e);
}
}
public static MessageDigest md5() {
return clone(MD5_DIGEST);
}
public static MessageDigest sha1() {
return clone(SHA_1_DIGEST);
}
public static MessageDigest sha256() {
return clone(SHA_256_DIGEST);
}
private static MessageDigest clone(MessageDigest messageDigest) {
try {
return (MessageDigest) messageDigest.clone();
} catch (CloneNotSupportedException e) {
throw new ElasticsearchException("Unexpected exception cloning MessageDigest instance", e);
}
}
private static final char[] HEX_DIGITS = "0123456789abcdef".toCharArray();
public static String toHexString(byte[] bytes) {
if (bytes == null) {
throw new NullPointerException("bytes");
}
StringBuilder sb = new StringBuilder(2 * bytes.length);
for (int i = 0; i < bytes.length; i++) {
byte b = bytes[i];
sb.append(HEX_DIGITS[b >> 4 & 0xf]).append(HEX_DIGITS[b & 0xf]);
}
return sb.toString();
}
}

View File

@ -19,19 +19,22 @@
package org.elasticsearch.common.http.client;
import java.nio.charset.StandardCharsets;
import com.google.common.hash.Hashing;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.*;
import org.elasticsearch.Build;
import org.elasticsearch.ElasticsearchCorruptionException;
import org.elasticsearch.ElasticsearchTimeoutException;
import org.elasticsearch.Version;
import org.elasticsearch.common.Base64;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.hash.MessageDigests;
import org.elasticsearch.common.unit.TimeValue;
import java.io.*;
import java.net.HttpURLConnection;
import java.net.URL;
import java.net.URLConnection;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.NoSuchFileException;
import java.nio.file.Path;
@ -96,7 +99,7 @@ public class HttpDownloadHelper {
public static Checksummer SHA1_CHECKSUM = new Checksummer() {
@Override
public String checksum(byte[] filebytes) {
return Hashing.sha1().hashBytes(filebytes).toString();
return MessageDigests.toHexString(MessageDigests.sha1().digest(filebytes));
}
@Override
@ -109,7 +112,7 @@ public class HttpDownloadHelper {
public static Checksummer MD5_CHECKSUM = new Checksummer() {
@Override
public String checksum(byte[] filebytes) {
return Hashing.md5().hashBytes(filebytes).toString();
return MessageDigests.toHexString(MessageDigests.md5().digest(filebytes));
}
@Override

View File

@ -90,12 +90,14 @@ public class LogConfigurator {
loaded = true;
// TODO: this is partly a copy of InternalSettingsPreparer...we should pass in Environment and not do all this...
Environment environment = new Environment(settings);
Settings.Builder settingsBuilder = settingsBuilder().put(settings);
Settings.Builder settingsBuilder = settingsBuilder();
resolveConfig(environment, settingsBuilder);
settingsBuilder
.putProperties("elasticsearch.", System.getProperties())
.putProperties("es.", System.getProperties())
.replacePropertyPlaceholders();
.putProperties("es.", System.getProperties());
// add custom settings after config was added so that they are not overwritten by config
settingsBuilder.put(settings);
settingsBuilder.replacePropertyPlaceholders();
Properties props = new Properties();
for (Map.Entry<String, String> entry : settingsBuilder.build().getAsMap().entrySet()) {
String key = "log4j." + entry.getKey();

View File

@ -20,21 +20,37 @@
package org.elasticsearch.index;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.inject.util.Providers;
import org.elasticsearch.index.aliases.IndexAliasesService;
import org.elasticsearch.index.engine.EngineFactory;
import org.elasticsearch.index.engine.InternalEngineFactory;
import org.elasticsearch.index.fielddata.IndexFieldDataService;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.shard.IndexSearcherWrapper;
/**
*
*/
public class IndexModule extends AbstractModule {
private final Settings settings;
public IndexModule(Settings settings) {
this.settings = settings;
}
// pkg private so tests can mock
Class<? extends EngineFactory> engineFactoryImpl = InternalEngineFactory.class;
Class<? extends IndexSearcherWrapper> indexSearcherWrapper = null;
@Override
protected void configure() {
bind(EngineFactory.class).to(engineFactoryImpl).asEagerSingleton();
if (indexSearcherWrapper == null) {
bind(IndexSearcherWrapper.class).toProvider(Providers.of(null));
} else {
bind(IndexSearcherWrapper.class).to(indexSearcherWrapper).asEagerSingleton();
}
bind(IndexService.class).asEagerSingleton();
bind(IndexServicesProvider.class).asEagerSingleton();
bind(MapperService.class).asEagerSingleton();
bind(IndexAliasesService.class).asEagerSingleton();
bind(IndexFieldDataService.class).asEagerSingleton();
}
}

View File

@ -24,16 +24,9 @@ import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.inject.CreationException;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.inject.Injector;
import org.elasticsearch.common.inject.Injectors;
import org.elasticsearch.common.inject.Module;
import org.elasticsearch.common.inject.ModulesBuilder;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.env.ShardLock;
@ -49,20 +42,12 @@ import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.query.IndexQueryParserService;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.settings.IndexSettingsService;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.IndexShardModule;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardNotFoundException;
import org.elasticsearch.index.shard.ShardPath;
import org.elasticsearch.index.shard.*;
import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.index.store.IndexStore;
import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.store.StoreModule;
import org.elasticsearch.indices.IndicesLifecycle;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.InternalIndicesLifecycle;
import org.elasticsearch.indices.cache.query.IndicesQueryCache;
import org.elasticsearch.plugins.PluginsService;
import java.io.Closeable;
import java.io.IOException;
@ -81,86 +66,42 @@ import static org.elasticsearch.common.collect.MapBuilder.newMapBuilder;
*/
public class IndexService extends AbstractIndexComponent implements IndexComponent, Iterable<IndexShard> {
private final Injector injector;
private final Settings indexSettings;
private final PluginsService pluginsService;
private final InternalIndicesLifecycle indicesLifecycle;
private final AnalysisService analysisService;
private final MapperService mapperService;
private final IndexQueryParserService queryParserService;
private final SimilarityService similarityService;
private final IndexAliasesService aliasesService;
private final IndexCache indexCache;
private final IndexFieldDataService indexFieldData;
private final BitsetFilterCache bitsetFilterCache;
private final IndexSettingsService settingsService;
private final NodeEnvironment nodeEnv;
private final IndicesService indicesServices;
private volatile ImmutableMap<Integer, IndexShardInjectorPair> shards = ImmutableMap.of();
private static class IndexShardInjectorPair {
private final IndexShard indexShard;
private final Injector injector;
public IndexShardInjectorPair(IndexShard indexShard, Injector injector) {
this.indexShard = indexShard;
this.injector = injector;
}
public IndexShard getIndexShard() {
return indexShard;
}
public Injector getInjector() {
return injector;
}
}
private final IndexServicesProvider indexServicesProvider;
private final IndexStore indexStore;
private volatile ImmutableMap<Integer, IndexShard> shards = ImmutableMap.of();
private final AtomicBoolean closed = new AtomicBoolean(false);
private final AtomicBoolean deleted = new AtomicBoolean(false);
@Inject
public IndexService(Injector injector, Index index, @IndexSettings Settings indexSettings, NodeEnvironment nodeEnv,
AnalysisService analysisService, MapperService mapperService, IndexQueryParserService queryParserService,
SimilarityService similarityService, IndexAliasesService aliasesService, IndexCache indexCache,
public IndexService(Index index, @IndexSettings Settings indexSettings, NodeEnvironment nodeEnv,
AnalysisService analysisService,
IndexSettingsService settingsService,
IndexFieldDataService indexFieldData, BitsetFilterCache bitSetFilterCache, IndicesService indicesServices) {
IndexFieldDataService indexFieldData,
BitsetFilterCache bitSetFilterCache,
IndicesService indicesServices,
IndexServicesProvider indexServicesProvider,
IndexStore indexStore) {
super(index, indexSettings);
this.injector = injector;
this.indexSettings = indexSettings;
this.analysisService = analysisService;
this.mapperService = mapperService;
this.queryParserService = queryParserService;
this.similarityService = similarityService;
this.aliasesService = aliasesService;
this.indexCache = indexCache;
this.indexFieldData = indexFieldData;
this.settingsService = settingsService;
this.bitsetFilterCache = bitSetFilterCache;
this.pluginsService = injector.getInstance(PluginsService.class);
this.indicesServices = indicesServices;
this.indicesLifecycle = (InternalIndicesLifecycle) injector.getInstance(IndicesLifecycle.class);
// inject workarounds for cyclic dep
this.indicesLifecycle = (InternalIndicesLifecycle) indexServicesProvider.getIndicesLifecycle();
this.nodeEnv = nodeEnv;
this.indexServicesProvider = indexServicesProvider;
this.indexStore = indexStore;
indexFieldData.setListener(new FieldDataCacheListener(this));
bitSetFilterCache.setListener(new BitsetCacheListener(this));
this.nodeEnv = nodeEnv;
}
public int numberOfShards() {
@ -173,7 +114,7 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
@Override
public Iterator<IndexShard> iterator() {
return shards.values().stream().map((p) -> p.getIndexShard()).iterator();
return shards.values().iterator();
}
public boolean hasShard(int shardId) {
@ -184,19 +125,15 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
* Return the shard with the provided id, or null if there is no such shard.
*/
@Nullable
public IndexShard shard(int shardId) {
IndexShardInjectorPair indexShardInjectorPair = shards.get(shardId);
if (indexShardInjectorPair != null) {
return indexShardInjectorPair.getIndexShard();
}
return null;
public IndexShard getShardOrNull(int shardId) {
return shards.get(shardId);
}
/**
* Return the shard with the provided id, or throw an exception if it doesn't exist.
*/
public IndexShard shardSafe(int shardId) {
IndexShard indexShard = shard(shardId);
public IndexShard getShard(int shardId) {
IndexShard indexShard = getShardOrNull(shardId);
if (indexShard == null) {
throw new ShardNotFoundException(new ShardId(index, shardId));
}
@ -207,16 +144,12 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
return shards.keySet();
}
public Injector injector() {
return injector;
}
public IndexSettingsService settingsService() {
return this.settingsService;
}
public IndexCache cache() {
return indexCache;
return indexServicesProvider.getIndexCache();
}
public IndexFieldDataService fieldData() {
@ -232,19 +165,19 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
}
public MapperService mapperService() {
return mapperService;
return indexServicesProvider.getMapperService();
}
public IndexQueryParserService queryParserService() {
return queryParserService;
return indexServicesProvider.getQueryParserService();
}
public SimilarityService similarityService() {
return similarityService;
return indexServicesProvider.getSimilarityService();
}
public IndexAliasesService aliasesService() {
return aliasesService;
return indexServicesProvider.getIndexAliasesService();
}
public synchronized void close(final String reason, boolean delete) {
@ -261,16 +194,6 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
}
}
/**
* Return the shard injector for the provided id, or throw an exception if there is no such shard.
*/
public Injector shardInjectorSafe(int shardId) {
IndexShardInjectorPair indexShardInjectorPair = shards.get(shardId);
if (indexShardInjectorPair == null) {
throw new ShardNotFoundException(new ShardId(index, shardId));
}
return indexShardInjectorPair.getInjector();
}
public String indexUUID() {
return indexSettings.get(IndexMetaData.SETTING_INDEX_UUID, IndexMetaData.INDEX_UUID_NA_VALUE);
@ -301,10 +224,14 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
if (closed.get()) {
throw new IllegalStateException("Can't create shard [" + index.name() + "][" + sShardId + "], closed");
}
if (indexSettings.get("index.translog.type") != null) { // TODO remove?
throw new IllegalStateException("a custom translog type is no longer supported. got [" + indexSettings.get("index.translog.type") + "]");
}
final ShardId shardId = new ShardId(index, sShardId);
ShardLock lock = null;
boolean success = false;
Injector shardInjector = null;
Store store = null;
IndexShard indexShard = null;
try {
lock = nodeEnv.shardLock(shardId, TimeUnit.SECONDS.toMillis(5));
indicesLifecycle.beforeIndexShardCreated(shardId, indexSettings);
@ -325,7 +252,6 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
if (path == null) {
// TODO: we should, instead, hold a "bytes reserved" of how large we anticipate this shard will be, e.g. for a shard
// that's being relocated/replicated we know how large it will become once it's done copying:
// Count up how many shards are currently on each data path:
Map<Path,Integer> dataPathToShardCount = new HashMap<>();
for(IndexShard shard : this) {
@ -351,39 +277,17 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
// if we are on a shared FS we only own the shard (ie. we can safely delete it) if we are the primary.
final boolean canDeleteShardContent = IndexMetaData.isOnSharedFilesystem(indexSettings) == false ||
(primary && IndexMetaData.isOnSharedFilesystem(indexSettings));
ModulesBuilder modules = new ModulesBuilder();
// plugin modules must be added here, before others or we can get crazy injection errors...
for (Module pluginModule : pluginsService.shardModules(indexSettings)) {
modules.add(pluginModule);
}
modules.add(new IndexShardModule(shardId, primary, indexSettings));
modules.add(new StoreModule(injector.getInstance(IndexStore.class).shardDirectory(), lock,
new StoreCloseListener(shardId, canDeleteShardContent, new Closeable() {
@Override
public void close() throws IOException {
injector.getInstance(IndicesQueryCache.class).onClose(shardId);
}
}), path));
pluginsService.processModules(modules);
try {
shardInjector = modules.createChildInjector(injector);
} catch (CreationException e) {
ElasticsearchException ex = new ElasticsearchException("failed to create shard", Injectors.getFirstErrorFailure(e));
ex.setShard(shardId);
throw ex;
} catch (Throwable e) {
ElasticsearchException ex = new ElasticsearchException("failed to create shard", e);
ex.setShard(shardId);
throw ex;
store = new Store(shardId, indexSettings, indexStore.newDirectoryService(path), lock, new StoreCloseListener(shardId, canDeleteShardContent, () -> indexServicesProvider.getIndicesQueryCache().onClose(shardId)));
if (useShadowEngine(primary, indexSettings)) {
indexShard = new ShadowIndexShard(shardId, indexSettings, path, store, indexServicesProvider);
} else {
indexShard = new IndexShard(shardId, indexSettings, path, store, indexServicesProvider);
}
IndexShard indexShard = shardInjector.getInstance(IndexShard.class);
indicesLifecycle.indexShardStateChanged(indexShard, null, "shard created");
indicesLifecycle.afterIndexShardCreated(indexShard);
shards = newMapBuilder(shards).put(shardId.id(), new IndexShardInjectorPair(indexShard, shardInjector)).immutableMap();
settingsService.addListener(indexShard);
shards = newMapBuilder(shards).put(shardId.id(), indexShard).immutableMap();
success = true;
return indexShard;
} catch (IOException e) {
@ -393,45 +297,35 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
} finally {
if (success == false) {
IOUtils.closeWhileHandlingException(lock);
if (shardInjector != null) {
IndexShard indexShard = shardInjector.getInstance(IndexShard.class);
closeShardInjector("initialization failed", shardId, shardInjector, indexShard);
}
closeShard("initialization failed", shardId, indexShard, store);
}
}
}
static boolean useShadowEngine(boolean primary, Settings indexSettings) {
return primary == false && IndexMetaData.isIndexUsingShadowReplicas(indexSettings);
}
public synchronized void removeShard(int shardId, String reason) {
final ShardId sId = new ShardId(index, shardId);
final Injector shardInjector;
final IndexShard indexShard;
if (shards.containsKey(shardId) == false) {
return;
}
logger.debug("[{}] closing... (reason: [{}])", shardId, reason);
HashMap<Integer, IndexShardInjectorPair> tmpShardsMap = new HashMap<>(shards);
IndexShardInjectorPair indexShardInjectorPair = tmpShardsMap.remove(shardId);
indexShard = indexShardInjectorPair.getIndexShard();
shardInjector = indexShardInjectorPair.getInjector();
HashMap<Integer, IndexShard> tmpShardsMap = new HashMap<>(shards);
indexShard = tmpShardsMap.remove(shardId);
shards = ImmutableMap.copyOf(tmpShardsMap);
closeShardInjector(reason, sId, shardInjector, indexShard);
closeShard(reason, sId, indexShard, indexShard.store());
logger.debug("[{}] closed (reason: [{}])", shardId, reason);
}
private void closeShardInjector(String reason, ShardId sId, Injector shardInjector, IndexShard indexShard) {
private void closeShard(String reason, ShardId sId, IndexShard indexShard, Store store) {
final int shardId = sId.id();
try {
try {
indicesLifecycle.beforeIndexShardClosed(sId, indexShard, indexSettings);
} finally {
// close everything else even if the beforeIndexShardClosed threw an exception
for (Class<? extends Closeable> closeable : pluginsService.shardServices()) {
try {
shardInjector.getInstance(closeable).close();
} catch (Throwable e) {
logger.debug("[{}] failed to clean plugin shard service [{}]", e, shardId, closeable);
}
}
// this logic is tricky, we want to close the engine so we rollback the changes done to it
// and close the shard so no operations are allowed to it
if (indexShard != null) {
@ -449,30 +343,13 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
}
} finally {
try {
shardInjector.getInstance(Store.class).close();
store.close();
} catch (Throwable e) {
logger.warn("[{}] failed to close store on shard removal (reason: [{}])", e, shardId, reason);
}
}
}
/**
* Closes an optional resource. Returns true if the resource was found;
* NOTE: this method swallows all exceptions thrown from the close method of the injector and logs them as debug log
*/
private boolean closeInjectorOptionalResource(ShardId shardId, Injector shardInjector, Class<? extends Closeable> toClose) {
try {
final Closeable instance = shardInjector.getInstance(toClose);
if (instance == null) {
return false;
}
IOUtils.close(instance);
} catch (Throwable t) {
logger.debug("{} failed to close {}", t, shardId, Strings.toUnderscoreCase(toClose.getSimpleName()));
}
return true;
}
private void onShardClose(ShardLock lock, boolean ownsShard) {
if (deleted.get()) { // we remove that shards content if this index has been deleted
@ -492,6 +369,10 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
}
}
public IndexServicesProvider getIndexServices() {
return indexServicesProvider;
}
private class StoreCloseListener implements Store.OnClose {
private final ShardId shardId;
private final boolean ownsShard;
@ -533,7 +414,7 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
@Override
public void onCache(ShardId shardId, Accountable accountable) {
if (shardId != null) {
final IndexShard shard = indexService.shard(shardId.id());
final IndexShard shard = indexService.getShardOrNull(shardId.id());
if (shard != null) {
long ramBytesUsed = accountable != null ? accountable.ramBytesUsed() : 0l;
shard.shardBitsetFilterCache().onCached(ramBytesUsed);
@ -544,7 +425,7 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
@Override
public void onRemoval(ShardId shardId, Accountable accountable) {
if (shardId != null) {
final IndexShard shard = indexService.shard(shardId.id());
final IndexShard shard = indexService.getShardOrNull(shardId.id());
if (shard != null) {
long ramBytesUsed = accountable != null ? accountable.ramBytesUsed() : 0l;
shard.shardBitsetFilterCache().onRemoval(ramBytesUsed);
@ -563,7 +444,7 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
@Override
public void onCache(ShardId shardId, MappedFieldType.Names fieldNames, FieldDataType fieldDataType, Accountable ramUsage) {
if (shardId != null) {
final IndexShard shard = indexService.shard(shardId.id());
final IndexShard shard = indexService.getShardOrNull(shardId.id());
if (shard != null) {
shard.fieldData().onCache(shardId, fieldNames, fieldDataType, ramUsage);
}
@ -573,7 +454,7 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
@Override
public void onRemoval(ShardId shardId, MappedFieldType.Names fieldNames, FieldDataType fieldDataType, boolean wasEvicted, long sizeInBytes) {
if (shardId != null) {
final IndexShard shard = indexService.shard(shardId.id());
final IndexShard shard = indexService.getShardOrNull(shardId.id());
if (shard != null) {
shard.fieldData().onRemoval(shardId, fieldNames, fieldDataType, wasEvicted, sizeInBytes);
}

View File

@ -0,0 +1,148 @@
/*
* 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.index;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.index.aliases.IndexAliasesService;
import org.elasticsearch.index.cache.IndexCache;
import org.elasticsearch.index.codec.CodecService;
import org.elasticsearch.index.engine.EngineFactory;
import org.elasticsearch.index.fielddata.IndexFieldDataService;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.query.IndexQueryParserService;
import org.elasticsearch.index.shard.IndexSearcherWrapper;
import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.index.termvectors.TermVectorsService;
import org.elasticsearch.indices.IndicesLifecycle;
import org.elasticsearch.indices.IndicesWarmer;
import org.elasticsearch.indices.cache.query.IndicesQueryCache;
import org.elasticsearch.indices.memory.IndexingMemoryController;
import org.elasticsearch.threadpool.ThreadPool;
/**
* Simple provider class that holds the Index and Node level services used by
* a shard.
* This is just a temporary solution until we cleaned up index creation and removed injectors on that level as well.
*/
public final class IndexServicesProvider {
private final IndicesLifecycle indicesLifecycle;
private final ThreadPool threadPool;
private final MapperService mapperService;
private final IndexQueryParserService queryParserService;
private final IndexCache indexCache;
private final IndexAliasesService indexAliasesService;
private final IndicesQueryCache indicesQueryCache;
private final CodecService codecService;
private final TermVectorsService termVectorsService;
private final IndexFieldDataService indexFieldDataService;
private final IndicesWarmer warmer;
private final SimilarityService similarityService;
private final EngineFactory factory;
private final BigArrays bigArrays;
private final IndexSearcherWrapper indexSearcherWrapper;
private final IndexingMemoryController indexingMemoryController;
@Inject
public IndexServicesProvider(IndicesLifecycle indicesLifecycle, ThreadPool threadPool, MapperService mapperService, IndexQueryParserService queryParserService, IndexCache indexCache, IndexAliasesService indexAliasesService, IndicesQueryCache indicesQueryCache, CodecService codecService, TermVectorsService termVectorsService, IndexFieldDataService indexFieldDataService, @Nullable IndicesWarmer warmer, SimilarityService similarityService, EngineFactory factory, BigArrays bigArrays, @Nullable IndexSearcherWrapper indexSearcherWrapper, IndexingMemoryController indexingMemoryController) {
this.indicesLifecycle = indicesLifecycle;
this.threadPool = threadPool;
this.mapperService = mapperService;
this.queryParserService = queryParserService;
this.indexCache = indexCache;
this.indexAliasesService = indexAliasesService;
this.indicesQueryCache = indicesQueryCache;
this.codecService = codecService;
this.termVectorsService = termVectorsService;
this.indexFieldDataService = indexFieldDataService;
this.warmer = warmer;
this.similarityService = similarityService;
this.factory = factory;
this.bigArrays = bigArrays;
this.indexSearcherWrapper = indexSearcherWrapper;
this.indexingMemoryController = indexingMemoryController;
}
public IndicesLifecycle getIndicesLifecycle() {
return indicesLifecycle;
}
public ThreadPool getThreadPool() {
return threadPool;
}
public MapperService getMapperService() {
return mapperService;
}
public IndexQueryParserService getQueryParserService() {
return queryParserService;
}
public IndexCache getIndexCache() {
return indexCache;
}
public IndexAliasesService getIndexAliasesService() {
return indexAliasesService;
}
public IndicesQueryCache getIndicesQueryCache() {
return indicesQueryCache;
}
public CodecService getCodecService() {
return codecService;
}
public TermVectorsService getTermVectorsService() {
return termVectorsService;
}
public IndexFieldDataService getIndexFieldDataService() {
return indexFieldDataService;
}
public IndicesWarmer getWarmer() {
return warmer;
}
public SimilarityService getSimilarityService() {
return similarityService;
}
public EngineFactory getFactory() {
return factory;
}
public BigArrays getBigArrays() {
return bigArrays;
}
public IndexSearcherWrapper getIndexSearcherWrapper() {
return indexSearcherWrapper;
}
public IndexingMemoryController getIndexingMemoryController() {
return indexingMemoryController;
}
}

View File

@ -1,33 +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.index.aliases;
import org.elasticsearch.common.inject.AbstractModule;
/**
*
*/
public class IndexAliasesServiceModule extends AbstractModule {
@Override
protected void configure() {
bind(IndexAliasesService.class).asEagerSingleton();
}
}

View File

@ -59,6 +59,8 @@ import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.Function;
import java.util.function.Supplier;
/**
*
@ -78,7 +80,6 @@ public abstract class Engine implements Closeable {
protected final ReentrantReadWriteLock rwl = new ReentrantReadWriteLock();
protected final ReleasableLock readLock = new ReleasableLock(rwl.readLock());
protected final ReleasableLock writeLock = new ReleasableLock(rwl.writeLock());
protected volatile Throwable failedEngine = null;
protected Engine(EngineConfig engineConfig) {
@ -227,8 +228,8 @@ public abstract class Engine implements Closeable {
PENDING_OPERATIONS
}
final protected GetResult getFromSearcher(Get get) throws EngineException {
final Searcher searcher = acquireSearcher("get");
final protected GetResult getFromSearcher(Get get, Function<String, Searcher> searcherFactory) throws EngineException {
final Searcher searcher = searcherFactory.apply("get");
final Versions.DocIdAndVersion docIdAndVersion;
try {
docIdAndVersion = Versions.loadDocIdAndVersion(searcher.reader(), get.uid());
@ -256,7 +257,11 @@ public abstract class Engine implements Closeable {
}
}
public abstract GetResult get(Get get) throws EngineException;
public final GetResult get(Get get) throws EngineException {
return get(get, this::acquireSearcher);
}
public abstract GetResult get(Get get, Function<String, Searcher> searcherFactory) throws EngineException;
/**
* Returns a new searcher instance. The consumer of this
@ -279,7 +284,7 @@ public abstract class Engine implements Closeable {
try {
final Searcher retVal = newSearcher(source, searcher, manager);
success = true;
return config().getWrappingService().wrap(engineConfig, retVal);
return retVal;
} finally {
if (!success) {
manager.release(searcher);

View File

@ -25,6 +25,7 @@ import org.apache.lucene.index.SnapshotDeletionPolicy;
import org.apache.lucene.search.QueryCache;
import org.apache.lucene.search.QueryCachingPolicy;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.util.SetOnce;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
@ -32,6 +33,7 @@ import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.codec.CodecService;
import org.elasticsearch.index.indexing.ShardIndexingService;
import org.elasticsearch.index.shard.IndexSearcherWrapper;
import org.elasticsearch.index.shard.MergeSchedulerConfig;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.TranslogRecoveryPerformer;
@ -73,7 +75,7 @@ public final class EngineConfig {
private final boolean forceNewTranslog;
private final QueryCache queryCache;
private final QueryCachingPolicy queryCachingPolicy;
private final IndexSearcherWrappingService wrappingService;
private final SetOnce<IndexSearcherWrapper> searcherWrapper = new SetOnce<>();
/**
* Index setting for compound file on flush. This setting is realtime updateable.
@ -121,7 +123,7 @@ public final class EngineConfig {
Settings indexSettings, IndicesWarmer warmer, Store store, SnapshotDeletionPolicy deletionPolicy,
MergePolicy mergePolicy, MergeSchedulerConfig mergeSchedulerConfig, Analyzer analyzer,
Similarity similarity, CodecService codecService, Engine.FailedEngineListener failedEngineListener,
TranslogRecoveryPerformer translogRecoveryPerformer, QueryCache queryCache, QueryCachingPolicy queryCachingPolicy, IndexSearcherWrappingService wrappingService, TranslogConfig translogConfig) {
TranslogRecoveryPerformer translogRecoveryPerformer, QueryCache queryCache, QueryCachingPolicy queryCachingPolicy, TranslogConfig translogConfig) {
this.shardId = shardId;
this.indexSettings = indexSettings;
this.threadPool = threadPool;
@ -135,7 +137,6 @@ public final class EngineConfig {
this.similarity = similarity;
this.codecService = codecService;
this.failedEngineListener = failedEngineListener;
this.wrappingService = wrappingService;
this.compoundOnFlush = indexSettings.getAsBoolean(EngineConfig.INDEX_COMPOUND_ON_FLUSH, compoundOnFlush);
codecName = indexSettings.get(EngineConfig.INDEX_CODEC_SETTING, EngineConfig.DEFAULT_CODEC_NAME);
indexingBufferSize = DEFAULT_INDEX_BUFFER_SIZE;
@ -380,10 +381,6 @@ public final class EngineConfig {
return queryCachingPolicy;
}
public IndexSearcherWrappingService getWrappingService() {
return wrappingService;
}
/**
* Returns the translog config for this engine
*/

View File

@ -1,47 +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.index.engine;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.search.IndexSearcher;
/**
* Extension point to add custom functionality at request time to the {@link DirectoryReader}
* and {@link IndexSearcher} managed by the {@link Engine}.
*/
public interface IndexSearcherWrapper {
/**
* @param reader The provided directory reader to be wrapped to add custom functionality
* @return a new directory reader wrapping the provided directory reader or if no wrapping was performed
* the provided directory reader
*/
DirectoryReader wrap(DirectoryReader reader);
/**
* @param engineConfig The engine config which can be used to get the query cache and query cache policy from
* when creating a new index searcher
* @param searcher The provided index searcher to be wrapped to add custom functionality
* @return a new index searcher wrapping the provided index searcher or if no wrapping was performed
* the provided index searcher
*/
IndexSearcher wrap(EngineConfig engineConfig, IndexSearcher searcher) throws EngineException;
}

View File

@ -66,6 +66,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import java.util.function.Function;
import java.util.function.Supplier;
/**
*
@ -303,7 +305,7 @@ public class InternalEngine extends Engine {
}
@Override
public GetResult get(Get get) throws EngineException {
public GetResult get(Get get, Function<String, Searcher> searcherFactory) throws EngineException {
try (ReleasableLock lock = readLock.acquire()) {
ensureOpen();
if (get.realtime()) {
@ -324,7 +326,7 @@ public class InternalEngine extends Engine {
}
// no version, get the version from the index, we know that we refresh on flush
return getFromSearcher(get);
return getFromSearcher(get, searcherFactory);
}
}

View File

@ -35,6 +35,7 @@ import org.elasticsearch.index.translog.Translog;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.function.Function;
/**
* ShadowEngine is a specialized engine that only allows read-only operations
@ -168,9 +169,9 @@ public class ShadowEngine extends Engine {
}
@Override
public GetResult get(Get get) throws EngineException {
public GetResult get(Get get, Function<String, Searcher> searcherFacotry) throws EngineException {
// There is no translog, so we can get it directly from the searcher
return getFromSearcher(get);
return getFromSearcher(get, searcherFacotry);
}
@Override

View File

@ -1,38 +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.index.fielddata;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.settings.Settings;
/**
*/
public class IndexFieldDataModule extends AbstractModule {
private final Settings settings;
public IndexFieldDataModule(Settings settings) {
this.settings = settings;
}
@Override
protected void configure() {
bind(IndexFieldDataService.class).asEagerSingleton();
}
}

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.percolator.stats;
package org.elasticsearch.index.percolator;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;

View File

@ -19,6 +19,7 @@
package org.elasticsearch.index.percolator;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
@ -27,6 +28,8 @@ import org.apache.lucene.util.BytesRef;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.metrics.CounterMetric;
import org.elasticsearch.common.metrics.MeanMetric;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -41,20 +44,18 @@ import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.DocumentTypeListener;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.internal.TypeFieldMapper;
import org.elasticsearch.index.percolator.stats.ShardPercolateService;
import org.elasticsearch.index.query.IndexQueryParserService;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.AbstractIndexShardComponent;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.IndicesLifecycle;
import org.elasticsearch.percolator.PercolatorService;
import java.io.Closeable;
import java.io.IOException;
import java.util.Map;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
/**
@ -64,39 +65,35 @@ import java.util.concurrent.atomic.AtomicBoolean;
* Once a document type has been created, the real-time percolator will start to listen to write events and update the
* this registry with queries in real time.
*/
public class PercolatorQueriesRegistry extends AbstractIndexShardComponent implements Closeable{
public final class PercolatorQueriesRegistry extends AbstractIndexShardComponent implements Closeable {
public final String MAP_UNMAPPED_FIELDS_AS_STRING = "index.percolator.map_unmapped_fields_as_string";
// This is a shard level service, but these below are index level service:
private final IndexQueryParserService queryParserService;
private final MapperService mapperService;
private final IndicesLifecycle indicesLifecycle;
private final IndexFieldDataService indexFieldDataService;
private final ShardIndexingService indexingService;
private final ShardPercolateService shardPercolateService;
private final ConcurrentMap<BytesRef, Query> percolateQueries = ConcurrentCollections.newConcurrentMapWithAggressiveConcurrency();
private final ShardLifecycleListener shardLifecycleListener = new ShardLifecycleListener();
private final RealTimePercolatorOperationListener realTimePercolatorOperationListener = new RealTimePercolatorOperationListener();
private final PercolateTypeListener percolateTypeListener = new PercolateTypeListener();
private final AtomicBoolean realTimePercolatorEnabled = new AtomicBoolean(false);
private boolean mapUnmappedFieldsAsString;
private final MeanMetric percolateMetric = new MeanMetric();
private final CounterMetric currentMetric = new CounterMetric();
private final CounterMetric numberOfQueries = new CounterMetric();
public PercolatorQueriesRegistry(ShardId shardId, @IndexSettings Settings indexSettings, IndexQueryParserService queryParserService,
ShardIndexingService indexingService, IndicesLifecycle indicesLifecycle, MapperService mapperService,
IndexFieldDataService indexFieldDataService, ShardPercolateService shardPercolateService) {
ShardIndexingService indexingService, MapperService mapperService,
IndexFieldDataService indexFieldDataService) {
super(shardId, indexSettings);
this.queryParserService = queryParserService;
this.mapperService = mapperService;
this.indicesLifecycle = indicesLifecycle;
this.indexingService = indexingService;
this.indexFieldDataService = indexFieldDataService;
this.shardPercolateService = shardPercolateService;
this.mapUnmappedFieldsAsString = indexSettings.getAsBoolean(MAP_UNMAPPED_FIELDS_AS_STRING, false);
indicesLifecycle.addListener(shardLifecycleListener);
mapperService.addTypeListener(percolateTypeListener);
}
@ -107,7 +104,6 @@ public class PercolatorQueriesRegistry extends AbstractIndexShardComponent imple
@Override
public void close() {
mapperService.removeTypeListener(percolateTypeListener);
indicesLifecycle.removeListener(shardLifecycleListener);
indexingService.removeListener(realTimePercolatorOperationListener);
clear();
}
@ -116,30 +112,25 @@ public class PercolatorQueriesRegistry extends AbstractIndexShardComponent imple
percolateQueries.clear();
}
void enableRealTimePercolator() {
public void enableRealTimePercolator() {
if (realTimePercolatorEnabled.compareAndSet(false, true)) {
indexingService.addListener(realTimePercolatorOperationListener);
}
}
void disableRealTimePercolator() {
if (realTimePercolatorEnabled.compareAndSet(true, false)) {
indexingService.removeListener(realTimePercolatorOperationListener);
}
}
public void addPercolateQuery(String idAsString, BytesReference source) {
Query newquery = parsePercolatorDocument(idAsString, source);
BytesRef id = new BytesRef(idAsString);
Query previousQuery = percolateQueries.put(id, newquery);
shardPercolateService.addedQuery(id, previousQuery, newquery);
percolateQueries.put(id, newquery);
numberOfQueries.inc();
}
public void removePercolateQuery(String idAsString) {
BytesRef id = new BytesRef(idAsString);
Query query = percolateQueries.remove(id);
if (query != null) {
shardPercolateService.removedQuery(id, query);
numberOfQueries.dec();
}
}
@ -225,55 +216,27 @@ public class PercolatorQueriesRegistry extends AbstractIndexShardComponent imple
enableRealTimePercolator();
}
}
}
private class ShardLifecycleListener extends IndicesLifecycle.Listener {
@Override
public void afterIndexShardCreated(IndexShard indexShard) {
if (hasPercolatorType(indexShard)) {
enableRealTimePercolator();
public void loadQueries(IndexReader reader) {
logger.trace("loading percolator queries...");
final int loadedQueries;
try {
Query query = new TermQuery(new Term(TypeFieldMapper.NAME, PercolatorService.TYPE_NAME));
QueriesLoaderCollector queryCollector = new QueriesLoaderCollector(PercolatorQueriesRegistry.this, logger, mapperService, indexFieldDataService);
IndexSearcher indexSearcher = new IndexSearcher(reader);
indexSearcher.setQueryCache(null);
indexSearcher.search(query, queryCollector);
Map<BytesRef, Query> queries = queryCollector.queries();
for (Map.Entry<BytesRef, Query> entry : queries.entrySet()) {
percolateQueries.put(entry.getKey(), entry.getValue());
numberOfQueries.inc();
}
loadedQueries = queries.size();
} catch (Exception e) {
throw new PercolatorException(shardId.index(), "failed to load queries from percolator index", e);
}
@Override
public void beforeIndexShardPostRecovery(IndexShard indexShard) {
if (hasPercolatorType(indexShard)) {
// percolator index has started, fetch what we can from it and initialize the indices
// we have
logger.trace("loading percolator queries for [{}]...", shardId);
int loadedQueries = loadQueries(indexShard);
logger.debug("done loading [{}] percolator queries for [{}]", loadedQueries, shardId);
}
}
private boolean hasPercolatorType(IndexShard indexShard) {
ShardId otherShardId = indexShard.shardId();
return shardId.equals(otherShardId) && mapperService.hasMapping(PercolatorService.TYPE_NAME);
}
private int loadQueries(IndexShard shard) {
shard.refresh("percolator_load_queries");
// NOTE: we acquire the searcher via the engine directly here since this is executed right
// before the shard is marked as POST_RECOVERY
try (Engine.Searcher searcher = shard.engine().acquireSearcher("percolator_load_queries")) {
Query query = new TermQuery(new Term(TypeFieldMapper.NAME, PercolatorService.TYPE_NAME));
QueriesLoaderCollector queryCollector = new QueriesLoaderCollector(PercolatorQueriesRegistry.this, logger, mapperService, indexFieldDataService);
IndexSearcher indexSearcher = new IndexSearcher(searcher.reader());
indexSearcher.setQueryCache(null);
indexSearcher.search(query, queryCollector);
Map<BytesRef, Query> queries = queryCollector.queries();
for (Map.Entry<BytesRef, Query> entry : queries.entrySet()) {
Query previousQuery = percolateQueries.put(entry.getKey(), entry.getValue());
shardPercolateService.addedQuery(entry.getKey(), previousQuery, entry.getValue());
}
return queries.size();
} catch (Exception e) {
throw new PercolatorException(shardId.index(), "failed to load queries from percolator index", e);
}
}
logger.debug("done loading [{}] percolator queries", loadedQueries);
}
private class RealTimePercolatorOperationListener extends IndexingOperationListener {
@ -320,4 +283,35 @@ public class PercolatorQueriesRegistry extends AbstractIndexShardComponent imple
}
}
}
public void prePercolate() {
currentMetric.inc();
}
public void postPercolate(long tookInNanos) {
currentMetric.dec();
percolateMetric.inc(tookInNanos);
}
/**
* @return The current metrics
*/
public PercolateStats stats() {
return new PercolateStats(percolateMetric.count(), TimeUnit.NANOSECONDS.toMillis(percolateMetric.sum()), currentMetric.count(), -1, numberOfQueries.count());
}
// Enable when a more efficient manner is found for estimating the size of a Lucene query.
/*private static long computeSizeInMemory(HashedBytesRef id, Query query) {
long size = (3 * RamUsageEstimator.NUM_BYTES_INT) + RamUsageEstimator.NUM_BYTES_OBJECT_REF + RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + id.bytes.bytes.length;
size += RamEstimator.sizeOf(query);
return size;
}
private static final class RamEstimator {
// we move this into it's own class to exclude it from the forbidden API checks
// it's fine to use here!
static long sizeOf(Query query) {
return RamUsageEstimator.sizeOf(query);
}
}*/
}

View File

@ -1,93 +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.index.percolator.stats;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.metrics.CounterMetric;
import org.elasticsearch.common.metrics.MeanMetric;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.AbstractIndexShardComponent;
import org.elasticsearch.index.shard.ShardId;
import java.util.concurrent.TimeUnit;
/**
* Shard level percolator service that maintains percolator metrics:
* <ul>
* <li> total time spent in percolate api
* <li> the current number of percolate requests
* <li> number of registered percolate queries
* </ul>
*/
public class ShardPercolateService extends AbstractIndexShardComponent {
@Inject
public ShardPercolateService(ShardId shardId, @IndexSettings Settings indexSettings) {
super(shardId, indexSettings);
}
private final MeanMetric percolateMetric = new MeanMetric();
private final CounterMetric currentMetric = new CounterMetric();
private final CounterMetric numberOfQueries = new CounterMetric();
public void prePercolate() {
currentMetric.inc();
}
public void postPercolate(long tookInNanos) {
currentMetric.dec();
percolateMetric.inc(tookInNanos);
}
public void addedQuery(BytesRef id, Query previousQuery, Query newQuery) {
numberOfQueries.inc();
}
public void removedQuery(BytesRef id, Query query) {
numberOfQueries.dec();
}
/**
* @return The current metrics
*/
public PercolateStats stats() {
return new PercolateStats(percolateMetric.count(), TimeUnit.NANOSECONDS.toMillis(percolateMetric.sum()), currentMetric.count(), -1, numberOfQueries.count());
}
// Enable when a more efficient manner is found for estimating the size of a Lucene query.
/*private static long computeSizeInMemory(HashedBytesRef id, Query query) {
long size = (3 * RamUsageEstimator.NUM_BYTES_INT) + RamUsageEstimator.NUM_BYTES_OBJECT_REF + RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + id.bytes.bytes.length;
size += RamEstimator.sizeOf(query);
return size;
}
private static final class RamEstimator {
// we move this into it's own class to exclude it from the forbidden API checks
// it's fine to use here!
static long sizeOf(Query query) {
return RamUsageEstimator.sizeOf(query);
}
}*/
}

View File

@ -22,7 +22,6 @@ package org.elasticsearch.index.query;
import org.apache.lucene.queries.TermsQuery;
import org.apache.lucene.search.Query;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lucene.search.Queries;
@ -47,9 +46,19 @@ public class IdsQueryBuilder extends AbstractQueryBuilder<IdsQueryBuilder> {
static final IdsQueryBuilder PROTOTYPE = new IdsQueryBuilder();
/**
* Creates a new IdsQueryBuilder by optionally providing the types of the documents to look for
* Creates a new IdsQueryBuilder without providing the types of the documents to look for
*/
public IdsQueryBuilder(@Nullable String... types) {
public IdsQueryBuilder() {
this.types = new String[0];
}
/**
* Creates a new IdsQueryBuilder by providing the types of the documents to look for
*/
public IdsQueryBuilder(String... types) {
if (types == null) {
throw new IllegalArgumentException("[ids] types cannot be null");
}
this.types = types;
}
@ -64,32 +73,13 @@ public class IdsQueryBuilder extends AbstractQueryBuilder<IdsQueryBuilder> {
* Adds ids to the query.
*/
public IdsQueryBuilder addIds(String... ids) {
if (ids == null) {
throw new IllegalArgumentException("[ids] ids cannot be null");
}
Collections.addAll(this.ids, ids);
return this;
}
/**
* Adds ids to the query.
*/
public IdsQueryBuilder addIds(Collection<String> ids) {
this.ids.addAll(ids);
return this;
}
/**
* Adds ids to the filter.
*/
public IdsQueryBuilder ids(String... ids) {
return addIds(ids);
}
/**
* Adds ids to the filter.
*/
public IdsQueryBuilder ids(Collection<String> ids) {
return addIds(ids);
}
/**
* Returns the ids for the query.
*/
@ -100,13 +90,7 @@ public class IdsQueryBuilder extends AbstractQueryBuilder<IdsQueryBuilder> {
@Override
protected void doXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(NAME);
if (types != null) {
if (types.length == 1) {
builder.field("type", types[0]);
} else {
builder.array("types", types);
}
}
builder.array("types", types);
builder.startArray("values");
for (String value : ids) {
builder.value(value);
@ -128,7 +112,7 @@ public class IdsQueryBuilder extends AbstractQueryBuilder<IdsQueryBuilder> {
query = Queries.newMatchNoDocsQuery();
} else {
Collection<String> typesForQuery;
if (types == null || types.length == 0) {
if (types.length == 0) {
typesForQuery = context.queryTypes();
} else if (types.length == 1 && MetaData.ALL.equals(types[0])) {
typesForQuery = context.mapperService().types();

View File

@ -197,15 +197,6 @@ public class IndexQueryParserService extends AbstractIndexComponent {
}
}
@Nullable
public Query parseInnerQuery(QueryShardContext context) throws IOException {
Query query = context.parseContext().parseInnerQueryBuilder().toQuery(context);
if (query == null) {
query = Queries.newMatchNoDocsQuery();
}
return query;
}
public QueryShardContext getShardContext() {
return cache.get();
}
@ -258,16 +249,41 @@ public class IndexQueryParserService extends AbstractIndexComponent {
context.reset(parser);
try {
context.parseFieldMatcher(parseFieldMatcher);
Query query = context.parseContext().parseInnerQueryBuilder().toQuery(context);
if (query == null) {
query = Queries.newMatchNoDocsQuery();
}
Query query = parseInnerQuery(context);
return new ParsedQuery(query, context.copyNamedQueries());
} finally {
context.reset(null);
}
}
public Query parseInnerQuery(QueryShardContext context) throws IOException {
return toQuery(context.parseContext().parseInnerQueryBuilder(), context);
}
public ParsedQuery toQuery(QueryBuilder<?> queryBuilder) {
QueryShardContext context = cache.get();
context.reset();
context.parseFieldMatcher(parseFieldMatcher);
try {
Query query = toQuery(queryBuilder, context);
return new ParsedQuery(query, context.copyNamedQueries());
} catch(QueryShardException | ParsingException e ) {
throw e;
} catch(Exception e) {
throw new QueryShardException(context, "failed to create query: {}", e, queryBuilder);
} finally {
context.reset();
}
}
private static Query toQuery(QueryBuilder<?> queryBuilder, QueryShardContext context) throws IOException {
Query query = queryBuilder.toQuery(context);
if (query == null) {
query = Queries.newMatchNoDocsQuery();
}
return query;
}
public ParseFieldMatcher parseFieldMatcher() {
return parseFieldMatcher;
}

View File

@ -19,7 +19,6 @@
package org.elasticsearch.index.query;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.geo.ShapeRelation;
@ -109,12 +108,19 @@ public abstract class QueryBuilders {
return new DisMaxQueryBuilder();
}
/**
* Constructs a query that will match only specific ids within all types.
*/
public static IdsQueryBuilder idsQuery() {
return new IdsQueryBuilder();
}
/**
* Constructs a query that will match only specific ids within types.
*
* @param types The mapping/doc type
*/
public static IdsQueryBuilder idsQuery(@Nullable String... types) {
public static IdsQueryBuilder idsQuery(String... types) {
return new IdsQueryBuilder(types);
}

View File

@ -17,59 +17,47 @@
* under the License.
*/
package org.elasticsearch.index.engine;
package org.elasticsearch.index.shard;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.search.IndexSearcher;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.index.engine.Engine.Searcher;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.engine.EngineConfig;
import org.elasticsearch.index.engine.EngineException;
import java.util.Set;
import java.io.IOException;
/**
* Service responsible for wrapping the {@link DirectoryReader} and {@link IndexSearcher} of a {@link Searcher} via the
* configured {@link IndexSearcherWrapper} instance. This allows custom functionally to be added the {@link Searcher}
* before being used to do an operation (search, get, field stats etc.)
* Extension point to add custom functionality at request time to the {@link DirectoryReader}
* and {@link IndexSearcher} managed by the {@link Engine}.
*/
// TODO: This needs extension point is a bit hacky now, because the IndexSearch from the engine can only be wrapped once,
// if we allowed the IndexSearcher to be wrapped multiple times then a custom IndexSearcherWrapper needs have good
// control over its location in the wrapping chain
public final class IndexSearcherWrappingService {
public interface IndexSearcherWrapper {
private final IndexSearcherWrapper wrapper;
/**
* @param reader The provided directory reader to be wrapped to add custom functionality
* @return a new directory reader wrapping the provided directory reader or if no wrapping was performed
* the provided directory reader
*/
DirectoryReader wrap(DirectoryReader reader) throws IOException;
// for unit tests:
IndexSearcherWrappingService() {
this.wrapper = null;
}
@Inject
// Use a Set parameter here, because constructor parameter can't be optional
// and I prefer to keep the `wrapper` field final.
public IndexSearcherWrappingService(Set<IndexSearcherWrapper> wrappers) {
if (wrappers.size() > 1) {
throw new IllegalStateException("wrapping of the index searcher by more than one wrappers is forbidden, found the following wrappers [" + wrappers + "]");
}
if (wrappers.isEmpty()) {
this.wrapper = null;
} else {
this.wrapper = wrappers.iterator().next();
}
}
/**
* @param engineConfig The engine config which can be used to get the query cache and query cache policy from
* when creating a new index searcher
* @param searcher The provided index searcher to be wrapped to add custom functionality
* @return a new index searcher wrapping the provided index searcher or if no wrapping was performed
* the provided index searcher
*/
IndexSearcher wrap(EngineConfig engineConfig, IndexSearcher searcher) throws IOException;
/**
* If there are configured {@link IndexSearcherWrapper} instances, the {@link IndexSearcher} of the provided engine searcher
* gets wrapped and a new {@link Searcher} instances is returned, otherwise the provided {@link Searcher} is returned.
* gets wrapped and a new {@link Engine.Searcher} instances is returned, otherwise the provided {@link Engine.Searcher} is returned.
*
* This is invoked each time a {@link Searcher} is requested to do an operation. (for example search)
* This is invoked each time a {@link Engine.Searcher} is requested to do an operation. (for example search)
*/
public Searcher wrap(EngineConfig engineConfig, final Searcher engineSearcher) throws EngineException {
if (wrapper == null) {
return engineSearcher;
}
DirectoryReader reader = wrapper.wrap((DirectoryReader) engineSearcher.reader());
default Engine.Searcher wrap(EngineConfig engineConfig, Engine.Searcher engineSearcher) throws IOException {
DirectoryReader reader = wrap((DirectoryReader) engineSearcher.reader());
IndexSearcher innerIndexSearcher = new IndexSearcher(reader);
innerIndexSearcher.setQueryCache(engineConfig.getQueryCache());
innerIndexSearcher.setQueryCachingPolicy(engineConfig.getQueryCachingPolicy());
@ -77,12 +65,11 @@ public final class IndexSearcherWrappingService {
// TODO: Right now IndexSearcher isn't wrapper friendly, when it becomes wrapper friendly we should revise this extension point
// For example if IndexSearcher#rewrite() is overwritten than also IndexSearcher#createNormalizedWeight needs to be overwritten
// This needs to be fixed before we can allow the IndexSearcher from Engine to be wrapped multiple times
IndexSearcher indexSearcher = wrapper.wrap(engineConfig, innerIndexSearcher);
IndexSearcher indexSearcher = wrap(engineConfig, innerIndexSearcher);
if (reader == engineSearcher.reader() && indexSearcher == innerIndexSearcher) {
return engineSearcher;
} else {
return new Engine.Searcher(engineSearcher.source(), indexSearcher) {
@Override
public void close() throws ElasticsearchException {
engineSearcher.close();

View File

@ -20,10 +20,7 @@
package org.elasticsearch.index.shard;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.index.CheckIndex;
import org.apache.lucene.index.IndexCommit;
import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy;
import org.apache.lucene.index.SnapshotDeletionPolicy;
import org.apache.lucene.index.*;
import org.apache.lucene.search.QueryCachingPolicy;
import org.apache.lucene.search.UsageTrackingQueryCachingPolicy;
import org.apache.lucene.store.AlreadyClosedException;
@ -36,6 +33,7 @@ import org.elasticsearch.action.admin.indices.optimize.OptimizeRequest;
import org.elasticsearch.action.admin.indices.upgrade.post.UpgradeRequest;
import org.elasticsearch.action.termvectors.TermVectorsRequest;
import org.elasticsearch.action.termvectors.TermVectorsResponse;
import org.elasticsearch.bootstrap.Elasticsearch;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.ShardRouting;
@ -51,11 +49,11 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.concurrent.AbstractRefCounted;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.FutureUtils;
import org.elasticsearch.gateway.MetaDataStateFormat;
import org.elasticsearch.index.IndexServicesProvider;
import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.aliases.IndexAliasesService;
import org.elasticsearch.index.cache.IndexCache;
@ -75,8 +73,8 @@ import org.elasticsearch.index.indexing.IndexingStats;
import org.elasticsearch.index.indexing.ShardIndexingService;
import org.elasticsearch.index.mapper.*;
import org.elasticsearch.index.merge.MergeStats;
import org.elasticsearch.index.percolator.PercolateStats;
import org.elasticsearch.index.percolator.PercolatorQueriesRegistry;
import org.elasticsearch.index.percolator.stats.ShardPercolateService;
import org.elasticsearch.index.query.IndexQueryParserService;
import org.elasticsearch.index.recovery.RecoveryStats;
import org.elasticsearch.index.refresh.RefreshStats;
@ -99,13 +97,13 @@ import org.elasticsearch.index.translog.TranslogStats;
import org.elasticsearch.index.translog.TranslogWriter;
import org.elasticsearch.index.warmer.ShardIndexWarmerService;
import org.elasticsearch.index.warmer.WarmerStats;
import org.elasticsearch.indices.IndicesLifecycle;
import org.elasticsearch.indices.IndicesWarmer;
import org.elasticsearch.indices.InternalIndicesLifecycle;
import org.elasticsearch.indices.cache.query.IndicesQueryCache;
import org.elasticsearch.indices.memory.IndexingMemoryController;
import org.elasticsearch.indices.recovery.RecoveryFailedException;
import org.elasticsearch.indices.recovery.RecoveryState;
import org.elasticsearch.percolator.PercolatorService;
import org.elasticsearch.search.suggest.completion.Completion090PostingsFormat;
import org.elasticsearch.search.suggest.completion.CompletionStats;
import org.elasticsearch.threadpool.ThreadPool;
@ -139,7 +137,6 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
private final ShardRequestCache shardQueryCache;
private final ShardFieldData shardFieldData;
private final PercolatorQueriesRegistry percolatorQueriesRegistry;
private final ShardPercolateService shardPercolateService;
private final TermVectorsService termVectorsService;
private final IndexFieldDataService indexFieldDataService;
private final ShardSuggestMetric shardSuggestMetric = new ShardSuggestMetric();
@ -163,7 +160,6 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
protected volatile IndexShardState state;
protected final AtomicReference<Engine> currentEngineReference = new AtomicReference<>();
protected final EngineFactory engineFactory;
private final IndexSearcherWrappingService wrappingService;
@Nullable
private RecoveryState recoveryState;
@ -192,47 +188,40 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
private final IndexShardOperationCounter indexShardOperationCounter;
private EnumSet<IndexShardState> readAllowedStates = EnumSet.of(IndexShardState.STARTED, IndexShardState.RELOCATED, IndexShardState.POST_RECOVERY);
private final EnumSet<IndexShardState> readAllowedStates = EnumSet.of(IndexShardState.STARTED, IndexShardState.RELOCATED, IndexShardState.POST_RECOVERY);
private final IndexSearcherWrapper searcherWrapper;
private final AtomicBoolean active = new AtomicBoolean();
private volatile long lastWriteNS;
private final IndexingMemoryController indexingMemoryController;
@Inject
public IndexShard(ShardId shardId, @IndexSettings Settings indexSettings, IndicesLifecycle indicesLifecycle, Store store,
ThreadPool threadPool, MapperService mapperService, IndexQueryParserService queryParserService, IndexCache indexCache, IndexAliasesService indexAliasesService,
IndicesQueryCache indicesQueryCache, CodecService codecService,
TermVectorsService termVectorsService, IndexFieldDataService indexFieldDataService,
@Nullable IndicesWarmer warmer, SimilarityService similarityService, EngineFactory factory,
ShardPath path, BigArrays bigArrays, IndexSearcherWrappingService wrappingService,
IndexingMemoryController indexingMemoryController) {
public IndexShard(ShardId shardId, @IndexSettings Settings indexSettings, ShardPath path, Store store, IndexServicesProvider provider) {
super(shardId, indexSettings);
this.codecService = codecService;
this.warmer = warmer;
this.codecService = provider.getCodecService();
this.warmer = provider.getWarmer();
this.deletionPolicy = new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy());
this.similarityService = similarityService;
this.wrappingService = wrappingService;
this.similarityService = provider.getSimilarityService();
Objects.requireNonNull(store, "Store must be provided to the index shard");
this.engineFactory = factory;
this.indicesLifecycle = (InternalIndicesLifecycle) indicesLifecycle;
this.engineFactory = provider.getFactory();
this.indicesLifecycle = (InternalIndicesLifecycle) provider.getIndicesLifecycle();
this.store = store;
this.mergeSchedulerConfig = new MergeSchedulerConfig(indexSettings);
this.threadPool = threadPool;
this.mapperService = mapperService;
this.queryParserService = queryParserService;
this.indexCache = indexCache;
this.indexAliasesService = indexAliasesService;
this.threadPool = provider.getThreadPool();
this.mapperService = provider.getMapperService();
this.queryParserService = provider.getQueryParserService();
this.indexCache = provider.getIndexCache();
this.indexAliasesService = provider.getIndexAliasesService();
this.indexingService = new ShardIndexingService(shardId, indexSettings);
this.getService = new ShardGetService(this, mapperService);
this.termVectorsService = termVectorsService;
this.termVectorsService = provider.getTermVectorsService();
this.searchService = new ShardSearchStats(indexSettings);
this.shardWarmerService = new ShardIndexWarmerService(shardId, indexSettings);
this.indicesQueryCache = indicesQueryCache;
this.indicesQueryCache = provider.getIndicesQueryCache();
this.shardQueryCache = new ShardRequestCache(shardId, indexSettings);
this.shardFieldData = new ShardFieldData();
this.shardPercolateService = new ShardPercolateService(shardId, indexSettings);
this.percolatorQueriesRegistry = new PercolatorQueriesRegistry(shardId, indexSettings, queryParserService, indexingService, indicesLifecycle, mapperService, indexFieldDataService, shardPercolateService);
this.indexFieldDataService = indexFieldDataService;
this.indexFieldDataService = provider.getIndexFieldDataService();
this.shardBitsetFilterCache = new ShardBitsetFilterCache(shardId, indexSettings);
state = IndexShardState.CREATED;
this.refreshInterval = indexSettings.getAsTime(INDEX_REFRESH_INTERVAL, EngineConfig.DEFAULT_REFRESH_INTERVAL);
@ -245,7 +234,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
this.checkIndexOnStartup = indexSettings.get("index.shard.check_on_startup", "false");
this.translogConfig = new TranslogConfig(shardId, shardPath().resolveTranslog(), indexSettings, getFromSettings(logger, indexSettings, Translog.Durabilty.REQUEST),
bigArrays, threadPool);
provider.getBigArrays(), threadPool);
final QueryCachingPolicy cachingPolicy;
// the query cache is a node-level thing, however we want the most popular filters
// to be computed on a per-shard basis
@ -259,7 +248,13 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
this.flushThresholdSize = indexSettings.getAsBytesSize(INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE, new ByteSizeValue(512, ByteSizeUnit.MB));
this.disableFlush = indexSettings.getAsBoolean(INDEX_TRANSLOG_DISABLE_FLUSH, false);
this.indexShardOperationCounter = new IndexShardOperationCounter(logger, shardId);
this.indexingMemoryController = indexingMemoryController;
this.indexingMemoryController = provider.getIndexingMemoryController();
this.searcherWrapper = provider.getIndexSearcherWrapper();
this.percolatorQueriesRegistry = new PercolatorQueriesRegistry(shardId, indexSettings, queryParserService, indexingService, mapperService, indexFieldDataService);
if (mapperService.hasMapping(PercolatorService.TYPE_NAME)) {
percolatorQueriesRegistry.enableRealTimePercolator();
}
// TODO: can we somehow call IMC.forceCheck here? Since we just became active, it can divvy up the RAM
active.set(true);
@ -355,7 +350,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
if (newRouting.state() == ShardRoutingState.STARTED || newRouting.state() == ShardRoutingState.RELOCATING) {
// we want to refresh *before* we move to internal STARTED state
try {
engine().refresh("cluster_state_started");
getEngine().refresh("cluster_state_started");
} catch (Throwable t) {
logger.debug("failed to refresh due to move to cluster wide started", t);
}
@ -464,7 +459,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
if (logger.isTraceEnabled()) {
logger.trace("index [{}][{}]{}", create.type(), create.id(), create.docs());
}
engine().create(create);
getEngine().create(create);
create.endTime(System.nanoTime());
} catch (Throwable ex) {
indexingService.postCreate(create, ex);
@ -503,7 +498,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
if (logger.isTraceEnabled()) {
logger.trace("index [{}][{}]{}", index.type(), index.id(), index.docs());
}
created = engine().index(index);
created = getEngine().index(index);
index.endTime(System.nanoTime());
} catch (Throwable ex) {
indexingService.postIndex(index, ex);
@ -526,7 +521,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
if (logger.isTraceEnabled()) {
logger.trace("delete [{}]", delete.uid().text());
}
engine().delete(delete);
getEngine().delete(delete);
delete.endTime(System.nanoTime());
} catch (Throwable ex) {
indexingService.postDelete(delete, ex);
@ -537,7 +532,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
public Engine.GetResult get(Engine.Get get) {
readAllowed();
return engine().get(get);
return getEngine().get(get, this::acquireSearcher);
}
public void refresh(String source) {
@ -546,7 +541,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
logger.trace("refresh with source: {}", source);
}
long time = System.nanoTime();
engine().refresh(source);
getEngine().refresh(source);
refreshMetric.inc(System.nanoTime() - time);
}
@ -572,7 +567,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
*/
@Nullable
public CommitStats commitStats() {
Engine engine = engineUnsafe();
Engine engine = getEngineOrNull();
return engine == null ? null : engine.commitStats();
}
@ -599,7 +594,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
}
public MergeStats mergeStats() {
final Engine engine = engineUnsafe();
final Engine engine = getEngineOrNull();
if (engine == null) {
return new MergeStats();
}
@ -607,7 +602,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
}
public SegmentsStats segmentStats() {
SegmentsStats segmentsStats = engine().segmentsStats();
SegmentsStats segmentsStats = getEngine().segmentsStats();
segmentsStats.addBitsetMemoryInBytes(shardBitsetFilterCache.getMemorySizeInBytes());
return segmentsStats;
}
@ -632,12 +627,8 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
return percolatorQueriesRegistry;
}
public ShardPercolateService shardPercolateService() {
return shardPercolateService;
}
public TranslogStats translogStats() {
return engine().getTranslog().stats();
return getEngine().getTranslog().stats();
}
public SuggestStats suggestStats() {
@ -662,7 +653,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
public Engine.SyncedFlushResult syncFlush(String syncId, Engine.CommitId expectedCommitId) {
verifyStartedOrRecovering();
logger.trace("trying to sync flush. sync id [{}]. expected commit id [{}]]", syncId, expectedCommitId);
return engine().syncFlush(syncId, expectedCommitId);
return getEngine().syncFlush(syncId, expectedCommitId);
}
public Engine.CommitId flush(FlushRequest request) throws ElasticsearchException {
@ -677,7 +668,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
verifyStartedOrRecovering();
long time = System.nanoTime();
Engine.CommitId commitId = engine().flush(force, waitIfOngoing);
Engine.CommitId commitId = getEngine().flush(force, waitIfOngoing);
flushMetric.inc(System.nanoTime() - time);
return commitId;
@ -688,7 +679,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
if (logger.isTraceEnabled()) {
logger.trace("optimize with {}", optimize);
}
engine().forceMerge(optimize.flush(), optimize.maxNumSegments(), optimize.onlyExpungeDeletes(), false, false);
getEngine().forceMerge(optimize.flush(), optimize.maxNumSegments(), optimize.onlyExpungeDeletes(), false, false);
}
/**
@ -701,7 +692,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
}
org.apache.lucene.util.Version previousVersion = minimumCompatibleVersion();
// we just want to upgrade the segments, not actually optimize to a single segment
engine().forceMerge(true, // we need to flush at the end to make sure the upgrade is durable
getEngine().forceMerge(true, // we need to flush at the end to make sure the upgrade is durable
Integer.MAX_VALUE, // we just want to upgrade the segments, not actually optimize to a single segment
false, true, upgrade.upgradeOnlyAncientSegments());
org.apache.lucene.util.Version version = minimumCompatibleVersion();
@ -714,7 +705,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
public org.apache.lucene.util.Version minimumCompatibleVersion() {
org.apache.lucene.util.Version luceneVersion = null;
for (Segment segment : engine().segments(false)) {
for (Segment segment : getEngine().segments(false)) {
if (luceneVersion == null || luceneVersion.onOrAfter(segment.getVersion())) {
luceneVersion = segment.getVersion();
}
@ -732,7 +723,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
IndexShardState state = this.state; // one time volatile read
// we allow snapshot on closed index shard, since we want to do one after we close the shard and before we close the engine
if (state == IndexShardState.STARTED || state == IndexShardState.RELOCATED || state == IndexShardState.CLOSED) {
return engine().snapshotIndex(flushFirst);
return getEngine().snapshotIndex(flushFirst);
} else {
throw new IllegalIndexShardStateException(shardId, state, "snapshot is not allowed");
}
@ -753,12 +744,17 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
*/
public void failShard(String reason, @Nullable Throwable e) {
// fail the engine. This will cause this shard to also be removed from the node's index service.
engine().failEngine(reason, e);
getEngine().failEngine(reason, e);
}
public Engine.Searcher acquireSearcher(String source) {
readAllowed();
return engine().acquireSearcher(source);
Engine engine = getEngine();
try {
return searcherWrapper == null ? engine.acquireSearcher(source) : searcherWrapper.wrap(engineConfig, engine.acquireSearcher(source));
} catch (IOException ex) {
throw new ElasticsearchException("failed to wrap searcher", ex);
}
}
public void close(String reason, boolean flushEngine) throws IOException {
@ -785,8 +781,14 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
}
}
public IndexShard postRecovery(String reason) throws IndexShardStartedException, IndexShardRelocatedException, IndexShardClosedException {
indicesLifecycle.beforeIndexShardPostRecovery(this);
if (mapperService.hasMapping(PercolatorService.TYPE_NAME)) {
refresh("percolator_load_queries");
try (Engine.Searcher searcher = getEngine().acquireSearcher("percolator_load_queries")) {
this.percolatorQueriesRegistry.loadQueries(searcher.reader());
}
}
synchronized (mutex) {
if (state == IndexShardState.CLOSED) {
throw new IndexShardClosedException(shardId);
@ -800,7 +802,6 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
recoveryState.setStage(RecoveryState.Stage.DONE);
changeState(IndexShardState.POST_RECOVERY, reason);
}
indicesLifecycle.afterIndexShardPostRecovery(this);
return this;
}
@ -824,7 +825,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
if (state != IndexShardState.RECOVERING) {
throw new IndexShardNotRecoveringException(shardId, state);
}
return engineConfig.getTranslogRecoveryPerformer().performBatchRecovery(engine(), operations);
return engineConfig.getTranslogRecoveryPerformer().performBatchRecovery(getEngine(), operations);
}
/**
@ -863,7 +864,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
* a remote peer.
*/
public void skipTranslogRecovery() throws IOException {
assert engineUnsafe() == null : "engine was already created";
assert getEngineOrNull() == null : "engine was already created";
internalPerformTranslogRecovery(true, true);
assert recoveryState.getTranslog().recoveredOperations() == 0;
}
@ -903,7 +904,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
*/
public void finalizeRecovery() {
recoveryState().setStage(RecoveryState.Stage.FINALIZE);
engine().refresh("recovery_finalization");
getEngine().refresh("recovery_finalization");
startScheduledTasksIfNeeded();
engineConfig.setEnableGcDeletes(true);
}
@ -1008,7 +1009,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
config.setIndexingBufferSize(shardIndexingBufferSize);
Engine engine = engineUnsafe();
Engine engine = getEngineOrNull();
if (engine == null) {
logger.debug("updateBufferSize: engine is closed; skipping");
return;
@ -1094,7 +1095,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
*/
boolean shouldFlush() {
if (disableFlush == false) {
Engine engine = engineUnsafe();
Engine engine = getEngineOrNull();
if (engine != null) {
try {
Translog translog = engine.getTranslog();
@ -1208,15 +1209,37 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
searchService.onRefreshSettings(settings);
indexingService.onRefreshSettings(settings);
if (change) {
engine().onSettingsChanged();
getEngine().onSettingsChanged();
}
}
public Translog.View acquireTranslogView() {
Engine engine = getEngine();
assert engine.getTranslog() != null : "translog must not be null";
return engine.getTranslog().newView();
}
public List<Segment> segments(boolean verbose) {
return getEngine().segments(verbose);
}
public void flushAndCloseEngine() throws IOException {
getEngine().flushAndClose();
}
public Translog getTranslog() {
return getEngine().getTranslog();
}
public PercolateStats percolateStats() {
return percolatorQueriesRegistry.stats();
}
class EngineRefresher implements Runnable {
@Override
public void run() {
// we check before if a refresh is needed, if not, we reschedule, otherwise, we fork, refresh, and then reschedule
if (!engine().refreshNeeded()) {
if (!getEngine().refreshNeeded()) {
reschedule();
return;
}
@ -1224,7 +1247,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
@Override
public void run() {
try {
if (engine().refreshNeeded()) {
if (getEngine().refreshNeeded()) {
refresh("schedule");
}
} catch (EngineClosedException e) {
@ -1337,8 +1360,8 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
recoveryState.getVerifyIndex().checkIndexTime(Math.max(0, TimeValue.nsecToMSec(System.nanoTime() - timeNS)));
}
public Engine engine() {
Engine engine = engineUnsafe();
Engine getEngine() {
Engine engine = getEngineOrNull();
if (engine == null) {
throw new EngineClosedException(shardId);
}
@ -1347,7 +1370,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
/** NOTE: returns null if engine is not yet started (e.g. recovery phase 1, copying over index files, is still running), or if engine is
* closed. */
protected Engine engineUnsafe() {
protected Engine getEngineOrNull() {
return this.currentEngineReference.get();
}
@ -1440,7 +1463,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
};
return new EngineConfig(shardId,
threadPool, indexingService, indexSettings, warmer, store, deletionPolicy, mergePolicyConfig.getMergePolicy(), mergeSchedulerConfig,
mapperService.indexAnalyzer(), similarityService.similarity(), codecService, failedEngineListener, translogRecoveryPerformer, indexCache.query(), cachingPolicy, wrappingService, translogConfig);
mapperService.indexAnalyzer(), similarityService.similarity(), codecService, failedEngineListener, translogRecoveryPerformer, indexCache.query(), cachingPolicy, translogConfig);
}
private static class IndexShardOperationCounter extends AbstractRefCounted {
@ -1481,7 +1504,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
*/
public void sync(Translog.Location location) {
try {
final Engine engine = engine();
final Engine engine = getEngine();
engine.getTranslog().ensureSynced(location);
} catch (EngineClosedException ex) {
// that's fine since we already synced everything on engine close - this also is conform with the methods documentation
@ -1552,4 +1575,5 @@ public class IndexShard extends AbstractIndexShardComponent implements IndexSett
}
return false;
}
}

View File

@ -1,76 +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.index.shard;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.inject.multibindings.Multibinder;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.engine.IndexSearcherWrapper;
import org.elasticsearch.index.engine.IndexSearcherWrappingService;
import org.elasticsearch.index.engine.EngineFactory;
import org.elasticsearch.index.engine.InternalEngineFactory;
/**
* The {@code IndexShardModule} module is responsible for binding the correct
* shard id, index shard, engine factory, and warming service for a newly
* created shard.
*/
public class IndexShardModule extends AbstractModule {
private final ShardId shardId;
private final Settings settings;
private final boolean primary;
// pkg private so tests can mock
Class<? extends EngineFactory> engineFactoryImpl = InternalEngineFactory.class;
public IndexShardModule(ShardId shardId, boolean primary, Settings settings) {
this.settings = settings;
this.shardId = shardId;
this.primary = primary;
if (settings.get("index.translog.type") != null) {
throw new IllegalStateException("a custom translog type is no longer supported. got [" + settings.get("index.translog.type") + "]");
}
}
/** Return true if a shadow engine should be used */
protected boolean useShadowEngine() {
return primary == false && IndexMetaData.isIndexUsingShadowReplicas(settings);
}
@Override
protected void configure() {
bind(ShardId.class).toInstance(shardId);
if (useShadowEngine()) {
bind(IndexShard.class).to(ShadowIndexShard.class).asEagerSingleton();
} else {
bind(IndexShard.class).asEagerSingleton();
}
bind(EngineFactory.class).to(engineFactoryImpl);
bind(IndexSearcherWrappingService.class).asEagerSingleton();
// this injects an empty set in IndexSearcherWrappingService, otherwise guice can't construct IndexSearcherWrappingService
Multibinder<IndexSearcherWrapper> multibinder
= Multibinder.newSetBinder(binder(), IndexSearcherWrapper.class);
}
}

View File

@ -20,33 +20,14 @@ package org.elasticsearch.index.shard;
import java.io.IOException;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.index.aliases.IndexAliasesService;
import org.elasticsearch.index.cache.IndexCache;
import org.elasticsearch.index.codec.CodecService;
import org.elasticsearch.index.IndexServicesProvider;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.engine.EngineConfig;
import org.elasticsearch.index.engine.EngineFactory;
import org.elasticsearch.index.engine.IndexSearcherWrappingService;
import org.elasticsearch.index.fielddata.IndexFieldDataService;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.merge.MergeStats;
import org.elasticsearch.index.query.IndexQueryParserService;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.settings.IndexSettingsService;
import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.termvectors.TermVectorsService;
import org.elasticsearch.indices.IndicesLifecycle;
import org.elasticsearch.indices.IndicesWarmer;
import org.elasticsearch.indices.cache.query.IndicesQueryCache;
import org.elasticsearch.indices.memory.IndexingMemoryController;
import org.elasticsearch.threadpool.ThreadPool;
/**
* ShadowIndexShard extends {@link IndexShard} to add file synchronization
@ -56,25 +37,8 @@ import org.elasticsearch.threadpool.ThreadPool;
*/
public final class ShadowIndexShard extends IndexShard {
@Inject
public ShadowIndexShard(ShardId shardId, @IndexSettings Settings indexSettings,
IndicesLifecycle indicesLifecycle, Store store,
ThreadPool threadPool, MapperService mapperService,
IndexQueryParserService queryParserService, IndexCache indexCache,
IndexAliasesService indexAliasesService, IndicesQueryCache indicesQueryCache,
CodecService codecService, TermVectorsService termVectorsService, IndexFieldDataService indexFieldDataService,
@Nullable IndicesWarmer warmer,
SimilarityService similarityService,
EngineFactory factory,
ShardPath path, BigArrays bigArrays, IndexSearcherWrappingService wrappingService,
IndexingMemoryController indexingMemoryController) throws IOException {
super(shardId, indexSettings, indicesLifecycle, store,
threadPool, mapperService, queryParserService, indexCache, indexAliasesService,
indicesQueryCache, codecService,
termVectorsService, indexFieldDataService,
warmer, similarityService,
factory, path, bigArrays, wrappingService,
indexingMemoryController);
public ShadowIndexShard(ShardId shardId, @IndexSettings Settings indexSettings, ShardPath path, Store store, IndexServicesProvider provider) throws IOException {
super(shardId, indexSettings, path, store, provider);
}
/**

View File

@ -26,6 +26,7 @@ import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.RateLimiter;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterService;
@ -93,6 +94,8 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
private RateLimitingInputStream.Listener snapshotThrottleListener;
private RateLimitingInputStream.Listener restoreThrottleListener;
private boolean compress;
private final ParseFieldMatcher parseFieldMatcher;
@ -147,6 +150,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
this.restoreRateLimiter = restoreRateLimiter;
this.rateLimiterListener = rateLimiterListener;
this.snapshotThrottleListener = nanos -> rateLimiterListener.onSnapshotPause(nanos);
this.restoreThrottleListener = nanos -> rateLimiterListener.onRestorePause(nanos);
this.compress = compress;
indexShardSnapshotFormat = new ChecksumBlobStoreFormat<>(SNAPSHOT_CODEC, SNAPSHOT_NAME_FORMAT, BlobStoreIndexShardSnapshot.PROTO, parseFieldMatcher, isCompress());
indexShardSnapshotLegacyFormat = new LegacyBlobStoreFormat<>(LEGACY_SNAPSHOT_NAME_FORMAT, BlobStoreIndexShardSnapshot.PROTO, parseFieldMatcher);
@ -486,7 +490,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
public SnapshotContext(SnapshotId snapshotId, ShardId shardId, IndexShardSnapshotStatus snapshotStatus) {
super(snapshotId, Version.CURRENT, shardId);
IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
store = indexService.shard(shardId.id()).store();
store = indexService.getShardOrNull(shardId.id()).store();
this.snapshotStatus = snapshotStatus;
}
@ -770,7 +774,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
*/
public RestoreContext(SnapshotId snapshotId, Version version, ShardId shardId, ShardId snapshotShardId, RecoveryState recoveryState) {
super(snapshotId, version, shardId, snapshotShardId);
store = indicesService.indexServiceSafe(shardId.getIndex()).shard(shardId.id()).store();
store = indicesService.indexServiceSafe(shardId.getIndex()).getShardOrNull(shardId.id()).store();
this.recoveryState = recoveryState;
}
@ -891,16 +895,20 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
*/
private void restoreFile(final FileInfo fileInfo) throws IOException {
boolean success = false;
try (InputStream stream = new PartSliceStream(blobContainer, fileInfo)) {
try (InputStream partSliceStream = new PartSliceStream(blobContainer, fileInfo)) {
final InputStream stream;
if (restoreRateLimiter == null) {
stream = partSliceStream;
} else {
stream = new RateLimitingInputStream(partSliceStream, restoreRateLimiter, restoreThrottleListener);
}
try (final IndexOutput indexOutput = store.createVerifyingOutput(fileInfo.physicalName(), fileInfo.metadata(), IOContext.DEFAULT)) {
final byte[] buffer = new byte[BUFFER_SIZE];
int length;
while ((length = stream.read(buffer)) > 0) {
indexOutput.writeBytes(buffer, 0, length);
recoveryState.getIndex().addRecoveredBytesToFile(fileInfo.name(), length);
if (restoreRateLimiter != null) {
rateLimiterListener.onRestorePause(restoreRateLimiter.pause(length));
}
}
Store.verify(indexOutput);
indexOutput.close();

View File

@ -27,6 +27,7 @@ import org.elasticsearch.index.AbstractIndexComponent;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.settings.IndexSettingsService;
import org.elasticsearch.index.shard.ShardPath;
import org.elasticsearch.indices.store.IndicesStore;
import java.io.Closeable;
@ -112,7 +113,7 @@ public class IndexStore extends AbstractIndexComponent implements Closeable {
/**
* The shard store class that should be used for each shard.
*/
public Class<? extends DirectoryService> shardDirectory() {
return FsDirectoryService.class;
public DirectoryService newDirectoryService(ShardPath path) {
return new FsDirectoryService(indexSettings, this, path);
}
}

View File

@ -1286,14 +1286,15 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
@Override
public void writeByte(byte b) throws IOException {
final long writtenBytes = this.writtenBytes++;
if (writtenBytes == checksumPosition) {
readAndCompareChecksum();
} else if (writtenBytes > checksumPosition) { // we are writing parts of the checksum....
if (writtenBytes >= checksumPosition) { // we are writing parts of the checksum....
if (writtenBytes == checksumPosition) {
readAndCompareChecksum();
}
final int index = Math.toIntExact(writtenBytes - checksumPosition);
if (index < footerChecksum.length) {
footerChecksum[index] = b;
if (index == footerChecksum.length-1) {
verify();// we have recorded the entire checksum
verify(); // we have recorded the entire checksum
}
} else {
verify(); // fail if we write more than expected
@ -1315,16 +1316,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
@Override
public void writeBytes(byte[] b, int offset, int length) throws IOException {
if (writtenBytes + length > checksumPosition) {
if (actualChecksum == null) {
assert writtenBytes <= checksumPosition;
final int bytesToWrite = (int) (checksumPosition - writtenBytes);
out.writeBytes(b, offset, bytesToWrite);
readAndCompareChecksum();
offset += bytesToWrite;
length -= bytesToWrite;
writtenBytes += bytesToWrite;
}
for (int i = 0; i < length; i++) {
for (int i = 0; i < length; i++) { // don't optimze writing the last block of bytes
writeByte(b[offset+i]);
}
} else {
@ -1332,7 +1324,6 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
writtenBytes += length;
}
}
}
/**

View File

@ -97,17 +97,6 @@ public interface IndicesLifecycle {
}
/**
* Called right after the shard is moved into POST_RECOVERY mode
*/
public void afterIndexShardPostRecovery(IndexShard indexShard) {}
/**
* Called right before the shard is moved into POST_RECOVERY mode.
* The shard is ready to be used but not yet marked as POST_RECOVERY.
*/
public void beforeIndexShardPostRecovery(IndexShard indexShard) {}
/**
* Called after the index shard has been started.
*/

View File

@ -53,18 +53,15 @@ import org.elasticsearch.index.IndexNameModule;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.LocalNodeIdModule;
import org.elasticsearch.index.aliases.IndexAliasesServiceModule;
import org.elasticsearch.index.analysis.AnalysisModule;
import org.elasticsearch.index.analysis.AnalysisService;
import org.elasticsearch.index.cache.IndexCache;
import org.elasticsearch.index.cache.IndexCacheModule;
import org.elasticsearch.index.fielddata.IndexFieldDataModule;
import org.elasticsearch.index.fielddata.IndexFieldDataService;
import org.elasticsearch.index.flush.FlushStats;
import org.elasticsearch.index.get.GetStats;
import org.elasticsearch.index.indexing.IndexingStats;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.MapperServiceModule;
import org.elasticsearch.index.merge.MergeStats;
import org.elasticsearch.index.query.IndexQueryParserService;
import org.elasticsearch.index.recovery.RecoveryStats;
@ -343,10 +340,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
modules.add(new AnalysisModule(indexSettings, indicesAnalysisService));
modules.add(new SimilarityModule(indexSettings));
modules.add(new IndexCacheModule(indexSettings));
modules.add(new IndexFieldDataModule(indexSettings));
modules.add(new MapperServiceModule());
modules.add(new IndexAliasesServiceModule());
modules.add(new IndexModule(indexSettings));
modules.add(new IndexModule());
pluginsService.processModules(modules);

View File

@ -87,7 +87,7 @@ public final class IndicesWarmer extends AbstractComponent {
if (indexService == null) {
return;
}
final IndexShard indexShard = indexService.shard(context.shardId().id());
final IndexShard indexShard = indexService.getShardOrNull(context.shardId().id());
if (indexShard == null) {
return;
}

View File

@ -121,28 +121,6 @@ public class InternalIndicesLifecycle extends AbstractComponent implements Indic
}
}
public void beforeIndexShardPostRecovery(IndexShard indexShard) {
for (Listener listener : listeners) {
try {
listener.beforeIndexShardPostRecovery(indexShard);
} catch (Throwable t) {
logger.warn("{} failed to invoke before shard post recovery callback", t, indexShard.shardId());
throw t;
}
}
}
public void afterIndexShardPostRecovery(IndexShard indexShard) {
for (Listener listener : listeners) {
try {
listener.afterIndexShardPostRecovery(indexShard);
} catch (Throwable t) {
logger.warn("{} failed to invoke after shard post recovery callback", t, indexShard.shardId());
throw t;
}
}
}
public void afterIndexShardStarted(IndexShard indexShard) {
for (Listener listener : listeners) {

View File

@ -38,7 +38,7 @@ import org.elasticsearch.index.flush.FlushStats;
import org.elasticsearch.index.get.GetStats;
import org.elasticsearch.index.indexing.IndexingStats;
import org.elasticsearch.index.merge.MergeStats;
import org.elasticsearch.index.percolator.stats.PercolateStats;
import org.elasticsearch.index.percolator.PercolateStats;
import org.elasticsearch.index.recovery.RecoveryStats;
import org.elasticsearch.index.refresh.RefreshStats;
import org.elasticsearch.index.search.stats.SearchStats;

View File

@ -327,7 +327,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
// already deleted on us, ignore it
continue;
}
IndexSettingsService indexSettingsService = indexService.injector().getInstance(IndexSettingsService.class);
IndexSettingsService indexSettingsService = indexService.settingsService();
indexSettingsService.refreshSettings(indexMetaData.settings());
}
}
@ -505,7 +505,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
continue;
}
IndexShard indexShard = indexService.shard(shardId);
IndexShard indexShard = indexService.getShardOrNull(shardId);
if (indexShard != null) {
ShardRouting currentRoutingEntry = indexShard.routingEntry();
// if the current and global routing are initializing, but are still not the same, its a different "shard" being allocated
@ -591,7 +591,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
final int shardId = shardRouting.id();
if (indexService.hasShard(shardId)) {
IndexShard indexShard = indexService.shardSafe(shardId);
IndexShard indexShard = indexService.getShard(shardId);
if (indexShard.state() == IndexShardState.STARTED || indexShard.state() == IndexShardState.POST_RECOVERY) {
// the master thinks we are initializing, but we are already started or on POST_RECOVERY and waiting
// for master to confirm a shard started message (either master failover, or a cluster event before
@ -647,7 +647,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
return;
}
}
final IndexShard indexShard = indexService.shardSafe(shardId);
final IndexShard indexShard = indexService.getShard(shardId);
if (indexShard.ignoreRecoveryAttempt()) {
// we are already recovering (we can get to this state since the cluster event can happen several
@ -835,7 +835,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
ShardRouting shardRouting = null;
final IndexService indexService = indicesService.indexService(shardId.index().name());
if (indexService != null) {
IndexShard indexShard = indexService.shard(shardId.id());
IndexShard indexShard = indexService.getShardOrNull(shardId.id());
if (indexShard != null) {
shardRouting = indexShard.routingEntry();
}

View File

@ -398,7 +398,7 @@ public class SyncedFlushService extends AbstractComponent {
}
private PreSyncedFlushResponse performPreSyncedFlush(PreSyncedFlushRequest request) {
IndexShard indexShard = indicesService.indexServiceSafe(request.shardId().getIndex()).shardSafe(request.shardId().id());
IndexShard indexShard = indicesService.indexServiceSafe(request.shardId().getIndex()).getShard(request.shardId().id());
FlushRequest flushRequest = new FlushRequest().force(false).waitIfOngoing(true);
logger.trace("{} performing pre sync flush", request.shardId());
Engine.CommitId commitId = indexShard.flush(flushRequest);
@ -408,7 +408,7 @@ public class SyncedFlushService extends AbstractComponent {
private SyncedFlushResponse performSyncedFlush(SyncedFlushRequest request) {
IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
IndexShard indexShard = indexService.shardSafe(request.shardId().id());
IndexShard indexShard = indexService.getShard(request.shardId().id());
logger.trace("{} performing sync flush. sync id [{}], expected commit id {}", request.shardId(), request.syncId(), request.expectedCommitId());
Engine.SyncedFlushResult result = indexShard.syncFlush(request.syncId(), request.expectedCommitId());
logger.trace("{} sync flush done. sync id [{}], result [{}]", request.shardId(), request.syncId(), result);
@ -426,7 +426,7 @@ public class SyncedFlushService extends AbstractComponent {
private InFlightOpsResponse performInFlightOps(InFlightOpsRequest request) {
IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
IndexShard indexShard = indexService.shardSafe(request.shardId().id());
IndexShard indexShard = indexService.getShard(request.shardId().id());
if (indexShard.routingEntry().primary() == false) {
throw new IllegalStateException("[" + request.shardId() +"] expected a primary shard");
}

View File

@ -230,7 +230,8 @@ public class IndexingMemoryController extends AbstractLifecycleComponent<Indexin
protected IndexShard getShard(ShardId shardId) {
IndexService indexService = indicesService.indexService(shardId.index().name());
if (indexService != null) {
return indexService.shard(shardId.id());
IndexShard indexShard = indexService.getShardOrNull(shardId.id());
return indexShard;
}
return null;
}

View File

@ -89,7 +89,7 @@ public class RecoverySource extends AbstractComponent {
private RecoveryResponse recover(final StartRecoveryRequest request) {
final IndexService indexService = indicesService.indexServiceSafe(request.shardId().index().name());
final IndexShard shard = indexService.shardSafe(request.shardId().id());
final IndexShard shard = indexService.getShard(request.shardId().id());
// starting recovery from that our (the source) shard state is marking the shard to be in recovery mode as well, otherwise
// the index operations will not be routed to it properly

View File

@ -120,9 +120,7 @@ public class RecoverySourceHandler {
* performs the recovery from the local engine to the target
*/
public RecoveryResponse recoverToTarget() {
final Engine engine = shard.engine();
assert engine.getTranslog() != null : "translog must not be null";
try (Translog.View translogView = engine.getTranslog().newView()) {
try (Translog.View translogView = shard.acquireTranslogView()) {
logger.trace("captured translog id [{}] for recovery", translogView.minTranslogGeneration());
final IndexCommit phase1Snapshot;
try {
@ -179,7 +177,7 @@ public class RecoverySourceHandler {
try {
recoverySourceMetadata = store.getMetadata(snapshot);
} catch (CorruptIndexException | IndexFormatTooOldException | IndexFormatTooNewException ex) {
shard.engine().failEngine("recovery", ex);
shard.failShard("recovery", ex);
throw ex;
}
for (String name : snapshot.getFileNames()) {
@ -287,7 +285,7 @@ public class RecoverySourceHandler {
for (StoreFileMetaData md : metadata) {
logger.debug("{} checking integrity for file {} after remove corruption exception", shard.shardId(), md);
if (store.checkIntegrityNoException(md) == false) { // we are corrupted on the primary -- fail!
shard.engine().failEngine("recovery", corruptIndexException);
shard.failShard("recovery", corruptIndexException);
logger.warn("{} Corrupted file detected {} checksum mismatch", shard.shardId(), md);
throw corruptIndexException;
}
@ -641,7 +639,7 @@ public class RecoverySourceHandler {
}
protected void failEngine(IOException cause) {
shard.engine().failEngine("recovery", cause);
shard.failShard("recovery", cause);
}
Future<Void>[] asyncSendFiles(Store store, StoreFileMetaData[] files, Function<StoreFileMetaData, OutputStream> outputStreamFactory) {

View File

@ -52,7 +52,7 @@ public class SharedFSRecoverySourceHandler extends RecoverySourceHandler {
// if we relocate we need to close the engine in order to open a new
// IndexWriter on the other end of the relocation
engineClosed = true;
shard.engine().flushAndClose();
shard.flushAndCloseEngine();
} catch (IOException e) {
logger.warn("close engine failed", e);
shard.failShard("failed to close engine (phase1)", e);

View File

@ -395,7 +395,7 @@ public class IndicesStore extends AbstractComponent implements ClusterStateListe
ShardId shardId = request.shardId;
IndexService indexService = indicesService.indexService(shardId.index().getName());
if (indexService != null && indexService.indexUUID().equals(request.indexUUID)) {
return indexService.shard(shardId.id());
return indexService.getShardOrNull(shardId.id());
}
return null;
}

View File

@ -152,7 +152,7 @@ public class TransportNodesListShardStoreMetaData extends TransportNodesAction<T
try {
IndexService indexService = indicesService.indexService(shardId.index().name());
if (indexService != null) {
IndexShard indexShard = indexService.shard(shardId.id());
IndexShard indexShard = indexService.getShardOrNull(shardId.id());
if (indexShard != null) {
final Store store = indexShard.store();
store.incRef();

View File

@ -50,6 +50,7 @@ import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.object.ObjectMapper;
import org.elasticsearch.index.percolator.PercolatorQueriesRegistry;
import org.elasticsearch.index.query.IndexQueryParserService;
import org.elasticsearch.index.query.ParsedQuery;
import org.elasticsearch.index.shard.IndexShard;
@ -89,6 +90,7 @@ import java.util.concurrent.ConcurrentMap;
*/
public class PercolateContext extends SearchContext {
private final PercolatorQueriesRegistry percolateQueryRegistry;
public boolean limit;
private int size;
public boolean doSort;
@ -102,7 +104,6 @@ public class PercolateContext extends SearchContext {
private final PageCacheRecycler pageCacheRecycler;
private final BigArrays bigArrays;
private final ScriptService scriptService;
private final ConcurrentMap<BytesRef, Query> percolateQueries;
private final int numberOfShards;
private final Query aliasFilter;
private final long originNanoTime = System.nanoTime();
@ -133,7 +134,7 @@ public class PercolateContext extends SearchContext {
this.indexService = indexService;
this.fieldDataService = indexService.fieldData();
this.searchShardTarget = searchShardTarget;
this.percolateQueries = indexShard.percolateRegistry().percolateQueries();
this.percolateQueryRegistry = indexShard.percolateRegistry();
this.types = new String[]{request.documentType()};
this.pageCacheRecycler = pageCacheRecycler;
this.bigArrays = bigArrays.withCircuitBreaking();
@ -179,7 +180,7 @@ public class PercolateContext extends SearchContext {
}
public ConcurrentMap<BytesRef, Query> percolateQueries() {
return percolateQueries;
return percolateQueryRegistry.percolateQueries();
}
public Query percolateQuery() {

View File

@ -71,7 +71,7 @@ import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.Uid;
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import org.elasticsearch.index.percolator.stats.ShardPercolateService;
import org.elasticsearch.index.percolator.PercolatorQueriesRegistry;
import org.elasticsearch.index.query.ParsedQuery;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.indices.IndicesService;
@ -86,7 +86,6 @@ import org.elasticsearch.search.aggregations.AggregationPhase;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregation.ReduceContext;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.pipeline.SiblingPipelineAggregator;
import org.elasticsearch.search.highlight.HighlightField;
import org.elasticsearch.search.highlight.HighlightPhase;
@ -177,11 +176,10 @@ public class PercolatorService extends AbstractComponent {
public PercolateShardResponse percolate(PercolateShardRequest request) {
IndexService percolateIndexService = indicesService.indexServiceSafe(request.shardId().getIndex());
IndexShard indexShard = percolateIndexService.shardSafe(request.shardId().id());
IndexShard indexShard = percolateIndexService.getShard(request.shardId().id());
indexShard.readAllowed(); // check if we can read the shard...
ShardPercolateService shardPercolateService = indexShard.shardPercolateService();
shardPercolateService.prePercolate();
PercolatorQueriesRegistry percolateQueryRegistry = indexShard.percolateRegistry();
percolateQueryRegistry.prePercolate();
long startTime = System.nanoTime();
// TODO: The filteringAliases should be looked up at the coordinating node and serialized with all shard request,
@ -255,7 +253,7 @@ public class PercolatorService extends AbstractComponent {
} finally {
SearchContext.removeCurrent();
context.close();
shardPercolateService.postPercolate(System.nanoTime() - startTime);
percolateQueryRegistry.postPercolate(System.nanoTime() - startTime);
}
}

View File

@ -73,20 +73,6 @@ public abstract class Plugin {
return Collections.emptyList();
}
/**
* Per index shard module.
*/
public Collection<Module> shardModules(Settings indexSettings) {
return Collections.emptyList();
}
/**
* Per index shard service that will be automatically closed.
*/
public Collection<Class<? extends Closeable>> shardServices() {
return Collections.emptyList();
}
/**
* Additional node settings loaded by the plugin. Note that settings that are explicit in the nodes settings can't be
* overwritten with the additional settings. These settings added if they don't exist.

View File

@ -250,22 +250,6 @@ public class PluginsService extends AbstractComponent {
return services;
}
public Collection<Module> shardModules(Settings indexSettings) {
List<Module> modules = new ArrayList<>();
for (Tuple<PluginInfo, Plugin> plugin : plugins) {
modules.addAll(plugin.v2().shardModules(indexSettings));
}
return modules;
}
public Collection<Class<? extends Closeable>> shardServices() {
List<Class<? extends Closeable>> services = new ArrayList<>();
for (Tuple<PluginInfo, Plugin> plugin : plugins) {
services.addAll(plugin.v2().shardServices());
}
return services;
}
/**
* Get information about plugins (jvm and site plugins).
*/

View File

@ -43,7 +43,7 @@ import org.elasticsearch.index.flush.FlushStats;
import org.elasticsearch.index.get.GetStats;
import org.elasticsearch.index.indexing.IndexingStats;
import org.elasticsearch.index.merge.MergeStats;
import org.elasticsearch.index.percolator.stats.PercolateStats;
import org.elasticsearch.index.percolator.PercolateStats;
import org.elasticsearch.index.refresh.RefreshStats;
import org.elasticsearch.index.search.stats.SearchStats;
import org.elasticsearch.index.suggest.stats.SuggestStats;

View File

@ -559,7 +559,7 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> {
final SearchContext createContext(ShardSearchRequest request, @Nullable Engine.Searcher searcher) {
IndexService indexService = indicesService.indexServiceSafe(request.index());
IndexShard indexShard = indexService.shardSafe(request.shardId());
IndexShard indexShard = indexService.getShard(request.shardId());
SearchShardTarget shardTarget = new SearchShardTarget(clusterService.localNode().id(), request.index(), request.shardId());

View File

@ -289,7 +289,7 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
for (final Map.Entry<ShardId, IndexShardSnapshotStatus> shardEntry : entry.getValue().entrySet()) {
final ShardId shardId = shardEntry.getKey();
try {
final IndexShard indexShard = indicesService.indexServiceSafe(shardId.getIndex()).shard(shardId.id());
final IndexShard indexShard = indicesService.indexServiceSafe(shardId.getIndex()).getShardOrNull(shardId.id());
executor.execute(new AbstractRunnable() {
@Override
public void doRun() {

View File

@ -69,8 +69,8 @@ grant codeBase "${es.security.plugin.lang-groovy}" {
permission java.lang.RuntimePermission "accessClassInPackage.sun.reflect";
// needed by GroovyScriptEngineService to close its classloader (why?)
permission java.lang.RuntimePermission "closeClassLoader";
// Allow executing groovy scripts with codesource of /groovy/script
permission groovy.security.GroovyCodeSourcePermission "/groovy/script";
// Allow executing groovy scripts with codesource of /untrusted
permission groovy.security.GroovyCodeSourcePermission "/untrusted";
};
grant codeBase "${es.security.plugin.lang-javascript}" {

View File

@ -18,8 +18,8 @@
*/
/*
* Limited security policy for groovy scripts.
* This is what is needed for its invokeDynamic functionality to work.
* Limited security policy for scripts.
* This is what is needed for invokeDynamic functionality to work.
*/
grant {

View File

@ -158,7 +158,7 @@ public class IndicesShardStoreRequestIT extends ESIntegTestCase {
IndicesService indexServices = internalCluster().getInstance(IndicesService.class, node);
IndexService indexShards = indexServices.indexServiceSafe(index);
for (Integer shardId : indexShards.shardIds()) {
IndexShard shard = indexShards.shardSafe(shardId);
IndexShard shard = indexShards.getShard(shardId);
if (randomBoolean()) {
shard.failShard("test", new CorruptIndexException("test corrupted", ""));
Set<String> nodes = corruptedShardIDMap.get(shardId);

View File

@ -65,7 +65,7 @@ public class UpgradeReallyOldIndexIT extends StaticIndexBackwardCompatibilityIT
for (IndicesService services : internalCluster().getInstances(IndicesService.class)) {
IndexService indexService = services.indexService(index);
if (indexService != null) {
assertEquals(version, indexService.shard(0).minimumCompatibleVersion());
assertEquals(version, indexService.getShardOrNull(0).minimumCompatibleVersion());
}
}

View File

@ -24,7 +24,9 @@ import org.elasticsearch.test.ESTestCase;
import java.io.FilePermission;
import java.security.AccessControlContext;
import java.security.AccessController;
import java.security.AllPermission;
import java.security.CodeSource;
import java.security.Permission;
import java.security.PermissionCollection;
import java.security.Permissions;
import java.security.PrivilegedAction;
@ -48,8 +50,13 @@ public class ESPolicyTests extends ESTestCase {
*/
public void testNullCodeSource() throws Exception {
assumeTrue("test cannot run with security manager", System.getSecurityManager() == null);
// create a policy with AllPermission
Permission all = new AllPermission();
PermissionCollection allCollection = all.newPermissionCollection();
allCollection.add(all);
ESPolicy policy = new ESPolicy(allCollection);
// restrict ourselves to NoPermission
PermissionCollection noPermissions = new Permissions();
ESPolicy policy = new ESPolicy(noPermissions);
assertFalse(policy.implies(new ProtectionDomain(null, noPermissions), new FilePermission("foo", "read")));
}

View File

@ -35,7 +35,6 @@ import java.security.ProtectionDomain;
import java.security.cert.Certificate;
import java.util.Collections;
import java.util.HashSet;
import java.util.Objects;
import java.util.Set;
/**
@ -99,18 +98,24 @@ final class MockPluginPolicy extends Policy {
excludedSources.add(RandomizedRunner.class.getProtectionDomain().getCodeSource());
// junit library
excludedSources.add(Assert.class.getProtectionDomain().getCodeSource());
// groovy scripts
excludedSources.add(new CodeSource(new URL("file:/groovy/script"), (Certificate[])null));
// scripts
excludedSources.add(new CodeSource(new URL("file:" + BootstrapInfo.UNTRUSTED_CODEBASE), (Certificate[])null));
Loggers.getLogger(getClass()).debug("Apply permissions [{}] excluding codebases [{}]", extraPermissions, excludedSources);
}
@Override
public boolean implies(ProtectionDomain domain, Permission permission) {
CodeSource codeSource = domain.getCodeSource();
// codesource can be null when reducing privileges via doPrivileged()
if (codeSource == null) {
return false;
}
if (standardPolicy.implies(domain, permission)) {
return true;
} else if (excludedSources.contains(domain.getCodeSource()) == false &&
Objects.toString(domain.getCodeSource()).contains("test-classes") == false) {
} else if (excludedSources.contains(codeSource) == false &&
codeSource.toString().contains("test-classes") == false) {
return extraPermissions.implies(permission);
} else {
return false;

View File

@ -179,7 +179,7 @@ public class ClusterInfoServiceIT extends ESIntegTestCase {
DiscoveryNode discoveryNode = state.getNodes().get(nodeId);
IndicesService indicesService = internalTestCluster.getInstance(IndicesService.class, discoveryNode.getName());
IndexService indexService = indicesService.indexService(shard.index());
IndexShard indexShard = indexService.shard(shard.id());
IndexShard indexShard = indexService.getShardOrNull(shard.id());
assertEquals(indexShard.shardPath().getRootDataPath().toString(), dataPath);
}

View File

@ -50,13 +50,11 @@ import org.elasticsearch.test.ESAllocationTestCase;
import org.elasticsearch.test.gateway.NoopGatewayAllocator;
import org.junit.Test;
import java.util.AbstractMap;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING;
import static org.elasticsearch.cluster.routing.ShardRoutingState.RELOCATING;
@ -912,6 +910,137 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
assertThat(result.routingTable().index("test").getShards().get(1).primaryShard().relocatingNodeId(), equalTo("node2"));
}
public void testForSingleDataNode() {
Settings diskSettings = settingsBuilder()
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED, true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS, true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK, "60%")
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, "70%").build();
Map<String, DiskUsage> usages = new HashMap<>();
usages.put("node1", new DiskUsage("node1", "n1", "/dev/null", 100, 100)); // 0% used
usages.put("node2", new DiskUsage("node2", "n2", "/dev/null", 100, 20)); // 80% used
usages.put("node3", new DiskUsage("node3", "n3", "/dev/null", 100, 100)); // 0% used
// We have an index with 1 primary shards each taking 40 bytes. Each node has 100 bytes available
Map<String, Long> shardSizes = new HashMap<>();
shardSizes.put("[test][0][p]", 40L);
shardSizes.put("[test][1][p]", 40L);
final ClusterInfo clusterInfo = new ClusterInfo(Collections.unmodifiableMap(usages), Collections.unmodifiableMap(usages), Collections.unmodifiableMap(shardSizes), MockInternalClusterInfoService.DEV_NULL_MAP);
DiskThresholdDecider diskThresholdDecider = new DiskThresholdDecider(diskSettings);
MetaData metaData = MetaData.builder()
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(2).numberOfReplicas(0))
.build();
RoutingTable routingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
logger.info("--> adding one master node, one data node");
Map<String, String> masterNodeAttributes = new HashMap<>();
masterNodeAttributes.put("master", "true");
masterNodeAttributes.put("data", "false");
Map<String, String> dataNodeAttributes = new HashMap<>();
dataNodeAttributes.put("master", "false");
dataNodeAttributes.put("data", "true");
DiscoveryNode discoveryNode1 = new DiscoveryNode("", "node1", new LocalTransportAddress("1"), masterNodeAttributes, Version.CURRENT);
DiscoveryNode discoveryNode2 = new DiscoveryNode("", "node2", new LocalTransportAddress("2"), dataNodeAttributes, Version.CURRENT);
DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().put(discoveryNode1).put(discoveryNode2).build();
ClusterState baseClusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT)
.metaData(metaData)
.routingTable(routingTable)
.nodes(discoveryNodes)
.build();
// Two shards consumes 80% of disk space in data node, but we have only one data node, shards should remain.
ShardRouting firstRouting = TestShardRouting.newShardRouting("test", 0, "node2", null, null, true, ShardRoutingState.STARTED, 1);
ShardRouting secondRouting = TestShardRouting.newShardRouting("test", 1, "node2", null, null, true, ShardRoutingState.STARTED, 1);
RoutingNode firstRoutingNode = new RoutingNode("node2", discoveryNode2, Arrays.asList(firstRouting, secondRouting));
RoutingTable.Builder builder = RoutingTable.builder().add(
IndexRoutingTable.builder("test")
.addIndexShard(new IndexShardRoutingTable.Builder(new ShardId("test", 0))
.addShard(firstRouting)
.build()
)
.addIndexShard(new IndexShardRoutingTable.Builder(new ShardId("test", 1))
.addShard(secondRouting)
.build()
)
);
ClusterState clusterState = ClusterState.builder(baseClusterState).routingTable(builder).build();
RoutingAllocation routingAllocation = new RoutingAllocation(null, new RoutingNodes(clusterState), discoveryNodes, clusterInfo);
Decision decision = diskThresholdDecider.canRemain(firstRouting, firstRoutingNode, routingAllocation);
// Two shards should start happily
assertThat(decision.type(), equalTo(Decision.Type.YES));
ClusterInfoService cis = new ClusterInfoService() {
@Override
public ClusterInfo getClusterInfo() {
logger.info("--> calling fake getClusterInfo");
return clusterInfo;
}
@Override
public void addListener(Listener listener) {
}
};
AllocationDeciders deciders = new AllocationDeciders(Settings.EMPTY, new HashSet<>(Arrays.asList(
new SameShardAllocationDecider(Settings.EMPTY), diskThresholdDecider
)));
AllocationService strategy = new AllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, makeShardsAllocators(), cis);
RoutingAllocation.Result result = strategy.reroute(clusterState);
assertThat(result.routingTable().index("test").getShards().get(0).primaryShard().state(), equalTo(STARTED));
assertThat(result.routingTable().index("test").getShards().get(0).primaryShard().currentNodeId(), equalTo("node2"));
assertThat(result.routingTable().index("test").getShards().get(0).primaryShard().relocatingNodeId(), nullValue());
assertThat(result.routingTable().index("test").getShards().get(1).primaryShard().state(), equalTo(STARTED));
assertThat(result.routingTable().index("test").getShards().get(1).primaryShard().currentNodeId(), equalTo("node2"));
assertThat(result.routingTable().index("test").getShards().get(1).primaryShard().relocatingNodeId(), nullValue());
// Add another datanode, it should relocate.
logger.info("--> adding node3");
DiscoveryNode discoveryNode3 = new DiscoveryNode("", "node3", new LocalTransportAddress("3"), dataNodeAttributes, Version.CURRENT);
ClusterState updateClusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.put(discoveryNode3)).build();
firstRouting = TestShardRouting.newShardRouting("test", 0, "node2", null, null, true, ShardRoutingState.STARTED, 1);
secondRouting = TestShardRouting.newShardRouting("test", 1, "node2", "node3", null, true, ShardRoutingState.RELOCATING, 1);
firstRoutingNode = new RoutingNode("node2", discoveryNode2, Arrays.asList(firstRouting, secondRouting));
builder = RoutingTable.builder().add(
IndexRoutingTable.builder("test")
.addIndexShard(new IndexShardRoutingTable.Builder(new ShardId("test", 0))
.addShard(firstRouting)
.build()
)
.addIndexShard(new IndexShardRoutingTable.Builder(new ShardId("test", 1))
.addShard(secondRouting)
.build()
)
);
clusterState = ClusterState.builder(updateClusterState).routingTable(builder).build();
routingAllocation = new RoutingAllocation(null, new RoutingNodes(clusterState), discoveryNodes, clusterInfo);
decision = diskThresholdDecider.canRemain(firstRouting, firstRoutingNode, routingAllocation);
assertThat(decision.type(), equalTo(Decision.Type.YES));
result = strategy.reroute(clusterState);
assertThat(result.routingTable().index("test").getShards().get(0).primaryShard().state(), equalTo(STARTED));
assertThat(result.routingTable().index("test").getShards().get(0).primaryShard().currentNodeId(), equalTo("node2"));
assertThat(result.routingTable().index("test").getShards().get(0).primaryShard().relocatingNodeId(), nullValue());
assertThat(result.routingTable().index("test").getShards().get(1).primaryShard().state(), equalTo(RELOCATING));
assertThat(result.routingTable().index("test").getShards().get(1).primaryShard().currentNodeId(), equalTo("node2"));
assertThat(result.routingTable().index("test").getShards().get(1).primaryShard().relocatingNodeId(), equalTo("node3"));
}
public void logShardStates(ClusterState state) {
RoutingNodes rn = state.getRoutingNodes();
logger.info("--> counts: total: {}, unassigned: {}, initializing: {}, relocating: {}, started: {}",

View File

@ -19,26 +19,24 @@
package org.elasticsearch.cluster.routing.operation.hash.murmur3;
import com.carrotsearch.randomizedtesting.generators.RandomInts;
import com.carrotsearch.randomizedtesting.generators.RandomStrings;
import com.google.common.hash.HashFunction;
import com.google.common.hash.Hashing;
import org.elasticsearch.cluster.routing.Murmur3HashFunction;
import org.elasticsearch.test.ESTestCase;
public class Murmur3HashFunctionTests extends ESTestCase {
public void test() {
// Make sure that we agree with guava
Murmur3HashFunction murmur3 = new Murmur3HashFunction();
HashFunction guavaMurmur3 = Hashing.murmur3_32();
for (int i = 0; i < 100; ++i) {
final String id = RandomStrings.randomRealisticUnicodeOfCodepointLength(getRandom(), RandomInts.randomIntBetween(getRandom(), 1, 20));
//final String id = "0";
final int hash1 = guavaMurmur3.newHasher().putUnencodedChars(id).hash().asInt();
final int hash2 = murmur3.hash(id);
assertEquals(hash1, hash2);
}
private static Murmur3HashFunction HASH = new Murmur3HashFunction();
public void testKnownValues() {
assertHash(0x5a0cb7c3, "hell");
assertHash(0xd7c31989, "hello");
assertHash(0x22ab2984, "hello w");
assertHash(0xdf0ca123, "hello wo");
assertHash(0xe7744d61, "hello wor");
assertHash(0xe07db09c, "The quick brown fox jumps over the lazy dog");
assertHash(0x4e63d2ad, "The quick brown fox jumps over the lazy cog");
}
private static void assertHash(int expected, String stringInput) {
assertEquals(expected, HASH.hash(stringInput));
}
}

View File

@ -0,0 +1,81 @@
/*
* 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.hash;
import org.elasticsearch.test.ESTestCase;
import org.junit.Test;
import java.math.BigInteger;
import java.nio.charset.StandardCharsets;
import java.security.MessageDigest;
import static org.junit.Assert.*;
public class MessageDigestsTests extends ESTestCase {
private void assertHash(String expected, String test, MessageDigest messageDigest) {
String actual = MessageDigests.toHexString(messageDigest.digest(test.getBytes(StandardCharsets.UTF_8)));
assertEquals(expected, actual);
}
@Test
public void testMd5() throws Exception {
assertHash("d41d8cd98f00b204e9800998ecf8427e", "", MessageDigests.md5());
assertHash("900150983cd24fb0d6963f7d28e17f72", "abc", MessageDigests.md5());
assertHash("8215ef0796a20bcaaae116d3876c664a", "abcdbcdecdefdefgefghfghighijhijkijkljklmklmnlmnomnopnopq", MessageDigests.md5());
assertHash("7707d6ae4e027c70eea2a935c2296f21", new String(new char[1000000]).replace("\0", "a"), MessageDigests.md5());
assertHash("9e107d9d372bb6826bd81d3542a419d6", "The quick brown fox jumps over the lazy dog", MessageDigests.md5());
assertHash("1055d3e698d289f2af8663725127bd4b", "The quick brown fox jumps over the lazy cog", MessageDigests.md5());
}
@Test
public void testSha1() throws Exception {
assertHash("da39a3ee5e6b4b0d3255bfef95601890afd80709", "", MessageDigests.sha1());
assertHash("a9993e364706816aba3e25717850c26c9cd0d89d", "abc", MessageDigests.sha1());
assertHash("84983e441c3bd26ebaae4aa1f95129e5e54670f1", "abcdbcdecdefdefgefghfghighijhijkijkljklmklmnlmnomnopnopq", MessageDigests.sha1());
assertHash("34aa973cd4c4daa4f61eeb2bdbad27316534016f", new String(new char[1000000]).replace("\0", "a"), MessageDigests.sha1());
assertHash("2fd4e1c67a2d28fced849ee1bb76e7391b93eb12", "The quick brown fox jumps over the lazy dog", MessageDigests.sha1());
assertHash("de9f2c7fd25e1b3afad3e85a0bd17d9b100db4b3", "The quick brown fox jumps over the lazy cog", MessageDigests.sha1());
}
@Test
public void testSha256() throws Exception {
assertHash("e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", "", MessageDigests.sha256());
assertHash("ba7816bf8f01cfea414140de5dae2223b00361a396177a9cb410ff61f20015ad", "abc", MessageDigests.sha256());
assertHash("248d6a61d20638b8e5c026930c3e6039a33ce45964ff2167f6ecedd419db06c1", "abcdbcdecdefdefgefghfghighijhijkijkljklmklmnlmnomnopnopq", MessageDigests.sha256());
assertHash("cdc76e5c9914fb9281a1c7e284d73e67f1809a48a497200e046d39ccc7112cd0", new String(new char[1000000]).replace("\0", "a"), MessageDigests.sha256());
assertHash("d7a8fbb307d7809469ca9abcb0082e4f8d5651e46d3cdb762d02d0bf37c9e592", "The quick brown fox jumps over the lazy dog", MessageDigests.sha256());
assertHash("e4c4d8f3bf76b692de791a173e05321150f7a345b46484fe427f6acc7ecc81be", "The quick brown fox jumps over the lazy cog", MessageDigests.sha256());
}
@Test
public void testToHexString() throws Exception {
for (int i = 0; i < 1024; i++) {
BigInteger expected = BigInteger.probablePrime(256, random());
byte[] bytes = expected.toByteArray();
String hex = MessageDigests.toHexString(bytes);
String zeros = new String(new char[2 * bytes.length]).replace("\0", "0");
String expectedAsString = expected.toString(16);
String expectedHex = zeros.substring(expectedAsString.length()) + expectedAsString;
assertEquals(expectedHex, hex);
BigInteger actual = new BigInteger(hex, 16);
assertEquals(expected, actual);
}
}
}

View File

@ -19,37 +19,34 @@
package org.elasticsearch.common.hashing;
import com.google.common.hash.HashCode;
import com.google.common.hash.Hashing;
import org.elasticsearch.common.hash.MurmurHash3;
import org.elasticsearch.test.ESTestCase;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.nio.LongBuffer;
import java.io.UnsupportedEncodingException;
import java.nio.charset.StandardCharsets;
public class MurmurHash3Tests extends ESTestCase {
public void testHash128() {
final int iters = scaledRandomIntBetween(100, 5000);
for (int i = 0; i < iters; ++i) {
final int seed = randomInt();
final int offset = randomInt(20);
final int len = randomInt(randomBoolean() ? 20 : 200);
final byte[] bytes = new byte[len + offset + randomInt(3)];
getRandom().nextBytes(bytes);
HashCode h1 = Hashing.murmur3_128(seed).hashBytes(bytes, offset, len);
MurmurHash3.Hash128 h2 = MurmurHash3.hash128(bytes, offset, len, seed, new MurmurHash3.Hash128());
assertEquals(h1, h2);
}
public void testKnownValues() throws UnsupportedEncodingException {
assertHash(0x629942693e10f867L, 0x92db0b82baeb5347L, "hell", 0);
assertHash(0xa78ddff5adae8d10L, 0x128900ef20900135L, "hello", 1);
assertHash(0x8a486b23f422e826L, 0xf962a2c58947765fL, "hello ", 2);
assertHash(0x2ea59f466f6bed8cL, 0xc610990acc428a17L, "hello w", 3);
assertHash(0x79f6305a386c572cL, 0x46305aed3483b94eL, "hello wo", 4);
assertHash(0xc2219d213ec1f1b5L, 0xa1d8e2e0a52785bdL, "hello wor", 5);
assertHash(0xe34bbc7bbc071b6cL, 0x7a433ca9c49a9347L, "The quick brown fox jumps over the lazy dog", 0);
assertHash(0x658ca970ff85269aL, 0x43fee3eaa68e5c3eL, "The quick brown fox jumps over the lazy cog", 0);
}
private void assertEquals(HashCode h1, MurmurHash3.Hash128 h2) {
final LongBuffer longs = ByteBuffer.wrap(h1.asBytes()).order(ByteOrder.LITTLE_ENDIAN).asLongBuffer();
assertEquals(2, longs.limit());
assertEquals(h1.asLong(), h2.h1);
assertEquals(longs.get(), h2.h1);
assertEquals(longs.get(), h2.h2);
private static void assertHash(long lower, long upper, String inputString, long seed) {
byte[] bytes = inputString.getBytes(StandardCharsets.UTF_8);
MurmurHash3.Hash128 expected = new MurmurHash3.Hash128();
expected.h1 = lower;
expected.h2 = upper;
assertHash(expected, MurmurHash3.hash128(bytes, 0, bytes.length, seed, new MurmurHash3.Hash128()));
}
private static void assertHash(MurmurHash3.Hash128 expected, MurmurHash3.Hash128 actual) {
assertEquals(expected.h1, actual.h1);
assertEquals(expected.h2, actual.h2);
}
}

View File

@ -60,6 +60,22 @@ public abstract class ModuleTestCase extends ESTestCase {
fail("Did not find any binding to " + to.getName() + ". Found these bindings:\n" + s);
}
// /** Configures the module and asserts "instance" is bound to "to". */
// public void assertInstanceBinding(Module module, Class to, Object instance) {
// List<Element> elements = Elements.getElements(module);
// for (Element element : elements) {
// if (element instanceof ProviderInstanceBinding) {
// assertEquals(instance, ((ProviderInstanceBinding) element).getProviderInstance().get());
// return;
// }
// }
// StringBuilder s = new StringBuilder();
// for (Element element : elements) {
// s.append(element + "\n");
// }
// fail("Did not find any binding to " + to.getName() + ". Found these bindings:\n" + s);
// }
/**
* Attempts to configure the module, and asserts an {@link IllegalArgumentException} is
* caught, containing the given messages
@ -164,6 +180,10 @@ public abstract class ModuleTestCase extends ESTestCase {
return;
}
}
} else if (element instanceof ProviderInstanceBinding) {
ProviderInstanceBinding binding = (ProviderInstanceBinding) element;
assertTrue(tester.test(to.cast(binding.getProviderInstance().get())));
return;
}
}
StringBuilder s = new StringBuilder();

View File

@ -21,20 +21,21 @@ package org.elasticsearch.common.logging.log4j;
import org.apache.log4j.Appender;
import org.apache.log4j.Logger;
import org.elasticsearch.common.cli.CliToolTestCase;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.node.internal.InternalSettingsPreparer;
import org.elasticsearch.test.ESTestCase;
import org.hamcrest.Matchers;
import org.junit.Before;
import org.junit.Test;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.StandardOpenOption;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.*;
/**
*
@ -148,7 +149,34 @@ public class LoggingConfigurationTests extends ESTestCase {
LogConfigurator.resolveConfig(environment, builder);
Settings logSettings = builder.build();
assertThat(logSettings.get("yml"), Matchers.nullValue());
assertThat(logSettings.get("yml"), nullValue());
}
// tests that custom settings are not overwritten by settings in the config file
@Test
public void testResolveOrder() throws Exception {
Path tmpDir = createTempDir();
Path loggingConf = tmpDir.resolve(loggingConfiguration("yaml"));
Files.write(loggingConf, "logger.test: INFO, file\n".getBytes(StandardCharsets.UTF_8));
Files.write(loggingConf, "appender.file.type: file\n".getBytes(StandardCharsets.UTF_8), StandardOpenOption.APPEND);
Environment environment = InternalSettingsPreparer.prepareEnvironment(
Settings.builder()
.put("path.conf", tmpDir.toAbsolutePath())
.put("path.home", createTempDir().toString())
.put("logger.test", "TRACE, console")
.put("appender.console.type", "console")
.put("appender.console.layout.type", "consolePattern")
.put("appender.console.layout.conversionPattern", "[%d{ISO8601}][%-5p][%-25c] %m%n")
.build(), new CliToolTestCase.MockTerminal());
LogConfigurator.configure(environment.settings());
// args should overwrite whatever is in the config
ESLogger esLogger = Log4jESLoggerFactory.getLogger("test");
Logger logger = ((Log4jESLogger) esLogger).logger();
Appender appender = logger.getAppender("console");
assertThat(appender, notNullValue());
assertTrue(logger.isTraceEnabled());
appender = logger.getAppender("file");
assertThat(appender, nullValue());
}
private static String loggingConfiguration(String suffix) {

View File

@ -0,0 +1,66 @@
/*
* 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.index;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.search.IndexSearcher;
import org.elasticsearch.common.inject.ModuleTestCase;
import org.elasticsearch.index.engine.EngineConfig;
import org.elasticsearch.index.engine.EngineException;
import org.elasticsearch.index.engine.EngineFactory;
import org.elasticsearch.index.engine.InternalEngineFactory;
import org.elasticsearch.index.shard.IndexSearcherWrapper;
import org.elasticsearch.test.engine.MockEngineFactory;
public class IndexModuleTests extends ModuleTestCase {
public void testWrapperIsBound() {
IndexModule module = new IndexModule();
assertInstanceBinding(module, IndexSearcherWrapper.class,(x) -> x == null);
module.indexSearcherWrapper = Wrapper.class;
assertBinding(module, IndexSearcherWrapper.class, Wrapper.class);
}
public void testEngineFactoryBound() {
IndexModule module = new IndexModule();
assertBinding(module, EngineFactory.class, InternalEngineFactory.class);
module.engineFactoryImpl = MockEngineFactory.class;
assertBinding(module, EngineFactory.class, MockEngineFactory.class);
}
public void testOtherServiceBound() {
IndexModule module = new IndexModule();
assertBinding(module, IndexService.class, IndexService.class);
assertBinding(module, IndexServicesProvider.class, IndexServicesProvider.class);
}
public static final class Wrapper implements IndexSearcherWrapper {
@Override
public DirectoryReader wrap(DirectoryReader reader) {
return null;
}
@Override
public IndexSearcher wrap(EngineConfig engineConfig, IndexSearcher searcher) throws EngineException {
return null;
}
}
}

View File

@ -17,19 +17,19 @@
* under the License.
*/
package org.elasticsearch.index.shard;
package org.elasticsearch.index;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.test.ESTestCase;
import org.junit.Test;
/** Unit test(s) for IndexShardModule */
public class IndexShardModuleTests extends ESTestCase {
/** Unit test(s) for IndexService */
public class IndexServiceTests extends ESTestCase {
@Test
public void testDetermineShadowEngineShouldBeUsed() {
ShardId shardId = new ShardId("myindex", 0);
Settings regularSettings = Settings.builder()
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 2)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)
@ -41,14 +41,9 @@ public class IndexShardModuleTests extends ESTestCase {
.put(IndexMetaData.SETTING_SHADOW_REPLICAS, true)
.build();
IndexShardModule ism1 = new IndexShardModule(shardId, true, regularSettings);
IndexShardModule ism2 = new IndexShardModule(shardId, false, regularSettings);
IndexShardModule ism3 = new IndexShardModule(shardId, true, shadowSettings);
IndexShardModule ism4 = new IndexShardModule(shardId, false, shadowSettings);
assertFalse("no shadow replicas for normal settings", ism1.useShadowEngine());
assertFalse("no shadow replicas for normal settings", ism2.useShadowEngine());
assertFalse("no shadow replicas for primary shard with shadow settings", ism3.useShadowEngine());
assertTrue("shadow replicas for replica shards with shadow settings", ism4.useShadowEngine());
assertFalse("no shadow replicas for normal settings", IndexService.useShadowEngine(true, regularSettings));
assertFalse("no shadow replicas for normal settings", IndexService.useShadowEngine(false, regularSettings));
assertFalse("no shadow replicas for primary shard with shadow settings", IndexService.useShadowEngine(true, shadowSettings));
assertTrue("shadow replicas for replica shards with shadow settings",IndexService.useShadowEngine(false, shadowSettings));
}
}

View File

@ -150,7 +150,7 @@ public class IndexWithShadowReplicasIT extends ESIntegTestCase {
for (IndicesService service : internalCluster().getDataNodeInstances(IndicesService.class)) {
if (service.hasIndex("foo-copy")) {
IndexShard shard = service.indexServiceSafe("foo-copy").shard(0);
IndexShard shard = service.indexServiceSafe("foo-copy").getShardOrNull(0);
if (shard.routingEntry().primary()) {
assertFalse(shard instanceof ShadowIndexShard);
} else {

View File

@ -16,10 +16,11 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.shard;
package org.elasticsearch.index;
import org.elasticsearch.common.inject.Module;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexModule;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.engine.MockEngineFactory;
import org.elasticsearch.test.engine.MockEngineSupportModule;
@ -27,7 +28,7 @@ import org.elasticsearch.test.engine.MockEngineSupportModule;
import java.util.Collection;
import java.util.Collections;
// this must exist in the same package as IndexShardModule to allow access to setting the impl
// this must exist in the same package as IndexModule to allow access to setting the impl
public class MockEngineFactoryPlugin extends Plugin {
@Override
public String name() {
@ -41,7 +42,7 @@ public class MockEngineFactoryPlugin extends Plugin {
public Collection<Module> indexModules(Settings indexSettings) {
return Collections.<Module>singletonList(new MockEngineSupportModule());
}
public void onModule(IndexShardModule module) {
public void onModule(IndexModule module) {
module.engineFactoryImpl = MockEngineFactory.class;
}
}

View File

@ -97,7 +97,7 @@ public class CodecTests extends ESSingleNodeTestCase {
private static CodecService createCodecService(Settings settings) {
IndexService indexService = createIndex("test", settings);
return indexService.injector().getInstance(CodecService.class);
return indexService.getIndexServices().getCodecService();
}
}

View File

@ -21,6 +21,7 @@ package org.elasticsearch.index.engine;
import org.apache.lucene.index.LiveIndexWriterConfig;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.shard.EngineAccess;
import org.elasticsearch.test.ESSingleNodeTestCase;
import java.util.concurrent.TimeUnit;
@ -33,7 +34,7 @@ public class InternalEngineSettingsTests extends ESSingleNodeTestCase {
public void testSettingsUpdate() {
final IndexService service = createIndex("foo");
// INDEX_COMPOUND_ON_FLUSH
InternalEngine engine = ((InternalEngine)engine(service));
InternalEngine engine = ((InternalEngine) EngineAccess.engine(service.getShardOrNull(0)));
assertThat(engine.getCurrentIndexWriterConfig().getUseCompoundFile(), is(true));
client().admin().indices().prepareUpdateSettings("foo").setSettings(Settings.builder().put(EngineConfig.INDEX_COMPOUND_ON_FLUSH, false).build()).get();
assertThat(engine.getCurrentIndexWriterConfig().getUseCompoundFile(), is(false));

View File

@ -67,10 +67,7 @@ import org.elasticsearch.index.mapper.ParseContext.Document;
import org.elasticsearch.index.mapper.internal.SourceFieldMapper;
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import org.elasticsearch.index.mapper.object.RootObjectMapper;
import org.elasticsearch.index.shard.MergeSchedulerConfig;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardUtils;
import org.elasticsearch.index.shard.TranslogRecoveryPerformer;
import org.elasticsearch.index.shard.*;
import org.elasticsearch.index.similarity.SimilarityLookupService;
import org.elasticsearch.index.store.DirectoryService;
import org.elasticsearch.index.store.DirectoryUtils;
@ -232,15 +229,15 @@ public class InternalEngineTests extends ESTestCase {
return new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy());
}
protected InternalEngine createEngine(Store store, Path translogPath, IndexSearcherWrapper... wrappers) {
return createEngine(defaultSettings, store, translogPath, new MergeSchedulerConfig(defaultSettings), newMergePolicy(), wrappers);
protected InternalEngine createEngine(Store store, Path translogPath) {
return createEngine(defaultSettings, store, translogPath, new MergeSchedulerConfig(defaultSettings), newMergePolicy());
}
protected InternalEngine createEngine(Settings indexSettings, Store store, Path translogPath, MergeSchedulerConfig mergeSchedulerConfig, MergePolicy mergePolicy, IndexSearcherWrapper... wrappers) {
return new InternalEngine(config(indexSettings, store, translogPath, mergeSchedulerConfig, mergePolicy, wrappers), false);
protected InternalEngine createEngine(Settings indexSettings, Store store, Path translogPath, MergeSchedulerConfig mergeSchedulerConfig, MergePolicy mergePolicy) {
return new InternalEngine(config(indexSettings, store, translogPath, mergeSchedulerConfig, mergePolicy), false);
}
public EngineConfig config(Settings indexSettings, Store store, Path translogPath, MergeSchedulerConfig mergeSchedulerConfig, MergePolicy mergePolicy, IndexSearcherWrapper... wrappers) {
public EngineConfig config(Settings indexSettings, Store store, Path translogPath, MergeSchedulerConfig mergeSchedulerConfig, MergePolicy mergePolicy) {
IndexWriterConfig iwc = newIndexWriterConfig();
TranslogConfig translogConfig = new TranslogConfig(shardId, translogPath, indexSettings, Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, threadPool);
@ -251,7 +248,7 @@ public class InternalEngineTests extends ESTestCase {
public void onFailedEngine(ShardId shardId, String reason, @Nullable Throwable t) {
// we don't need to notify anybody in this test
}
}, new TranslogHandler(shardId.index().getName(), logger), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), new IndexSearcherWrappingService(new HashSet<>(Arrays.asList(wrappers))), translogConfig);
}, new TranslogHandler(shardId.index().getName(), logger), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig);
try {
config.setCreate(Lucene.indexExists(store.directory()) == false);
} catch (IOException e) {
@ -491,8 +488,7 @@ public class InternalEngineTests extends ESTestCase {
assertThat(stats2.getUserData(), hasKey(Translog.TRANSLOG_GENERATION_KEY));
assertThat(stats2.getUserData(), hasKey(Translog.TRANSLOG_UUID_KEY));
assertThat(stats2.getUserData().get(Translog.TRANSLOG_GENERATION_KEY), not(equalTo(stats1.getUserData().get(Translog.TRANSLOG_GENERATION_KEY))));
assertThat(stats2.getUserData().get(Translog.TRANSLOG_UUID_KEY), equalTo(stats1.getUserData().get(Translog.TRANSLOG_UUID_KEY)))
;
assertThat(stats2.getUserData().get(Translog.TRANSLOG_UUID_KEY), equalTo(stats1.getUserData().get(Translog.TRANSLOG_UUID_KEY)));
}
@Test
@ -514,8 +510,11 @@ public class InternalEngineTests extends ESTestCase {
};
Store store = createStore();
Path translog = createTempDir("translog-test");
InternalEngine engine = createEngine(store, translog, wrapper);
Engine.Searcher searcher = engine.acquireSearcher("test");
InternalEngine engine = createEngine(store, translog);
engine.close();
engine = new InternalEngine(engine.config(), false);
Engine.Searcher searcher = wrapper.wrap(engine.config(), engine.acquireSearcher("test"));
assertThat(counter.get(), equalTo(2));
searcher.close();
IOUtils.close(store, engine);
@ -1951,7 +1950,7 @@ public class InternalEngineTests extends ESTestCase {
EngineConfig brokenConfig = new EngineConfig(shardId, threadPool, config.getIndexingService(), config.getIndexSettings()
, null, store, createSnapshotDeletionPolicy(), newMergePolicy(), config.getMergeSchedulerConfig(),
config.getAnalyzer(), config.getSimilarity(), new CodecService(shardId.index()), config.getFailedEngineListener()
, config.getTranslogRecoveryPerformer(), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), new IndexSearcherWrappingService(), translogConfig);
, config.getTranslogRecoveryPerformer(), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig);
try {
new InternalEngine(brokenConfig, false);

View File

@ -216,7 +216,7 @@ public class ShadowEngineTests extends ESTestCase {
@Override
public void onFailedEngine(ShardId shardId, String reason, @Nullable Throwable t) {
// we don't need to notify anybody in this test
}}, null, IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), new IndexSearcherWrappingService(), translogConfig);
}}, null, IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig);
try {
config.setCreate(Lucene.indexExists(store.directory()) == false);
} catch (IOException e) {

View File

@ -102,7 +102,7 @@ public abstract class AbstractFieldDataTestCase extends ESSingleNodeTestCase {
Settings settings = Settings.builder().put("index.fielddata.cache", "none").build();
indexService = createIndex("test", settings);
mapperService = indexService.mapperService();
indicesFieldDataCache = indexService.injector().getInstance(IndicesFieldDataCache.class);
indicesFieldDataCache = getInstanceFromNode(IndicesFieldDataCache.class);
ifdService = indexService.fieldData();
// LogByteSizeMP to preserve doc ID order
writer = new IndexWriter(new RAMDirectory(), new IndexWriterConfig(new StandardAnalyzer()).setMergePolicy(new LogByteSizeMergePolicy()));

View File

@ -433,7 +433,7 @@ public class SimpleAllMapperTests extends ESSingleNodeTestCase {
client().prepareIndex(index, "type").setSource("foo", "bar").get();
client().admin().indices().prepareRefresh(index).get();
Query query = indexService.mapperService().documentMapper("type").allFieldMapper().fieldType().termQuery("bar", null);
try (Searcher searcher = indexService.shard(0).acquireSearcher("tests")) {
try (Searcher searcher = indexService.getShardOrNull(0).acquireSearcher("tests")) {
query = searcher.searcher().rewrite(query);
final Class<?> expected = boost ? AllTermQuery.class : TermQuery.class;
assertThat(query, Matchers.instanceOf(expected));

View File

@ -190,7 +190,36 @@ public class HasChildQueryBuilderTests extends AbstractQueryTestCase<HasChildQue
}
public void testParseFromJSON() throws IOException {
String query = copyToStringFromClasspath("/org/elasticsearch/index/query/has-child-with-inner-hits.json");
String query = "{\n" +
" \"has_child\" : {\n" +
" \"query\" : {\n" +
" \"range\" : {\n" +
" \"mapped_string\" : {\n" +
" \"from\" : \"agJhRET\",\n" +
" \"to\" : \"zvqIq\",\n" +
" \"include_lower\" : true,\n" +
" \"include_upper\" : true,\n" +
" \"boost\" : 1.0\n" +
" }\n" +
" }\n" +
" },\n" +
" \"child_type\" : \"child\",\n" +
" \"score_mode\" : \"avg\",\n" +
" \"min_children\" : 883170873,\n" +
" \"max_children\" : 1217235442,\n" +
" \"boost\" : 2.0,\n" +
" \"_name\" : \"WNzYMJKRwePuRBh\",\n" +
" \"inner_hits\" : {\n" +
" \"name\" : \"inner_hits_name\",\n" +
" \"size\" : 100,\n" +
" \"sort\" : [ {\n" +
" \"mapped_string\" : {\n" +
" \"order\" : \"asc\"\n" +
" }\n" +
" } ]\n" +
" }\n" +
" }\n" +
"}";
HasChildQueryBuilder queryBuilder = (HasChildQueryBuilder) parseQuery(query);
assertEquals(query, queryBuilder.maxChildren(), 1217235442);
assertEquals(query, queryBuilder.minChildren(), 883170873);

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