Merge branch 'master' into dots2

This commit is contained in:
Ryan Ernst 2016-02-25 13:05:27 -08:00
commit 09df7379c5
82 changed files with 1897 additions and 1405 deletions

View File

@ -400,7 +400,7 @@ class BuildPlugin implements Plugin<Project> {
// we use './temp' since this is per JVM and tests are forbidden from writing to CWD
systemProperty 'java.io.tmpdir', './temp'
systemProperty 'java.awt.headless', 'true'
systemProperty 'tests.maven', 'true' // TODO: rename this once we've switched to gradle!
systemProperty 'tests.gradle', 'true'
systemProperty 'tests.artifact', project.name
systemProperty 'tests.task', path
systemProperty 'tests.security.manager', 'true'

View File

@ -245,7 +245,8 @@ class ClusterFormationTasks {
return setup
}
Copy copyConfig = project.tasks.create(name: name, type: Copy, dependsOn: setup)
copyConfig.into(new File(node.homeDir, 'config')) // copy must always have a general dest dir, even though we don't use it
File configDir = new File(node.homeDir, 'config')
copyConfig.into(configDir) // copy must always have a general dest dir, even though we don't use it
for (Map.Entry<String,Object> extraConfigFile : node.config.extraConfigFiles.entrySet()) {
copyConfig.doFirst {
// make sure the copy won't be a no-op or act on a directory
@ -258,9 +259,12 @@ class ClusterFormationTasks {
}
}
File destConfigFile = new File(node.homeDir, 'config/' + extraConfigFile.getKey())
copyConfig.into(destConfigFile.canonicalFile.parentFile)
.from({ extraConfigFile.getValue() }) // wrap in closure to delay resolution to execution time
.rename { destConfigFile.name }
// wrap source file in closure to delay resolution to execution time
copyConfig.from({ extraConfigFile.getValue() }) {
// this must be in a closure so it is only applied to the single file specified in from above
into(configDir.toPath().relativize(destConfigFile.canonicalFile.parentFile.toPath()).toFile())
rename { destConfigFile.name }
}
}
return copyConfig
}

View File

@ -13,6 +13,8 @@ jna = 4.1.0
# test dependencies
randomizedrunner = 2.3.2
junit = 4.11
# TODO: Upgrade httpclient to a version > 4.5.1 once released. Then remove o.e.test.rest.client.StrictHostnameVerifier* and use
# DefaultHostnameVerifier instead since we no longer need to workaround https://issues.apache.org/jira/browse/HTTPCLIENT-1698
httpclient = 4.3.6
httpcore = 4.3.3
commonslogging = 1.1.3

View File

@ -36,14 +36,6 @@ public class CancelTasksRequest extends BaseTasksRequest<CancelTasksRequest> {
private String reason = DEFAULT_REASON;
/**
* Cancel tasks on the specified nodes. If none are passed, all cancellable tasks on
* all nodes will be cancelled.
*/
public CancelTasksRequest(String... nodesIds) {
super(nodesIds);
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
@ -54,7 +46,6 @@ public class CancelTasksRequest extends BaseTasksRequest<CancelTasksRequest> {
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeString(reason);
}
@Override

View File

@ -24,7 +24,6 @@ import org.elasticsearch.action.FailedNodeException;
import org.elasticsearch.action.TaskOperationFailure;
import org.elasticsearch.action.admin.cluster.node.tasks.list.TaskInfo;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.tasks.BaseTasksRequest;
import org.elasticsearch.action.support.tasks.TransportTasksAction;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterService;
@ -36,6 +35,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.tasks.CancellableTask;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.EmptyTransportResponseHandler;
import org.elasticsearch.transport.TransportChannel;
@ -84,9 +84,9 @@ public class TransportCancelTasksAction extends TransportTasksAction<Cancellable
}
protected void processTasks(CancelTasksRequest request, Consumer<CancellableTask> operation) {
if (request.taskId() != BaseTasksRequest.ALL_TASKS) {
if (request.taskId().isSet() == false) {
// we are only checking one task, we can optimize it
CancellableTask task = taskManager.getCancellableTask(request.taskId());
CancellableTask task = taskManager.getCancellableTask(request.taskId().getId());
if (task != null) {
if (request.match(task)) {
operation.accept(task);
@ -94,7 +94,7 @@ public class TransportCancelTasksAction extends TransportTasksAction<Cancellable
throw new IllegalArgumentException("task [" + request.taskId() + "] doesn't support this operation");
}
} else {
if (taskManager.getTask(request.taskId()) != null) {
if (taskManager.getTask(request.taskId().getId()) != null) {
// The task exists, but doesn't support cancellation
throw new IllegalArgumentException("task [" + request.taskId() + "] doesn't support cancellation");
} else {
@ -135,11 +135,14 @@ public class TransportCancelTasksAction extends TransportTasksAction<Cancellable
}
private void setBanOnNodes(String reason, CancellableTask task, Set<String> nodes, BanLock banLock) {
sendSetBanRequest(nodes, new BanParentTaskRequest(clusterService.localNode().getId(), task.getId(), reason), banLock);
sendSetBanRequest(nodes,
BanParentTaskRequest.createSetBanParentTaskRequest(new TaskId(clusterService.localNode().getId(), task.getId()), reason),
banLock);
}
private void removeBanOnNodes(CancellableTask task, Set<String> nodes) {
sendRemoveBanRequest(nodes, new BanParentTaskRequest(clusterService.localNode().getId(), task.getId()));
sendRemoveBanRequest(nodes,
BanParentTaskRequest.createRemoveBanParentTaskRequest(new TaskId(clusterService.localNode().getId(), task.getId())));
}
private void sendSetBanRequest(Set<String> nodes, BanParentTaskRequest request, BanLock banLock) {
@ -148,8 +151,8 @@ public class TransportCancelTasksAction extends TransportTasksAction<Cancellable
DiscoveryNode discoveryNode = clusterState.getNodes().get(node);
if (discoveryNode != null) {
// Check if node still in the cluster
logger.debug("Sending ban for tasks with the parent [{}:{}] to the node [{}], ban [{}]", request.parentNodeId, request
.parentTaskId, node, request.ban);
logger.debug("Sending ban for tasks with the parent [{}] to the node [{}], ban [{}]", request.parentTaskId, node,
request.ban);
transportService.sendRequest(discoveryNode, BAN_PARENT_ACTION_NAME, request,
new EmptyTransportResponseHandler(ThreadPool.Names.SAME) {
@Override
@ -164,8 +167,8 @@ public class TransportCancelTasksAction extends TransportTasksAction<Cancellable
});
} else {
banLock.onBanSet();
logger.debug("Cannot send ban for tasks with the parent [{}:{}] to the node [{}] - the node no longer in the cluster",
request.parentNodeId, request.parentTaskId, node);
logger.debug("Cannot send ban for tasks with the parent [{}] to the node [{}] - the node no longer in the cluster",
request.parentTaskId, node);
}
}
}
@ -176,13 +179,12 @@ public class TransportCancelTasksAction extends TransportTasksAction<Cancellable
DiscoveryNode discoveryNode = clusterState.getNodes().get(node);
if (discoveryNode != null) {
// Check if node still in the cluster
logger.debug("Sending remove ban for tasks with the parent [{}:{}] to the node [{}]", request.parentNodeId,
request.parentTaskId, node);
logger.debug("Sending remove ban for tasks with the parent [{}] to the node [{}]", request.parentTaskId, node);
transportService.sendRequest(discoveryNode, BAN_PARENT_ACTION_NAME, request, EmptyTransportResponseHandler
.INSTANCE_SAME);
} else {
logger.debug("Cannot send remove ban request for tasks with the parent [{}:{}] to the node [{}] - the node no longer in " +
"the cluster", request.parentNodeId, request.parentTaskId, node);
logger.debug("Cannot send remove ban request for tasks with the parent [{}] to the node [{}] - the node no longer in " +
"the cluster", request.parentTaskId, node);
}
}
}
@ -218,23 +220,27 @@ public class TransportCancelTasksAction extends TransportTasksAction<Cancellable
private static class BanParentTaskRequest extends TransportRequest {
private String parentNodeId;
private long parentTaskId;
private TaskId parentTaskId;
private boolean ban;
private String reason;
BanParentTaskRequest(String parentNodeId, long parentTaskId, String reason) {
this.parentNodeId = parentNodeId;
static BanParentTaskRequest createSetBanParentTaskRequest(TaskId parentTaskId, String reason) {
return new BanParentTaskRequest(parentTaskId, reason);
}
static BanParentTaskRequest createRemoveBanParentTaskRequest(TaskId parentTaskId) {
return new BanParentTaskRequest(parentTaskId);
}
private BanParentTaskRequest(TaskId parentTaskId, String reason) {
this.parentTaskId = parentTaskId;
this.ban = true;
this.reason = reason;
}
BanParentTaskRequest(String parentNodeId, long parentTaskId) {
this.parentNodeId = parentNodeId;
private BanParentTaskRequest(TaskId parentTaskId) {
this.parentTaskId = parentTaskId;
this.ban = false;
}
@ -245,8 +251,7 @@ public class TransportCancelTasksAction extends TransportTasksAction<Cancellable
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
parentNodeId = in.readString();
parentTaskId = in.readLong();
parentTaskId = new TaskId(in);
ban = in.readBoolean();
if (ban) {
reason = in.readString();
@ -256,8 +261,7 @@ public class TransportCancelTasksAction extends TransportTasksAction<Cancellable
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeString(parentNodeId);
out.writeLong(parentTaskId);
parentTaskId.writeTo(out);
out.writeBoolean(ban);
if (ban) {
out.writeString(reason);
@ -269,13 +273,13 @@ public class TransportCancelTasksAction extends TransportTasksAction<Cancellable
@Override
public void messageReceived(final BanParentTaskRequest request, final TransportChannel channel) throws Exception {
if (request.ban) {
logger.debug("Received ban for the parent [{}:{}] on the node [{}], reason: [{}]", request.parentNodeId, request
.parentTaskId, clusterService.localNode().getId(), request.reason);
taskManager.setBan(request.parentNodeId, request.parentTaskId, request.reason);
logger.debug("Received ban for the parent [{}] on the node [{}], reason: [{}]", request.parentTaskId,
clusterService.localNode().getId(), request.reason);
taskManager.setBan(request.parentTaskId, request.reason);
} else {
logger.debug("Removing ban for the parent [{}:{}] on the node [{}]", request.parentNodeId, request.parentTaskId,
logger.debug("Removing ban for the parent [{}] on the node [{}]", request.parentTaskId,
clusterService.localNode().getId());
taskManager.removeBan(request.parentNodeId, request.parentTaskId);
taskManager.removeBan(request.parentTaskId);
}
channel.sendResponse(TransportResponse.Empty.INSTANCE);
}

View File

@ -32,14 +32,6 @@ public class ListTasksRequest extends BaseTasksRequest<ListTasksRequest> {
private boolean detailed = false;
/**
* Get information from nodes based on the nodes ids specified. If none are passed, information
* for all nodes will be returned.
*/
public ListTasksRequest(String... nodesIds) {
super(nodesIds);
}
/**
* Should the detailed task information be returned.
*/
@ -48,7 +40,7 @@ public class ListTasksRequest extends BaseTasksRequest<ListTasksRequest> {
}
/**
* Should the node settings be returned.
* Should the detailed task information be returned.
*/
public ListTasksRequest detailed(boolean detailed) {
this.detailed = detailed;

View File

@ -138,11 +138,13 @@ public class ListTasksResponse extends BaseTasksResponse implements ToXContent {
}
builder.endObject();
}
builder.startArray("tasks");
builder.startObject("tasks");
for(TaskInfo task : entry.getValue()) {
builder.startObject(task.getTaskId().toString(), XContentBuilder.FieldCaseConversion.NONE);
task.toXContent(builder, params);
builder.endObject();
}
builder.endArray();
builder.endObject();
builder.endObject();
}
builder.endObject();

View File

@ -26,6 +26,7 @@ import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskId;
import java.io.IOException;
@ -41,7 +42,7 @@ public class TaskInfo implements Writeable<TaskInfo>, ToXContent {
private final DiscoveryNode node;
private final long id;
private final TaskId taskId;
private final String type;
@ -51,28 +52,21 @@ public class TaskInfo implements Writeable<TaskInfo>, ToXContent {
private final Task.Status status;
private final String parentNode;
private final TaskId parentTaskId;
private final long parentId;
public TaskInfo(DiscoveryNode node, long id, String type, String action, String description, Task.Status status) {
this(node, id, type, action, description, status, null, -1L);
}
public TaskInfo(DiscoveryNode node, long id, String type, String action, String description, Task.Status status, String parentNode, long parentId) {
public TaskInfo(DiscoveryNode node, long id, String type, String action, String description, Task.Status status, TaskId parentTaskId) {
this.node = node;
this.id = id;
this.taskId = new TaskId(node.getId(), id);
this.type = type;
this.action = action;
this.description = description;
this.status = status;
this.parentNode = parentNode;
this.parentId = parentId;
this.parentTaskId = parentTaskId;
}
public TaskInfo(StreamInput in) throws IOException {
node = DiscoveryNode.readNode(in);
id = in.readLong();
taskId = new TaskId(node.getId(), in.readLong());
type = in.readString();
action = in.readString();
description = in.readOptionalString();
@ -81,8 +75,11 @@ public class TaskInfo implements Writeable<TaskInfo>, ToXContent {
} else {
status = null;
}
parentNode = in.readOptionalString();
parentId = in.readLong();
parentTaskId = new TaskId(in);
}
public TaskId getTaskId() {
return taskId;
}
public DiscoveryNode getNode() {
@ -90,7 +87,7 @@ public class TaskInfo implements Writeable<TaskInfo>, ToXContent {
}
public long getId() {
return id;
return taskId.getId();
}
public String getType() {
@ -113,12 +110,8 @@ public class TaskInfo implements Writeable<TaskInfo>, ToXContent {
return status;
}
public String getParentNode() {
return parentNode;
}
public long getParentId() {
return parentId;
public TaskId getParentTaskId() {
return parentTaskId;
}
@Override
@ -129,7 +122,7 @@ public class TaskInfo implements Writeable<TaskInfo>, ToXContent {
@Override
public void writeTo(StreamOutput out) throws IOException {
node.writeTo(out);
out.writeLong(id);
out.writeLong(taskId.getId());
out.writeString(type);
out.writeString(action);
out.writeOptionalString(description);
@ -139,15 +132,13 @@ public class TaskInfo implements Writeable<TaskInfo>, ToXContent {
} else {
out.writeBoolean(false);
}
out.writeOptionalString(parentNode);
out.writeLong(parentId);
parentTaskId.writeTo(out);
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject();
builder.field("node", node.getId());
builder.field("id", id);
builder.field("id", taskId.getId());
builder.field("type", type);
builder.field("action", action);
if (status != null) {
@ -156,11 +147,9 @@ public class TaskInfo implements Writeable<TaskInfo>, ToXContent {
if (description != null) {
builder.field("description", description);
}
if (parentNode != null) {
builder.field("parent_node", parentNode);
builder.field("parent_id", parentId);
if (parentTaskId.isSet() == false) {
builder.field("parent_task_id", parentTaskId.toString());
}
builder.endObject();
return builder;
}
}

View File

@ -30,6 +30,7 @@ import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.action.index.TransportIndexAction;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.replication.ReplicationRequest;
import org.elasticsearch.action.support.replication.TransportReplicationAction;
import org.elasticsearch.action.update.UpdateHelper;
import org.elasticsearch.action.update.UpdateRequest;
@ -111,185 +112,7 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
Translog.Location location = null;
for (int requestIndex = 0; requestIndex < request.items().length; requestIndex++) {
BulkItemRequest item = request.items()[requestIndex];
if (item.request() instanceof IndexRequest) {
IndexRequest indexRequest = (IndexRequest) item.request();
preVersions[requestIndex] = indexRequest.version();
preVersionTypes[requestIndex] = indexRequest.versionType();
try {
WriteResult<IndexResponse> result = shardIndexOperation(request, indexRequest, metaData, indexShard, true);
location = locationToSync(location, result.location);
// add the response
IndexResponse indexResponse = result.response();
setResponse(item, new BulkItemResponse(item.id(), indexRequest.opType().lowercase(), indexResponse));
} catch (Throwable e) {
// rethrow the failure if we are going to retry on primary and let parent failure to handle it
if (retryPrimaryException(e)) {
// restore updated versions...
for (int j = 0; j < requestIndex; j++) {
applyVersion(request.items()[j], preVersions[j], preVersionTypes[j]);
}
throw (ElasticsearchException) e;
}
if (ExceptionsHelper.status(e) == RestStatus.CONFLICT) {
logger.trace("{} failed to execute bulk item (index) {}", e, request.shardId(), indexRequest);
} else {
logger.debug("{} failed to execute bulk item (index) {}", e, request.shardId(), indexRequest);
}
// if its a conflict failure, and we already executed the request on a primary (and we execute it
// again, due to primary relocation and only processing up to N bulk items when the shard gets closed)
// then just use the response we got from the successful execution
if (item.getPrimaryResponse() != null && isConflictException(e)) {
setResponse(item, item.getPrimaryResponse());
} else {
setResponse(item, new BulkItemResponse(item.id(), indexRequest.opType().lowercase(),
new BulkItemResponse.Failure(request.index(), indexRequest.type(), indexRequest.id(), e)));
}
}
} else if (item.request() instanceof DeleteRequest) {
DeleteRequest deleteRequest = (DeleteRequest) item.request();
preVersions[requestIndex] = deleteRequest.version();
preVersionTypes[requestIndex] = deleteRequest.versionType();
try {
// add the response
final WriteResult<DeleteResponse> writeResult = TransportDeleteAction.executeDeleteRequestOnPrimary(deleteRequest, indexShard);
DeleteResponse deleteResponse = writeResult.response();
location = locationToSync(location, writeResult.location);
setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_DELETE, deleteResponse));
} catch (Throwable e) {
// rethrow the failure if we are going to retry on primary and let parent failure to handle it
if (retryPrimaryException(e)) {
// restore updated versions...
for (int j = 0; j < requestIndex; j++) {
applyVersion(request.items()[j], preVersions[j], preVersionTypes[j]);
}
throw (ElasticsearchException) e;
}
if (ExceptionsHelper.status(e) == RestStatus.CONFLICT) {
logger.trace("{} failed to execute bulk item (delete) {}", e, request.shardId(), deleteRequest);
} else {
logger.debug("{} failed to execute bulk item (delete) {}", e, request.shardId(), deleteRequest);
}
// if its a conflict failure, and we already executed the request on a primary (and we execute it
// again, due to primary relocation and only processing up to N bulk items when the shard gets closed)
// then just use the response we got from the successful execution
if (item.getPrimaryResponse() != null && isConflictException(e)) {
setResponse(item, item.getPrimaryResponse());
} else {
setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_DELETE,
new BulkItemResponse.Failure(request.index(), deleteRequest.type(), deleteRequest.id(), e)));
}
}
} else if (item.request() instanceof UpdateRequest) {
UpdateRequest updateRequest = (UpdateRequest) item.request();
preVersions[requestIndex] = updateRequest.version();
preVersionTypes[requestIndex] = updateRequest.versionType();
// We need to do the requested retries plus the initial attempt. We don't do < 1+retry_on_conflict because retry_on_conflict may be Integer.MAX_VALUE
for (int updateAttemptsCount = 0; updateAttemptsCount <= updateRequest.retryOnConflict(); updateAttemptsCount++) {
UpdateResult updateResult;
try {
updateResult = shardUpdateOperation(metaData, request, updateRequest, indexShard);
} catch (Throwable t) {
updateResult = new UpdateResult(null, null, false, t, null);
}
if (updateResult.success()) {
if (updateResult.writeResult != null) {
location = locationToSync(location, updateResult.writeResult.location);
}
switch (updateResult.result.operation()) {
case UPSERT:
case INDEX:
WriteResult<IndexResponse> result = updateResult.writeResult;
IndexRequest indexRequest = updateResult.request();
BytesReference indexSourceAsBytes = indexRequest.source();
// add the response
IndexResponse indexResponse = result.response();
UpdateResponse updateResponse = new UpdateResponse(indexResponse.getShardInfo(), indexResponse.getShardId(), indexResponse.getType(), indexResponse.getId(), indexResponse.getVersion(), indexResponse.isCreated());
if (updateRequest.fields() != null && updateRequest.fields().length > 0) {
Tuple<XContentType, Map<String, Object>> sourceAndContent = XContentHelper.convertToMap(indexSourceAsBytes, true);
updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, request.index(), indexResponse.getVersion(), sourceAndContent.v2(), sourceAndContent.v1(), indexSourceAsBytes));
}
item = request.items()[requestIndex] = new BulkItemRequest(request.items()[requestIndex].id(), indexRequest);
setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_UPDATE, updateResponse));
break;
case DELETE:
WriteResult<DeleteResponse> writeResult = updateResult.writeResult;
DeleteResponse response = writeResult.response();
DeleteRequest deleteRequest = updateResult.request();
updateResponse = new UpdateResponse(response.getShardInfo(), response.getShardId(), response.getType(), response.getId(), response.getVersion(), false);
updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, request.index(), response.getVersion(), updateResult.result.updatedSourceAsMap(), updateResult.result.updateSourceContentType(), null));
// Replace the update request to the translated delete request to execute on the replica.
item = request.items()[requestIndex] = new BulkItemRequest(request.items()[requestIndex].id(), deleteRequest);
setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_UPDATE, updateResponse));
break;
case NONE:
setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_UPDATE, updateResult.noopResult));
item.setIgnoreOnReplica(); // no need to go to the replica
break;
}
// NOTE: Breaking out of the retry_on_conflict loop!
break;
} else if (updateResult.failure()) {
Throwable t = updateResult.error;
if (updateResult.retry) {
// updateAttemptCount is 0 based and marks current attempt, if it's equal to retryOnConflict we are going out of the iteration
if (updateAttemptsCount >= updateRequest.retryOnConflict()) {
setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_UPDATE,
new BulkItemResponse.Failure(request.index(), updateRequest.type(), updateRequest.id(), t)));
}
} else {
// rethrow the failure if we are going to retry on primary and let parent failure to handle it
if (retryPrimaryException(t)) {
// restore updated versions...
for (int j = 0; j < requestIndex; j++) {
applyVersion(request.items()[j], preVersions[j], preVersionTypes[j]);
}
throw (ElasticsearchException) t;
}
// if its a conflict failure, and we already executed the request on a primary (and we execute it
// again, due to primary relocation and only processing up to N bulk items when the shard gets closed)
// then just use the response we got from the successful execution
if (item.getPrimaryResponse() != null && isConflictException(t)) {
setResponse(item, item.getPrimaryResponse());
} else if (updateResult.result == null) {
setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_UPDATE, new BulkItemResponse.Failure(request.index(), updateRequest.type(), updateRequest.id(), t)));
} else {
switch (updateResult.result.operation()) {
case UPSERT:
case INDEX:
IndexRequest indexRequest = updateResult.request();
if (ExceptionsHelper.status(t) == RestStatus.CONFLICT) {
logger.trace("{} failed to execute bulk item (index) {}", t, request.shardId(), indexRequest);
} else {
logger.debug("{} failed to execute bulk item (index) {}", t, request.shardId(), indexRequest);
}
setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_UPDATE,
new BulkItemResponse.Failure(request.index(), indexRequest.type(), indexRequest.id(), t)));
break;
case DELETE:
DeleteRequest deleteRequest = updateResult.request();
if (ExceptionsHelper.status(t) == RestStatus.CONFLICT) {
logger.trace("{} failed to execute bulk item (delete) {}", t, request.shardId(), deleteRequest);
} else {
logger.debug("{} failed to execute bulk item (delete) {}", t, request.shardId(), deleteRequest);
}
setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_DELETE,
new BulkItemResponse.Failure(request.index(), deleteRequest.type(), deleteRequest.id(), t)));
break;
}
}
// NOTE: Breaking out of the retry_on_conflict loop!
break;
}
}
}
} else {
throw new IllegalStateException("Unexpected index operation: " + item.request());
}
assert item.getPrimaryResponse() != null;
assert preVersionTypes[requestIndex] != null;
location = handleItem(metaData, request, indexShard, preVersions, preVersionTypes, location, requestIndex, item);
}
processAfterWrite(request.refresh(), indexShard, location);
@ -301,6 +124,198 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
return new Tuple<>(new BulkShardResponse(request.shardId(), responses), request);
}
private Translog.Location handleItem(MetaData metaData, BulkShardRequest request, IndexShard indexShard, long[] preVersions, VersionType[] preVersionTypes, Translog.Location location, int requestIndex, BulkItemRequest item) {
if (item.request() instanceof IndexRequest) {
location = index(metaData, request, indexShard, preVersions, preVersionTypes, location, requestIndex, item);
} else if (item.request() instanceof DeleteRequest) {
location = delete(request, indexShard, preVersions, preVersionTypes, location, requestIndex, item);
} else if (item.request() instanceof UpdateRequest) {
Tuple<Translog.Location, BulkItemRequest> tuple = update(metaData, request, indexShard, preVersions, preVersionTypes, location, requestIndex, item);
location = tuple.v1();
item = tuple.v2();
} else {
throw new IllegalStateException("Unexpected index operation: " + item.request());
}
assert item.getPrimaryResponse() != null;
assert preVersionTypes[requestIndex] != null;
return location;
}
private Translog.Location index(MetaData metaData, BulkShardRequest request, IndexShard indexShard, long[] preVersions, VersionType[] preVersionTypes, Translog.Location location, int requestIndex, BulkItemRequest item) {
IndexRequest indexRequest = (IndexRequest) item.request();
preVersions[requestIndex] = indexRequest.version();
preVersionTypes[requestIndex] = indexRequest.versionType();
try {
WriteResult<IndexResponse> result = shardIndexOperation(request, indexRequest, metaData, indexShard, true);
location = locationToSync(location, result.location);
// add the response
IndexResponse indexResponse = result.response();
setResponse(item, new BulkItemResponse(item.id(), indexRequest.opType().lowercase(), indexResponse));
} catch (Throwable e) {
// rethrow the failure if we are going to retry on primary and let parent failure to handle it
if (retryPrimaryException(e)) {
// restore updated versions...
for (int j = 0; j < requestIndex; j++) {
applyVersion(request.items()[j], preVersions[j], preVersionTypes[j]);
}
throw (ElasticsearchException) e;
}
logFailure(e, "index", request.shardId(), indexRequest);
// if its a conflict failure, and we already executed the request on a primary (and we execute it
// again, due to primary relocation and only processing up to N bulk items when the shard gets closed)
// then just use the response we got from the successful execution
if (item.getPrimaryResponse() != null && isConflictException(e)) {
setResponse(item, item.getPrimaryResponse());
} else {
setResponse(item, new BulkItemResponse(item.id(), indexRequest.opType().lowercase(),
new BulkItemResponse.Failure(request.index(), indexRequest.type(), indexRequest.id(), e)));
}
}
return location;
}
private <ReplicationRequestT extends ReplicationRequest<ReplicationRequestT>> void logFailure(Throwable e, String operation, ShardId shardId, ReplicationRequest<ReplicationRequestT> request) {
if (ExceptionsHelper.status(e) == RestStatus.CONFLICT) {
logger.trace("{} failed to execute bulk item ({}) {}", e, shardId, operation, request);
} else {
logger.debug("{} failed to execute bulk item ({}) {}", e, shardId, operation, request);
}
}
private Translog.Location delete(BulkShardRequest request, IndexShard indexShard, long[] preVersions, VersionType[] preVersionTypes, Translog.Location location, int requestIndex, BulkItemRequest item) {
DeleteRequest deleteRequest = (DeleteRequest) item.request();
preVersions[requestIndex] = deleteRequest.version();
preVersionTypes[requestIndex] = deleteRequest.versionType();
try {
// add the response
final WriteResult<DeleteResponse> writeResult = TransportDeleteAction.executeDeleteRequestOnPrimary(deleteRequest, indexShard);
DeleteResponse deleteResponse = writeResult.response();
location = locationToSync(location, writeResult.location);
setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_DELETE, deleteResponse));
} catch (Throwable e) {
// rethrow the failure if we are going to retry on primary and let parent failure to handle it
if (retryPrimaryException(e)) {
// restore updated versions...
for (int j = 0; j < requestIndex; j++) {
applyVersion(request.items()[j], preVersions[j], preVersionTypes[j]);
}
throw (ElasticsearchException) e;
}
logFailure(e, "delete", request.shardId(), deleteRequest);
// if its a conflict failure, and we already executed the request on a primary (and we execute it
// again, due to primary relocation and only processing up to N bulk items when the shard gets closed)
// then just use the response we got from the successful execution
if (item.getPrimaryResponse() != null && isConflictException(e)) {
setResponse(item, item.getPrimaryResponse());
} else {
setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_DELETE,
new BulkItemResponse.Failure(request.index(), deleteRequest.type(), deleteRequest.id(), e)));
}
}
return location;
}
private Tuple<Translog.Location, BulkItemRequest> update(MetaData metaData, BulkShardRequest request, IndexShard indexShard, long[] preVersions, VersionType[] preVersionTypes, Translog.Location location, int requestIndex, BulkItemRequest item) {
UpdateRequest updateRequest = (UpdateRequest) item.request();
preVersions[requestIndex] = updateRequest.version();
preVersionTypes[requestIndex] = updateRequest.versionType();
// We need to do the requested retries plus the initial attempt. We don't do < 1+retry_on_conflict because retry_on_conflict may be Integer.MAX_VALUE
for (int updateAttemptsCount = 0; updateAttemptsCount <= updateRequest.retryOnConflict(); updateAttemptsCount++) {
UpdateResult updateResult;
try {
updateResult = shardUpdateOperation(metaData, request, updateRequest, indexShard);
} catch (Throwable t) {
updateResult = new UpdateResult(null, null, false, t, null);
}
if (updateResult.success()) {
if (updateResult.writeResult != null) {
location = locationToSync(location, updateResult.writeResult.location);
}
switch (updateResult.result.operation()) {
case UPSERT:
case INDEX:
WriteResult<IndexResponse> result = updateResult.writeResult;
IndexRequest indexRequest = updateResult.request();
BytesReference indexSourceAsBytes = indexRequest.source();
// add the response
IndexResponse indexResponse = result.response();
UpdateResponse updateResponse = new UpdateResponse(indexResponse.getShardInfo(), indexResponse.getShardId(), indexResponse.getType(), indexResponse.getId(), indexResponse.getVersion(), indexResponse.isCreated());
if (updateRequest.fields() != null && updateRequest.fields().length > 0) {
Tuple<XContentType, Map<String, Object>> sourceAndContent = XContentHelper.convertToMap(indexSourceAsBytes, true);
updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, request.index(), indexResponse.getVersion(), sourceAndContent.v2(), sourceAndContent.v1(), indexSourceAsBytes));
}
item = request.items()[requestIndex] = new BulkItemRequest(request.items()[requestIndex].id(), indexRequest);
setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_UPDATE, updateResponse));
break;
case DELETE:
WriteResult<DeleteResponse> writeResult = updateResult.writeResult;
DeleteResponse response = writeResult.response();
DeleteRequest deleteRequest = updateResult.request();
updateResponse = new UpdateResponse(response.getShardInfo(), response.getShardId(), response.getType(), response.getId(), response.getVersion(), false);
updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, request.index(), response.getVersion(), updateResult.result.updatedSourceAsMap(), updateResult.result.updateSourceContentType(), null));
// Replace the update request to the translated delete request to execute on the replica.
item = request.items()[requestIndex] = new BulkItemRequest(request.items()[requestIndex].id(), deleteRequest);
setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_UPDATE, updateResponse));
break;
case NONE:
setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_UPDATE, updateResult.noopResult));
item.setIgnoreOnReplica(); // no need to go to the replica
break;
}
// NOTE: Breaking out of the retry_on_conflict loop!
break;
} else if (updateResult.failure()) {
Throwable t = updateResult.error;
if (updateResult.retry) {
// updateAttemptCount is 0 based and marks current attempt, if it's equal to retryOnConflict we are going out of the iteration
if (updateAttemptsCount >= updateRequest.retryOnConflict()) {
setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_UPDATE,
new BulkItemResponse.Failure(request.index(), updateRequest.type(), updateRequest.id(), t)));
}
} else {
// rethrow the failure if we are going to retry on primary and let parent failure to handle it
if (retryPrimaryException(t)) {
// restore updated versions...
for (int j = 0; j < requestIndex; j++) {
applyVersion(request.items()[j], preVersions[j], preVersionTypes[j]);
}
throw (ElasticsearchException) t;
}
// if its a conflict failure, and we already executed the request on a primary (and we execute it
// again, due to primary relocation and only processing up to N bulk items when the shard gets closed)
// then just use the response we got from the successful execution
if (item.getPrimaryResponse() != null && isConflictException(t)) {
setResponse(item, item.getPrimaryResponse());
} else if (updateResult.result == null) {
setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_UPDATE, new BulkItemResponse.Failure(request.index(), updateRequest.type(), updateRequest.id(), t)));
} else {
switch (updateResult.result.operation()) {
case UPSERT:
case INDEX:
IndexRequest indexRequest = updateResult.request();
logFailure(t, "index", request.shardId(), indexRequest);
setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_UPDATE,
new BulkItemResponse.Failure(request.index(), indexRequest.type(), indexRequest.id(), t)));
break;
case DELETE:
DeleteRequest deleteRequest = updateResult.request();
logFailure(t, "delete", request.shardId(), deleteRequest);
setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_DELETE,
new BulkItemResponse.Failure(request.index(), deleteRequest.type(), deleteRequest.id(), t)));
break;
}
}
// NOTE: Breaking out of the retry_on_conflict loop!
break;
}
}
}
return Tuple.tuple(location, item);
}
private void setResponse(BulkItemRequest request, BulkItemResponse response) {
request.setPrimaryResponse(response);
if (response.isFailed()) {

View File

@ -23,6 +23,7 @@ import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskId;
import java.io.IOException;
@ -31,40 +32,35 @@ import java.io.IOException;
*/
public abstract class ChildTaskActionRequest<Request extends ActionRequest<Request>> extends ActionRequest<Request> {
private String parentTaskNode;
private long parentTaskId;
private TaskId parentTaskId = TaskId.EMPTY_TASK_ID;
protected ChildTaskActionRequest() {
}
public void setParentTask(String parentTaskNode, long parentTaskId) {
this.parentTaskNode = parentTaskNode;
this.parentTaskId = parentTaskId;
this.parentTaskId = new TaskId(parentTaskNode, parentTaskId);
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
parentTaskNode = in.readOptionalString();
parentTaskId = in.readLong();
parentTaskId = new TaskId(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeOptionalString(parentTaskNode);
out.writeLong(parentTaskId);
parentTaskId.writeTo(out);
}
@Override
public final Task createTask(long id, String type, String action) {
return createTask(id, type, action, parentTaskNode, parentTaskId);
return createTask(id, type, action, parentTaskId);
}
public Task createTask(long id, String type, String action, String parentTaskNode, long parentTaskId) {
return new Task(id, type, action, getDescription(), parentTaskNode, parentTaskId);
public Task createTask(long id, String type, String action, TaskId parentTaskId) {
return new Task(id, type, action, getDescription(), parentTaskId);
}
}

View File

@ -22,6 +22,7 @@ package org.elasticsearch.action.support;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.transport.TransportRequest;
import java.io.IOException;
@ -31,38 +32,33 @@ import java.io.IOException;
*/
public class ChildTaskRequest extends TransportRequest {
private String parentTaskNode;
private long parentTaskId;
private TaskId parentTaskId = TaskId.EMPTY_TASK_ID;
protected ChildTaskRequest() {
}
public void setParentTask(String parentTaskNode, long parentTaskId) {
this.parentTaskNode = parentTaskNode;
this.parentTaskId = parentTaskId;
this.parentTaskId = new TaskId(parentTaskNode, parentTaskId);
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
parentTaskNode = in.readOptionalString();
parentTaskId = in.readLong();
parentTaskId = new TaskId(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeOptionalString(parentTaskNode);
out.writeLong(parentTaskId);
parentTaskId.writeTo(out);
}
@Override
public final Task createTask(long id, String type, String action) {
return createTask(id, type, action, parentTaskNode, parentTaskId);
return createTask(id, type, action, parentTaskId);
}
public Task createTask(long id, String type, String action, String parentTaskNode, long parentTaskId) {
return new Task(id, type, action, getDescription(), parentTaskNode, parentTaskId);
public Task createTask(long id, String type, String action, TaskId parentTaskId) {
return new Task(id, type, action, getDescription(), parentTaskId);
}
}

View File

@ -30,6 +30,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskId;
import java.io.IOException;
import java.util.concurrent.TimeUnit;
@ -186,8 +187,8 @@ public abstract class ReplicationRequest<Request extends ReplicationRequest<Requ
}
@Override
public Task createTask(long id, String type, String action, String parentTaskNode, long parentTaskId) {
return new ReplicationTask(id, type, action, getDescription(), parentTaskNode, parentTaskId);
public Task createTask(long id, String type, String action, TaskId parentTaskId) {
return new ReplicationTask(id, type, action, getDescription(), parentTaskId);
}
/**

View File

@ -19,11 +19,11 @@
package org.elasticsearch.action.support.replication;
import org.elasticsearch.common.inject.Provider;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskId;
import java.io.IOException;
@ -35,8 +35,8 @@ import static java.util.Objects.requireNonNull;
public class ReplicationTask extends Task {
private volatile String phase = "starting";
public ReplicationTask(long id, String type, String action, String description, String parentNode, long parentId) {
super(id, type, action, description, parentNode, parentId);
public ReplicationTask(long id, String type, String action, String description, TaskId parentTaskId) {
super(id, type, action, description, parentTaskId);
}
/**

View File

@ -83,6 +83,7 @@ import java.util.Map;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Consumer;
import java.util.function.Supplier;
/**
@ -461,60 +462,90 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
protected void doRun() {
setPhase(task, "routing");
final ClusterState state = observer.observedState();
ClusterBlockException blockException = state.blocks().globalBlockedException(globalBlockLevel());
if (blockException != null) {
handleBlockException(blockException);
return;
}
final String concreteIndex = resolveIndex() ? indexNameExpressionResolver.concreteSingleIndex(state, request) : request.index();
blockException = state.blocks().indexBlockedException(indexBlockLevel(), concreteIndex);
if (blockException != null) {
handleBlockException(blockException);
if (handleBlockExceptions(state)) {
return;
}
// request does not have a shardId yet, we need to pass the concrete index to resolve shardId
final String concreteIndex = concreteIndex(state);
resolveRequest(state.metaData(), concreteIndex, request);
assert request.shardId() != null : "request shardId must be set in resolveRequest";
IndexShardRoutingTable indexShard = state.getRoutingTable().shardRoutingTable(request.shardId());
final ShardRouting primary = indexShard.primaryShard();
if (primary == null || primary.active() == false) {
logger.trace("primary shard [{}] is not yet active, scheduling a retry: action [{}], request [{}], cluster state version [{}]", request.shardId(), actionName, request, state.version());
retryBecauseUnavailable(request.shardId(), "primary shard is not active");
return;
}
if (state.nodes().nodeExists(primary.currentNodeId()) == false) {
logger.trace("primary shard [{}] is assigned to an unknown node [{}], scheduling a retry: action [{}], request [{}], cluster state version [{}]", request.shardId(), primary.currentNodeId(), actionName, request, state.version());
retryBecauseUnavailable(request.shardId(), "primary shard isn't assigned to a known node.");
final ShardRouting primary = primary(state);
if (retryIfUnavailable(state, primary)) {
return;
}
final DiscoveryNode node = state.nodes().get(primary.currentNodeId());
taskManager.registerChildTask(task, node.getId());
if (primary.currentNodeId().equals(state.nodes().localNodeId())) {
setPhase(task, "waiting_on_primary");
if (logger.isTraceEnabled()) {
logger.trace("send action [{}] on primary [{}] for request [{}] with cluster state version [{}] to [{}] ", transportPrimaryAction, request.shardId(), request, state.version(), primary.currentNodeId());
}
performAction(node, transportPrimaryAction, true);
performLocalAction(state, primary, node);
} else {
if (state.version() < request.routedBasedOnClusterVersion()) {
logger.trace("failed to find primary [{}] for request [{}] despite sender thinking it would be here. Local cluster state version [{}]] is older than on sending node (version [{}]), scheduling a retry...", request.shardId(), request, state.version(), request.routedBasedOnClusterVersion());
retryBecauseUnavailable(request.shardId(), "failed to find primary as current cluster state with version [" + state.version() + "] is stale (expected at least [" + request.routedBasedOnClusterVersion() + "]");
return;
} else {
// chasing the node with the active primary for a second hop requires that we are at least up-to-date with the current cluster state version
// this prevents redirect loops between two nodes when a primary was relocated and the relocation target is not aware that it is the active primary shard already.
request.routedBasedOnClusterVersion(state.version());
}
if (logger.isTraceEnabled()) {
logger.trace("send action [{}] on primary [{}] for request [{}] with cluster state version [{}] to [{}]", actionName, request.shardId(), request, state.version(), primary.currentNodeId());
}
setPhase(task, "rerouted");
performAction(node, actionName, false);
performRemoteAction(state, primary, node);
}
}
private void performLocalAction(ClusterState state, ShardRouting primary, DiscoveryNode node) {
setPhase(task, "waiting_on_primary");
if (logger.isTraceEnabled()) {
logger.trace("send action [{}] on primary [{}] for request [{}] with cluster state version [{}] to [{}] ", transportPrimaryAction, request.shardId(), request, state.version(), primary.currentNodeId());
}
performAction(node, transportPrimaryAction, true);
}
private void performRemoteAction(ClusterState state, ShardRouting primary, DiscoveryNode node) {
if (state.version() < request.routedBasedOnClusterVersion()) {
logger.trace("failed to find primary [{}] for request [{}] despite sender thinking it would be here. Local cluster state version [{}]] is older than on sending node (version [{}]), scheduling a retry...", request.shardId(), request, state.version(), request.routedBasedOnClusterVersion());
retryBecauseUnavailable(request.shardId(), "failed to find primary as current cluster state with version [" + state.version() + "] is stale (expected at least [" + request.routedBasedOnClusterVersion() + "]");
return;
} else {
// chasing the node with the active primary for a second hop requires that we are at least up-to-date with the current cluster state version
// this prevents redirect loops between two nodes when a primary was relocated and the relocation target is not aware that it is the active primary shard already.
request.routedBasedOnClusterVersion(state.version());
}
if (logger.isTraceEnabled()) {
logger.trace("send action [{}] on primary [{}] for request [{}] with cluster state version [{}] to [{}]", actionName, request.shardId(), request, state.version(), primary.currentNodeId());
}
setPhase(task, "rerouted");
performAction(node, actionName, false);
}
private boolean retryIfUnavailable(ClusterState state, ShardRouting primary) {
if (primary == null || primary.active() == false) {
logger.trace("primary shard [{}] is not yet active, scheduling a retry: action [{}], request [{}], cluster state version [{}]", request.shardId(), actionName, request, state.version());
retryBecauseUnavailable(request.shardId(), "primary shard is not active");
return true;
}
if (state.nodes().nodeExists(primary.currentNodeId()) == false) {
logger.trace("primary shard [{}] is assigned to an unknown node [{}], scheduling a retry: action [{}], request [{}], cluster state version [{}]", request.shardId(), primary.currentNodeId(), actionName, request, state.version());
retryBecauseUnavailable(request.shardId(), "primary shard isn't assigned to a known node.");
return true;
}
return false;
}
private String concreteIndex(ClusterState state) {
return resolveIndex() ? indexNameExpressionResolver.concreteSingleIndex(state, request) : request.index();
}
private ShardRouting primary(ClusterState state) {
IndexShardRoutingTable indexShard = state.getRoutingTable().shardRoutingTable(request.shardId());
return indexShard.primaryShard();
}
private boolean handleBlockExceptions(ClusterState state) {
ClusterBlockException blockException = state.blocks().globalBlockedException(globalBlockLevel());
if (blockException != null) {
handleBlockException(blockException);
return true;
}
blockException = state.blocks().indexBlockedException(indexBlockLevel(), concreteIndex(state));
if (blockException != null) {
handleBlockException(blockException);
return true;
}
return false;
}
private void handleBlockException(ClusterBlockException blockException) {
if (blockException.retryable()) {
logger.trace("cluster is blocked ({}), scheduling a retry", blockException.getMessage());
@ -677,27 +708,36 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
// closed in finishAsFailed(e) in the case of error
indexShardReference = getIndexShardReferenceOnPrimary(shardId);
if (indexShardReference.isRelocated() == false) {
// execute locally
Tuple<Response, ReplicaRequest> primaryResponse = shardOperationOnPrimary(state.metaData(), request);
if (logger.isTraceEnabled()) {
logger.trace("action [{}] completed on shard [{}] for request [{}] with cluster state version [{}]", transportPrimaryAction, shardId, request, state.version());
}
ReplicationPhase replicationPhase = new ReplicationPhase(task, primaryResponse.v2(), primaryResponse.v1(), shardId, channel, indexShardReference);
finishAndMoveToReplication(replicationPhase);
executeLocally();
} else {
// delegate primary phase to relocation target
// it is safe to execute primary phase on relocation target as there are no more in-flight operations where primary
// phase is executed on local shard and all subsequent operations are executed on relocation target as primary phase.
final ShardRouting primary = indexShardReference.routingEntry();
indexShardReference.close();
assert primary.relocating() : "indexShard is marked as relocated but routing isn't" + primary;
DiscoveryNode relocatingNode = state.nodes().get(primary.relocatingNodeId());
transportService.sendRequest(relocatingNode, transportPrimaryAction, request, transportOptions,
TransportChannelResponseHandler.responseHandler(logger, TransportReplicationAction.this::newResponseInstance, channel,
"rerouting indexing to target primary " + primary));
executeRemotely();
}
}
private void executeLocally() throws Exception {
// execute locally
Tuple<Response, ReplicaRequest> primaryResponse = shardOperationOnPrimary(state.metaData(), request);
if (logger.isTraceEnabled()) {
logger.trace("action [{}] completed on shard [{}] for request [{}] with cluster state version [{}]", transportPrimaryAction, shardId, request, state.version());
}
ReplicationPhase replicationPhase = new ReplicationPhase(task, primaryResponse.v2(), primaryResponse.v1(), shardId, channel, indexShardReference);
finishAndMoveToReplication(replicationPhase);
}
private void executeRemotely() {
// delegate primary phase to relocation target
// it is safe to execute primary phase on relocation target as there are no more in-flight operations where primary
// phase is executed on local shard and all subsequent operations are executed on relocation target as primary phase.
final ShardRouting primary = indexShardReference.routingEntry();
indexShardReference.close();
assert primary.relocating() : "indexShard is marked as relocated but routing isn't" + primary;
DiscoveryNode relocatingNode = state.nodes().get(primary.relocatingNodeId());
transportService.sendRequest(relocatingNode, transportPrimaryAction, request, transportOptions,
TransportChannelResponseHandler.responseHandler(logger, TransportReplicationAction.this::newResponseInstance, channel,
"rerouting indexing to target primary " + primary));
}
/**
* checks whether we can perform a write based on the write consistency setting
* returns **null* if OK to proceed, or a string describing the reason to stop
@ -835,23 +875,48 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
// to the recovery target. If we use an old cluster state, we may miss a relocation that has started since then.
// If the index gets deleted after primary operation, we skip replication
final ClusterState state = clusterService.state();
final IndexRoutingTable index = state.getRoutingTable().index(shardId.getIndex());
final IndexShardRoutingTable shardRoutingTable = (index != null) ? index.shard(shardId.id()) : null;
final IndexShardRoutingTable shardRoutingTable = state.getRoutingTable().shardRoutingTableOrNull(shardId);
final IndexMetaData indexMetaData = state.getMetaData().index(shardId.getIndex());
this.shards = (shardRoutingTable != null) ? shardRoutingTable.shards() : Collections.emptyList();
this.executeOnReplica = (indexMetaData == null) || shouldExecuteReplication(indexMetaData.getSettings());
this.nodes = state.getNodes();
List<ShardRouting> shards = shards(shardRoutingTable);
boolean executeOnReplica = (indexMetaData == null) || shouldExecuteReplication(indexMetaData.getSettings());
DiscoveryNodes nodes = state.getNodes();
if (shards.isEmpty()) {
logger.debug("replication phase for request [{}] on [{}] is skipped due to index deletion after primary operation", replicaRequest, shardId);
}
// we calculate number of target nodes to send replication operations, including nodes with relocating shards
AtomicInteger numberOfPendingShardInstances = new AtomicInteger();
this.totalShards = countTotalAndPending(shards, executeOnReplica, nodes, numberOfPendingShardInstances);
this.pending = numberOfPendingShardInstances;
this.shards = shards;
this.executeOnReplica = executeOnReplica;
this.nodes = nodes;
if (logger.isTraceEnabled()) {
logger.trace("replication phase started. pending [{}], action [{}], request [{}], cluster state version used [{}]", pending.get(),
transportReplicaAction, replicaRequest, state.version());
}
}
private int countTotalAndPending(List<ShardRouting> shards, boolean executeOnReplica, DiscoveryNodes nodes, AtomicInteger pending) {
assert pending.get() == 0;
int numberOfIgnoredShardInstances = performOnShards(shards, executeOnReplica, nodes, shard -> pending.incrementAndGet(), shard -> pending.incrementAndGet());
// one for the local primary copy
return 1 + numberOfIgnoredShardInstances + pending.get();
}
private int performOnShards(List<ShardRouting> shards, boolean executeOnReplica, DiscoveryNodes nodes, Consumer<ShardRouting> onLocalShard, Consumer<ShardRouting> onRelocatingShard) {
int numberOfIgnoredShardInstances = 0;
int numberOfPendingShardInstances = 0;
for (ShardRouting shard : shards) {
// the following logic to select the shards to replicate to is mirrored and explained in the doRun method below
if (shard.primary() == false && executeOnReplica == false) {
// If the replicas use shadow replicas, there is no reason to
// perform the action on the replica, so skip it and
// immediately return
// this delays mapping updates on replicas because they have
// to wait until they get the new mapping through the cluster
// state, which is why we recommend pre-defined mappings for
// indices using shadow replicas
numberOfIgnoredShardInstances++;
continue;
}
@ -859,20 +924,26 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
numberOfIgnoredShardInstances++;
continue;
}
// we index on a replica that is initializing as well since we might not have got the event
// yet that it was started. We will get an exception IllegalShardState exception if its not started
// and that's fine, we will ignore it
// we never execute replication operation locally as primary operation has already completed locally
// hence, we ignore any local shard for replication
if (nodes.localNodeId().equals(shard.currentNodeId()) == false) {
numberOfPendingShardInstances++;
onLocalShard.accept(shard);
}
// send operation to relocating shard
// local shard can be a relocation target of a primary that is in relocated state
if (shard.relocating() && nodes.localNodeId().equals(shard.relocatingNodeId()) == false) {
numberOfPendingShardInstances++;
onRelocatingShard.accept(shard);
}
}
// one for the local primary copy
this.totalShards = 1 + numberOfPendingShardInstances + numberOfIgnoredShardInstances;
this.pending = new AtomicInteger(numberOfPendingShardInstances);
if (logger.isTraceEnabled()) {
logger.trace("replication phase started. pending [{}], action [{}], request [{}], cluster state version used [{}]", pending.get(),
transportReplicaAction, replicaRequest, state.version());
}
return numberOfIgnoredShardInstances;
}
private List<ShardRouting> shards(IndexShardRoutingTable shardRoutingTable) {
return (shardRoutingTable != null) ? shardRoutingTable.shards() : Collections.emptyList();
}
/**
@ -912,36 +983,7 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
doFinish();
return;
}
for (ShardRouting shard : shards) {
if (shard.primary() == false && executeOnReplica == false) {
// If the replicas use shadow replicas, there is no reason to
// perform the action on the replica, so skip it and
// immediately return
// this delays mapping updates on replicas because they have
// to wait until they get the new mapping through the cluster
// state, which is why we recommend pre-defined mappings for
// indices using shadow replicas
continue;
}
if (shard.unassigned()) {
continue;
}
// we index on a replica that is initializing as well since we might not have got the event
// yet that it was started. We will get an exception IllegalShardState exception if its not started
// and that's fine, we will ignore it
// we never execute replication operation locally as primary operation has already completed locally
// hence, we ignore any local shard for replication
if (nodes.localNodeId().equals(shard.currentNodeId()) == false) {
performOnReplica(shard);
}
// send operation to relocating shard
// local shard can be a relocation target of a primary that is in relocated state
if (shard.relocating() && nodes.localNodeId().equals(shard.relocatingNodeId()) == false) {
performOnReplica(shard.buildTargetRelocatingShard());
}
}
performOnShards(shards, executeOnReplica, nodes, shard -> performOnReplica(shard), shard -> performOnReplica(shard.buildTargetRelocatingShard()));
}
/**

View File

@ -27,9 +27,12 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskId;
import java.io.IOException;
import static org.elasticsearch.action.ValidateActions.addValidationError;
/**
* A base class for task requests
*/
@ -47,26 +50,21 @@ public class BaseTasksRequest<Request extends BaseTasksRequest<Request>> extends
private String[] actions = ALL_ACTIONS;
private String parentNode;
private TaskId parentTaskId = TaskId.EMPTY_TASK_ID;
private long parentTaskId = ALL_TASKS;
private long taskId = ALL_TASKS;
private TaskId taskId = TaskId.EMPTY_TASK_ID;
public BaseTasksRequest() {
}
@Override
public ActionRequestValidationException validate() {
return null;
}
/**
* Get information about tasks from nodes based on the nodes ids specified.
* If none are passed, information for all nodes will be returned.
*/
public BaseTasksRequest(String... nodesIds) {
this.nodesIds = nodesIds;
ActionRequestValidationException validationException = null;
if (taskId.isSet() == false && nodesIds.length > 0) {
validationException = addValidationError("task id cannot be used together with node ids",
validationException);
}
return validationException;
}
/**
@ -100,39 +98,26 @@ public class BaseTasksRequest<Request extends BaseTasksRequest<Request>> extends
*
* By default tasks with any ids are returned.
*/
public long taskId() {
public TaskId taskId() {
return taskId;
}
@SuppressWarnings("unchecked")
public final Request taskId(long taskId) {
public final Request taskId(TaskId taskId) {
this.taskId = taskId;
return (Request) this;
}
/**
* Returns the parent node id that tasks should be filtered by
*/
public String parentNode() {
return parentNode;
}
@SuppressWarnings("unchecked")
public Request parentNode(String parentNode) {
this.parentNode = parentNode;
return (Request) this;
}
/**
* Returns the parent task id that tasks should be filtered by
*/
public long parentTaskId() {
public TaskId parentTaskId() {
return parentTaskId;
}
@SuppressWarnings("unchecked")
public Request parentTaskId(long parentTaskId) {
public Request parentTaskId(TaskId parentTaskId) {
this.parentTaskId = parentTaskId;
return (Request) this;
}
@ -157,11 +142,10 @@ public class BaseTasksRequest<Request extends BaseTasksRequest<Request>> extends
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
taskId = new TaskId(in);
parentTaskId = new TaskId(in);
nodesIds = in.readStringArray();
taskId = in.readLong();
actions = in.readStringArray();
parentNode = in.readOptionalString();
parentTaskId = in.readLong();
if (in.readBoolean()) {
timeout = TimeValue.readTimeValue(in);
}
@ -170,11 +154,10 @@ public class BaseTasksRequest<Request extends BaseTasksRequest<Request>> extends
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
taskId.writeTo(out);
parentTaskId.writeTo(out);
out.writeStringArrayNullable(nodesIds);
out.writeLong(taskId);
out.writeStringArrayNullable(actions);
out.writeOptionalString(parentNode);
out.writeLong(parentTaskId);
out.writeOptionalStreamable(timeout);
}
@ -182,18 +165,13 @@ public class BaseTasksRequest<Request extends BaseTasksRequest<Request>> extends
if (actions() != null && actions().length > 0 && Regex.simpleMatch(actions(), task.getAction()) == false) {
return false;
}
if (taskId() != ALL_TASKS) {
if(taskId() != task.getId()) {
if (taskId().isSet() == false) {
if(taskId().getId() != task.getId()) {
return false;
}
}
if (parentNode() != null) {
if (parentNode().equals(task.getParentNode()) == false) {
return false;
}
}
if (parentTaskId() != ALL_TASKS) {
if (parentTaskId() != task.getParentId()) {
if (parentTaskId.isSet() == false) {
if (parentTaskId.equals(task.getParentTaskId()) == false) {
return false;
}
}

View File

@ -124,13 +124,17 @@ public abstract class TransportTasksAction<
}
protected String[] resolveNodes(TasksRequest request, ClusterState clusterState) {
return clusterState.nodes().resolveNodesIds(request.nodesIds());
if (request.taskId().isSet()) {
return clusterState.nodes().resolveNodesIds(request.nodesIds());
} else {
return new String[]{request.taskId().getNodeId()};
}
}
protected void processTasks(TasksRequest request, Consumer<OperationTask> operation) {
if (request.taskId() != BaseTasksRequest.ALL_TASKS) {
if (request.taskId().isSet() == false) {
// we are only checking one task, we can optimize it
Task task = taskManager.getTask(request.taskId());
Task task = taskManager.getTask(request.taskId().getId());
if (task != null) {
if (request.match(task)) {
operation.accept((OperationTask) task);
@ -143,13 +147,14 @@ public abstract class TransportTasksAction<
} else {
for (Task task : taskManager.getTasks().values()) {
if (request.match(task)) {
operation.accept((OperationTask)task);
operation.accept((OperationTask) task);
}
}
}
}
protected abstract TasksResponse newResponse(TasksRequest request, List<TaskResponse> tasks, List<TaskOperationFailure> taskOperationFailures, List<FailedNodeException> failedNodeExceptions);
protected abstract TasksResponse newResponse(TasksRequest request, List<TaskResponse> tasks, List<TaskOperationFailure>
taskOperationFailures, List<FailedNodeException> failedNodeExceptions);
@SuppressWarnings("unchecked")
protected TasksResponse newResponse(TasksRequest request, AtomicReferenceArray responses) {
@ -232,34 +237,36 @@ public abstract class TransportTasksAction<
onFailure(idx, nodeId, new NoSuchNodeException(nodeId));
} else if (!clusterService.localNode().shouldConnectTo(node) && !clusterService.localNode().equals(node)) {
// the check "!clusterService.localNode().equals(node)" is to maintain backward comp. where before
// we allowed to connect from "local" client node to itself, certain tests rely on it, if we remove it, we need to fix
// we allowed to connect from "local" client node to itself, certain tests rely on it, if we remove it, we
// need to fix
// those (and they randomize the client node usage, so tricky to find when)
onFailure(idx, nodeId, new NodeShouldNotConnectException(clusterService.localNode(), node));
} else {
NodeTaskRequest nodeRequest = new NodeTaskRequest(request);
nodeRequest.setParentTask(clusterService.localNode().id(), task.getId());
taskManager.registerChildTask(task, node.getId());
transportService.sendRequest(node, transportNodeAction, nodeRequest, builder.build(), new BaseTransportResponseHandler<NodeTasksResponse>() {
@Override
public NodeTasksResponse newInstance() {
return new NodeTasksResponse();
}
transportService.sendRequest(node, transportNodeAction, nodeRequest, builder.build(),
new BaseTransportResponseHandler<NodeTasksResponse>() {
@Override
public NodeTasksResponse newInstance() {
return new NodeTasksResponse();
}
@Override
public void handleResponse(NodeTasksResponse response) {
onOperation(idx, response);
}
@Override
public void handleResponse(NodeTasksResponse response) {
onOperation(idx, response);
}
@Override
public void handleException(TransportException exp) {
onFailure(idx, node.id(), exp);
}
@Override
public void handleException(TransportException exp) {
onFailure(idx, node.id(), exp);
}
@Override
public String executor() {
return ThreadPool.Names.SAME;
}
});
@Override
public String executor() {
return ThreadPool.Names.SAME;
}
});
}
} catch (Throwable t) {
onFailure(idx, nodeId, t);

View File

@ -98,7 +98,7 @@ import java.util.Map;
* <p>
* When running tests you have to pass it to the test runner like this:
* <pre>
* mvn test -Dtests.jvm.argline="-Djava.security.debug=access,failure" ...
* gradle test -Dtests.jvm.argline="-Djava.security.debug=access,failure" ...
* </pre>
* See <a href="https://docs.oracle.com/javase/7/docs/technotes/guides/security/troubleshooting-security.html">
* Troubleshooting Security</a> for information.

View File

@ -272,7 +272,7 @@ public interface ClusterAdminClient extends ElasticsearchClient {
*
* @param request The nodes tasks request
* @return The result future
* @see org.elasticsearch.client.Requests#listTasksRequest(String...)
* @see org.elasticsearch.client.Requests#listTasksRequest()
*/
ActionFuture<ListTasksResponse> listTasks(ListTasksRequest request);
@ -281,7 +281,7 @@ public interface ClusterAdminClient extends ElasticsearchClient {
*
* @param request The nodes tasks request
* @param listener A listener to be notified with a result
* @see org.elasticsearch.client.Requests#listTasksRequest(String...)
* @see org.elasticsearch.client.Requests#listTasksRequest()
*/
void listTasks(ListTasksRequest request, ActionListener<ListTasksResponse> listener);
@ -295,7 +295,7 @@ public interface ClusterAdminClient extends ElasticsearchClient {
*
* @param request The nodes tasks request
* @return The result future
* @see org.elasticsearch.client.Requests#cancelTasksRequest(String...)
* @see org.elasticsearch.client.Requests#cancelTasksRequest()
*/
ActionFuture<CancelTasksResponse> cancelTasks(CancelTasksRequest request);
@ -304,7 +304,7 @@ public interface ClusterAdminClient extends ElasticsearchClient {
*
* @param request The nodes tasks request
* @param listener A cancelener to be notified with a result
* @see org.elasticsearch.client.Requests#cancelTasksRequest(String...)
* @see org.elasticsearch.client.Requests#cancelTasksRequest()
*/
void cancelTasks(CancelTasksRequest request, ActionListener<CancelTasksResponse> listener);

View File

@ -419,23 +419,11 @@ public class Requests {
/**
* Creates a nodes tasks request against one or more nodes. Pass <tt>null</tt> or an empty array for all nodes.
*
* @param nodesIds The nodes ids to get the tasks for
* @return The nodes tasks request
* @see org.elasticsearch.client.ClusterAdminClient#listTasks(ListTasksRequest)
*/
public static ListTasksRequest listTasksRequest(String... nodesIds) {
return new ListTasksRequest(nodesIds);
}
/**
* Creates a nodes tasks request against one or more nodes. Pass <tt>null</tt> or an empty array for all nodes.
*
* @param nodesIds The nodes ids to cancel the tasks on
* @return The nodes tasks request
* @see org.elasticsearch.client.ClusterAdminClient#cancelTasks(CancelTasksRequest)
*/
public static CancelTasksRequest cancelTasksRequest(String... nodesIds) {
return new CancelTasksRequest(nodesIds);
public static CancelTasksRequest cancelTasksRequest() {
return new CancelTasksRequest();
}
/**

View File

@ -39,7 +39,6 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.network.NetworkModule;
import org.elasticsearch.common.network.NetworkService;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.SettingsFilter;
import org.elasticsearch.common.settings.SettingsModule;
import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.indices.breaker.CircuitBreakerModule;
@ -155,7 +154,10 @@ public class TransportClient extends AbstractClient {
pluginsService.processModules(modules);
Injector injector = modules.createInjector();
injector.getInstance(TransportService.class).start();
final TransportService transportService = injector.getInstance(TransportService.class);
transportService.start();
transportService.acceptIncomingRequests();
TransportClient transportClient = new TransportClient(injector);
success = true;
return transportClient;

View File

@ -32,7 +32,6 @@ import org.elasticsearch.common.joda.DateMathParser;
import org.elasticsearch.common.joda.FormatDateTimeFormatter;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.indices.IndexClosedException;
import org.joda.time.DateTimeZone;
@ -527,29 +526,35 @@ public class IndexNameExpressionResolver extends AbstractComponent {
return expressions;
}
if (expressions.isEmpty() || (expressions.size() == 1 && (MetaData.ALL.equals(expressions.get(0))) || Regex.isMatchAllPattern(expressions.get(0)))) {
if (options.expandWildcardsOpen() && options.expandWildcardsClosed()) {
return Arrays.asList(metaData.concreteAllIndices());
} else if (options.expandWildcardsOpen()) {
return Arrays.asList(metaData.concreteAllOpenIndices());
} else if (options.expandWildcardsClosed()) {
return Arrays.asList(metaData.concreteAllClosedIndices());
} else {
return Collections.emptyList();
}
if (isEmptyOrTrivialWildcard(expressions)) {
return resolveEmptyOrTrivialWildcard(options, metaData, true);
}
Set<String> result = innerResolve(context, expressions, options, metaData);
if (result == null) {
return expressions;
}
if (result.isEmpty() && !options.allowNoIndices()) {
IndexNotFoundException infe = new IndexNotFoundException((String)null);
infe.setResources("index_or_alias", expressions.toArray(new String[0]));
throw infe;
}
return new ArrayList<>(result);
}
private Set<String> innerResolve(Context context, List<String> expressions, IndicesOptions options, MetaData metaData) {
Set<String> result = null;
for (int i = 0; i < expressions.size(); i++) {
String expression = expressions.get(i);
if (metaData.getAliasAndIndexLookup().containsKey(expression)) {
if (aliasOrIndexExists(metaData, expression)) {
if (result != null) {
result.add(expression);
}
continue;
}
if (Strings.isEmpty(expression)) {
throw new IndexNotFoundException(expression);
throw infe(expression);
}
boolean add = true;
if (expression.charAt(0) == '+') {
@ -557,32 +562,19 @@ public class IndexNameExpressionResolver extends AbstractComponent {
if (i == 0) {
result = new HashSet<>();
}
add = true;
expression = expression.substring(1);
} else if (expression.charAt(0) == '-') {
// if its the first, fill it with all the indices...
if (i == 0) {
String[] concreteIndices;
if (options.expandWildcardsOpen() && options.expandWildcardsClosed()) {
concreteIndices = metaData.concreteAllIndices();
} else if (options.expandWildcardsOpen()) {
concreteIndices = metaData.concreteAllOpenIndices();
} else if (options.expandWildcardsClosed()) {
concreteIndices = metaData.concreteAllClosedIndices();
} else {
assert false : "Shouldn't end up here";
concreteIndices = Strings.EMPTY_ARRAY;
}
result = new HashSet<>(Arrays.asList(concreteIndices));
List<String> concreteIndices = resolveEmptyOrTrivialWildcard(options, metaData, false);
result = new HashSet<>(concreteIndices);
}
add = false;
expression = expression.substring(1);
}
if (!Regex.isSimpleMatchPattern(expression)) {
if (!options.ignoreUnavailable() && !metaData.getAliasAndIndexLookup().containsKey(expression)) {
IndexNotFoundException infe = new IndexNotFoundException(expression);
infe.setResources("index_or_alias", expression);
throw infe;
if (!unavailableIgnoredOrExists(options, metaData, expression)) {
throw infe(expression);
}
if (result != null) {
if (add) {
@ -595,77 +587,119 @@ public class IndexNameExpressionResolver extends AbstractComponent {
}
if (result == null) {
// add all the previous ones...
result = new HashSet<>();
result.addAll(expressions.subList(0, i));
result = new HashSet<>(expressions.subList(0, i));
}
final IndexMetaData.State excludeState;
if (options.expandWildcardsOpen() && options.expandWildcardsClosed()){
excludeState = null;
} else if (options.expandWildcardsOpen() && options.expandWildcardsClosed() == false) {
excludeState = IndexMetaData.State.CLOSE;
} else if (options.expandWildcardsClosed() && options.expandWildcardsOpen() == false) {
excludeState = IndexMetaData.State.OPEN;
} else {
assert false : "this shouldn't get called if wildcards expand to none";
excludeState = null;
}
final Map<String, AliasOrIndex> matches;
if (Regex.isMatchAllPattern(expression)) {
// Can only happen if the expressions was initially: '-*'
matches = metaData.getAliasAndIndexLookup();
} else if (expression.indexOf("*") == expression.length() - 1) {
// Suffix wildcard:
assert expression.length() >= 2 : "expression [" + expression + "] should have at least a length of 2";
String fromPrefix = expression.substring(0, expression.length() - 1);
char[] toPrefixCharArr = fromPrefix.toCharArray();
toPrefixCharArr[toPrefixCharArr.length - 1]++;
String toPrefix = new String(toPrefixCharArr);
matches = metaData.getAliasAndIndexLookup().subMap(fromPrefix, toPrefix);
} else {
// Other wildcard expressions:
final String pattern = expression;
matches = metaData.getAliasAndIndexLookup()
.entrySet()
.stream()
.filter(e -> Regex.simpleMatch(pattern, e.getKey()))
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
}
Set<String> expand = new HashSet<>();
for (Map.Entry<String, AliasOrIndex> entry : matches.entrySet()) {
AliasOrIndex aliasOrIndex = entry.getValue();
if (context.isPreserveAliases() && aliasOrIndex.isAlias()) {
expand.add(entry.getKey());
} else {
for (IndexMetaData meta : aliasOrIndex.getIndices()) {
if (excludeState == null || meta.getState() != excludeState) {
expand.add(meta.getIndex().getName());
}
}
}
}
final IndexMetaData.State excludeState = excludeState(options);
final Map<String, AliasOrIndex> matches = matches(metaData, expression);
Set<String> expand = expand(context, excludeState, matches);
if (add) {
result.addAll(expand);
} else {
result.removeAll(expand);
}
if (matches.isEmpty() && options.allowNoIndices() == false) {
IndexNotFoundException infe = new IndexNotFoundException(expression);
infe.setResources("index_or_alias", expression);
throw infe;
if (!noIndicesAllowedOrMatches(options, matches)) {
throw infe(expression);
}
}
if (result == null) {
return expressions;
return result;
}
private boolean noIndicesAllowedOrMatches(IndicesOptions options, Map<String, AliasOrIndex> matches) {
return options.allowNoIndices() || !matches.isEmpty();
}
private boolean unavailableIgnoredOrExists(IndicesOptions options, MetaData metaData, String expression) {
return options.ignoreUnavailable() || aliasOrIndexExists(metaData, expression);
}
private boolean aliasOrIndexExists(MetaData metaData, String expression) {
return metaData.getAliasAndIndexLookup().containsKey(expression);
}
private static IndexNotFoundException infe(String expression) {
IndexNotFoundException infe = new IndexNotFoundException(expression);
infe.setResources("index_or_alias", expression);
return infe;
}
private static IndexMetaData.State excludeState(IndicesOptions options) {
final IndexMetaData.State excludeState;
if (options.expandWildcardsOpen() && options.expandWildcardsClosed()) {
excludeState = null;
} else if (options.expandWildcardsOpen() && options.expandWildcardsClosed() == false) {
excludeState = IndexMetaData.State.CLOSE;
} else if (options.expandWildcardsClosed() && options.expandWildcardsOpen() == false) {
excludeState = IndexMetaData.State.OPEN;
} else {
assert false : "this shouldn't get called if wildcards expand to none";
excludeState = null;
}
if (result.isEmpty() && !options.allowNoIndices()) {
IndexNotFoundException infe = new IndexNotFoundException((String)null);
infe.setResources("index_or_alias", expressions.toArray(new String[0]));
throw infe;
return excludeState;
}
private static Map<String, AliasOrIndex> matches(MetaData metaData, String expression) {
if (Regex.isMatchAllPattern(expression)) {
// Can only happen if the expressions was initially: '-*'
return metaData.getAliasAndIndexLookup();
} else if (expression.indexOf("*") == expression.length() - 1) {
return suffixWildcard(metaData, expression);
} else {
return otherWildcard(metaData, expression);
}
}
private static Map<String, AliasOrIndex> suffixWildcard(MetaData metaData, String expression) {
assert expression.length() >= 2 : "expression [" + expression + "] should have at least a length of 2";
String fromPrefix = expression.substring(0, expression.length() - 1);
char[] toPrefixCharArr = fromPrefix.toCharArray();
toPrefixCharArr[toPrefixCharArr.length - 1]++;
String toPrefix = new String(toPrefixCharArr);
return metaData.getAliasAndIndexLookup().subMap(fromPrefix, toPrefix);
}
private static Map<String, AliasOrIndex> otherWildcard(MetaData metaData, String expression) {
final String pattern = expression;
return metaData.getAliasAndIndexLookup()
.entrySet()
.stream()
.filter(e -> Regex.simpleMatch(pattern, e.getKey()))
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
}
private static Set<String> expand(Context context, IndexMetaData.State excludeState, Map<String, AliasOrIndex> matches) {
Set<String> expand = new HashSet<>();
for (Map.Entry<String, AliasOrIndex> entry : matches.entrySet()) {
AliasOrIndex aliasOrIndex = entry.getValue();
if (context.isPreserveAliases() && aliasOrIndex.isAlias()) {
expand.add(entry.getKey());
} else {
for (IndexMetaData meta : aliasOrIndex.getIndices()) {
if (excludeState == null || meta.getState() != excludeState) {
expand.add(meta.getIndex().getName());
}
}
}
}
return expand;
}
private boolean isEmptyOrTrivialWildcard(List<String> expressions) {
return expressions.isEmpty() || (expressions.size() == 1 && (MetaData.ALL.equals(expressions.get(0))) || Regex.isMatchAllPattern(expressions.get(0)));
}
private List<String> resolveEmptyOrTrivialWildcard(IndicesOptions options, MetaData metaData, boolean assertEmpty) {
if (options.expandWildcardsOpen() && options.expandWildcardsClosed()) {
return Arrays.asList(metaData.concreteAllIndices());
} else if (options.expandWildcardsOpen()) {
return Arrays.asList(metaData.concreteAllOpenIndices());
} else if (options.expandWildcardsClosed()) {
return Arrays.asList(metaData.concreteAllClosedIndices());
} else {
assert assertEmpty : "Shouldn't end up here";
return Collections.emptyList();
}
return new ArrayList<>(result);
}
}

View File

@ -399,27 +399,16 @@ public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData>, Fr
// in the index,bulk,update and delete apis.
public String resolveIndexRouting(@Nullable String parent, @Nullable String routing, String aliasOrIndex) {
if (aliasOrIndex == null) {
if (routing == null) {
return parent;
}
return routing;
return routingOrParent(parent, routing);
}
AliasOrIndex result = getAliasAndIndexLookup().get(aliasOrIndex);
if (result == null || result.isAlias() == false) {
if (routing == null) {
return parent;
}
return routing;
return routingOrParent(parent, routing);
}
AliasOrIndex.Alias alias = (AliasOrIndex.Alias) result;
if (result.getIndices().size() > 1) {
String[] indexNames = new String[result.getIndices().size()];
int i = 0;
for (IndexMetaData indexMetaData : result.getIndices()) {
indexNames[i++] = indexMetaData.getIndex().getName();
}
throw new IllegalArgumentException("Alias [" + aliasOrIndex + "] has more than one index associated with it [" + Arrays.toString(indexNames) + "], can't execute a single index op");
rejectSingleIndexOperation(aliasOrIndex, result);
}
AliasMetaData aliasMd = alias.getFirstAliasMetaData();
if (aliasMd.indexRouting() != null) {
@ -434,6 +423,19 @@ public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData>, Fr
// Alias routing overrides the parent routing (if any).
return aliasMd.indexRouting();
}
return routingOrParent(parent, routing);
}
private void rejectSingleIndexOperation(String aliasOrIndex, AliasOrIndex result) {
String[] indexNames = new String[result.getIndices().size()];
int i = 0;
for (IndexMetaData indexMetaData : result.getIndices()) {
indexNames[i++] = indexMetaData.getIndex().getName();
}
throw new IllegalArgumentException("Alias [" + aliasOrIndex + "] has more than one index associated with it [" + Arrays.toString(indexNames) + "], can't execute a single index op");
}
private String routingOrParent(@Nullable String parent, @Nullable String routing) {
if (routing == null) {
return parent;
}

View File

@ -18,8 +18,10 @@
*/
package org.elasticsearch.common;
import java.io.IOException;
import java.nio.charset.Charset;
import java.util.Locale;
import java.util.Objects;
/**
* <p>Encodes and decodes to and from Base64 notation.</p>
@ -161,7 +163,7 @@ import java.util.Locale;
* @author rob@iharder.net
* @version 2.3.7
*/
public class Base64 {
public final class Base64 {
/* ******** P U B L I C F I E L D S ******** */
@ -791,10 +793,7 @@ public class Base64 {
* @since 2.3.1
*/
public static byte[] encodeBytesToBytes(byte[] source, int off, int len, int options) throws java.io.IOException {
if (source == null) {
throw new NullPointerException("Cannot serialize a null array.");
} // end if: null
Objects.requireNonNull(source, "Cannot serialize a null array.");
if (off < 0) {
throw new IllegalArgumentException("Cannot have negative offset: " + off);
@ -809,103 +808,109 @@ public class Base64 {
String.format(Locale.ROOT, "Cannot have offset of %d and length of %d with array of length %d", off, len, source.length));
} // end if: off < 0
// Compress?
if ((options & GZIP) != 0) {
java.io.ByteArrayOutputStream baos = null;
java.util.zip.GZIPOutputStream gzos = null;
Base64.OutputStream b64os = null;
try {
// GZip -> Base64 -> ByteArray
baos = new java.io.ByteArrayOutputStream();
b64os = new Base64.OutputStream(baos, ENCODE | options);
gzos = new java.util.zip.GZIPOutputStream(b64os);
gzos.write(source, off, len);
gzos.close();
} // end try
catch (java.io.IOException e) {
// Catch it and then throw it immediately so that
// the finally{} block is called for cleanup.
throw e;
} // end catch
finally {
try {
gzos.close();
} catch (Exception e) {
}
try {
b64os.close();
} catch (Exception e) {
}
try {
baos.close();
} catch (Exception e) {
}
} // end finally
return baos.toByteArray();
return encodeCompressedBytes(source, off, len, options);
} // end if: compress
// Else, don't compress. Better not to use streams at all then.
else {
boolean breakLines = (options & DO_BREAK_LINES) != 0;
//int len43 = len * 4 / 3;
//byte[] outBuff = new byte[ ( len43 ) // Main 4:3
// + ( (len % 3) > 0 ? 4 : 0 ) // Account for padding
// + (breakLines ? ( len43 / MAX_LINE_LENGTH ) : 0) ]; // New lines
// Try to determine more precisely how big the array needs to be.
// If we get it right, we don't have to do an array copy, and
// we save a bunch of memory.
int encLen = (len / 3) * 4 + (len % 3 > 0 ? 4 : 0); // Bytes needed for actual encoding
if (breakLines) {
encLen += encLen / MAX_LINE_LENGTH; // Plus extra newline characters
}
byte[] outBuff = new byte[encLen];
int d = 0;
int e = 0;
int len2 = len - 2;
int lineLength = 0;
for (; d < len2; d += 3, e += 4) {
encode3to4(source, d + off, 3, outBuff, e, options);
lineLength += 4;
if (breakLines && lineLength >= MAX_LINE_LENGTH) {
outBuff[e + 4] = NEW_LINE;
e++;
lineLength = 0;
} // end if: end of line
} // en dfor: each piece of array
if (d < len) {
encode3to4(source, d + off, len - d, outBuff, e, options);
e += 4;
} // end if: some padding needed
// Only resize array if we didn't guess it right.
if (e <= outBuff.length - 1) {
// If breaking lines and the last byte falls right at
// the line length (76 bytes per line), there will be
// one extra byte, and the array will need to be resized.
// Not too bad of an estimate on array size, I'd say.
byte[] finalOut = new byte[e];
System.arraycopy(outBuff, 0, finalOut, 0, e);
//System.err.println("Having to resize array from " + outBuff.length + " to " + e );
return finalOut;
} else {
//System.err.println("No need to resize array.");
return outBuff;
}
return encodeNonCompressedBytes(source, off, len, options);
} // end else: don't compress
} // end encodeBytesToBytes
private static byte[] encodeNonCompressedBytes(byte[] source, int off, int len, int options) {
boolean breakLines = (options & DO_BREAK_LINES) != 0;
//int len43 = len * 4 / 3;
//byte[] outBuff = new byte[ ( len43 ) // Main 4:3
// + ( (len % 3) > 0 ? 4 : 0 ) // Account for padding
// + (breakLines ? ( len43 / MAX_LINE_LENGTH ) : 0) ]; // New lines
// Try to determine more precisely how big the array needs to be.
// If we get it right, we don't have to do an array copy, and
// we save a bunch of memory.
int encLen = (len / 3) * 4 + (len % 3 > 0 ? 4 : 0); // Bytes needed for actual encoding
if (breakLines) {
encLen += encLen / MAX_LINE_LENGTH; // Plus extra newline characters
}
byte[] outBuff = new byte[encLen];
int d = 0;
int e = 0;
int len2 = len - 2;
int lineLength = 0;
for (; d < len2; d += 3, e += 4) {
encode3to4(source, d + off, 3, outBuff, e, options);
lineLength += 4;
if (breakLines && lineLength >= MAX_LINE_LENGTH) {
outBuff[e + 4] = NEW_LINE;
e++;
lineLength = 0;
} // end if: end of line
} // en dfor: each piece of array
if (d < len) {
encode3to4(source, d + off, len - d, outBuff, e, options);
e += 4;
} // end if: some padding needed
// Only resize array if we didn't guess it right.
if (e <= outBuff.length - 1) {
// If breaking lines and the last byte falls right at
// the line length (76 bytes per line), there will be
// one extra byte, and the array will need to be resized.
// Not too bad of an estimate on array size, I'd say.
byte[] finalOut = new byte[e];
System.arraycopy(outBuff, 0, finalOut, 0, e);
//System.err.println("Having to resize array from " + outBuff.length + " to " + e );
return finalOut;
} else {
//System.err.println("No need to resize array.");
return outBuff;
}
}
private static byte[] encodeCompressedBytes(byte[] source, int off, int len, int options) throws IOException {
java.io.ByteArrayOutputStream baos = null;
java.util.zip.GZIPOutputStream gzos = null;
OutputStream b64os = null;
try {
// GZip -> Base64 -> ByteArray
baos = new java.io.ByteArrayOutputStream();
b64os = new OutputStream(baos, ENCODE | options);
gzos = new java.util.zip.GZIPOutputStream(b64os);
gzos.write(source, off, len);
gzos.close();
} // end try
catch (IOException e) {
// Catch it and then throw it immediately so that
// the finally{} block is called for cleanup.
throw e;
} // end catch
finally {
try {
gzos.close();
} catch (Exception e) {
}
try {
b64os.close();
} catch (Exception e) {
}
try {
baos.close();
} catch (Exception e) {
}
} // end finally
return baos.toByteArray();
}
/* ******** D E C O D I N G M E T H O D S ******** */
@ -937,17 +942,10 @@ public class Base64 {
* or there is not enough room in the array.
* @since 1.3
*/
private static int decode4to3(
byte[] source, int srcOffset,
byte[] destination, int destOffset, int options) {
private static int decode4to3(byte[] source, int srcOffset, byte[] destination, int destOffset, int options) {
// Lots of error checking and exception throwing
if (source == null) {
throw new NullPointerException("Source array was null.");
} // end if
if (destination == null) {
throw new NullPointerException("Destination array was null.");
} // end if
Objects.requireNonNull(source, "Source array was null.");
Objects.requireNonNull(destination, "Destination array was null.");
if (srcOffset < 0 || srcOffset + 3 >= source.length) {
throw new IllegalArgumentException(String.format(Locale.ROOT,
"Source array with length %d cannot have offset of %d and still process four bytes.", source.length, srcOffset));
@ -957,56 +955,36 @@ public class Base64 {
"Destination array with length %d cannot have offset of %d and still store three bytes.", destination.length, destOffset));
} // end if
byte[] DECODABET = getDecodabet(options);
// Two ways to do the same thing. Don't know which way I like best.
//int outBuff = ( ( DECODABET[ source[ srcOffset ] ] << 24 ) >>> 6 )
// | ( ( DECODABET[ source[ srcOffset + 1] ] << 24 ) >>> 12 );
int outBuff = ((DECODABET[source[srcOffset]] & 0xFF) << 18)
| ((DECODABET[source[srcOffset + 1]] & 0xFF) << 12);
destination[destOffset] = (byte) (outBuff >>> 16);
// Example: Dk==
if (source[srcOffset + 2] == EQUALS_SIGN) {
// Two ways to do the same thing. Don't know which way I like best.
//int outBuff = ( ( DECODABET[ source[ srcOffset ] ] << 24 ) >>> 6 )
// | ( ( DECODABET[ source[ srcOffset + 1] ] << 24 ) >>> 12 );
int outBuff = ((DECODABET[source[srcOffset]] & 0xFF) << 18)
| ((DECODABET[source[srcOffset + 1]] & 0xFF) << 12);
destination[destOffset] = (byte) (outBuff >>> 16);
return 1;
}
// Example: DkL=
else if (source[srcOffset + 3] == EQUALS_SIGN) {
// Two ways to do the same thing. Don't know which way I like best.
//int outBuff = ( ( DECODABET[ source[ srcOffset ] ] << 24 ) >>> 6 )
// | ( ( DECODABET[ source[ srcOffset + 1 ] ] << 24 ) >>> 12 )
// | ( ( DECODABET[ source[ srcOffset + 2 ] ] << 24 ) >>> 18 );
int outBuff = ((DECODABET[source[srcOffset]] & 0xFF) << 18)
| ((DECODABET[source[srcOffset + 1]] & 0xFF) << 12)
| ((DECODABET[source[srcOffset + 2]] & 0xFF) << 6);
outBuff |= ((DECODABET[source[srcOffset + 2]] & 0xFF) << 6);
destination[destOffset + 1] = (byte) (outBuff >>> 8);
destination[destOffset] = (byte) (outBuff >>> 16);
destination[destOffset + 1] = (byte) (outBuff >>> 8);
// Example: DkL=
if (source[srcOffset + 3] == EQUALS_SIGN) {
return 2;
}
outBuff |= ((DECODABET[source[srcOffset + 3]] & 0xFF));
destination[destOffset + 2] = (byte) (outBuff);
// Example: DkLE
else {
// Two ways to do the same thing. Don't know which way I like best.
//int outBuff = ( ( DECODABET[ source[ srcOffset ] ] << 24 ) >>> 6 )
// | ( ( DECODABET[ source[ srcOffset + 1 ] ] << 24 ) >>> 12 )
// | ( ( DECODABET[ source[ srcOffset + 2 ] ] << 24 ) >>> 18 )
// | ( ( DECODABET[ source[ srcOffset + 3 ] ] << 24 ) >>> 24 );
int outBuff = ((DECODABET[source[srcOffset]] & 0xFF) << 18)
| ((DECODABET[source[srcOffset + 1]] & 0xFF) << 12)
| ((DECODABET[source[srcOffset + 2]] & 0xFF) << 6)
| ((DECODABET[source[srcOffset + 3]] & 0xFF));
destination[destOffset] = (byte) (outBuff >> 16);
destination[destOffset + 1] = (byte) (outBuff >> 8);
destination[destOffset + 2] = (byte) (outBuff);
return 3;
}
} // end decodeToBytes
return 3;
}
/**
@ -1051,13 +1029,9 @@ public class Base64 {
* @throws java.io.IOException If bogus characters exist in source data
* @since 1.3
*/
public static byte[] decode(byte[] source, int off, int len, int options)
throws java.io.IOException {
public static byte[] decode(byte[] source, int off, int len, int options) throws java.io.IOException {
// Lots of error checking and exception throwing
if (source == null) {
throw new NullPointerException("Cannot decode null source array.");
} // end if
Objects.requireNonNull(source, "Cannot decode null source array.");
if (off < 0 || off + len > source.length) {
throw new IllegalArgumentException(String.format(Locale.ROOT,
"Source array with length %d cannot have offset of %d and process %d bytes.", source.length, off, len));
@ -1074,16 +1048,21 @@ public class Base64 {
int len34 = len * 3 / 4; // Estimate on array size
byte[] outBuff = new byte[len34]; // Upper limit on size of output
int outBuffPosn = 0; // Keep track of where we're writing
int outBuffPosn = decode(source, off, len, options, DECODABET, outBuff);
byte[] out = new byte[outBuffPosn];
System.arraycopy(outBuff, 0, out, 0, outBuffPosn);
return out;
} // end decode
private static int decode(byte[] source, int off, int len, int options, byte[] DECODABET, byte[] outBuff) throws IOException {
int outBuffPosn = 0; // Keep track of where we're writing
byte[] b4 = new byte[4]; // Four byte buffer from source, eliminating white space
int b4Posn = 0; // Keep track of four byte input buffer
int i = 0; // Source array counter
byte sbiDecode = 0; // Special value from DECODABET
for (int i = off; i < off + len; i++) { // Loop through source
for (i = off; i < off + len; i++) { // Loop through source
sbiDecode = DECODABET[source[i] & 0xFF];
byte sbiDecode = DECODABET[source[i] & 0xFF];
// White space, Equals sign, or legit Base64 character
// Note the values such as -5 and -9 in the
@ -1099,7 +1078,7 @@ public class Base64 {
if (source[i] == EQUALS_SIGN) {
// check if the equals sign is somewhere in between
if (i+1 < len + off) {
throw new java.io.IOException(String.format(Locale.ROOT,
throw new IOException(String.format(Locale.ROOT,
"Found equals sign at position %d of the base64 string, not at the end", i));
}
break;
@ -1107,7 +1086,7 @@ public class Base64 {
} // end if: quartet built
else {
if (source[i] == EQUALS_SIGN && len + off > i && source[i+1] != EQUALS_SIGN) {
throw new java.io.IOException(String.format(Locale.ROOT,
throw new IOException(String.format(Locale.ROOT,
"Found equals sign at position %d of the base64 string, not at the end", i));
} // enf if: equals sign and next character not as well
} // end else:
@ -1115,15 +1094,12 @@ public class Base64 {
} // end if: white space, equals sign or better
else {
// There's a bad input character in the Base64 stream.
throw new java.io.IOException(String.format(Locale.ROOT,
throw new IOException(String.format(Locale.ROOT,
"Bad Base64 input character decimal %d in array position %d", ((int) source[i]) & 0xFF, i));
} // end else:
} // each input character
byte[] out = new byte[outBuffPosn];
System.arraycopy(outBuff, 0, out, 0, outBuffPosn);
return out;
} // end decode
return outBuffPosn;
}
/**

View File

@ -118,37 +118,9 @@ public class ChildMemoryCircuitBreaker implements CircuitBreaker {
// .addAndGet() instead of looping (because we don't have to check a
// limit), which makes the RamAccountingTermsEnum case faster.
if (this.memoryBytesLimit == -1) {
newUsed = this.used.addAndGet(bytes);
if (logger.isTraceEnabled()) {
logger.trace("[{}] Adding [{}][{}] to used bytes [new used: [{}], limit: [-1b]]",
this.name, new ByteSizeValue(bytes), label, new ByteSizeValue(newUsed));
}
newUsed = noLimit(bytes, label);
} else {
// Otherwise, check the addition and commit the addition, looping if
// there are conflicts. May result in additional logging, but it's
// trace logging and shouldn't be counted on for additions.
long currentUsed;
do {
currentUsed = this.used.get();
newUsed = currentUsed + bytes;
long newUsedWithOverhead = (long) (newUsed * overheadConstant);
if (logger.isTraceEnabled()) {
logger.trace("[{}] Adding [{}][{}] to used bytes [new used: [{}], limit: {} [{}], estimate: {} [{}]]",
this.name,
new ByteSizeValue(bytes), label, new ByteSizeValue(newUsed),
memoryBytesLimit, new ByteSizeValue(memoryBytesLimit),
newUsedWithOverhead, new ByteSizeValue(newUsedWithOverhead));
}
if (memoryBytesLimit > 0 && newUsedWithOverhead > memoryBytesLimit) {
logger.warn("[{}] New used memory {} [{}] for data of [{}] would be larger than configured breaker: {} [{}], breaking",
this.name,
newUsedWithOverhead, new ByteSizeValue(newUsedWithOverhead), label,
memoryBytesLimit, new ByteSizeValue(memoryBytesLimit));
circuitBreak(label, newUsedWithOverhead);
}
// Attempt to set the new used value, but make sure it hasn't changed
// underneath us, if it has, keep trying until we are able to set it
} while (!this.used.compareAndSet(currentUsed, newUsed));
newUsed = limit(bytes, label);
}
// Additionally, we need to check that we haven't exceeded the parent's limit
@ -164,6 +136,45 @@ public class ChildMemoryCircuitBreaker implements CircuitBreaker {
return newUsed;
}
private long noLimit(long bytes, String label) {
long newUsed;
newUsed = this.used.addAndGet(bytes);
if (logger.isTraceEnabled()) {
logger.trace("[{}] Adding [{}][{}] to used bytes [new used: [{}], limit: [-1b]]",
this.name, new ByteSizeValue(bytes), label, new ByteSizeValue(newUsed));
}
return newUsed;
}
private long limit(long bytes, String label) {
long newUsed;// Otherwise, check the addition and commit the addition, looping if
// there are conflicts. May result in additional logging, but it's
// trace logging and shouldn't be counted on for additions.
long currentUsed;
do {
currentUsed = this.used.get();
newUsed = currentUsed + bytes;
long newUsedWithOverhead = (long) (newUsed * overheadConstant);
if (logger.isTraceEnabled()) {
logger.trace("[{}] Adding [{}][{}] to used bytes [new used: [{}], limit: {} [{}], estimate: {} [{}]]",
this.name,
new ByteSizeValue(bytes), label, new ByteSizeValue(newUsed),
memoryBytesLimit, new ByteSizeValue(memoryBytesLimit),
newUsedWithOverhead, new ByteSizeValue(newUsedWithOverhead));
}
if (memoryBytesLimit > 0 && newUsedWithOverhead > memoryBytesLimit) {
logger.warn("[{}] New used memory {} [{}] for data of [{}] would be larger than configured breaker: {} [{}], breaking",
this.name,
newUsedWithOverhead, new ByteSizeValue(newUsedWithOverhead), label,
memoryBytesLimit, new ByteSizeValue(memoryBytesLimit));
circuitBreak(label, newUsedWithOverhead);
}
// Attempt to set the new used value, but make sure it hasn't changed
// underneath us, if it has, keep trying until we are able to set it
} while (!this.used.compareAndSet(currentUsed, newUsed));
return newUsed;
}
/**
* Add an <b>exact</b> number of bytes, not checking for tripping the
* circuit breaker. This bypasses the overheadConstant multiplication.

View File

@ -265,17 +265,17 @@ public class TimeValue implements Streamable {
long millis;
String lowerSValue = sValue.toLowerCase(Locale.ROOT).trim();
if (lowerSValue.endsWith("ms")) {
millis = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 2)));
millis = parse(lowerSValue, 2, 1);
} else if (lowerSValue.endsWith("s")) {
millis = (long) Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 1)) * 1000;
millis = parse(lowerSValue, 1, 1000);
} else if (lowerSValue.endsWith("m")) {
millis = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 1)) * 60 * 1000);
millis = parse(lowerSValue, 1, 60 * 1000);
} else if (lowerSValue.endsWith("h")) {
millis = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 1)) * 60 * 60 * 1000);
millis = parse(lowerSValue, 1, 60 * 60 * 1000);
} else if (lowerSValue.endsWith("d")) {
millis = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 1)) * 24 * 60 * 60 * 1000);
millis = parse(lowerSValue, 1, 24 * 60 * 60 * 1000);
} else if (lowerSValue.endsWith("w")) {
millis = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 1)) * 7 * 24 * 60 * 60 * 1000);
millis = parse(lowerSValue, 1, 7 * 24 * 60 * 60 * 1000);
} else if (lowerSValue.equals("-1")) {
// Allow this special value to be unit-less:
millis = -1;
@ -292,6 +292,10 @@ public class TimeValue implements Streamable {
}
}
private static long parse(String s, int suffixLength, long scale) {
return (long) (Double.parseDouble(s.substring(0, s.length() - suffixLength)) * scale);
}
static final long C0 = 1L;
static final long C1 = C0 * 1000L;
static final long C2 = C1 * 1000L;

View File

@ -36,8 +36,6 @@ import java.util.concurrent.LinkedTransferQueue;
*/
public abstract class ConcurrentCollections {
private final static boolean useLinkedTransferQueue = Boolean.parseBoolean(System.getProperty("es.useLinkedTransferQueue", "false"));
static final int aggressiveConcurrencyLevel;
static {
@ -71,9 +69,6 @@ public abstract class ConcurrentCollections {
}
public static <T> Queue<T> newQueue() {
if (useLinkedTransferQueue) {
return new LinkedTransferQueue<>();
}
return new ConcurrentLinkedQueue<>();
}

View File

@ -43,9 +43,12 @@ import java.math.RoundingMode;
import java.nio.charset.StandardCharsets;
import java.nio.file.Path;
import java.util.Calendar;
import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.Locale;
import java.util.Map;
import java.util.function.BiConsumer;
/**
*
@ -1227,52 +1230,101 @@ public final class XContentBuilder implements BytesStream, Releasable {
generator.writeEndObject();
}
@FunctionalInterface
interface Writer {
void write(XContentGenerator g, Object v) throws IOException;
}
private final static Map<Class<?>, Writer> MAP;
static {
Map<Class<?>, Writer> map = new HashMap<>();
map.put(String.class, (g, v) -> g.writeString((String) v));
map.put(Integer.class, (g, v) -> g.writeNumber((Integer) v));
map.put(Long.class, (g, v) -> g.writeNumber((Long) v));
map.put(Float.class, (g, v) -> g.writeNumber((Float) v));
map.put(Double.class, (g, v) -> g.writeNumber((Double) v));
map.put(Byte.class, (g, v) -> g.writeNumber((Byte) v));
map.put(Short.class, (g, v) -> g.writeNumber((Short) v));
map.put(Boolean.class, (g, v) -> g.writeBoolean((Boolean) v));
map.put(GeoPoint.class, (g, v) -> {
g.writeStartObject();
g.writeNumberField("lat", ((GeoPoint) v).lat());
g.writeNumberField("lon", ((GeoPoint) v).lon());
g.writeEndObject();
});
map.put(int[].class, (g, v) -> {
g.writeStartArray();
for (int item : (int[]) v) {
g.writeNumber(item);
}
g.writeEndArray();
});
map.put(long[].class, (g, v) -> {
g.writeStartArray();
for (long item : (long[]) v) {
g.writeNumber(item);
}
g.writeEndArray();
});
map.put(float[].class, (g, v) -> {
g.writeStartArray();
for (float item : (float[]) v) {
g.writeNumber(item);
}
g.writeEndArray();
});
map.put(double[].class, (g, v) -> {
g.writeStartArray();
for (double item : (double[])v) {
g.writeNumber(item);
}
g.writeEndArray();
});
map.put(byte[].class, (g, v) -> g.writeBinary((byte[]) v));
map.put(short[].class, (g, v) -> {
g.writeStartArray();
for (short item : (short[])v) {
g.writeNumber(item);
}
g.writeEndArray();
});
map.put(BytesRef.class, (g, v) -> {
BytesRef bytes = (BytesRef) v;
g.writeBinary(bytes.bytes, bytes.offset, bytes.length);
});
map.put(Text.class, (g, v) -> {
Text text = (Text) v;
if (text.hasBytes() && text.bytes().hasArray()) {
g.writeUTF8String(text.bytes().array(), text.bytes().arrayOffset(), text.bytes().length());
} else if (text.hasString()) {
g.writeString(text.string());
} else {
BytesArray bytesArray = text.bytes().toBytesArray();
g.writeUTF8String(bytesArray.array(), bytesArray.arrayOffset(), bytesArray.length());
}
});
MAP = Collections.unmodifiableMap(map);
}
private void writeValue(Object value) throws IOException {
if (value == null) {
generator.writeNull();
return;
}
Class<?> type = value.getClass();
if (type == String.class) {
generator.writeString((String) value);
} else if (type == Integer.class) {
generator.writeNumber(((Integer) value).intValue());
} else if (type == Long.class) {
generator.writeNumber(((Long) value).longValue());
} else if (type == Float.class) {
generator.writeNumber(((Float) value).floatValue());
} else if (type == Double.class) {
generator.writeNumber(((Double) value).doubleValue());
} else if (type == Byte.class) {
generator.writeNumber(((Byte)value).byteValue());
} else if (type == Short.class) {
generator.writeNumber(((Short) value).shortValue());
} else if (type == Boolean.class) {
generator.writeBoolean(((Boolean) value).booleanValue());
} else if (type == GeoPoint.class) {
generator.writeStartObject();
generator.writeNumberField("lat", ((GeoPoint) value).lat());
generator.writeNumberField("lon", ((GeoPoint) value).lon());
generator.writeEndObject();
Writer writer = MAP.get(type);
if (writer != null) {
writer.write(generator, value);
} else if (value instanceof Map) {
writeMap((Map) value);
} else if (value instanceof Path) {
//Path implements Iterable<Path> and causes endless recursion and a StackOverFlow if treated as an Iterable here
generator.writeString(value.toString());
} else if (value instanceof Iterable) {
generator.writeStartArray();
for (Object v : (Iterable<?>) value) {
writeValue(v);
}
generator.writeEndArray();
writeIterable((Iterable<?>) value);
} else if (value instanceof Object[]) {
generator.writeStartArray();
for (Object v : (Object[]) value) {
writeValue(v);
}
generator.writeEndArray();
} else if (type == byte[].class) {
generator.writeBinary((byte[]) value);
writeObjectArray((Object[]) value);
} else if (value instanceof Date) {
generator.writeString(XContentBuilder.defaultDatePrinter.print(((Date) value).getTime()));
} else if (value instanceof Calendar) {
@ -1280,56 +1332,9 @@ public final class XContentBuilder implements BytesStream, Releasable {
} else if (value instanceof ReadableInstant) {
generator.writeString(XContentBuilder.defaultDatePrinter.print((((ReadableInstant) value)).getMillis()));
} else if (value instanceof BytesReference) {
BytesReference bytes = (BytesReference) value;
if (!bytes.hasArray()) {
bytes = bytes.toBytesArray();
}
generator.writeBinary(bytes.array(), bytes.arrayOffset(), bytes.length());
} else if (value instanceof BytesRef) {
BytesRef bytes = (BytesRef) value;
generator.writeBinary(bytes.bytes, bytes.offset, bytes.length);
} else if (value instanceof Text) {
Text text = (Text) value;
if (text.hasBytes() && text.bytes().hasArray()) {
generator.writeUTF8String(text.bytes().array(), text.bytes().arrayOffset(), text.bytes().length());
} else if (text.hasString()) {
generator.writeString(text.string());
} else {
BytesArray bytesArray = text.bytes().toBytesArray();
generator.writeUTF8String(bytesArray.array(), bytesArray.arrayOffset(), bytesArray.length());
}
writeBytesReference((BytesReference) value);
} else if (value instanceof ToXContent) {
((ToXContent) value).toXContent(this, ToXContent.EMPTY_PARAMS);
} else if (value instanceof double[]) {
generator.writeStartArray();
for (double v : (double[]) value) {
generator.writeNumber(v);
}
generator.writeEndArray();
} else if (value instanceof long[]) {
generator.writeStartArray();
for (long v : (long[]) value) {
generator.writeNumber(v);
}
generator.writeEndArray();
} else if (value instanceof int[]) {
generator.writeStartArray();
for (int v : (int[]) value) {
generator.writeNumber(v);
}
generator.writeEndArray();
} else if (value instanceof float[]) {
generator.writeStartArray();
for (float v : (float[]) value) {
generator.writeNumber(v);
}
generator.writeEndArray();
} else if (value instanceof short[]) {
generator.writeStartArray();
for (short v : (short[]) value) {
generator.writeNumber(v);
}
generator.writeEndArray();
} else {
// if this is a "value" object, like enum, DistanceUnit, ..., just toString it
// yea, it can be misleading when toString a Java class, but really, jackson should be used in that case
@ -1337,4 +1342,29 @@ public final class XContentBuilder implements BytesStream, Releasable {
//throw new ElasticsearchIllegalArgumentException("type not supported for generic value conversion: " + type);
}
}
private void writeBytesReference(BytesReference value) throws IOException {
BytesReference bytes = value;
if (!bytes.hasArray()) {
bytes = bytes.toBytesArray();
}
generator.writeBinary(bytes.array(), bytes.arrayOffset(), bytes.length());
}
private void writeIterable(Iterable<?> value) throws IOException {
generator.writeStartArray();
for (Object v : value) {
writeValue(v);
}
generator.writeEndArray();
}
private void writeObjectArray(Object[] value) throws IOException {
generator.writeStartArray();
for (Object v : value) {
writeValue(v);
}
generator.writeEndArray();
}
}

View File

@ -94,6 +94,12 @@ public interface Discovery extends LifecycleComponent<Discovery> {
DiscoveryStats stats();
/**
* Triggers the first join cycle
*/
void startInitialJoin();
/***
* @return the current value of minimum master nodes, or -1 for not set
*/

View File

@ -87,8 +87,9 @@ public class DiscoveryService extends AbstractLifecycleComponent<DiscoveryServic
logger.info(discovery.nodeDescription());
}
public void waitForInitialState() {
public void joinClusterAndWaitForInitialState() {
try {
discovery.startInitialJoin();
if (!initialStateListener.waitForInitialState(initialStateTimeout)) {
logger.warn("waited for {} and no initial state was set by the discovery", initialStateTimeout);
}

View File

@ -100,6 +100,11 @@ public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implem
@Override
protected void doStart() {
}
@Override
public void startInitialJoin() {
synchronized (clusterGroups) {
ClusterGroup clusterGroup = clusterGroups.get(clusterName);
if (clusterGroup == null) {

View File

@ -216,7 +216,10 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
joinThreadControl.start();
pingService.start();
this.nodeJoinController = new NodeJoinController(clusterService, routingService, discoverySettings, settings);
}
@Override
public void startInitialJoin() {
// start the join thread from a cluster state update. See {@link JoinThreadControl} for details.
clusterService.submitStateUpdateTask("initial_join", new ClusterStateUpdateTask() {

View File

@ -133,27 +133,6 @@ public class GatewayService extends AbstractLifecycleComponent<GatewayService> i
@Override
protected void doStart() {
clusterService.addLast(this);
// check we didn't miss any cluster state that came in until now / during the addition
clusterService.submitStateUpdateTask("gateway_initial_state_recovery", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
checkStateMeetsSettingsAndMaybeRecover(currentState);
return currentState;
}
@Override
public boolean runOnlyOnMaster() {
// It's OK to run on non masters as checkStateMeetsSettingsAndMaybeRecover checks for this
// we return false to avoid unneeded failure logs
return false;
}
@Override
public void onFailure(String source, Throwable t) {
logger.warn("unexpected failure while checking if state can be recovered. another attempt will be made with the next cluster state change", t);
}
});
}
@Override
@ -170,10 +149,9 @@ public class GatewayService extends AbstractLifecycleComponent<GatewayService> i
if (lifecycle.stoppedOrClosed()) {
return;
}
checkStateMeetsSettingsAndMaybeRecover(event.state());
}
protected void checkStateMeetsSettingsAndMaybeRecover(ClusterState state) {
final ClusterState state = event.state();
if (state.nodes().localNodeMaster() == false) {
// not our job to recover
return;

View File

@ -43,6 +43,8 @@ import org.jboss.netty.handler.codec.http.HttpResponse;
import org.jboss.netty.handler.codec.http.HttpResponseStatus;
import org.jboss.netty.handler.codec.http.HttpVersion;
import java.util.Collections;
import java.util.EnumMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -51,10 +53,7 @@ import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONNECTION;
import static org.jboss.netty.handler.codec.http.HttpHeaders.Values.CLOSE;
import static org.jboss.netty.handler.codec.http.HttpHeaders.Values.KEEP_ALIVE;
/**
*
*/
public class NettyHttpChannel extends HttpChannel {
public final class NettyHttpChannel extends HttpChannel {
private final NettyHttpServerTransport transport;
private final Channel channel;
@ -92,18 +91,11 @@ public class NettyHttpChannel extends HttpChannel {
String opaque = nettyRequest.headers().get("X-Opaque-Id");
if (opaque != null) {
resp.headers().add("X-Opaque-Id", opaque);
setHeaderField(resp, "X-Opaque-Id", opaque);
}
// Add all custom headers
Map<String, List<String>> customHeaders = response.getHeaders();
if (customHeaders != null) {
for (Map.Entry<String, List<String>> headerEntry : customHeaders.entrySet()) {
for (String headerValue : headerEntry.getValue()) {
resp.headers().add(headerEntry.getKey(), headerValue);
}
}
}
addCustomHeaders(response, resp);
BytesReference content = response.content();
ChannelBuffer buffer;
@ -113,30 +105,11 @@ public class NettyHttpChannel extends HttpChannel {
resp.setContent(buffer);
// If our response doesn't specify a content-type header, set one
if (!resp.headers().contains(HttpHeaders.Names.CONTENT_TYPE)) {
resp.headers().add(HttpHeaders.Names.CONTENT_TYPE, response.contentType());
}
setHeaderField(resp, HttpHeaders.Names.CONTENT_TYPE, response.contentType(), false);
// If our response has no content-length, calculate and set one
if (!resp.headers().contains(HttpHeaders.Names.CONTENT_LENGTH)) {
resp.headers().add(HttpHeaders.Names.CONTENT_LENGTH, String.valueOf(buffer.readableBytes()));
}
setHeaderField(resp, HttpHeaders.Names.CONTENT_LENGTH, String.valueOf(buffer.readableBytes()), false);
if (transport.resetCookies) {
String cookieString = nettyRequest.headers().get(HttpHeaders.Names.COOKIE);
if (cookieString != null) {
CookieDecoder cookieDecoder = new CookieDecoder();
Set<Cookie> cookies = cookieDecoder.decode(cookieString);
if (!cookies.isEmpty()) {
// Reset the cookies if necessary.
CookieEncoder cookieEncoder = new CookieEncoder(true);
for (Cookie cookie : cookies) {
cookieEncoder.addCookie(cookie);
}
resp.headers().add(HttpHeaders.Names.SET_COOKIE, cookieEncoder.encode());
}
}
}
addCookies(resp);
ChannelFuture future;
@ -164,6 +137,45 @@ public class NettyHttpChannel extends HttpChannel {
}
}
private void setHeaderField(HttpResponse resp, String headerField, String value) {
setHeaderField(resp, headerField, value, true);
}
private void setHeaderField(HttpResponse resp, String headerField, String value, boolean override) {
if (override || !resp.headers().contains(headerField)) {
resp.headers().add(headerField, value);
}
}
private void addCookies(HttpResponse resp) {
if (transport.resetCookies) {
String cookieString = nettyRequest.headers().get(HttpHeaders.Names.COOKIE);
if (cookieString != null) {
CookieDecoder cookieDecoder = new CookieDecoder();
Set<Cookie> cookies = cookieDecoder.decode(cookieString);
if (!cookies.isEmpty()) {
// Reset the cookies if necessary.
CookieEncoder cookieEncoder = new CookieEncoder(true);
for (Cookie cookie : cookies) {
cookieEncoder.addCookie(cookie);
}
setHeaderField(resp, HttpHeaders.Names.SET_COOKIE, cookieEncoder.encode());
}
}
}
}
private void addCustomHeaders(RestResponse response, HttpResponse resp) {
Map<String, List<String>> customHeaders = response.getHeaders();
if (customHeaders != null) {
for (Map.Entry<String, List<String>> headerEntry : customHeaders.entrySet()) {
for (String headerValue : headerEntry.getValue()) {
setHeaderField(resp, headerEntry.getKey(), headerValue);
}
}
}
}
// Determine if the request protocol version is HTTP 1.0
private boolean isHttp10() {
return nettyRequest.getProtocolVersion().equals(HttpVersion.HTTP_1_0);
@ -196,101 +208,59 @@ public class NettyHttpChannel extends HttpChannel {
private static final HttpResponseStatus TOO_MANY_REQUESTS = new HttpResponseStatus(429, "Too Many Requests");
private HttpResponseStatus getStatus(RestStatus status) {
switch (status) {
case CONTINUE:
return HttpResponseStatus.CONTINUE;
case SWITCHING_PROTOCOLS:
return HttpResponseStatus.SWITCHING_PROTOCOLS;
case OK:
return HttpResponseStatus.OK;
case CREATED:
return HttpResponseStatus.CREATED;
case ACCEPTED:
return HttpResponseStatus.ACCEPTED;
case NON_AUTHORITATIVE_INFORMATION:
return HttpResponseStatus.NON_AUTHORITATIVE_INFORMATION;
case NO_CONTENT:
return HttpResponseStatus.NO_CONTENT;
case RESET_CONTENT:
return HttpResponseStatus.RESET_CONTENT;
case PARTIAL_CONTENT:
return HttpResponseStatus.PARTIAL_CONTENT;
case MULTI_STATUS:
// no status for this??
return HttpResponseStatus.INTERNAL_SERVER_ERROR;
case MULTIPLE_CHOICES:
return HttpResponseStatus.MULTIPLE_CHOICES;
case MOVED_PERMANENTLY:
return HttpResponseStatus.MOVED_PERMANENTLY;
case FOUND:
return HttpResponseStatus.FOUND;
case SEE_OTHER:
return HttpResponseStatus.SEE_OTHER;
case NOT_MODIFIED:
return HttpResponseStatus.NOT_MODIFIED;
case USE_PROXY:
return HttpResponseStatus.USE_PROXY;
case TEMPORARY_REDIRECT:
return HttpResponseStatus.TEMPORARY_REDIRECT;
case BAD_REQUEST:
return HttpResponseStatus.BAD_REQUEST;
case UNAUTHORIZED:
return HttpResponseStatus.UNAUTHORIZED;
case PAYMENT_REQUIRED:
return HttpResponseStatus.PAYMENT_REQUIRED;
case FORBIDDEN:
return HttpResponseStatus.FORBIDDEN;
case NOT_FOUND:
return HttpResponseStatus.NOT_FOUND;
case METHOD_NOT_ALLOWED:
return HttpResponseStatus.METHOD_NOT_ALLOWED;
case NOT_ACCEPTABLE:
return HttpResponseStatus.NOT_ACCEPTABLE;
case PROXY_AUTHENTICATION:
return HttpResponseStatus.PROXY_AUTHENTICATION_REQUIRED;
case REQUEST_TIMEOUT:
return HttpResponseStatus.REQUEST_TIMEOUT;
case CONFLICT:
return HttpResponseStatus.CONFLICT;
case GONE:
return HttpResponseStatus.GONE;
case LENGTH_REQUIRED:
return HttpResponseStatus.LENGTH_REQUIRED;
case PRECONDITION_FAILED:
return HttpResponseStatus.PRECONDITION_FAILED;
case REQUEST_ENTITY_TOO_LARGE:
return HttpResponseStatus.REQUEST_ENTITY_TOO_LARGE;
case REQUEST_URI_TOO_LONG:
return HttpResponseStatus.REQUEST_URI_TOO_LONG;
case UNSUPPORTED_MEDIA_TYPE:
return HttpResponseStatus.UNSUPPORTED_MEDIA_TYPE;
case REQUESTED_RANGE_NOT_SATISFIED:
return HttpResponseStatus.REQUESTED_RANGE_NOT_SATISFIABLE;
case EXPECTATION_FAILED:
return HttpResponseStatus.EXPECTATION_FAILED;
case UNPROCESSABLE_ENTITY:
return HttpResponseStatus.BAD_REQUEST;
case LOCKED:
return HttpResponseStatus.BAD_REQUEST;
case FAILED_DEPENDENCY:
return HttpResponseStatus.BAD_REQUEST;
case TOO_MANY_REQUESTS:
return TOO_MANY_REQUESTS;
case INTERNAL_SERVER_ERROR:
return HttpResponseStatus.INTERNAL_SERVER_ERROR;
case NOT_IMPLEMENTED:
return HttpResponseStatus.NOT_IMPLEMENTED;
case BAD_GATEWAY:
return HttpResponseStatus.BAD_GATEWAY;
case SERVICE_UNAVAILABLE:
return HttpResponseStatus.SERVICE_UNAVAILABLE;
case GATEWAY_TIMEOUT:
return HttpResponseStatus.GATEWAY_TIMEOUT;
case HTTP_VERSION_NOT_SUPPORTED:
return HttpResponseStatus.HTTP_VERSION_NOT_SUPPORTED;
default:
return HttpResponseStatus.INTERNAL_SERVER_ERROR;
}
static Map<RestStatus, HttpResponseStatus> MAP;
static {
EnumMap<RestStatus, HttpResponseStatus> map = new EnumMap<>(RestStatus.class);
map.put(RestStatus.CONTINUE, HttpResponseStatus.CONTINUE);
map.put(RestStatus.SWITCHING_PROTOCOLS, HttpResponseStatus.SWITCHING_PROTOCOLS);
map.put(RestStatus.OK, HttpResponseStatus.OK);
map.put(RestStatus.CREATED, HttpResponseStatus.CREATED);
map.put(RestStatus.ACCEPTED, HttpResponseStatus.ACCEPTED);
map.put(RestStatus.NON_AUTHORITATIVE_INFORMATION, HttpResponseStatus.NON_AUTHORITATIVE_INFORMATION);
map.put(RestStatus.NO_CONTENT, HttpResponseStatus.NO_CONTENT);
map.put(RestStatus.RESET_CONTENT, HttpResponseStatus.RESET_CONTENT);
map.put(RestStatus.PARTIAL_CONTENT, HttpResponseStatus.PARTIAL_CONTENT);
map.put(RestStatus.MULTI_STATUS, HttpResponseStatus.INTERNAL_SERVER_ERROR); // no status for this??
map.put(RestStatus.MULTIPLE_CHOICES, HttpResponseStatus.MULTIPLE_CHOICES);
map.put(RestStatus.MOVED_PERMANENTLY, HttpResponseStatus.MOVED_PERMANENTLY);
map.put(RestStatus.FOUND, HttpResponseStatus.FOUND);
map.put(RestStatus.SEE_OTHER, HttpResponseStatus.SEE_OTHER);
map.put(RestStatus.NOT_MODIFIED, HttpResponseStatus.NOT_MODIFIED);
map.put(RestStatus.USE_PROXY, HttpResponseStatus.USE_PROXY);
map.put(RestStatus.TEMPORARY_REDIRECT, HttpResponseStatus.TEMPORARY_REDIRECT);
map.put(RestStatus.BAD_REQUEST, HttpResponseStatus.BAD_REQUEST);
map.put(RestStatus.UNAUTHORIZED, HttpResponseStatus.UNAUTHORIZED);
map.put(RestStatus.PAYMENT_REQUIRED, HttpResponseStatus.PAYMENT_REQUIRED);
map.put(RestStatus.FORBIDDEN, HttpResponseStatus.FORBIDDEN);
map.put(RestStatus.NOT_FOUND, HttpResponseStatus.NOT_FOUND);
map.put(RestStatus.METHOD_NOT_ALLOWED, HttpResponseStatus.METHOD_NOT_ALLOWED);
map.put(RestStatus.NOT_ACCEPTABLE, HttpResponseStatus.NOT_ACCEPTABLE);
map.put(RestStatus.PROXY_AUTHENTICATION, HttpResponseStatus.PROXY_AUTHENTICATION_REQUIRED);
map.put(RestStatus.REQUEST_TIMEOUT, HttpResponseStatus.REQUEST_TIMEOUT);
map.put(RestStatus.CONFLICT, HttpResponseStatus.CONFLICT);
map.put(RestStatus.GONE, HttpResponseStatus.GONE);
map.put(RestStatus.LENGTH_REQUIRED, HttpResponseStatus.LENGTH_REQUIRED);
map.put(RestStatus.PRECONDITION_FAILED, HttpResponseStatus.PRECONDITION_FAILED);
map.put(RestStatus.REQUEST_ENTITY_TOO_LARGE, HttpResponseStatus.REQUEST_ENTITY_TOO_LARGE);
map.put(RestStatus.REQUEST_URI_TOO_LONG, HttpResponseStatus.REQUEST_URI_TOO_LONG);
map.put(RestStatus.UNSUPPORTED_MEDIA_TYPE, HttpResponseStatus.UNSUPPORTED_MEDIA_TYPE);
map.put(RestStatus.REQUESTED_RANGE_NOT_SATISFIED, HttpResponseStatus.REQUESTED_RANGE_NOT_SATISFIABLE);
map.put(RestStatus.EXPECTATION_FAILED, HttpResponseStatus.EXPECTATION_FAILED);
map.put(RestStatus.UNPROCESSABLE_ENTITY, HttpResponseStatus.BAD_REQUEST);
map.put(RestStatus.LOCKED, HttpResponseStatus.BAD_REQUEST);
map.put(RestStatus.FAILED_DEPENDENCY, HttpResponseStatus.BAD_REQUEST);
map.put(RestStatus.TOO_MANY_REQUESTS, TOO_MANY_REQUESTS);
map.put(RestStatus.INTERNAL_SERVER_ERROR, HttpResponseStatus.INTERNAL_SERVER_ERROR);
map.put(RestStatus.NOT_IMPLEMENTED, HttpResponseStatus.NOT_IMPLEMENTED);
map.put(RestStatus.BAD_GATEWAY, HttpResponseStatus.BAD_GATEWAY);
map.put(RestStatus.SERVICE_UNAVAILABLE, HttpResponseStatus.SERVICE_UNAVAILABLE);
map.put(RestStatus.GATEWAY_TIMEOUT, HttpResponseStatus.GATEWAY_TIMEOUT);
map.put(RestStatus.HTTP_VERSION_NOT_SUPPORTED, HttpResponseStatus.HTTP_VERSION_NOT_SUPPORTED);
MAP = Collections.unmodifiableMap(map);
}
private static HttpResponseStatus getStatus(RestStatus status) {
return MAP.getOrDefault(status, HttpResponseStatus.INTERNAL_SERVER_ERROR);
}
}

View File

@ -52,7 +52,7 @@ public class SynonymTokenFilterFactory extends AbstractTokenFilterFactory {
List<String> rules = Analysis.getWordList(env, settings, "synonyms");
StringBuilder sb = new StringBuilder();
for (String line : rules) {
sb.append(line).append(System.getProperty("line.separator"));
sb.append(line).append(System.lineSeparator());
}
rulesReader = new FastStringReader(sb.toString());
} else if (settings.get("synonyms_path") != null) {

View File

@ -362,13 +362,12 @@ public class InternalEngine extends Engine {
}
long expectedVersion = index.version();
if (index.versionType().isVersionConflictForWrites(currentVersion, expectedVersion, deleted)) {
if (index.origin() == Operation.Origin.RECOVERY) {
return false;
} else {
if (isVersionConflictForWrites(index, currentVersion, deleted, expectedVersion)) {
if (index.origin() != Operation.Origin.RECOVERY) {
throw new VersionConflictEngineException(shardId, index.type(), index.id(),
index.versionType().explainConflictForWrites(currentVersion, expectedVersion, deleted));
index.versionType().explainConflictForWrites(currentVersion, expectedVersion, deleted));
}
return false;
}
long updatedVersion = index.versionType().updateVersion(currentVersion, expectedVersion);
@ -378,22 +377,9 @@ public class InternalEngine extends Engine {
if (currentVersion == Versions.NOT_FOUND) {
// document does not exists, we can optimize for create
created = true;
if (index.docs().size() > 1) {
indexWriter.addDocuments(index.docs());
} else {
indexWriter.addDocument(index.docs().get(0));
}
index(index, indexWriter);
} else {
if (versionValue != null) {
created = versionValue.delete(); // we have a delete which is not GC'ed...
} else {
created = false;
}
if (index.docs().size() > 1) {
indexWriter.updateDocuments(index.uid(), index.docs());
} else {
indexWriter.updateDocument(index.uid(), index.docs().get(0));
}
created = update(index, versionValue, indexWriter);
}
Translog.Location translogLocation = translog.add(new Translog.Index(index));
@ -403,6 +389,33 @@ public class InternalEngine extends Engine {
}
}
private static boolean update(Index index, VersionValue versionValue, IndexWriter indexWriter) throws IOException {
boolean created;
if (versionValue != null) {
created = versionValue.delete(); // we have a delete which is not GC'ed...
} else {
created = false;
}
if (index.docs().size() > 1) {
indexWriter.updateDocuments(index.uid(), index.docs());
} else {
indexWriter.updateDocument(index.uid(), index.docs().get(0));
}
return created;
}
private static void index(Index index, IndexWriter indexWriter) throws IOException {
if (index.docs().size() > 1) {
indexWriter.addDocuments(index.docs());
} else {
indexWriter.addDocument(index.docs().get(0));
}
}
private boolean isVersionConflictForWrites(Index index, long currentVersion, boolean deleted, long expectedVersion) {
return index.versionType().isVersionConflictForWrites(currentVersion, expectedVersion, deleted);
}
@Override
public void delete(Delete delete) throws EngineException {
try (ReleasableLock lock = readLock.acquire()) {

View File

@ -190,48 +190,55 @@ public final class OrdinalsBuilder implements Closeable {
public int addOrdinal(int docID, long ordinal) {
final long position = positions.get(docID);
if (position == 0L) { // on the first level
// 0 or 1 ordinal
if (firstOrdinals.get(docID) == 0L) {
firstOrdinals.set(docID, ordinal + 1);
return 1;
} else {
final long newSlice = newSlice(1);
if (firstNextLevelSlices == null) {
firstNextLevelSlices = new PagedGrowableWriter(firstOrdinals.size(), PAGE_SIZE, 3, acceptableOverheadRatio);
}
firstNextLevelSlices.set(docID, newSlice);
final long offset = startOffset(1, newSlice);
ordinals[1].set(offset, ordinal + 1);
positions.set(docID, position(1, offset)); // current position is on the 1st level and not allocated yet
return 2;
}
return firstLevel(docID, ordinal);
} else {
int level = level(position);
long offset = offset(position, level);
assert offset != 0L;
if (((offset + 1) & slotsMask(level)) == 0L) {
// reached the end of the slice, allocate a new one on the next level
final long newSlice = newSlice(level + 1);
if (nextLevelSlices[level] == null) {
nextLevelSlices[level] = new PagedGrowableWriter(sizes[level], PAGE_SIZE, 1, acceptableOverheadRatio);
}
nextLevelSlices[level].set(sliceID(level, offset), newSlice);
++level;
offset = startOffset(level, newSlice);
assert (offset & slotsMask(level)) == 0L;
} else {
// just go to the next slot
++offset;
}
ordinals[level].set(offset, ordinal + 1);
final long newPosition = position(level, offset);
positions.set(docID, newPosition);
return numOrdinals(level, offset);
return nonFirstLevel(docID, ordinal, position);
}
}
private int firstLevel(int docID, long ordinal) {
// 0 or 1 ordinal
if (firstOrdinals.get(docID) == 0L) {
firstOrdinals.set(docID, ordinal + 1);
return 1;
} else {
final long newSlice = newSlice(1);
if (firstNextLevelSlices == null) {
firstNextLevelSlices = new PagedGrowableWriter(firstOrdinals.size(), PAGE_SIZE, 3, acceptableOverheadRatio);
}
firstNextLevelSlices.set(docID, newSlice);
final long offset = startOffset(1, newSlice);
ordinals[1].set(offset, ordinal + 1);
positions.set(docID, position(1, offset)); // current position is on the 1st level and not allocated yet
return 2;
}
}
private int nonFirstLevel(int docID, long ordinal, long position) {
int level = level(position);
long offset = offset(position, level);
assert offset != 0L;
if (((offset + 1) & slotsMask(level)) == 0L) {
// reached the end of the slice, allocate a new one on the next level
final long newSlice = newSlice(level + 1);
if (nextLevelSlices[level] == null) {
nextLevelSlices[level] = new PagedGrowableWriter(sizes[level], PAGE_SIZE, 1, acceptableOverheadRatio);
}
nextLevelSlices[level].set(sliceID(level, offset), newSlice);
++level;
offset = startOffset(level, newSlice);
assert (offset & slotsMask(level)) == 0L;
} else {
// just go to the next slot
++offset;
}
ordinals[level].set(offset, ordinal + 1);
final long newPosition = position(level, offset);
positions.set(docID, newPosition);
return numOrdinals(level, offset);
}
public void appendOrdinals(int docID, LongsRef ords) {
// First level
final long firstOrd = firstOrdinals.get(docID);

View File

@ -77,90 +77,115 @@ final class DocumentParser implements Closeable {
this.docMapper = docMapper;
}
public ParsedDocument parseDocument(SourceToParse source) throws MapperParsingException {
final ParsedDocument parseDocument(SourceToParse source) throws MapperParsingException {
validateType(source);
source.type(docMapper.type());
final Mapping mapping = docMapper.mapping();
final ParseContext.InternalParseContext context = cache.get();
XContentParser parser = null;
try {
parser = parser(source);
context.reset(parser, new ParseContext.Document(), source);
validateStart(parser);
internalParseDocument(mapping, context, parser);
validateEnd(source, parser);
} catch (Throwable t) {
throw wrapInMapperParsingException(source, t);
} finally {
// only close the parser when its not provided externally
if (internalParser(source, parser)) {
parser.close();
}
}
reverseOrder(context);
applyDocBoost(context);
ParsedDocument doc = parsedDocument(source, context, createDynamicUpdate(mapping, docMapper, context.getDynamicMappers()));
// reset the context to free up memory
context.reset(null, null, null);
return doc;
}
private static void internalParseDocument(Mapping mapping, ParseContext.InternalParseContext context, XContentParser parser) throws IOException {
final boolean emptyDoc = isEmptyDoc(mapping, parser);
for (MetadataFieldMapper metadataMapper : mapping.metadataMappers) {
metadataMapper.preParse(context);
}
if (mapping.root.isEnabled() == false) {
// entire type is disabled
parser.skipChildren();
} else if (emptyDoc == false) {
parseObjectOrNested(context, mapping.root, true);
}
for (MetadataFieldMapper metadataMapper : mapping.metadataMappers) {
metadataMapper.postParse(context);
}
}
private void validateType(SourceToParse source) {
if (docMapper.type().equals(MapperService.DEFAULT_MAPPING)) {
throw new IllegalArgumentException("It is forbidden to index into the default mapping [" + MapperService.DEFAULT_MAPPING + "]");
}
ParseContext.InternalParseContext context = cache.get();
final Mapping mapping = docMapper.mapping();
if (source.type() != null && !source.type().equals(docMapper.type())) {
throw new MapperParsingException("Type mismatch, provide type [" + source.type() + "] but mapper is of type [" + docMapper.type() + "]");
}
source.type(docMapper.type());
}
XContentParser parser = source.parser();
try {
if (parser == null) {
parser = XContentHelper.createParser(source.source());
}
context.reset(parser, new ParseContext.Document(), source);
private static XContentParser parser(SourceToParse source) throws IOException {
return source.parser() == null ? XContentHelper.createParser(source.source()) : source.parser();
}
// will result in START_OBJECT
XContentParser.Token token = parser.nextToken();
if (token != XContentParser.Token.START_OBJECT) {
throw new MapperParsingException("Malformed content, must start with an object");
}
private static boolean internalParser(SourceToParse source, XContentParser parser) {
return source.parser() == null && parser != null;
}
boolean emptyDoc = false;
if (mapping.root.isEnabled()) {
token = parser.nextToken();
if (token == XContentParser.Token.END_OBJECT) {
// empty doc, we can handle it...
emptyDoc = true;
} else if (token != XContentParser.Token.FIELD_NAME) {
throw new MapperParsingException("Malformed content, after first object, either the type field or the actual properties should exist");
}
}
for (MetadataFieldMapper metadataMapper : mapping.metadataMappers) {
metadataMapper.preParse(context);
}
if (mapping.root.isEnabled() == false) {
// entire type is disabled
parser.skipChildren();
} else if (emptyDoc == false) {
parseObjectOrNested(context, mapping.root, true);
}
for (MetadataFieldMapper metadataMapper : mapping.metadataMappers) {
metadataMapper.postParse(context);
}
private static void validateStart(XContentParser parser) throws IOException {
// will result in START_OBJECT
XContentParser.Token token = parser.nextToken();
if (token != XContentParser.Token.START_OBJECT) {
throw new MapperParsingException("Malformed content, must start with an object");
}
}
private static void validateEnd(SourceToParse source, XContentParser parser) throws IOException {
XContentParser.Token token;// only check for end of tokens if we created the parser here
if (internalParser(source, parser)) {
// try to parse the next token, this should be null if the object is ended properly
// but will throw a JSON exception if the extra tokens is not valid JSON (this will be handled by the catch)
if (source.parser() == null && parser != null) {
// only check for end of tokens if we created the parser here
token = parser.nextToken();
if (token != null) {
throw new IllegalArgumentException("Malformed content, found extra data after parsing: " + token);
}
}
} catch (Throwable e) {
// if its already a mapper parsing exception, no need to wrap it...
if (e instanceof MapperParsingException) {
throw (MapperParsingException) e;
}
// Throw a more meaningful message if the document is empty.
if (source.source() != null && source.source().length() == 0) {
throw new MapperParsingException("failed to parse, document is empty");
}
throw new MapperParsingException("failed to parse", e);
} finally {
// only close the parser when its not provided externally
if (source.parser() == null && parser != null) {
parser.close();
token = parser.nextToken();
if (token != null) {
throw new IllegalArgumentException("Malformed content, found extra data after parsing: " + token);
}
}
}
private static boolean isEmptyDoc(Mapping mapping, XContentParser parser) throws IOException {
if (mapping.root.isEnabled()) {
final XContentParser.Token token = parser.nextToken();
if (token == XContentParser.Token.END_OBJECT) {
// empty doc, we can handle it...
return true;
} else if (token != XContentParser.Token.FIELD_NAME) {
throw new MapperParsingException("Malformed content, after first object, either the type field or the actual properties should exist");
}
}
return false;
}
private static void reverseOrder(ParseContext.InternalParseContext context) {
// reverse the order of docs for nested docs support, parent should be last
if (context.docs().size() > 1) {
Collections.reverse(context.docs());
}
}
private static void applyDocBoost(ParseContext.InternalParseContext context) {
// apply doc boost
if (context.docBoost() != 1.0f) {
Set<String> encounteredFields = new HashSet<>();
@ -176,13 +201,36 @@ final class DocumentParser implements Closeable {
}
}
}
}
Mapping update = createDynamicUpdate(mapping, docMapper, context.getDynamicMappers());
ParsedDocument doc = new ParsedDocument(context.uid(), context.version(), context.id(), context.type(), source.routing(), source.timestamp(), source.ttl(), context.docs(),
context.source(), update).parent(source.parent());
// reset the context to free up memory
context.reset(null, null, null);
return doc;
private static ParsedDocument parsedDocument(SourceToParse source, ParseContext.InternalParseContext context, Mapping update) {
return new ParsedDocument(
context.uid(),
context.version(),
context.id(),
context.type(),
source.routing(),
source.timestamp(),
source.ttl(),
context.docs(),
context.source(),
update
).parent(source.parent());
}
private static MapperParsingException wrapInMapperParsingException(SourceToParse source, Throwable e) {
// if its already a mapper parsing exception, no need to wrap it...
if (e instanceof MapperParsingException) {
return (MapperParsingException) e;
}
// Throw a more meaningful message if the document is empty.
if (source.source() != null && source.source().length() == 0) {
return new MapperParsingException("failed to parse, document is empty");
}
return new MapperParsingException("failed to parse", e);
}
/** Creates a Mapping containing any dynamically added fields, or returns null if there were no dynamic mappings. */
@ -287,22 +335,7 @@ final class DocumentParser implements Closeable {
ObjectMapper.Nested nested = mapper.nested();
if (nested.isNested()) {
context = context.createNestedContext(mapper.fullPath());
ParseContext.Document nestedDoc = context.doc();
ParseContext.Document parentDoc = nestedDoc.getParent();
// pre add the uid field if possible (id was already provided)
IndexableField uidField = parentDoc.getField(UidFieldMapper.NAME);
if (uidField != null) {
// we don't need to add it as a full uid field in nested docs, since we don't need versioning
// we also rely on this for UidField#loadVersion
// this is a deeply nested field
nestedDoc.add(new Field(UidFieldMapper.NAME, uidField.stringValue(), UidFieldMapper.Defaults.NESTED_FIELD_TYPE));
}
// the type of the nested doc starts with __, so we can identify that its a nested one in filters
// note, we don't prefix it with the type of the doc since it allows us to execute a nested query
// across types (for example, with similar nested objects)
nestedDoc.add(new Field(TypeFieldMapper.NAME, mapper.nestedTypePathAsString(), TypeFieldMapper.Defaults.FIELD_TYPE));
context = nestedContext(context, mapper);
}
// if we are at the end of the previous object, advance
@ -315,8 +348,15 @@ final class DocumentParser implements Closeable {
}
ObjectMapper update = null;
innerParseObject(context, mapper, parser, currentFieldName, token);
// restore the enable path flag
if (nested.isNested()) {
nested(context, nested);
}
}
private static void innerParseObject(ParseContext context, ObjectMapper mapper, XContentParser parser, String currentFieldName, XContentParser.Token token) throws IOException {
while (token != XContentParser.Token.END_OBJECT) {
ObjectMapper newUpdate = null;
if (token == XContentParser.Token.START_OBJECT) {
parseObject(context, mapper, currentFieldName);
} else if (token == XContentParser.Token.START_ARRAY) {
@ -331,43 +371,52 @@ final class DocumentParser implements Closeable {
parseValue(context, mapper, currentFieldName, token);
}
token = parser.nextToken();
if (newUpdate != null) {
if (update == null) {
update = newUpdate;
} else {
update = update.merge(newUpdate, false);
}
}
}
private static void nested(ParseContext context, ObjectMapper.Nested nested) {
ParseContext.Document nestedDoc = context.doc();
ParseContext.Document parentDoc = nestedDoc.getParent();
if (nested.isIncludeInParent()) {
addFields(nestedDoc, parentDoc);
}
if (nested.isIncludeInRoot()) {
ParseContext.Document rootDoc = context.rootDoc();
// don't add it twice, if its included in parent, and we are handling the master doc...
if (!nested.isIncludeInParent() || parentDoc != rootDoc) {
addFields(nestedDoc, rootDoc);
}
}
// restore the enable path flag
if (nested.isNested()) {
ParseContext.Document nestedDoc = context.doc();
ParseContext.Document parentDoc = nestedDoc.getParent();
if (nested.isIncludeInParent()) {
for (IndexableField field : nestedDoc.getFields()) {
if (field.name().equals(UidFieldMapper.NAME) || field.name().equals(TypeFieldMapper.NAME)) {
continue;
} else {
parentDoc.add(field);
}
}
}
if (nested.isIncludeInRoot()) {
ParseContext.Document rootDoc = context.rootDoc();
// don't add it twice, if its included in parent, and we are handling the master doc...
if (!nested.isIncludeInParent() || parentDoc != rootDoc) {
for (IndexableField field : nestedDoc.getFields()) {
if (field.name().equals(UidFieldMapper.NAME) || field.name().equals(TypeFieldMapper.NAME)) {
continue;
} else {
rootDoc.add(field);
}
}
}
}
private static void addFields(ParseContext.Document nestedDoc, ParseContext.Document rootDoc) {
for (IndexableField field : nestedDoc.getFields()) {
if (!field.name().equals(UidFieldMapper.NAME) && !field.name().equals(TypeFieldMapper.NAME)) {
rootDoc.add(field);
}
}
}
private static ParseContext nestedContext(ParseContext context, ObjectMapper mapper) {
context = context.createNestedContext(mapper.fullPath());
ParseContext.Document nestedDoc = context.doc();
ParseContext.Document parentDoc = nestedDoc.getParent();
// pre add the uid field if possible (id was already provided)
IndexableField uidField = parentDoc.getField(UidFieldMapper.NAME);
if (uidField != null) {
// we don't need to add it as a full uid field in nested docs, since we don't need versioning
// we also rely on this for UidField#loadVersion
// this is a deeply nested field
nestedDoc.add(new Field(UidFieldMapper.NAME, uidField.stringValue(), UidFieldMapper.Defaults.NESTED_FIELD_TYPE));
}
// the type of the nested doc starts with __, so we can identify that its a nested one in filters
// note, we don't prefix it with the type of the doc since it allows us to execute a nested query
// across types (for example, with similar nested objects)
nestedDoc.add(new Field(TypeFieldMapper.NAME, mapper.nestedTypePathAsString(), TypeFieldMapper.Defaults.FIELD_TYPE));
return context;
}
private static void parseObjectOrField(ParseContext context, Mapper mapper) throws IOException {
if (mapper instanceof ObjectMapper) {
parseObjectOrNested(context, (ObjectMapper) mapper, false);

View File

@ -40,7 +40,6 @@ import org.elasticsearch.common.inject.Injector;
import org.elasticsearch.common.inject.Module;
import org.elasticsearch.common.inject.ModulesBuilder;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
@ -50,7 +49,6 @@ import org.elasticsearch.common.network.NetworkService;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.SettingsFilter;
import org.elasticsearch.common.settings.SettingsModule;
import org.elasticsearch.common.transport.BoundTransportAddress;
import org.elasticsearch.common.transport.TransportAddress;
@ -279,8 +277,6 @@ public class Node implements Closeable {
injector.getInstance(IndicesTTLService.class).start();
injector.getInstance(SnapshotsService.class).start();
injector.getInstance(SnapshotShardsService.class).start();
injector.getInstance(TransportService.class).start();
injector.getInstance(ClusterService.class).start();
injector.getInstance(RoutingService.class).start();
injector.getInstance(SearchService.class).start();
injector.getInstance(MonitorService.class).start();
@ -289,16 +285,24 @@ public class Node implements Closeable {
// TODO hack around circular dependencies problems
injector.getInstance(GatewayAllocator.class).setReallocation(injector.getInstance(ClusterService.class), injector.getInstance(RoutingService.class));
DiscoveryService discoService = injector.getInstance(DiscoveryService.class).start();
discoService.waitForInitialState();
// gateway should start after disco, so it can try and recovery from gateway on "start"
injector.getInstance(ResourceWatcherService.class).start();
injector.getInstance(GatewayService.class).start();
// Start the transport service now so the publish address will be added to the local disco node in ClusterService
TransportService transportService = injector.getInstance(TransportService.class);
transportService.start();
injector.getInstance(ClusterService.class).start();
// start after cluster service so the local disco is known
DiscoveryService discoService = injector.getInstance(DiscoveryService.class).start();
transportService.acceptIncomingRequests();
discoService.joinClusterAndWaitForInitialState();
if (settings.getAsBoolean("http.enabled", true)) {
injector.getInstance(HttpServer.class).start();
}
injector.getInstance(ResourceWatcherService.class).start();
injector.getInstance(TribeService.class).start();
if (WRITE_PORTS_FIELD_SETTING.get(settings)) {

View File

@ -30,6 +30,7 @@ import org.elasticsearch.rest.RestChannel;
import org.elasticsearch.rest.RestController;
import org.elasticsearch.rest.RestRequest;
import org.elasticsearch.rest.action.support.RestToXContentListener;
import org.elasticsearch.tasks.TaskId;
import static org.elasticsearch.rest.RestRequest.Method.POST;
@ -40,22 +41,20 @@ public class RestCancelTasksAction extends BaseRestHandler {
public RestCancelTasksAction(Settings settings, RestController controller, Client client) {
super(settings, client);
controller.registerHandler(POST, "/_tasks/_cancel", this);
controller.registerHandler(POST, "/_tasks/{nodeId}/_cancel", this);
controller.registerHandler(POST, "/_tasks/{nodeId}/{taskId}/_cancel", this);
controller.registerHandler(POST, "/_tasks/{taskId}/_cancel", this);
}
@Override
public void handleRequest(final RestRequest request, final RestChannel channel, final Client client) {
String[] nodesIds = Strings.splitStringByCommaToArray(request.param("nodeId"));
long taskId = request.paramAsLong("taskId", ListTasksRequest.ALL_TASKS);
TaskId taskId = new TaskId(request.param("taskId"));
String[] actions = Strings.splitStringByCommaToArray(request.param("actions"));
String parentNode = request.param("parent_node");
long parentTaskId = request.paramAsLong("parent_task", ListTasksRequest.ALL_TASKS);
TaskId parentTaskId = new TaskId(request.param("parent_task_id"));
CancelTasksRequest cancelTasksRequest = new CancelTasksRequest(nodesIds);
CancelTasksRequest cancelTasksRequest = new CancelTasksRequest();
cancelTasksRequest.taskId(taskId);
cancelTasksRequest.nodesIds(nodesIds);
cancelTasksRequest.actions(actions);
cancelTasksRequest.parentNode(parentNode);
cancelTasksRequest.parentTaskId(parentTaskId);
client.admin().cluster().cancelTasks(cancelTasksRequest, new RestToXContentListener<>(channel));
}

View File

@ -29,6 +29,7 @@ import org.elasticsearch.rest.RestChannel;
import org.elasticsearch.rest.RestController;
import org.elasticsearch.rest.RestRequest;
import org.elasticsearch.rest.action.support.RestToXContentListener;
import org.elasticsearch.tasks.TaskId;
import static org.elasticsearch.rest.RestRequest.Method.GET;
@ -39,24 +40,22 @@ public class RestListTasksAction extends BaseRestHandler {
public RestListTasksAction(Settings settings, RestController controller, Client client) {
super(settings, client);
controller.registerHandler(GET, "/_tasks", this);
controller.registerHandler(GET, "/_tasks/{nodeId}", this);
controller.registerHandler(GET, "/_tasks/{nodeId}/{taskId}", this);
controller.registerHandler(GET, "/_tasks/{taskId}", this);
}
@Override
public void handleRequest(final RestRequest request, final RestChannel channel, final Client client) {
boolean detailed = request.paramAsBoolean("detailed", false);
String[] nodesIds = Strings.splitStringByCommaToArray(request.param("nodeId"));
long taskId = request.paramAsLong("taskId", ListTasksRequest.ALL_TASKS);
String[] nodesIds = Strings.splitStringByCommaToArray(request.param("node_id"));
TaskId taskId = new TaskId(request.param("taskId"));
String[] actions = Strings.splitStringByCommaToArray(request.param("actions"));
String parentNode = request.param("parent_node");
long parentTaskId = request.paramAsLong("parent_task", ListTasksRequest.ALL_TASKS);
TaskId parentTaskId = new TaskId(request.param("parent_task_id"));
ListTasksRequest listTasksRequest = new ListTasksRequest(nodesIds);
ListTasksRequest listTasksRequest = new ListTasksRequest();
listTasksRequest.taskId(taskId);
listTasksRequest.nodesIds(nodesIds);
listTasksRequest.detailed(detailed);
listTasksRequest.actions(actions);
listTasksRequest.parentNode(parentNode);
listTasksRequest.parentTaskId(parentTaskId);
client.admin().cluster().listTasks(listTasksRequest, new RestToXContentListener<>(channel));
}

View File

@ -142,13 +142,22 @@ public class RestUtils {
* @throws IllegalArgumentException if the string contains a malformed
* escape sequence.
*/
@SuppressWarnings("fallthrough")
public static String decodeComponent(final String s, final Charset charset) {
if (s == null) {
return "";
}
final int size = s.length();
boolean modified = false;
if (!decodingNeeded(s, size)) {
return s;
}
final byte[] buf = new byte[size];
int pos = decode(s, size, buf);
return new String(buf, 0, pos, charset);
}
@SuppressWarnings("fallthrough")
private static boolean decodingNeeded(String s, int size) {
boolean decodingNeeded = false;
for (int i = 0; i < size; i++) {
final char c = s.charAt(i);
switch (c) {
@ -156,14 +165,15 @@ public class RestUtils {
i++; // We can skip at least one char, e.g. `%%'.
// Fall through.
case '+':
modified = true;
decodingNeeded = true;
break;
}
}
if (!modified) {
return s;
}
final byte[] buf = new byte[size];
return decodingNeeded;
}
@SuppressWarnings("fallthrough")
private static int decode(String s, int size, byte[] buf) {
int pos = 0; // position in `buf'.
for (int i = 0; i < size; i++) {
char c = s.charAt(i);
@ -173,24 +183,22 @@ public class RestUtils {
break;
case '%':
if (i == size - 1) {
throw new IllegalArgumentException("unterminated escape"
+ " sequence at end of string: " + s);
throw new IllegalArgumentException("unterminated escape sequence at end of string: " + s);
}
c = s.charAt(++i);
if (c == '%') {
buf[pos++] = '%'; // "%%" -> "%"
break;
} else if (i == size - 1) {
throw new IllegalArgumentException("partial escape"
+ " sequence at end of string: " + s);
throw new IllegalArgumentException("partial escape sequence at end of string: " + s);
}
c = decodeHexNibble(c);
final char c2 = decodeHexNibble(s.charAt(++i));
if (c == Character.MAX_VALUE || c2 == Character.MAX_VALUE) {
throw new IllegalArgumentException(
"invalid escape sequence `%" + s.charAt(i - 1)
+ s.charAt(i) + "' at index " + (i - 2)
+ " of: " + s);
"invalid escape sequence `%" + s.charAt(i - 1)
+ s.charAt(i) + "' at index " + (i - 2)
+ " of: " + s);
}
c = (char) (c * 16 + c2);
// Fall through.
@ -199,7 +207,7 @@ public class RestUtils {
break;
}
}
return new String(buf, 0, pos, charset);
return pos;
}
/**

View File

@ -33,6 +33,7 @@ import org.elasticsearch.common.geo.builders.PolygonBuilder;
import org.elasticsearch.common.geo.builders.ShapeBuilder;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.inject.multibindings.Multibinder;
import org.elasticsearch.common.io.stream.NamedWriteable;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.query.BoolQueryParser;
@ -283,14 +284,15 @@ public class SearchModule extends AbstractModule {
/**
* Register a new ScoreFunctionParser.
*/
public void registerFunctionScoreParser(ScoreFunctionParser<?> parser) {
public void registerFunctionScoreParser(ScoreFunctionParser<? extends ScoreFunctionBuilder> parser) {
for (String name: parser.getNames()) {
Object oldValue = functionScoreParsers.putIfAbsent(name, parser);
if (oldValue != null) {
throw new IllegalArgumentException("Function score parser [" + oldValue + "] already registered for name [" + name + "]");
}
}
namedWriteableRegistry.registerPrototype(ScoreFunctionBuilder.class, parser.getBuilderPrototype());
@SuppressWarnings("unchecked") NamedWriteable<? extends ScoreFunctionBuilder> sfb = parser.getBuilderPrototype();
namedWriteableRegistry.registerPrototype(ScoreFunctionBuilder.class, sfb);
}
public void registerQueryParser(Supplier<QueryParser<?>> parser) {
@ -354,14 +356,15 @@ public class SearchModule extends AbstractModule {
public IndicesQueriesRegistry buildQueryParserRegistry() {
Map<String, QueryParser<?>> queryParsersMap = new HashMap<>();
for (Supplier<QueryParser<?>> parserSupplier : queryParsers) {
QueryParser<?> parser = parserSupplier.get();
QueryParser<? extends QueryBuilder> parser = parserSupplier.get();
for (String name: parser.names()) {
Object oldValue = queryParsersMap.putIfAbsent(name, parser);
if (oldValue != null) {
throw new IllegalArgumentException("Query parser [" + oldValue + "] already registered for name [" + name + "] while trying to register [" + parser + "]");
}
}
namedWriteableRegistry.registerPrototype(QueryBuilder.class, parser.getBuilderPrototype());
@SuppressWarnings("unchecked") NamedWriteable<? extends QueryBuilder> qb = parser.getBuilderPrototype();
namedWriteableRegistry.registerPrototype(QueryBuilder.class, qb);
}
return new IndicesQueriesRegistry(settings, queryParsersMap);
}

View File

@ -32,8 +32,8 @@ public class CancellableTask extends Task {
super(id, type, action, description);
}
public CancellableTask(long id, String type, String action, String description, String parentNode, long parentId) {
super(id, type, action, description, parentNode, parentId);
public CancellableTask(long id, String type, String action, String description, TaskId parentTaskId) {
super(id, type, action, description, parentTaskId);
}
/**

View File

@ -30,8 +30,6 @@ import org.elasticsearch.common.xcontent.ToXContent;
*/
public class Task {
public static final long NO_PARENT_ID = 0;
private final long id;
private final String type;
@ -40,22 +38,18 @@ public class Task {
private final String description;
private final String parentNode;
private final long parentId;
private final TaskId parentTask;
public Task(long id, String type, String action, String description) {
this(id, type, action, description, null, NO_PARENT_ID);
this(id, type, action, description, TaskId.EMPTY_TASK_ID);
}
public Task(long id, String type, String action, String description, String parentNode, long parentId) {
public Task(long id, String type, String action, String description, TaskId parentTask) {
this.id = id;
this.type = type;
this.action = action;
this.description = description;
this.parentNode = parentNode;
this.parentId = parentId;
this.parentTask = parentTask;
}
/**
@ -75,7 +69,7 @@ public class Task {
description = getDescription();
status = getStatus();
}
return new TaskInfo(node, getId(), getType(), getAction(), description, status, parentNode, parentId);
return new TaskInfo(node, getId(), getType(), getAction(), description, status, parentTask);
}
/**
@ -106,18 +100,11 @@ public class Task {
return description;
}
/**
* Returns the parent node of the task or null if the task doesn't have any parent tasks
*/
public String getParentNode() {
return parentNode;
}
/**
* Returns id of the parent task or NO_PARENT_ID if the task doesn't have any parent tasks
*/
public long getParentId() {
return parentId;
public TaskId getParentTaskId() {
return parentTask;
}
/**

View File

@ -0,0 +1,118 @@
/*
* 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.tasks;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
import java.io.IOException;
/**
* Task id that consists of node id and id of the task on the node
*/
public final class TaskId implements Writeable<TaskId> {
public final static TaskId EMPTY_TASK_ID = new TaskId("", -1L);
private final String nodeId;
private final long id;
public TaskId(String nodeId, long id) {
this.nodeId = nodeId;
this.id = id;
}
public TaskId(String taskId) {
if (Strings.hasLength(taskId) && "unset".equals(taskId) == false) {
String[] s = Strings.split(taskId, ":");
if (s == null || s.length != 2) {
throw new IllegalArgumentException("malformed task id " + taskId);
}
this.nodeId = s[0];
try {
this.id = Long.parseLong(s[1]);
} catch (NumberFormatException ex) {
throw new IllegalArgumentException("malformed task id " + taskId, ex);
}
} else {
nodeId = "";
id = -1L;
}
}
public TaskId(StreamInput in) throws IOException {
nodeId = in.readString();
id = in.readLong();
}
public String getNodeId() {
return nodeId;
}
public long getId() {
return id;
}
public boolean isSet() {
return id == -1L;
}
@Override
public String toString() {
if (isSet()) {
return "unset";
} else {
return nodeId + ":" + id;
}
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(nodeId);
out.writeLong(id);
}
@Override
public TaskId readFrom(StreamInput in) throws IOException {
return new TaskId(in);
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
TaskId taskId = (TaskId) o;
if (id != taskId.id) return false;
return nodeId.equals(taskId.nodeId);
}
@Override
public int hashCode() {
int result = nodeId.hashCode();
result = 31 * result + (int) (id ^ (id >>> 32));
return result;
}
}

View File

@ -22,7 +22,6 @@ package org.elasticsearch.tasks;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterStateListener;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
@ -51,7 +50,7 @@ public class TaskManager extends AbstractComponent implements ClusterStateListen
private final AtomicLong taskIdGenerator = new AtomicLong();
private final Map<Tuple<String, Long>, String> banedParents = new ConcurrentHashMap<>();
private final Map<TaskId, String> banedParents = new ConcurrentHashMap<>();
public TaskManager(Settings settings) {
super(settings);
@ -77,8 +76,8 @@ public class TaskManager extends AbstractComponent implements ClusterStateListen
CancellableTaskHolder oldHolder = cancellableTasks.put(task.getId(), holder);
assert oldHolder == null;
// Check if this task was banned before we start it
if (task.getParentNode() != null && banedParents.isEmpty() == false) {
String reason = banedParents.get(new Tuple<>(task.getParentNode(), task.getParentId()));
if (task.getParentTaskId().isSet() == false && banedParents.isEmpty() == false) {
String reason = banedParents.get(task.getParentTaskId());
if (reason != null) {
try {
holder.cancel(reason);
@ -191,22 +190,21 @@ public class TaskManager extends AbstractComponent implements ClusterStateListen
* <p>
* This method is called when a parent task that has children is cancelled.
*/
public void setBan(String parentNode, long parentId, String reason) {
logger.trace("setting ban for the parent task {}:{} {}", parentNode, parentId, reason);
public void setBan(TaskId parentTaskId, String reason) {
logger.trace("setting ban for the parent task {} {}", parentTaskId, reason);
// Set the ban first, so the newly created tasks cannot be registered
Tuple<String, Long> ban = new Tuple<>(parentNode, parentId);
synchronized (banedParents) {
if (lastDiscoveryNodes.nodeExists(parentNode)) {
if (lastDiscoveryNodes.nodeExists(parentTaskId.getNodeId())) {
// Only set the ban if the node is the part of the cluster
banedParents.put(ban, reason);
banedParents.put(parentTaskId, reason);
}
}
// Now go through already running tasks and cancel them
for (Map.Entry<Long, CancellableTaskHolder> taskEntry : cancellableTasks.entrySet()) {
CancellableTaskHolder holder = taskEntry.getValue();
if (holder.hasParent(parentNode, parentId)) {
if (holder.hasParent(parentTaskId)) {
holder.cancel(reason);
}
}
@ -217,9 +215,9 @@ public class TaskManager extends AbstractComponent implements ClusterStateListen
* <p>
* This method is called when a previously banned task finally cancelled
*/
public void removeBan(String parentNode, long parentId) {
logger.trace("removing ban for the parent task {}:{} {}", parentNode, parentId);
banedParents.remove(new Tuple<>(parentNode, parentId));
public void removeBan(TaskId parentTaskId) {
logger.trace("removing ban for the parent task {}", parentTaskId);
banedParents.remove(parentTaskId);
}
@Override
@ -228,14 +226,12 @@ public class TaskManager extends AbstractComponent implements ClusterStateListen
synchronized (banedParents) {
lastDiscoveryNodes = event.state().getNodes();
// Remove all bans that were registered by nodes that are no longer in the cluster state
Iterator<Tuple<String, Long>> banIterator = banedParents.keySet().iterator();
Iterator<TaskId> banIterator = banedParents.keySet().iterator();
while (banIterator.hasNext()) {
Tuple<String, Long> nodeAndTaskId = banIterator.next();
String nodeId = nodeAndTaskId.v1();
Long taskId = nodeAndTaskId.v2();
if (lastDiscoveryNodes.nodeExists(nodeId) == false) {
logger.debug("Removing ban for the parent [{}:{}] on the node [{}], reason: the parent node is gone", nodeId,
taskId, event.state().getNodes().localNode());
TaskId taskId = banIterator.next();
if (lastDiscoveryNodes.nodeExists(taskId.getNodeId()) == false) {
logger.debug("Removing ban for the parent [{}] on the node [{}], reason: the parent node is gone", taskId,
event.state().getNodes().localNode());
banIterator.remove();
}
}
@ -244,10 +240,10 @@ public class TaskManager extends AbstractComponent implements ClusterStateListen
for (Map.Entry<Long, CancellableTaskHolder> taskEntry : cancellableTasks.entrySet()) {
CancellableTaskHolder holder = taskEntry.getValue();
CancellableTask task = holder.getTask();
String parent = task.getParentNode();
if (parent != null && lastDiscoveryNodes.nodeExists(parent) == false) {
TaskId parentTaskId = task.getParentTaskId();
if (parentTaskId.isSet() == false && lastDiscoveryNodes.nodeExists(parentTaskId.getNodeId()) == false) {
if (task.cancelOnParentLeaving()) {
holder.cancel("Coordinating node [" + parent + "] left the cluster");
holder.cancel("Coordinating node [" + parentTaskId.getNodeId() + "] left the cluster");
}
}
}
@ -340,8 +336,8 @@ public class TaskManager extends AbstractComponent implements ClusterStateListen
}
public boolean hasParent(String parentNode, long parentId) {
return parentId == task.getParentId() && parentNode.equals(task.getParentNode());
public boolean hasParent(TaskId parentTaskId) {
return task.getParentTaskId().equals(parentTaskId);
}
public CancellableTask getTask() {

View File

@ -54,8 +54,8 @@ import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
import java.util.function.Supplier;
@ -71,7 +71,7 @@ public class TransportService extends AbstractLifecycleComponent<TransportServic
public static final String DIRECT_RESPONSE_PROFILE = ".direct";
private final AtomicBoolean started = new AtomicBoolean(false);
private final CountDownLatch blockIncomingRequestsLatch = new CountDownLatch(1);
protected final Transport transport;
protected final ThreadPool threadPool;
protected final TaskManager taskManager;
@ -167,6 +167,7 @@ public class TransportService extends AbstractLifecycleComponent<TransportServic
void setTracerLogExclude(List<String> tracelLogExclude) {
this.tracelLogExclude = tracelLogExclude.toArray(Strings.EMPTY_ARRAY);
}
@Override
protected void doStart() {
adapter.rxMetric.clear();
@ -179,14 +180,10 @@ public class TransportService extends AbstractLifecycleComponent<TransportServic
logger.info("profile [{}]: {}", entry.getKey(), entry.getValue());
}
}
boolean setStarted = started.compareAndSet(false, true);
assert setStarted : "service was already started";
}
@Override
protected void doStop() {
final boolean setStopped = started.compareAndSet(true, false);
assert setStopped : "service has already been stopped";
try {
transport.stop();
} finally {
@ -213,6 +210,15 @@ public class TransportService extends AbstractLifecycleComponent<TransportServic
transport.close();
}
/**
* start accepting incoming requests.
* when the transport layer starts up it will block any incoming requests until
* this method is called
*/
public void acceptIncomingRequests() {
blockIncomingRequestsLatch.countDown();
}
public boolean addressSupported(Class<? extends TransportAddress> address) {
return transport.addressSupported(address);
}
@ -302,7 +308,7 @@ public class TransportService extends AbstractLifecycleComponent<TransportServic
timeoutHandler = new TimeoutHandler(requestId);
}
clientHandlers.put(requestId, new RequestHolder<>(new ContextRestoreResponseHandler<T>(threadPool.getThreadContext().newStoredContext(), handler), node, action, timeoutHandler));
if (started.get() == false) {
if (lifecycle.stoppedOrClosed()) {
// if we are not started the exception handling will remove the RequestHolder again and calls the handler to notify the caller.
// it will only notify if the toStop code hasn't done the work yet.
throw new TransportException("TransportService is closed stopped can't send request");
@ -405,10 +411,11 @@ public class TransportService extends AbstractLifecycleComponent<TransportServic
/**
* Registers a new request handler
* @param action The action the request handler is associated with
*
* @param action The action the request handler is associated with
* @param requestFactory a callable to be used construct new instances for streaming
* @param executor The executor the request handling will be executed on
* @param handler The handler itself that implements the request handling
* @param executor The executor the request handling will be executed on
* @param handler The handler itself that implements the request handling
*/
public <Request extends TransportRequest> void registerRequestHandler(String action, Supplier<Request> requestFactory, String executor, TransportRequestHandler<Request> handler) {
RequestHandlerRegistry<Request> reg = new RequestHandlerRegistry<>(action, requestFactory, taskManager, handler, executor, false);
@ -417,11 +424,12 @@ public class TransportService extends AbstractLifecycleComponent<TransportServic
/**
* Registers a new request handler
* @param action The action the request handler is associated with
* @param request The request class that will be used to constrcut new instances for streaming
* @param executor The executor the request handling will be executed on
*
* @param action The action the request handler is associated with
* @param request The request class that will be used to constrcut new instances for streaming
* @param executor The executor the request handling will be executed on
* @param forceExecution Force execution on the executor queue and never reject it
* @param handler The handler itself that implements the request handling
* @param handler The handler itself that implements the request handling
*/
public <Request extends TransportRequest> void registerRequestHandler(String action, Supplier<Request> request, String executor, boolean forceExecution, TransportRequestHandler<Request> handler) {
RequestHandlerRegistry<Request> reg = new RequestHandlerRegistry<>(action, request, taskManager, handler, executor, forceExecution);
@ -494,6 +502,11 @@ public class TransportService extends AbstractLifecycleComponent<TransportServic
@Override
public void onRequestReceived(long requestId, String action) {
try {
blockIncomingRequestsLatch.await();
} catch (InterruptedException e) {
logger.trace("interrupted while waiting for incoming requests block to be removed");
}
if (traceEnabled() && shouldTraceAction(action)) {
traceReceivedRequest(requestId, action);
}
@ -729,6 +742,7 @@ public class TransportService extends AbstractLifecycleComponent<TransportServic
private final static class ContextRestoreResponseHandler<T extends TransportResponse> implements TransportResponseHandler<T> {
private final TransportResponseHandler<T> delegate;
private final ThreadContext.StoredContext threadContext;
private ContextRestoreResponseHandler(ThreadContext.StoredContext threadContext, TransportResponseHandler<T> delegate) {
this.delegate = delegate;
this.threadContext = threadContext;
@ -766,7 +780,7 @@ public class TransportService extends AbstractLifecycleComponent<TransportServic
final ThreadPool threadPool;
public DirectResponseChannel(ESLogger logger, DiscoveryNode localNode, String action, long requestId,
TransportServiceAdapter adapter, ThreadPool threadPool) {
TransportServiceAdapter adapter, ThreadPool threadPool) {
this.logger = logger;
this.localNode = localNode;
this.action = action;

View File

@ -36,6 +36,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.tasks.CancellableTask;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
@ -87,8 +88,8 @@ public class CancellableTasksTests extends TaskManagerTestCase {
}
@Override
public Task createTask(long id, String type, String action, String parentTaskNode, long parentTaskId) {
return new CancellableTask(id, type, action, getDescription(), parentTaskNode, parentTaskId);
public Task createTask(long id, String type, String action, TaskId parentTaskId) {
return new CancellableTask(id, type, action, getDescription(), parentTaskId);
}
}
@ -235,9 +236,9 @@ public class CancellableTasksTests extends TaskManagerTestCase {
});
// Cancel main task
CancelTasksRequest request = new CancelTasksRequest(testNodes[0].discoveryNode.getId());
CancelTasksRequest request = new CancelTasksRequest();
request.reason("Testing Cancellation");
request.taskId(mainTask.getId());
request.taskId(new TaskId(testNodes[0].discoveryNode.getId(), mainTask.getId()));
// And send the cancellation request to a random node
CancelTasksResponse response = testNodes[randomIntBetween(0, testNodes.length - 1)].transportCancelTasksAction.execute(request)
.get();
@ -269,7 +270,8 @@ public class CancellableTasksTests extends TaskManagerTestCase {
// Make sure that tasks are no longer running
ListTasksResponse listTasksResponse = testNodes[randomIntBetween(0, testNodes.length - 1)]
.transportListTasksAction.execute(new ListTasksRequest(testNodes[0].discoveryNode.getId()).taskId(mainTask.getId())).get();
.transportListTasksAction.execute(new ListTasksRequest().taskId(
new TaskId(testNodes[0].discoveryNode.getId(), mainTask.getId()))).get();
assertEquals(0, listTasksResponse.getTasks().size());
// Make sure that there are no leftover bans, the ban removal is async, so we might return from the cancellation
@ -311,7 +313,7 @@ public class CancellableTasksTests extends TaskManagerTestCase {
// Make sure that tasks are running
ListTasksResponse listTasksResponse = testNodes[randomIntBetween(0, testNodes.length - 1)]
.transportListTasksAction.execute(new ListTasksRequest().parentNode(mainNode).taskId(mainTask.getId())).get();
.transportListTasksAction.execute(new ListTasksRequest().parentTaskId(new TaskId(mainNode, mainTask.getId()))).get();
assertThat(listTasksResponse.getTasks().size(), greaterThanOrEqualTo(blockOnNodes.size()));
// Simulate the coordinating node leaving the cluster
@ -328,9 +330,9 @@ public class CancellableTasksTests extends TaskManagerTestCase {
if (simulateBanBeforeLeaving) {
logger.info("--> Simulate issuing cancel request on the node that is about to leave the cluster");
// Simulate issuing cancel request on the node that is about to leave the cluster
CancelTasksRequest request = new CancelTasksRequest(testNodes[0].discoveryNode.getId());
CancelTasksRequest request = new CancelTasksRequest();
request.reason("Testing Cancellation");
request.taskId(mainTask.getId());
request.taskId(new TaskId(testNodes[0].discoveryNode.getId(), mainTask.getId()));
// And send the cancellation request to a random node
CancelTasksResponse response = testNodes[0].transportCancelTasksAction.execute(request).get();
logger.info("--> Done simulating issuing cancel request on the node that is about to leave the cluster");
@ -354,7 +356,7 @@ public class CancellableTasksTests extends TaskManagerTestCase {
// Make sure that tasks are no longer running
try {
ListTasksResponse listTasksResponse1 = testNodes[randomIntBetween(1, testNodes.length - 1)]
.transportListTasksAction.execute(new ListTasksRequest().parentNode(mainNode).taskId(mainTask.getId())).get();
.transportListTasksAction.execute(new ListTasksRequest().taskId(new TaskId(mainNode, mainTask.getId()))).get();
assertEquals(0, listTasksResponse1.getTasks().size());
} catch (InterruptedException ex) {
Thread.currentThread().interrupt();

View File

@ -203,6 +203,7 @@ public abstract class TaskManagerTestCase extends ESTestCase {
actionFilters, indexNameExpressionResolver);
transportCancelTasksAction = new TransportCancelTasksAction(settings, clusterName, threadPool, clusterService, transportService,
actionFilters, indexNameExpressionResolver);
transportService.acceptIncomingRequests();
}
public final TestClusterService clusterService;

View File

@ -110,7 +110,7 @@ public class TasksIT extends ESIntegTestCase {
List<TaskInfo> tasks = findEvents(ClusterHealthAction.NAME, Tuple::v1);
// Verify that one of these tasks is a parent of another task
if (tasks.get(0).getParentNode() == null) {
if (tasks.get(0).getParentTaskId().isSet()) {
assertParentTask(Collections.singletonList(tasks.get(1)), tasks.get(0));
} else {
assertParentTask(Collections.singletonList(tasks.get(0)), tasks.get(1));
@ -227,7 +227,9 @@ public class TasksIT extends ESIntegTestCase {
} else {
// A [s][r] level task should have a corresponding [s] level task on the a different node (where primary is located)
sTask = findEvents(RefreshAction.NAME + "[s]",
event -> event.v1() && taskInfo.getParentNode().equals(event.v2().getNode().getId()) && taskInfo.getDescription().equals(event.v2().getDescription()));
event -> event.v1() && taskInfo.getParentTaskId().getNodeId().equals(event.v2().getNode().getId()) && taskInfo
.getDescription()
.equals(event.v2().getDescription()));
}
// There should be only one parent task
assertEquals(1, sTask.size());
@ -393,9 +395,10 @@ public class TasksIT extends ESIntegTestCase {
*/
private void assertParentTask(List<TaskInfo> tasks, TaskInfo parentTask) {
for (TaskInfo task : tasks) {
assertNotNull(task.getParentNode());
assertEquals(parentTask.getNode().getId(), task.getParentNode());
assertEquals(parentTask.getId(), task.getParentId());
assertFalse(task.getParentTaskId().isSet());
assertEquals(parentTask.getNode().getId(), task.getParentTaskId().getNodeId());
assertTrue(Strings.hasLength(task.getParentTaskId().getNodeId()));
assertEquals(parentTask.getId(), task.getParentTaskId().getId());
}
}
}

View File

@ -47,6 +47,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.tasks.CancellableTask;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
@ -84,8 +85,8 @@ public class TestTaskPlugin extends Plugin {
private volatile boolean blocked = true;
public TestTask(long id, String type, String action, String description, String parentNode, long parentId) {
super(id, type, action, description, parentNode, parentId);
public TestTask(long id, String type, String action, String description, TaskId parentTaskId) {
super(id, type, action, description, parentTaskId);
}
public boolean isBlocked() {
@ -172,8 +173,8 @@ public class TestTaskPlugin extends Plugin {
}
@Override
public Task createTask(long id, String type, String action, String parentTaskNode, long parentTaskId) {
return new TestTask(id, type, action, this.getDescription(), parentTaskNode, parentTaskId);
public Task createTask(long id, String type, String action, TaskId parentTaskId) {
return new TestTask(id, type, action, this.getDescription(), parentTaskId);
}
}

View File

@ -43,13 +43,11 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.test.tasks.MockTaskManager;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.transport.local.LocalTransport;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import java.io.IOException;
import java.util.ArrayList;
@ -103,9 +101,9 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
}
@Override
public Task createTask(long id, String type, String action, String parentTaskNode, long parentTaskId) {
public Task createTask(long id, String type, String action, TaskId parentTaskId) {
if (enableTaskManager) {
return super.createTask(id, type, action, parentTaskNode, parentTaskId);
return super.createTask(id, type, action, parentTaskId);
} else {
return null;
}
@ -313,7 +311,7 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
}
Task task = actions[0].execute(request, listener);
logger.info("Awaiting for all actions to start");
actionLatch.await();
assertTrue(actionLatch.await(10, TimeUnit.SECONDS));
logger.info("Done waiting for all actions to start");
return task;
}
@ -426,14 +424,13 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
// Find tasks with common parent
listTasksRequest = new ListTasksRequest();
listTasksRequest.parentNode(parentNode);
listTasksRequest.parentTaskId(parentTaskId);
listTasksRequest.parentTaskId(new TaskId(parentNode, parentTaskId));
response = testNode.transportListTasksAction.execute(listTasksRequest).get();
assertEquals(testNodes.length, response.getTasks().size());
for (TaskInfo task : response.getTasks()) {
assertEquals("testAction[n]", task.getAction());
assertEquals(parentNode, task.getParentNode());
assertEquals(parentTaskId, task.getParentId());
assertEquals(parentNode, task.getParentTaskId().getNodeId());
assertEquals(parentTaskId, task.getParentTaskId().getId());
}
// Release all tasks and wait for response
@ -514,7 +511,8 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
String actionName = "testAction"; // only pick the main action
// Try to cancel main task using action name
CancelTasksRequest request = new CancelTasksRequest(testNodes[0].discoveryNode.getId());
CancelTasksRequest request = new CancelTasksRequest();
request.nodesIds(testNodes[0].discoveryNode.getId());
request.reason("Testing Cancellation");
request.actions(actionName);
CancelTasksResponse response = testNodes[randomIntBetween(0, testNodes.length - 1)].transportCancelTasksAction.execute(request)
@ -527,9 +525,9 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
// Try to cancel main task using id
request = new CancelTasksRequest(testNodes[0].discoveryNode.getId());
request = new CancelTasksRequest();
request.reason("Testing Cancellation");
request.taskId(task.getId());
request.taskId(new TaskId(testNodes[0].discoveryNode.getId(), task.getId()));
response = testNodes[randomIntBetween(0, testNodes.length - 1)].transportCancelTasksAction.execute(request).get();
// Shouldn't match any tasks since testAction doesn't support cancellation
@ -601,7 +599,7 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
@Override
protected TestTaskResponse taskOperation(TestTasksRequest request, Task task) {
logger.info("Task action on node " + node);
if (failTaskOnNode == node && task.getParentNode() != null) {
if (failTaskOnNode == node && task.getParentTaskId().isSet() == false) {
logger.info("Failing on node " + node);
throw new RuntimeException("Task level failure");
}

View File

@ -185,6 +185,7 @@ public class TransportBroadcastByNodeActionTests extends ESTestCase {
clusterService = new TestClusterService(THREAD_POOL);
final TransportService transportService = new TransportService(transport, THREAD_POOL);
transportService.start();
transportService.acceptIncomingRequests();
setClusterState(clusterService, TEST_INDEX);
action = new TestTransportBroadcastByNodeAction(
Settings.EMPTY,

View File

@ -84,6 +84,7 @@ public class TransportMasterNodeActionTests extends ESTestCase {
clusterService = new TestClusterService(threadPool);
transportService = new TransportService(transport, threadPool);
transportService.start();
transportService.acceptIncomingRequests();
localNode = new DiscoveryNode("local_node", DummyTransportAddress.INSTANCE, Version.CURRENT);
remoteNode = new DiscoveryNode("remote_node", DummyTransportAddress.INSTANCE, Version.CURRENT);
allNodes = new DiscoveryNode[] { localNode, remoteNode };

View File

@ -88,6 +88,7 @@ public class BroadcastReplicationTests extends ESTestCase {
clusterService = new TestClusterService(threadPool);
transportService = new TransportService(transport, threadPool);
transportService.start();
transportService.acceptIncomingRequests();
broadcastReplicationAction = new TestBroadcastReplicationAction(Settings.EMPTY, threadPool, clusterService, transportService, new ActionFilters(new HashSet<ActionFilter>()), new IndexNameExpressionResolver(Settings.EMPTY), null);
}

View File

@ -18,8 +18,6 @@
*/
package org.elasticsearch.action.support.replication;
import com.carrotsearch.randomizedtesting.annotations.Repeat;
import org.apache.lucene.index.CorruptIndexException;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.ReplicationResponse;
@ -126,6 +124,7 @@ public class TransportReplicationActionTests extends ESTestCase {
clusterService = new TestClusterService(threadPool);
transportService = new TransportService(transport, threadPool);
transportService.start();
transportService.acceptIncomingRequests();
action = new Action(Settings.EMPTY, "testAction", transportService, clusterService, threadPool);
count.set(1);
}
@ -1016,7 +1015,7 @@ public class TransportReplicationActionTests extends ESTestCase {
* half the time.
*/
private ReplicationTask maybeTask() {
return random().nextBoolean() ? new ReplicationTask(0, null, null, null, null, 0) : null;
return random().nextBoolean() ? new ReplicationTask(0, null, null, null, null) : null;
}
/**

View File

@ -141,6 +141,7 @@ public class TransportInstanceSingleOperationActionTests extends ESTestCase {
clusterService = new TestClusterService(THREAD_POOL);
transportService = new TransportService(transport, THREAD_POOL);
transportService.start();
transportService.acceptIncomingRequests();
action = new TestTransportInstanceSingleOperationAction(
Settings.EMPTY,
"indices:admin/test",

View File

@ -73,6 +73,7 @@ public class TransportClientNodesServiceTests extends ESTestCase {
};
transportService = new TransportService(Settings.EMPTY, transport, threadPool, new NamedWriteableRegistry());
transportService.start();
transportService.acceptIncomingRequests();
transportClientNodesService = new TransportClientNodesService(Settings.EMPTY, ClusterName.DEFAULT, transportService, threadPool, Version.CURRENT);
nodesCount = randomIntBetween(1, 10);

View File

@ -107,6 +107,7 @@ public class ShardStateActionTests extends ESTestCase {
clusterService = new TestClusterService(THREAD_POOL);
transportService = new TransportService(transport, THREAD_POOL);
transportService.start();
transportService.acceptIncomingRequests();
shardStateAction = new TestShardStateAction(Settings.EMPTY, clusterService, transportService, null, null);
shardStateAction.setOnBeforeWaitForNewMasterAndRetry(() -> {});
shardStateAction.setOnAfterWaitForNewMasterAndRetry(() -> {});

View File

@ -108,6 +108,7 @@ public class ZenFaultDetectionTests extends ESTestCase {
MockTransportService transportService = new MockTransportService(Settings.EMPTY,
new LocalTransport(settings, threadPool, version, namedWriteableRegistry), threadPool, namedWriteableRegistry);
transportService.start();
transportService.acceptIncomingRequests();
return transportService;
}

View File

@ -58,12 +58,14 @@ public class UnicastZenPingIT extends ESTestCase {
NettyTransport transportA = new NettyTransport(settings, threadPool, networkService, BigArrays.NON_RECYCLING_INSTANCE, Version.CURRENT, new NamedWriteableRegistry());
final TransportService transportServiceA = new TransportService(transportA, threadPool).start();
transportServiceA.acceptIncomingRequests();
final DiscoveryNode nodeA = new DiscoveryNode("UZP_A", transportServiceA.boundAddress().publishAddress(), Version.CURRENT);
InetSocketTransportAddress addressA = (InetSocketTransportAddress) transportA.boundAddress().publishAddress();
NettyTransport transportB = new NettyTransport(settings, threadPool, networkService, BigArrays.NON_RECYCLING_INSTANCE, Version.CURRENT, new NamedWriteableRegistry());
final TransportService transportServiceB = new TransportService(transportB, threadPool).start();
transportServiceB.acceptIncomingRequests();
final DiscoveryNode nodeB = new DiscoveryNode("UZP_B", transportServiceA.boundAddress().publishAddress(), Version.CURRENT);
InetSocketTransportAddress addressB = (InetSocketTransportAddress) transportB.boundAddress().publishAddress();

View File

@ -232,6 +232,7 @@ public class PublishClusterStateActionTests extends ESTestCase {
protected MockTransportService buildTransportService(Settings settings, Version version) {
MockTransportService transportService = MockTransportService.local(Settings.EMPTY, version, threadPool);
transportService.start();
transportService.acceptIncomingRequests();
return transportService;
}

View File

@ -72,6 +72,11 @@ public class NoopDiscovery implements Discovery {
return null;
}
@Override
public void startInitialJoin() {
}
@Override
public int getMinimumMasterNodes() {
return -1;

View File

@ -40,6 +40,7 @@ import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import static java.util.Collections.emptyMap;
@ -74,11 +75,13 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
Settings.builder().put("name", "TS_A", TransportService.TRACE_LOG_INCLUDE_SETTING.getKey(), "", TransportService.TRACE_LOG_EXCLUDE_SETTING.getKey(), "NOTHING").build(),
version0, new NamedWriteableRegistry()
);
serviceA.acceptIncomingRequests();
nodeA = new DiscoveryNode("TS_A", "TS_A", serviceA.boundAddress().publishAddress(), emptyMap(), version0);
serviceB = build(
Settings.builder().put("name", "TS_B", TransportService.TRACE_LOG_INCLUDE_SETTING.getKey(), "", TransportService.TRACE_LOG_EXCLUDE_SETTING.getKey(), "NOTHING").build(),
version1, new NamedWriteableRegistry()
);
serviceB.acceptIncomingRequests();
nodeB = new DiscoveryNode("TS_B", "TS_B", serviceB.boundAddress().publishAddress(), emptyMap(), version1);
// wait till all nodes are properly connected and the event has been sent, so tests in this class
@ -1254,6 +1257,54 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
assertTrue(nodeB.address().sameHost(addressB.get()));
}
public void testBlockingIncomingRequests() throws Exception {
TransportService service = build(
Settings.builder().put("name", "TS_TEST", TransportService.TRACE_LOG_INCLUDE_SETTING.getKey(), "", TransportService.TRACE_LOG_EXCLUDE_SETTING.getKey(), "NOTHING").build(),
version0, new NamedWriteableRegistry()
);
AtomicBoolean requestProcessed = new AtomicBoolean();
service.registerRequestHandler("action", TestRequest::new, ThreadPool.Names.SAME,
(request, channel) -> {
requestProcessed.set(true);
channel.sendResponse(TransportResponse.Empty.INSTANCE);
});
DiscoveryNode node = new DiscoveryNode("TS_TEST", "TS_TEST", service.boundAddress().publishAddress(), emptyMap(), version0);
serviceA.connectToNode(node);
CountDownLatch latch = new CountDownLatch(1);
serviceA.sendRequest(node, "action", new TestRequest(), new TransportResponseHandler<TestResponse>() {
@Override
public TestResponse newInstance() {
return new TestResponse();
}
@Override
public void handleResponse(TestResponse response) {
latch.countDown();
}
@Override
public void handleException(TransportException exp) {
latch.countDown();
}
@Override
public String executor() {
return ThreadPool.Names.SAME;
}
});
assertFalse(requestProcessed.get());
service.acceptIncomingRequests();
assertBusy(() -> assertTrue(requestProcessed.get()));
latch.await();
service.close();
}
public static class TestRequest extends TransportRequest {
}

View File

@ -56,11 +56,13 @@ public class NettyScheduledPingTests extends ESTestCase {
final NettyTransport nettyA = new NettyTransport(settings, threadPool, new NetworkService(settings), BigArrays.NON_RECYCLING_INSTANCE, Version.CURRENT, registryA);
MockTransportService serviceA = new MockTransportService(settings, nettyA, threadPool, registryA);
serviceA.start();
serviceA.acceptIncomingRequests();
NamedWriteableRegistry registryB = new NamedWriteableRegistry();
final NettyTransport nettyB = new NettyTransport(settings, threadPool, new NetworkService(settings), BigArrays.NON_RECYCLING_INSTANCE, Version.CURRENT, registryB);
MockTransportService serviceB = new MockTransportService(settings, nettyB, threadPool, registryB);
serviceB.start();
serviceB.acceptIncomingRequests();
DiscoveryNode nodeA = new DiscoveryNode("TS_A", "TS_A", serviceA.boundAddress().publishAddress(), emptyMap(), Version.CURRENT);
DiscoveryNode nodeB = new DiscoveryNode("TS_B", "TS_B", serviceB.boundAddress().publishAddress(), emptyMap(), Version.CURRENT);

View File

@ -24,12 +24,12 @@ import org.elasticsearch.test.rest.parser.RestTestParseException;
import java.io.IOException;
/** Rest integration test. runs against external cluster in 'mvn verify' */
/** Rest integration test. Runs against a cluster started by {@code gradle integTest} */
public class RestIT extends ESRestTestCase {
public RestIT(RestTestCandidate testCandidate) {
super(testCandidate);
}
// we run them all sequentially: start simple!
@ParametersFactory
public static Iterable<Object[]> parameters() throws IOException, RestTestParseException {
return createParameters(0, 1);

View File

@ -24,12 +24,12 @@ import org.elasticsearch.test.rest.parser.RestTestParseException;
import java.io.IOException;
/** Rest integration test. runs against external cluster in 'mvn verify' */
/** Rest integration test. Runs against a cluster started by {@code gradle integTest} */
public class RestIT extends ESRestTestCase {
public RestIT(RestTestCandidate testCandidate) {
super(testCandidate);
}
// we run them all sequentially: start simple!
@ParametersFactory
public static Iterable<Object[]> parameters() throws IOException, RestTestParseException {
return createParameters(0, 1);

View File

@ -24,12 +24,12 @@ import org.elasticsearch.test.rest.parser.RestTestParseException;
import java.io.IOException;
/** Rest integration test. runs against external cluster in 'mvn verify' */
/** Rest integration test. Runs against a cluster started by {@code gradle integTest} */
public class RestIT extends ESRestTestCase {
public RestIT(RestTestCandidate testCandidate) {
super(testCandidate);
}
// we run them all sequentially: start simple!
@ParametersFactory
public static Iterable<Object[]> parameters() throws IOException, RestTestParseException {
return createParameters(0, 1);

View File

@ -24,12 +24,12 @@ import org.elasticsearch.test.rest.parser.RestTestParseException;
import java.io.IOException;
/** Rest integration test. runs against external cluster in 'mvn verify' */
/** Rest integration test. Runs against a cluster started by {@code gradle integTest} */
public class RestIT extends ESRestTestCase {
public RestIT(RestTestCandidate testCandidate) {
super(testCandidate);
}
// we run them all sequentially: start simple!
@ParametersFactory
public static Iterable<Object[]> parameters() throws IOException, RestTestParseException {
return createParameters(0, 1);

View File

@ -24,12 +24,12 @@ import org.elasticsearch.test.rest.parser.RestTestParseException;
import java.io.IOException;
/** Rest integration test. runs against external cluster in 'mvn verify' */
/** Rest integration test. Runs against a cluster started by {@code gradle integTest} */
public class RestIT extends ESRestTestCase {
public RestIT(RestTestCandidate testCandidate) {
super(testCandidate);
}
// we run them all sequentially: start simple!
@ParametersFactory
public static Iterable<Object[]> parameters() throws IOException, RestTestParseException {
return createParameters(0, 1);

View File

@ -102,6 +102,7 @@ descriptors percentage |1
|`file_desc.max` |`fdm`, `fileDescriptorMax` |No |Maximum number of file
descriptors |1024
|`load` |`l` |No |Most recent load average |0.22
|`cpu` | |No |Recent system CPU usage as percent |12
|`uptime` |`u` |No |Node uptime |17.3m
|`node.role` |`r`, `role`, `dc`, `nodeRole` |Yes |Data node (d); Client
node (c) |d

View File

@ -323,6 +323,13 @@ hatch of setting the system property "es.netty.gathering" to "false".
Time has proven enabling gathering by default is a non-issue and this
non-documented setting has been removed.
==== Removed es.useLinkedTransferQueue
The system property `es.useLinkedTransferQueue` could be used to
control the queue implementation used in the cluster service and the
handling of ping responses during discovery. This was an undocumented
setting and has been removed.
[[breaking_30_mapping_changes]]
=== Mapping changes

View File

@ -43,13 +43,14 @@ import java.util.Locale;
public class TestIndexableBinaryStringTools extends LuceneTestCase {
private static int NUM_RANDOM_TESTS;
private static int MAX_RANDOM_BINARY_LENGTH;
private static final String LINE_SEPARATOR = System.lineSeparator();
@BeforeClass
public static void beforeClass() throws Exception {
NUM_RANDOM_TESTS = atLeast(200);
MAX_RANDOM_BINARY_LENGTH = atLeast(300);
}
public void testSingleBinaryRoundTrip() {
byte[] binary = new byte[] { (byte) 0x23, (byte) 0x98, (byte) 0x13,
(byte) 0xE4, (byte) 0x76, (byte) 0x41, (byte) 0xB2, (byte) 0xC9,
@ -68,15 +69,15 @@ public class TestIndexableBinaryStringTools extends LuceneTestCase {
decoded.length);
assertEquals("Round trip decode/decode returned different results:"
+ System.getProperty("line.separator") + "original: "
+ LINE_SEPARATOR + "original: "
+ binaryDump(binary, binary.length)
+ System.getProperty("line.separator") + " encoded: "
+ LINE_SEPARATOR + " encoded: "
+ charArrayDump(encoded, encoded.length)
+ System.getProperty("line.separator") + " decoded: "
+ LINE_SEPARATOR + " decoded: "
+ binaryDump(decoded, decoded.length),
binaryDump(binary, binary.length), binaryDump(decoded, decoded.length));
}
public void testEncodedSortability() {
byte[] originalArray1 = new byte[MAX_RANDOM_BINARY_LENGTH];
char[] originalString1 = new char[MAX_RANDOM_BINARY_LENGTH];
@ -127,15 +128,15 @@ public class TestIndexableBinaryStringTools extends LuceneTestCase {
assertEquals("Test #" + (testNum + 1)
+ ": Original bytes and encoded chars compare differently:"
+ System.getProperty("line.separator") + " binary 1: "
+ LINE_SEPARATOR + " binary 1: "
+ binaryDump(originalArray1, numBytes1)
+ System.getProperty("line.separator") + " binary 2: "
+ LINE_SEPARATOR + " binary 2: "
+ binaryDump(original2, numBytes2)
+ System.getProperty("line.separator") + "encoded 1: "
+ LINE_SEPARATOR + "encoded 1: "
+ charArrayDump(encoded1, encodedLen1)
+ System.getProperty("line.separator") + "encoded 2: "
+ LINE_SEPARATOR + "encoded 2: "
+ charArrayDump(encoded2, encodedLen2)
+ System.getProperty("line.separator"), originalComparison,
+ LINE_SEPARATOR, originalComparison,
encodedComparison);
}
}
@ -157,7 +158,7 @@ public class TestIndexableBinaryStringTools extends LuceneTestCase {
assertEquals("decoded empty input was not empty", decoded.length, 0);
}
public void testAllNullInput() {
byte[] binary = new byte[] { 0, 0, 0, 0, 0, 0, 0, 0, 0 };
@ -174,19 +175,19 @@ public class TestIndexableBinaryStringTools extends LuceneTestCase {
decoded.length);
assertEquals("Round trip decode/decode returned different results:"
+ System.getProperty("line.separator") + " original: "
+ LINE_SEPARATOR + " original: "
+ binaryDump(binary, binary.length)
+ System.getProperty("line.separator") + "decodedBuf: "
+ LINE_SEPARATOR + "decodedBuf: "
+ binaryDump(decoded, decoded.length),
binaryDump(binary, binary.length), binaryDump(decoded, decoded.length));
}
public void testRandomBinaryRoundTrip() {
byte[] binary = new byte[MAX_RANDOM_BINARY_LENGTH];
char[] encoded = new char[MAX_RANDOM_BINARY_LENGTH * 10];
byte[] decoded = new byte[MAX_RANDOM_BINARY_LENGTH];
for (int testNum = 0; testNum < NUM_RANDOM_TESTS; ++testNum) {
int numBytes = random().nextInt(MAX_RANDOM_BINARY_LENGTH - 1) + 1; // Min == 1
int numBytes = random().nextInt(MAX_RANDOM_BINARY_LENGTH - 1) + 1; // Min == 1
for (int byteNum = 0; byteNum < numBytes; ++byteNum) {
binary[byteNum] = (byte) random().nextInt(0x100);
@ -206,15 +207,15 @@ public class TestIndexableBinaryStringTools extends LuceneTestCase {
assertEquals("Test #" + (testNum + 1)
+ ": Round trip decode/decode returned different results:"
+ System.getProperty("line.separator") + " original: "
+ binaryDump(binary, numBytes) + System.getProperty("line.separator")
+ LINE_SEPARATOR + " original: "
+ binaryDump(binary, numBytes) + LINE_SEPARATOR
+ "encodedBuf: " + charArrayDump(encoded, encodedLen)
+ System.getProperty("line.separator") + "decodedBuf: "
+ LINE_SEPARATOR + "decodedBuf: "
+ binaryDump(decoded, decodedLen), binaryDump(binary, numBytes),
binaryDump(decoded, decodedLen));
}
}
public String binaryDump(byte[] binary, int numBytes) {
StringBuilder buf = new StringBuilder();
for (int byteNum = 0 ; byteNum < numBytes ; ++byteNum) {

View File

@ -48,17 +48,14 @@ import static com.carrotsearch.randomizedtesting.RandomizedTest.randomAsciiOfLen
import static org.hamcrest.Matchers.notNullValue;
/**
* {@link ESSmokeClientTestCase} is an abstract base class to run integration
* tests against an external Elasticsearch Cluster.
* An abstract base class to run integration tests against an Elasticsearch cluster running outside of the test process.
* <p>
* You can define a list of transport addresses from where you can reach your cluster
* by setting "tests.cluster" system property. It defaults to "localhost:9300".
* You can define a list of transport addresses from where you can reach your cluster by setting "tests.cluster" system
* property. It defaults to "localhost:9300". If you run this from `gradle integTest` then it will start the clsuter for
* you and set up the property.
* <p>
* All tests can be run from maven using mvn install as maven will start an external cluster first.
* <p>
* If you want to debug this module from your IDE, then start an external cluster by yourself
* then run JUnit. If you changed the default port, set "tests.cluster=localhost:PORT" when running
* your test.
* If you want to debug this module from your IDE, then start an external cluster by yourself, maybe with `gradle run`,
* then run JUnit. If you changed the default port, set "-Dtests.cluster=localhost:PORT" when running your test.
*/
@LuceneTestCase.SuppressSysoutChecks(bugUrl = "we log a lot on purpose")
public abstract class ESSmokeClientTestCase extends LuceneTestCase {

View File

@ -4,18 +4,18 @@
"methods": ["POST"],
"url": {
"path": "/_tasks",
"paths": ["/_tasks/_cancel", "/_tasks/{node_id}/_cancel", "/_tasks/{node_id}/{task_id}/_cancel"],
"paths": ["/_tasks/_cancel", "/_tasks/{task_id}/_cancel"],
"parts": {
"node_id": {
"type": "list",
"description": "A comma-separated list of node IDs or names to limit the request; use `_local` to cancel only tasks on the node you're connecting to, leave empty to cancel tasks on all nodes"
},
"task_id": {
"type": "number",
"description": "Cancel the task with specified id"
}
},
"params": {
"node_id": {
"type": "list",
"description": "A comma-separated list of node IDs or names to limit the returned information; use `_local` to return information from the node you're connecting to, leave empty to get information from all nodes"
},
"actions": {
"type": "list",
"description": "A comma-separated list of actions that should be cancelled. Leave empty to cancel all."

View File

@ -4,18 +4,18 @@
"methods": ["GET"],
"url": {
"path": "/_tasks",
"paths": ["/_tasks", "/_tasks/{node_id}", "/_tasks/{node_id}/{task_id}"],
"paths": ["/_tasks", "/_tasks/{task_id}"],
"parts": {
"node_id": {
"type": "list",
"description": "A comma-separated list of node IDs or names to limit the returned information; use `_local` to return information from the node you're connecting to, leave empty to get information from all nodes"
},
"task_id": {
"type": "number",
"description": "Return the task with specified id"
}
},
"params": {
"node_id": {
"type": "list",
"description": "A comma-separated list of node IDs or names to limit the returned information; use `_local` to return information from the node you're connecting to, leave empty to get information from all nodes"
},
"actions": {
"type": "list",
"description": "A comma-separated list of actions that should be returned. Leave empty to return all."

View File

@ -2,7 +2,6 @@
"tasks_cancel test":
- do:
tasks.cancel:
node_id: _local
task_id: 1
actions: "unknown_action"
- length: { nodes: 0 }

View File

@ -122,7 +122,7 @@ public class BootstrapForTesting {
}
// intellij hack: intellij test runner wants setIO and will
// screw up all test logging without it!
if (System.getProperty("tests.maven") == null) {
if (System.getProperty("tests.gradle") == null) {
perms.add(new RuntimePermission("setIO"));
}

View File

@ -42,8 +42,8 @@ import static org.elasticsearch.test.rest.ESRestTestCase.REST_TESTS_SPEC;
import static org.elasticsearch.test.rest.ESRestTestCase.REST_TESTS_SUITE;
/**
* A {@link RunListener} that emits to {@link System#err} a string with command
* line parameters allowing quick test re-run under MVN command line.
* A {@link RunListener} that emits a command you can use to re-run a failing test with the failing random seed to
* {@link System#err}.
*/
public class ReproduceInfoPrinter extends RunListener {
@ -60,7 +60,7 @@ public class ReproduceInfoPrinter extends RunListener {
}
/**
* true if we are running maven integration tests (mvn verify)
* Are we in the integ test phase?
*/
static boolean inVerifyPhase() {
return Boolean.parseBoolean(System.getProperty("tests.verify.phase"));
@ -75,7 +75,7 @@ public class ReproduceInfoPrinter extends RunListener {
final StringBuilder b = new StringBuilder("REPRODUCE WITH: gradle ");
String task = System.getProperty("tests.task");
// TODO: enforce (intellij still runs the runner?) or use default "test" but that wont' work for integ
// TODO: enforce (intellij still runs the runner?) or use default "test" but that won't work for integ
b.append(task);
GradleMessageBuilder gradleMessageBuilder = new GradleMessageBuilder(b);
@ -140,7 +140,8 @@ public class ReproduceInfoPrinter extends RunListener {
appendProperties("es.logger.level");
if (inVerifyPhase()) {
// these properties only make sense for integration tests
appendProperties("es.node.mode", "es.node.local", TESTS_CLUSTER, ESIntegTestCase.TESTS_ENABLE_MOCK_MODULES);
appendProperties("es.node.mode", "es.node.local", TESTS_CLUSTER,
ESIntegTestCase.TESTS_ENABLE_MOCK_MODULES);
}
appendProperties("tests.assertion.disabled", "tests.security.manager", "tests.nightly", "tests.jvms",
"tests.client.ratio", "tests.heap.size", "tests.bwc", "tests.bwc.version");

View File

@ -34,8 +34,6 @@ import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.PathUtils;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.network.InetAddresses;
import org.elasticsearch.common.network.NetworkAddress;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.ThreadContext;
import org.elasticsearch.common.util.set.Sets;
@ -48,7 +46,6 @@ import javax.net.ssl.SSLContext;
import java.io.Closeable;
import java.io.IOException;
import java.io.InputStream;
import java.net.InetSocketAddress;
import java.net.URL;
import java.nio.file.Files;
import java.nio.file.Path;
@ -284,7 +281,7 @@ public class RestClient implements Closeable {
SSLContext sslcontext = SSLContexts.custom()
.loadTrustMaterial(keyStore, null)
.build();
sslsf = new SSLConnectionSocketFactory(sslcontext);
sslsf = new SSLConnectionSocketFactory(sslcontext, StrictHostnameVerifier.INSTANCE);
} catch (KeyStoreException|NoSuchAlgorithmException|KeyManagementException|CertificateException e) {
throw new RuntimeException(e);
}

View File

@ -0,0 +1,76 @@
/*
* 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.test.rest.client;
import org.apache.http.conn.ssl.X509HostnameVerifier;
import org.apache.http.conn.util.InetAddressUtils;
import javax.net.ssl.SSLException;
import javax.net.ssl.SSLSession;
import javax.net.ssl.SSLSocket;
import java.io.IOException;
import java.security.cert.X509Certificate;
/**
* A custom {@link X509HostnameVerifier} implementation that wraps calls to the {@link org.apache.http.conn.ssl.StrictHostnameVerifier} and
* properly handles IPv6 addresses that come from a URL in the form <code>http://[::1]:9200/</code> by removing the surrounding brackets.
*
* This is a variation of the fix for <a href="https://issues.apache.org/jira/browse/HTTPCLIENT-1698">HTTPCLIENT-1698</a>, which is not
* released yet as of Apache HttpClient 4.5.1
*/
final class StrictHostnameVerifier implements X509HostnameVerifier {
static final StrictHostnameVerifier INSTANCE = new StrictHostnameVerifier();
// We need to wrap the default verifier for HttpClient since we use an older version and the following issue is not
// fixed in a released version yet https://issues.apache.org/jira/browse/HTTPCLIENT-1698
// TL;DR we need to strip '[' and ']' from IPv6 addresses if they come from a URL
private final X509HostnameVerifier verifier = new org.apache.http.conn.ssl.StrictHostnameVerifier();
private StrictHostnameVerifier() {}
@Override
public boolean verify(String host, SSLSession sslSession) {
return verifier.verify(stripBracketsIfNecessary(host), sslSession);
}
@Override
public void verify(String host, SSLSocket ssl) throws IOException {
verifier.verify(stripBracketsIfNecessary(host), ssl);
}
@Override
public void verify(String host, X509Certificate cert) throws SSLException {
verifier.verify(stripBracketsIfNecessary(host), cert);
}
@Override
public void verify(String host, String[] cns, String[] subjectAlts) throws SSLException {
verifier.verify(stripBracketsIfNecessary(host), cns, subjectAlts);
}
private String stripBracketsIfNecessary(String host) {
if (host.startsWith("[") && host.endsWith("]")) {
String newHost = host.substring(1, host.length() - 1);
assert InetAddressUtils.isIPv6Address(newHost);
return newHost;
}
return host;
}
}

View File

@ -0,0 +1,120 @@
/*
* 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.test.rest.client;
import org.elasticsearch.test.ESTestCase;
import org.junit.Before;
import javax.net.ssl.SSLSession;
import javax.net.ssl.SSLSocket;
import javax.security.auth.x500.X500Principal;
import java.security.cert.Certificate;
import java.security.cert.X509Certificate;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
/**
* Tests for the {@link StrictHostnameVerifier} to validate that it can verify IPv6 addresses with and without bracket notation, in
* addition to other address types.
*/
public class StrictHostnameVerifierTests extends ESTestCase {
private static final int IP_SAN_TYPE = 7;
private static final int DNS_SAN_TYPE = 2;
private static final String[] CNS = new String[] { "my node" };
private static final String[] IP_SANS = new String[] { "127.0.0.1", "192.168.1.1", "::1" };
private static final String[] DNS_SANS = new String[] { "localhost", "computer", "localhost6" };
private SSLSocket sslSocket;
private SSLSession sslSession;
private X509Certificate certificate;
@Before
public void setupMocks() throws Exception {
sslSocket = mock(SSLSocket.class);
sslSession = mock(SSLSession.class);
certificate = mock(X509Certificate.class);
Collection<List<?>> subjectAlternativeNames = new ArrayList<>();
for (String san : IP_SANS) {
subjectAlternativeNames.add(Arrays.asList(IP_SAN_TYPE, san));
}
for (String san : DNS_SANS) {
subjectAlternativeNames.add(Arrays.asList(DNS_SAN_TYPE, san));
}
when(sslSocket.getSession()).thenReturn(sslSession);
when(sslSession.getPeerCertificates()).thenReturn(new Certificate[] { certificate });
when(certificate.getSubjectX500Principal()).thenReturn(new X500Principal("CN=" + CNS[0]));
when(certificate.getSubjectAlternativeNames()).thenReturn(subjectAlternativeNames);
}
public void testThatIPv6WithBracketsWorks() throws Exception {
final String ipv6Host = "[::1]";
// an exception will be thrown if verification fails
StrictHostnameVerifier.INSTANCE.verify(ipv6Host, CNS, IP_SANS);
StrictHostnameVerifier.INSTANCE.verify(ipv6Host, sslSocket);
StrictHostnameVerifier.INSTANCE.verify(ipv6Host, certificate);
// this is the only one we can assert on
assertTrue(StrictHostnameVerifier.INSTANCE.verify(ipv6Host, sslSession));
}
public void testThatIPV6WithoutBracketWorks() throws Exception {
final String ipv6Host = "::1";
// an exception will be thrown if verification fails
StrictHostnameVerifier.INSTANCE.verify(ipv6Host, CNS, IP_SANS);
StrictHostnameVerifier.INSTANCE.verify(ipv6Host, sslSocket);
StrictHostnameVerifier.INSTANCE.verify(ipv6Host, certificate);
// this is the only one we can assert on
assertTrue(StrictHostnameVerifier.INSTANCE.verify(ipv6Host, sslSession));
}
public void testThatIPV4Works() throws Exception {
final String ipv4Host = randomFrom("127.0.0.1", "192.168.1.1");
// an exception will be thrown if verification fails
StrictHostnameVerifier.INSTANCE.verify(ipv4Host, CNS, IP_SANS);
StrictHostnameVerifier.INSTANCE.verify(ipv4Host, sslSocket);
StrictHostnameVerifier.INSTANCE.verify(ipv4Host, certificate);
// this is the only one we can assert on
assertTrue(StrictHostnameVerifier.INSTANCE.verify(ipv4Host, sslSession));
}
public void testThatHostnameWorks() throws Exception {
final String host = randomFrom(DNS_SANS);
// an exception will be thrown if verification fails
StrictHostnameVerifier.INSTANCE.verify(host, CNS, DNS_SANS);
StrictHostnameVerifier.INSTANCE.verify(host, sslSocket);
StrictHostnameVerifier.INSTANCE.verify(host, certificate);
// this is the only one we can assert on
assertTrue(StrictHostnameVerifier.INSTANCE.verify(host, sslSession));
}
}