Merge branch 'master' into feature/client_aggs_parsing
This commit is contained in:
commit
01b976071f
|
@ -35,7 +35,6 @@ import org.elasticsearch.cluster.routing.ShardRouting;
|
||||||
import org.elasticsearch.common.Nullable;
|
import org.elasticsearch.common.Nullable;
|
||||||
import org.elasticsearch.common.io.stream.StreamInput;
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
import org.elasticsearch.common.util.set.Sets;
|
import org.elasticsearch.common.util.set.Sets;
|
||||||
import org.elasticsearch.index.engine.VersionConflictEngineException;
|
|
||||||
import org.elasticsearch.index.shard.ShardId;
|
import org.elasticsearch.index.shard.ShardId;
|
||||||
import org.elasticsearch.rest.RestStatus;
|
import org.elasticsearch.rest.RestStatus;
|
||||||
|
|
||||||
|
@ -126,7 +125,7 @@ public class ReplicationOperation<
|
||||||
|
|
||||||
markUnavailableShardsAsStale(replicaRequest, inSyncAllocationIds, shards);
|
markUnavailableShardsAsStale(replicaRequest, inSyncAllocationIds, shards);
|
||||||
|
|
||||||
performOnReplicas(replicaRequest, shards);
|
performOnReplicas(replicaRequest, primary.globalCheckpoint(), shards);
|
||||||
}
|
}
|
||||||
|
|
||||||
successfulShards.incrementAndGet(); // mark primary as successful
|
successfulShards.incrementAndGet(); // mark primary as successful
|
||||||
|
@ -154,7 +153,7 @@ public class ReplicationOperation<
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void performOnReplicas(ReplicaRequest replicaRequest, List<ShardRouting> shards) {
|
private void performOnReplicas(final ReplicaRequest replicaRequest, final long globalCheckpoint, final List<ShardRouting> shards) {
|
||||||
final String localNodeId = primary.routingEntry().currentNodeId();
|
final String localNodeId = primary.routingEntry().currentNodeId();
|
||||||
// If the index gets deleted after primary operation, we skip replication
|
// If the index gets deleted after primary operation, we skip replication
|
||||||
for (final ShardRouting shard : shards) {
|
for (final ShardRouting shard : shards) {
|
||||||
|
@ -166,23 +165,23 @@ public class ReplicationOperation<
|
||||||
}
|
}
|
||||||
|
|
||||||
if (shard.currentNodeId().equals(localNodeId) == false) {
|
if (shard.currentNodeId().equals(localNodeId) == false) {
|
||||||
performOnReplica(shard, replicaRequest);
|
performOnReplica(shard, replicaRequest, globalCheckpoint);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (shard.relocating() && shard.relocatingNodeId().equals(localNodeId) == false) {
|
if (shard.relocating() && shard.relocatingNodeId().equals(localNodeId) == false) {
|
||||||
performOnReplica(shard.getTargetRelocatingShard(), replicaRequest);
|
performOnReplica(shard.getTargetRelocatingShard(), replicaRequest, globalCheckpoint);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void performOnReplica(final ShardRouting shard, final ReplicaRequest replicaRequest) {
|
private void performOnReplica(final ShardRouting shard, final ReplicaRequest replicaRequest, final long globalCheckpoint) {
|
||||||
if (logger.isTraceEnabled()) {
|
if (logger.isTraceEnabled()) {
|
||||||
logger.trace("[{}] sending op [{}] to replica {} for request [{}]", shard.shardId(), opType, shard, replicaRequest);
|
logger.trace("[{}] sending op [{}] to replica {} for request [{}]", shard.shardId(), opType, shard, replicaRequest);
|
||||||
}
|
}
|
||||||
|
|
||||||
totalShards.incrementAndGet();
|
totalShards.incrementAndGet();
|
||||||
pendingActions.incrementAndGet();
|
pendingActions.incrementAndGet();
|
||||||
replicasProxy.performOn(shard, replicaRequest, new ActionListener<ReplicaResponse>() {
|
replicasProxy.performOn(shard, replicaRequest, globalCheckpoint, new ActionListener<ReplicaResponse>() {
|
||||||
@Override
|
@Override
|
||||||
public void onResponse(ReplicaResponse response) {
|
public void onResponse(ReplicaResponse response) {
|
||||||
successfulShards.incrementAndGet();
|
successfulShards.incrementAndGet();
|
||||||
|
@ -347,8 +346,20 @@ public class ReplicationOperation<
|
||||||
*/
|
*/
|
||||||
void updateLocalCheckpointForShard(String allocationId, long checkpoint);
|
void updateLocalCheckpointForShard(String allocationId, long checkpoint);
|
||||||
|
|
||||||
/** returns the local checkpoint of the primary shard */
|
/**
|
||||||
|
* Returns the local checkpoint on the primary shard.
|
||||||
|
*
|
||||||
|
* @return the local checkpoint
|
||||||
|
*/
|
||||||
long localCheckpoint();
|
long localCheckpoint();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the global checkpoint on the primary shard.
|
||||||
|
*
|
||||||
|
* @return the global checkpoint
|
||||||
|
*/
|
||||||
|
long globalCheckpoint();
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -357,13 +368,14 @@ public class ReplicationOperation<
|
||||||
public interface Replicas<RequestT extends ReplicationRequest<RequestT>> {
|
public interface Replicas<RequestT extends ReplicationRequest<RequestT>> {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* performs the the given request on the specified replica
|
* Performs the the specified request on the specified replica.
|
||||||
*
|
*
|
||||||
* @param replica {@link ShardRouting} of the shard this request should be executed on
|
* @param replica the shard this request should be executed on
|
||||||
* @param replicaRequest operation to perform
|
* @param replicaRequest the operation to perform
|
||||||
* @param listener a callback to call once the operation has been complicated, either successfully or with an error.
|
* @param globalCheckpoint the global checkpoint on the primary
|
||||||
|
* @param listener callback for handling the response or failure
|
||||||
*/
|
*/
|
||||||
void performOn(ShardRouting replica, RequestT replicaRequest, ActionListener<ReplicaResponse> listener);
|
void performOn(ShardRouting replica, RequestT replicaRequest, long globalCheckpoint, ActionListener<ReplicaResponse> listener);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Fail the specified shard if needed, removing it from the current set
|
* Fail the specified shard if needed, removing it from the current set
|
||||||
|
|
|
@ -129,7 +129,7 @@ public abstract class TransportReplicationAction<
|
||||||
new PrimaryOperationTransportHandler());
|
new PrimaryOperationTransportHandler());
|
||||||
// we must never reject on because of thread pool capacity on replicas
|
// we must never reject on because of thread pool capacity on replicas
|
||||||
transportService.registerRequestHandler(transportReplicaAction,
|
transportService.registerRequestHandler(transportReplicaAction,
|
||||||
() -> new ConcreteShardRequest<>(replicaRequest),
|
() -> new ConcreteReplicaRequest<>(replicaRequest),
|
||||||
executor, true, true,
|
executor, true, true,
|
||||||
new ReplicaOperationTransportHandler());
|
new ReplicaOperationTransportHandler());
|
||||||
|
|
||||||
|
@ -155,15 +155,13 @@ public abstract class TransportReplicationAction<
|
||||||
protected abstract Response newResponseInstance();
|
protected abstract Response newResponseInstance();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Resolves derived values in the request. For example, the target shard id of the incoming request,
|
* Resolves derived values in the request. For example, the target shard id of the incoming request, if not set at request construction.
|
||||||
* if not set at request construction
|
|
||||||
* Additional processing or validation of the request should be done here.
|
* Additional processing or validation of the request should be done here.
|
||||||
*
|
*
|
||||||
* @param metaData cluster state metadata
|
|
||||||
* @param indexMetaData index metadata of the concrete index this request is going to operate on
|
* @param indexMetaData index metadata of the concrete index this request is going to operate on
|
||||||
* @param request the request to resolve
|
* @param request the request to resolve
|
||||||
*/
|
*/
|
||||||
protected void resolveRequest(MetaData metaData, IndexMetaData indexMetaData, Request request) {
|
protected void resolveRequest(final IndexMetaData indexMetaData, final Request request) {
|
||||||
if (request.waitForActiveShards() == ActiveShardCount.DEFAULT) {
|
if (request.waitForActiveShards() == ActiveShardCount.DEFAULT) {
|
||||||
// if the wait for active shard count has not been set in the request,
|
// if the wait for active shard count has not been set in the request,
|
||||||
// resolve it from the index settings
|
// resolve it from the index settings
|
||||||
|
@ -441,18 +439,28 @@ public abstract class TransportReplicationAction<
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
class ReplicaOperationTransportHandler implements TransportRequestHandler<ConcreteShardRequest<ReplicaRequest>> {
|
class ReplicaOperationTransportHandler implements TransportRequestHandler<ConcreteReplicaRequest<ReplicaRequest>> {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void messageReceived(final ConcreteShardRequest<ReplicaRequest> request, final TransportChannel channel)
|
public void messageReceived(
|
||||||
throws Exception {
|
final ConcreteReplicaRequest<ReplicaRequest> replicaRequest, final TransportChannel channel) throws Exception {
|
||||||
throw new UnsupportedOperationException("the task parameter is required for this operation");
|
throw new UnsupportedOperationException("the task parameter is required for this operation");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void messageReceived(ConcreteShardRequest<ReplicaRequest> requestWithAID, TransportChannel channel, Task task)
|
public void messageReceived(
|
||||||
|
final ConcreteReplicaRequest<ReplicaRequest> replicaRequest,
|
||||||
|
final TransportChannel channel,
|
||||||
|
final Task task)
|
||||||
throws Exception {
|
throws Exception {
|
||||||
new AsyncReplicaAction(requestWithAID.request, requestWithAID.targetAllocationID, channel, (ReplicationTask) task).run();
|
new AsyncReplicaAction(
|
||||||
|
replicaRequest.getRequest(),
|
||||||
|
replicaRequest.getTargetAllocationID(),
|
||||||
|
replicaRequest.getGlobalCheckpoint(),
|
||||||
|
channel,
|
||||||
|
(ReplicationTask) task).run();
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class RetryOnReplicaException extends ElasticsearchException {
|
public static class RetryOnReplicaException extends ElasticsearchException {
|
||||||
|
@ -471,6 +479,7 @@ public abstract class TransportReplicationAction<
|
||||||
private final ReplicaRequest request;
|
private final ReplicaRequest request;
|
||||||
// allocation id of the replica this request is meant for
|
// allocation id of the replica this request is meant for
|
||||||
private final String targetAllocationID;
|
private final String targetAllocationID;
|
||||||
|
private final long globalCheckpoint;
|
||||||
private final TransportChannel channel;
|
private final TransportChannel channel;
|
||||||
private final IndexShard replica;
|
private final IndexShard replica;
|
||||||
/**
|
/**
|
||||||
|
@ -481,11 +490,17 @@ public abstract class TransportReplicationAction<
|
||||||
// something we want to avoid at all costs
|
// something we want to avoid at all costs
|
||||||
private final ClusterStateObserver observer = new ClusterStateObserver(clusterService, null, logger, threadPool.getThreadContext());
|
private final ClusterStateObserver observer = new ClusterStateObserver(clusterService, null, logger, threadPool.getThreadContext());
|
||||||
|
|
||||||
AsyncReplicaAction(ReplicaRequest request, String targetAllocationID, TransportChannel channel, ReplicationTask task) {
|
AsyncReplicaAction(
|
||||||
|
ReplicaRequest request,
|
||||||
|
String targetAllocationID,
|
||||||
|
long globalCheckpoint,
|
||||||
|
TransportChannel channel,
|
||||||
|
ReplicationTask task) {
|
||||||
this.request = request;
|
this.request = request;
|
||||||
this.channel = channel;
|
this.channel = channel;
|
||||||
this.task = task;
|
this.task = task;
|
||||||
this.targetAllocationID = targetAllocationID;
|
this.targetAllocationID = targetAllocationID;
|
||||||
|
this.globalCheckpoint = globalCheckpoint;
|
||||||
final ShardId shardId = request.shardId();
|
final ShardId shardId = request.shardId();
|
||||||
assert shardId != null : "request shardId must be set";
|
assert shardId != null : "request shardId must be set";
|
||||||
this.replica = getIndexShard(shardId);
|
this.replica = getIndexShard(shardId);
|
||||||
|
@ -494,12 +509,13 @@ public abstract class TransportReplicationAction<
|
||||||
@Override
|
@Override
|
||||||
public void onResponse(Releasable releasable) {
|
public void onResponse(Releasable releasable) {
|
||||||
try {
|
try {
|
||||||
ReplicaResult replicaResult = shardOperationOnReplica(request, replica);
|
replica.updateGlobalCheckpointOnReplica(globalCheckpoint);
|
||||||
|
final ReplicaResult replicaResult = shardOperationOnReplica(request, replica);
|
||||||
releasable.close(); // release shard operation lock before responding to caller
|
releasable.close(); // release shard operation lock before responding to caller
|
||||||
final TransportReplicationAction.ReplicaResponse response =
|
final TransportReplicationAction.ReplicaResponse response =
|
||||||
new ReplicaResponse(replica.routingEntry().allocationId().getId(), replica.getLocalCheckpoint());
|
new ReplicaResponse(replica.routingEntry().allocationId().getId(), replica.getLocalCheckpoint());
|
||||||
replicaResult.respond(new ResponseListener(response));
|
replicaResult.respond(new ResponseListener(response));
|
||||||
} catch (Exception e) {
|
} catch (final Exception e) {
|
||||||
Releasables.closeWhileHandlingException(releasable); // release shard operation lock before responding to caller
|
Releasables.closeWhileHandlingException(releasable); // release shard operation lock before responding to caller
|
||||||
AsyncReplicaAction.this.onFailure(e);
|
AsyncReplicaAction.this.onFailure(e);
|
||||||
}
|
}
|
||||||
|
@ -656,7 +672,7 @@ public abstract class TransportReplicationAction<
|
||||||
}
|
}
|
||||||
|
|
||||||
// resolve all derived request fields, so we can route and apply it
|
// resolve all derived request fields, so we can route and apply it
|
||||||
resolveRequest(state.metaData(), indexMetaData, request);
|
resolveRequest(indexMetaData, request);
|
||||||
assert request.shardId() != null : "request shardId must be set in resolveRequest";
|
assert request.shardId() != null : "request shardId must be set in resolveRequest";
|
||||||
assert request.waitForActiveShards() != ActiveShardCount.DEFAULT : "request waitForActiveShards must be set in resolveRequest";
|
assert request.waitForActiveShards() != ActiveShardCount.DEFAULT : "request waitForActiveShards must be set in resolveRequest";
|
||||||
|
|
||||||
|
@ -974,6 +990,11 @@ public abstract class TransportReplicationAction<
|
||||||
return indexShard.getLocalCheckpoint();
|
return indexShard.getLocalCheckpoint();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long globalCheckpoint() {
|
||||||
|
return indexShard.getGlobalCheckpoint();
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -1033,16 +1054,20 @@ public abstract class TransportReplicationAction<
|
||||||
class ReplicasProxy implements ReplicationOperation.Replicas<ReplicaRequest> {
|
class ReplicasProxy implements ReplicationOperation.Replicas<ReplicaRequest> {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void performOn(ShardRouting replica, ReplicaRequest request, ActionListener<ReplicationOperation.ReplicaResponse> listener) {
|
public void performOn(
|
||||||
|
final ShardRouting replica,
|
||||||
|
final ReplicaRequest request,
|
||||||
|
final long globalCheckpoint,
|
||||||
|
final ActionListener<ReplicationOperation.ReplicaResponse> listener) {
|
||||||
String nodeId = replica.currentNodeId();
|
String nodeId = replica.currentNodeId();
|
||||||
final DiscoveryNode node = clusterService.state().nodes().get(nodeId);
|
final DiscoveryNode node = clusterService.state().nodes().get(nodeId);
|
||||||
if (node == null) {
|
if (node == null) {
|
||||||
listener.onFailure(new NoNodeAvailableException("unknown node [" + nodeId + "]"));
|
listener.onFailure(new NoNodeAvailableException("unknown node [" + nodeId + "]"));
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
final ConcreteShardRequest<ReplicaRequest> concreteShardRequest =
|
final ConcreteReplicaRequest<ReplicaRequest> replicaRequest =
|
||||||
new ConcreteShardRequest<>(request, replica.allocationId().getId());
|
new ConcreteReplicaRequest<>(request, replica.allocationId().getId(), globalCheckpoint);
|
||||||
sendReplicaRequest(concreteShardRequest, node, listener);
|
sendReplicaRequest(replicaRequest, node, listener);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -1066,16 +1091,23 @@ public abstract class TransportReplicationAction<
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/** sends the given replica request to the supplied nodes */
|
/**
|
||||||
protected void sendReplicaRequest(ConcreteShardRequest<ReplicaRequest> concreteShardRequest, DiscoveryNode node,
|
* Sends the specified replica request to the specified node.
|
||||||
ActionListener<ReplicationOperation.ReplicaResponse> listener) {
|
*
|
||||||
transportService.sendRequest(node, transportReplicaAction, concreteShardRequest, transportOptions,
|
* @param replicaRequest the replica request
|
||||||
// Eclipse can't handle when this is <> so we specify the type here.
|
* @param node the node to send the request to
|
||||||
new ActionListenerResponseHandler<ReplicaResponse>(listener, ReplicaResponse::new));
|
* @param listener callback for handling the response or failure
|
||||||
|
*/
|
||||||
|
protected void sendReplicaRequest(
|
||||||
|
final ConcreteReplicaRequest<ReplicaRequest> replicaRequest,
|
||||||
|
final DiscoveryNode node,
|
||||||
|
final ActionListener<ReplicationOperation.ReplicaResponse> listener) {
|
||||||
|
final ActionListenerResponseHandler<ReplicaResponse> handler = new ActionListenerResponseHandler<>(listener, ReplicaResponse::new);
|
||||||
|
transportService.sendRequest(node, transportReplicaAction, replicaRequest, transportOptions, handler);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** a wrapper class to encapsulate a request when being sent to a specific allocation id **/
|
/** a wrapper class to encapsulate a request when being sent to a specific allocation id **/
|
||||||
public static final class ConcreteShardRequest<R extends TransportRequest> extends TransportRequest {
|
public static class ConcreteShardRequest<R extends TransportRequest> extends TransportRequest {
|
||||||
|
|
||||||
/** {@link AllocationId#getId()} of the shard this request is sent to **/
|
/** {@link AllocationId#getId()} of the shard this request is sent to **/
|
||||||
private String targetAllocationID;
|
private String targetAllocationID;
|
||||||
|
@ -1145,6 +1177,49 @@ public abstract class TransportReplicationAction<
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected static final class ConcreteReplicaRequest<R extends TransportRequest> extends ConcreteShardRequest<R> {
|
||||||
|
|
||||||
|
private long globalCheckpoint;
|
||||||
|
|
||||||
|
public ConcreteReplicaRequest(final Supplier<R> requestSupplier) {
|
||||||
|
super(requestSupplier);
|
||||||
|
}
|
||||||
|
|
||||||
|
public ConcreteReplicaRequest(final R request, final String targetAllocationID, final long globalCheckpoint) {
|
||||||
|
super(request, targetAllocationID);
|
||||||
|
this.globalCheckpoint = globalCheckpoint;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void readFrom(StreamInput in) throws IOException {
|
||||||
|
super.readFrom(in);
|
||||||
|
if (in.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) {
|
||||||
|
globalCheckpoint = in.readZLong();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
super.writeTo(out);
|
||||||
|
if (out.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) {
|
||||||
|
out.writeZLong(globalCheckpoint);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getGlobalCheckpoint() {
|
||||||
|
return globalCheckpoint;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return "ConcreteReplicaRequest{" +
|
||||||
|
"targetAllocationID='" + getTargetAllocationID() + '\'' +
|
||||||
|
", request=" + getRequest() +
|
||||||
|
", globalCheckpoint=" + globalCheckpoint +
|
||||||
|
'}';
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Sets the current phase on the task if it isn't null. Pulled into its own
|
* Sets the current phase on the task if it isn't null. Pulled into its own
|
||||||
* method because its more convenient that way.
|
* method because its more convenient that way.
|
||||||
|
|
|
@ -363,7 +363,7 @@ public class MetaDataCreateIndexService extends AbstractComponent {
|
||||||
(tmpImd.getNumberOfReplicas() + 1) + "]");
|
(tmpImd.getNumberOfReplicas() + 1) + "]");
|
||||||
}
|
}
|
||||||
// create the index here (on the master) to validate it can be created, as well as adding the mapping
|
// create the index here (on the master) to validate it can be created, as well as adding the mapping
|
||||||
final IndexService indexService = indicesService.createIndex(tmpImd, Collections.emptyList(), shardId -> {});
|
final IndexService indexService = indicesService.createIndex(tmpImd, Collections.emptyList());
|
||||||
createdIndex = indexService.index();
|
createdIndex = indexService.index();
|
||||||
// now add the mappings
|
// now add the mappings
|
||||||
MapperService mapperService = indexService.mapperService();
|
MapperService mapperService = indexService.mapperService();
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package org.elasticsearch.cluster.metadata;
|
package org.elasticsearch.cluster.metadata;
|
||||||
|
|
||||||
import com.carrotsearch.hppc.cursors.ObjectCursor;
|
|
||||||
import org.elasticsearch.ElasticsearchException;
|
import org.elasticsearch.ElasticsearchException;
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
import org.elasticsearch.action.admin.indices.alias.IndicesAliasesClusterStateUpdateRequest;
|
import org.elasticsearch.action.admin.indices.alias.IndicesAliasesClusterStateUpdateRequest;
|
||||||
|
@ -140,7 +139,7 @@ public class MetaDataIndexAliasesService extends AbstractComponent {
|
||||||
if (indexService == null) {
|
if (indexService == null) {
|
||||||
// temporarily create the index and add mappings so we can parse the filter
|
// temporarily create the index and add mappings so we can parse the filter
|
||||||
try {
|
try {
|
||||||
indexService = indicesService.createIndex(index, emptyList(), shardId -> {});
|
indexService = indicesService.createIndex(index, emptyList());
|
||||||
indicesToClose.add(index.getIndex());
|
indicesToClose.add(index.getIndex());
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new ElasticsearchException("Failed to create temporary index for parsing the alias", e);
|
throw new ElasticsearchException("Failed to create temporary index for parsing the alias", e);
|
||||||
|
|
|
@ -213,7 +213,7 @@ public class MetaDataIndexTemplateService extends AbstractComponent {
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
final IndexMetaData tmpIndexMetadata = IndexMetaData.builder(temporaryIndexName).settings(dummySettings).build();
|
final IndexMetaData tmpIndexMetadata = IndexMetaData.builder(temporaryIndexName).settings(dummySettings).build();
|
||||||
IndexService dummyIndexService = indicesService.createIndex(tmpIndexMetadata, Collections.emptyList(), shardId -> {});
|
IndexService dummyIndexService = indicesService.createIndex(tmpIndexMetadata, Collections.emptyList());
|
||||||
createdIndex = dummyIndexService.index();
|
createdIndex = dummyIndexService.index();
|
||||||
|
|
||||||
templateBuilder.order(request.order);
|
templateBuilder.order(request.order);
|
||||||
|
|
|
@ -145,7 +145,7 @@ public class MetaDataMappingService extends AbstractComponent {
|
||||||
IndexService indexService = indicesService.indexService(indexMetaData.getIndex());
|
IndexService indexService = indicesService.indexService(indexMetaData.getIndex());
|
||||||
if (indexService == null) {
|
if (indexService == null) {
|
||||||
// we need to create the index here, and add the current mapping to it, so we can merge
|
// we need to create the index here, and add the current mapping to it, so we can merge
|
||||||
indexService = indicesService.createIndex(indexMetaData, Collections.emptyList(), shardId -> {});
|
indexService = indicesService.createIndex(indexMetaData, Collections.emptyList());
|
||||||
removeIndex = true;
|
removeIndex = true;
|
||||||
indexService.mapperService().merge(indexMetaData, MergeReason.MAPPING_RECOVERY, true);
|
indexService.mapperService().merge(indexMetaData, MergeReason.MAPPING_RECOVERY, true);
|
||||||
}
|
}
|
||||||
|
|
|
@ -115,7 +115,6 @@ public final class IndexScopedSettings extends AbstractScopedSettings {
|
||||||
IndexSettings.QUERY_STRING_LENIENT_SETTING,
|
IndexSettings.QUERY_STRING_LENIENT_SETTING,
|
||||||
IndexSettings.ALLOW_UNMAPPED,
|
IndexSettings.ALLOW_UNMAPPED,
|
||||||
IndexSettings.INDEX_CHECK_ON_STARTUP,
|
IndexSettings.INDEX_CHECK_ON_STARTUP,
|
||||||
IndexSettings.INDEX_SEQ_NO_CHECKPOINT_SYNC_INTERVAL,
|
|
||||||
LocalCheckpointTracker.SETTINGS_BIT_ARRAYS_SIZE,
|
LocalCheckpointTracker.SETTINGS_BIT_ARRAYS_SIZE,
|
||||||
IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD,
|
IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD,
|
||||||
IndexSettings.MAX_SLICES_PER_SCROLL,
|
IndexSettings.MAX_SLICES_PER_SCROLL,
|
||||||
|
|
|
@ -55,7 +55,8 @@ public class Environment {
|
||||||
public static final Setting<String> DEFAULT_PATH_CONF_SETTING = Setting.simpleString("default.path.conf", Property.NodeScope);
|
public static final Setting<String> DEFAULT_PATH_CONF_SETTING = Setting.simpleString("default.path.conf", Property.NodeScope);
|
||||||
public static final Setting<String> PATH_CONF_SETTING =
|
public static final Setting<String> PATH_CONF_SETTING =
|
||||||
new Setting<>("path.conf", DEFAULT_PATH_CONF_SETTING, Function.identity(), Property.NodeScope);
|
new Setting<>("path.conf", DEFAULT_PATH_CONF_SETTING, Function.identity(), Property.NodeScope);
|
||||||
public static final Setting<String> PATH_SCRIPTS_SETTING = Setting.simpleString("path.scripts", Property.NodeScope);
|
public static final Setting<String> PATH_SCRIPTS_SETTING =
|
||||||
|
Setting.simpleString("path.scripts", Property.NodeScope, Property.Deprecated);
|
||||||
public static final Setting<List<String>> DEFAULT_PATH_DATA_SETTING =
|
public static final Setting<List<String>> DEFAULT_PATH_DATA_SETTING =
|
||||||
Setting.listSetting("default.path.data", Collections.emptyList(), Function.identity(), Property.NodeScope);
|
Setting.listSetting("default.path.data", Collections.emptyList(), Function.identity(), Property.NodeScope);
|
||||||
public static final Setting<List<String>> PATH_DATA_SETTING =
|
public static final Setting<List<String>> PATH_DATA_SETTING =
|
||||||
|
|
|
@ -320,19 +320,17 @@ public final class IndexModule {
|
||||||
}
|
}
|
||||||
|
|
||||||
public IndexService newIndexService(
|
public IndexService newIndexService(
|
||||||
NodeEnvironment environment,
|
NodeEnvironment environment,
|
||||||
NamedXContentRegistry xContentRegistry,
|
NamedXContentRegistry xContentRegistry,
|
||||||
IndexService.ShardStoreDeleter shardStoreDeleter,
|
IndexService.ShardStoreDeleter shardStoreDeleter,
|
||||||
CircuitBreakerService circuitBreakerService,
|
CircuitBreakerService circuitBreakerService,
|
||||||
BigArrays bigArrays,
|
BigArrays bigArrays,
|
||||||
ThreadPool threadPool,
|
ThreadPool threadPool,
|
||||||
ScriptService scriptService,
|
ScriptService scriptService,
|
||||||
ClusterService clusterService,
|
Client client,
|
||||||
Client client,
|
IndicesQueryCache indicesQueryCache,
|
||||||
IndicesQueryCache indicesQueryCache,
|
MapperRegistry mapperRegistry,
|
||||||
MapperRegistry mapperRegistry,
|
IndicesFieldDataCache indicesFieldDataCache)
|
||||||
Consumer<ShardId> globalCheckpointSyncer,
|
|
||||||
IndicesFieldDataCache indicesFieldDataCache)
|
|
||||||
throws IOException {
|
throws IOException {
|
||||||
final IndexEventListener eventListener = freeze();
|
final IndexEventListener eventListener = freeze();
|
||||||
IndexSearcherWrapperFactory searcherWrapperFactory = indexSearcherWrapper.get() == null
|
IndexSearcherWrapperFactory searcherWrapperFactory = indexSearcherWrapper.get() == null
|
||||||
|
@ -365,8 +363,8 @@ public final class IndexModule {
|
||||||
}
|
}
|
||||||
return new IndexService(indexSettings, environment, xContentRegistry, new SimilarityService(indexSettings, similarities),
|
return new IndexService(indexSettings, environment, xContentRegistry, new SimilarityService(indexSettings, similarities),
|
||||||
shardStoreDeleter, analysisRegistry, engineFactory.get(), circuitBreakerService, bigArrays, threadPool, scriptService,
|
shardStoreDeleter, analysisRegistry, engineFactory.get(), circuitBreakerService, bigArrays, threadPool, scriptService,
|
||||||
clusterService, client, queryCache, store, eventListener, searcherWrapperFactory, mapperRegistry,
|
client, queryCache, store, eventListener, searcherWrapperFactory, mapperRegistry,
|
||||||
indicesFieldDataCache, globalCheckpointSyncer, searchOperationListeners, indexOperationListeners);
|
indicesFieldDataCache, searchOperationListeners, indexOperationListeners);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -28,7 +28,6 @@ import org.apache.lucene.util.IOUtils;
|
||||||
import org.elasticsearch.client.Client;
|
import org.elasticsearch.client.Client;
|
||||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||||
import org.elasticsearch.cluster.service.ClusterService;
|
|
||||||
import org.elasticsearch.common.Nullable;
|
import org.elasticsearch.common.Nullable;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.unit.TimeValue;
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
|
@ -105,7 +104,6 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
|
||||||
private final SimilarityService similarityService;
|
private final SimilarityService similarityService;
|
||||||
private final EngineFactory engineFactory;
|
private final EngineFactory engineFactory;
|
||||||
private final IndexWarmer warmer;
|
private final IndexWarmer warmer;
|
||||||
private final Consumer<ShardId> globalCheckpointSyncer;
|
|
||||||
private volatile Map<Integer, IndexShard> shards = emptyMap();
|
private volatile Map<Integer, IndexShard> shards = emptyMap();
|
||||||
private final AtomicBoolean closed = new AtomicBoolean(false);
|
private final AtomicBoolean closed = new AtomicBoolean(false);
|
||||||
private final AtomicBoolean deleted = new AtomicBoolean(false);
|
private final AtomicBoolean deleted = new AtomicBoolean(false);
|
||||||
|
@ -116,36 +114,33 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
|
||||||
private volatile AsyncTranslogFSync fsyncTask;
|
private volatile AsyncTranslogFSync fsyncTask;
|
||||||
private final ThreadPool threadPool;
|
private final ThreadPool threadPool;
|
||||||
private final BigArrays bigArrays;
|
private final BigArrays bigArrays;
|
||||||
private final AsyncGlobalCheckpointTask globalCheckpointTask;
|
|
||||||
private final ScriptService scriptService;
|
private final ScriptService scriptService;
|
||||||
private final ClusterService clusterService;
|
|
||||||
private final Client client;
|
private final Client client;
|
||||||
private Supplier<Sort> indexSortSupplier;
|
private Supplier<Sort> indexSortSupplier;
|
||||||
|
|
||||||
public IndexService(IndexSettings indexSettings, NodeEnvironment nodeEnv,
|
public IndexService(
|
||||||
NamedXContentRegistry xContentRegistry,
|
IndexSettings indexSettings,
|
||||||
SimilarityService similarityService,
|
NodeEnvironment nodeEnv,
|
||||||
ShardStoreDeleter shardStoreDeleter,
|
NamedXContentRegistry xContentRegistry,
|
||||||
AnalysisRegistry registry,
|
SimilarityService similarityService,
|
||||||
@Nullable EngineFactory engineFactory,
|
ShardStoreDeleter shardStoreDeleter,
|
||||||
CircuitBreakerService circuitBreakerService,
|
AnalysisRegistry registry,
|
||||||
BigArrays bigArrays,
|
@Nullable EngineFactory engineFactory,
|
||||||
ThreadPool threadPool,
|
CircuitBreakerService circuitBreakerService,
|
||||||
ScriptService scriptService,
|
BigArrays bigArrays,
|
||||||
ClusterService clusterService,
|
ThreadPool threadPool,
|
||||||
Client client,
|
ScriptService scriptService,
|
||||||
QueryCache queryCache,
|
Client client,
|
||||||
IndexStore indexStore,
|
QueryCache queryCache,
|
||||||
IndexEventListener eventListener,
|
IndexStore indexStore,
|
||||||
IndexModule.IndexSearcherWrapperFactory wrapperFactory,
|
IndexEventListener eventListener,
|
||||||
MapperRegistry mapperRegistry,
|
IndexModule.IndexSearcherWrapperFactory wrapperFactory,
|
||||||
IndicesFieldDataCache indicesFieldDataCache,
|
MapperRegistry mapperRegistry,
|
||||||
Consumer<ShardId> globalCheckpointSyncer,
|
IndicesFieldDataCache indicesFieldDataCache,
|
||||||
List<SearchOperationListener> searchOperationListeners,
|
List<SearchOperationListener> searchOperationListeners,
|
||||||
List<IndexingOperationListener> indexingOperationListeners) throws IOException {
|
List<IndexingOperationListener> indexingOperationListeners) throws IOException {
|
||||||
super(indexSettings);
|
super(indexSettings);
|
||||||
this.indexSettings = indexSettings;
|
this.indexSettings = indexSettings;
|
||||||
this.globalCheckpointSyncer = globalCheckpointSyncer;
|
|
||||||
this.xContentRegistry = xContentRegistry;
|
this.xContentRegistry = xContentRegistry;
|
||||||
this.similarityService = similarityService;
|
this.similarityService = similarityService;
|
||||||
this.mapperService = new MapperService(indexSettings, registry.build(indexSettings), xContentRegistry, similarityService,
|
this.mapperService = new MapperService(indexSettings, registry.build(indexSettings), xContentRegistry, similarityService,
|
||||||
|
@ -169,7 +164,6 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
|
||||||
this.bigArrays = bigArrays;
|
this.bigArrays = bigArrays;
|
||||||
this.threadPool = threadPool;
|
this.threadPool = threadPool;
|
||||||
this.scriptService = scriptService;
|
this.scriptService = scriptService;
|
||||||
this.clusterService = clusterService;
|
|
||||||
this.client = client;
|
this.client = client;
|
||||||
this.eventListener = eventListener;
|
this.eventListener = eventListener;
|
||||||
this.nodeEnv = nodeEnv;
|
this.nodeEnv = nodeEnv;
|
||||||
|
@ -182,7 +176,6 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
|
||||||
this.engineFactory = engineFactory;
|
this.engineFactory = engineFactory;
|
||||||
// initialize this last -- otherwise if the wrapper requires any other member to be non-null we fail with an NPE
|
// initialize this last -- otherwise if the wrapper requires any other member to be non-null we fail with an NPE
|
||||||
this.searcherWrapper = wrapperFactory.newWrapper(this);
|
this.searcherWrapper = wrapperFactory.newWrapper(this);
|
||||||
this.globalCheckpointTask = new AsyncGlobalCheckpointTask(this);
|
|
||||||
this.indexingOperationListeners = Collections.unmodifiableList(indexingOperationListeners);
|
this.indexingOperationListeners = Collections.unmodifiableList(indexingOperationListeners);
|
||||||
this.searchOperationListeners = Collections.unmodifiableList(searchOperationListeners);
|
this.searchOperationListeners = Collections.unmodifiableList(searchOperationListeners);
|
||||||
// kick off async ops for the first shard in this index
|
// kick off async ops for the first shard in this index
|
||||||
|
@ -272,7 +265,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
IOUtils.close(bitsetFilterCache, indexCache, indexFieldData, mapperService, refreshTask, fsyncTask, globalCheckpointTask);
|
IOUtils.close(bitsetFilterCache, indexCache, indexFieldData, mapperService, refreshTask, fsyncTask);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -369,7 +362,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
|
||||||
indexShard = new IndexShard(routing, this.indexSettings, path, store, indexSortSupplier,
|
indexShard = new IndexShard(routing, this.indexSettings, path, store, indexSortSupplier,
|
||||||
indexCache, mapperService, similarityService, indexFieldData, engineFactory,
|
indexCache, mapperService, similarityService, indexFieldData, engineFactory,
|
||||||
eventListener, searcherWrapper, threadPool, bigArrays, engineWarmer,
|
eventListener, searcherWrapper, threadPool, bigArrays, engineWarmer,
|
||||||
() -> globalCheckpointSyncer.accept(shardId), searchOperationListeners, indexingOperationListeners);
|
searchOperationListeners, indexingOperationListeners);
|
||||||
eventListener.indexShardStateChanged(indexShard, null, indexShard.state(), "shard created");
|
eventListener.indexShardStateChanged(indexShard, null, indexShard.state(), "shard created");
|
||||||
eventListener.afterIndexShardCreated(indexShard);
|
eventListener.afterIndexShardCreated(indexShard);
|
||||||
shards = newMapBuilder(shards).put(shardId.id(), indexShard).immutableMap();
|
shards = newMapBuilder(shards).put(shardId.id(), indexShard).immutableMap();
|
||||||
|
@ -708,31 +701,6 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void maybeUpdateGlobalCheckpoints() {
|
|
||||||
for (IndexShard shard : this.shards.values()) {
|
|
||||||
if (shard.routingEntry().primary()) {
|
|
||||||
switch (shard.state()) {
|
|
||||||
case CREATED:
|
|
||||||
case RECOVERING:
|
|
||||||
case CLOSED:
|
|
||||||
case RELOCATED:
|
|
||||||
continue;
|
|
||||||
case POST_RECOVERY:
|
|
||||||
case STARTED:
|
|
||||||
try {
|
|
||||||
shard.updateGlobalCheckpointOnPrimary();
|
|
||||||
} catch (AlreadyClosedException ex) {
|
|
||||||
// fine - continue, the shard was concurrently closed on us.
|
|
||||||
}
|
|
||||||
continue;
|
|
||||||
default:
|
|
||||||
throw new IllegalStateException("unknown state: " + shard.state());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
abstract static class BaseAsyncTask implements Runnable, Closeable {
|
abstract static class BaseAsyncTask implements Runnable, Closeable {
|
||||||
protected final IndexService indexService;
|
protected final IndexService indexService;
|
||||||
protected final ThreadPool threadPool;
|
protected final ThreadPool threadPool;
|
||||||
|
@ -877,23 +845,6 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
final class AsyncGlobalCheckpointTask extends BaseAsyncTask {
|
|
||||||
|
|
||||||
AsyncGlobalCheckpointTask(IndexService indexService) {
|
|
||||||
super(indexService, indexService.getIndexSettings().getGlobalCheckpointInterval());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void runInternal() {
|
|
||||||
indexService.maybeUpdateGlobalCheckpoints();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String toString() {
|
|
||||||
return "global_checkpoint";
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
AsyncRefreshTask getRefreshTask() { // for tests
|
AsyncRefreshTask getRefreshTask() { // for tests
|
||||||
return refreshTask;
|
return refreshTask;
|
||||||
}
|
}
|
||||||
|
@ -902,7 +853,4 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
|
||||||
return fsyncTask;
|
return fsyncTask;
|
||||||
}
|
}
|
||||||
|
|
||||||
AsyncGlobalCheckpointTask getGlobalCheckpointTask() { // for tests
|
|
||||||
return globalCheckpointTask;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -130,10 +130,6 @@ public final class IndexSettings {
|
||||||
new ByteSizeValue(Long.MAX_VALUE, ByteSizeUnit.BYTES),
|
new ByteSizeValue(Long.MAX_VALUE, ByteSizeUnit.BYTES),
|
||||||
new Property[]{Property.Dynamic, Property.IndexScope});
|
new Property[]{Property.Dynamic, Property.IndexScope});
|
||||||
|
|
||||||
public static final Setting<TimeValue> INDEX_SEQ_NO_CHECKPOINT_SYNC_INTERVAL =
|
|
||||||
Setting.timeSetting("index.seq_no.checkpoint_sync_interval", new TimeValue(30, TimeUnit.SECONDS),
|
|
||||||
new TimeValue(-1, TimeUnit.MILLISECONDS), Property.Dynamic, Property.IndexScope);
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Index setting to enable / disable deletes garbage collection.
|
* Index setting to enable / disable deletes garbage collection.
|
||||||
* This setting is realtime updateable
|
* This setting is realtime updateable
|
||||||
|
@ -171,7 +167,6 @@ public final class IndexSettings {
|
||||||
private volatile Translog.Durability durability;
|
private volatile Translog.Durability durability;
|
||||||
private final TimeValue syncInterval;
|
private final TimeValue syncInterval;
|
||||||
private volatile TimeValue refreshInterval;
|
private volatile TimeValue refreshInterval;
|
||||||
private final TimeValue globalCheckpointInterval;
|
|
||||||
private volatile ByteSizeValue flushThresholdSize;
|
private volatile ByteSizeValue flushThresholdSize;
|
||||||
private volatile ByteSizeValue generationThresholdSize;
|
private volatile ByteSizeValue generationThresholdSize;
|
||||||
private final MergeSchedulerConfig mergeSchedulerConfig;
|
private final MergeSchedulerConfig mergeSchedulerConfig;
|
||||||
|
@ -269,7 +264,6 @@ public final class IndexSettings {
|
||||||
this.durability = scopedSettings.get(INDEX_TRANSLOG_DURABILITY_SETTING);
|
this.durability = scopedSettings.get(INDEX_TRANSLOG_DURABILITY_SETTING);
|
||||||
syncInterval = INDEX_TRANSLOG_SYNC_INTERVAL_SETTING.get(settings);
|
syncInterval = INDEX_TRANSLOG_SYNC_INTERVAL_SETTING.get(settings);
|
||||||
refreshInterval = scopedSettings.get(INDEX_REFRESH_INTERVAL_SETTING);
|
refreshInterval = scopedSettings.get(INDEX_REFRESH_INTERVAL_SETTING);
|
||||||
globalCheckpointInterval = scopedSettings.get(INDEX_SEQ_NO_CHECKPOINT_SYNC_INTERVAL);
|
|
||||||
flushThresholdSize = scopedSettings.get(INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING);
|
flushThresholdSize = scopedSettings.get(INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING);
|
||||||
generationThresholdSize = scopedSettings.get(INDEX_TRANSLOG_GENERATION_THRESHOLD_SIZE_SETTING);
|
generationThresholdSize = scopedSettings.get(INDEX_TRANSLOG_GENERATION_THRESHOLD_SIZE_SETTING);
|
||||||
mergeSchedulerConfig = new MergeSchedulerConfig(this);
|
mergeSchedulerConfig = new MergeSchedulerConfig(this);
|
||||||
|
@ -470,13 +464,6 @@ public final class IndexSettings {
|
||||||
return refreshInterval;
|
return refreshInterval;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns this interval in which the primary shards of this index should check and advance the global checkpoint
|
|
||||||
*/
|
|
||||||
public TimeValue getGlobalCheckpointInterval() {
|
|
||||||
return globalCheckpointInterval;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the transaction log threshold size when to forcefully flush the index and clear the transaction log.
|
* Returns the transaction log threshold size when to forcefully flush the index and clear the transaction log.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -36,7 +36,6 @@ import org.elasticsearch.indices.analysis.AnalysisModule;
|
||||||
import org.elasticsearch.indices.analysis.AnalysisModule.AnalysisProvider;
|
import org.elasticsearch.indices.analysis.AnalysisModule.AnalysisProvider;
|
||||||
import org.elasticsearch.indices.analysis.PreBuiltAnalyzers;
|
import org.elasticsearch.indices.analysis.PreBuiltAnalyzers;
|
||||||
import org.elasticsearch.indices.analysis.PreBuiltCharFilters;
|
import org.elasticsearch.indices.analysis.PreBuiltCharFilters;
|
||||||
import org.elasticsearch.indices.analysis.PreBuiltTokenFilters;
|
|
||||||
import org.elasticsearch.indices.analysis.PreBuiltTokenizers;
|
import org.elasticsearch.indices.analysis.PreBuiltTokenizers;
|
||||||
|
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
|
@ -59,7 +58,7 @@ public final class AnalysisRegistry implements Closeable {
|
||||||
public static final String INDEX_ANALYSIS_CHAR_FILTER = "index.analysis.char_filter";
|
public static final String INDEX_ANALYSIS_CHAR_FILTER = "index.analysis.char_filter";
|
||||||
public static final String INDEX_ANALYSIS_FILTER = "index.analysis.filter";
|
public static final String INDEX_ANALYSIS_FILTER = "index.analysis.filter";
|
||||||
public static final String INDEX_ANALYSIS_TOKENIZER = "index.analysis.tokenizer";
|
public static final String INDEX_ANALYSIS_TOKENIZER = "index.analysis.tokenizer";
|
||||||
private final PrebuiltAnalysis prebuiltAnalysis = new PrebuiltAnalysis();
|
private final PrebuiltAnalysis prebuiltAnalysis;
|
||||||
private final Map<String, Analyzer> cachedAnalyzer = new ConcurrentHashMap<>();
|
private final Map<String, Analyzer> cachedAnalyzer = new ConcurrentHashMap<>();
|
||||||
|
|
||||||
private final Environment environment;
|
private final Environment environment;
|
||||||
|
@ -74,13 +73,15 @@ public final class AnalysisRegistry implements Closeable {
|
||||||
Map<String, AnalysisProvider<TokenFilterFactory>> tokenFilters,
|
Map<String, AnalysisProvider<TokenFilterFactory>> tokenFilters,
|
||||||
Map<String, AnalysisProvider<TokenizerFactory>> tokenizers,
|
Map<String, AnalysisProvider<TokenizerFactory>> tokenizers,
|
||||||
Map<String, AnalysisProvider<AnalyzerProvider<?>>> analyzers,
|
Map<String, AnalysisProvider<AnalyzerProvider<?>>> analyzers,
|
||||||
Map<String, AnalysisProvider<AnalyzerProvider<?>>> normalizers) {
|
Map<String, AnalysisProvider<AnalyzerProvider<?>>> normalizers,
|
||||||
|
Map<String, PreConfiguredTokenFilter> preConfiguredTokenFilters) {
|
||||||
this.environment = environment;
|
this.environment = environment;
|
||||||
this.charFilters = unmodifiableMap(charFilters);
|
this.charFilters = unmodifiableMap(charFilters);
|
||||||
this.tokenFilters = unmodifiableMap(tokenFilters);
|
this.tokenFilters = unmodifiableMap(tokenFilters);
|
||||||
this.tokenizers = unmodifiableMap(tokenizers);
|
this.tokenizers = unmodifiableMap(tokenizers);
|
||||||
this.analyzers = unmodifiableMap(analyzers);
|
this.analyzers = unmodifiableMap(analyzers);
|
||||||
this.normalizers = unmodifiableMap(normalizers);
|
this.normalizers = unmodifiableMap(normalizers);
|
||||||
|
prebuiltAnalysis = new PrebuiltAnalysis(preConfiguredTokenFilters);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -305,8 +306,8 @@ public final class AnalysisRegistry implements Closeable {
|
||||||
}
|
}
|
||||||
|
|
||||||
private <T> Map<String, T> buildMapping(Component component, IndexSettings settings, Map<String, Settings> settingsMap,
|
private <T> Map<String, T> buildMapping(Component component, IndexSettings settings, Map<String, Settings> settingsMap,
|
||||||
Map<String, AnalysisModule.AnalysisProvider<T>> providerMap, Map<String, AnalysisModule.AnalysisProvider<T>> defaultInstance)
|
Map<String, ? extends AnalysisModule.AnalysisProvider<T>> providerMap,
|
||||||
throws IOException {
|
Map<String, ? extends AnalysisModule.AnalysisProvider<T>> defaultInstance) throws IOException {
|
||||||
Settings defaultSettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, settings.getIndexVersionCreated()).build();
|
Settings defaultSettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, settings.getIndexVersionCreated()).build();
|
||||||
Map<String, T> factories = new HashMap<>();
|
Map<String, T> factories = new HashMap<>();
|
||||||
for (Map.Entry<String, Settings> entry : settingsMap.entrySet()) {
|
for (Map.Entry<String, Settings> entry : settingsMap.entrySet()) {
|
||||||
|
@ -344,7 +345,7 @@ public final class AnalysisRegistry implements Closeable {
|
||||||
|
|
||||||
}
|
}
|
||||||
// go over the char filters in the bindings and register the ones that are not configured
|
// go over the char filters in the bindings and register the ones that are not configured
|
||||||
for (Map.Entry<String, AnalysisModule.AnalysisProvider<T>> entry : providerMap.entrySet()) {
|
for (Map.Entry<String, ? extends AnalysisModule.AnalysisProvider<T>> entry : providerMap.entrySet()) {
|
||||||
String name = entry.getKey();
|
String name = entry.getKey();
|
||||||
AnalysisModule.AnalysisProvider<T> provider = entry.getValue();
|
AnalysisModule.AnalysisProvider<T> provider = entry.getValue();
|
||||||
// we don't want to re-register one that already exists
|
// we don't want to re-register one that already exists
|
||||||
|
@ -365,7 +366,7 @@ public final class AnalysisRegistry implements Closeable {
|
||||||
factories.put(name, instance);
|
factories.put(name, instance);
|
||||||
}
|
}
|
||||||
|
|
||||||
for (Map.Entry<String, AnalysisModule.AnalysisProvider<T>> entry : defaultInstance.entrySet()) {
|
for (Map.Entry<String, ? extends AnalysisModule.AnalysisProvider<T>> entry : defaultInstance.entrySet()) {
|
||||||
final String name = entry.getKey();
|
final String name = entry.getKey();
|
||||||
final AnalysisModule.AnalysisProvider<T> provider = entry.getValue();
|
final AnalysisModule.AnalysisProvider<T> provider = entry.getValue();
|
||||||
if (factories.containsKey(name) == false) {
|
if (factories.containsKey(name) == false) {
|
||||||
|
@ -378,7 +379,8 @@ public final class AnalysisRegistry implements Closeable {
|
||||||
return factories;
|
return factories;
|
||||||
}
|
}
|
||||||
|
|
||||||
private <T> AnalysisProvider<T> getAnalysisProvider(Component component, Map<String, AnalysisProvider<T>> providerMap, String name, String typeName) {
|
private <T> AnalysisProvider<T> getAnalysisProvider(Component component, Map<String, ? extends AnalysisProvider<T>> providerMap,
|
||||||
|
String name, String typeName) {
|
||||||
if (typeName == null) {
|
if (typeName == null) {
|
||||||
throw new IllegalArgumentException(component + " [" + name + "] must specify either an analyzer type, or a tokenizer");
|
throw new IllegalArgumentException(component + " [" + name + "] must specify either an analyzer type, or a tokenizer");
|
||||||
}
|
}
|
||||||
|
@ -393,13 +395,12 @@ public final class AnalysisRegistry implements Closeable {
|
||||||
|
|
||||||
final Map<String, AnalysisModule.AnalysisProvider<AnalyzerProvider<?>>> analyzerProviderFactories;
|
final Map<String, AnalysisModule.AnalysisProvider<AnalyzerProvider<?>>> analyzerProviderFactories;
|
||||||
final Map<String, AnalysisModule.AnalysisProvider<TokenizerFactory>> tokenizerFactories;
|
final Map<String, AnalysisModule.AnalysisProvider<TokenizerFactory>> tokenizerFactories;
|
||||||
final Map<String, AnalysisModule.AnalysisProvider<TokenFilterFactory>> tokenFilterFactories;
|
final Map<String, ? extends AnalysisProvider<TokenFilterFactory>> tokenFilterFactories;
|
||||||
final Map<String, AnalysisModule.AnalysisProvider<CharFilterFactory>> charFilterFactories;
|
final Map<String, AnalysisModule.AnalysisProvider<CharFilterFactory>> charFilterFactories;
|
||||||
|
|
||||||
private PrebuiltAnalysis() {
|
private PrebuiltAnalysis(Map<String, PreConfiguredTokenFilter> preConfiguredTokenFilters) {
|
||||||
Map<String, PreBuiltAnalyzerProviderFactory> analyzerProviderFactories = new HashMap<>();
|
Map<String, PreBuiltAnalyzerProviderFactory> analyzerProviderFactories = new HashMap<>();
|
||||||
Map<String, PreBuiltTokenizerFactoryFactory> tokenizerFactories = new HashMap<>();
|
Map<String, PreBuiltTokenizerFactoryFactory> tokenizerFactories = new HashMap<>();
|
||||||
Map<String, PreBuiltTokenFilterFactoryFactory> tokenFilterFactories = new HashMap<>();
|
|
||||||
Map<String, PreBuiltCharFilterFactoryFactory> charFilterFactories = new HashMap<>();
|
Map<String, PreBuiltCharFilterFactoryFactory> charFilterFactories = new HashMap<>();
|
||||||
// Analyzers
|
// Analyzers
|
||||||
for (PreBuiltAnalyzers preBuiltAnalyzerEnum : PreBuiltAnalyzers.values()) {
|
for (PreBuiltAnalyzers preBuiltAnalyzerEnum : PreBuiltAnalyzers.values()) {
|
||||||
|
@ -418,17 +419,6 @@ public final class AnalysisRegistry implements Closeable {
|
||||||
tokenizerFactories.put("edgeNGram", new PreBuiltTokenizerFactoryFactory(PreBuiltTokenizers.EDGE_NGRAM.getTokenizerFactory(Version.CURRENT)));
|
tokenizerFactories.put("edgeNGram", new PreBuiltTokenizerFactoryFactory(PreBuiltTokenizers.EDGE_NGRAM.getTokenizerFactory(Version.CURRENT)));
|
||||||
tokenizerFactories.put("PathHierarchy", new PreBuiltTokenizerFactoryFactory(PreBuiltTokenizers.PATH_HIERARCHY.getTokenizerFactory(Version.CURRENT)));
|
tokenizerFactories.put("PathHierarchy", new PreBuiltTokenizerFactoryFactory(PreBuiltTokenizers.PATH_HIERARCHY.getTokenizerFactory(Version.CURRENT)));
|
||||||
|
|
||||||
|
|
||||||
// Token filters
|
|
||||||
for (PreBuiltTokenFilters preBuiltTokenFilter : PreBuiltTokenFilters.values()) {
|
|
||||||
String name = preBuiltTokenFilter.name().toLowerCase(Locale.ROOT);
|
|
||||||
tokenFilterFactories.put(name, new PreBuiltTokenFilterFactoryFactory(preBuiltTokenFilter.getTokenFilterFactory(Version.CURRENT)));
|
|
||||||
}
|
|
||||||
// Token filter aliases
|
|
||||||
tokenFilterFactories.put("nGram", new PreBuiltTokenFilterFactoryFactory(PreBuiltTokenFilters.NGRAM.getTokenFilterFactory(Version.CURRENT)));
|
|
||||||
tokenFilterFactories.put("edgeNGram", new PreBuiltTokenFilterFactoryFactory(PreBuiltTokenFilters.EDGE_NGRAM.getTokenFilterFactory(Version.CURRENT)));
|
|
||||||
|
|
||||||
|
|
||||||
// Char Filters
|
// Char Filters
|
||||||
for (PreBuiltCharFilters preBuiltCharFilter : PreBuiltCharFilters.values()) {
|
for (PreBuiltCharFilters preBuiltCharFilter : PreBuiltCharFilters.values()) {
|
||||||
String name = preBuiltCharFilter.name().toLowerCase(Locale.ROOT);
|
String name = preBuiltCharFilter.name().toLowerCase(Locale.ROOT);
|
||||||
|
@ -436,10 +426,11 @@ public final class AnalysisRegistry implements Closeable {
|
||||||
}
|
}
|
||||||
// Char filter aliases
|
// Char filter aliases
|
||||||
charFilterFactories.put("htmlStrip", new PreBuiltCharFilterFactoryFactory(PreBuiltCharFilters.HTML_STRIP.getCharFilterFactory(Version.CURRENT)));
|
charFilterFactories.put("htmlStrip", new PreBuiltCharFilterFactoryFactory(PreBuiltCharFilters.HTML_STRIP.getCharFilterFactory(Version.CURRENT)));
|
||||||
|
|
||||||
this.analyzerProviderFactories = Collections.unmodifiableMap(analyzerProviderFactories);
|
this.analyzerProviderFactories = Collections.unmodifiableMap(analyzerProviderFactories);
|
||||||
this.charFilterFactories = Collections.unmodifiableMap(charFilterFactories);
|
this.charFilterFactories = Collections.unmodifiableMap(charFilterFactories);
|
||||||
this.tokenFilterFactories = Collections.unmodifiableMap(tokenFilterFactories);
|
|
||||||
this.tokenizerFactories = Collections.unmodifiableMap(tokenizerFactories);
|
this.tokenizerFactories = Collections.unmodifiableMap(tokenizerFactories);
|
||||||
|
tokenFilterFactories = preConfiguredTokenFilters;
|
||||||
}
|
}
|
||||||
|
|
||||||
public AnalysisModule.AnalysisProvider<CharFilterFactory> getCharFilterFactory(String name) {
|
public AnalysisModule.AnalysisProvider<CharFilterFactory> getCharFilterFactory(String name) {
|
||||||
|
|
|
@ -1,50 +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.analysis;
|
|
||||||
|
|
||||||
import org.elasticsearch.Version;
|
|
||||||
import org.elasticsearch.common.settings.Settings;
|
|
||||||
import org.elasticsearch.env.Environment;
|
|
||||||
import org.elasticsearch.index.IndexSettings;
|
|
||||||
import org.elasticsearch.indices.analysis.AnalysisModule;
|
|
||||||
import org.elasticsearch.indices.analysis.PreBuiltTokenFilters;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
public class PreBuiltTokenFilterFactoryFactory implements AnalysisModule.AnalysisProvider<TokenFilterFactory> {
|
|
||||||
|
|
||||||
private final TokenFilterFactory tokenFilterFactory;
|
|
||||||
|
|
||||||
public PreBuiltTokenFilterFactoryFactory(TokenFilterFactory tokenFilterFactory) {
|
|
||||||
this.tokenFilterFactory = tokenFilterFactory;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public TokenFilterFactory get(IndexSettings indexSettings, Environment environment, String name, Settings settings) throws IOException {
|
|
||||||
Version indexVersion = Version.indexCreated(settings);
|
|
||||||
if (!Version.CURRENT.equals(indexVersion)) {
|
|
||||||
PreBuiltTokenFilters preBuiltTokenFilters = PreBuiltTokenFilters.getOrDefault(name, null);
|
|
||||||
if (preBuiltTokenFilters != null) {
|
|
||||||
return preBuiltTokenFilters.getTokenFilterFactory(indexVersion);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return tokenFilterFactory;
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -0,0 +1,123 @@
|
||||||
|
/*
|
||||||
|
* 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.analysis;
|
||||||
|
|
||||||
|
import org.apache.lucene.analysis.TokenFilter;
|
||||||
|
import org.apache.lucene.analysis.TokenStream;
|
||||||
|
import org.elasticsearch.Version;
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.env.Environment;
|
||||||
|
import org.elasticsearch.index.IndexSettings;
|
||||||
|
import org.elasticsearch.indices.analysis.AnalysisModule;
|
||||||
|
import org.elasticsearch.indices.analysis.PreBuiltCacheFactory;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.function.BiFunction;
|
||||||
|
import java.util.function.Function;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Provides pre-configured, shared {@link TokenFilter}s.
|
||||||
|
*/
|
||||||
|
public final class PreConfiguredTokenFilter implements AnalysisModule.AnalysisProvider<TokenFilterFactory> {
|
||||||
|
private final String name;
|
||||||
|
private final boolean useFilterForMultitermQueries;
|
||||||
|
private final PreBuiltCacheFactory.PreBuiltCache<TokenFilterFactory> cache;
|
||||||
|
private final BiFunction<TokenStream, Version, TokenStream> create;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Standard ctor with all the power.
|
||||||
|
*/
|
||||||
|
public PreConfiguredTokenFilter(String name, boolean useFilterForMultitermQueries,
|
||||||
|
PreBuiltCacheFactory.CachingStrategy cachingStrategy, BiFunction<TokenStream, Version, TokenStream> create) {
|
||||||
|
this.name = name;
|
||||||
|
this.useFilterForMultitermQueries = useFilterForMultitermQueries;
|
||||||
|
cache = PreBuiltCacheFactory.getCache(cachingStrategy);
|
||||||
|
this.create = create;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Convenience ctor for token streams that don't vary based on version.
|
||||||
|
*/
|
||||||
|
public PreConfiguredTokenFilter(String name, boolean useFilterForMultitermQueries,
|
||||||
|
PreBuiltCacheFactory.CachingStrategy cachingStrategy, Function<TokenStream, TokenStream> create) {
|
||||||
|
this(name, useFilterForMultitermQueries, cachingStrategy, (input, version) -> create.apply(input));
|
||||||
|
// TODO why oh why aren't these all CachingStrategy.ONE? They *can't* vary based on version because they don't get it, right?!
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public TokenFilterFactory get(IndexSettings indexSettings, Environment environment, String name, Settings settings) throws IOException {
|
||||||
|
return getTokenFilterFactory(Version.indexCreated(settings));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The name of the {@link TokenFilter} in the API.
|
||||||
|
*/
|
||||||
|
public String getName() {
|
||||||
|
return name;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Can this {@link TokenFilter} be used in multi-term queries?
|
||||||
|
*/
|
||||||
|
public boolean shouldUseFilterForMultitermQueries() {
|
||||||
|
return useFilterForMultitermQueries;
|
||||||
|
}
|
||||||
|
|
||||||
|
private interface MultiTermAwareTokenFilterFactory extends TokenFilterFactory, MultiTermAwareComponent {}
|
||||||
|
|
||||||
|
private synchronized TokenFilterFactory getTokenFilterFactory(final Version version) {
|
||||||
|
TokenFilterFactory factory = cache.get(version);
|
||||||
|
if (factory == null) {
|
||||||
|
if (useFilterForMultitermQueries) {
|
||||||
|
factory = new MultiTermAwareTokenFilterFactory() {
|
||||||
|
@Override
|
||||||
|
public String name() {
|
||||||
|
return name;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public TokenStream create(TokenStream tokenStream) {
|
||||||
|
return create.apply(tokenStream, version);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object getMultiTermComponent() {
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
} else {
|
||||||
|
factory = new TokenFilterFactory() {
|
||||||
|
@Override
|
||||||
|
public String name() {
|
||||||
|
return name;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public TokenStream create(TokenStream tokenStream) {
|
||||||
|
return create.apply(tokenStream, version);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
cache.put(version, factory);
|
||||||
|
}
|
||||||
|
|
||||||
|
return factory;
|
||||||
|
}
|
||||||
|
}
|
|
@ -16,10 +16,9 @@
|
||||||
* specific language governing permissions and limitations
|
* specific language governing permissions and limitations
|
||||||
* under the License.
|
* under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package org.elasticsearch.index.seqno;
|
package org.elasticsearch.index.seqno;
|
||||||
|
|
||||||
import org.apache.logging.log4j.message.ParameterizedMessage;
|
|
||||||
import org.apache.logging.log4j.util.Supplier;
|
|
||||||
import org.elasticsearch.Version;
|
import org.elasticsearch.Version;
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
import org.elasticsearch.action.support.ActionFilters;
|
import org.elasticsearch.action.support.ActionFilters;
|
||||||
|
@ -32,31 +31,49 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||||
import org.elasticsearch.cluster.service.ClusterService;
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
import org.elasticsearch.common.inject.Inject;
|
import org.elasticsearch.common.inject.Inject;
|
||||||
import org.elasticsearch.common.io.stream.StreamInput;
|
|
||||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.index.shard.IndexEventListener;
|
||||||
import org.elasticsearch.index.shard.IndexShard;
|
import org.elasticsearch.index.shard.IndexShard;
|
||||||
import org.elasticsearch.index.shard.ShardId;
|
import org.elasticsearch.index.shard.ShardId;
|
||||||
import org.elasticsearch.indices.IndicesService;
|
import org.elasticsearch.indices.IndicesService;
|
||||||
import org.elasticsearch.threadpool.ThreadPool;
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
import org.elasticsearch.transport.TransportService;
|
import org.elasticsearch.transport.TransportService;
|
||||||
|
|
||||||
import java.io.IOException;
|
/**
|
||||||
|
* Background global checkpoint sync action initiated when a shard goes inactive. This is needed because while we send the global checkpoint
|
||||||
public class GlobalCheckpointSyncAction extends TransportReplicationAction<GlobalCheckpointSyncAction.PrimaryRequest,
|
* on every replication operation, after the last operation completes the global checkpoint could advance but without a follow-up operation
|
||||||
GlobalCheckpointSyncAction.ReplicaRequest, ReplicationResponse> {
|
* the global checkpoint will never be synced to the replicas.
|
||||||
|
*/
|
||||||
|
public class GlobalCheckpointSyncAction extends TransportReplicationAction<
|
||||||
|
GlobalCheckpointSyncAction.Request,
|
||||||
|
GlobalCheckpointSyncAction.Request,
|
||||||
|
ReplicationResponse> implements IndexEventListener {
|
||||||
|
|
||||||
public static String ACTION_NAME = "indices:admin/seq_no/global_checkpoint_sync";
|
public static String ACTION_NAME = "indices:admin/seq_no/global_checkpoint_sync";
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public GlobalCheckpointSyncAction(Settings settings, TransportService transportService,
|
public GlobalCheckpointSyncAction(
|
||||||
ClusterService clusterService, IndicesService indicesService,
|
final Settings settings,
|
||||||
ThreadPool threadPool, ShardStateAction shardStateAction,
|
final TransportService transportService,
|
||||||
ActionFilters actionFilters,
|
final ClusterService clusterService,
|
||||||
IndexNameExpressionResolver indexNameExpressionResolver) {
|
final IndicesService indicesService,
|
||||||
super(settings, ACTION_NAME, transportService, clusterService, indicesService, threadPool, shardStateAction,
|
final ThreadPool threadPool,
|
||||||
actionFilters, indexNameExpressionResolver, PrimaryRequest::new, ReplicaRequest::new,
|
final ShardStateAction shardStateAction,
|
||||||
ThreadPool.Names.SAME);
|
final ActionFilters actionFilters,
|
||||||
|
final IndexNameExpressionResolver indexNameExpressionResolver) {
|
||||||
|
super(
|
||||||
|
settings,
|
||||||
|
ACTION_NAME,
|
||||||
|
transportService,
|
||||||
|
clusterService,
|
||||||
|
indicesService,
|
||||||
|
threadPool,
|
||||||
|
shardStateAction,
|
||||||
|
actionFilters,
|
||||||
|
indexNameExpressionResolver,
|
||||||
|
Request::new,
|
||||||
|
Request::new,
|
||||||
|
ThreadPool.Names.SAME);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -65,97 +82,53 @@ public class GlobalCheckpointSyncAction extends TransportReplicationAction<Globa
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void sendReplicaRequest(ConcreteShardRequest<ReplicaRequest> concreteShardRequest, DiscoveryNode node,
|
protected void sendReplicaRequest(
|
||||||
ActionListener<ReplicationOperation.ReplicaResponse> listener) {
|
final ConcreteReplicaRequest<Request> replicaRequest,
|
||||||
|
final DiscoveryNode node,
|
||||||
|
final ActionListener<ReplicationOperation.ReplicaResponse> listener) {
|
||||||
if (node.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) {
|
if (node.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) {
|
||||||
super.sendReplicaRequest(concreteShardRequest, node, listener);
|
super.sendReplicaRequest(replicaRequest, node, listener);
|
||||||
} else {
|
} else {
|
||||||
listener.onResponse(
|
listener.onResponse(new ReplicaResponse(replicaRequest.getTargetAllocationID(), SequenceNumbersService.UNASSIGNED_SEQ_NO));
|
||||||
new ReplicaResponse(concreteShardRequest.getTargetAllocationID(), SequenceNumbersService.UNASSIGNED_SEQ_NO));
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected PrimaryResult shardOperationOnPrimary(PrimaryRequest request, IndexShard indexShard) throws Exception {
|
public void onShardInactive(final IndexShard indexShard) {
|
||||||
long checkpoint = indexShard.getGlobalCheckpoint();
|
execute(new Request(indexShard.shardId()));
|
||||||
indexShard.getTranslog().sync();
|
|
||||||
return new PrimaryResult(new ReplicaRequest(request, checkpoint), new ReplicationResponse());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected ReplicaResult shardOperationOnReplica(ReplicaRequest request, IndexShard indexShard) throws Exception {
|
protected PrimaryResult<Request, ReplicationResponse> shardOperationOnPrimary(
|
||||||
indexShard.updateGlobalCheckpointOnReplica(request.checkpoint);
|
final Request request, final IndexShard indexShard) throws Exception {
|
||||||
|
indexShard.getTranslog().sync();
|
||||||
|
return new PrimaryResult<>(request, new ReplicationResponse());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected ReplicaResult shardOperationOnReplica(final Request request, final IndexShard indexShard) throws Exception {
|
||||||
indexShard.getTranslog().sync();
|
indexShard.getTranslog().sync();
|
||||||
return new ReplicaResult();
|
return new ReplicaResult();
|
||||||
}
|
}
|
||||||
|
|
||||||
public void updateCheckpointForShard(ShardId shardId) {
|
public static final class Request extends ReplicationRequest<Request> {
|
||||||
execute(new PrimaryRequest(shardId), new ActionListener<ReplicationResponse>() {
|
|
||||||
@Override
|
|
||||||
public void onResponse(ReplicationResponse replicationResponse) {
|
|
||||||
if (logger.isTraceEnabled()) {
|
|
||||||
logger.trace("{} global checkpoint successfully updated (shard info [{}])", shardId,
|
|
||||||
replicationResponse.getShardInfo());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
private Request() {
|
||||||
public void onFailure(Exception e) {
|
|
||||||
logger.debug((Supplier<?>) () -> new ParameterizedMessage("{} failed to update global checkpoint", shardId), e);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
}
|
|
||||||
|
|
||||||
public static final class PrimaryRequest extends ReplicationRequest<PrimaryRequest> {
|
|
||||||
|
|
||||||
private PrimaryRequest() {
|
|
||||||
super();
|
super();
|
||||||
}
|
}
|
||||||
|
|
||||||
public PrimaryRequest(ShardId shardId) {
|
public Request(final ShardId shardId) {
|
||||||
super(shardId);
|
super(shardId);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return "GlobalCkpSyncPrimary{" + shardId + "}";
|
return "GlobalCheckpointSyncAction.Request{" +
|
||||||
}
|
"shardId=" + shardId +
|
||||||
}
|
", timeout=" + timeout +
|
||||||
|
", index='" + index + '\'' +
|
||||||
public static final class ReplicaRequest extends ReplicationRequest<GlobalCheckpointSyncAction.ReplicaRequest> {
|
", waitForActiveShards=" + waitForActiveShards +
|
||||||
|
"}";
|
||||||
private long checkpoint;
|
|
||||||
|
|
||||||
private ReplicaRequest() {
|
|
||||||
}
|
|
||||||
|
|
||||||
public ReplicaRequest(PrimaryRequest primaryRequest, long checkpoint) {
|
|
||||||
super(primaryRequest.shardId());
|
|
||||||
this.checkpoint = checkpoint;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void readFrom(StreamInput in) throws IOException {
|
|
||||||
super.readFrom(in);
|
|
||||||
checkpoint = in.readZLong();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void writeTo(StreamOutput out) throws IOException {
|
|
||||||
super.writeTo(out);
|
|
||||||
out.writeZLong(checkpoint);
|
|
||||||
}
|
|
||||||
|
|
||||||
public long getCheckpoint() {
|
|
||||||
return checkpoint;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String toString() {
|
|
||||||
return "GlobalCkpSyncReplica{" +
|
|
||||||
"checkpoint=" + checkpoint +
|
|
||||||
", shardId=" + shardId +
|
|
||||||
'}';
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -87,16 +87,17 @@ public class GlobalCheckpointTracker extends AbstractIndexShardComponent {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Notifies the service to update the local checkpoint for the shard with the provided allocation ID. If the checkpoint is lower than
|
* Notifies the service to update the local checkpoint for the shard with the provided allocation ID. If the checkpoint is lower than
|
||||||
* the currently known one, this is a no-op. If the allocation ID is not in sync, it is ignored. This is to prevent late arrivals from
|
* the currently known one, this is a no-op. If the allocation ID is not tracked, it is ignored. This is to prevent late arrivals from
|
||||||
* shards that are removed to be re-added.
|
* shards that are removed to be re-added.
|
||||||
*
|
*
|
||||||
* @param allocationId the allocation ID of the shard to update the local checkpoint for
|
* @param allocationId the allocation ID of the shard to update the local checkpoint for
|
||||||
* @param localCheckpoint the local checkpoint for the shard
|
* @param localCheckpoint the local checkpoint for the shard
|
||||||
*/
|
*/
|
||||||
public synchronized void updateLocalCheckpoint(final String allocationId, final long localCheckpoint) {
|
public synchronized void updateLocalCheckpoint(final String allocationId, final long localCheckpoint) {
|
||||||
final boolean updated;
|
final boolean updated;
|
||||||
if (updateLocalCheckpoint(allocationId, localCheckpoint, inSyncLocalCheckpoints, "in-sync")) {
|
if (updateLocalCheckpoint(allocationId, localCheckpoint, inSyncLocalCheckpoints, "in-sync")) {
|
||||||
updated = true;
|
updated = true;
|
||||||
|
updateGlobalCheckpointOnPrimary();
|
||||||
} else if (updateLocalCheckpoint(allocationId, localCheckpoint, trackingLocalCheckpoints, "tracking")) {
|
} else if (updateLocalCheckpoint(allocationId, localCheckpoint, trackingLocalCheckpoints, "tracking")) {
|
||||||
updated = true;
|
updated = true;
|
||||||
} else {
|
} else {
|
||||||
|
@ -108,11 +109,25 @@ public class GlobalCheckpointTracker extends AbstractIndexShardComponent {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Notify all threads waiting on the monitor on this tracker. These threads should be waiting for the local checkpoint on a specific
|
||||||
|
* allocation ID to catch up to the global checkpoint.
|
||||||
|
*/
|
||||||
@SuppressForbidden(reason = "Object#notifyAll waiters for local checkpoint advancement")
|
@SuppressForbidden(reason = "Object#notifyAll waiters for local checkpoint advancement")
|
||||||
private synchronized void notifyAllWaiters() {
|
private synchronized void notifyAllWaiters() {
|
||||||
this.notifyAll();
|
this.notifyAll();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Update the local checkpoint for the specified allocation ID in the specified tracking map. If the checkpoint is lower than the
|
||||||
|
* currently known one, this is a no-op. If the allocation ID is not tracked, it is ignored.
|
||||||
|
*
|
||||||
|
* @param allocationId the allocation ID of the shard to update the local checkpoint for
|
||||||
|
* @param localCheckpoint the local checkpoint for the shard
|
||||||
|
* @param map the tracking map
|
||||||
|
* @param reason the reason for the update (used for logging)
|
||||||
|
* @return {@code true} if the local checkpoint was updated, otherwise {@code false} if this was a no-op
|
||||||
|
*/
|
||||||
private boolean updateLocalCheckpoint(
|
private boolean updateLocalCheckpoint(
|
||||||
final String allocationId, final long localCheckpoint, ObjectLongMap<String> map, final String reason) {
|
final String allocationId, final long localCheckpoint, ObjectLongMap<String> map, final String reason) {
|
||||||
final int index = map.indexOf(allocationId);
|
final int index = map.indexOf(allocationId);
|
||||||
|
@ -137,19 +152,16 @@ public class GlobalCheckpointTracker extends AbstractIndexShardComponent {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Scans through the currently known local checkpoint and updates the global checkpoint accordingly.
|
* Scans through the currently known local checkpoint and updates the global checkpoint accordingly.
|
||||||
*
|
|
||||||
* @return {@code true} if the checkpoint has been updated or if it can not be updated since the local checkpoints of one of the active
|
|
||||||
* allocations is not known.
|
|
||||||
*/
|
*/
|
||||||
synchronized boolean updateCheckpointOnPrimary() {
|
private synchronized void updateGlobalCheckpointOnPrimary() {
|
||||||
long minLocalCheckpoint = Long.MAX_VALUE;
|
long minLocalCheckpoint = Long.MAX_VALUE;
|
||||||
if (inSyncLocalCheckpoints.isEmpty() || !pendingInSync.isEmpty()) {
|
if (inSyncLocalCheckpoints.isEmpty() || !pendingInSync.isEmpty()) {
|
||||||
return false;
|
return;
|
||||||
}
|
}
|
||||||
for (final ObjectLongCursor<String> localCheckpoint : inSyncLocalCheckpoints) {
|
for (final ObjectLongCursor<String> localCheckpoint : inSyncLocalCheckpoints) {
|
||||||
if (localCheckpoint.value == SequenceNumbersService.UNASSIGNED_SEQ_NO) {
|
if (localCheckpoint.value == SequenceNumbersService.UNASSIGNED_SEQ_NO) {
|
||||||
logger.trace("unknown local checkpoint for active allocation ID [{}], requesting a sync", localCheckpoint.key);
|
logger.trace("unknown local checkpoint for active allocation ID [{}], requesting a sync", localCheckpoint.key);
|
||||||
return true;
|
return;
|
||||||
}
|
}
|
||||||
minLocalCheckpoint = Math.min(localCheckpoint.value, minLocalCheckpoint);
|
minLocalCheckpoint = Math.min(localCheckpoint.value, minLocalCheckpoint);
|
||||||
}
|
}
|
||||||
|
@ -166,9 +178,6 @@ public class GlobalCheckpointTracker extends AbstractIndexShardComponent {
|
||||||
if (globalCheckpoint != minLocalCheckpoint) {
|
if (globalCheckpoint != minLocalCheckpoint) {
|
||||||
logger.trace("global checkpoint updated to [{}]", minLocalCheckpoint);
|
logger.trace("global checkpoint updated to [{}]", minLocalCheckpoint);
|
||||||
globalCheckpoint = minLocalCheckpoint;
|
globalCheckpoint = minLocalCheckpoint;
|
||||||
return true;
|
|
||||||
} else {
|
|
||||||
return false;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -177,7 +186,7 @@ public class GlobalCheckpointTracker extends AbstractIndexShardComponent {
|
||||||
*
|
*
|
||||||
* @return the global checkpoint
|
* @return the global checkpoint
|
||||||
*/
|
*/
|
||||||
public synchronized long getCheckpoint() {
|
public synchronized long getGlobalCheckpoint() {
|
||||||
return globalCheckpoint;
|
return globalCheckpoint;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -235,6 +244,8 @@ public class GlobalCheckpointTracker extends AbstractIndexShardComponent {
|
||||||
trackingLocalCheckpoints.put(a, SequenceNumbersService.UNASSIGNED_SEQ_NO);
|
trackingLocalCheckpoints.put(a, SequenceNumbersService.UNASSIGNED_SEQ_NO);
|
||||||
logger.trace("tracking [{}] via cluster state update from master", a);
|
logger.trace("tracking [{}] via cluster state update from master", a);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
updateGlobalCheckpointOnPrimary();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -257,47 +268,68 @@ public class GlobalCheckpointTracker extends AbstractIndexShardComponent {
|
||||||
}
|
}
|
||||||
|
|
||||||
updateLocalCheckpoint(allocationId, localCheckpoint, trackingLocalCheckpoints, "tracking");
|
updateLocalCheckpoint(allocationId, localCheckpoint, trackingLocalCheckpoints, "tracking");
|
||||||
waitForAllocationIdToBeInSync(allocationId);
|
|
||||||
}
|
|
||||||
|
|
||||||
private synchronized void waitForAllocationIdToBeInSync(final String allocationId) throws InterruptedException {
|
|
||||||
if (!pendingInSync.add(allocationId)) {
|
if (!pendingInSync.add(allocationId)) {
|
||||||
throw new IllegalStateException("there is already a pending sync in progress for allocation ID [" + allocationId + "]");
|
throw new IllegalStateException("there is already a pending sync in progress for allocation ID [" + allocationId + "]");
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
while (true) {
|
waitForAllocationIdToBeInSync(allocationId);
|
||||||
/*
|
|
||||||
* If the allocation has been cancelled and so removed from the tracking map from a cluster state update from the master it
|
|
||||||
* means that this recovery will be cancelled; we are here on a cancellable recovery thread and so this thread will throw
|
|
||||||
* an interrupted exception as soon as it tries to wait on the monitor.
|
|
||||||
*/
|
|
||||||
final long current = trackingLocalCheckpoints.getOrDefault(allocationId, Long.MIN_VALUE);
|
|
||||||
if (current >= globalCheckpoint) {
|
|
||||||
logger.trace("marked [{}] as in-sync with local checkpoint [{}]", allocationId, current);
|
|
||||||
trackingLocalCheckpoints.remove(allocationId);
|
|
||||||
/*
|
|
||||||
* This is prematurely adding the allocation ID to the in-sync map as at this point recovery is not yet finished and
|
|
||||||
* could still abort. At this point we will end up with a shard in the in-sync map holding back the global checkpoint
|
|
||||||
* because the shard never recovered and we would have to wait until either the recovery retries and completes
|
|
||||||
* successfully, or the master fails the shard and issues a cluster state update that removes the shard from the set of
|
|
||||||
* active allocation IDs.
|
|
||||||
*/
|
|
||||||
inSyncLocalCheckpoints.put(allocationId, current);
|
|
||||||
break;
|
|
||||||
} else {
|
|
||||||
waitForLocalCheckpointToAdvance();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
} finally {
|
} finally {
|
||||||
pendingInSync.remove(allocationId);
|
pendingInSync.remove(allocationId);
|
||||||
|
updateGlobalCheckpointOnPrimary();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Wait for knowledge of the local checkpoint for the specified allocation ID to advance to the global checkpoint. Global checkpoint
|
||||||
|
* advancement is blocked while there are any allocation IDs waiting to catch up to the global checkpoint.
|
||||||
|
*
|
||||||
|
* @param allocationId the allocation ID
|
||||||
|
* @throws InterruptedException if this thread was interrupted before of during waiting
|
||||||
|
*/
|
||||||
|
private synchronized void waitForAllocationIdToBeInSync(final String allocationId) throws InterruptedException {
|
||||||
|
while (true) {
|
||||||
|
/*
|
||||||
|
* If the allocation has been cancelled and so removed from the tracking map from a cluster state update from the master it
|
||||||
|
* means that this recovery will be cancelled; we are here on a cancellable recovery thread and so this thread will throw an
|
||||||
|
* interrupted exception as soon as it tries to wait on the monitor.
|
||||||
|
*/
|
||||||
|
final long current = trackingLocalCheckpoints.getOrDefault(allocationId, Long.MIN_VALUE);
|
||||||
|
if (current >= globalCheckpoint) {
|
||||||
|
logger.trace("marked [{}] as in-sync with local checkpoint [{}]", allocationId, current);
|
||||||
|
trackingLocalCheckpoints.remove(allocationId);
|
||||||
|
/*
|
||||||
|
* This is prematurely adding the allocation ID to the in-sync map as at this point recovery is not yet finished and could
|
||||||
|
* still abort. At this point we will end up with a shard in the in-sync map holding back the global checkpoint because the
|
||||||
|
* shard never recovered and we would have to wait until either the recovery retries and completes successfully, or the
|
||||||
|
* master fails the shard and issues a cluster state update that removes the shard from the set of active allocation IDs.
|
||||||
|
*/
|
||||||
|
inSyncLocalCheckpoints.put(allocationId, current);
|
||||||
|
break;
|
||||||
|
} else {
|
||||||
|
waitForLocalCheckpointToAdvance();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Wait for the local checkpoint to advance to the global checkpoint.
|
||||||
|
*
|
||||||
|
* @throws InterruptedException if this thread was interrupted before of during waiting
|
||||||
|
*/
|
||||||
@SuppressForbidden(reason = "Object#wait for local checkpoint advancement")
|
@SuppressForbidden(reason = "Object#wait for local checkpoint advancement")
|
||||||
private synchronized void waitForLocalCheckpointToAdvance() throws InterruptedException {
|
private synchronized void waitForLocalCheckpointToAdvance() throws InterruptedException {
|
||||||
this.wait();
|
this.wait();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check if there are any recoveries pending in-sync.
|
||||||
|
*
|
||||||
|
* @return {@code true} if there is at least one shard pending in-sync, otherwise false
|
||||||
|
*/
|
||||||
|
public boolean pendingInSync() {
|
||||||
|
return !pendingInSync.isEmpty();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the local checkpoint for the shard with the specified allocation ID, or {@link SequenceNumbersService#UNASSIGNED_SEQ_NO} if
|
* Returns the local checkpoint for the shard with the specified allocation ID, or {@link SequenceNumbersService#UNASSIGNED_SEQ_NO} if
|
||||||
* the shard is not in-sync.
|
* the shard is not in-sync.
|
||||||
|
|
|
@ -151,17 +151,7 @@ public class SequenceNumbersService extends AbstractIndexShardComponent {
|
||||||
* @return the global checkpoint
|
* @return the global checkpoint
|
||||||
*/
|
*/
|
||||||
public long getGlobalCheckpoint() {
|
public long getGlobalCheckpoint() {
|
||||||
return globalCheckpointTracker.getCheckpoint();
|
return globalCheckpointTracker.getGlobalCheckpoint();
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Scans through the currently known local checkpoint and updates the global checkpoint accordingly.
|
|
||||||
*
|
|
||||||
* @return {@code true} if the checkpoint has been updated or if it can not be updated since one of the local checkpoints of one of the
|
|
||||||
* active allocations is not known.
|
|
||||||
*/
|
|
||||||
public boolean updateGlobalCheckpointOnPrimary() {
|
|
||||||
return globalCheckpointTracker.updateCheckpointOnPrimary();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -184,4 +174,13 @@ public class SequenceNumbersService extends AbstractIndexShardComponent {
|
||||||
globalCheckpointTracker.updateAllocationIdsFromMaster(activeAllocationIds, initializingAllocationIds);
|
globalCheckpointTracker.updateAllocationIdsFromMaster(activeAllocationIds, initializingAllocationIds);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check if there are any recoveries pending in-sync.
|
||||||
|
*
|
||||||
|
* @return {@code true} if there is at least one shard pending in-sync, otherwise false
|
||||||
|
*/
|
||||||
|
public boolean pendingInSync() {
|
||||||
|
return globalCheckpointTracker.pendingInSync();
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -211,8 +211,6 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
|
|
||||||
private final IndexSearcherWrapper searcherWrapper;
|
private final IndexSearcherWrapper searcherWrapper;
|
||||||
|
|
||||||
private final Runnable globalCheckpointSyncer;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* True if this shard is still indexing (recently) and false if we've been idle for long enough (as periodically checked by {@link
|
* True if this shard is still indexing (recently) and false if we've been idle for long enough (as periodically checked by {@link
|
||||||
* IndexingMemoryController}).
|
* IndexingMemoryController}).
|
||||||
|
@ -227,7 +225,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
Supplier<Sort> indexSortSupplier, IndexCache indexCache, MapperService mapperService, SimilarityService similarityService,
|
Supplier<Sort> indexSortSupplier, IndexCache indexCache, MapperService mapperService, SimilarityService similarityService,
|
||||||
IndexFieldDataService indexFieldDataService, @Nullable EngineFactory engineFactory,
|
IndexFieldDataService indexFieldDataService, @Nullable EngineFactory engineFactory,
|
||||||
IndexEventListener indexEventListener, IndexSearcherWrapper indexSearcherWrapper, ThreadPool threadPool, BigArrays bigArrays,
|
IndexEventListener indexEventListener, IndexSearcherWrapper indexSearcherWrapper, ThreadPool threadPool, BigArrays bigArrays,
|
||||||
Engine.Warmer warmer, Runnable globalCheckpointSyncer, List<SearchOperationListener> searchOperationListener, List<IndexingOperationListener> listeners) throws IOException {
|
Engine.Warmer warmer, List<SearchOperationListener> searchOperationListener, List<IndexingOperationListener> listeners) throws IOException {
|
||||||
super(shardRouting.shardId(), indexSettings);
|
super(shardRouting.shardId(), indexSettings);
|
||||||
assert shardRouting.initializing();
|
assert shardRouting.initializing();
|
||||||
this.shardRouting = shardRouting;
|
this.shardRouting = shardRouting;
|
||||||
|
@ -251,7 +249,6 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
final List<SearchOperationListener> searchListenersList = new ArrayList<>(searchOperationListener);
|
final List<SearchOperationListener> searchListenersList = new ArrayList<>(searchOperationListener);
|
||||||
searchListenersList.add(searchStats);
|
searchListenersList.add(searchStats);
|
||||||
this.searchOperationListener = new SearchOperationListener.CompositeListener(searchListenersList, logger);
|
this.searchOperationListener = new SearchOperationListener.CompositeListener(searchListenersList, logger);
|
||||||
this.globalCheckpointSyncer = globalCheckpointSyncer;
|
|
||||||
this.getService = new ShardGetService(indexSettings, this, mapperService);
|
this.getService = new ShardGetService(indexSettings, this, mapperService);
|
||||||
this.shardWarmerService = new ShardIndexWarmerService(shardId, indexSettings);
|
this.shardWarmerService = new ShardIndexWarmerService(shardId, indexSettings);
|
||||||
this.requestCacheStats = new ShardRequestCache();
|
this.requestCacheStats = new ShardRequestCache();
|
||||||
|
@ -1486,6 +1483,11 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
public void markAllocationIdAsInSync(final String allocationId, final long localCheckpoint) throws InterruptedException {
|
public void markAllocationIdAsInSync(final String allocationId, final long localCheckpoint) throws InterruptedException {
|
||||||
verifyPrimary();
|
verifyPrimary();
|
||||||
getEngine().seqNoService().markAllocationIdAsInSync(allocationId, localCheckpoint);
|
getEngine().seqNoService().markAllocationIdAsInSync(allocationId, localCheckpoint);
|
||||||
|
/*
|
||||||
|
* We could have blocked waiting for the replica to catch up that we fell idle and there will not be a background sync to the
|
||||||
|
* replica; mark our self as active to force a future background sync.
|
||||||
|
*/
|
||||||
|
active.compareAndSet(false, true);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -1506,17 +1508,6 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
return getEngine().seqNoService().getGlobalCheckpoint();
|
return getEngine().seqNoService().getGlobalCheckpoint();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Checks whether the global checkpoint can be updated based on current knowledge of local checkpoints on the different shard copies.
|
|
||||||
* The checkpoint is updated or if more information is required from the replica, a global checkpoint sync is initiated.
|
|
||||||
*/
|
|
||||||
public void updateGlobalCheckpointOnPrimary() {
|
|
||||||
verifyPrimary();
|
|
||||||
if (getEngine().seqNoService().updateGlobalCheckpointOnPrimary()) {
|
|
||||||
globalCheckpointSyncer.run();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Updates the global checkpoint on a replica shard after it has been updated by the primary.
|
* Updates the global checkpoint on a replica shard after it has been updated by the primary.
|
||||||
*
|
*
|
||||||
|
@ -1561,6 +1552,16 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check if there are any recoveries pending in-sync.
|
||||||
|
*
|
||||||
|
* @return {@code true} if there is at least one shard pending in-sync, otherwise false
|
||||||
|
*/
|
||||||
|
public boolean pendingInSync() {
|
||||||
|
verifyPrimary();
|
||||||
|
return getEngine().seqNoService().pendingInSync();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Should be called for each no-op update operation to increment relevant statistics.
|
* Should be called for each no-op update operation to increment relevant statistics.
|
||||||
*
|
*
|
||||||
|
@ -2074,9 +2075,4 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// for tests
|
|
||||||
Runnable getGlobalCheckpointSyncer() {
|
|
||||||
return globalCheckpointSyncer;
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -182,7 +182,7 @@ public class IndicesService extends AbstractLifecycleComponent
|
||||||
}
|
}
|
||||||
|
|
||||||
public IndicesService(Settings settings, PluginsService pluginsService, NodeEnvironment nodeEnv, NamedXContentRegistry xContentRegistry,
|
public IndicesService(Settings settings, PluginsService pluginsService, NodeEnvironment nodeEnv, NamedXContentRegistry xContentRegistry,
|
||||||
ClusterSettings clusterSettings, AnalysisRegistry analysisRegistry,
|
AnalysisRegistry analysisRegistry,
|
||||||
IndexNameExpressionResolver indexNameExpressionResolver,
|
IndexNameExpressionResolver indexNameExpressionResolver,
|
||||||
MapperRegistry mapperRegistry, NamedWriteableRegistry namedWriteableRegistry,
|
MapperRegistry mapperRegistry, NamedWriteableRegistry namedWriteableRegistry,
|
||||||
ThreadPool threadPool, IndexScopedSettings indexScopedSettings, CircuitBreakerService circuitBreakerService,
|
ThreadPool threadPool, IndexScopedSettings indexScopedSettings, CircuitBreakerService circuitBreakerService,
|
||||||
|
@ -373,7 +373,7 @@ public class IndicesService extends AbstractLifecycleComponent
|
||||||
* @throws ResourceAlreadyExistsException if the index already exists.
|
* @throws ResourceAlreadyExistsException if the index already exists.
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public synchronized IndexService createIndex(IndexMetaData indexMetaData, List<IndexEventListener> builtInListeners, Consumer<ShardId> globalCheckpointSyncer) throws IOException {
|
public synchronized IndexService createIndex(IndexMetaData indexMetaData, List<IndexEventListener> builtInListeners) throws IOException {
|
||||||
ensureChangesAllowed();
|
ensureChangesAllowed();
|
||||||
if (indexMetaData.getIndexUUID().equals(IndexMetaData.INDEX_UUID_NA_VALUE)) {
|
if (indexMetaData.getIndexUUID().equals(IndexMetaData.INDEX_UUID_NA_VALUE)) {
|
||||||
throw new IllegalArgumentException("index must have a real UUID found value: [" + indexMetaData.getIndexUUID() + "]");
|
throw new IllegalArgumentException("index must have a real UUID found value: [" + indexMetaData.getIndexUUID() + "]");
|
||||||
|
@ -398,8 +398,7 @@ public class IndicesService extends AbstractLifecycleComponent
|
||||||
indicesQueryCache,
|
indicesQueryCache,
|
||||||
indicesFieldDataCache,
|
indicesFieldDataCache,
|
||||||
finalListeners,
|
finalListeners,
|
||||||
globalCheckpointSyncer,
|
indexingMemoryController);
|
||||||
indexingMemoryController);
|
|
||||||
boolean success = false;
|
boolean success = false;
|
||||||
try {
|
try {
|
||||||
indexService.getIndexEventListener().afterIndexCreated(indexService);
|
indexService.getIndexEventListener().afterIndexCreated(indexService);
|
||||||
|
@ -420,9 +419,7 @@ public class IndicesService extends AbstractLifecycleComponent
|
||||||
IndexMetaData indexMetaData, IndicesQueryCache indicesQueryCache,
|
IndexMetaData indexMetaData, IndicesQueryCache indicesQueryCache,
|
||||||
IndicesFieldDataCache indicesFieldDataCache,
|
IndicesFieldDataCache indicesFieldDataCache,
|
||||||
List<IndexEventListener> builtInListeners,
|
List<IndexEventListener> builtInListeners,
|
||||||
Consumer<ShardId> globalCheckpointSyncer,
|
|
||||||
IndexingOperationListener... indexingOperationListeners) throws IOException {
|
IndexingOperationListener... indexingOperationListeners) throws IOException {
|
||||||
final Index index = indexMetaData.getIndex();
|
|
||||||
final IndexSettings idxSettings = new IndexSettings(indexMetaData, this.settings, indexScopeSetting);
|
final IndexSettings idxSettings = new IndexSettings(indexMetaData, this.settings, indexScopeSetting);
|
||||||
logger.debug("creating Index [{}], shards [{}]/[{}] - reason [{}]",
|
logger.debug("creating Index [{}], shards [{}]/[{}] - reason [{}]",
|
||||||
indexMetaData.getIndex(),
|
indexMetaData.getIndex(),
|
||||||
|
@ -439,19 +436,17 @@ public class IndicesService extends AbstractLifecycleComponent
|
||||||
indexModule.addIndexEventListener(listener);
|
indexModule.addIndexEventListener(listener);
|
||||||
}
|
}
|
||||||
return indexModule.newIndexService(
|
return indexModule.newIndexService(
|
||||||
nodeEnv,
|
nodeEnv,
|
||||||
xContentRegistry,
|
xContentRegistry,
|
||||||
this,
|
this,
|
||||||
circuitBreakerService,
|
circuitBreakerService,
|
||||||
bigArrays,
|
bigArrays,
|
||||||
threadPool,
|
threadPool,
|
||||||
scriptService,
|
scriptService,
|
||||||
clusterService,
|
client,
|
||||||
client,
|
indicesQueryCache,
|
||||||
indicesQueryCache,
|
mapperRegistry,
|
||||||
mapperRegistry,
|
indicesFieldDataCache);
|
||||||
globalCheckpointSyncer,
|
|
||||||
indicesFieldDataCache);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -482,7 +477,7 @@ public class IndicesService extends AbstractLifecycleComponent
|
||||||
closeables.add(indicesQueryCache);
|
closeables.add(indicesQueryCache);
|
||||||
// this will also fail if some plugin fails etc. which is nice since we can verify that early
|
// this will also fail if some plugin fails etc. which is nice since we can verify that early
|
||||||
final IndexService service =
|
final IndexService service =
|
||||||
createIndexService("metadata verification", metaData, indicesQueryCache, indicesFieldDataCache, emptyList(), s -> {});
|
createIndexService("metadata verification", metaData, indicesQueryCache, indicesFieldDataCache, emptyList());
|
||||||
closeables.add(() -> service.close("metadata verification", false));
|
closeables.add(() -> service.close("metadata verification", false));
|
||||||
service.mapperService().merge(metaData, MapperService.MergeReason.MAPPING_RECOVERY, true);
|
service.mapperService().merge(metaData, MapperService.MergeReason.MAPPING_RECOVERY, true);
|
||||||
if (metaData.equals(metaDataUpdate) == false) {
|
if (metaData.equals(metaDataUpdate) == false) {
|
||||||
|
|
|
@ -19,6 +19,8 @@
|
||||||
|
|
||||||
package org.elasticsearch.indices.analysis;
|
package org.elasticsearch.indices.analysis;
|
||||||
|
|
||||||
|
import org.apache.lucene.analysis.LowerCaseFilter;
|
||||||
|
import org.apache.lucene.analysis.standard.StandardFilter;
|
||||||
import org.elasticsearch.Version;
|
import org.elasticsearch.Version;
|
||||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||||
import org.elasticsearch.common.NamedRegistry;
|
import org.elasticsearch.common.NamedRegistry;
|
||||||
|
@ -101,6 +103,7 @@ import org.elasticsearch.index.analysis.PersianAnalyzerProvider;
|
||||||
import org.elasticsearch.index.analysis.PersianNormalizationFilterFactory;
|
import org.elasticsearch.index.analysis.PersianNormalizationFilterFactory;
|
||||||
import org.elasticsearch.index.analysis.PorterStemTokenFilterFactory;
|
import org.elasticsearch.index.analysis.PorterStemTokenFilterFactory;
|
||||||
import org.elasticsearch.index.analysis.PortugueseAnalyzerProvider;
|
import org.elasticsearch.index.analysis.PortugueseAnalyzerProvider;
|
||||||
|
import org.elasticsearch.index.analysis.PreConfiguredTokenFilter;
|
||||||
import org.elasticsearch.index.analysis.ReverseTokenFilterFactory;
|
import org.elasticsearch.index.analysis.ReverseTokenFilterFactory;
|
||||||
import org.elasticsearch.index.analysis.RomanianAnalyzerProvider;
|
import org.elasticsearch.index.analysis.RomanianAnalyzerProvider;
|
||||||
import org.elasticsearch.index.analysis.RussianAnalyzerProvider;
|
import org.elasticsearch.index.analysis.RussianAnalyzerProvider;
|
||||||
|
@ -138,11 +141,15 @@ import org.elasticsearch.index.analysis.WhitespaceAnalyzerProvider;
|
||||||
import org.elasticsearch.index.analysis.WhitespaceTokenizerFactory;
|
import org.elasticsearch.index.analysis.WhitespaceTokenizerFactory;
|
||||||
import org.elasticsearch.index.analysis.compound.DictionaryCompoundWordTokenFilterFactory;
|
import org.elasticsearch.index.analysis.compound.DictionaryCompoundWordTokenFilterFactory;
|
||||||
import org.elasticsearch.index.analysis.compound.HyphenationCompoundWordTokenFilterFactory;
|
import org.elasticsearch.index.analysis.compound.HyphenationCompoundWordTokenFilterFactory;
|
||||||
|
import org.elasticsearch.indices.analysis.PreBuiltCacheFactory.CachingStrategy;
|
||||||
import org.elasticsearch.plugins.AnalysisPlugin;
|
import org.elasticsearch.plugins.AnalysisPlugin;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Locale;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import static java.util.Collections.unmodifiableMap;
|
||||||
import static org.elasticsearch.plugins.AnalysisPlugin.requriesAnalysisSettings;
|
import static org.elasticsearch.plugins.AnalysisPlugin.requriesAnalysisSettings;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -169,8 +176,11 @@ public final class AnalysisModule {
|
||||||
NamedRegistry<AnalysisProvider<TokenizerFactory>> tokenizers = setupTokenizers(plugins);
|
NamedRegistry<AnalysisProvider<TokenizerFactory>> tokenizers = setupTokenizers(plugins);
|
||||||
NamedRegistry<AnalysisProvider<AnalyzerProvider<?>>> analyzers = setupAnalyzers(plugins);
|
NamedRegistry<AnalysisProvider<AnalyzerProvider<?>>> analyzers = setupAnalyzers(plugins);
|
||||||
NamedRegistry<AnalysisProvider<AnalyzerProvider<?>>> normalizers = setupNormalizers(plugins);
|
NamedRegistry<AnalysisProvider<AnalyzerProvider<?>>> normalizers = setupNormalizers(plugins);
|
||||||
|
|
||||||
|
Map<String, PreConfiguredTokenFilter> preConfiguredTokenFilters = setupPreConfiguredTokenFilters(plugins);
|
||||||
|
|
||||||
analysisRegistry = new AnalysisRegistry(environment, charFilters.getRegistry(), tokenFilters.getRegistry(), tokenizers
|
analysisRegistry = new AnalysisRegistry(environment, charFilters.getRegistry(), tokenFilters.getRegistry(), tokenizers
|
||||||
.getRegistry(), analyzers.getRegistry(), normalizers.getRegistry());
|
.getRegistry(), analyzers.getRegistry(), normalizers.getRegistry(), preConfiguredTokenFilters);
|
||||||
}
|
}
|
||||||
|
|
||||||
HunspellService getHunspellService() {
|
HunspellService getHunspellService() {
|
||||||
|
@ -258,6 +268,40 @@ public final class AnalysisModule {
|
||||||
return tokenFilters;
|
return tokenFilters;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
static Map<String, PreConfiguredTokenFilter> setupPreConfiguredTokenFilters(List<AnalysisPlugin> plugins) {
|
||||||
|
NamedRegistry<PreConfiguredTokenFilter> preConfiguredTokenFilters = new NamedRegistry<>("pre-configured token_filter");
|
||||||
|
|
||||||
|
// Add filters available in lucene-core
|
||||||
|
preConfiguredTokenFilters.register("lowercase",
|
||||||
|
new PreConfiguredTokenFilter("lowercase", true, CachingStrategy.LUCENE, LowerCaseFilter::new));
|
||||||
|
preConfiguredTokenFilters.register("standard",
|
||||||
|
new PreConfiguredTokenFilter("standard", false, CachingStrategy.LUCENE, StandardFilter::new));
|
||||||
|
/* Note that "stop" is available in lucene-core but it's pre-built
|
||||||
|
* version uses a set of English stop words that are in
|
||||||
|
* lucene-analyzers-common so "stop" is defined in the analysis-common
|
||||||
|
* module. */
|
||||||
|
|
||||||
|
// Add token filters declared in PreBuiltTokenFilters until they have all been migrated
|
||||||
|
for (PreBuiltTokenFilters preBuilt : PreBuiltTokenFilters.values()) {
|
||||||
|
switch (preBuilt) {
|
||||||
|
case LOWERCASE:
|
||||||
|
// This has been migrated but has to stick around until PreBuiltTokenizers is removed.
|
||||||
|
continue;
|
||||||
|
default:
|
||||||
|
String name = preBuilt.name().toLowerCase(Locale.ROOT);
|
||||||
|
preConfiguredTokenFilters.register(name,
|
||||||
|
new PreConfiguredTokenFilter(name, preBuilt.isMultiTermAware(), preBuilt.getCachingStrategy(), preBuilt::create));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
for (AnalysisPlugin plugin: plugins) {
|
||||||
|
for (PreConfiguredTokenFilter filter : plugin.getPreConfiguredTokenFilters()) {
|
||||||
|
preConfiguredTokenFilters.register(filter.getName(), filter);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return unmodifiableMap(preConfiguredTokenFilters.getRegistry());
|
||||||
|
}
|
||||||
|
|
||||||
private NamedRegistry<AnalysisProvider<TokenizerFactory>> setupTokenizers(List<AnalysisPlugin> plugins) {
|
private NamedRegistry<AnalysisProvider<TokenizerFactory>> setupTokenizers(List<AnalysisPlugin> plugins) {
|
||||||
NamedRegistry<AnalysisProvider<TokenizerFactory>> tokenizers = new NamedRegistry<>("tokenizer");
|
NamedRegistry<AnalysisProvider<TokenizerFactory>> tokenizers = new NamedRegistry<>("tokenizer");
|
||||||
tokenizers.register("standard", StandardTokenizerFactory::new);
|
tokenizers.register("standard", StandardTokenizerFactory::new);
|
||||||
|
|
|
@ -42,7 +42,7 @@ public class PreBuiltCacheFactory {
|
||||||
|
|
||||||
private PreBuiltCacheFactory() {}
|
private PreBuiltCacheFactory() {}
|
||||||
|
|
||||||
static <T> PreBuiltCache<T> getCache(CachingStrategy cachingStrategy) {
|
public static <T> PreBuiltCache<T> getCache(CachingStrategy cachingStrategy) {
|
||||||
switch (cachingStrategy) {
|
switch (cachingStrategy) {
|
||||||
case ONE:
|
case ONE:
|
||||||
return new PreBuiltCacheStrategyOne<>();
|
return new PreBuiltCacheStrategyOne<>();
|
||||||
|
|
|
@ -18,9 +18,7 @@
|
||||||
*/
|
*/
|
||||||
package org.elasticsearch.indices.analysis;
|
package org.elasticsearch.indices.analysis;
|
||||||
|
|
||||||
import org.apache.lucene.analysis.CharArraySet;
|
|
||||||
import org.apache.lucene.analysis.LowerCaseFilter;
|
import org.apache.lucene.analysis.LowerCaseFilter;
|
||||||
import org.apache.lucene.analysis.StopFilter;
|
|
||||||
import org.apache.lucene.analysis.TokenStream;
|
import org.apache.lucene.analysis.TokenStream;
|
||||||
import org.apache.lucene.analysis.ar.ArabicNormalizationFilter;
|
import org.apache.lucene.analysis.ar.ArabicNormalizationFilter;
|
||||||
import org.apache.lucene.analysis.ar.ArabicStemFilter;
|
import org.apache.lucene.analysis.ar.ArabicStemFilter;
|
||||||
|
@ -28,39 +26,23 @@ import org.apache.lucene.analysis.br.BrazilianStemFilter;
|
||||||
import org.apache.lucene.analysis.cjk.CJKBigramFilter;
|
import org.apache.lucene.analysis.cjk.CJKBigramFilter;
|
||||||
import org.apache.lucene.analysis.cjk.CJKWidthFilter;
|
import org.apache.lucene.analysis.cjk.CJKWidthFilter;
|
||||||
import org.apache.lucene.analysis.ckb.SoraniNormalizationFilter;
|
import org.apache.lucene.analysis.ckb.SoraniNormalizationFilter;
|
||||||
import org.apache.lucene.analysis.commongrams.CommonGramsFilter;
|
|
||||||
import org.apache.lucene.analysis.core.DecimalDigitFilter;
|
import org.apache.lucene.analysis.core.DecimalDigitFilter;
|
||||||
import org.apache.lucene.analysis.core.StopAnalyzer;
|
|
||||||
import org.apache.lucene.analysis.core.UpperCaseFilter;
|
|
||||||
import org.apache.lucene.analysis.cz.CzechStemFilter;
|
import org.apache.lucene.analysis.cz.CzechStemFilter;
|
||||||
import org.apache.lucene.analysis.de.GermanNormalizationFilter;
|
import org.apache.lucene.analysis.de.GermanNormalizationFilter;
|
||||||
import org.apache.lucene.analysis.de.GermanStemFilter;
|
import org.apache.lucene.analysis.de.GermanStemFilter;
|
||||||
import org.apache.lucene.analysis.en.KStemFilter;
|
|
||||||
import org.apache.lucene.analysis.en.PorterStemFilter;
|
import org.apache.lucene.analysis.en.PorterStemFilter;
|
||||||
import org.apache.lucene.analysis.fa.PersianNormalizationFilter;
|
import org.apache.lucene.analysis.fa.PersianNormalizationFilter;
|
||||||
import org.apache.lucene.analysis.fr.FrenchAnalyzer;
|
import org.apache.lucene.analysis.fr.FrenchAnalyzer;
|
||||||
import org.apache.lucene.analysis.hi.HindiNormalizationFilter;
|
import org.apache.lucene.analysis.hi.HindiNormalizationFilter;
|
||||||
import org.apache.lucene.analysis.in.IndicNormalizationFilter;
|
import org.apache.lucene.analysis.in.IndicNormalizationFilter;
|
||||||
import org.apache.lucene.analysis.miscellaneous.ASCIIFoldingFilter;
|
|
||||||
import org.apache.lucene.analysis.miscellaneous.KeywordRepeatFilter;
|
import org.apache.lucene.analysis.miscellaneous.KeywordRepeatFilter;
|
||||||
import org.apache.lucene.analysis.miscellaneous.LengthFilter;
|
|
||||||
import org.apache.lucene.analysis.miscellaneous.LimitTokenCountFilter;
|
import org.apache.lucene.analysis.miscellaneous.LimitTokenCountFilter;
|
||||||
import org.apache.lucene.analysis.miscellaneous.ScandinavianFoldingFilter;
|
import org.apache.lucene.analysis.miscellaneous.ScandinavianFoldingFilter;
|
||||||
import org.apache.lucene.analysis.miscellaneous.ScandinavianNormalizationFilter;
|
import org.apache.lucene.analysis.miscellaneous.ScandinavianNormalizationFilter;
|
||||||
import org.apache.lucene.analysis.miscellaneous.TrimFilter;
|
|
||||||
import org.apache.lucene.analysis.miscellaneous.TruncateTokenFilter;
|
|
||||||
import org.apache.lucene.analysis.miscellaneous.UniqueTokenFilter;
|
|
||||||
import org.apache.lucene.analysis.miscellaneous.WordDelimiterFilter;
|
|
||||||
import org.apache.lucene.analysis.miscellaneous.WordDelimiterGraphFilter;
|
|
||||||
import org.apache.lucene.analysis.ngram.EdgeNGramTokenFilter;
|
|
||||||
import org.apache.lucene.analysis.ngram.NGramTokenFilter;
|
|
||||||
import org.apache.lucene.analysis.payloads.DelimitedPayloadTokenFilter;
|
import org.apache.lucene.analysis.payloads.DelimitedPayloadTokenFilter;
|
||||||
import org.apache.lucene.analysis.payloads.TypeAsPayloadTokenFilter;
|
import org.apache.lucene.analysis.payloads.TypeAsPayloadTokenFilter;
|
||||||
import org.apache.lucene.analysis.reverse.ReverseStringFilter;
|
|
||||||
import org.apache.lucene.analysis.shingle.ShingleFilter;
|
import org.apache.lucene.analysis.shingle.ShingleFilter;
|
||||||
import org.apache.lucene.analysis.snowball.SnowballFilter;
|
import org.apache.lucene.analysis.snowball.SnowballFilter;
|
||||||
import org.apache.lucene.analysis.standard.ClassicFilter;
|
|
||||||
import org.apache.lucene.analysis.standard.StandardFilter;
|
|
||||||
import org.apache.lucene.analysis.tr.ApostropheFilter;
|
import org.apache.lucene.analysis.tr.ApostropheFilter;
|
||||||
import org.apache.lucene.analysis.util.ElisionFilter;
|
import org.apache.lucene.analysis.util.ElisionFilter;
|
||||||
import org.elasticsearch.Version;
|
import org.elasticsearch.Version;
|
||||||
|
@ -75,77 +57,7 @@ import org.tartarus.snowball.ext.FrenchStemmer;
|
||||||
import java.util.Locale;
|
import java.util.Locale;
|
||||||
|
|
||||||
public enum PreBuiltTokenFilters {
|
public enum PreBuiltTokenFilters {
|
||||||
|
// TODO remove this entire class when PreBuiltTokenizers no longer needs it.....
|
||||||
WORD_DELIMITER(CachingStrategy.ONE) {
|
|
||||||
@Override
|
|
||||||
public TokenStream create(TokenStream tokenStream, Version version) {
|
|
||||||
return new WordDelimiterFilter(tokenStream,
|
|
||||||
WordDelimiterFilter.GENERATE_WORD_PARTS |
|
|
||||||
WordDelimiterFilter.GENERATE_NUMBER_PARTS |
|
|
||||||
WordDelimiterFilter.SPLIT_ON_CASE_CHANGE |
|
|
||||||
WordDelimiterFilter.SPLIT_ON_NUMERICS |
|
|
||||||
WordDelimiterFilter.STEM_ENGLISH_POSSESSIVE, null);
|
|
||||||
}
|
|
||||||
},
|
|
||||||
|
|
||||||
WORD_DELIMITER_GRAPH(CachingStrategy.ONE) {
|
|
||||||
@Override
|
|
||||||
public TokenStream create(TokenStream tokenStream, Version version) {
|
|
||||||
return new WordDelimiterGraphFilter(tokenStream,
|
|
||||||
WordDelimiterGraphFilter.GENERATE_WORD_PARTS |
|
|
||||||
WordDelimiterGraphFilter.GENERATE_NUMBER_PARTS |
|
|
||||||
WordDelimiterGraphFilter.SPLIT_ON_CASE_CHANGE |
|
|
||||||
WordDelimiterGraphFilter.SPLIT_ON_NUMERICS |
|
|
||||||
WordDelimiterGraphFilter.STEM_ENGLISH_POSSESSIVE, null);
|
|
||||||
}
|
|
||||||
},
|
|
||||||
|
|
||||||
STOP(CachingStrategy.LUCENE) {
|
|
||||||
@Override
|
|
||||||
public TokenStream create(TokenStream tokenStream, Version version) {
|
|
||||||
return new StopFilter(tokenStream, StopAnalyzer.ENGLISH_STOP_WORDS_SET);
|
|
||||||
}
|
|
||||||
},
|
|
||||||
|
|
||||||
TRIM(CachingStrategy.LUCENE) {
|
|
||||||
@Override
|
|
||||||
public TokenStream create(TokenStream tokenStream, Version version) {
|
|
||||||
return new TrimFilter(tokenStream);
|
|
||||||
}
|
|
||||||
},
|
|
||||||
|
|
||||||
REVERSE(CachingStrategy.LUCENE) {
|
|
||||||
@Override
|
|
||||||
public TokenStream create(TokenStream tokenStream, Version version) {
|
|
||||||
return new ReverseStringFilter(tokenStream);
|
|
||||||
}
|
|
||||||
},
|
|
||||||
|
|
||||||
ASCIIFOLDING(CachingStrategy.ONE) {
|
|
||||||
@Override
|
|
||||||
public TokenStream create(TokenStream tokenStream, Version version) {
|
|
||||||
return new ASCIIFoldingFilter(tokenStream);
|
|
||||||
}
|
|
||||||
@Override
|
|
||||||
protected boolean isMultiTermAware() {
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
},
|
|
||||||
|
|
||||||
LENGTH(CachingStrategy.LUCENE) {
|
|
||||||
@Override
|
|
||||||
public TokenStream create(TokenStream tokenStream, Version version) {
|
|
||||||
return new LengthFilter(tokenStream, 0, Integer.MAX_VALUE);
|
|
||||||
}
|
|
||||||
},
|
|
||||||
|
|
||||||
COMMON_GRAMS(CachingStrategy.LUCENE) {
|
|
||||||
@Override
|
|
||||||
public TokenStream create(TokenStream tokenStream, Version version) {
|
|
||||||
return new CommonGramsFilter(tokenStream, CharArraySet.EMPTY_SET);
|
|
||||||
}
|
|
||||||
},
|
|
||||||
|
|
||||||
LOWERCASE(CachingStrategy.LUCENE) {
|
LOWERCASE(CachingStrategy.LUCENE) {
|
||||||
@Override
|
@Override
|
||||||
public TokenStream create(TokenStream tokenStream, Version version) {
|
public TokenStream create(TokenStream tokenStream, Version version) {
|
||||||
|
@ -157,73 +69,6 @@ public enum PreBuiltTokenFilters {
|
||||||
}
|
}
|
||||||
},
|
},
|
||||||
|
|
||||||
UPPERCASE(CachingStrategy.LUCENE) {
|
|
||||||
@Override
|
|
||||||
public TokenStream create(TokenStream tokenStream, Version version) {
|
|
||||||
return new UpperCaseFilter(tokenStream);
|
|
||||||
}
|
|
||||||
@Override
|
|
||||||
protected boolean isMultiTermAware() {
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
},
|
|
||||||
|
|
||||||
KSTEM(CachingStrategy.ONE) {
|
|
||||||
@Override
|
|
||||||
public TokenStream create(TokenStream tokenStream, Version version) {
|
|
||||||
return new KStemFilter(tokenStream);
|
|
||||||
}
|
|
||||||
},
|
|
||||||
|
|
||||||
PORTER_STEM(CachingStrategy.ONE) {
|
|
||||||
@Override
|
|
||||||
public TokenStream create(TokenStream tokenStream, Version version) {
|
|
||||||
return new PorterStemFilter(tokenStream);
|
|
||||||
}
|
|
||||||
},
|
|
||||||
|
|
||||||
STANDARD(CachingStrategy.LUCENE) {
|
|
||||||
@Override
|
|
||||||
public TokenStream create(TokenStream tokenStream, Version version) {
|
|
||||||
return new StandardFilter(tokenStream);
|
|
||||||
}
|
|
||||||
},
|
|
||||||
|
|
||||||
CLASSIC(CachingStrategy.ONE) {
|
|
||||||
@Override
|
|
||||||
public TokenStream create(TokenStream tokenStream, Version version) {
|
|
||||||
return new ClassicFilter(tokenStream);
|
|
||||||
}
|
|
||||||
},
|
|
||||||
|
|
||||||
NGRAM(CachingStrategy.LUCENE) {
|
|
||||||
@Override
|
|
||||||
public TokenStream create(TokenStream tokenStream, Version version) {
|
|
||||||
return new NGramTokenFilter(tokenStream);
|
|
||||||
}
|
|
||||||
},
|
|
||||||
|
|
||||||
EDGE_NGRAM(CachingStrategy.LUCENE) {
|
|
||||||
@Override
|
|
||||||
public TokenStream create(TokenStream tokenStream, Version version) {
|
|
||||||
return new EdgeNGramTokenFilter(tokenStream, EdgeNGramTokenFilter.DEFAULT_MIN_GRAM_SIZE, EdgeNGramTokenFilter.DEFAULT_MAX_GRAM_SIZE);
|
|
||||||
}
|
|
||||||
},
|
|
||||||
|
|
||||||
UNIQUE(CachingStrategy.ONE) {
|
|
||||||
@Override
|
|
||||||
public TokenStream create(TokenStream tokenStream, Version version) {
|
|
||||||
return new UniqueTokenFilter(tokenStream);
|
|
||||||
}
|
|
||||||
},
|
|
||||||
|
|
||||||
TRUNCATE(CachingStrategy.ONE) {
|
|
||||||
@Override
|
|
||||||
public TokenStream create(TokenStream tokenStream, Version version) {
|
|
||||||
return new TruncateTokenFilter(tokenStream, 10);
|
|
||||||
}
|
|
||||||
},
|
|
||||||
|
|
||||||
// Extended Token Filters
|
// Extended Token Filters
|
||||||
SNOWBALL(CachingStrategy.ONE) {
|
SNOWBALL(CachingStrategy.ONE) {
|
||||||
@Override
|
@Override
|
||||||
|
@ -469,10 +314,16 @@ public enum PreBuiltTokenFilters {
|
||||||
protected final PreBuiltCacheFactory.PreBuiltCache<TokenFilterFactory> cache;
|
protected final PreBuiltCacheFactory.PreBuiltCache<TokenFilterFactory> cache;
|
||||||
|
|
||||||
|
|
||||||
|
private final CachingStrategy cachingStrategy;
|
||||||
PreBuiltTokenFilters(CachingStrategy cachingStrategy) {
|
PreBuiltTokenFilters(CachingStrategy cachingStrategy) {
|
||||||
|
this.cachingStrategy = cachingStrategy;
|
||||||
cache = PreBuiltCacheFactory.getCache(cachingStrategy);
|
cache = PreBuiltCacheFactory.getCache(cachingStrategy);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public CachingStrategy getCachingStrategy() {
|
||||||
|
return cachingStrategy;
|
||||||
|
}
|
||||||
|
|
||||||
private interface MultiTermAwareTokenFilterFactory extends TokenFilterFactory, MultiTermAwareComponent {}
|
private interface MultiTermAwareTokenFilterFactory extends TokenFilterFactory, MultiTermAwareComponent {}
|
||||||
|
|
||||||
public synchronized TokenFilterFactory getTokenFilterFactory(final Version version) {
|
public synchronized TokenFilterFactory getTokenFilterFactory(final Version version) {
|
||||||
|
@ -514,17 +365,4 @@ public enum PreBuiltTokenFilters {
|
||||||
|
|
||||||
return factory;
|
return factory;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Get a pre built TokenFilter by its name or fallback to the default one
|
|
||||||
* @param name TokenFilter name
|
|
||||||
* @param defaultTokenFilter default TokenFilter if name not found
|
|
||||||
*/
|
|
||||||
public static PreBuiltTokenFilters getOrDefault(String name, PreBuiltTokenFilters defaultTokenFilter) {
|
|
||||||
try {
|
|
||||||
return valueOf(name.toUpperCase(Locale.ROOT));
|
|
||||||
} catch (IllegalArgumentException e) {
|
|
||||||
return defaultTokenFilter;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -99,7 +99,6 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple
|
||||||
private final PeerRecoveryTargetService recoveryTargetService;
|
private final PeerRecoveryTargetService recoveryTargetService;
|
||||||
private final ShardStateAction shardStateAction;
|
private final ShardStateAction shardStateAction;
|
||||||
private final NodeMappingRefreshAction nodeMappingRefreshAction;
|
private final NodeMappingRefreshAction nodeMappingRefreshAction;
|
||||||
private final Consumer<ShardId> globalCheckpointSyncer;
|
|
||||||
|
|
||||||
private static final ShardStateAction.Listener SHARD_STATE_ACTION_LISTENER = new ShardStateAction.Listener() {
|
private static final ShardStateAction.Listener SHARD_STATE_ACTION_LISTENER = new ShardStateAction.Listener() {
|
||||||
};
|
};
|
||||||
|
@ -123,10 +122,10 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple
|
||||||
SearchService searchService, SyncedFlushService syncedFlushService,
|
SearchService searchService, SyncedFlushService syncedFlushService,
|
||||||
PeerRecoverySourceService peerRecoverySourceService, SnapshotShardsService snapshotShardsService,
|
PeerRecoverySourceService peerRecoverySourceService, SnapshotShardsService snapshotShardsService,
|
||||||
GlobalCheckpointSyncAction globalCheckpointSyncAction) {
|
GlobalCheckpointSyncAction globalCheckpointSyncAction) {
|
||||||
this(settings, indicesService,
|
this(settings, (AllocatedIndices<? extends Shard, ? extends AllocatedIndex<? extends Shard>>) indicesService,
|
||||||
clusterService, threadPool, recoveryTargetService, shardStateAction,
|
clusterService, threadPool, recoveryTargetService, shardStateAction,
|
||||||
nodeMappingRefreshAction, repositoriesService, searchService, syncedFlushService, peerRecoverySourceService,
|
nodeMappingRefreshAction, repositoriesService, searchService, syncedFlushService, peerRecoverySourceService,
|
||||||
snapshotShardsService, globalCheckpointSyncAction::updateCheckpointForShard);
|
snapshotShardsService, globalCheckpointSyncAction);
|
||||||
}
|
}
|
||||||
|
|
||||||
// for tests
|
// for tests
|
||||||
|
@ -139,10 +138,16 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple
|
||||||
RepositoriesService repositoriesService,
|
RepositoriesService repositoriesService,
|
||||||
SearchService searchService, SyncedFlushService syncedFlushService,
|
SearchService searchService, SyncedFlushService syncedFlushService,
|
||||||
PeerRecoverySourceService peerRecoverySourceService, SnapshotShardsService snapshotShardsService,
|
PeerRecoverySourceService peerRecoverySourceService, SnapshotShardsService snapshotShardsService,
|
||||||
Consumer<ShardId> globalCheckpointSyncer) {
|
GlobalCheckpointSyncAction globalCheckpointSyncAction) {
|
||||||
super(settings);
|
super(settings);
|
||||||
this.buildInIndexListener = Arrays.asList(peerRecoverySourceService, recoveryTargetService, searchService, syncedFlushService,
|
this.buildInIndexListener =
|
||||||
snapshotShardsService);
|
Arrays.asList(
|
||||||
|
peerRecoverySourceService,
|
||||||
|
recoveryTargetService,
|
||||||
|
searchService,
|
||||||
|
syncedFlushService,
|
||||||
|
snapshotShardsService,
|
||||||
|
globalCheckpointSyncAction);
|
||||||
this.indicesService = indicesService;
|
this.indicesService = indicesService;
|
||||||
this.clusterService = clusterService;
|
this.clusterService = clusterService;
|
||||||
this.threadPool = threadPool;
|
this.threadPool = threadPool;
|
||||||
|
@ -151,7 +156,6 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple
|
||||||
this.nodeMappingRefreshAction = nodeMappingRefreshAction;
|
this.nodeMappingRefreshAction = nodeMappingRefreshAction;
|
||||||
this.repositoriesService = repositoriesService;
|
this.repositoriesService = repositoriesService;
|
||||||
this.sendRefreshMapping = this.settings.getAsBoolean("indices.cluster.send_refresh_mapping", true);
|
this.sendRefreshMapping = this.settings.getAsBoolean("indices.cluster.send_refresh_mapping", true);
|
||||||
this.globalCheckpointSyncer = globalCheckpointSyncer;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -432,7 +436,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple
|
||||||
|
|
||||||
AllocatedIndex<? extends Shard> indexService = null;
|
AllocatedIndex<? extends Shard> indexService = null;
|
||||||
try {
|
try {
|
||||||
indexService = indicesService.createIndex(indexMetaData, buildInIndexListener, globalCheckpointSyncer);
|
indexService = indicesService.createIndex(indexMetaData, buildInIndexListener);
|
||||||
if (indexService.updateMapping(indexMetaData) && sendRefreshMapping) {
|
if (indexService.updateMapping(indexMetaData) && sendRefreshMapping) {
|
||||||
nodeMappingRefreshAction.nodeMappingRefresh(state.nodes().getMasterNode(),
|
nodeMappingRefreshAction.nodeMappingRefresh(state.nodes().getMasterNode(),
|
||||||
new NodeMappingRefreshAction.NodeMappingRefreshRequest(indexMetaData.getIndex().getName(),
|
new NodeMappingRefreshAction.NodeMappingRefreshRequest(indexMetaData.getIndex().getName(),
|
||||||
|
@ -769,12 +773,10 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple
|
||||||
* @param indexMetaData the index metadata to create the index for
|
* @param indexMetaData the index metadata to create the index for
|
||||||
* @param builtInIndexListener a list of built-in lifecycle {@link IndexEventListener} that should should be used along side with
|
* @param builtInIndexListener a list of built-in lifecycle {@link IndexEventListener} that should should be used along side with
|
||||||
* the per-index listeners
|
* the per-index listeners
|
||||||
* @param globalCheckpointSyncer the global checkpoint syncer
|
|
||||||
* @throws ResourceAlreadyExistsException if the index already exists.
|
* @throws ResourceAlreadyExistsException if the index already exists.
|
||||||
*/
|
*/
|
||||||
U createIndex(IndexMetaData indexMetaData,
|
U createIndex(IndexMetaData indexMetaData,
|
||||||
List<IndexEventListener> builtInIndexListener,
|
List<IndexEventListener> builtInIndexListener) throws IOException;
|
||||||
Consumer<ShardId> globalCheckpointSyncer) throws IOException;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Verify that the contents on disk for the given index is deleted; if not, delete the contents.
|
* Verify that the contents on disk for the given index is deleted; if not, delete the contents.
|
||||||
|
|
|
@ -392,7 +392,7 @@ public class Node implements Closeable {
|
||||||
modules.add(new RepositoriesModule(this.environment, pluginsService.filterPlugins(RepositoryPlugin.class), xContentRegistry));
|
modules.add(new RepositoriesModule(this.environment, pluginsService.filterPlugins(RepositoryPlugin.class), xContentRegistry));
|
||||||
final MetaStateService metaStateService = new MetaStateService(settings, nodeEnvironment, xContentRegistry);
|
final MetaStateService metaStateService = new MetaStateService(settings, nodeEnvironment, xContentRegistry);
|
||||||
final IndicesService indicesService = new IndicesService(settings, pluginsService, nodeEnvironment, xContentRegistry,
|
final IndicesService indicesService = new IndicesService(settings, pluginsService, nodeEnvironment, xContentRegistry,
|
||||||
settingsModule.getClusterSettings(), analysisModule.getAnalysisRegistry(),
|
analysisModule.getAnalysisRegistry(),
|
||||||
clusterModule.getIndexNameExpressionResolver(), indicesModule.getMapperRegistry(), namedWriteableRegistry,
|
clusterModule.getIndexNameExpressionResolver(), indicesModule.getMapperRegistry(), namedWriteableRegistry,
|
||||||
threadPool, settingsModule.getIndexScopedSettings(), circuitBreakerService, bigArrays, scriptModule.getScriptService(),
|
threadPool, settingsModule.getIndexScopedSettings(), circuitBreakerService, bigArrays, scriptModule.getScriptService(),
|
||||||
clusterService, client, metaStateService);
|
clusterService, client, metaStateService);
|
||||||
|
|
|
@ -22,19 +22,26 @@ package org.elasticsearch.plugins;
|
||||||
import org.apache.lucene.analysis.Analyzer;
|
import org.apache.lucene.analysis.Analyzer;
|
||||||
import org.apache.lucene.analysis.CharFilter;
|
import org.apache.lucene.analysis.CharFilter;
|
||||||
import org.apache.lucene.analysis.TokenFilter;
|
import org.apache.lucene.analysis.TokenFilter;
|
||||||
|
import org.apache.lucene.analysis.TokenStream;
|
||||||
import org.apache.lucene.analysis.Tokenizer;
|
import org.apache.lucene.analysis.Tokenizer;
|
||||||
|
import org.elasticsearch.Version;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.env.Environment;
|
import org.elasticsearch.env.Environment;
|
||||||
import org.elasticsearch.index.IndexSettings;
|
import org.elasticsearch.index.IndexSettings;
|
||||||
import org.elasticsearch.index.analysis.AnalyzerProvider;
|
import org.elasticsearch.index.analysis.AnalyzerProvider;
|
||||||
import org.elasticsearch.index.analysis.CharFilterFactory;
|
import org.elasticsearch.index.analysis.CharFilterFactory;
|
||||||
|
import org.elasticsearch.index.analysis.PreConfiguredTokenFilter;
|
||||||
import org.elasticsearch.index.analysis.TokenFilterFactory;
|
import org.elasticsearch.index.analysis.TokenFilterFactory;
|
||||||
import org.elasticsearch.index.analysis.TokenizerFactory;
|
import org.elasticsearch.index.analysis.TokenizerFactory;
|
||||||
import org.elasticsearch.indices.analysis.AnalysisModule.AnalysisProvider;
|
import org.elasticsearch.indices.analysis.AnalysisModule.AnalysisProvider;
|
||||||
|
import org.elasticsearch.indices.analysis.PreBuiltCacheFactory;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.function.BiFunction;
|
||||||
|
|
||||||
|
import static java.util.Collections.emptyList;
|
||||||
import static java.util.Collections.emptyMap;
|
import static java.util.Collections.emptyMap;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -87,6 +94,13 @@ public interface AnalysisPlugin {
|
||||||
return emptyMap();
|
return emptyMap();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Override to add additional pre-configured token filters.
|
||||||
|
*/
|
||||||
|
default List<PreConfiguredTokenFilter> getPreConfiguredTokenFilters() {
|
||||||
|
return emptyList();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Override to add additional hunspell {@link org.apache.lucene.analysis.hunspell.Dictionary}s.
|
* Override to add additional hunspell {@link org.apache.lucene.analysis.hunspell.Dictionary}s.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -86,7 +86,7 @@ public class ScriptService extends AbstractComponent implements Closeable, Clust
|
||||||
public static final Setting<TimeValue> SCRIPT_CACHE_EXPIRE_SETTING =
|
public static final Setting<TimeValue> SCRIPT_CACHE_EXPIRE_SETTING =
|
||||||
Setting.positiveTimeSetting("script.cache.expire", TimeValue.timeValueMillis(0), Property.NodeScope);
|
Setting.positiveTimeSetting("script.cache.expire", TimeValue.timeValueMillis(0), Property.NodeScope);
|
||||||
public static final Setting<Boolean> SCRIPT_AUTO_RELOAD_ENABLED_SETTING =
|
public static final Setting<Boolean> SCRIPT_AUTO_RELOAD_ENABLED_SETTING =
|
||||||
Setting.boolSetting("script.auto_reload_enabled", true, Property.NodeScope);
|
Setting.boolSetting("script.auto_reload_enabled", true, Property.NodeScope, Property.Deprecated);
|
||||||
public static final Setting<Integer> SCRIPT_MAX_SIZE_IN_BYTES =
|
public static final Setting<Integer> SCRIPT_MAX_SIZE_IN_BYTES =
|
||||||
Setting.intSetting("script.max_size_in_bytes", 65535, Property.NodeScope);
|
Setting.intSetting("script.max_size_in_bytes", 65535, Property.NodeScope);
|
||||||
public static final Setting<Integer> SCRIPT_MAX_COMPILATIONS_PER_MINUTE =
|
public static final Setting<Integer> SCRIPT_MAX_COMPILATIONS_PER_MINUTE =
|
||||||
|
@ -162,7 +162,7 @@ public class ScriptService extends AbstractComponent implements Closeable, Clust
|
||||||
FileWatcher fileWatcher = new FileWatcher(scriptsDirectory);
|
FileWatcher fileWatcher = new FileWatcher(scriptsDirectory);
|
||||||
fileWatcher.addListener(new ScriptChangesListener());
|
fileWatcher.addListener(new ScriptChangesListener());
|
||||||
|
|
||||||
if (SCRIPT_AUTO_RELOAD_ENABLED_SETTING.get(settings)) {
|
if (SCRIPT_AUTO_RELOAD_ENABLED_SETTING.get(settings) && resourceWatcherService != null) {
|
||||||
// automatic reload is enabled - register scripts
|
// automatic reload is enabled - register scripts
|
||||||
resourceWatcherService.add(fileWatcher);
|
resourceWatcherService.add(fileWatcher);
|
||||||
} else {
|
} else {
|
||||||
|
|
|
@ -48,8 +48,8 @@ import static java.util.Collections.singletonList;
|
||||||
import static java.util.Collections.singletonMap;
|
import static java.util.Collections.singletonMap;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Tests for {@link TransportAnalyzeAction}. See the more "intense" version of this test in the
|
* Tests for {@link TransportAnalyzeAction}. See the rest tests in the {@code analysis-common} module for places where this code gets a ton
|
||||||
* {@code common-analysis} module.
|
* more exercise.
|
||||||
*/
|
*/
|
||||||
public class TransportAnalyzeActionTests extends ESTestCase {
|
public class TransportAnalyzeActionTests extends ESTestCase {
|
||||||
|
|
||||||
|
@ -90,7 +90,11 @@ public class TransportAnalyzeActionTests extends ESTestCase {
|
||||||
indexAnalyzers = registry.build(idxSettings);
|
indexAnalyzers = registry.build(idxSettings);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test behavior when the named analysis component isn't defined on the index. In that case we should build with defaults.
|
||||||
|
*/
|
||||||
public void testNoIndexAnalyzers() throws IOException {
|
public void testNoIndexAnalyzers() throws IOException {
|
||||||
|
// Refer to an analyzer by its type so we get its default configuration
|
||||||
AnalyzeRequest request = new AnalyzeRequest();
|
AnalyzeRequest request = new AnalyzeRequest();
|
||||||
request.analyzer("standard");
|
request.analyzer("standard");
|
||||||
request.text("the quick brown fox");
|
request.text("the quick brown fox");
|
||||||
|
@ -98,33 +102,30 @@ public class TransportAnalyzeActionTests extends ESTestCase {
|
||||||
List<AnalyzeResponse.AnalyzeToken> tokens = analyze.getTokens();
|
List<AnalyzeResponse.AnalyzeToken> tokens = analyze.getTokens();
|
||||||
assertEquals(4, tokens.size());
|
assertEquals(4, tokens.size());
|
||||||
|
|
||||||
|
// Refer to a token filter by its type so we get its default configuration
|
||||||
request.analyzer(null);
|
request.analyzer(null);
|
||||||
request.tokenizer("whitespace");
|
request.tokenizer("whitespace");
|
||||||
request.addTokenFilter("lowercase");
|
request.addTokenFilter("mock");
|
||||||
request.addTokenFilter("word_delimiter");
|
|
||||||
request.text("the qu1ck brown fox");
|
request.text("the qu1ck brown fox");
|
||||||
analyze = TransportAnalyzeAction.analyze(request, AllFieldMapper.NAME, null, randomBoolean() ? indexAnalyzers : null, registry, environment);
|
analyze = TransportAnalyzeAction.analyze(request, AllFieldMapper.NAME, null, randomBoolean() ? indexAnalyzers : null, registry, environment);
|
||||||
tokens = analyze.getTokens();
|
tokens = analyze.getTokens();
|
||||||
assertEquals(6, tokens.size());
|
assertEquals(3, tokens.size());
|
||||||
assertEquals("qu", tokens.get(1).getTerm());
|
assertEquals("qu1ck", tokens.get(0).getTerm());
|
||||||
assertEquals("1", tokens.get(2).getTerm());
|
assertEquals("brown", tokens.get(1).getTerm());
|
||||||
assertEquals("ck", tokens.get(3).getTerm());
|
assertEquals("fox", tokens.get(2).getTerm());
|
||||||
|
|
||||||
|
// Refer to a char filter by its type so we get its default configuration
|
||||||
request.analyzer(null);
|
request.analyzer(null);
|
||||||
request.tokenizer("whitespace");
|
request.tokenizer("whitespace");
|
||||||
request.addCharFilter("html_strip");
|
request.addCharFilter("html_strip");
|
||||||
request.addTokenFilter("lowercase");
|
request.addTokenFilter("mock");
|
||||||
request.addTokenFilter("word_delimiter");
|
|
||||||
request.text("<p>the qu1ck brown fox</p>");
|
request.text("<p>the qu1ck brown fox</p>");
|
||||||
analyze = TransportAnalyzeAction.analyze(request, AllFieldMapper.NAME, null, randomBoolean() ? indexAnalyzers : null, registry, environment);
|
analyze = TransportAnalyzeAction.analyze(request, AllFieldMapper.NAME, null, randomBoolean() ? indexAnalyzers : null, registry, environment);
|
||||||
tokens = analyze.getTokens();
|
tokens = analyze.getTokens();
|
||||||
assertEquals(6, tokens.size());
|
assertEquals(3, tokens.size());
|
||||||
assertEquals("the", tokens.get(0).getTerm());
|
assertEquals("qu1ck", tokens.get(0).getTerm());
|
||||||
assertEquals("qu", tokens.get(1).getTerm());
|
assertEquals("brown", tokens.get(1).getTerm());
|
||||||
assertEquals("1", tokens.get(2).getTerm());
|
assertEquals("fox", tokens.get(2).getTerm());
|
||||||
assertEquals("ck", tokens.get(3).getTerm());
|
|
||||||
assertEquals("brown", tokens.get(4).getTerm());
|
|
||||||
assertEquals("fox", tokens.get(5).getTerm());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testFillsAttributes() throws IOException {
|
public void testFillsAttributes() throws IOException {
|
||||||
|
|
|
@ -345,12 +345,14 @@ public class ReplicationOperationTests extends ESTestCase {
|
||||||
final ShardRouting routing;
|
final ShardRouting routing;
|
||||||
final long term;
|
final long term;
|
||||||
final long localCheckpoint;
|
final long localCheckpoint;
|
||||||
|
final long globalCheckpoint;
|
||||||
final Map<String, Long> knownLocalCheckpoints = new HashMap<>();
|
final Map<String, Long> knownLocalCheckpoints = new HashMap<>();
|
||||||
|
|
||||||
TestPrimary(ShardRouting routing, long term) {
|
TestPrimary(ShardRouting routing, long term) {
|
||||||
this.routing = routing;
|
this.routing = routing;
|
||||||
this.term = term;
|
this.term = term;
|
||||||
this.localCheckpoint = random().nextLong();
|
this.localCheckpoint = random().nextLong();
|
||||||
|
this.globalCheckpoint = randomNonNegativeLong();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -404,6 +406,12 @@ public class ReplicationOperationTests extends ESTestCase {
|
||||||
public long localCheckpoint() {
|
public long localCheckpoint() {
|
||||||
return localCheckpoint;
|
return localCheckpoint;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long globalCheckpoint() {
|
||||||
|
return globalCheckpoint;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
static class ReplicaResponse implements ReplicationOperation.ReplicaResponse {
|
static class ReplicaResponse implements ReplicationOperation.ReplicaResponse {
|
||||||
|
@ -445,7 +453,11 @@ public class ReplicationOperationTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void performOn(ShardRouting replica, Request request, ActionListener<ReplicationOperation.ReplicaResponse> listener) {
|
public void performOn(
|
||||||
|
final ShardRouting replica,
|
||||||
|
final Request request,
|
||||||
|
final long globalCheckpoint,
|
||||||
|
final ActionListener<ReplicationOperation.ReplicaResponse> listener) {
|
||||||
assertTrue("replica request processed twice on [" + replica + "]", request.processedOnReplicas.add(replica));
|
assertTrue("replica request processed twice on [" + replica + "]", request.processedOnReplicas.add(replica));
|
||||||
if (opFailures.containsKey(replica)) {
|
if (opFailures.containsKey(replica)) {
|
||||||
listener.onFailure(opFailures.get(replica));
|
listener.onFailure(opFailures.get(replica));
|
||||||
|
|
|
@ -608,7 +608,9 @@ public class TransportReplicationActionTests extends ESTestCase {
|
||||||
PlainActionFuture<ReplicaResponse> listener = new PlainActionFuture<>();
|
PlainActionFuture<ReplicaResponse> listener = new PlainActionFuture<>();
|
||||||
proxy.performOn(
|
proxy.performOn(
|
||||||
TestShardRouting.newShardRouting(shardId, "NOT THERE", false, randomFrom(ShardRoutingState.values())),
|
TestShardRouting.newShardRouting(shardId, "NOT THERE", false, randomFrom(ShardRoutingState.values())),
|
||||||
new Request(), listener);
|
new Request(),
|
||||||
|
randomNonNegativeLong(),
|
||||||
|
listener);
|
||||||
assertTrue(listener.isDone());
|
assertTrue(listener.isDone());
|
||||||
assertListenerThrows("non existent node should throw a NoNodeAvailableException", listener, NoNodeAvailableException.class);
|
assertListenerThrows("non existent node should throw a NoNodeAvailableException", listener, NoNodeAvailableException.class);
|
||||||
|
|
||||||
|
@ -616,7 +618,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
||||||
final ShardRouting replica = randomFrom(shardRoutings.replicaShards().stream()
|
final ShardRouting replica = randomFrom(shardRoutings.replicaShards().stream()
|
||||||
.filter(ShardRouting::assignedToNode).collect(Collectors.toList()));
|
.filter(ShardRouting::assignedToNode).collect(Collectors.toList()));
|
||||||
listener = new PlainActionFuture<>();
|
listener = new PlainActionFuture<>();
|
||||||
proxy.performOn(replica, new Request(), listener);
|
proxy.performOn(replica, new Request(), randomNonNegativeLong(), listener);
|
||||||
assertFalse(listener.isDone());
|
assertFalse(listener.isDone());
|
||||||
|
|
||||||
CapturingTransport.CapturedRequest[] captures = transport.getCapturedRequestsAndClear();
|
CapturingTransport.CapturedRequest[] captures = transport.getCapturedRequestsAndClear();
|
||||||
|
@ -767,8 +769,8 @@ public class TransportReplicationActionTests extends ESTestCase {
|
||||||
final TestAction.ReplicaOperationTransportHandler replicaOperationTransportHandler = action.new ReplicaOperationTransportHandler();
|
final TestAction.ReplicaOperationTransportHandler replicaOperationTransportHandler = action.new ReplicaOperationTransportHandler();
|
||||||
try {
|
try {
|
||||||
replicaOperationTransportHandler.messageReceived(
|
replicaOperationTransportHandler.messageReceived(
|
||||||
new TransportReplicationAction.ConcreteShardRequest<>(
|
new TransportReplicationAction.ConcreteReplicaRequest<>(
|
||||||
new Request().setShardId(shardId), replicaRouting.allocationId().getId()),
|
new Request().setShardId(shardId), replicaRouting.allocationId().getId(), randomNonNegativeLong()),
|
||||||
createTransportChannel(new PlainActionFuture<>()), task);
|
createTransportChannel(new PlainActionFuture<>()), task);
|
||||||
} catch (ElasticsearchException e) {
|
} catch (ElasticsearchException e) {
|
||||||
assertThat(e.getMessage(), containsString("simulated"));
|
assertThat(e.getMessage(), containsString("simulated"));
|
||||||
|
@ -800,13 +802,13 @@ public class TransportReplicationActionTests extends ESTestCase {
|
||||||
state = ClusterState.builder(state).metaData(metaDataBuilder).build();
|
state = ClusterState.builder(state).metaData(metaDataBuilder).build();
|
||||||
setState(clusterService, state);
|
setState(clusterService, state);
|
||||||
Request request = new Request(shardId).waitForActiveShards(ActiveShardCount.DEFAULT); // set to default so index settings are used
|
Request request = new Request(shardId).waitForActiveShards(ActiveShardCount.DEFAULT); // set to default so index settings are used
|
||||||
action.resolveRequest(state.metaData(), state.metaData().index(indexName), request);
|
action.resolveRequest(state.metaData().index(indexName), request);
|
||||||
assertEquals(ActiveShardCount.from(idxSettingWaitForActiveShards), request.waitForActiveShards());
|
assertEquals(ActiveShardCount.from(idxSettingWaitForActiveShards), request.waitForActiveShards());
|
||||||
|
|
||||||
// test wait_for_active_shards when default not set on the request (request value should be honored over index setting)
|
// test wait_for_active_shards when default not set on the request (request value should be honored over index setting)
|
||||||
int requestWaitForActiveShards = randomIntBetween(0, numReplicas + 1);
|
int requestWaitForActiveShards = randomIntBetween(0, numReplicas + 1);
|
||||||
request = new Request(shardId).waitForActiveShards(ActiveShardCount.from(requestWaitForActiveShards));
|
request = new Request(shardId).waitForActiveShards(ActiveShardCount.from(requestWaitForActiveShards));
|
||||||
action.resolveRequest(state.metaData(), state.metaData().index(indexName), request);
|
action.resolveRequest(state.metaData().index(indexName), request);
|
||||||
assertEquals(ActiveShardCount.from(requestWaitForActiveShards), request.waitForActiveShards());
|
assertEquals(ActiveShardCount.from(requestWaitForActiveShards), request.waitForActiveShards());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -844,7 +846,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
||||||
PlainActionFuture<TestResponse> listener = new PlainActionFuture<>();
|
PlainActionFuture<TestResponse> listener = new PlainActionFuture<>();
|
||||||
Request request = new Request(shardId).timeout("1ms");
|
Request request = new Request(shardId).timeout("1ms");
|
||||||
action.new ReplicaOperationTransportHandler().messageReceived(
|
action.new ReplicaOperationTransportHandler().messageReceived(
|
||||||
new TransportReplicationAction.ConcreteShardRequest<>(request, "_not_a_valid_aid_"),
|
new TransportReplicationAction.ConcreteReplicaRequest<>(request, "_not_a_valid_aid_", randomNonNegativeLong()),
|
||||||
createTransportChannel(listener), maybeTask()
|
createTransportChannel(listener), maybeTask()
|
||||||
);
|
);
|
||||||
try {
|
try {
|
||||||
|
@ -888,7 +890,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
||||||
final Request request = new Request().setShardId(shardId);
|
final Request request = new Request().setShardId(shardId);
|
||||||
request.primaryTerm(state.metaData().getIndexSafe(shardId.getIndex()).primaryTerm(shardId.id()));
|
request.primaryTerm(state.metaData().getIndexSafe(shardId.getIndex()).primaryTerm(shardId.id()));
|
||||||
replicaOperationTransportHandler.messageReceived(
|
replicaOperationTransportHandler.messageReceived(
|
||||||
new TransportReplicationAction.ConcreteShardRequest<>(request, replica.allocationId().getId()),
|
new TransportReplicationAction.ConcreteReplicaRequest<>(request, replica.allocationId().getId(), randomNonNegativeLong()),
|
||||||
createTransportChannel(listener), task);
|
createTransportChannel(listener), task);
|
||||||
if (listener.isDone()) {
|
if (listener.isDone()) {
|
||||||
listener.get(); // fail with the exception if there
|
listener.get(); // fail with the exception if there
|
||||||
|
|
|
@ -270,8 +270,9 @@ public class TransportWriteActionTests extends ESTestCase {
|
||||||
// check that at unknown node fails
|
// check that at unknown node fails
|
||||||
PlainActionFuture<ReplicaResponse> listener = new PlainActionFuture<>();
|
PlainActionFuture<ReplicaResponse> listener = new PlainActionFuture<>();
|
||||||
proxy.performOn(
|
proxy.performOn(
|
||||||
TestShardRouting.newShardRouting(shardId, "NOT THERE", false, randomFrom(ShardRoutingState.values())),
|
TestShardRouting.newShardRouting(shardId, "NOT THERE", false, randomFrom(ShardRoutingState.values())),
|
||||||
new TestRequest(), listener);
|
new TestRequest(),
|
||||||
|
randomNonNegativeLong(), listener);
|
||||||
assertTrue(listener.isDone());
|
assertTrue(listener.isDone());
|
||||||
assertListenerThrows("non existent node should throw a NoNodeAvailableException", listener, NoNodeAvailableException.class);
|
assertListenerThrows("non existent node should throw a NoNodeAvailableException", listener, NoNodeAvailableException.class);
|
||||||
|
|
||||||
|
@ -279,7 +280,7 @@ public class TransportWriteActionTests extends ESTestCase {
|
||||||
final ShardRouting replica = randomFrom(shardRoutings.replicaShards().stream()
|
final ShardRouting replica = randomFrom(shardRoutings.replicaShards().stream()
|
||||||
.filter(ShardRouting::assignedToNode).collect(Collectors.toList()));
|
.filter(ShardRouting::assignedToNode).collect(Collectors.toList()));
|
||||||
listener = new PlainActionFuture<>();
|
listener = new PlainActionFuture<>();
|
||||||
proxy.performOn(replica, new TestRequest(), listener);
|
proxy.performOn(replica, new TestRequest(), randomNonNegativeLong(), listener);
|
||||||
assertFalse(listener.isDone());
|
assertFalse(listener.isDone());
|
||||||
|
|
||||||
CapturingTransport.CapturedRequest[] captures = transport.getCapturedRequestsAndClear();
|
CapturingTransport.CapturedRequest[] captures = transport.getCapturedRequestsAndClear();
|
||||||
|
|
|
@ -493,7 +493,6 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
|
||||||
.setSettings(Settings.builder()
|
.setSettings(Settings.builder()
|
||||||
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1 + randomInt(2))
|
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1 + randomInt(2))
|
||||||
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, randomInt(2))
|
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, randomInt(2))
|
||||||
.put(IndexSettings.INDEX_SEQ_NO_CHECKPOINT_SYNC_INTERVAL.getKey(), randomBoolean() ? "5s" : "200ms")
|
|
||||||
));
|
));
|
||||||
ensureGreen();
|
ensureGreen();
|
||||||
|
|
||||||
|
|
|
@ -94,6 +94,7 @@ public class IndexModuleTests extends ESTestCase {
|
||||||
private Settings settings;
|
private Settings settings;
|
||||||
private IndexSettings indexSettings;
|
private IndexSettings indexSettings;
|
||||||
private Environment environment;
|
private Environment environment;
|
||||||
|
private AnalysisRegistry emptyAnalysisRegistry;
|
||||||
private NodeEnvironment nodeEnvironment;
|
private NodeEnvironment nodeEnvironment;
|
||||||
private IndicesQueryCache indicesQueryCache;
|
private IndicesQueryCache indicesQueryCache;
|
||||||
|
|
||||||
|
@ -123,6 +124,7 @@ public class IndexModuleTests extends ESTestCase {
|
||||||
indexSettings = IndexSettingsModule.newIndexSettings("foo", settings);
|
indexSettings = IndexSettingsModule.newIndexSettings("foo", settings);
|
||||||
index = indexSettings.getIndex();
|
index = indexSettings.getIndex();
|
||||||
environment = new Environment(settings);
|
environment = new Environment(settings);
|
||||||
|
emptyAnalysisRegistry = new AnalysisRegistry(environment, emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap());
|
||||||
threadPool = new TestThreadPool("test");
|
threadPool = new TestThreadPool("test");
|
||||||
circuitBreakerService = new NoneCircuitBreakerService();
|
circuitBreakerService = new NoneCircuitBreakerService();
|
||||||
bigArrays = new BigArrays(settings, circuitBreakerService);
|
bigArrays = new BigArrays(settings, circuitBreakerService);
|
||||||
|
@ -145,13 +147,12 @@ public class IndexModuleTests extends ESTestCase {
|
||||||
|
|
||||||
private IndexService newIndexService(IndexModule module) throws IOException {
|
private IndexService newIndexService(IndexModule module) throws IOException {
|
||||||
return module.newIndexService(nodeEnvironment, xContentRegistry(), deleter, circuitBreakerService, bigArrays, threadPool,
|
return module.newIndexService(nodeEnvironment, xContentRegistry(), deleter, circuitBreakerService, bigArrays, threadPool,
|
||||||
scriptService, clusterService, null, indicesQueryCache, mapperRegistry, shardId -> {},
|
scriptService, null, indicesQueryCache, mapperRegistry,
|
||||||
new IndicesFieldDataCache(settings, listener));
|
new IndicesFieldDataCache(settings, listener));
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testWrapperIsBound() throws IOException {
|
public void testWrapperIsBound() throws IOException {
|
||||||
IndexModule module = new IndexModule(indexSettings,
|
IndexModule module = new IndexModule(indexSettings, emptyAnalysisRegistry);
|
||||||
new AnalysisRegistry(environment, emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap()));
|
|
||||||
module.setSearcherWrapper((s) -> new Wrapper());
|
module.setSearcherWrapper((s) -> new Wrapper());
|
||||||
module.engineFactory.set(new MockEngineFactory(AssertingDirectoryReader.class));
|
module.engineFactory.set(new MockEngineFactory(AssertingDirectoryReader.class));
|
||||||
|
|
||||||
|
@ -170,8 +171,7 @@ public class IndexModuleTests extends ESTestCase {
|
||||||
.put(IndexModule.INDEX_STORE_TYPE_SETTING.getKey(), "foo_store")
|
.put(IndexModule.INDEX_STORE_TYPE_SETTING.getKey(), "foo_store")
|
||||||
.build();
|
.build();
|
||||||
IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(index, settings);
|
IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(index, settings);
|
||||||
IndexModule module = new IndexModule(indexSettings,
|
IndexModule module = new IndexModule(indexSettings, emptyAnalysisRegistry);
|
||||||
new AnalysisRegistry(environment, emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap()));
|
|
||||||
module.addIndexStore("foo_store", FooStore::new);
|
module.addIndexStore("foo_store", FooStore::new);
|
||||||
try {
|
try {
|
||||||
module.addIndexStore("foo_store", FooStore::new);
|
module.addIndexStore("foo_store", FooStore::new);
|
||||||
|
@ -195,8 +195,7 @@ public class IndexModuleTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(index, settings);
|
IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(index, settings);
|
||||||
IndexModule module = new IndexModule(indexSettings,
|
IndexModule module = new IndexModule(indexSettings, emptyAnalysisRegistry);
|
||||||
new AnalysisRegistry(environment, emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap()));
|
|
||||||
module.addIndexEventListener(eventListener);
|
module.addIndexEventListener(eventListener);
|
||||||
IndexService indexService = newIndexService(module);
|
IndexService indexService = newIndexService(module);
|
||||||
IndexSettings x = indexService.getIndexSettings();
|
IndexSettings x = indexService.getIndexSettings();
|
||||||
|
@ -210,8 +209,7 @@ public class IndexModuleTests extends ESTestCase {
|
||||||
|
|
||||||
public void testListener() throws IOException {
|
public void testListener() throws IOException {
|
||||||
Setting<Boolean> booleanSetting = Setting.boolSetting("index.foo.bar", false, Property.Dynamic, Property.IndexScope);
|
Setting<Boolean> booleanSetting = Setting.boolSetting("index.foo.bar", false, Property.Dynamic, Property.IndexScope);
|
||||||
IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(index, settings, booleanSetting),
|
IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(index, settings, booleanSetting), emptyAnalysisRegistry);
|
||||||
new AnalysisRegistry(environment, emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap()));
|
|
||||||
Setting<Boolean> booleanSetting2 = Setting.boolSetting("index.foo.bar.baz", false, Property.Dynamic, Property.IndexScope);
|
Setting<Boolean> booleanSetting2 = Setting.boolSetting("index.foo.bar.baz", false, Property.Dynamic, Property.IndexScope);
|
||||||
AtomicBoolean atomicBoolean = new AtomicBoolean(false);
|
AtomicBoolean atomicBoolean = new AtomicBoolean(false);
|
||||||
module.addSettingsUpdateConsumer(booleanSetting, atomicBoolean::set);
|
module.addSettingsUpdateConsumer(booleanSetting, atomicBoolean::set);
|
||||||
|
@ -230,8 +228,7 @@ public class IndexModuleTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testAddIndexOperationListener() throws IOException {
|
public void testAddIndexOperationListener() throws IOException {
|
||||||
IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(index, settings),
|
IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(index, settings), emptyAnalysisRegistry);
|
||||||
new AnalysisRegistry(environment, emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap()));
|
|
||||||
AtomicBoolean executed = new AtomicBoolean(false);
|
AtomicBoolean executed = new AtomicBoolean(false);
|
||||||
IndexingOperationListener listener = new IndexingOperationListener() {
|
IndexingOperationListener listener = new IndexingOperationListener() {
|
||||||
@Override
|
@Override
|
||||||
|
@ -261,8 +258,7 @@ public class IndexModuleTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testAddSearchOperationListener() throws IOException {
|
public void testAddSearchOperationListener() throws IOException {
|
||||||
IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(index, settings),
|
IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(index, settings), emptyAnalysisRegistry);
|
||||||
new AnalysisRegistry(environment, emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap()));
|
|
||||||
AtomicBoolean executed = new AtomicBoolean(false);
|
AtomicBoolean executed = new AtomicBoolean(false);
|
||||||
SearchOperationListener listener = new SearchOperationListener() {
|
SearchOperationListener listener = new SearchOperationListener() {
|
||||||
|
|
||||||
|
@ -295,8 +291,7 @@ public class IndexModuleTests extends ESTestCase {
|
||||||
.put("index.similarity.my_similarity.key", "there is a key")
|
.put("index.similarity.my_similarity.key", "there is a key")
|
||||||
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString())
|
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString())
|
||||||
.build();
|
.build();
|
||||||
IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings("foo", indexSettings),
|
IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings("foo", indexSettings), emptyAnalysisRegistry);
|
||||||
new AnalysisRegistry(environment, emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap()));
|
|
||||||
module.addSimilarity("test_similarity", (string, providerSettings, indexLevelSettings) -> new SimilarityProvider() {
|
module.addSimilarity("test_similarity", (string, providerSettings, indexLevelSettings) -> new SimilarityProvider() {
|
||||||
@Override
|
@Override
|
||||||
public String name() {
|
public String name() {
|
||||||
|
@ -319,8 +314,7 @@ public class IndexModuleTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testFrozen() {
|
public void testFrozen() {
|
||||||
IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(index, settings),
|
IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(index, settings), emptyAnalysisRegistry);
|
||||||
new AnalysisRegistry(environment, emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap()));
|
|
||||||
module.freeze();
|
module.freeze();
|
||||||
String msg = "Can't modify IndexModule once the index service has been created";
|
String msg = "Can't modify IndexModule once the index service has been created";
|
||||||
assertEquals(msg, expectThrows(IllegalStateException.class, () -> module.addSearchOperationListener(null)).getMessage());
|
assertEquals(msg, expectThrows(IllegalStateException.class, () -> module.addSearchOperationListener(null)).getMessage());
|
||||||
|
@ -338,8 +332,7 @@ public class IndexModuleTests extends ESTestCase {
|
||||||
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
|
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
|
||||||
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString())
|
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString())
|
||||||
.build();
|
.build();
|
||||||
IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings("foo", indexSettings),
|
IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings("foo", indexSettings), emptyAnalysisRegistry);
|
||||||
new AnalysisRegistry(environment, emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap()));
|
|
||||||
Exception ex = expectThrows(IllegalArgumentException.class, () -> newIndexService(module));
|
Exception ex = expectThrows(IllegalArgumentException.class, () -> newIndexService(module));
|
||||||
assertEquals("Unknown Similarity type [test_similarity] for [my_similarity]", ex.getMessage());
|
assertEquals("Unknown Similarity type [test_similarity] for [my_similarity]", ex.getMessage());
|
||||||
}
|
}
|
||||||
|
@ -350,8 +343,7 @@ public class IndexModuleTests extends ESTestCase {
|
||||||
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString())
|
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString())
|
||||||
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
|
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
|
||||||
.build();
|
.build();
|
||||||
IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings("foo", indexSettings),
|
IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings("foo", indexSettings), emptyAnalysisRegistry);
|
||||||
new AnalysisRegistry(environment, emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap()));
|
|
||||||
Exception ex = expectThrows(IllegalArgumentException.class, () -> newIndexService(module));
|
Exception ex = expectThrows(IllegalArgumentException.class, () -> newIndexService(module));
|
||||||
assertEquals("Similarity [my_similarity] must have an associated type", ex.getMessage());
|
assertEquals("Similarity [my_similarity] must have an associated type", ex.getMessage());
|
||||||
}
|
}
|
||||||
|
@ -360,8 +352,7 @@ public class IndexModuleTests extends ESTestCase {
|
||||||
Settings indexSettings = Settings.builder()
|
Settings indexSettings = Settings.builder()
|
||||||
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString())
|
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString())
|
||||||
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
|
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
|
||||||
IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings("foo", indexSettings),
|
IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings("foo", indexSettings), emptyAnalysisRegistry);
|
||||||
new AnalysisRegistry(environment, emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap()));
|
|
||||||
module.forceQueryCacheProvider((a, b) -> new CustomQueryCache());
|
module.forceQueryCacheProvider((a, b) -> new CustomQueryCache());
|
||||||
expectThrows(AlreadySetException.class, () -> module.forceQueryCacheProvider((a, b) -> new CustomQueryCache()));
|
expectThrows(AlreadySetException.class, () -> module.forceQueryCacheProvider((a, b) -> new CustomQueryCache()));
|
||||||
IndexService indexService = newIndexService(module);
|
IndexService indexService = newIndexService(module);
|
||||||
|
@ -373,8 +364,7 @@ public class IndexModuleTests extends ESTestCase {
|
||||||
Settings indexSettings = Settings.builder()
|
Settings indexSettings = Settings.builder()
|
||||||
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString())
|
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString())
|
||||||
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
|
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
|
||||||
IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings("foo", indexSettings),
|
IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings("foo", indexSettings), emptyAnalysisRegistry);
|
||||||
new AnalysisRegistry(environment, emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap()));
|
|
||||||
IndexService indexService = newIndexService(module);
|
IndexService indexService = newIndexService(module);
|
||||||
assertTrue(indexService.cache().query() instanceof IndexQueryCache);
|
assertTrue(indexService.cache().query() instanceof IndexQueryCache);
|
||||||
indexService.close("simon says", false);
|
indexService.close("simon says", false);
|
||||||
|
@ -385,8 +375,7 @@ public class IndexModuleTests extends ESTestCase {
|
||||||
.put(IndexModule.INDEX_QUERY_CACHE_ENABLED_SETTING.getKey(), false)
|
.put(IndexModule.INDEX_QUERY_CACHE_ENABLED_SETTING.getKey(), false)
|
||||||
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString())
|
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString())
|
||||||
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
|
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
|
||||||
IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings("foo", indexSettings),
|
IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings("foo", indexSettings), emptyAnalysisRegistry);
|
||||||
new AnalysisRegistry(environment, emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap()));
|
|
||||||
module.forceQueryCacheProvider((a, b) -> new CustomQueryCache());
|
module.forceQueryCacheProvider((a, b) -> new CustomQueryCache());
|
||||||
IndexService indexService = newIndexService(module);
|
IndexService indexService = newIndexService(module);
|
||||||
assertTrue(indexService.cache().query() instanceof DisabledQueryCache);
|
assertTrue(indexService.cache().query() instanceof DisabledQueryCache);
|
||||||
|
|
|
@ -179,20 +179,6 @@ public class IndexServiceTests extends ESSingleNodeTestCase {
|
||||||
assertNull(indexService.getFsyncTask());
|
assertNull(indexService.getFsyncTask());
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testGlobalCheckpointTaskIsRunning() throws IOException {
|
|
||||||
IndexService indexService = createIndex("test", Settings.EMPTY);
|
|
||||||
IndexService.AsyncGlobalCheckpointTask task = indexService.getGlobalCheckpointTask();
|
|
||||||
assertNotNull(task);
|
|
||||||
assertEquals(IndexSettings.INDEX_SEQ_NO_CHECKPOINT_SYNC_INTERVAL.getDefault(Settings.EMPTY), task.getInterval());
|
|
||||||
assertTrue(task.mustReschedule());
|
|
||||||
assertTrue(task.isScheduled());
|
|
||||||
|
|
||||||
indexService.close("simon says", false);
|
|
||||||
assertFalse(task.isScheduled());
|
|
||||||
assertTrue(task.isClosed());
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
public void testRefreshActuallyWorks() throws Exception {
|
public void testRefreshActuallyWorks() throws Exception {
|
||||||
IndexService indexService = createIndex("test", Settings.EMPTY);
|
IndexService indexService = createIndex("test", Settings.EMPTY);
|
||||||
ensureGreen("test");
|
ensureGreen("test");
|
||||||
|
|
|
@ -1,26 +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.analysis;
|
|
||||||
|
|
||||||
import org.elasticsearch.AnalysisFactoryTestCase;
|
|
||||||
|
|
||||||
public class AnalysisFactoryTests extends AnalysisFactoryTestCase {
|
|
||||||
// tests are inherited and nothing needs to be defined here
|
|
||||||
}
|
|
|
@ -34,6 +34,7 @@ import org.elasticsearch.index.IndexSettings;
|
||||||
import org.elasticsearch.indices.analysis.AnalysisModule;
|
import org.elasticsearch.indices.analysis.AnalysisModule;
|
||||||
import org.elasticsearch.indices.analysis.AnalysisModule.AnalysisProvider;
|
import org.elasticsearch.indices.analysis.AnalysisModule.AnalysisProvider;
|
||||||
import org.elasticsearch.indices.analysis.PreBuiltAnalyzers;
|
import org.elasticsearch.indices.analysis.PreBuiltAnalyzers;
|
||||||
|
import org.elasticsearch.indices.analysis.PreBuiltCacheFactory;
|
||||||
import org.elasticsearch.plugins.AnalysisPlugin;
|
import org.elasticsearch.plugins.AnalysisPlugin;
|
||||||
import org.elasticsearch.test.ESTestCase;
|
import org.elasticsearch.test.ESTestCase;
|
||||||
import org.elasticsearch.test.IndexSettingsModule;
|
import org.elasticsearch.test.IndexSettingsModule;
|
||||||
|
@ -41,6 +42,7 @@ import org.elasticsearch.test.VersionUtils;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
|
|
||||||
import static java.util.Collections.emptyMap;
|
import static java.util.Collections.emptyMap;
|
||||||
import static java.util.Collections.singletonList;
|
import static java.util.Collections.singletonList;
|
||||||
|
@ -50,7 +52,9 @@ import static org.hamcrest.Matchers.instanceOf;
|
||||||
|
|
||||||
public class AnalysisRegistryTests extends ESTestCase {
|
public class AnalysisRegistryTests extends ESTestCase {
|
||||||
|
|
||||||
private AnalysisRegistry registry;
|
private Environment emptyEnvironment;
|
||||||
|
private AnalysisRegistry emptyRegistry;
|
||||||
|
private IndexSettings emptyIndexSettingsOfCurrentVersion;
|
||||||
|
|
||||||
private static AnalyzerProvider<?> analyzerProvider(final String name) {
|
private static AnalyzerProvider<?> analyzerProvider(final String name) {
|
||||||
return new PreBuiltAnalyzerProvider(name, AnalyzerScope.INDEX, new EnglishAnalyzer());
|
return new PreBuiltAnalyzerProvider(name, AnalyzerScope.INDEX, new EnglishAnalyzer());
|
||||||
|
@ -59,12 +63,13 @@ public class AnalysisRegistryTests extends ESTestCase {
|
||||||
@Override
|
@Override
|
||||||
public void setUp() throws Exception {
|
public void setUp() throws Exception {
|
||||||
super.setUp();
|
super.setUp();
|
||||||
Settings settings = Settings
|
emptyEnvironment = new Environment(Settings.builder()
|
||||||
.builder()
|
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString())
|
||||||
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString())
|
.build());
|
||||||
.build();
|
emptyRegistry = new AnalysisRegistry(emptyEnvironment, emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap());
|
||||||
registry = new AnalysisRegistry(new Environment(settings),
|
emptyIndexSettingsOfCurrentVersion = IndexSettingsModule.newIndexSettings("index", Settings.builder()
|
||||||
emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap());
|
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
|
||||||
|
.build());
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testDefaultAnalyzers() throws IOException {
|
public void testDefaultAnalyzers() throws IOException {
|
||||||
|
@ -75,9 +80,7 @@ public class AnalysisRegistryTests extends ESTestCase {
|
||||||
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString())
|
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString())
|
||||||
.build();
|
.build();
|
||||||
IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("index", settings);
|
IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("index", settings);
|
||||||
IndexAnalyzers indexAnalyzers = new AnalysisRegistry(new Environment(settings),
|
IndexAnalyzers indexAnalyzers = emptyRegistry.build(idxSettings);
|
||||||
emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap())
|
|
||||||
.build(idxSettings);
|
|
||||||
assertThat(indexAnalyzers.getDefaultIndexAnalyzer().analyzer(), instanceOf(StandardAnalyzer.class));
|
assertThat(indexAnalyzers.getDefaultIndexAnalyzer().analyzer(), instanceOf(StandardAnalyzer.class));
|
||||||
assertThat(indexAnalyzers.getDefaultSearchAnalyzer().analyzer(), instanceOf(StandardAnalyzer.class));
|
assertThat(indexAnalyzers.getDefaultSearchAnalyzer().analyzer(), instanceOf(StandardAnalyzer.class));
|
||||||
assertThat(indexAnalyzers.getDefaultSearchQuoteAnalyzer().analyzer(), instanceOf(StandardAnalyzer.class));
|
assertThat(indexAnalyzers.getDefaultSearchQuoteAnalyzer().analyzer(), instanceOf(StandardAnalyzer.class));
|
||||||
|
@ -86,7 +89,7 @@ public class AnalysisRegistryTests extends ESTestCase {
|
||||||
public void testOverrideDefaultAnalyzer() throws IOException {
|
public void testOverrideDefaultAnalyzer() throws IOException {
|
||||||
Version version = VersionUtils.randomVersion(random());
|
Version version = VersionUtils.randomVersion(random());
|
||||||
Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
|
Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
|
||||||
IndexAnalyzers indexAnalyzers = registry.build(IndexSettingsModule.newIndexSettings("index", settings),
|
IndexAnalyzers indexAnalyzers = emptyRegistry.build(IndexSettingsModule.newIndexSettings("index", settings),
|
||||||
singletonMap("default", analyzerProvider("default"))
|
singletonMap("default", analyzerProvider("default"))
|
||||||
, emptyMap(), emptyMap(), emptyMap(), emptyMap());
|
, emptyMap(), emptyMap(), emptyMap(), emptyMap());
|
||||||
assertThat(indexAnalyzers.getDefaultIndexAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class));
|
assertThat(indexAnalyzers.getDefaultIndexAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class));
|
||||||
|
@ -99,7 +102,7 @@ public class AnalysisRegistryTests extends ESTestCase {
|
||||||
Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
|
Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
|
||||||
AnalyzerProvider<?> defaultIndex = new PreBuiltAnalyzerProvider("default_index", AnalyzerScope.INDEX, new EnglishAnalyzer());
|
AnalyzerProvider<?> defaultIndex = new PreBuiltAnalyzerProvider("default_index", AnalyzerScope.INDEX, new EnglishAnalyzer());
|
||||||
IllegalArgumentException e = expectThrows(IllegalArgumentException.class,
|
IllegalArgumentException e = expectThrows(IllegalArgumentException.class,
|
||||||
() -> registry.build(IndexSettingsModule.newIndexSettings("index", settings),
|
() -> emptyRegistry.build(IndexSettingsModule.newIndexSettings("index", settings),
|
||||||
singletonMap("default_index", defaultIndex), emptyMap(), emptyMap(), emptyMap(), emptyMap()));
|
singletonMap("default_index", defaultIndex), emptyMap(), emptyMap(), emptyMap(), emptyMap()));
|
||||||
assertTrue(e.getMessage().contains("[index.analysis.analyzer.default_index] is not supported"));
|
assertTrue(e.getMessage().contains("[index.analysis.analyzer.default_index] is not supported"));
|
||||||
}
|
}
|
||||||
|
@ -107,7 +110,7 @@ public class AnalysisRegistryTests extends ESTestCase {
|
||||||
public void testOverrideDefaultSearchAnalyzer() {
|
public void testOverrideDefaultSearchAnalyzer() {
|
||||||
Version version = VersionUtils.randomVersion(random());
|
Version version = VersionUtils.randomVersion(random());
|
||||||
Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
|
Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
|
||||||
IndexAnalyzers indexAnalyzers = registry.build(IndexSettingsModule.newIndexSettings("index", settings),
|
IndexAnalyzers indexAnalyzers = emptyRegistry.build(IndexSettingsModule.newIndexSettings("index", settings),
|
||||||
singletonMap("default_search", analyzerProvider("default_search")), emptyMap(), emptyMap(), emptyMap(), emptyMap());
|
singletonMap("default_search", analyzerProvider("default_search")), emptyMap(), emptyMap(), emptyMap(), emptyMap());
|
||||||
assertThat(indexAnalyzers.getDefaultIndexAnalyzer().analyzer(), instanceOf(StandardAnalyzer.class));
|
assertThat(indexAnalyzers.getDefaultIndexAnalyzer().analyzer(), instanceOf(StandardAnalyzer.class));
|
||||||
assertThat(indexAnalyzers.getDefaultSearchAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class));
|
assertThat(indexAnalyzers.getDefaultSearchAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class));
|
||||||
|
@ -189,11 +192,12 @@ public class AnalysisRegistryTests extends ESTestCase {
|
||||||
Settings indexSettings = Settings.builder()
|
Settings indexSettings = Settings.builder()
|
||||||
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
|
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
|
||||||
IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("index", indexSettings);
|
IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("index", indexSettings);
|
||||||
IndexAnalyzers indexAnalyzers = new AnalysisRegistry(new Environment(settings),
|
IndexAnalyzers indexAnalyzers =
|
||||||
emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap())
|
new AnalysisRegistry(new Environment(settings), emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap())
|
||||||
.build(idxSettings);
|
.build(idxSettings);
|
||||||
IndexAnalyzers otherIndexAnalyzers = new AnalysisRegistry(new Environment(settings), emptyMap(), emptyMap(), emptyMap(),
|
IndexAnalyzers otherIndexAnalyzers =
|
||||||
emptyMap(), emptyMap()).build(idxSettings);
|
new AnalysisRegistry(new Environment(settings), emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap())
|
||||||
|
.build(idxSettings);
|
||||||
final int numIters = randomIntBetween(5, 20);
|
final int numIters = randomIntBetween(5, 20);
|
||||||
for (int i = 0; i < numIters; i++) {
|
for (int i = 0; i < numIters; i++) {
|
||||||
PreBuiltAnalyzers preBuiltAnalyzers = RandomPicks.randomFrom(random(), PreBuiltAnalyzers.values());
|
PreBuiltAnalyzers preBuiltAnalyzers = RandomPicks.randomFrom(random(), PreBuiltAnalyzers.values());
|
||||||
|
@ -201,6 +205,23 @@ public class AnalysisRegistryTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void testPreConfiguredTokenFiltersAreCached() throws IOException {
|
||||||
|
AtomicBoolean built = new AtomicBoolean(false);
|
||||||
|
PreConfiguredTokenFilter assertsBuiltOnce = new PreConfiguredTokenFilter("asserts_built_once", false,
|
||||||
|
PreBuiltCacheFactory.CachingStrategy.ONE, (tokens, version) -> {
|
||||||
|
if (false == built.compareAndSet(false, true)) {
|
||||||
|
fail("Attempted to build the token filter twice when it should have been cached");
|
||||||
|
}
|
||||||
|
return new MockTokenFilter(tokens, MockTokenFilter.EMPTY_STOPSET);
|
||||||
|
});
|
||||||
|
try (AnalysisRegistry registryWithPreBuiltTokenFilter = new AnalysisRegistry(emptyEnvironment, emptyMap(), emptyMap(), emptyMap(),
|
||||||
|
emptyMap(), emptyMap(), singletonMap("asserts_built_once", assertsBuiltOnce))) {
|
||||||
|
IndexAnalyzers indexAnalyzers = registryWithPreBuiltTokenFilter.build(emptyIndexSettingsOfCurrentVersion);
|
||||||
|
IndexAnalyzers otherIndexAnalyzers = registryWithPreBuiltTokenFilter.build(emptyIndexSettingsOfCurrentVersion);
|
||||||
|
assertSame(indexAnalyzers.get("asserts_built_once"), otherIndexAnalyzers.get("asserts_built_once"));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public void testNoTypeOrTokenizerErrorMessage() throws IOException {
|
public void testNoTypeOrTokenizerErrorMessage() throws IOException {
|
||||||
Version version = VersionUtils.randomVersion(random());
|
Version version = VersionUtils.randomVersion(random());
|
||||||
Settings settings = Settings
|
Settings settings = Settings
|
||||||
|
@ -212,20 +233,14 @@ public class AnalysisRegistryTests extends ESTestCase {
|
||||||
.build();
|
.build();
|
||||||
IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("index", settings);
|
IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("index", settings);
|
||||||
|
|
||||||
IllegalArgumentException e = expectThrows(IllegalArgumentException.class,
|
IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () ->
|
||||||
() -> new AnalysisRegistry(new Environment(settings),
|
new AnalysisRegistry(new Environment(settings), emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap())
|
||||||
emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap()).build(idxSettings));
|
.build(idxSettings));
|
||||||
assertThat(e.getMessage(), equalTo("analyzer [test_analyzer] must specify either an analyzer type, or a tokenizer"));
|
assertThat(e.getMessage(), equalTo("analyzer [test_analyzer] must specify either an analyzer type, or a tokenizer"));
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testCloseIndexAnalyzersMultipleTimes() throws IOException {
|
public void testCloseIndexAnalyzersMultipleTimes() throws IOException {
|
||||||
Settings settings = Settings.builder().put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()).build();
|
IndexAnalyzers indexAnalyzers = emptyRegistry.build(emptyIndexSettingsOfCurrentVersion);
|
||||||
Settings indexSettings = Settings.builder()
|
|
||||||
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
|
|
||||||
IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("index", indexSettings);
|
|
||||||
IndexAnalyzers indexAnalyzers = new AnalysisRegistry(new Environment(settings),
|
|
||||||
emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap())
|
|
||||||
.build(idxSettings);
|
|
||||||
indexAnalyzers.close();
|
indexAnalyzers.close();
|
||||||
indexAnalyzers.close();
|
indexAnalyzers.close();
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,37 @@
|
||||||
|
/*
|
||||||
|
* 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.analysis;
|
||||||
|
|
||||||
|
import org.elasticsearch.indices.analysis.AnalysisFactoryTestCase;
|
||||||
|
import org.elasticsearch.plugins.AnalysisPlugin;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Checks on the analysis components that are part of core to make sure that any that are added
|
||||||
|
* to lucene are either enabled or explicitly not enabled. During the migration of analysis
|
||||||
|
* components to the {@code analysis-common} module this test ignores many components that are
|
||||||
|
* available to es-core but mapping in {@code analysis-common}. When the migration is complete
|
||||||
|
* no such ignoring will be needed because the analysis components won't be available to core.
|
||||||
|
*/
|
||||||
|
public class CoreAnalysisFactoryTests extends AnalysisFactoryTestCase {
|
||||||
|
public CoreAnalysisFactoryTests() {
|
||||||
|
// Use an empty plugin that doesn't define anything so the test doesn't need a ton of null checks.
|
||||||
|
super(new AnalysisPlugin() {});
|
||||||
|
}
|
||||||
|
}
|
|
@ -19,33 +19,39 @@
|
||||||
|
|
||||||
package org.elasticsearch.index.analysis;
|
package org.elasticsearch.index.analysis;
|
||||||
|
|
||||||
|
import org.apache.lucene.analysis.MockLowerCaseFilter;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.env.Environment;
|
import org.elasticsearch.env.Environment;
|
||||||
import org.elasticsearch.indices.analysis.AnalysisModule.AnalysisProvider;
|
import org.elasticsearch.indices.analysis.AnalysisModule.AnalysisProvider;
|
||||||
|
import org.elasticsearch.indices.analysis.PreBuiltCacheFactory.CachingStrategy;
|
||||||
import org.elasticsearch.plugins.AnalysisPlugin;
|
import org.elasticsearch.plugins.AnalysisPlugin;
|
||||||
import org.elasticsearch.test.ESTestCase;
|
import org.elasticsearch.test.ESTestCase;
|
||||||
import org.elasticsearch.test.ESTokenStreamTestCase;
|
import org.elasticsearch.test.ESTokenStreamTestCase;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.Reader;
|
import java.io.Reader;
|
||||||
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
|
import static java.util.Collections.singletonList;
|
||||||
import static java.util.Collections.singletonMap;
|
import static java.util.Collections.singletonMap;
|
||||||
|
|
||||||
public class CustomNormalizerTests extends ESTokenStreamTestCase {
|
public class CustomNormalizerTests extends ESTokenStreamTestCase {
|
||||||
|
private static final AnalysisPlugin MOCK_ANALYSIS_PLUGIN = new MockAnalysisPlugin();
|
||||||
|
|
||||||
public void testBasics() throws IOException {
|
public void testBasics() throws IOException {
|
||||||
Settings settings = Settings.builder()
|
Settings settings = Settings.builder()
|
||||||
.putArray("index.analysis.normalizer.my_normalizer.filter", "lowercase", "asciifolding")
|
.putArray("index.analysis.normalizer.my_normalizer.filter", "lowercase")
|
||||||
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString())
|
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString())
|
||||||
.build();
|
.build();
|
||||||
ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(settings);
|
ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(settings, MOCK_ANALYSIS_PLUGIN);
|
||||||
assertNull(analysis.indexAnalyzers.get("my_normalizer"));
|
assertNull(analysis.indexAnalyzers.get("my_normalizer"));
|
||||||
NamedAnalyzer normalizer = analysis.indexAnalyzers.getNormalizer("my_normalizer");
|
NamedAnalyzer normalizer = analysis.indexAnalyzers.getNormalizer("my_normalizer");
|
||||||
assertNotNull(normalizer);
|
assertNotNull(normalizer);
|
||||||
assertEquals("my_normalizer", normalizer.name());
|
assertEquals("my_normalizer", normalizer.name());
|
||||||
assertTokenStreamContents(normalizer.tokenStream("foo", "Cet été-là"), new String[] {"cet ete-la"});
|
assertTokenStreamContents(normalizer.tokenStream("foo", "Cet été-là"), new String[] {"cet été-là"});
|
||||||
assertEquals(new BytesRef("cet ete-la"), normalizer.normalize("foo", "Cet été-là"));
|
assertEquals(new BytesRef("cet été-là"), normalizer.normalize("foo", "Cet été-là"));
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testUnknownType() {
|
public void testUnknownType() {
|
||||||
|
@ -75,7 +81,7 @@ public class CustomNormalizerTests extends ESTokenStreamTestCase {
|
||||||
.putArray("index.analysis.normalizer.my_normalizer.char_filter", "my_mapping")
|
.putArray("index.analysis.normalizer.my_normalizer.char_filter", "my_mapping")
|
||||||
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString())
|
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString())
|
||||||
.build();
|
.build();
|
||||||
ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(settings, new MockCharFilterPlugin());
|
ESTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings(settings, MOCK_ANALYSIS_PLUGIN);
|
||||||
assertNull(analysis.indexAnalyzers.get("my_normalizer"));
|
assertNull(analysis.indexAnalyzers.get("my_normalizer"));
|
||||||
NamedAnalyzer normalizer = analysis.indexAnalyzers.getNormalizer("my_normalizer");
|
NamedAnalyzer normalizer = analysis.indexAnalyzers.getNormalizer("my_normalizer");
|
||||||
assertNotNull(normalizer);
|
assertNotNull(normalizer);
|
||||||
|
@ -86,12 +92,12 @@ public class CustomNormalizerTests extends ESTokenStreamTestCase {
|
||||||
|
|
||||||
public void testIllegalFilters() throws IOException {
|
public void testIllegalFilters() throws IOException {
|
||||||
Settings settings = Settings.builder()
|
Settings settings = Settings.builder()
|
||||||
.putArray("index.analysis.normalizer.my_normalizer.filter", "porter_stem")
|
.putArray("index.analysis.normalizer.my_normalizer.filter", "mock_forbidden")
|
||||||
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString())
|
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString())
|
||||||
.build();
|
.build();
|
||||||
IllegalArgumentException e = expectThrows(IllegalArgumentException.class,
|
IllegalArgumentException e = expectThrows(IllegalArgumentException.class,
|
||||||
() -> AnalysisTestsHelper.createTestAnalysisFromSettings(settings));
|
() -> AnalysisTestsHelper.createTestAnalysisFromSettings(settings, MOCK_ANALYSIS_PLUGIN));
|
||||||
assertEquals("Custom normalizer [my_normalizer] may not use filter [porter_stem]", e.getMessage());
|
assertEquals("Custom normalizer [my_normalizer] may not use filter [mock_forbidden]", e.getMessage());
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testIllegalCharFilters() throws IOException {
|
public void testIllegalCharFilters() throws IOException {
|
||||||
|
@ -104,7 +110,12 @@ public class CustomNormalizerTests extends ESTokenStreamTestCase {
|
||||||
assertEquals("Custom normalizer [my_normalizer] may not use char filter [html_strip]", e.getMessage());
|
assertEquals("Custom normalizer [my_normalizer] may not use char filter [html_strip]", e.getMessage());
|
||||||
}
|
}
|
||||||
|
|
||||||
private class MockCharFilterPlugin implements AnalysisPlugin {
|
private static class MockAnalysisPlugin implements AnalysisPlugin {
|
||||||
|
@Override
|
||||||
|
public List<PreConfiguredTokenFilter> getPreConfiguredTokenFilters() {
|
||||||
|
return singletonList(new PreConfiguredTokenFilter("mock_forbidden", false, CachingStrategy.ONE, MockLowerCaseFilter::new));
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Map<String, AnalysisProvider<CharFilterFactory>> getCharFilters() {
|
public Map<String, AnalysisProvider<CharFilterFactory>> getCharFilters() {
|
||||||
return singletonMap("mock_char_filter", (indexSettings, env, name, settings) -> {
|
return singletonMap("mock_char_filter", (indexSettings, env, name, settings) -> {
|
||||||
|
@ -116,22 +127,21 @@ public class CustomNormalizerTests extends ESTokenStreamTestCase {
|
||||||
@Override
|
@Override
|
||||||
public Reader create(Reader reader) {
|
public Reader create(Reader reader) {
|
||||||
return new Reader() {
|
return new Reader() {
|
||||||
|
@Override
|
||||||
|
public int read(char[] cbuf, int off, int len) throws IOException {
|
||||||
|
int result = reader.read(cbuf, off, len);
|
||||||
|
for (int i = off; i < result; i++) {
|
||||||
|
if (cbuf[i] == 'a') {
|
||||||
|
cbuf[i] = 'z';
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int read(char[] cbuf, int off, int len) throws IOException {
|
public void close() throws IOException {
|
||||||
int result = reader.read(cbuf, off, len);
|
reader.close();
|
||||||
for (int i = off; i < result; i++) {
|
}
|
||||||
if (cbuf[i] == 'a') {
|
|
||||||
cbuf[i] = 'z';
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void close() throws IOException {
|
|
||||||
reader.close();
|
|
||||||
}
|
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -2071,13 +2071,11 @@ public class InternalEngineTests extends ESTestCase {
|
||||||
if (rarely()) {
|
if (rarely()) {
|
||||||
localCheckpoint = primarySeqNo;
|
localCheckpoint = primarySeqNo;
|
||||||
maxSeqNo = primarySeqNo;
|
maxSeqNo = primarySeqNo;
|
||||||
initialEngine.seqNoService().updateGlobalCheckpointOnPrimary();
|
|
||||||
initialEngine.flush(true, true);
|
initialEngine.flush(true, true);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
logger.info("localcheckpoint {}, global {}", replicaLocalCheckpoint, primarySeqNo);
|
logger.info("localcheckpoint {}, global {}", replicaLocalCheckpoint, primarySeqNo);
|
||||||
initialEngine.seqNoService().updateGlobalCheckpointOnPrimary();
|
|
||||||
globalCheckpoint = initialEngine.seqNoService().getGlobalCheckpoint();
|
globalCheckpoint = initialEngine.seqNoService().getGlobalCheckpoint();
|
||||||
|
|
||||||
assertEquals(primarySeqNo, initialEngine.seqNoService().getMaxSeqNo());
|
assertEquals(primarySeqNo, initialEngine.seqNoService().getMaxSeqNo());
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package org.elasticsearch.index.mapper;
|
package org.elasticsearch.index.mapper;
|
||||||
|
|
||||||
|
import org.apache.lucene.analysis.MockLowerCaseFilter;
|
||||||
import org.apache.lucene.index.DocValuesType;
|
import org.apache.lucene.index.DocValuesType;
|
||||||
import org.apache.lucene.index.IndexOptions;
|
import org.apache.lucene.index.IndexOptions;
|
||||||
import org.apache.lucene.index.IndexableField;
|
import org.apache.lucene.index.IndexableField;
|
||||||
|
@ -29,7 +30,10 @@ import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.xcontent.XContentFactory;
|
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||||
import org.elasticsearch.common.xcontent.XContentType;
|
import org.elasticsearch.common.xcontent.XContentType;
|
||||||
import org.elasticsearch.index.IndexService;
|
import org.elasticsearch.index.IndexService;
|
||||||
|
import org.elasticsearch.index.analysis.PreConfiguredTokenFilter;
|
||||||
import org.elasticsearch.index.mapper.MapperService.MergeReason;
|
import org.elasticsearch.index.mapper.MapperService.MergeReason;
|
||||||
|
import org.elasticsearch.indices.analysis.PreBuiltCacheFactory.CachingStrategy;
|
||||||
|
import org.elasticsearch.plugins.AnalysisPlugin;
|
||||||
import org.elasticsearch.plugins.Plugin;
|
import org.elasticsearch.plugins.Plugin;
|
||||||
import org.elasticsearch.test.ESSingleNodeTestCase;
|
import org.elasticsearch.test.ESSingleNodeTestCase;
|
||||||
import org.elasticsearch.test.InternalSettingsPlugin;
|
import org.elasticsearch.test.InternalSettingsPlugin;
|
||||||
|
@ -38,15 +42,26 @@ import org.junit.Before;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
import static java.util.Collections.singletonList;
|
||||||
import static org.hamcrest.Matchers.containsString;
|
import static org.hamcrest.Matchers.containsString;
|
||||||
import static org.hamcrest.Matchers.equalTo;
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
|
|
||||||
public class KeywordFieldMapperTests extends ESSingleNodeTestCase {
|
public class KeywordFieldMapperTests extends ESSingleNodeTestCase {
|
||||||
|
/**
|
||||||
|
* Creates a copy of the lowercase token filter which we use for testing merge errors.
|
||||||
|
*/
|
||||||
|
public static class MockAnalysisPlugin extends Plugin implements AnalysisPlugin {
|
||||||
|
@Override
|
||||||
|
public List<PreConfiguredTokenFilter> getPreConfiguredTokenFilters() {
|
||||||
|
return singletonList(new PreConfiguredTokenFilter("mock_other_lowercase", true, CachingStrategy.ONE, MockLowerCaseFilter::new));
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected Collection<Class<? extends Plugin>> getPlugins() {
|
protected Collection<Class<? extends Plugin>> getPlugins() {
|
||||||
return pluginList(InternalSettingsPlugin.class);
|
return pluginList(InternalSettingsPlugin.class, MockAnalysisPlugin.class);
|
||||||
}
|
}
|
||||||
|
|
||||||
IndexService indexService;
|
IndexService indexService;
|
||||||
|
@ -57,8 +72,8 @@ public class KeywordFieldMapperTests extends ESSingleNodeTestCase {
|
||||||
indexService = createIndex("test", Settings.builder()
|
indexService = createIndex("test", Settings.builder()
|
||||||
.put("index.analysis.normalizer.my_lowercase.type", "custom")
|
.put("index.analysis.normalizer.my_lowercase.type", "custom")
|
||||||
.putArray("index.analysis.normalizer.my_lowercase.filter", "lowercase")
|
.putArray("index.analysis.normalizer.my_lowercase.filter", "lowercase")
|
||||||
.put("index.analysis.normalizer.my_asciifolding.type", "custom")
|
.put("index.analysis.normalizer.my_other_lowercase.type", "custom")
|
||||||
.putArray("index.analysis.normalizer.my_asciifolding.filter", "asciifolding").build());
|
.putArray("index.analysis.normalizer.my_other_lowercase.filter", "mock_other_lowercase").build());
|
||||||
parser = indexService.mapperService().documentMapperParser();
|
parser = indexService.mapperService().documentMapperParser();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -348,7 +363,7 @@ public class KeywordFieldMapperTests extends ESSingleNodeTestCase {
|
||||||
|
|
||||||
String mapping2 = XContentFactory.jsonBuilder().startObject().startObject("type")
|
String mapping2 = XContentFactory.jsonBuilder().startObject().startObject("type")
|
||||||
.startObject("properties").startObject("field")
|
.startObject("properties").startObject("field")
|
||||||
.field("type", "keyword").field("normalizer", "my_asciifolding").endObject().endObject()
|
.field("type", "keyword").field("normalizer", "my_other_lowercase").endObject().endObject()
|
||||||
.endObject().endObject().string();
|
.endObject().endObject().string();
|
||||||
IllegalArgumentException e = expectThrows(IllegalArgumentException.class,
|
IllegalArgumentException e = expectThrows(IllegalArgumentException.class,
|
||||||
() -> indexService.mapperService().merge("type",
|
() -> indexService.mapperService().merge("type",
|
||||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.lucene.search.MatchNoDocsQuery;
|
||||||
import org.apache.lucene.search.Query;
|
import org.apache.lucene.search.Query;
|
||||||
import org.elasticsearch.cluster.metadata.MetaData;
|
import org.elasticsearch.cluster.metadata.MetaData;
|
||||||
import org.elasticsearch.common.ParsingException;
|
import org.elasticsearch.common.ParsingException;
|
||||||
|
import org.elasticsearch.index.mapper.UidFieldMapper;
|
||||||
import org.elasticsearch.search.internal.SearchContext;
|
import org.elasticsearch.search.internal.SearchContext;
|
||||||
import org.elasticsearch.test.AbstractQueryTestCase;
|
import org.elasticsearch.test.AbstractQueryTestCase;
|
||||||
|
|
||||||
|
@ -73,7 +74,7 @@ public class IdsQueryBuilderTests extends AbstractQueryTestCase<IdsQueryBuilder>
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void doAssertLuceneQuery(IdsQueryBuilder queryBuilder, Query query, SearchContext context) throws IOException {
|
protected void doAssertLuceneQuery(IdsQueryBuilder queryBuilder, Query query, SearchContext context) throws IOException {
|
||||||
if (queryBuilder.ids().size() == 0) {
|
if (queryBuilder.ids().size() == 0 || context.getQueryShardContext().fieldMapper(UidFieldMapper.NAME) == null) {
|
||||||
assertThat(query, instanceOf(MatchNoDocsQuery.class));
|
assertThat(query, instanceOf(MatchNoDocsQuery.class));
|
||||||
} else {
|
} else {
|
||||||
assertThat(query, instanceOf(TermInSetQuery.class));
|
assertThat(query, instanceOf(TermInSetQuery.class));
|
||||||
|
|
|
@ -59,6 +59,7 @@ import org.elasticsearch.index.shard.ShardPath;
|
||||||
import org.elasticsearch.index.translog.Translog;
|
import org.elasticsearch.index.translog.Translog;
|
||||||
import org.elasticsearch.indices.recovery.RecoveryState;
|
import org.elasticsearch.indices.recovery.RecoveryState;
|
||||||
import org.elasticsearch.indices.recovery.RecoveryTarget;
|
import org.elasticsearch.indices.recovery.RecoveryTarget;
|
||||||
|
import org.elasticsearch.transport.TransportRequest;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
@ -123,7 +124,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
||||||
|
|
||||||
ReplicationGroup(final IndexMetaData indexMetaData) throws IOException {
|
ReplicationGroup(final IndexMetaData indexMetaData) throws IOException {
|
||||||
final ShardRouting primaryRouting = this.createShardRouting("s0", true);
|
final ShardRouting primaryRouting = this.createShardRouting("s0", true);
|
||||||
primary = newShard(primaryRouting, indexMetaData, null, this::syncGlobalCheckpoint, getEngineFactory(primaryRouting));
|
primary = newShard(primaryRouting, indexMetaData, null, getEngineFactory(primaryRouting));
|
||||||
replicas = new ArrayList<>();
|
replicas = new ArrayList<>();
|
||||||
this.indexMetaData = indexMetaData;
|
this.indexMetaData = indexMetaData;
|
||||||
updateAllocationIDsOnPrimary();
|
updateAllocationIDsOnPrimary();
|
||||||
|
@ -152,7 +153,6 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
||||||
assertEquals(DocWriteResponse.Result.CREATED, response.getResponse().getResult());
|
assertEquals(DocWriteResponse.Result.CREATED, response.getResponse().getResult());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
primary.updateGlobalCheckpointOnPrimary();
|
|
||||||
return numOfDoc;
|
return numOfDoc;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -166,7 +166,6 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
||||||
assertEquals(DocWriteResponse.Result.CREATED, response.getResponse().getResult());
|
assertEquals(DocWriteResponse.Result.CREATED, response.getResponse().getResult());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
primary.updateGlobalCheckpointOnPrimary();
|
|
||||||
return numOfDoc;
|
return numOfDoc;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -208,16 +207,16 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
||||||
primary.markAsRecovering("store", new RecoveryState(primary.routingEntry(), pNode, null));
|
primary.markAsRecovering("store", new RecoveryState(primary.routingEntry(), pNode, null));
|
||||||
primary.recoverFromStore();
|
primary.recoverFromStore();
|
||||||
primary.updateRoutingEntry(ShardRoutingHelper.moveToStarted(primary.routingEntry()));
|
primary.updateRoutingEntry(ShardRoutingHelper.moveToStarted(primary.routingEntry()));
|
||||||
|
updateAllocationIDsOnPrimary();
|
||||||
for (final IndexShard replica : replicas) {
|
for (final IndexShard replica : replicas) {
|
||||||
recoverReplica(replica);
|
recoverReplica(replica);
|
||||||
}
|
}
|
||||||
updateAllocationIDsOnPrimary();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public IndexShard addReplica() throws IOException {
|
public IndexShard addReplica() throws IOException {
|
||||||
final ShardRouting replicaRouting = createShardRouting("s" + replicaId.incrementAndGet(), false);
|
final ShardRouting replicaRouting = createShardRouting("s" + replicaId.incrementAndGet(), false);
|
||||||
final IndexShard replica =
|
final IndexShard replica =
|
||||||
newShard(replicaRouting, indexMetaData, null, this::syncGlobalCheckpoint, getEngineFactory(replicaRouting));
|
newShard(replicaRouting, indexMetaData, null, getEngineFactory(replicaRouting));
|
||||||
addReplica(replica);
|
addReplica(replica);
|
||||||
return replica;
|
return replica;
|
||||||
}
|
}
|
||||||
|
@ -240,7 +239,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
||||||
RecoverySource.PeerRecoverySource.INSTANCE);
|
RecoverySource.PeerRecoverySource.INSTANCE);
|
||||||
|
|
||||||
final IndexShard newReplica = newShard(shardRouting, shardPath, indexMetaData, null,
|
final IndexShard newReplica = newShard(shardRouting, shardPath, indexMetaData, null,
|
||||||
this::syncGlobalCheckpoint, getEngineFactory(shardRouting));
|
getEngineFactory(shardRouting));
|
||||||
replicas.add(newReplica);
|
replicas.add(newReplica);
|
||||||
updateAllocationIDsOnPrimary();
|
updateAllocationIDsOnPrimary();
|
||||||
return newReplica;
|
return newReplica;
|
||||||
|
@ -358,7 +357,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
||||||
return primary;
|
return primary;
|
||||||
}
|
}
|
||||||
|
|
||||||
private void syncGlobalCheckpoint() {
|
public void syncGlobalCheckpoint() {
|
||||||
PlainActionFuture<ReplicationResponse> listener = new PlainActionFuture<>();
|
PlainActionFuture<ReplicationResponse> listener = new PlainActionFuture<>();
|
||||||
try {
|
try {
|
||||||
new GlobalCheckpointSync(listener, this).execute();
|
new GlobalCheckpointSync(listener, this).execute();
|
||||||
|
@ -390,8 +389,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
||||||
private final ReplicationGroup replicationGroup;
|
private final ReplicationGroup replicationGroup;
|
||||||
private final String opType;
|
private final String opType;
|
||||||
|
|
||||||
ReplicationAction(Request request, ActionListener<Response> listener,
|
ReplicationAction(Request request, ActionListener<Response> listener, ReplicationGroup group, String opType) {
|
||||||
ReplicationGroup group, String opType) {
|
|
||||||
this.request = request;
|
this.request = request;
|
||||||
this.listener = listener;
|
this.listener = listener;
|
||||||
this.replicationGroup = group;
|
this.replicationGroup = group;
|
||||||
|
@ -462,18 +460,26 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
||||||
public long localCheckpoint() {
|
public long localCheckpoint() {
|
||||||
return replicationGroup.getPrimary().getLocalCheckpoint();
|
return replicationGroup.getPrimary().getLocalCheckpoint();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long globalCheckpoint() {
|
||||||
|
return replicationGroup.getPrimary().getGlobalCheckpoint();
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
class ReplicasRef implements ReplicationOperation.Replicas<ReplicaRequest> {
|
class ReplicasRef implements ReplicationOperation.Replicas<ReplicaRequest> {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void performOn(
|
public void performOn(
|
||||||
ShardRouting replicaRouting,
|
final ShardRouting replicaRouting,
|
||||||
ReplicaRequest request,
|
final ReplicaRequest request,
|
||||||
ActionListener<ReplicationOperation.ReplicaResponse> listener) {
|
final long globalCheckpoint,
|
||||||
|
final ActionListener<ReplicationOperation.ReplicaResponse> listener) {
|
||||||
try {
|
try {
|
||||||
IndexShard replica = replicationGroup.replicas.stream()
|
IndexShard replica = replicationGroup.replicas.stream()
|
||||||
.filter(s -> replicaRouting.isSameAllocation(s.routingEntry())).findFirst().get();
|
.filter(s -> replicaRouting.isSameAllocation(s.routingEntry())).findFirst().get();
|
||||||
|
replica.updateGlobalCheckpointOnReplica(globalCheckpoint);
|
||||||
performOnReplica(request, replica);
|
performOnReplica(request, replica);
|
||||||
listener.onResponse(new ReplicaResponse(replica.routingEntry().allocationId().getId(), replica.getLocalCheckpoint()));
|
listener.onResponse(new ReplicaResponse(replica.routingEntry().allocationId().getId(), replica.getLocalCheckpoint()));
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
|
@ -578,26 +584,30 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
||||||
executeShardBulkOnReplica(replica, request);
|
executeShardBulkOnReplica(replica, request);
|
||||||
}
|
}
|
||||||
|
|
||||||
class GlobalCheckpointSync extends ReplicationAction<GlobalCheckpointSyncAction.PrimaryRequest,
|
class GlobalCheckpointSync extends ReplicationAction<
|
||||||
GlobalCheckpointSyncAction.ReplicaRequest, ReplicationResponse> {
|
GlobalCheckpointSyncAction.Request,
|
||||||
|
GlobalCheckpointSyncAction.Request,
|
||||||
|
ReplicationResponse> {
|
||||||
|
|
||||||
GlobalCheckpointSync(ActionListener<ReplicationResponse> listener, ReplicationGroup replicationGroup) {
|
GlobalCheckpointSync(final ActionListener<ReplicationResponse> listener, final ReplicationGroup replicationGroup) {
|
||||||
super(new GlobalCheckpointSyncAction.PrimaryRequest(replicationGroup.getPrimary().shardId()), listener,
|
super(
|
||||||
replicationGroup, "global_ckp");
|
new GlobalCheckpointSyncAction.Request(replicationGroup.getPrimary().shardId()),
|
||||||
|
listener,
|
||||||
|
replicationGroup,
|
||||||
|
"global_checkpoint_sync");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected PrimaryResult performOnPrimary(IndexShard primary,
|
protected PrimaryResult performOnPrimary(
|
||||||
GlobalCheckpointSyncAction.PrimaryRequest request) throws Exception {
|
final IndexShard primary, final GlobalCheckpointSyncAction.Request request) throws Exception {
|
||||||
primary.getTranslog().sync();
|
primary.getTranslog().sync();
|
||||||
return new PrimaryResult(new GlobalCheckpointSyncAction.ReplicaRequest(request, primary.getGlobalCheckpoint()),
|
return new PrimaryResult(request, new ReplicationResponse());
|
||||||
new ReplicationResponse());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void performOnReplica(GlobalCheckpointSyncAction.ReplicaRequest request, IndexShard replica) throws IOException {
|
protected void performOnReplica(final GlobalCheckpointSyncAction.Request request, final IndexShard replica) throws IOException {
|
||||||
replica.updateGlobalCheckpointOnReplica(request.getCheckpoint());
|
|
||||||
replica.getTranslog().sync();
|
replica.getTranslog().sync();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -38,11 +38,14 @@ import org.elasticsearch.index.engine.InternalEngineTests;
|
||||||
import org.elasticsearch.index.engine.SegmentsStats;
|
import org.elasticsearch.index.engine.SegmentsStats;
|
||||||
import org.elasticsearch.index.engine.VersionConflictEngineException;
|
import org.elasticsearch.index.engine.VersionConflictEngineException;
|
||||||
import org.elasticsearch.index.seqno.SeqNoStats;
|
import org.elasticsearch.index.seqno.SeqNoStats;
|
||||||
|
import org.elasticsearch.index.seqno.SequenceNumbersService;
|
||||||
import org.elasticsearch.index.shard.IndexShard;
|
import org.elasticsearch.index.shard.IndexShard;
|
||||||
import org.elasticsearch.index.shard.IndexShardTests;
|
import org.elasticsearch.index.shard.IndexShardTests;
|
||||||
import org.elasticsearch.index.store.Store;
|
import org.elasticsearch.index.store.Store;
|
||||||
import org.elasticsearch.index.translog.Translog;
|
import org.elasticsearch.index.translog.Translog;
|
||||||
import org.elasticsearch.indices.recovery.RecoveryTarget;
|
import org.elasticsearch.indices.recovery.RecoveryTarget;
|
||||||
|
import org.elasticsearch.test.junit.annotations.TestLogging;
|
||||||
|
import org.hamcrest.Matcher;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
@ -50,6 +53,7 @@ import java.util.Map;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.Future;
|
import java.util.concurrent.Future;
|
||||||
|
|
||||||
|
import static org.hamcrest.Matchers.anyOf;
|
||||||
import static org.hamcrest.Matchers.containsString;
|
import static org.hamcrest.Matchers.containsString;
|
||||||
import static org.hamcrest.Matchers.equalTo;
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
import static org.hamcrest.Matchers.instanceOf;
|
import static org.hamcrest.Matchers.instanceOf;
|
||||||
|
@ -148,18 +152,40 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase
|
||||||
startedShards = shards.startReplicas(randomIntBetween(1, 2));
|
startedShards = shards.startReplicas(randomIntBetween(1, 2));
|
||||||
} while (startedShards > 0);
|
} while (startedShards > 0);
|
||||||
|
|
||||||
if (numDocs == 0 || randomBoolean()) {
|
final long unassignedSeqNo = SequenceNumbersService.UNASSIGNED_SEQ_NO;
|
||||||
// in the case we have no indexing, we simulate the background global checkpoint sync
|
|
||||||
shards.getPrimary().updateGlobalCheckpointOnPrimary();
|
|
||||||
}
|
|
||||||
for (IndexShard shard : shards) {
|
for (IndexShard shard : shards) {
|
||||||
final SeqNoStats shardStats = shard.seqNoStats();
|
final SeqNoStats shardStats = shard.seqNoStats();
|
||||||
final ShardRouting shardRouting = shard.routingEntry();
|
final ShardRouting shardRouting = shard.routingEntry();
|
||||||
logger.debug("seq_no stats for {}: {}", shardRouting, XContentHelper.toString(shardStats,
|
|
||||||
new ToXContent.MapParams(Collections.singletonMap("pretty", "false"))));
|
|
||||||
assertThat(shardRouting + " local checkpoint mismatch", shardStats.getLocalCheckpoint(), equalTo(numDocs - 1L));
|
assertThat(shardRouting + " local checkpoint mismatch", shardStats.getLocalCheckpoint(), equalTo(numDocs - 1L));
|
||||||
|
/*
|
||||||
|
* After the last indexing operation completes, the primary will advance its global checkpoint. Without an other indexing
|
||||||
|
* operation, or a background sync, the primary will not have broadcast this global checkpoint to its replicas. However, a
|
||||||
|
* shard could have recovered from the primary in which case its global checkpoint will be in-sync with the primary.
|
||||||
|
* Therefore, we can only assert that the global checkpoint is number of docs minus one (matching the primary, in case of a
|
||||||
|
* recovery), or number of docs minus two (received indexing operations but has not received a global checkpoint sync after
|
||||||
|
* the last operation completed).
|
||||||
|
*/
|
||||||
|
final Matcher<Long> globalCheckpointMatcher;
|
||||||
|
if (shardRouting.primary()) {
|
||||||
|
globalCheckpointMatcher = numDocs == 0 ? equalTo(unassignedSeqNo) : equalTo(numDocs - 1L);
|
||||||
|
} else {
|
||||||
|
globalCheckpointMatcher = numDocs == 0 ? equalTo(unassignedSeqNo) : anyOf(equalTo(numDocs - 1L), equalTo(numDocs - 2L));
|
||||||
|
}
|
||||||
|
assertThat(shardRouting + " global checkpoint mismatch", shardStats.getGlobalCheckpoint(), globalCheckpointMatcher);
|
||||||
|
assertThat(shardRouting + " max seq no mismatch", shardStats.getMaxSeqNo(), equalTo(numDocs - 1L));
|
||||||
|
}
|
||||||
|
|
||||||
assertThat(shardRouting + " global checkpoint mismatch", shardStats.getGlobalCheckpoint(), equalTo(numDocs - 1L));
|
// simulate a background global checkpoint sync at which point we expect the global checkpoint to advance on the replicas
|
||||||
|
shards.syncGlobalCheckpoint();
|
||||||
|
|
||||||
|
for (IndexShard shard : shards) {
|
||||||
|
final SeqNoStats shardStats = shard.seqNoStats();
|
||||||
|
final ShardRouting shardRouting = shard.routingEntry();
|
||||||
|
assertThat(shardRouting + " local checkpoint mismatch", shardStats.getLocalCheckpoint(), equalTo(numDocs - 1L));
|
||||||
|
assertThat(
|
||||||
|
shardRouting + " global checkpoint mismatch",
|
||||||
|
shardStats.getGlobalCheckpoint(),
|
||||||
|
numDocs == 0 ? equalTo(unassignedSeqNo) : equalTo(numDocs - 1L));
|
||||||
assertThat(shardRouting + " max seq no mismatch", shardStats.getMaxSeqNo(), equalTo(numDocs - 1L));
|
assertThat(shardRouting + " max seq no mismatch", shardStats.getMaxSeqNo(), equalTo(numDocs - 1L));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -29,6 +29,7 @@ import org.elasticsearch.action.index.IndexRequest;
|
||||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||||
|
import org.elasticsearch.common.util.concurrent.CountDown;
|
||||||
import org.elasticsearch.common.xcontent.XContentType;
|
import org.elasticsearch.common.xcontent.XContentType;
|
||||||
import org.elasticsearch.index.engine.Engine;
|
import org.elasticsearch.index.engine.Engine;
|
||||||
import org.elasticsearch.index.engine.EngineConfig;
|
import org.elasticsearch.index.engine.EngineConfig;
|
||||||
|
@ -51,6 +52,7 @@ import java.util.concurrent.Future;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
import java.util.concurrent.atomic.AtomicReference;
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
|
|
||||||
|
import static org.hamcrest.Matchers.anyOf;
|
||||||
import static org.hamcrest.Matchers.empty;
|
import static org.hamcrest.Matchers.empty;
|
||||||
import static org.hamcrest.Matchers.equalTo;
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
import static org.hamcrest.Matchers.lessThan;
|
import static org.hamcrest.Matchers.lessThan;
|
||||||
|
@ -84,7 +86,6 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
|
||||||
shards.startAll();
|
shards.startAll();
|
||||||
int docs = shards.indexDocs(randomInt(50));
|
int docs = shards.indexDocs(randomInt(50));
|
||||||
shards.flush();
|
shards.flush();
|
||||||
shards.getPrimary().updateGlobalCheckpointOnPrimary();
|
|
||||||
final IndexShard originalReplica = shards.getReplicas().get(0);
|
final IndexShard originalReplica = shards.getReplicas().get(0);
|
||||||
long replicaCommittedLocalCheckpoint = docs - 1;
|
long replicaCommittedLocalCheckpoint = docs - 1;
|
||||||
boolean replicaHasDocsSinceLastFlushedCheckpoint = false;
|
boolean replicaHasDocsSinceLastFlushedCheckpoint = false;
|
||||||
|
@ -101,23 +102,17 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
|
||||||
replicaHasDocsSinceLastFlushedCheckpoint = false;
|
replicaHasDocsSinceLastFlushedCheckpoint = false;
|
||||||
replicaCommittedLocalCheckpoint = docs - 1;
|
replicaCommittedLocalCheckpoint = docs - 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
final boolean sync = randomBoolean();
|
|
||||||
if (sync) {
|
|
||||||
shards.getPrimary().updateGlobalCheckpointOnPrimary();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// simulate a background global checkpoint sync at which point we expect the global checkpoint to advance on the replicas
|
||||||
|
shards.syncGlobalCheckpoint();
|
||||||
|
|
||||||
shards.removeReplica(originalReplica);
|
shards.removeReplica(originalReplica);
|
||||||
|
|
||||||
final int missingOnReplica = shards.indexDocs(randomInt(5));
|
final int missingOnReplica = shards.indexDocs(randomInt(5));
|
||||||
docs += missingOnReplica;
|
docs += missingOnReplica;
|
||||||
replicaHasDocsSinceLastFlushedCheckpoint |= missingOnReplica > 0;
|
replicaHasDocsSinceLastFlushedCheckpoint |= missingOnReplica > 0;
|
||||||
|
|
||||||
if (randomBoolean()) {
|
|
||||||
shards.getPrimary().updateGlobalCheckpointOnPrimary();
|
|
||||||
}
|
|
||||||
|
|
||||||
final boolean flushPrimary = randomBoolean();
|
final boolean flushPrimary = randomBoolean();
|
||||||
if (flushPrimary) {
|
if (flushPrimary) {
|
||||||
shards.flush();
|
shards.flush();
|
||||||
|
@ -234,6 +229,8 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
|
||||||
}) {
|
}) {
|
||||||
shards.startAll();
|
shards.startAll();
|
||||||
int docs = shards.indexDocs(randomIntBetween(1, 10));
|
int docs = shards.indexDocs(randomIntBetween(1, 10));
|
||||||
|
// simulate a background global checkpoint sync at which point we expect the global checkpoint to advance on the replicas
|
||||||
|
shards.syncGlobalCheckpoint();
|
||||||
IndexShard replica = shards.getReplicas().get(0);
|
IndexShard replica = shards.getReplicas().get(0);
|
||||||
shards.removeReplica(replica);
|
shards.removeReplica(replica);
|
||||||
closeShards(replica);
|
closeShards(replica);
|
||||||
|
@ -330,6 +327,7 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
|
||||||
logger.info("indexed [{}] docs", docs);
|
logger.info("indexed [{}] docs", docs);
|
||||||
final CountDownLatch pendingDocDone = new CountDownLatch(1);
|
final CountDownLatch pendingDocDone = new CountDownLatch(1);
|
||||||
final CountDownLatch pendingDocActiveWithExtraDocIndexed = new CountDownLatch(1);
|
final CountDownLatch pendingDocActiveWithExtraDocIndexed = new CountDownLatch(1);
|
||||||
|
final CountDownLatch phaseTwoStartLatch = new CountDownLatch(1);
|
||||||
final IndexShard replica = shards.addReplica();
|
final IndexShard replica = shards.addReplica();
|
||||||
final Future<Void> recoveryFuture = shards.asyncRecoverReplica(
|
final Future<Void> recoveryFuture = shards.asyncRecoverReplica(
|
||||||
replica,
|
replica,
|
||||||
|
@ -353,17 +351,15 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
|
||||||
// unblock indexing for the next doc
|
// unblock indexing for the next doc
|
||||||
replicaEngineFactory.allowIndexing();
|
replicaEngineFactory.allowIndexing();
|
||||||
shards.index(new IndexRequest(index.getName(), "type", "completed").source("{}", XContentType.JSON));
|
shards.index(new IndexRequest(index.getName(), "type", "completed").source("{}", XContentType.JSON));
|
||||||
/*
|
|
||||||
* We want to test that the global checkpoint is blocked from advancing on the primary when a replica shard
|
|
||||||
* is pending being marked in-sync. We also want to test the the global checkpoint does not advance on the
|
|
||||||
* replica when its local checkpoint is behind the global checkpoint on the primary. Finally, advancing the
|
|
||||||
* global checkpoint here forces recovery to block until the pending doc is indexing on the replica.
|
|
||||||
*/
|
|
||||||
shards.getPrimary().updateGlobalCheckpointOnPrimary();
|
|
||||||
pendingDocActiveWithExtraDocIndexed.countDown();
|
pendingDocActiveWithExtraDocIndexed.countDown();
|
||||||
} catch (final Exception e) {
|
} catch (final Exception e) {
|
||||||
throw new AssertionError(e);
|
throw new AssertionError(e);
|
||||||
}
|
}
|
||||||
|
try {
|
||||||
|
phaseTwoStartLatch.await();
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
throw new AssertionError(e);
|
||||||
|
}
|
||||||
return super.indexTranslogOperations(operations, totalTranslogOps);
|
return super.indexTranslogOperations(operations, totalTranslogOps);
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
@ -372,18 +368,23 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
|
||||||
{
|
{
|
||||||
final long expectedDocs = docs + 2L;
|
final long expectedDocs = docs + 2L;
|
||||||
assertThat(shards.getPrimary().getLocalCheckpoint(), equalTo(expectedDocs - 1));
|
assertThat(shards.getPrimary().getLocalCheckpoint(), equalTo(expectedDocs - 1));
|
||||||
// recovery has not completed, therefore the global checkpoint can have advance on the primary
|
// recovery has not completed, therefore the global checkpoint can have advanced on the primary
|
||||||
assertThat(shards.getPrimary().getGlobalCheckpoint(), equalTo(expectedDocs - 1));
|
assertThat(shards.getPrimary().getGlobalCheckpoint(), equalTo(expectedDocs - 1));
|
||||||
// the pending document is not done, the checkpoints can not have advanced on the replica
|
// the pending document is not done, the checkpoints can not have advanced on the replica
|
||||||
assertThat(replica.getLocalCheckpoint(), lessThan(expectedDocs - 1));
|
assertThat(replica.getLocalCheckpoint(), lessThan(expectedDocs - 1));
|
||||||
assertThat(replica.getGlobalCheckpoint(), lessThan(expectedDocs - 1));
|
assertThat(replica.getGlobalCheckpoint(), lessThan(expectedDocs - 1));
|
||||||
}
|
}
|
||||||
|
|
||||||
shards.getPrimary().updateGlobalCheckpointOnPrimary();
|
// wait for recovery to enter the translog phase
|
||||||
|
phaseTwoStartLatch.countDown();
|
||||||
|
|
||||||
|
// wait for the translog phase to complete and the recovery to block global checkpoint advancement
|
||||||
|
awaitBusy(() -> shards.getPrimary().pendingInSync());
|
||||||
{
|
{
|
||||||
final long expectedDocs = docs + 3L;
|
|
||||||
shards.index(new IndexRequest(index.getName(), "type", "last").source("{}", XContentType.JSON));
|
shards.index(new IndexRequest(index.getName(), "type", "last").source("{}", XContentType.JSON));
|
||||||
|
final long expectedDocs = docs + 3L;
|
||||||
assertThat(shards.getPrimary().getLocalCheckpoint(), equalTo(expectedDocs - 1));
|
assertThat(shards.getPrimary().getLocalCheckpoint(), equalTo(expectedDocs - 1));
|
||||||
|
// recovery is now in the process of being completed, therefore the global checkpoint can not have advanced on the primary
|
||||||
assertThat(shards.getPrimary().getGlobalCheckpoint(), equalTo(expectedDocs - 2));
|
assertThat(shards.getPrimary().getGlobalCheckpoint(), equalTo(expectedDocs - 2));
|
||||||
assertThat(replica.getLocalCheckpoint(), lessThan(expectedDocs - 2));
|
assertThat(replica.getLocalCheckpoint(), lessThan(expectedDocs - 2));
|
||||||
assertThat(replica.getGlobalCheckpoint(), lessThan(expectedDocs - 2));
|
assertThat(replica.getGlobalCheckpoint(), lessThan(expectedDocs - 2));
|
||||||
|
@ -392,14 +393,14 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
|
||||||
replicaEngineFactory.releaseLatchedIndexers();
|
replicaEngineFactory.releaseLatchedIndexers();
|
||||||
pendingDocDone.await();
|
pendingDocDone.await();
|
||||||
recoveryFuture.get();
|
recoveryFuture.get();
|
||||||
shards.getPrimary().updateGlobalCheckpointOnPrimary();
|
|
||||||
{
|
{
|
||||||
final long expectedDocs = docs + 3L;
|
final long expectedDocs = docs + 3L;
|
||||||
assertBusy(() -> {
|
assertBusy(() -> {
|
||||||
assertThat(shards.getPrimary().getLocalCheckpoint(), equalTo(expectedDocs - 1));
|
assertThat(shards.getPrimary().getLocalCheckpoint(), equalTo(expectedDocs - 1));
|
||||||
assertThat(shards.getPrimary().getGlobalCheckpoint(), equalTo(expectedDocs - 1));
|
assertThat(shards.getPrimary().getGlobalCheckpoint(), equalTo(expectedDocs - 1));
|
||||||
assertThat(replica.getLocalCheckpoint(), equalTo(expectedDocs - 1));
|
assertThat(replica.getLocalCheckpoint(), equalTo(expectedDocs - 1));
|
||||||
assertThat(replica.getGlobalCheckpoint(), equalTo(expectedDocs - 1));
|
// the global checkpoint advances can only advance here if a background global checkpoint sync fires
|
||||||
|
assertThat(replica.getGlobalCheckpoint(), anyOf(equalTo(expectedDocs - 1), equalTo(expectedDocs - 2)));
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -83,6 +83,9 @@ public class GlobalCheckpointSyncActionTests extends ESTestCase {
|
||||||
final IndexShard indexShard = mock(IndexShard.class);
|
final IndexShard indexShard = mock(IndexShard.class);
|
||||||
when(indexService.getShard(id)).thenReturn(indexShard);
|
when(indexService.getShard(id)).thenReturn(indexShard);
|
||||||
|
|
||||||
|
final ShardId shardId = new ShardId(index, id);
|
||||||
|
when(indexShard.shardId()).thenReturn(shardId);
|
||||||
|
|
||||||
final Translog translog = mock(Translog.class);
|
final Translog translog = mock(Translog.class);
|
||||||
when(indexShard.getTranslog()).thenReturn(translog);
|
when(indexShard.getTranslog()).thenReturn(translog);
|
||||||
|
|
||||||
|
@ -95,13 +98,11 @@ public class GlobalCheckpointSyncActionTests extends ESTestCase {
|
||||||
shardStateAction,
|
shardStateAction,
|
||||||
new ActionFilters(Collections.emptySet()),
|
new ActionFilters(Collections.emptySet()),
|
||||||
new IndexNameExpressionResolver(Settings.EMPTY));
|
new IndexNameExpressionResolver(Settings.EMPTY));
|
||||||
final ShardId shardId = new ShardId(index, id);
|
final GlobalCheckpointSyncAction.Request primaryRequest = new GlobalCheckpointSyncAction.Request(indexShard.shardId());
|
||||||
final GlobalCheckpointSyncAction.PrimaryRequest primaryRequest = new GlobalCheckpointSyncAction.PrimaryRequest(shardId);
|
|
||||||
if (randomBoolean()) {
|
if (randomBoolean()) {
|
||||||
action.shardOperationOnPrimary(primaryRequest, indexShard);
|
action.shardOperationOnPrimary(primaryRequest, indexShard);
|
||||||
} else {
|
} else {
|
||||||
action.shardOperationOnReplica(
|
action.shardOperationOnReplica(new GlobalCheckpointSyncAction.Request(indexShard.shardId()), indexShard);
|
||||||
new GlobalCheckpointSyncAction.ReplicaRequest(primaryRequest, randomNonNegativeLong()), indexShard);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
verify(translog).sync();
|
verify(translog).sync();
|
||||||
|
|
|
@ -29,6 +29,7 @@ import org.junit.Before;
|
||||||
|
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
import java.util.Comparator;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -44,10 +45,8 @@ import java.util.stream.IntStream;
|
||||||
import java.util.stream.Stream;
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
import static org.elasticsearch.index.seqno.SequenceNumbersService.UNASSIGNED_SEQ_NO;
|
import static org.elasticsearch.index.seqno.SequenceNumbersService.UNASSIGNED_SEQ_NO;
|
||||||
import static org.hamcrest.Matchers.comparesEqualTo;
|
|
||||||
import static org.hamcrest.Matchers.equalTo;
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
import static org.hamcrest.Matchers.greaterThan;
|
import static org.hamcrest.Matchers.greaterThan;
|
||||||
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
|
||||||
import static org.hamcrest.Matchers.not;
|
import static org.hamcrest.Matchers.not;
|
||||||
|
|
||||||
public class GlobalCheckpointTrackerTests extends ESTestCase {
|
public class GlobalCheckpointTrackerTests extends ESTestCase {
|
||||||
|
@ -66,8 +65,7 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testEmptyShards() {
|
public void testEmptyShards() {
|
||||||
assertFalse("checkpoint shouldn't be updated when the are no active shards", tracker.updateCheckpointOnPrimary());
|
assertThat(tracker.getGlobalCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
||||||
assertThat(tracker.getCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private final AtomicInteger aIdGenerator = new AtomicInteger();
|
private final AtomicInteger aIdGenerator = new AtomicInteger();
|
||||||
|
@ -92,9 +90,9 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
||||||
|
|
||||||
// note: allocations can never be empty in practice as we always have at least one primary shard active/in sync
|
// note: allocations can never be empty in practice as we always have at least one primary shard active/in sync
|
||||||
// it is however nice not to assume this on this level and check we do the right thing.
|
// it is however nice not to assume this on this level and check we do the right thing.
|
||||||
final long maxLocalCheckpoint = allocations.values().stream().min(Long::compare).orElse(UNASSIGNED_SEQ_NO);
|
final long minLocalCheckpoint = allocations.values().stream().min(Long::compare).orElse(UNASSIGNED_SEQ_NO);
|
||||||
|
|
||||||
assertThat(tracker.getCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
assertThat(tracker.getGlobalCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
||||||
|
|
||||||
logger.info("--> using allocations");
|
logger.info("--> using allocations");
|
||||||
allocations.keySet().forEach(aId -> {
|
allocations.keySet().forEach(aId -> {
|
||||||
|
@ -110,41 +108,35 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
||||||
});
|
});
|
||||||
|
|
||||||
tracker.updateAllocationIdsFromMaster(active, initializing);
|
tracker.updateAllocationIdsFromMaster(active, initializing);
|
||||||
initializing.forEach(aId -> markAllocationIdAsInSyncQuietly(tracker, aId, tracker.getCheckpoint()));
|
initializing.forEach(aId -> markAllocationIdAsInSyncQuietly(tracker, aId, tracker.getGlobalCheckpoint()));
|
||||||
allocations.keySet().forEach(aId -> tracker.updateLocalCheckpoint(aId, allocations.get(aId)));
|
allocations.keySet().forEach(aId -> tracker.updateLocalCheckpoint(aId, allocations.get(aId)));
|
||||||
|
|
||||||
|
assertThat(tracker.getGlobalCheckpoint(), equalTo(minLocalCheckpoint));
|
||||||
assertThat(tracker.getCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
|
||||||
|
|
||||||
assertThat(tracker.updateCheckpointOnPrimary(), equalTo(maxLocalCheckpoint != UNASSIGNED_SEQ_NO));
|
|
||||||
assertThat(tracker.getCheckpoint(), equalTo(maxLocalCheckpoint));
|
|
||||||
|
|
||||||
// increment checkpoints
|
// increment checkpoints
|
||||||
active.forEach(aId -> allocations.put(aId, allocations.get(aId) + 1 + randomInt(4)));
|
active.forEach(aId -> allocations.put(aId, allocations.get(aId) + 1 + randomInt(4)));
|
||||||
initializing.forEach(aId -> allocations.put(aId, allocations.get(aId) + 1 + randomInt(4)));
|
initializing.forEach(aId -> allocations.put(aId, allocations.get(aId) + 1 + randomInt(4)));
|
||||||
allocations.keySet().forEach(aId -> tracker.updateLocalCheckpoint(aId, allocations.get(aId)));
|
allocations.keySet().forEach(aId -> tracker.updateLocalCheckpoint(aId, allocations.get(aId)));
|
||||||
|
|
||||||
|
final long minLocalCheckpointAfterUpdates =
|
||||||
|
allocations.entrySet().stream().map(Map.Entry::getValue).min(Long::compareTo).orElse(UNASSIGNED_SEQ_NO);
|
||||||
|
|
||||||
// now insert an unknown active/insync id , the checkpoint shouldn't change but a refresh should be requested.
|
// now insert an unknown active/insync id , the checkpoint shouldn't change but a refresh should be requested.
|
||||||
final String extraId = "extra_" + randomAlphaOfLength(5);
|
final String extraId = "extra_" + randomAlphaOfLength(5);
|
||||||
|
|
||||||
// first check that adding it without the master blessing doesn't change anything.
|
// first check that adding it without the master blessing doesn't change anything.
|
||||||
tracker.updateLocalCheckpoint(extraId, maxLocalCheckpoint + 1 + randomInt(4));
|
tracker.updateLocalCheckpoint(extraId, minLocalCheckpointAfterUpdates + 1 + randomInt(4));
|
||||||
assertThat(tracker.getLocalCheckpointForAllocationId(extraId), equalTo(UNASSIGNED_SEQ_NO));
|
assertThat(tracker.getLocalCheckpointForAllocationId(extraId), equalTo(UNASSIGNED_SEQ_NO));
|
||||||
|
|
||||||
Set<String> newActive = new HashSet<>(active);
|
Set<String> newActive = new HashSet<>(active);
|
||||||
newActive.add(extraId);
|
newActive.add(extraId);
|
||||||
tracker.updateAllocationIdsFromMaster(newActive, initializing);
|
tracker.updateAllocationIdsFromMaster(newActive, initializing);
|
||||||
|
|
||||||
// we should ask for a refresh , but not update the checkpoint
|
|
||||||
assertTrue(tracker.updateCheckpointOnPrimary());
|
|
||||||
assertThat(tracker.getCheckpoint(), equalTo(maxLocalCheckpoint));
|
|
||||||
|
|
||||||
// now notify for the new id
|
// now notify for the new id
|
||||||
tracker.updateLocalCheckpoint(extraId, maxLocalCheckpoint + 1 + randomInt(4));
|
tracker.updateLocalCheckpoint(extraId, minLocalCheckpointAfterUpdates + 1 + randomInt(4));
|
||||||
|
|
||||||
// now it should be incremented
|
// now it should be incremented
|
||||||
assertTrue(tracker.updateCheckpointOnPrimary());
|
assertThat(tracker.getGlobalCheckpoint(), greaterThan(minLocalCheckpoint));
|
||||||
assertThat(tracker.getCheckpoint(), greaterThan(maxLocalCheckpoint));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testMissingActiveIdsPreventAdvance() {
|
public void testMissingActiveIdsPreventAdvance() {
|
||||||
|
@ -153,43 +145,36 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
||||||
final Map<String, Long> assigned = new HashMap<>();
|
final Map<String, Long> assigned = new HashMap<>();
|
||||||
assigned.putAll(active);
|
assigned.putAll(active);
|
||||||
assigned.putAll(initializing);
|
assigned.putAll(initializing);
|
||||||
|
final String maxActiveID = active.entrySet().stream().max(Comparator.comparing(Map.Entry::getValue)).get().getKey();
|
||||||
tracker.updateAllocationIdsFromMaster(
|
tracker.updateAllocationIdsFromMaster(
|
||||||
new HashSet<>(randomSubsetOf(randomInt(active.size() - 1), active.keySet())),
|
active.entrySet().stream().filter(e -> !e.getKey().equals(maxActiveID)).map(Map.Entry::getKey).collect(Collectors.toSet()),
|
||||||
initializing.keySet());
|
initializing.keySet());
|
||||||
randomSubsetOf(initializing.keySet()).forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, tracker.getCheckpoint()));
|
randomSubsetOf(initializing.keySet()).forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, tracker.getGlobalCheckpoint()));
|
||||||
assigned.forEach(tracker::updateLocalCheckpoint);
|
assigned.forEach(tracker::updateLocalCheckpoint);
|
||||||
|
|
||||||
// now mark all active shards
|
// now mark all active shards
|
||||||
tracker.updateAllocationIdsFromMaster(active.keySet(), initializing.keySet());
|
tracker.updateAllocationIdsFromMaster(active.keySet(), initializing.keySet());
|
||||||
|
|
||||||
// global checkpoint can't be advanced, but we need a sync
|
|
||||||
assertTrue(tracker.updateCheckpointOnPrimary());
|
|
||||||
assertThat(tracker.getCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
|
||||||
|
|
||||||
// update again
|
// update again
|
||||||
assigned.forEach(tracker::updateLocalCheckpoint);
|
assigned.forEach(tracker::updateLocalCheckpoint);
|
||||||
assertTrue(tracker.updateCheckpointOnPrimary());
|
assertThat(tracker.getGlobalCheckpoint(), not(equalTo(UNASSIGNED_SEQ_NO)));
|
||||||
assertThat(tracker.getCheckpoint(), not(equalTo(UNASSIGNED_SEQ_NO)));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testMissingInSyncIdsPreventAdvance() {
|
public void testMissingInSyncIdsPreventAdvance() {
|
||||||
final Map<String, Long> active = randomAllocationsWithLocalCheckpoints(0, 5);
|
final Map<String, Long> active = randomAllocationsWithLocalCheckpoints(0, 5);
|
||||||
final Map<String, Long> initializing = randomAllocationsWithLocalCheckpoints(1, 5);
|
final Map<String, Long> initializing = randomAllocationsWithLocalCheckpoints(1, 5);
|
||||||
tracker.updateAllocationIdsFromMaster(active.keySet(), initializing.keySet());
|
tracker.updateAllocationIdsFromMaster(active.keySet(), initializing.keySet());
|
||||||
initializing.keySet().forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, tracker.getCheckpoint()));
|
initializing.keySet().forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, tracker.getGlobalCheckpoint()));
|
||||||
randomSubsetOf(randomInt(initializing.size() - 1),
|
randomSubsetOf(randomInt(initializing.size() - 1),
|
||||||
initializing.keySet()).forEach(aId -> tracker.updateLocalCheckpoint(aId, initializing.get(aId)));
|
initializing.keySet()).forEach(aId -> tracker.updateLocalCheckpoint(aId, initializing.get(aId)));
|
||||||
|
|
||||||
active.forEach(tracker::updateLocalCheckpoint);
|
active.forEach(tracker::updateLocalCheckpoint);
|
||||||
|
|
||||||
// global checkpoint can't be advanced, but we need a sync
|
assertThat(tracker.getGlobalCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
||||||
assertTrue(tracker.updateCheckpointOnPrimary());
|
|
||||||
assertThat(tracker.getCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
|
||||||
|
|
||||||
// update again
|
// update again
|
||||||
initializing.forEach(tracker::updateLocalCheckpoint);
|
initializing.forEach(tracker::updateLocalCheckpoint);
|
||||||
assertTrue(tracker.updateCheckpointOnPrimary());
|
assertThat(tracker.getGlobalCheckpoint(), not(equalTo(UNASSIGNED_SEQ_NO)));
|
||||||
assertThat(tracker.getCheckpoint(), not(equalTo(UNASSIGNED_SEQ_NO)));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testInSyncIdsAreIgnoredIfNotValidatedByMaster() {
|
public void testInSyncIdsAreIgnoredIfNotValidatedByMaster() {
|
||||||
|
@ -197,16 +182,14 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
||||||
final Map<String, Long> initializing = randomAllocationsWithLocalCheckpoints(1, 5);
|
final Map<String, Long> initializing = randomAllocationsWithLocalCheckpoints(1, 5);
|
||||||
final Map<String, Long> nonApproved = randomAllocationsWithLocalCheckpoints(1, 5);
|
final Map<String, Long> nonApproved = randomAllocationsWithLocalCheckpoints(1, 5);
|
||||||
tracker.updateAllocationIdsFromMaster(active.keySet(), initializing.keySet());
|
tracker.updateAllocationIdsFromMaster(active.keySet(), initializing.keySet());
|
||||||
initializing.keySet().forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, tracker.getCheckpoint()));
|
initializing.keySet().forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, tracker.getGlobalCheckpoint()));
|
||||||
nonApproved.keySet().forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, tracker.getCheckpoint()));
|
nonApproved.keySet().forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, tracker.getGlobalCheckpoint()));
|
||||||
|
|
||||||
List<Map<String, Long>> allocations = Arrays.asList(active, initializing, nonApproved);
|
List<Map<String, Long>> allocations = Arrays.asList(active, initializing, nonApproved);
|
||||||
Collections.shuffle(allocations, random());
|
Collections.shuffle(allocations, random());
|
||||||
allocations.forEach(a -> a.forEach(tracker::updateLocalCheckpoint));
|
allocations.forEach(a -> a.forEach(tracker::updateLocalCheckpoint));
|
||||||
|
|
||||||
// global checkpoint can be advanced, but we need a sync
|
assertThat(tracker.getGlobalCheckpoint(), not(equalTo(UNASSIGNED_SEQ_NO)));
|
||||||
assertTrue(tracker.updateCheckpointOnPrimary());
|
|
||||||
assertThat(tracker.getCheckpoint(), not(equalTo(UNASSIGNED_SEQ_NO)));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testInSyncIdsAreRemovedIfNotValidatedByMaster() {
|
public void testInSyncIdsAreRemovedIfNotValidatedByMaster() {
|
||||||
|
@ -227,17 +210,14 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
tracker.updateAllocationIdsFromMaster(active, initializing);
|
tracker.updateAllocationIdsFromMaster(active, initializing);
|
||||||
if (randomBoolean()) {
|
if (randomBoolean()) {
|
||||||
initializingToStay.keySet().forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, tracker.getCheckpoint()));
|
initializingToStay.keySet().forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, tracker.getGlobalCheckpoint()));
|
||||||
} else {
|
} else {
|
||||||
initializing.forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, tracker.getCheckpoint()));
|
initializing.forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, tracker.getGlobalCheckpoint()));
|
||||||
}
|
}
|
||||||
if (randomBoolean()) {
|
if (randomBoolean()) {
|
||||||
allocations.forEach(tracker::updateLocalCheckpoint);
|
allocations.forEach(tracker::updateLocalCheckpoint);
|
||||||
}
|
}
|
||||||
|
|
||||||
// global checkpoint may be advanced, but we need a sync in any case
|
|
||||||
assertTrue(tracker.updateCheckpointOnPrimary());
|
|
||||||
|
|
||||||
// now remove shards
|
// now remove shards
|
||||||
if (randomBoolean()) {
|
if (randomBoolean()) {
|
||||||
tracker.updateAllocationIdsFromMaster(activeToStay.keySet(), initializingToStay.keySet());
|
tracker.updateAllocationIdsFromMaster(activeToStay.keySet(), initializingToStay.keySet());
|
||||||
|
@ -250,9 +230,7 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
||||||
final long checkpoint = Stream.concat(activeToStay.values().stream(), initializingToStay.values().stream())
|
final long checkpoint = Stream.concat(activeToStay.values().stream(), initializingToStay.values().stream())
|
||||||
.min(Long::compare).get() + 10; // we added 10 to make sure it's advanced in the second time
|
.min(Long::compare).get() + 10; // we added 10 to make sure it's advanced in the second time
|
||||||
|
|
||||||
// global checkpoint is advanced and we need a sync
|
assertThat(tracker.getGlobalCheckpoint(), equalTo(checkpoint));
|
||||||
assertTrue(tracker.updateCheckpointOnPrimary());
|
|
||||||
assertThat(tracker.getCheckpoint(), equalTo(checkpoint));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testWaitForAllocationIdToBeInSync() throws BrokenBarrierException, InterruptedException {
|
public void testWaitForAllocationIdToBeInSync() throws BrokenBarrierException, InterruptedException {
|
||||||
|
@ -264,7 +242,6 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
||||||
final String trackingAllocationId = randomAlphaOfLength(16);
|
final String trackingAllocationId = randomAlphaOfLength(16);
|
||||||
tracker.updateAllocationIdsFromMaster(Collections.singleton(inSyncAllocationId), Collections.singleton(trackingAllocationId));
|
tracker.updateAllocationIdsFromMaster(Collections.singleton(inSyncAllocationId), Collections.singleton(trackingAllocationId));
|
||||||
tracker.updateLocalCheckpoint(inSyncAllocationId, globalCheckpoint);
|
tracker.updateLocalCheckpoint(inSyncAllocationId, globalCheckpoint);
|
||||||
tracker.updateCheckpointOnPrimary();
|
|
||||||
final Thread thread = new Thread(() -> {
|
final Thread thread = new Thread(() -> {
|
||||||
try {
|
try {
|
||||||
// synchronize starting with the test thread
|
// synchronize starting with the test thread
|
||||||
|
@ -313,7 +290,6 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
||||||
final String trackingAllocationId = randomAlphaOfLength(32);
|
final String trackingAllocationId = randomAlphaOfLength(32);
|
||||||
tracker.updateAllocationIdsFromMaster(Collections.singleton(inSyncAllocationId), Collections.singleton(trackingAllocationId));
|
tracker.updateAllocationIdsFromMaster(Collections.singleton(inSyncAllocationId), Collections.singleton(trackingAllocationId));
|
||||||
tracker.updateLocalCheckpoint(inSyncAllocationId, globalCheckpoint);
|
tracker.updateLocalCheckpoint(inSyncAllocationId, globalCheckpoint);
|
||||||
tracker.updateCheckpointOnPrimary();
|
|
||||||
final Thread thread = new Thread(() -> {
|
final Thread thread = new Thread(() -> {
|
||||||
try {
|
try {
|
||||||
// synchronize starting with the test thread
|
// synchronize starting with the test thread
|
||||||
|
@ -426,9 +402,8 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
||||||
.entrySet()
|
.entrySet()
|
||||||
.stream()
|
.stream()
|
||||||
.allMatch(e -> tracker.trackingLocalCheckpoints.get(e.getKey()) == e.getValue()));
|
.allMatch(e -> tracker.trackingLocalCheckpoints.get(e.getKey()) == e.getValue()));
|
||||||
assertTrue(tracker.updateCheckpointOnPrimary());
|
|
||||||
final long minimumActiveLocalCheckpoint = (long) activeLocalCheckpoints.values().stream().min(Integer::compareTo).get();
|
final long minimumActiveLocalCheckpoint = (long) activeLocalCheckpoints.values().stream().min(Integer::compareTo).get();
|
||||||
assertThat(tracker.getCheckpoint(), equalTo(minimumActiveLocalCheckpoint));
|
assertThat(tracker.getGlobalCheckpoint(), equalTo(minimumActiveLocalCheckpoint));
|
||||||
final long minimumInitailizingLocalCheckpoint = (long) initializingLocalCheckpoints.values().stream().min(Integer::compareTo).get();
|
final long minimumInitailizingLocalCheckpoint = (long) initializingLocalCheckpoints.values().stream().min(Integer::compareTo).get();
|
||||||
|
|
||||||
// now we are going to add a new allocation ID and bring it in sync which should move it to the in-sync allocation IDs
|
// now we are going to add a new allocation ID and bring it in sync which should move it to the in-sync allocation IDs
|
||||||
|
@ -477,6 +452,69 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
||||||
assertTrue(tracker.inSyncLocalCheckpoints.containsKey(newSyncingAllocationId));
|
assertTrue(tracker.inSyncLocalCheckpoints.containsKey(newSyncingAllocationId));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* If we do not update the global checkpoint in {@link GlobalCheckpointTracker#markAllocationIdAsInSync(String, long)} after adding the
|
||||||
|
* allocation ID to the in-sync set and removing it from pending, the local checkpoint update that freed the thread waiting for the
|
||||||
|
* local checkpoint to advance could miss updating the global checkpoint in a race if the the waiting thread did not add the allocation
|
||||||
|
* ID to the in-sync set and remove it from the pending set before the local checkpoint updating thread executed the global checkpoint
|
||||||
|
* update. This test fails without an additional call to {@link GlobalCheckpointTracker#updateGlobalCheckpointOnPrimary()} after
|
||||||
|
* removing the allocation ID from the pending set in {@link GlobalCheckpointTracker#markAllocationIdAsInSync(String, long)} (even if a
|
||||||
|
* call is added after notifying all waiters in {@link GlobalCheckpointTracker#updateLocalCheckpoint(String, long)}).
|
||||||
|
*
|
||||||
|
* @throws InterruptedException if the main test thread was interrupted while waiting
|
||||||
|
* @throws BrokenBarrierException if the barrier was broken while the main test thread was waiting
|
||||||
|
*/
|
||||||
|
public void testRaceUpdatingGlobalCheckpoint() throws InterruptedException, BrokenBarrierException {
|
||||||
|
|
||||||
|
final String active = randomAlphaOfLength(16);
|
||||||
|
final String initializing = randomAlphaOfLength(32);
|
||||||
|
tracker.updateAllocationIdsFromMaster(Collections.singleton(active), Collections.singleton(initializing));
|
||||||
|
|
||||||
|
final CyclicBarrier barrier = new CyclicBarrier(4);
|
||||||
|
|
||||||
|
final int activeLocalCheckpoint = randomIntBetween(0, Integer.MAX_VALUE - 1);
|
||||||
|
tracker.updateLocalCheckpoint(active, activeLocalCheckpoint);
|
||||||
|
final int nextActiveLocalCheckpoint = randomIntBetween(activeLocalCheckpoint + 1, Integer.MAX_VALUE);
|
||||||
|
final Thread activeThread = new Thread(() -> {
|
||||||
|
try {
|
||||||
|
barrier.await();
|
||||||
|
} catch (final BrokenBarrierException | InterruptedException e) {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
tracker.updateLocalCheckpoint(active, nextActiveLocalCheckpoint);
|
||||||
|
});
|
||||||
|
|
||||||
|
final int initializingLocalCheckpoint = randomIntBetween(0, nextActiveLocalCheckpoint - 1);
|
||||||
|
final Thread initializingThread = new Thread(() -> {
|
||||||
|
try {
|
||||||
|
barrier.await();
|
||||||
|
} catch (final BrokenBarrierException | InterruptedException e) {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
tracker.updateLocalCheckpoint(initializing, nextActiveLocalCheckpoint);
|
||||||
|
});
|
||||||
|
|
||||||
|
final Thread markingThread = new Thread(() -> {
|
||||||
|
try {
|
||||||
|
barrier.await();
|
||||||
|
tracker.markAllocationIdAsInSync(initializing, initializingLocalCheckpoint - 1);
|
||||||
|
} catch (final BrokenBarrierException | InterruptedException e) {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
activeThread.start();
|
||||||
|
initializingThread.start();
|
||||||
|
markingThread.start();
|
||||||
|
barrier.await();
|
||||||
|
|
||||||
|
activeThread.join();
|
||||||
|
initializingThread.join();
|
||||||
|
markingThread.join();
|
||||||
|
|
||||||
|
assertThat(tracker.getGlobalCheckpoint(), equalTo((long) nextActiveLocalCheckpoint));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
private void markAllocationIdAsInSyncQuietly(
|
private void markAllocationIdAsInSyncQuietly(
|
||||||
final GlobalCheckpointTracker tracker, final String allocationId, final long localCheckpoint) {
|
final GlobalCheckpointTracker tracker, final String allocationId, final long localCheckpoint) {
|
||||||
|
|
|
@ -161,21 +161,23 @@ public class IndexShardIT extends ESSingleNodeTestCase {
|
||||||
assertNotNull(indexStats.getShards()[0].getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID));
|
assertNotNull(indexStats.getShards()[0].getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID));
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testDurableFlagHasEffect() {
|
public void testDurableFlagHasEffect() throws Exception {
|
||||||
createIndex("test");
|
createIndex("test");
|
||||||
ensureGreen();
|
ensureGreen();
|
||||||
client().prepareIndex("test", "bar", "1").setSource("{}", XContentType.JSON).get();
|
client().prepareIndex("test", "bar", "1").setSource("{}", XContentType.JSON).get();
|
||||||
IndicesService indicesService = getInstanceFromNode(IndicesService.class);
|
IndicesService indicesService = getInstanceFromNode(IndicesService.class);
|
||||||
IndexService test = indicesService.indexService(resolveIndex("test"));
|
IndexService test = indicesService.indexService(resolveIndex("test"));
|
||||||
IndexShard shard = test.getShardOrNull(0);
|
IndexShard shard = test.getShardOrNull(0);
|
||||||
|
shard.checkIdle(Long.MIN_VALUE);
|
||||||
setDurability(shard, Translog.Durability.REQUEST);
|
setDurability(shard, Translog.Durability.REQUEST);
|
||||||
assertFalse(ShardUtilsTests.getShardEngine(shard).getTranslog().syncNeeded());
|
assertBusy(() -> assertFalse(ShardUtilsTests.getShardEngine(shard).getTranslog().syncNeeded()));
|
||||||
setDurability(shard, Translog.Durability.ASYNC);
|
setDurability(shard, Translog.Durability.ASYNC);
|
||||||
client().prepareIndex("test", "bar", "2").setSource("{}", XContentType.JSON).get();
|
client().prepareIndex("test", "bar", "2").setSource("{}", XContentType.JSON).get();
|
||||||
assertTrue(ShardUtilsTests.getShardEngine(shard).getTranslog().syncNeeded());
|
assertTrue(ShardUtilsTests.getShardEngine(shard).getTranslog().syncNeeded());
|
||||||
setDurability(shard, Translog.Durability.REQUEST);
|
setDurability(shard, Translog.Durability.REQUEST);
|
||||||
client().prepareDelete("test", "bar", "1").get();
|
client().prepareDelete("test", "bar", "1").get();
|
||||||
assertFalse(ShardUtilsTests.getShardEngine(shard).getTranslog().syncNeeded());
|
shard.checkIdle(Long.MIN_VALUE);
|
||||||
|
assertBusy(() -> assertFalse(ShardUtilsTests.getShardEngine(shard).getTranslog().syncNeeded()));
|
||||||
|
|
||||||
setDurability(shard, Translog.Durability.ASYNC);
|
setDurability(shard, Translog.Durability.ASYNC);
|
||||||
client().prepareDelete("test", "bar", "2").get();
|
client().prepareDelete("test", "bar", "2").get();
|
||||||
|
@ -184,7 +186,8 @@ public class IndexShardIT extends ESSingleNodeTestCase {
|
||||||
assertNoFailures(client().prepareBulk()
|
assertNoFailures(client().prepareBulk()
|
||||||
.add(client().prepareIndex("test", "bar", "3").setSource("{}", XContentType.JSON))
|
.add(client().prepareIndex("test", "bar", "3").setSource("{}", XContentType.JSON))
|
||||||
.add(client().prepareDelete("test", "bar", "1")).get());
|
.add(client().prepareDelete("test", "bar", "1")).get());
|
||||||
assertFalse(ShardUtilsTests.getShardEngine(shard).getTranslog().syncNeeded());
|
shard.checkIdle(Long.MIN_VALUE);
|
||||||
|
assertBusy(() -> assertFalse(ShardUtilsTests.getShardEngine(shard).getTranslog().syncNeeded()));
|
||||||
|
|
||||||
setDurability(shard, Translog.Durability.ASYNC);
|
setDurability(shard, Translog.Durability.ASYNC);
|
||||||
assertNoFailures(client().prepareBulk()
|
assertNoFailures(client().prepareBulk()
|
||||||
|
@ -532,7 +535,7 @@ public class IndexShardIT extends ESSingleNodeTestCase {
|
||||||
IndexShard newShard = new IndexShard(initializingShardRouting, indexService.getIndexSettings(), shard.shardPath(),
|
IndexShard newShard = new IndexShard(initializingShardRouting, indexService.getIndexSettings(), shard.shardPath(),
|
||||||
shard.store(), indexService.getIndexSortSupplier(), indexService.cache(), indexService.mapperService(), indexService.similarityService(),
|
shard.store(), indexService.getIndexSortSupplier(), indexService.cache(), indexService.mapperService(), indexService.similarityService(),
|
||||||
indexService.fieldData(), shard.getEngineFactory(), indexService.getIndexEventListener(), wrapper,
|
indexService.fieldData(), shard.getEngineFactory(), indexService.getIndexEventListener(), wrapper,
|
||||||
indexService.getThreadPool(), indexService.getBigArrays(), null, () -> {}, Collections.emptyList(), Arrays.asList(listeners));
|
indexService.getThreadPool(), indexService.getBigArrays(), null, Collections.emptyList(), Arrays.asList(listeners));
|
||||||
return newShard;
|
return newShard;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -1102,7 +1102,7 @@ public class IndexShardTests extends IndexShardTestCase {
|
||||||
};
|
};
|
||||||
closeShards(shard);
|
closeShards(shard);
|
||||||
IndexShard newShard = newShard(ShardRoutingHelper.reinitPrimary(shard.routingEntry()),
|
IndexShard newShard = newShard(ShardRoutingHelper.reinitPrimary(shard.routingEntry()),
|
||||||
shard.shardPath(), shard.indexSettings().getIndexMetaData(), wrapper, () -> {}, null);
|
shard.shardPath(), shard.indexSettings().getIndexMetaData(), wrapper, null);
|
||||||
|
|
||||||
recoveryShardFromStore(newShard);
|
recoveryShardFromStore(newShard);
|
||||||
|
|
||||||
|
@ -1243,7 +1243,7 @@ public class IndexShardTests extends IndexShardTestCase {
|
||||||
|
|
||||||
closeShards(shard);
|
closeShards(shard);
|
||||||
IndexShard newShard = newShard(ShardRoutingHelper.reinitPrimary(shard.routingEntry()),
|
IndexShard newShard = newShard(ShardRoutingHelper.reinitPrimary(shard.routingEntry()),
|
||||||
shard.shardPath(), shard.indexSettings().getIndexMetaData(), wrapper, () -> {}, null);
|
shard.shardPath(), shard.indexSettings().getIndexMetaData(), wrapper, null);
|
||||||
|
|
||||||
recoveryShardFromStore(newShard);
|
recoveryShardFromStore(newShard);
|
||||||
|
|
||||||
|
|
|
@ -120,7 +120,7 @@ public class IndicesLifecycleListenerSingleNodeTests extends ESSingleNodeTestCas
|
||||||
};
|
};
|
||||||
indicesService.removeIndex(idx, DELETED, "simon says");
|
indicesService.removeIndex(idx, DELETED, "simon says");
|
||||||
try {
|
try {
|
||||||
IndexService index = indicesService.createIndex(metaData, Arrays.asList(countingListener), s -> {});
|
IndexService index = indicesService.createIndex(metaData, Arrays.asList(countingListener));
|
||||||
assertEquals(3, counter.get());
|
assertEquals(3, counter.get());
|
||||||
idx = index.index();
|
idx = index.index();
|
||||||
ShardRouting newRouting = shardRouting;
|
ShardRouting newRouting = shardRouting;
|
||||||
|
|
|
@ -52,13 +52,11 @@ import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.ConcurrentMap;
|
import java.util.concurrent.ConcurrentMap;
|
||||||
import java.util.function.Consumer;
|
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import static java.util.Collections.emptyMap;
|
import static java.util.Collections.emptyMap;
|
||||||
import static java.util.Collections.unmodifiableMap;
|
import static java.util.Collections.unmodifiableMap;
|
||||||
import static org.elasticsearch.common.collect.MapBuilder.newMapBuilder;
|
import static org.elasticsearch.common.collect.MapBuilder.newMapBuilder;
|
||||||
import static org.hamcrest.Matchers.empty;
|
|
||||||
import static org.hamcrest.Matchers.equalTo;
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
||||||
|
|
||||||
|
@ -191,9 +189,8 @@ public abstract class AbstractIndicesClusterStateServiceTestCase extends ESTestC
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public synchronized MockIndexService createIndex(
|
public synchronized MockIndexService createIndex(
|
||||||
IndexMetaData indexMetaData,
|
IndexMetaData indexMetaData,
|
||||||
List<IndexEventListener> buildInIndexListener,
|
List<IndexEventListener> buildInIndexListener) throws IOException {
|
||||||
Consumer<ShardId> globalCheckPointSyncer) throws IOException {
|
|
||||||
MockIndexService indexService = new MockIndexService(new IndexSettings(indexMetaData, Settings.EMPTY));
|
MockIndexService indexService = new MockIndexService(new IndexSettings(indexMetaData, Settings.EMPTY));
|
||||||
indices = newMapBuilder(indices).put(indexMetaData.getIndexUUID(), indexService).immutableMap();
|
indices = newMapBuilder(indices).put(indexMetaData.getIndexUUID(), indexService).immutableMap();
|
||||||
return indexService;
|
return indexService;
|
||||||
|
|
|
@ -141,8 +141,7 @@ public class ClusterStateChanges extends AbstractComponent {
|
||||||
// MetaDataCreateIndexService creates indices using its IndicesService instance to check mappings -> fake it here
|
// MetaDataCreateIndexService creates indices using its IndicesService instance to check mappings -> fake it here
|
||||||
try {
|
try {
|
||||||
@SuppressWarnings("unchecked") final List<IndexEventListener> listeners = anyList();
|
@SuppressWarnings("unchecked") final List<IndexEventListener> listeners = anyList();
|
||||||
@SuppressWarnings("unchecked") final Consumer<ShardId> globalCheckpointSyncer = any(Consumer.class);
|
when(indicesService.createIndex(any(IndexMetaData.class), listeners))
|
||||||
when(indicesService.createIndex(any(IndexMetaData.class), listeners, globalCheckpointSyncer))
|
|
||||||
.then(invocationOnMock -> {
|
.then(invocationOnMock -> {
|
||||||
IndexService indexService = mock(IndexService.class);
|
IndexService indexService = mock(IndexService.class);
|
||||||
IndexMetaData indexMetaData = (IndexMetaData)invocationOnMock.getArguments()[0];
|
IndexMetaData indexMetaData = (IndexMetaData)invocationOnMock.getArguments()[0];
|
||||||
|
|
|
@ -420,7 +420,7 @@ public class IndicesClusterStateServiceRandomUpdatesTests extends AbstractIndice
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
shardId -> {});
|
null);
|
||||||
}
|
}
|
||||||
|
|
||||||
private class RecordingIndicesService extends MockIndicesService {
|
private class RecordingIndicesService extends MockIndicesService {
|
||||||
|
|
|
@ -84,7 +84,6 @@ import java.util.concurrent.Semaphore;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.stream.Stream;
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
import static org.elasticsearch.index.IndexSettings.INDEX_SEQ_NO_CHECKPOINT_SYNC_INTERVAL;
|
|
||||||
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
|
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
|
||||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
||||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
|
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
|
||||||
|
@ -104,12 +103,6 @@ public class RelocationIT extends ESIntegTestCase {
|
||||||
return Arrays.asList(MockTransportService.TestPlugin.class, MockIndexEventListener.TestPlugin.class);
|
return Arrays.asList(MockTransportService.TestPlugin.class, MockIndexEventListener.TestPlugin.class);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Settings indexSettings() {
|
|
||||||
return Settings.builder().put(super.indexSettings())
|
|
||||||
.put(INDEX_SEQ_NO_CHECKPOINT_SYNC_INTERVAL.getKey(), "200ms").build();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void beforeIndexDeletion() throws Exception {
|
protected void beforeIndexDeletion() throws Exception {
|
||||||
super.beforeIndexDeletion();
|
super.beforeIndexDeletion();
|
||||||
|
@ -131,7 +124,6 @@ public class RelocationIT extends ESIntegTestCase {
|
||||||
final SeqNoStats seqNoStats = shardStats.getSeqNoStats();
|
final SeqNoStats seqNoStats = shardStats.getSeqNoStats();
|
||||||
assertThat(shardStats.getShardRouting() + " local checkpoint mismatch",
|
assertThat(shardStats.getShardRouting() + " local checkpoint mismatch",
|
||||||
seqNoStats.getLocalCheckpoint(), equalTo(primarySeqNoStats.getLocalCheckpoint()));
|
seqNoStats.getLocalCheckpoint(), equalTo(primarySeqNoStats.getLocalCheckpoint()));
|
||||||
|
|
||||||
assertThat(shardStats.getShardRouting() + " global checkpoint mismatch",
|
assertThat(shardStats.getShardRouting() + " global checkpoint mismatch",
|
||||||
seqNoStats.getGlobalCheckpoint(), equalTo(primarySeqNoStats.getGlobalCheckpoint()));
|
seqNoStats.getGlobalCheckpoint(), equalTo(primarySeqNoStats.getGlobalCheckpoint()));
|
||||||
assertThat(shardStats.getShardRouting() + " max seq no mismatch",
|
assertThat(shardStats.getShardRouting() + " max seq no mismatch",
|
||||||
|
@ -375,6 +367,9 @@ public class RelocationIT extends ESIntegTestCase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// refresh is a replication action so this forces a global checkpoint sync which is needed as these are asserted on in tear down
|
||||||
|
client().admin().indices().prepareRefresh("test").get();
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -18,6 +18,7 @@
|
||||||
*/
|
*/
|
||||||
package org.elasticsearch.script;
|
package org.elasticsearch.script;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.settings.Setting;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.env.Environment;
|
import org.elasticsearch.env.Environment;
|
||||||
import org.elasticsearch.script.MockScriptEngine.MockCompiledScript;
|
import org.elasticsearch.script.MockScriptEngine.MockCompiledScript;
|
||||||
|
@ -59,7 +60,8 @@ public class FileScriptTests extends ESTestCase {
|
||||||
assertNotNull(compiledScript);
|
assertNotNull(compiledScript);
|
||||||
MockCompiledScript executable = (MockCompiledScript) compiledScript.compiled();
|
MockCompiledScript executable = (MockCompiledScript) compiledScript.compiled();
|
||||||
assertEquals("script1.mockscript", executable.getName());
|
assertEquals("script1.mockscript", executable.getName());
|
||||||
assertWarnings("File scripts are deprecated. Use stored or inline scripts instead.");
|
assertSettingDeprecationsAndWarnings(new Setting[] {ScriptService.SCRIPT_AUTO_RELOAD_ENABLED_SETTING},
|
||||||
|
"File scripts are deprecated. Use stored or inline scripts instead.");
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testAllOpsDisabled() throws Exception {
|
public void testAllOpsDisabled() throws Exception {
|
||||||
|
@ -79,6 +81,7 @@ public class FileScriptTests extends ESTestCase {
|
||||||
assertTrue(e.getMessage(), e.getMessage().contains("scripts of type [file], operation [" + context.getKey() + "] and lang [" + MockScriptEngine.NAME + "] are disabled"));
|
assertTrue(e.getMessage(), e.getMessage().contains("scripts of type [file], operation [" + context.getKey() + "] and lang [" + MockScriptEngine.NAME + "] are disabled"));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
assertWarnings("File scripts are deprecated. Use stored or inline scripts instead.");
|
assertSettingDeprecationsAndWarnings(new Setting[] {ScriptService.SCRIPT_AUTO_RELOAD_ENABLED_SETTING},
|
||||||
|
"File scripts are deprecated. Use stored or inline scripts instead.");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -45,7 +45,6 @@ public class NativeScriptTests extends ESTestCase {
|
||||||
Settings settings = Settings.builder()
|
Settings settings = Settings.builder()
|
||||||
.put("node.name", "testNativeScript")
|
.put("node.name", "testNativeScript")
|
||||||
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir())
|
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir())
|
||||||
.put(ScriptService.SCRIPT_AUTO_RELOAD_ENABLED_SETTING.getKey(), false)
|
|
||||||
.build();
|
.build();
|
||||||
ScriptModule scriptModule = new ScriptModule(settings, new Environment(settings), null,
|
ScriptModule scriptModule = new ScriptModule(settings, new Environment(settings), null,
|
||||||
singletonList(new NativeScriptEngineService(settings, singletonMap("my", new MyNativeScriptFactory()))), emptyList());
|
singletonList(new NativeScriptEngineService(settings, singletonMap("my", new MyNativeScriptFactory()))), emptyList());
|
||||||
|
|
|
@ -23,6 +23,7 @@ import org.elasticsearch.cluster.ClusterChangedEvent;
|
||||||
import org.elasticsearch.cluster.ClusterName;
|
import org.elasticsearch.cluster.ClusterName;
|
||||||
import org.elasticsearch.cluster.ClusterState;
|
import org.elasticsearch.cluster.ClusterState;
|
||||||
import org.elasticsearch.cluster.metadata.MetaData;
|
import org.elasticsearch.cluster.metadata.MetaData;
|
||||||
|
import org.elasticsearch.common.settings.Setting;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.env.Environment;
|
import org.elasticsearch.env.Environment;
|
||||||
import org.elasticsearch.test.ESTestCase;
|
import org.elasticsearch.test.ESTestCase;
|
||||||
|
@ -77,6 +78,7 @@ public class ScriptContextTests extends ESTestCase {
|
||||||
assertThat(e.getMessage(), containsString("scripts of type [" + scriptType + "], operation [" + PLUGIN_NAME + "_custom_globally_disabled_op] and lang [" + MockScriptEngine.NAME + "] are disabled"));
|
assertThat(e.getMessage(), containsString("scripts of type [" + scriptType + "], operation [" + PLUGIN_NAME + "_custom_globally_disabled_op] and lang [" + MockScriptEngine.NAME + "] are disabled"));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
assertSettingDeprecationsAndWarnings(new Setting[] {ScriptService.SCRIPT_AUTO_RELOAD_ENABLED_SETTING});
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testCustomScriptContextSettings() throws Exception {
|
public void testCustomScriptContextSettings() throws Exception {
|
||||||
|
@ -93,6 +95,7 @@ public class ScriptContextTests extends ESTestCase {
|
||||||
assertNotNull(scriptService.compile(script, ScriptContext.Standard.AGGS));
|
assertNotNull(scriptService.compile(script, ScriptContext.Standard.AGGS));
|
||||||
assertNotNull(scriptService.compile(script, ScriptContext.Standard.SEARCH));
|
assertNotNull(scriptService.compile(script, ScriptContext.Standard.SEARCH));
|
||||||
assertNotNull(scriptService.compile(script, new ScriptContext.Plugin(PLUGIN_NAME, "custom_op")));
|
assertNotNull(scriptService.compile(script, new ScriptContext.Plugin(PLUGIN_NAME, "custom_op")));
|
||||||
|
assertSettingDeprecationsAndWarnings(new Setting[] {ScriptService.SCRIPT_AUTO_RELOAD_ENABLED_SETTING});
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testUnknownPluginScriptContext() throws Exception {
|
public void testUnknownPluginScriptContext() throws Exception {
|
||||||
|
@ -106,6 +109,7 @@ public class ScriptContextTests extends ESTestCase {
|
||||||
assertTrue(e.getMessage(), e.getMessage().contains("script context [" + PLUGIN_NAME + "_unknown] not supported"));
|
assertTrue(e.getMessage(), e.getMessage().contains("script context [" + PLUGIN_NAME + "_unknown] not supported"));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
assertSettingDeprecationsAndWarnings(new Setting[] {ScriptService.SCRIPT_AUTO_RELOAD_ENABLED_SETTING});
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testUnknownCustomScriptContext() throws Exception {
|
public void testUnknownCustomScriptContext() throws Exception {
|
||||||
|
@ -125,6 +129,7 @@ public class ScriptContextTests extends ESTestCase {
|
||||||
assertTrue(e.getMessage(), e.getMessage().contains("script context [test] not supported"));
|
assertTrue(e.getMessage(), e.getMessage().contains("script context [test] not supported"));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
assertSettingDeprecationsAndWarnings(new Setting[] {ScriptService.SCRIPT_AUTO_RELOAD_ENABLED_SETTING});
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -52,8 +52,7 @@ public class AggregatorFactoriesTests extends ESTestCase {
|
||||||
// we have to prefer CURRENT since with the range of versions we support
|
// we have to prefer CURRENT since with the range of versions we support
|
||||||
// it's rather unlikely to get the current actually.
|
// it's rather unlikely to get the current actually.
|
||||||
Settings settings = Settings.builder().put("node.name", AbstractQueryTestCase.class.toString())
|
Settings settings = Settings.builder().put("node.name", AbstractQueryTestCase.class.toString())
|
||||||
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir())
|
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()).build();
|
||||||
.put(ScriptService.SCRIPT_AUTO_RELOAD_ENABLED_SETTING.getKey(), false).build();
|
|
||||||
// create some random type with some default field, those types will
|
// create some random type with some default field, those types will
|
||||||
// stick around for all of the subclasses
|
// stick around for all of the subclasses
|
||||||
currentTypes = new String[randomIntBetween(0, 5)];
|
currentTypes = new String[randomIntBetween(0, 5)];
|
||||||
|
|
|
@ -69,8 +69,6 @@ public class InternalScriptedMetricTests extends InternalAggregationTestCase<Int
|
||||||
protected ScriptService mockScriptService() {
|
protected ScriptService mockScriptService() {
|
||||||
Settings settings = Settings.builder()
|
Settings settings = Settings.builder()
|
||||||
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir())
|
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir())
|
||||||
// no file watching, so we don't need a ResourceWatcherService
|
|
||||||
.put(ScriptService.SCRIPT_AUTO_RELOAD_ENABLED_SETTING.getKey(), "false")
|
|
||||||
.build();
|
.build();
|
||||||
// mock script always retuns the size of the input aggs list as result
|
// mock script always retuns the size of the input aggs list as result
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
|
|
|
@ -198,9 +198,7 @@ public class ScriptedMetricAggregatorTests extends AggregatorTestCase {
|
||||||
@Override
|
@Override
|
||||||
protected QueryShardContext queryShardContextMock(MapperService mapperService, final MappedFieldType[] fieldTypes,
|
protected QueryShardContext queryShardContextMock(MapperService mapperService, final MappedFieldType[] fieldTypes,
|
||||||
CircuitBreakerService circuitBreakerService) {
|
CircuitBreakerService circuitBreakerService) {
|
||||||
Settings settings = Settings.builder().put(Environment.PATH_HOME_SETTING.getKey(), createTempDir())
|
Settings settings = Settings.builder().put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()).build();
|
||||||
// no file watching, so we don't need a ResourceWatcherService
|
|
||||||
.put(ScriptService.SCRIPT_AUTO_RELOAD_ENABLED_SETTING.getKey(), "false").build();
|
|
||||||
MockScriptEngine scriptEngine = new MockScriptEngine(MockScriptEngine.NAME, SCRIPTS);
|
MockScriptEngine scriptEngine = new MockScriptEngine(MockScriptEngine.NAME, SCRIPTS);
|
||||||
ScriptEngineRegistry scriptEngineRegistry = new ScriptEngineRegistry(Collections.singletonList(scriptEngine));
|
ScriptEngineRegistry scriptEngineRegistry = new ScriptEngineRegistry(Collections.singletonList(scriptEngine));
|
||||||
ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(Collections.emptyList());
|
ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(Collections.emptyList());
|
||||||
|
|
|
@ -23,8 +23,11 @@ import org.elasticsearch.cluster.NamedDiff;
|
||||||
import org.elasticsearch.cluster.metadata.MetaData;
|
import org.elasticsearch.cluster.metadata.MetaData;
|
||||||
import org.elasticsearch.common.io.stream.StreamInput;
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
import org.elasticsearch.common.network.NetworkModule;
|
import org.elasticsearch.common.network.NetworkModule;
|
||||||
|
import org.elasticsearch.common.settings.Setting;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.util.set.Sets;
|
import org.elasticsearch.common.util.set.Sets;
|
||||||
|
import org.elasticsearch.env.Environment;
|
||||||
|
import org.elasticsearch.script.ScriptService;
|
||||||
import org.elasticsearch.test.ESTestCase;
|
import org.elasticsearch.test.ESTestCase;
|
||||||
import org.elasticsearch.test.TestCustomMetaData;
|
import org.elasticsearch.test.TestCustomMetaData;
|
||||||
|
|
||||||
|
@ -76,6 +79,7 @@ public class TribeServiceTests extends ESTestCase {
|
||||||
TribeService.buildClientSettings("tribe1", "parent_id", globalSettings, tribeSettings);
|
TribeService.buildClientSettings("tribe1", "parent_id", globalSettings, tribeSettings);
|
||||||
});
|
});
|
||||||
assertTrue(e.getMessage(), e.getMessage().contains("Setting [path.home] not allowed in tribe client"));
|
assertTrue(e.getMessage(), e.getMessage().contains("Setting [path.home] not allowed in tribe client"));
|
||||||
|
assertSettingDeprecationsAndWarnings(new Setting[] {Environment.PATH_SCRIPTS_SETTING});
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testPassthroughSettings() {
|
public void testPassthroughSettings() {
|
||||||
|
|
|
@ -3,11 +3,11 @@
|
||||||
SETLOCAL enabledelayedexpansion
|
SETLOCAL enabledelayedexpansion
|
||||||
|
|
||||||
IF DEFINED JAVA_HOME (
|
IF DEFINED JAVA_HOME (
|
||||||
set JAVA=%JAVA_HOME%\bin\java.exe
|
set JAVA="%JAVA_HOME%\bin\java.exe"
|
||||||
) ELSE (
|
) ELSE (
|
||||||
FOR %%I IN (java.exe) DO set JAVA=%%~$PATH:I
|
FOR %%I IN (java.exe) DO set JAVA=%%~$PATH:I
|
||||||
)
|
)
|
||||||
IF NOT EXIST "%JAVA%" (
|
IF NOT EXIST %JAVA% (
|
||||||
ECHO Could not find any executable java binary. Please install java in your PATH or set JAVA_HOME 1>&2
|
ECHO Could not find any executable java binary. Please install java in your PATH or set JAVA_HOME 1>&2
|
||||||
EXIT /B 1
|
EXIT /B 1
|
||||||
)
|
)
|
||||||
|
@ -25,6 +25,6 @@ IF DEFINED CONF_DIR (
|
||||||
SET args=%*
|
SET args=%*
|
||||||
SET HOSTNAME=%COMPUTERNAME%
|
SET HOSTNAME=%COMPUTERNAME%
|
||||||
|
|
||||||
"%JAVA%" %ES_JAVA_OPTS% !path_props! -cp "%ES_HOME%/lib/*;" "org.elasticsearch.plugins.PluginCli" !args!
|
%JAVA% %ES_JAVA_OPTS% !path_props! -cp "%ES_HOME%/lib/*;" "org.elasticsearch.plugins.PluginCli" !args!
|
||||||
|
|
||||||
ENDLOCAL
|
ENDLOCAL
|
||||||
|
|
|
@ -33,7 +33,7 @@ A `extended_stats_bucket` aggregation looks like this in isolation:
|
||||||
|`sigma` |The number of standard deviations above/below the mean to display |Optional | 2
|
|`sigma` |The number of standard deviations above/below the mean to display |Optional | 2
|
||||||
|===
|
|===
|
||||||
|
|
||||||
The following snippet calculates the sum of all the total monthly `sales` buckets:
|
The following snippet calculates the extended stats for monthly `sales` bucket:
|
||||||
|
|
||||||
[source,js]
|
[source,js]
|
||||||
--------------------------------------------------
|
--------------------------------------------------
|
||||||
|
|
|
@ -31,7 +31,7 @@ A `percentiles_bucket` aggregation looks like this in isolation:
|
||||||
|`percents` |The list of percentiles to calculate |Optional | `[ 1, 5, 25, 50, 75, 95, 99 ]`
|
|`percents` |The list of percentiles to calculate |Optional | `[ 1, 5, 25, 50, 75, 95, 99 ]`
|
||||||
|===
|
|===
|
||||||
|
|
||||||
The following snippet calculates the sum of all the total monthly `sales` buckets:
|
The following snippet calculates the percentiles for the total monthly `sales` buckets:
|
||||||
|
|
||||||
[source,js]
|
[source,js]
|
||||||
--------------------------------------------------
|
--------------------------------------------------
|
||||||
|
|
|
@ -30,7 +30,7 @@ A `stats_bucket` aggregation looks like this in isolation:
|
||||||
|`format` |format to apply to the output value of this aggregation |Optional | `null`
|
|`format` |format to apply to the output value of this aggregation |Optional | `null`
|
||||||
|===
|
|===
|
||||||
|
|
||||||
The following snippet calculates the sum of all the total monthly `sales` buckets:
|
The following snippet calculates the stats for monthly `sales`:
|
||||||
|
|
||||||
[source,js]
|
[source,js]
|
||||||
--------------------------------------------------
|
--------------------------------------------------
|
||||||
|
|
|
@ -19,14 +19,35 @@
|
||||||
|
|
||||||
package org.elasticsearch.analysis.common;
|
package org.elasticsearch.analysis.common;
|
||||||
|
|
||||||
|
import org.apache.lucene.analysis.CharArraySet;
|
||||||
|
import org.apache.lucene.analysis.StopFilter;
|
||||||
|
import org.apache.lucene.analysis.commongrams.CommonGramsFilter;
|
||||||
|
import org.apache.lucene.analysis.core.StopAnalyzer;
|
||||||
|
import org.apache.lucene.analysis.core.UpperCaseFilter;
|
||||||
|
import org.apache.lucene.analysis.en.KStemFilter;
|
||||||
|
import org.apache.lucene.analysis.en.PorterStemFilter;
|
||||||
|
import org.apache.lucene.analysis.miscellaneous.ASCIIFoldingFilter;
|
||||||
|
import org.apache.lucene.analysis.miscellaneous.LengthFilter;
|
||||||
|
import org.apache.lucene.analysis.miscellaneous.TrimFilter;
|
||||||
|
import org.apache.lucene.analysis.miscellaneous.TruncateTokenFilter;
|
||||||
|
import org.apache.lucene.analysis.miscellaneous.UniqueTokenFilter;
|
||||||
|
import org.apache.lucene.analysis.miscellaneous.WordDelimiterFilter;
|
||||||
|
import org.apache.lucene.analysis.miscellaneous.WordDelimiterGraphFilter;
|
||||||
|
import org.apache.lucene.analysis.ngram.EdgeNGramTokenFilter;
|
||||||
|
import org.apache.lucene.analysis.ngram.NGramTokenFilter;
|
||||||
|
import org.apache.lucene.analysis.reverse.ReverseStringFilter;
|
||||||
|
import org.apache.lucene.analysis.standard.ClassicFilter;
|
||||||
import org.elasticsearch.index.analysis.CharFilterFactory;
|
import org.elasticsearch.index.analysis.CharFilterFactory;
|
||||||
import org.elasticsearch.index.analysis.HtmlStripCharFilterFactory;
|
import org.elasticsearch.index.analysis.HtmlStripCharFilterFactory;
|
||||||
|
import org.elasticsearch.index.analysis.PreConfiguredTokenFilter;
|
||||||
import org.elasticsearch.index.analysis.TokenFilterFactory;
|
import org.elasticsearch.index.analysis.TokenFilterFactory;
|
||||||
import org.elasticsearch.indices.analysis.AnalysisModule.AnalysisProvider;
|
import org.elasticsearch.indices.analysis.AnalysisModule.AnalysisProvider;
|
||||||
|
import org.elasticsearch.indices.analysis.PreBuiltCacheFactory.CachingStrategy;
|
||||||
import org.elasticsearch.plugins.AnalysisPlugin;
|
import org.elasticsearch.plugins.AnalysisPlugin;
|
||||||
import org.elasticsearch.plugins.Plugin;
|
import org.elasticsearch.plugins.Plugin;
|
||||||
|
|
||||||
import java.util.HashMap;
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.TreeMap;
|
import java.util.TreeMap;
|
||||||
|
|
||||||
|
@ -35,14 +56,13 @@ import static org.elasticsearch.plugins.AnalysisPlugin.requriesAnalysisSettings;
|
||||||
public class CommonAnalysisPlugin extends Plugin implements AnalysisPlugin {
|
public class CommonAnalysisPlugin extends Plugin implements AnalysisPlugin {
|
||||||
@Override
|
@Override
|
||||||
public Map<String, AnalysisProvider<TokenFilterFactory>> getTokenFilters() {
|
public Map<String, AnalysisProvider<TokenFilterFactory>> getTokenFilters() {
|
||||||
Map<String, AnalysisProvider<TokenFilterFactory>> filters = new HashMap<>();
|
Map<String, AnalysisProvider<TokenFilterFactory>> filters = new TreeMap<>();
|
||||||
filters.put("asciifolding", ASCIIFoldingTokenFilterFactory::new);
|
filters.put("asciifolding", ASCIIFoldingTokenFilterFactory::new);
|
||||||
filters.put("word_delimiter", WordDelimiterTokenFilterFactory::new);
|
filters.put("word_delimiter", WordDelimiterTokenFilterFactory::new);
|
||||||
filters.put("word_delimiter_graph", WordDelimiterGraphTokenFilterFactory::new);
|
filters.put("word_delimiter_graph", WordDelimiterGraphTokenFilterFactory::new);
|
||||||
return filters;
|
return filters;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Map<String, AnalysisProvider<CharFilterFactory>> getCharFilters() {
|
public Map<String, AnalysisProvider<CharFilterFactory>> getCharFilters() {
|
||||||
Map<String, AnalysisProvider<CharFilterFactory>> filters = new TreeMap<>();
|
Map<String, AnalysisProvider<CharFilterFactory>> filters = new TreeMap<>();
|
||||||
filters.put("html_strip", HtmlStripCharFilterFactory::new);
|
filters.put("html_strip", HtmlStripCharFilterFactory::new);
|
||||||
|
@ -50,4 +70,50 @@ public class CommonAnalysisPlugin extends Plugin implements AnalysisPlugin {
|
||||||
filters.put("mapping", requriesAnalysisSettings(MappingCharFilterFactory::new));
|
filters.put("mapping", requriesAnalysisSettings(MappingCharFilterFactory::new));
|
||||||
return filters;
|
return filters;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<PreConfiguredTokenFilter> getPreConfiguredTokenFilters() {
|
||||||
|
// TODO we should revisit the caching strategies.
|
||||||
|
List<PreConfiguredTokenFilter> filters = new ArrayList<>();
|
||||||
|
filters.add(new PreConfiguredTokenFilter("asciifolding", true, CachingStrategy.ONE, input -> new ASCIIFoldingFilter(input)));
|
||||||
|
filters.add(new PreConfiguredTokenFilter("classic", false, CachingStrategy.ONE, ClassicFilter::new));
|
||||||
|
filters.add(new PreConfiguredTokenFilter("common_grams", false, CachingStrategy.LUCENE, input ->
|
||||||
|
new CommonGramsFilter(input, CharArraySet.EMPTY_SET)));
|
||||||
|
filters.add(new PreConfiguredTokenFilter("edge_ngram", false, CachingStrategy.LUCENE, input ->
|
||||||
|
new EdgeNGramTokenFilter(input, EdgeNGramTokenFilter.DEFAULT_MIN_GRAM_SIZE, EdgeNGramTokenFilter.DEFAULT_MAX_GRAM_SIZE)));
|
||||||
|
// TODO deprecate edgeNGram
|
||||||
|
filters.add(new PreConfiguredTokenFilter("edgeNGram", false, CachingStrategy.LUCENE, input ->
|
||||||
|
new EdgeNGramTokenFilter(input, EdgeNGramTokenFilter.DEFAULT_MIN_GRAM_SIZE, EdgeNGramTokenFilter.DEFAULT_MAX_GRAM_SIZE)));
|
||||||
|
filters.add(new PreConfiguredTokenFilter("kstem", false, CachingStrategy.ONE, KStemFilter::new));
|
||||||
|
filters.add(new PreConfiguredTokenFilter("length", false, CachingStrategy.LUCENE, input ->
|
||||||
|
new LengthFilter(input, 0, Integer.MAX_VALUE))); // TODO this one seems useless
|
||||||
|
filters.add(new PreConfiguredTokenFilter("ngram", false, CachingStrategy.LUCENE, NGramTokenFilter::new));
|
||||||
|
// TODO deprecate nGram
|
||||||
|
filters.add(new PreConfiguredTokenFilter("nGram", false, CachingStrategy.LUCENE, NGramTokenFilter::new));
|
||||||
|
filters.add(new PreConfiguredTokenFilter("porter_stem", false, CachingStrategy.ONE, PorterStemFilter::new));
|
||||||
|
filters.add(new PreConfiguredTokenFilter("reverse", false, CachingStrategy.LUCENE, input -> new ReverseStringFilter(input)));
|
||||||
|
// The stop filter is in lucene-core but the English stop words set is in lucene-analyzers-common
|
||||||
|
filters.add(new PreConfiguredTokenFilter("stop", false, CachingStrategy.LUCENE, input ->
|
||||||
|
new StopFilter(input, StopAnalyzer.ENGLISH_STOP_WORDS_SET)));
|
||||||
|
filters.add(new PreConfiguredTokenFilter("trim", false, CachingStrategy.LUCENE, TrimFilter::new));
|
||||||
|
filters.add(new PreConfiguredTokenFilter("truncate", false, CachingStrategy.ONE, input ->
|
||||||
|
new TruncateTokenFilter(input, 10)));
|
||||||
|
filters.add(new PreConfiguredTokenFilter("unique", false, CachingStrategy.ONE, input -> new UniqueTokenFilter(input)));
|
||||||
|
filters.add(new PreConfiguredTokenFilter("uppercase", true, CachingStrategy.LUCENE, UpperCaseFilter::new));
|
||||||
|
filters.add(new PreConfiguredTokenFilter("word_delimiter", false, CachingStrategy.ONE, input ->
|
||||||
|
new WordDelimiterFilter(input,
|
||||||
|
WordDelimiterFilter.GENERATE_WORD_PARTS
|
||||||
|
| WordDelimiterFilter.GENERATE_NUMBER_PARTS
|
||||||
|
| WordDelimiterFilter.SPLIT_ON_CASE_CHANGE
|
||||||
|
| WordDelimiterFilter.SPLIT_ON_NUMERICS
|
||||||
|
| WordDelimiterFilter.STEM_ENGLISH_POSSESSIVE, null)));
|
||||||
|
filters.add(new PreConfiguredTokenFilter("word_delimiter_graph", false, CachingStrategy.ONE, input ->
|
||||||
|
new WordDelimiterGraphFilter(input,
|
||||||
|
WordDelimiterGraphFilter.GENERATE_WORD_PARTS
|
||||||
|
| WordDelimiterGraphFilter.GENERATE_NUMBER_PARTS
|
||||||
|
| WordDelimiterGraphFilter.SPLIT_ON_CASE_CHANGE
|
||||||
|
| WordDelimiterGraphFilter.SPLIT_ON_NUMERICS
|
||||||
|
| WordDelimiterGraphFilter.STEM_ENGLISH_POSSESSIVE, null)));
|
||||||
|
return filters;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,10 +19,10 @@
|
||||||
|
|
||||||
package org.elasticsearch.analysis.common;
|
package org.elasticsearch.analysis.common;
|
||||||
|
|
||||||
import org.elasticsearch.AnalysisFactoryTestCase;
|
import org.apache.lucene.analysis.reverse.ReverseStringFilterFactory;
|
||||||
import org.elasticsearch.index.analysis.HtmlStripCharFilterFactory;
|
import org.elasticsearch.index.analysis.HtmlStripCharFilterFactory;
|
||||||
|
import org.elasticsearch.indices.analysis.AnalysisFactoryTestCase;
|
||||||
|
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.TreeMap;
|
import java.util.TreeMap;
|
||||||
|
@ -31,15 +31,19 @@ import static java.util.Collections.emptyList;
|
||||||
import static java.util.stream.Collectors.toList;
|
import static java.util.stream.Collectors.toList;
|
||||||
|
|
||||||
public class CommonAnalysisFactoryTests extends AnalysisFactoryTestCase {
|
public class CommonAnalysisFactoryTests extends AnalysisFactoryTestCase {
|
||||||
|
public CommonAnalysisFactoryTests() {
|
||||||
|
super(new CommonAnalysisPlugin());
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected Map<String, Class<?>> getTokenizers() {
|
protected Map<String, Class<?>> getTokenizers() {
|
||||||
Map<String, Class<?>> tokenizers = new HashMap<>(super.getTokenizers());
|
Map<String, Class<?>> tokenizers = new TreeMap<>(super.getTokenizers());
|
||||||
return tokenizers;
|
return tokenizers;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected Map<String, Class<?>> getTokenFilters() {
|
protected Map<String, Class<?>> getTokenFilters() {
|
||||||
Map<String, Class<?>> filters = new HashMap<>(super.getTokenFilters());
|
Map<String, Class<?>> filters = new TreeMap<>(super.getTokenFilters());
|
||||||
filters.put("asciifolding", ASCIIFoldingTokenFilterFactory.class);
|
filters.put("asciifolding", ASCIIFoldingTokenFilterFactory.class);
|
||||||
filters.put("worddelimiter", WordDelimiterTokenFilterFactory.class);
|
filters.put("worddelimiter", WordDelimiterTokenFilterFactory.class);
|
||||||
filters.put("worddelimitergraph", WordDelimiterGraphTokenFilterFactory.class);
|
filters.put("worddelimitergraph", WordDelimiterGraphTokenFilterFactory.class);
|
||||||
|
@ -59,6 +63,30 @@ public class CommonAnalysisFactoryTests extends AnalysisFactoryTestCase {
|
||||||
return filters;
|
return filters;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Map<String, Class<?>> getPreConfiguredTokenFilters() {
|
||||||
|
Map<String, Class<?>> filters = new TreeMap<>(super.getPreConfiguredTokenFilters());
|
||||||
|
filters.put("asciifolding", null);
|
||||||
|
filters.put("classic", null);
|
||||||
|
filters.put("common_grams", null);
|
||||||
|
filters.put("edge_ngram", null);
|
||||||
|
filters.put("edgeNGram", null);
|
||||||
|
filters.put("kstem", null);
|
||||||
|
filters.put("length", null);
|
||||||
|
filters.put("ngram", null);
|
||||||
|
filters.put("nGram", null);
|
||||||
|
filters.put("porter_stem", null);
|
||||||
|
filters.put("reverse", ReverseStringFilterFactory.class);
|
||||||
|
filters.put("stop", null);
|
||||||
|
filters.put("trim", null);
|
||||||
|
filters.put("truncate", null);
|
||||||
|
filters.put("unique", Void.class);
|
||||||
|
filters.put("uppercase", null);
|
||||||
|
filters.put("word_delimiter", null);
|
||||||
|
filters.put("word_delimiter_graph", null);
|
||||||
|
return filters;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Fails if a tokenizer is marked in the superclass with {@link MovedToAnalysisCommon} but
|
* Fails if a tokenizer is marked in the superclass with {@link MovedToAnalysisCommon} but
|
||||||
* hasn't been marked in this class with its proper factory.
|
* hasn't been marked in this class with its proper factory.
|
||||||
|
|
|
@ -10,6 +10,18 @@
|
||||||
- length: { tokens: 1 }
|
- length: { tokens: 1 }
|
||||||
- match: { tokens.0.token: Musee d'Orsay }
|
- match: { tokens.0.token: Musee d'Orsay }
|
||||||
|
|
||||||
|
- do:
|
||||||
|
indices.analyze:
|
||||||
|
body:
|
||||||
|
text: Musée d'Orsay
|
||||||
|
tokenizer: keyword
|
||||||
|
filter:
|
||||||
|
- type: asciifolding
|
||||||
|
preserve_original: true
|
||||||
|
- length: { tokens: 2 }
|
||||||
|
- match: { tokens.0.token: Musee d'Orsay }
|
||||||
|
- match: { tokens.1.token: Musée d'Orsay }
|
||||||
|
|
||||||
---
|
---
|
||||||
"lowercase":
|
"lowercase":
|
||||||
- do:
|
- do:
|
||||||
|
|
|
@ -19,12 +19,16 @@
|
||||||
|
|
||||||
package org.elasticsearch.index.analysis;
|
package org.elasticsearch.index.analysis;
|
||||||
|
|
||||||
import org.elasticsearch.AnalysisFactoryTestCase;
|
import org.elasticsearch.indices.analysis.AnalysisFactoryTestCase;
|
||||||
|
import org.elasticsearch.plugin.analysis.icu.AnalysisICUPlugin;
|
||||||
|
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
public class AnalysisICUFactoryTests extends AnalysisFactoryTestCase {
|
public class AnalysisICUFactoryTests extends AnalysisFactoryTestCase {
|
||||||
|
public AnalysisICUFactoryTests() {
|
||||||
|
super(new AnalysisICUPlugin());
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected Map<String, Class<?>> getTokenizers() {
|
protected Map<String, Class<?>> getTokenizers() {
|
||||||
|
|
|
@ -20,12 +20,16 @@
|
||||||
package org.elasticsearch.index.analysis;
|
package org.elasticsearch.index.analysis;
|
||||||
|
|
||||||
import org.apache.lucene.analysis.ja.JapaneseTokenizerFactory;
|
import org.apache.lucene.analysis.ja.JapaneseTokenizerFactory;
|
||||||
import org.elasticsearch.AnalysisFactoryTestCase;
|
import org.elasticsearch.indices.analysis.AnalysisFactoryTestCase;
|
||||||
|
import org.elasticsearch.plugin.analysis.kuromoji.AnalysisKuromojiPlugin;
|
||||||
|
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
public class AnalysisKuromojiFactoryTests extends AnalysisFactoryTestCase {
|
public class AnalysisKuromojiFactoryTests extends AnalysisFactoryTestCase {
|
||||||
|
public AnalysisKuromojiFactoryTests() {
|
||||||
|
super(new AnalysisKuromojiPlugin());
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected Map<String, Class<?>> getTokenizers() {
|
protected Map<String, Class<?>> getTokenizers() {
|
||||||
|
|
|
@ -19,12 +19,16 @@
|
||||||
|
|
||||||
package org.elasticsearch.index.analysis;
|
package org.elasticsearch.index.analysis;
|
||||||
|
|
||||||
import org.elasticsearch.AnalysisFactoryTestCase;
|
import org.elasticsearch.indices.analysis.AnalysisFactoryTestCase;
|
||||||
|
import org.elasticsearch.plugin.analysis.AnalysisPhoneticPlugin;
|
||||||
|
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
public class AnalysisPhoneticFactoryTests extends AnalysisFactoryTestCase {
|
public class AnalysisPhoneticFactoryTests extends AnalysisFactoryTestCase {
|
||||||
|
public AnalysisPhoneticFactoryTests() {
|
||||||
|
super(new AnalysisPhoneticPlugin());
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected Map<String, Class<?>> getTokenFilters() {
|
protected Map<String, Class<?>> getTokenFilters() {
|
||||||
|
|
|
@ -19,13 +19,16 @@
|
||||||
|
|
||||||
package org.elasticsearch.index.analysis;
|
package org.elasticsearch.index.analysis;
|
||||||
|
|
||||||
import org.elasticsearch.AnalysisFactoryTestCase;
|
import org.elasticsearch.indices.analysis.AnalysisFactoryTestCase;
|
||||||
|
import org.elasticsearch.plugin.analysis.smartcn.AnalysisSmartChinesePlugin;
|
||||||
|
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
public class AnalysisSmartChineseFactoryTests extends AnalysisFactoryTestCase {
|
public class AnalysisSmartChineseFactoryTests extends AnalysisFactoryTestCase {
|
||||||
|
public AnalysisSmartChineseFactoryTests() {
|
||||||
|
super(new AnalysisSmartChinesePlugin());
|
||||||
|
}
|
||||||
@Override
|
@Override
|
||||||
protected Map<String, Class<?>> getTokenizers() {
|
protected Map<String, Class<?>> getTokenizers() {
|
||||||
Map<String, Class<?>> tokenizers = new HashMap<>(super.getTokenizers());
|
Map<String, Class<?>> tokenizers = new HashMap<>(super.getTokenizers());
|
||||||
|
|
|
@ -23,7 +23,6 @@ import org.apache.lucene.analysis.Analyzer;
|
||||||
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
|
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
|
||||||
import org.apache.lucene.analysis.MockTokenizer;
|
import org.apache.lucene.analysis.MockTokenizer;
|
||||||
import org.apache.lucene.analysis.Tokenizer;
|
import org.apache.lucene.analysis.Tokenizer;
|
||||||
import org.elasticsearch.AnalysisFactoryTestCase;
|
|
||||||
import org.elasticsearch.Version;
|
import org.elasticsearch.Version;
|
||||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||||
import org.elasticsearch.common.UUIDs;
|
import org.elasticsearch.common.UUIDs;
|
||||||
|
@ -31,12 +30,17 @@ import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.env.Environment;
|
import org.elasticsearch.env.Environment;
|
||||||
import org.elasticsearch.index.IndexSettings;
|
import org.elasticsearch.index.IndexSettings;
|
||||||
import org.elasticsearch.index.analysis.pl.PolishStemTokenFilterFactory;
|
import org.elasticsearch.index.analysis.pl.PolishStemTokenFilterFactory;
|
||||||
|
import org.elasticsearch.indices.analysis.AnalysisFactoryTestCase;
|
||||||
|
import org.elasticsearch.plugin.analysis.stempel.AnalysisStempelPlugin;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
public class AnalysisPolishFactoryTests extends AnalysisFactoryTestCase {
|
public class AnalysisPolishFactoryTests extends AnalysisFactoryTestCase {
|
||||||
|
public AnalysisPolishFactoryTests() {
|
||||||
|
super(new AnalysisStempelPlugin());
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected Map<String, Class<?>> getTokenFilters() {
|
protected Map<String, Class<?>> getTokenFilters() {
|
||||||
|
|
|
@ -210,9 +210,6 @@ public class IndexingIT extends ESRestTestCase {
|
||||||
final boolean checkGlobalCheckpoints = nodes.getMaster().getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED);
|
final boolean checkGlobalCheckpoints = nodes.getMaster().getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED);
|
||||||
logger.info("master version is [{}], global checkpoints will be [{}]", nodes.getMaster().getVersion(),
|
logger.info("master version is [{}], global checkpoints will be [{}]", nodes.getMaster().getVersion(),
|
||||||
checkGlobalCheckpoints ? "checked" : "not be checked");
|
checkGlobalCheckpoints ? "checked" : "not be checked");
|
||||||
if (checkGlobalCheckpoints) {
|
|
||||||
settings.put(IndexSettings.INDEX_SEQ_NO_CHECKPOINT_SYNC_INTERVAL.getKey(), "100ms");
|
|
||||||
}
|
|
||||||
final String index = "test";
|
final String index = "test";
|
||||||
createIndex(index, settings.build());
|
createIndex(index, settings.build());
|
||||||
try (RestClient newNodeClient = buildClient(restClientSettings(),
|
try (RestClient newNodeClient = buildClient(restClientSettings(),
|
||||||
|
@ -242,6 +239,7 @@ public class IndexingIT extends ESRestTestCase {
|
||||||
logger.info("indexing [{}] docs after moving primary", numberOfDocsAfterMovingPrimary);
|
logger.info("indexing [{}] docs after moving primary", numberOfDocsAfterMovingPrimary);
|
||||||
numDocsOnNewPrimary += indexDocs(index, numDocs, numberOfDocsAfterMovingPrimary);
|
numDocsOnNewPrimary += indexDocs(index, numDocs, numberOfDocsAfterMovingPrimary);
|
||||||
numDocs += numberOfDocsAfterMovingPrimary;
|
numDocs += numberOfDocsAfterMovingPrimary;
|
||||||
|
assertOK(client().performRequest("POST", index + "/_refresh")); // this forces a global checkpoint sync
|
||||||
assertSeqNoOnShards(index, nodes, checkGlobalCheckpoints, numDocsOnNewPrimary, newNodeClient);
|
assertSeqNoOnShards(index, nodes, checkGlobalCheckpoints, numDocsOnNewPrimary, newNodeClient);
|
||||||
/*
|
/*
|
||||||
* Dropping the number of replicas to zero, and then increasing it to one triggers a recovery thus exercising any BWC-logic in
|
* Dropping the number of replicas to zero, and then increasing it to one triggers a recovery thus exercising any BWC-logic in
|
||||||
|
|
|
@ -22,6 +22,7 @@ package org.elasticsearch.bootstrap;
|
||||||
import org.apache.lucene.util.Constants;
|
import org.apache.lucene.util.Constants;
|
||||||
import org.elasticsearch.common.SuppressForbidden;
|
import org.elasticsearch.common.SuppressForbidden;
|
||||||
import org.elasticsearch.common.io.PathUtils;
|
import org.elasticsearch.common.io.PathUtils;
|
||||||
|
import org.elasticsearch.common.settings.Setting;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.env.Environment;
|
import org.elasticsearch.env.Environment;
|
||||||
import org.elasticsearch.test.ESTestCase;
|
import org.elasticsearch.test.ESTestCase;
|
||||||
|
@ -118,6 +119,7 @@ public class EvilSecurityTests extends ESTestCase {
|
||||||
assertExactPermissions(new FilePermission(environment.configFile().toString(), "read,readlink"), permissions);
|
assertExactPermissions(new FilePermission(environment.configFile().toString(), "read,readlink"), permissions);
|
||||||
// scripts file: ro
|
// scripts file: ro
|
||||||
assertExactPermissions(new FilePermission(environment.scriptsFile().toString(), "read,readlink"), permissions);
|
assertExactPermissions(new FilePermission(environment.scriptsFile().toString(), "read,readlink"), permissions);
|
||||||
|
assertSettingDeprecationsAndWarnings(new Setting<?>[] {Environment.PATH_SCRIPTS_SETTING});
|
||||||
// plugins: ro
|
// plugins: ro
|
||||||
assertExactPermissions(new FilePermission(environment.pluginsFile().toString(), "read,readlink"), permissions);
|
assertExactPermissions(new FilePermission(environment.pluginsFile().toString(), "read,readlink"), permissions);
|
||||||
|
|
||||||
|
|
|
@ -40,7 +40,6 @@ public abstract class AbstractScriptTestCase extends ESTestCase {
|
||||||
public void init() throws Exception {
|
public void init() throws Exception {
|
||||||
Settings settings = Settings.builder()
|
Settings settings = Settings.builder()
|
||||||
.put("path.home", createTempDir())
|
.put("path.home", createTempDir())
|
||||||
.put(ScriptService.SCRIPT_AUTO_RELOAD_ENABLED_SETTING.getKey(), false)
|
|
||||||
.build();
|
.build();
|
||||||
ScriptEngineRegistry scriptEngineRegistry = new ScriptEngineRegistry(Arrays.asList(new MustacheScriptEngineService()));
|
ScriptEngineRegistry scriptEngineRegistry = new ScriptEngineRegistry(Arrays.asList(new MustacheScriptEngineService()));
|
||||||
ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(Collections.emptyList());
|
ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(Collections.emptyList());
|
||||||
|
|
|
@ -24,11 +24,23 @@
|
||||||
"value": "{{ _ingest.timestamp }}"
|
"value": "{{ _ingest.timestamp }}"
|
||||||
}
|
}
|
||||||
},
|
},
|
||||||
|
{
|
||||||
|
"grok" : {
|
||||||
|
"field" : "my_time",
|
||||||
|
"patterns": ["%{DAY:day} %{MONTH:month} %{MONTHDAY:monthday} %{TIME:time} %{WORD} %{YEAR:year}"]
|
||||||
|
}
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"set": {
|
||||||
|
"field": "my_time",
|
||||||
|
"value": "{{day}} {{month}} {{monthday}} {{time}} {{year}}"
|
||||||
|
}
|
||||||
|
},
|
||||||
{
|
{
|
||||||
"date" : {
|
"date" : {
|
||||||
"field" : "my_time",
|
"field" : "my_time",
|
||||||
"target_field": "my_time",
|
"target_field": "my_time",
|
||||||
"formats": ["EEE MMM dd HH:mm:ss zzz yyyy"]
|
"formats": ["EEE MMM dd HH:mm:ss yyyy"]
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
]
|
]
|
||||||
|
|
|
@ -190,7 +190,7 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
||||||
@Nullable IndexSearcherWrapper searcherWrapper) throws IOException {
|
@Nullable IndexSearcherWrapper searcherWrapper) throws IOException {
|
||||||
ShardRouting shardRouting = TestShardRouting.newShardRouting(shardId, nodeId, primary, ShardRoutingState.INITIALIZING,
|
ShardRouting shardRouting = TestShardRouting.newShardRouting(shardId, nodeId, primary, ShardRoutingState.INITIALIZING,
|
||||||
primary ? RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE);
|
primary ? RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE);
|
||||||
return newShard(shardRouting, indexMetaData, searcherWrapper, () -> {}, null);
|
return newShard(shardRouting, indexMetaData, searcherWrapper, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -206,7 +206,7 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
||||||
@Nullable IndexSearcherWrapper searcherWrapper) throws IOException {
|
@Nullable IndexSearcherWrapper searcherWrapper) throws IOException {
|
||||||
ShardRouting shardRouting = TestShardRouting.newShardRouting(shardId, nodeId, primary, ShardRoutingState.INITIALIZING,
|
ShardRouting shardRouting = TestShardRouting.newShardRouting(shardId, nodeId, primary, ShardRoutingState.INITIALIZING,
|
||||||
primary ? RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE);
|
primary ? RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE);
|
||||||
return newShard(shardRouting, indexMetaData, searcherWrapper, globalCheckpointSyncer, null);
|
return newShard(shardRouting, indexMetaData, searcherWrapper, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -220,21 +220,19 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
||||||
*/
|
*/
|
||||||
protected IndexShard newShard(ShardRouting routing, IndexMetaData indexMetaData, IndexingOperationListener... listeners)
|
protected IndexShard newShard(ShardRouting routing, IndexMetaData indexMetaData, IndexingOperationListener... listeners)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return newShard(routing, indexMetaData, null, () -> {}, null, listeners);
|
return newShard(routing, indexMetaData, null, null, listeners);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* creates a new initializing shard. The shard will will be put in its proper path under the
|
* creates a new initializing shard. The shard will will be put in its proper path under the
|
||||||
* current node id the shard is assigned to.
|
* current node id the shard is assigned to.
|
||||||
*
|
* @param routing shard routing to use
|
||||||
* @param routing shard routing to use
|
|
||||||
* @param indexMetaData indexMetaData for the shard, including any mapping
|
* @param indexMetaData indexMetaData for the shard, including any mapping
|
||||||
* @param indexSearcherWrapper an optional wrapper to be used during searchers
|
* @param indexSearcherWrapper an optional wrapper to be used during searchers
|
||||||
* @param globalCheckpointSyncer an runnable to run when the global check point needs syncing
|
|
||||||
* @param listeners an optional set of listeners to add to the shard
|
* @param listeners an optional set of listeners to add to the shard
|
||||||
*/
|
*/
|
||||||
protected IndexShard newShard(ShardRouting routing, IndexMetaData indexMetaData,
|
protected IndexShard newShard(ShardRouting routing, IndexMetaData indexMetaData,
|
||||||
@Nullable IndexSearcherWrapper indexSearcherWrapper, Runnable globalCheckpointSyncer,
|
@Nullable IndexSearcherWrapper indexSearcherWrapper,
|
||||||
@Nullable EngineFactory engineFactory,
|
@Nullable EngineFactory engineFactory,
|
||||||
IndexingOperationListener... listeners)
|
IndexingOperationListener... listeners)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
@ -242,12 +240,11 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
||||||
final ShardId shardId = routing.shardId();
|
final ShardId shardId = routing.shardId();
|
||||||
final NodeEnvironment.NodePath nodePath = new NodeEnvironment.NodePath(createTempDir());
|
final NodeEnvironment.NodePath nodePath = new NodeEnvironment.NodePath(createTempDir());
|
||||||
ShardPath shardPath = new ShardPath(false, nodePath.resolve(shardId), nodePath.resolve(shardId), shardId);
|
ShardPath shardPath = new ShardPath(false, nodePath.resolve(shardId), nodePath.resolve(shardId), shardId);
|
||||||
return newShard(routing, shardPath, indexMetaData, indexSearcherWrapper, globalCheckpointSyncer, engineFactory, listeners);
|
return newShard(routing, shardPath, indexMetaData, indexSearcherWrapper, engineFactory, listeners);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* creates a new initializing shard.
|
* creates a new initializing shard.
|
||||||
*
|
|
||||||
* @param routing shard routing to use
|
* @param routing shard routing to use
|
||||||
* @param shardPath path to use for shard data
|
* @param shardPath path to use for shard data
|
||||||
* @param indexMetaData indexMetaData for the shard, including any mapping
|
* @param indexMetaData indexMetaData for the shard, including any mapping
|
||||||
|
@ -256,7 +253,6 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
||||||
*/
|
*/
|
||||||
protected IndexShard newShard(ShardRouting routing, ShardPath shardPath, IndexMetaData indexMetaData,
|
protected IndexShard newShard(ShardRouting routing, ShardPath shardPath, IndexMetaData indexMetaData,
|
||||||
@Nullable IndexSearcherWrapper indexSearcherWrapper,
|
@Nullable IndexSearcherWrapper indexSearcherWrapper,
|
||||||
Runnable globalCheckpointSyncer,
|
|
||||||
@Nullable EngineFactory engineFactory,
|
@Nullable EngineFactory engineFactory,
|
||||||
IndexingOperationListener... listeners) throws IOException {
|
IndexingOperationListener... listeners) throws IOException {
|
||||||
final Settings nodeSettings = Settings.builder().put("node.name", routing.currentNodeId()).build();
|
final Settings nodeSettings = Settings.builder().put("node.name", routing.currentNodeId()).build();
|
||||||
|
@ -280,7 +276,7 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
||||||
new NoneCircuitBreakerService(), mapperService);
|
new NoneCircuitBreakerService(), mapperService);
|
||||||
indexShard = new IndexShard(routing, indexSettings, shardPath, store, () ->null, indexCache, mapperService, similarityService,
|
indexShard = new IndexShard(routing, indexSettings, shardPath, store, () ->null, indexCache, mapperService, similarityService,
|
||||||
indexFieldDataService, engineFactory, indexEventListener, indexSearcherWrapper, threadPool,
|
indexFieldDataService, engineFactory, indexEventListener, indexSearcherWrapper, threadPool,
|
||||||
BigArrays.NON_RECYCLING_INSTANCE, warmer, globalCheckpointSyncer, Collections.emptyList(), Arrays.asList(listeners));
|
BigArrays.NON_RECYCLING_INSTANCE, warmer, Collections.emptyList(), Arrays.asList(listeners));
|
||||||
success = true;
|
success = true;
|
||||||
} finally {
|
} finally {
|
||||||
if (success == false) {
|
if (success == false) {
|
||||||
|
@ -310,8 +306,7 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
||||||
*/
|
*/
|
||||||
protected IndexShard reinitShard(IndexShard current, ShardRouting routing, IndexingOperationListener... listeners) throws IOException {
|
protected IndexShard reinitShard(IndexShard current, ShardRouting routing, IndexingOperationListener... listeners) throws IOException {
|
||||||
closeShards(current);
|
closeShards(current);
|
||||||
return newShard(routing, current.shardPath(), current.indexSettings().getIndexMetaData(), null,
|
return newShard(routing, current.shardPath(), current.indexSettings().getIndexMetaData(), null, current.engineFactory, listeners);
|
||||||
current.getGlobalCheckpointSyncer(), current.engineFactory, listeners);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -17,14 +17,14 @@
|
||||||
* under the License.
|
* under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package org.elasticsearch;
|
package org.elasticsearch.indices.analysis;
|
||||||
|
|
||||||
import org.apache.lucene.analysis.en.PorterStemFilterFactory;
|
import org.apache.lucene.analysis.en.PorterStemFilterFactory;
|
||||||
import org.apache.lucene.analysis.reverse.ReverseStringFilterFactory;
|
|
||||||
import org.apache.lucene.analysis.snowball.SnowballPorterFilterFactory;
|
import org.apache.lucene.analysis.snowball.SnowballPorterFilterFactory;
|
||||||
import org.apache.lucene.analysis.util.CharFilterFactory;
|
import org.apache.lucene.analysis.util.CharFilterFactory;
|
||||||
import org.apache.lucene.analysis.util.TokenFilterFactory;
|
import org.apache.lucene.analysis.util.TokenFilterFactory;
|
||||||
import org.apache.lucene.analysis.util.TokenizerFactory;
|
import org.apache.lucene.analysis.util.TokenizerFactory;
|
||||||
|
import org.elasticsearch.Version;
|
||||||
import org.elasticsearch.common.collect.MapBuilder;
|
import org.elasticsearch.common.collect.MapBuilder;
|
||||||
import org.elasticsearch.index.analysis.ApostropheFilterFactory;
|
import org.elasticsearch.index.analysis.ApostropheFilterFactory;
|
||||||
import org.elasticsearch.index.analysis.ArabicNormalizationFilterFactory;
|
import org.elasticsearch.index.analysis.ArabicNormalizationFilterFactory;
|
||||||
|
@ -67,6 +67,7 @@ import org.elasticsearch.index.analysis.PatternReplaceTokenFilterFactory;
|
||||||
import org.elasticsearch.index.analysis.PatternTokenizerFactory;
|
import org.elasticsearch.index.analysis.PatternTokenizerFactory;
|
||||||
import org.elasticsearch.index.analysis.PersianNormalizationFilterFactory;
|
import org.elasticsearch.index.analysis.PersianNormalizationFilterFactory;
|
||||||
import org.elasticsearch.index.analysis.PorterStemTokenFilterFactory;
|
import org.elasticsearch.index.analysis.PorterStemTokenFilterFactory;
|
||||||
|
import org.elasticsearch.index.analysis.PreConfiguredTokenFilter;
|
||||||
import org.elasticsearch.index.analysis.ReverseTokenFilterFactory;
|
import org.elasticsearch.index.analysis.ReverseTokenFilterFactory;
|
||||||
import org.elasticsearch.index.analysis.ScandinavianFoldingFilterFactory;
|
import org.elasticsearch.index.analysis.ScandinavianFoldingFilterFactory;
|
||||||
import org.elasticsearch.index.analysis.ScandinavianNormalizationFilterFactory;
|
import org.elasticsearch.index.analysis.ScandinavianNormalizationFilterFactory;
|
||||||
|
@ -89,21 +90,23 @@ import org.elasticsearch.index.analysis.UpperCaseTokenFilterFactory;
|
||||||
import org.elasticsearch.index.analysis.WhitespaceTokenizerFactory;
|
import org.elasticsearch.index.analysis.WhitespaceTokenizerFactory;
|
||||||
import org.elasticsearch.index.analysis.compound.DictionaryCompoundWordTokenFilterFactory;
|
import org.elasticsearch.index.analysis.compound.DictionaryCompoundWordTokenFilterFactory;
|
||||||
import org.elasticsearch.index.analysis.compound.HyphenationCompoundWordTokenFilterFactory;
|
import org.elasticsearch.index.analysis.compound.HyphenationCompoundWordTokenFilterFactory;
|
||||||
import org.elasticsearch.indices.analysis.PreBuiltCharFilters;
|
import org.elasticsearch.plugins.AnalysisPlugin;
|
||||||
import org.elasticsearch.indices.analysis.PreBuiltTokenFilters;
|
|
||||||
import org.elasticsearch.indices.analysis.PreBuiltTokenizers;
|
|
||||||
import org.elasticsearch.test.ESTestCase;
|
import org.elasticsearch.test.ESTestCase;
|
||||||
|
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.EnumMap;
|
import java.util.EnumMap;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
|
import java.util.Locale;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Objects;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.TreeSet;
|
import java.util.TreeSet;
|
||||||
import java.util.regex.Matcher;
|
import java.util.regex.Matcher;
|
||||||
import java.util.regex.Pattern;
|
import java.util.regex.Pattern;
|
||||||
|
|
||||||
|
import static java.util.Collections.singletonList;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Alerts us if new analysis components are added to Lucene, so we don't miss them.
|
* Alerts us if new analysis components are added to Lucene, so we don't miss them.
|
||||||
* <p>
|
* <p>
|
||||||
|
@ -285,41 +288,6 @@ public abstract class AnalysisFactoryTestCase extends ESTestCase {
|
||||||
|
|
||||||
.immutableMap();
|
.immutableMap();
|
||||||
|
|
||||||
static final Map<PreBuiltTokenFilters, Class<?>> PREBUILT_TOKENFILTERS;
|
|
||||||
static {
|
|
||||||
PREBUILT_TOKENFILTERS = new EnumMap<>(PreBuiltTokenFilters.class);
|
|
||||||
for (PreBuiltTokenFilters tokenizer : PreBuiltTokenFilters.values()) {
|
|
||||||
Class<?> luceneFactoryClazz;
|
|
||||||
switch (tokenizer) {
|
|
||||||
case REVERSE:
|
|
||||||
luceneFactoryClazz = ReverseStringFilterFactory.class;
|
|
||||||
break;
|
|
||||||
case UNIQUE:
|
|
||||||
luceneFactoryClazz = Void.class;
|
|
||||||
break;
|
|
||||||
case SNOWBALL:
|
|
||||||
case DUTCH_STEM:
|
|
||||||
case FRENCH_STEM:
|
|
||||||
case RUSSIAN_STEM:
|
|
||||||
luceneFactoryClazz = SnowballPorterFilterFactory.class;
|
|
||||||
break;
|
|
||||||
case STEMMER:
|
|
||||||
luceneFactoryClazz = PorterStemFilterFactory.class;
|
|
||||||
break;
|
|
||||||
case DELIMITED_PAYLOAD_FILTER:
|
|
||||||
luceneFactoryClazz = org.apache.lucene.analysis.payloads.DelimitedPayloadTokenFilterFactory.class;
|
|
||||||
break;
|
|
||||||
case LIMIT:
|
|
||||||
luceneFactoryClazz = org.apache.lucene.analysis.miscellaneous.LimitTokenCountFilterFactory.class;
|
|
||||||
break;
|
|
||||||
default:
|
|
||||||
luceneFactoryClazz = org.apache.lucene.analysis.util.TokenFilterFactory.lookupClass(
|
|
||||||
toCamelCase(tokenizer.getTokenFilterFactory(Version.CURRENT).name()));
|
|
||||||
}
|
|
||||||
PREBUILT_TOKENFILTERS.put(tokenizer, luceneFactoryClazz);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
static final Map<String,Class<?>> KNOWN_CHARFILTERS = new MapBuilder<String,Class<?>>()
|
static final Map<String,Class<?>> KNOWN_CHARFILTERS = new MapBuilder<String,Class<?>>()
|
||||||
// exposed in ES
|
// exposed in ES
|
||||||
.put("htmlstrip", MovedToAnalysisCommon.class)
|
.put("htmlstrip", MovedToAnalysisCommon.class)
|
||||||
|
@ -345,6 +313,15 @@ public abstract class AnalysisFactoryTestCase extends ESTestCase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The plugin being tested. Core uses an "empty" plugin so we don't have to throw null checks all over the place.
|
||||||
|
*/
|
||||||
|
private final AnalysisPlugin plugin;
|
||||||
|
|
||||||
|
public AnalysisFactoryTestCase(AnalysisPlugin plugin) {
|
||||||
|
this.plugin = Objects.requireNonNull(plugin, "plugin is required. use an empty plugin for core");
|
||||||
|
}
|
||||||
|
|
||||||
protected Map<String, Class<?>> getTokenizers() {
|
protected Map<String, Class<?>> getTokenizers() {
|
||||||
return KNOWN_TOKENIZERS;
|
return KNOWN_TOKENIZERS;
|
||||||
}
|
}
|
||||||
|
@ -353,6 +330,49 @@ public abstract class AnalysisFactoryTestCase extends ESTestCase {
|
||||||
return KNOWN_TOKENFILTERS;
|
return KNOWN_TOKENFILTERS;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Map containing pre-configured token filters that should be available
|
||||||
|
* after installing this plugin. The map is from the name of the token
|
||||||
|
* filter to the class of the Lucene {@link TokenFilterFactory} that it
|
||||||
|
* is emulating. If the Lucene filter factory is {@code null} then the
|
||||||
|
* test will look it up for you from the name. If there is no Lucene
|
||||||
|
* {@linkplain TokenFilterFactory} then the right hand side should
|
||||||
|
* be {@link Void}.
|
||||||
|
*/
|
||||||
|
protected Map<String, Class<?>> getPreConfiguredTokenFilters() {
|
||||||
|
Map<String, Class<?>> filters = new HashMap<>();
|
||||||
|
filters.put("standard", null);
|
||||||
|
filters.put("lowercase", null);
|
||||||
|
// TODO remove the loop below once all the tokenizers are migrated out of PreBuiltTokenFilters
|
||||||
|
for (PreBuiltTokenFilters tokenizer : PreBuiltTokenFilters.values()) {
|
||||||
|
Class<?> luceneFactoryClass;
|
||||||
|
switch (tokenizer) {
|
||||||
|
case LOWERCASE:
|
||||||
|
// This has been migrated but has to stick around until PreBuiltTokenizers is removed.
|
||||||
|
continue;
|
||||||
|
case SNOWBALL:
|
||||||
|
case DUTCH_STEM:
|
||||||
|
case FRENCH_STEM:
|
||||||
|
case RUSSIAN_STEM:
|
||||||
|
luceneFactoryClass = SnowballPorterFilterFactory.class;
|
||||||
|
break;
|
||||||
|
case STEMMER:
|
||||||
|
luceneFactoryClass = PorterStemFilterFactory.class;
|
||||||
|
break;
|
||||||
|
case DELIMITED_PAYLOAD_FILTER:
|
||||||
|
luceneFactoryClass = org.apache.lucene.analysis.payloads.DelimitedPayloadTokenFilterFactory.class;
|
||||||
|
break;
|
||||||
|
case LIMIT:
|
||||||
|
luceneFactoryClass = org.apache.lucene.analysis.miscellaneous.LimitTokenCountFilterFactory.class;
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
luceneFactoryClass = null;
|
||||||
|
}
|
||||||
|
filters.put(tokenizer.name().toLowerCase(Locale.ROOT), luceneFactoryClass);
|
||||||
|
}
|
||||||
|
return filters;
|
||||||
|
}
|
||||||
|
|
||||||
protected Map<String, Class<?>> getCharFilters() {
|
protected Map<String, Class<?>> getCharFilters() {
|
||||||
return KNOWN_CHARFILTERS;
|
return KNOWN_CHARFILTERS;
|
||||||
}
|
}
|
||||||
|
@ -445,18 +465,24 @@ public abstract class AnalysisFactoryTestCase extends ESTestCase {
|
||||||
expected.add(tokenizer);
|
expected.add(tokenizer);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
for (Map.Entry<PreBuiltTokenFilters, Class<?>> entry : PREBUILT_TOKENFILTERS.entrySet()) {
|
Map<String, PreConfiguredTokenFilter> preBuiltTokenFilters = AnalysisModule.setupPreConfiguredTokenFilters(singletonList(plugin));
|
||||||
PreBuiltTokenFilters tokenFilter = entry.getKey();
|
for (Map.Entry<String, Class<?>> entry : getPreConfiguredTokenFilters().entrySet()) {
|
||||||
|
String name = entry.getKey();
|
||||||
Class<?> luceneFactory = entry.getValue();
|
Class<?> luceneFactory = entry.getValue();
|
||||||
if (luceneFactory == Void.class) {
|
if (luceneFactory == Void.class) {
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
if (luceneFactory == null) {
|
||||||
|
luceneFactory = TokenFilterFactory.lookupClass(toCamelCase(name));
|
||||||
|
}
|
||||||
assertTrue(TokenFilterFactory.class.isAssignableFrom(luceneFactory));
|
assertTrue(TokenFilterFactory.class.isAssignableFrom(luceneFactory));
|
||||||
if (tokenFilter.getTokenFilterFactory(Version.CURRENT) instanceof MultiTermAwareComponent) {
|
PreConfiguredTokenFilter filter = preBuiltTokenFilters.get(name);
|
||||||
actual.add(tokenFilter);
|
assertNotNull("test claims pre built token filter [" + name + "] should be available but it wasn't", filter);
|
||||||
|
if (filter.shouldUseFilterForMultitermQueries()) {
|
||||||
|
actual.add("token filter [" + name + "]");
|
||||||
}
|
}
|
||||||
if (org.apache.lucene.analysis.util.MultiTermAwareComponent.class.isAssignableFrom(luceneFactory)) {
|
if (org.apache.lucene.analysis.util.MultiTermAwareComponent.class.isAssignableFrom(luceneFactory)) {
|
||||||
expected.add(tokenFilter);
|
expected.add("token filter [" + name + "]");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
for (Map.Entry<PreBuiltCharFilters, Class<?>> entry : PREBUILT_CHARFILTERS.entrySet()) {
|
for (Map.Entry<PreBuiltCharFilters, Class<?>> entry : PREBUILT_CHARFILTERS.entrySet()) {
|
|
@ -178,7 +178,6 @@ public abstract class AbstractQueryTestCase<QB extends AbstractQueryBuilder<QB>>
|
||||||
nodeSettings = Settings.builder()
|
nodeSettings = Settings.builder()
|
||||||
.put("node.name", AbstractQueryTestCase.class.toString())
|
.put("node.name", AbstractQueryTestCase.class.toString())
|
||||||
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir())
|
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir())
|
||||||
.put(ScriptService.SCRIPT_AUTO_RELOAD_ENABLED_SETTING.getKey(), false)
|
|
||||||
.build();
|
.build();
|
||||||
indexSettings = Settings.builder()
|
indexSettings = Settings.builder()
|
||||||
.put(IndexMetaData.SETTING_VERSION_CREATED, indexVersionCreated)
|
.put(IndexMetaData.SETTING_VERSION_CREATED, indexVersionCreated)
|
||||||
|
@ -1091,8 +1090,6 @@ public abstract class AbstractQueryTestCase<QB extends AbstractQueryBuilder<QB>>
|
||||||
|
|
||||||
Settings settings = Settings.builder()
|
Settings settings = Settings.builder()
|
||||||
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir())
|
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir())
|
||||||
// no file watching, so we don't need a ResourceWatcherService
|
|
||||||
.put(ScriptService.SCRIPT_AUTO_RELOAD_ENABLED_SETTING.getKey(), false)
|
|
||||||
.build();
|
.build();
|
||||||
Environment environment = new Environment(settings);
|
Environment environment = new Environment(settings);
|
||||||
return ScriptModule.create(settings, environment, null, scriptPlugins);
|
return ScriptModule.create(settings, environment, null, scriptPlugins);
|
||||||
|
|
|
@ -1172,8 +1172,6 @@ public abstract class ESTestCase extends LuceneTestCase {
|
||||||
public static ScriptModule newTestScriptModule() {
|
public static ScriptModule newTestScriptModule() {
|
||||||
Settings settings = Settings.builder()
|
Settings settings = Settings.builder()
|
||||||
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir())
|
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir())
|
||||||
// no file watching, so we don't need a ResourceWatcherService
|
|
||||||
.put(ScriptService.SCRIPT_AUTO_RELOAD_ENABLED_SETTING.getKey(), false)
|
|
||||||
.build();
|
.build();
|
||||||
Environment environment = new Environment(settings);
|
Environment environment = new Environment(settings);
|
||||||
MockScriptEngine scriptEngine = new MockScriptEngine(MockScriptEngine.NAME, Collections.singletonMap("1", script -> "1"));
|
MockScriptEngine scriptEngine = new MockScriptEngine(MockScriptEngine.NAME, Collections.singletonMap("1", script -> "1"));
|
||||||
|
|
Loading…
Reference in New Issue