Convert several classes in 'server' to Writeable. (#44527)
* Convert FieldCapabilities*. * Convert MultiTermVectors*. * Convert SyncedFlush*. * Convert SearchTemplateRequest. * Convert MultiSearchTemplateRequest. * Convert GrokProcessorGet*. * Remove a stray reference to SearchTemplateRequest#readFrom. Relates to #34389.
This commit is contained in:
parent
2a2686e6e7
commit
34c6067018
|
@ -22,7 +22,7 @@ import org.elasticsearch.action.ActionListener;
|
|||
import org.elasticsearch.action.ActionRequest;
|
||||
import org.elasticsearch.action.ActionRequestValidationException;
|
||||
import org.elasticsearch.action.ActionResponse;
|
||||
import org.elasticsearch.action.StreamableResponseActionType;
|
||||
import org.elasticsearch.action.ActionType;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.HandledTransportAction;
|
||||
import org.elasticsearch.client.node.NodeClient;
|
||||
|
@ -45,18 +45,13 @@ import java.util.Map;
|
|||
import static org.elasticsearch.ingest.common.IngestCommonPlugin.GROK_PATTERNS;
|
||||
import static org.elasticsearch.rest.RestRequest.Method.GET;
|
||||
|
||||
public class GrokProcessorGetAction extends StreamableResponseActionType<GrokProcessorGetAction.Response> {
|
||||
public class GrokProcessorGetAction extends ActionType<GrokProcessorGetAction.Response> {
|
||||
|
||||
static final GrokProcessorGetAction INSTANCE = new GrokProcessorGetAction();
|
||||
static final String NAME = "cluster:admin/ingest/processor/grok/get";
|
||||
|
||||
private GrokProcessorGetAction() {
|
||||
super(NAME);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Response newResponse() {
|
||||
return new Response(null);
|
||||
super(NAME, Response::new);
|
||||
}
|
||||
|
||||
public static class Request extends ActionRequest {
|
||||
|
@ -64,15 +59,26 @@ public class GrokProcessorGetAction extends StreamableResponseActionType<GrokPro
|
|||
public ActionRequestValidationException validate() {
|
||||
return null;
|
||||
}
|
||||
|
||||
Request() {}
|
||||
|
||||
Request(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
||||
public static class Response extends ActionResponse implements ToXContentObject {
|
||||
private Map<String, String> grokPatterns;
|
||||
private final Map<String, String> grokPatterns;
|
||||
|
||||
Response(Map<String, String> grokPatterns) {
|
||||
this.grokPatterns = grokPatterns;
|
||||
}
|
||||
|
||||
Response(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
grokPatterns = in.readMap(StreamInput::readString, StreamInput::readString);
|
||||
}
|
||||
|
||||
public Map<String, String> getGrokPatterns() {
|
||||
return grokPatterns;
|
||||
}
|
||||
|
@ -87,9 +93,8 @@ public class GrokProcessorGetAction extends StreamableResponseActionType<GrokPro
|
|||
}
|
||||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
super.readFrom(in);
|
||||
grokPatterns = in.readMap(StreamInput::readString, StreamInput::readString);
|
||||
public void readFrom(StreamInput in) {
|
||||
throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -102,7 +107,7 @@ public class GrokProcessorGetAction extends StreamableResponseActionType<GrokPro
|
|||
|
||||
@Inject
|
||||
public TransportAction(TransportService transportService, ActionFilters actionFilters) {
|
||||
super(NAME, transportService, Request::new, actionFilters);
|
||||
super(NAME, transportService, actionFilters, Request::new);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -34,7 +34,6 @@ import java.util.Map;
|
|||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.core.IsNull.nullValue;
|
||||
|
||||
|
||||
public class GrokProcessorGetActionTests extends ESTestCase {
|
||||
private static final Map<String, String> TEST_PATTERNS = Collections.singletonMap("PATTERN", "foo");
|
||||
|
||||
|
@ -53,8 +52,7 @@ public class GrokProcessorGetActionTests extends ESTestCase {
|
|||
BytesStreamOutput out = new BytesStreamOutput();
|
||||
response.writeTo(out);
|
||||
StreamInput streamInput = out.bytes().streamInput();
|
||||
GrokProcessorGetAction.Response otherResponse = new GrokProcessorGetAction.Response(null);
|
||||
otherResponse.readFrom(streamInput);
|
||||
GrokProcessorGetAction.Response otherResponse = new GrokProcessorGetAction.Response(streamInput);
|
||||
assertThat(response.getGrokPatterns(), equalTo(TEST_PATTERNS));
|
||||
assertThat(response.getGrokPatterns(), equalTo(otherResponse.getGrokPatterns()));
|
||||
}
|
||||
|
|
|
@ -47,6 +47,14 @@ public class MultiSearchTemplateRequest extends ActionRequest implements Composi
|
|||
|
||||
private IndicesOptions indicesOptions = IndicesOptions.strictExpandOpenAndForbidClosedIgnoreThrottled();
|
||||
|
||||
public MultiSearchTemplateRequest() {}
|
||||
|
||||
public MultiSearchTemplateRequest(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
maxConcurrentSearchRequests = in.readVInt();
|
||||
requests = in.readList(SearchTemplateRequest::new);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a search template request to execute. Note, the order is important, the search response will be returned in the
|
||||
* same order as the search requests.
|
||||
|
@ -116,10 +124,8 @@ public class MultiSearchTemplateRequest extends ActionRequest implements Composi
|
|||
}
|
||||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
super.readFrom(in);
|
||||
maxConcurrentSearchRequests = in.readVInt();
|
||||
requests = in.readStreamableList(SearchTemplateRequest::new);
|
||||
public void readFrom(StreamInput in) {
|
||||
throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -54,13 +54,25 @@ public class SearchTemplateRequest extends ActionRequest implements CompositeInd
|
|||
private String script;
|
||||
private Map<String, Object> scriptParams;
|
||||
|
||||
public SearchTemplateRequest() {
|
||||
}
|
||||
public SearchTemplateRequest() {}
|
||||
|
||||
public SearchTemplateRequest(SearchRequest searchRequest) {
|
||||
this.request = searchRequest;
|
||||
}
|
||||
|
||||
public SearchTemplateRequest(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
request = in.readOptionalWriteable(SearchRequest::new);
|
||||
simulate = in.readBoolean();
|
||||
explain = in.readBoolean();
|
||||
profile = in.readBoolean();
|
||||
scriptType = ScriptType.readFrom(in);
|
||||
script = in.readOptionalString();
|
||||
if (in.readBoolean()) {
|
||||
scriptParams = in.readMap();
|
||||
}
|
||||
}
|
||||
|
||||
public void setRequest(SearchRequest request) {
|
||||
this.request = request;
|
||||
}
|
||||
|
@ -218,17 +230,8 @@ public class SearchTemplateRequest extends ActionRequest implements CompositeInd
|
|||
}
|
||||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
super.readFrom(in);
|
||||
request = in.readOptionalWriteable(SearchRequest::new);
|
||||
simulate = in.readBoolean();
|
||||
explain = in.readBoolean();
|
||||
profile = in.readBoolean();
|
||||
scriptType = ScriptType.readFrom(in);
|
||||
script = in.readOptionalString();
|
||||
if (in.readBoolean()) {
|
||||
scriptParams = in.readMap();
|
||||
}
|
||||
public void readFrom(StreamInput in) {
|
||||
throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -46,7 +46,7 @@ public class TransportMultiSearchTemplateAction extends HandledTransportAction<M
|
|||
@Inject
|
||||
public TransportMultiSearchTemplateAction(TransportService transportService, ActionFilters actionFilters, ScriptService scriptService,
|
||||
NamedXContentRegistry xContentRegistry, NodeClient client) {
|
||||
super(MultiSearchTemplateAction.NAME, transportService, MultiSearchTemplateRequest::new, actionFilters);
|
||||
super(MultiSearchTemplateAction.NAME, transportService, actionFilters, MultiSearchTemplateRequest::new);
|
||||
this.scriptService = scriptService;
|
||||
this.xContentRegistry = xContentRegistry;
|
||||
this.client = client;
|
||||
|
|
|
@ -54,7 +54,7 @@ public class TransportSearchTemplateAction extends HandledTransportAction<Search
|
|||
@Inject
|
||||
public TransportSearchTemplateAction(TransportService transportService, ActionFilters actionFilters,
|
||||
ScriptService scriptService, NamedXContentRegistry xContentRegistry, NodeClient client) {
|
||||
super(SearchTemplateAction.NAME, transportService, SearchTemplateRequest::new, actionFilters);
|
||||
super(SearchTemplateAction.NAME, transportService, actionFilters, SearchTemplateRequest::new);
|
||||
this.scriptService = scriptService;
|
||||
this.xContentRegistry = xContentRegistry;
|
||||
this.client = client;
|
||||
|
|
|
@ -19,10 +19,11 @@
|
|||
|
||||
package org.elasticsearch.script.mustache;
|
||||
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.script.ScriptType;
|
||||
import org.elasticsearch.search.RandomSearchRequestGenerator;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.test.AbstractStreamableTestCase;
|
||||
import org.elasticsearch.test.AbstractWireSerializingTestCase;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -31,18 +32,18 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.function.Consumer;
|
||||
|
||||
public class SearchTemplateRequestTests extends AbstractStreamableTestCase<SearchTemplateRequest> {
|
||||
|
||||
@Override
|
||||
protected SearchTemplateRequest createBlankInstance() {
|
||||
return new SearchTemplateRequest();
|
||||
}
|
||||
public class SearchTemplateRequestTests extends AbstractWireSerializingTestCase<SearchTemplateRequest> {
|
||||
|
||||
@Override
|
||||
protected SearchTemplateRequest createTestInstance() {
|
||||
return createRandomRequest();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Writeable.Reader<SearchTemplateRequest> instanceReader() {
|
||||
return SearchTemplateRequest::new;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected SearchTemplateRequest mutateInstance(SearchTemplateRequest instance) throws IOException {
|
||||
List<Consumer<SearchTemplateRequest>> mutators = new ArrayList<>();
|
||||
|
|
|
@ -19,20 +19,15 @@
|
|||
|
||||
package org.elasticsearch.action.admin.indices.flush;
|
||||
|
||||
import org.elasticsearch.action.StreamableResponseActionType;
|
||||
import org.elasticsearch.action.ActionType;
|
||||
|
||||
|
||||
public class SyncedFlushAction extends StreamableResponseActionType<SyncedFlushResponse> {
|
||||
public class SyncedFlushAction extends ActionType<SyncedFlushResponse> {
|
||||
|
||||
public static final SyncedFlushAction INSTANCE = new SyncedFlushAction();
|
||||
public static final String NAME = "indices:admin/synced_flush";
|
||||
|
||||
private SyncedFlushAction() {
|
||||
super(NAME);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SyncedFlushResponse newResponse() {
|
||||
return new SyncedFlushResponse();
|
||||
super(NAME, SyncedFlushResponse::new);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,7 +22,7 @@ import org.elasticsearch.action.ActionResponse;
|
|||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.io.stream.Streamable;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.common.util.iterable.Iterables;
|
||||
import org.elasticsearch.common.xcontent.ToXContentFragment;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
|
@ -44,12 +44,8 @@ import static java.util.Collections.unmodifiableMap;
|
|||
*/
|
||||
public class SyncedFlushResponse extends ActionResponse implements ToXContentFragment {
|
||||
|
||||
Map<String, List<ShardsSyncedFlushResult>> shardsResultPerIndex;
|
||||
ShardCounts shardCounts;
|
||||
|
||||
SyncedFlushResponse() {
|
||||
|
||||
}
|
||||
private final Map<String, List<ShardsSyncedFlushResult>> shardsResultPerIndex;
|
||||
private final ShardCounts shardCounts;
|
||||
|
||||
public SyncedFlushResponse(Map<String, List<ShardsSyncedFlushResult>> shardsResultPerIndex) {
|
||||
// shardsResultPerIndex is never modified after it is passed to this
|
||||
|
@ -59,6 +55,23 @@ public class SyncedFlushResponse extends ActionResponse implements ToXContentFra
|
|||
this.shardCounts = calculateShardCounts(Iterables.flatten(shardsResultPerIndex.values()));
|
||||
}
|
||||
|
||||
public SyncedFlushResponse(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
shardCounts = new ShardCounts(in);
|
||||
Map<String, List<ShardsSyncedFlushResult>> tmpShardsResultPerIndex = new HashMap<>();
|
||||
int numShardsResults = in.readInt();
|
||||
for (int i =0 ; i< numShardsResults; i++) {
|
||||
String index = in.readString();
|
||||
List<ShardsSyncedFlushResult> shardsSyncedFlushResults = new ArrayList<>();
|
||||
int numShards = in.readInt();
|
||||
for (int j =0; j< numShards; j++) {
|
||||
shardsSyncedFlushResults.add(ShardsSyncedFlushResult.readShardsSyncedFlushResult(in));
|
||||
}
|
||||
tmpShardsResultPerIndex.put(index, shardsSyncedFlushResults);
|
||||
}
|
||||
shardsResultPerIndex = Collections.unmodifiableMap(tmpShardsResultPerIndex);
|
||||
}
|
||||
|
||||
/**
|
||||
* total number shards, including replicas, both assigned and unassigned
|
||||
*/
|
||||
|
@ -140,11 +153,11 @@ public class SyncedFlushResponse extends ActionResponse implements ToXContentFra
|
|||
return new ShardCounts(total, successful, failed);
|
||||
}
|
||||
|
||||
static final class ShardCounts implements ToXContentFragment, Streamable {
|
||||
static final class ShardCounts implements ToXContentFragment, Writeable {
|
||||
|
||||
public int total;
|
||||
public int successful;
|
||||
public int failed;
|
||||
public final int total;
|
||||
public final int successful;
|
||||
public final int failed;
|
||||
|
||||
ShardCounts(int total, int successful, int failed) {
|
||||
this.total = total;
|
||||
|
@ -152,8 +165,10 @@ public class SyncedFlushResponse extends ActionResponse implements ToXContentFra
|
|||
this.failed = failed;
|
||||
}
|
||||
|
||||
ShardCounts() {
|
||||
|
||||
ShardCounts(StreamInput in) throws IOException {
|
||||
total = in.readInt();
|
||||
successful = in.readInt();
|
||||
failed = in.readInt();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -164,13 +179,6 @@ public class SyncedFlushResponse extends ActionResponse implements ToXContentFra
|
|||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
total = in.readInt();
|
||||
successful = in.readInt();
|
||||
failed = in.readInt();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
out.writeInt(total);
|
||||
|
@ -192,21 +200,7 @@ public class SyncedFlushResponse extends ActionResponse implements ToXContentFra
|
|||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
super.readFrom(in);
|
||||
shardCounts = new ShardCounts();
|
||||
shardCounts.readFrom(in);
|
||||
Map<String, List<ShardsSyncedFlushResult>> tmpShardsResultPerIndex = new HashMap<>();
|
||||
int numShardsResults = in.readInt();
|
||||
for (int i =0 ; i< numShardsResults; i++) {
|
||||
String index = in.readString();
|
||||
List<ShardsSyncedFlushResult> shardsSyncedFlushResults = new ArrayList<>();
|
||||
int numShards = in.readInt();
|
||||
for (int j =0; j< numShards; j++) {
|
||||
shardsSyncedFlushResults.add(ShardsSyncedFlushResult.readShardsSyncedFlushResult(in));
|
||||
}
|
||||
tmpShardsResultPerIndex.put(index, shardsSyncedFlushResults);
|
||||
}
|
||||
shardsResultPerIndex = Collections.unmodifiableMap(tmpShardsResultPerIndex);
|
||||
throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -19,19 +19,15 @@
|
|||
|
||||
package org.elasticsearch.action.fieldcaps;
|
||||
|
||||
import org.elasticsearch.action.StreamableResponseActionType;
|
||||
import org.elasticsearch.action.ActionType;
|
||||
|
||||
public class FieldCapabilitiesAction extends StreamableResponseActionType<FieldCapabilitiesResponse> {
|
||||
public class FieldCapabilitiesAction extends ActionType<FieldCapabilitiesResponse> {
|
||||
|
||||
public static final FieldCapabilitiesAction INSTANCE = new FieldCapabilitiesAction();
|
||||
public static final String NAME = "indices:data/read/field_caps";
|
||||
|
||||
private FieldCapabilitiesAction() {
|
||||
super(NAME);
|
||||
super(NAME, FieldCapabilitiesResponse::new);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldCapabilitiesResponse newResponse() {
|
||||
return new FieldCapabilitiesResponse();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -56,6 +56,19 @@ public final class FieldCapabilitiesRequest extends ActionRequest implements Ind
|
|||
PARSER.declareStringArray(fromList(String.class, FieldCapabilitiesRequest::fields), FIELDS_FIELD);
|
||||
}
|
||||
|
||||
public FieldCapabilitiesRequest(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
fields = in.readStringArray();
|
||||
indices = in.readStringArray();
|
||||
indicesOptions = IndicesOptions.readIndicesOptions(in);
|
||||
mergeResults = in.readBoolean();
|
||||
if (in.getVersion().onOrAfter(Version.V_7_2_0)) {
|
||||
includeUnmapped = in.readBoolean();
|
||||
} else {
|
||||
includeUnmapped = false;
|
||||
}
|
||||
}
|
||||
|
||||
public FieldCapabilitiesRequest() {}
|
||||
|
||||
/**
|
||||
|
@ -78,17 +91,8 @@ public final class FieldCapabilitiesRequest extends ActionRequest implements Ind
|
|||
}
|
||||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
super.readFrom(in);
|
||||
fields = in.readStringArray();
|
||||
indices = in.readStringArray();
|
||||
indicesOptions = IndicesOptions.readIndicesOptions(in);
|
||||
mergeResults = in.readBoolean();
|
||||
if (in.getVersion().onOrAfter(Version.V_7_2_0)) {
|
||||
includeUnmapped = in.readBoolean();
|
||||
} else {
|
||||
includeUnmapped = false;
|
||||
}
|
||||
public void readFrom(StreamInput in) {
|
||||
throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -48,9 +48,9 @@ public class FieldCapabilitiesResponse extends ActionResponse implements ToXCont
|
|||
private static final ParseField INDICES_FIELD = new ParseField("indices");
|
||||
private static final ParseField FIELDS_FIELD = new ParseField("fields");
|
||||
|
||||
private String[] indices;
|
||||
private Map<String, Map<String, FieldCapabilities>> responseMap;
|
||||
private List<FieldCapabilitiesIndexResponse> indexResponses;
|
||||
private final String[] indices;
|
||||
private final Map<String, Map<String, FieldCapabilities>> responseMap;
|
||||
private final List<FieldCapabilitiesIndexResponse> indexResponses;
|
||||
|
||||
FieldCapabilitiesResponse(String[] indices, Map<String, Map<String, FieldCapabilities>> responseMap) {
|
||||
this(indices, responseMap, Collections.emptyList());
|
||||
|
@ -67,6 +67,17 @@ public class FieldCapabilitiesResponse extends ActionResponse implements ToXCont
|
|||
this.indices = indices;
|
||||
}
|
||||
|
||||
public FieldCapabilitiesResponse(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
if (in.getVersion().onOrAfter(Version.V_7_2_0)) {
|
||||
indices = in.readStringArray();
|
||||
} else {
|
||||
indices = Strings.EMPTY_ARRAY;
|
||||
}
|
||||
this.responseMap = in.readMap(StreamInput::readString, FieldCapabilitiesResponse::readField);
|
||||
indexResponses = in.readList(FieldCapabilitiesIndexResponse::new);
|
||||
}
|
||||
|
||||
/**
|
||||
* Used for serialization
|
||||
*/
|
||||
|
@ -106,15 +117,8 @@ public class FieldCapabilitiesResponse extends ActionResponse implements ToXCont
|
|||
}
|
||||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
super.readFrom(in);
|
||||
if (in.getVersion().onOrAfter(Version.V_7_2_0)) {
|
||||
indices = in.readStringArray();
|
||||
} else {
|
||||
indices = Strings.EMPTY_ARRAY;
|
||||
}
|
||||
this.responseMap = in.readMap(StreamInput::readString, FieldCapabilitiesResponse::readField);
|
||||
indexResponses = in.readList(FieldCapabilitiesIndexResponse::new);
|
||||
public void readFrom(StreamInput in) {
|
||||
throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable");
|
||||
}
|
||||
|
||||
private static Map<String, FieldCapabilities> readField(StreamInput in) throws IOException {
|
||||
|
|
|
@ -57,7 +57,7 @@ public class TransportFieldCapabilitiesAction extends HandledTransportAction<Fie
|
|||
ClusterService clusterService, ThreadPool threadPool,
|
||||
TransportFieldCapabilitiesIndexAction shardAction,
|
||||
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) {
|
||||
super(FieldCapabilitiesAction.NAME, transportService, FieldCapabilitiesRequest::new, actionFilters);
|
||||
super(FieldCapabilitiesAction.NAME, transportService, actionFilters, FieldCapabilitiesRequest::new);
|
||||
this.threadPool = threadPool;
|
||||
this.clusterService = clusterService;
|
||||
this.remoteClusterService = transportService.getRemoteClusterService();
|
||||
|
|
|
@ -84,7 +84,7 @@ public class BroadcastRequest<Request extends BroadcastRequest<Request>> extends
|
|||
}
|
||||
|
||||
@Override
|
||||
public final void readFrom(StreamInput in) throws IOException {
|
||||
throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable " + getClass().getName());
|
||||
public final void readFrom(StreamInput in) {
|
||||
throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,19 +19,14 @@
|
|||
|
||||
package org.elasticsearch.action.termvectors;
|
||||
|
||||
import org.elasticsearch.action.StreamableResponseActionType;
|
||||
import org.elasticsearch.action.ActionType;
|
||||
|
||||
public class MultiTermVectorsAction extends StreamableResponseActionType<MultiTermVectorsResponse> {
|
||||
public class MultiTermVectorsAction extends ActionType<MultiTermVectorsResponse> {
|
||||
|
||||
public static final MultiTermVectorsAction INSTANCE = new MultiTermVectorsAction();
|
||||
public static final String NAME = "indices:data/read/mtv";
|
||||
|
||||
private MultiTermVectorsAction() {
|
||||
super(NAME);
|
||||
}
|
||||
|
||||
@Override
|
||||
public MultiTermVectorsResponse newResponse() {
|
||||
return new MultiTermVectorsResponse();
|
||||
super(NAME, MultiTermVectorsResponse::new);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,21 +21,17 @@ package org.elasticsearch.action.termvectors;
|
|||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.io.stream.Streamable;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* A single multi get response.
|
||||
*/
|
||||
public class MultiTermVectorsItemResponse implements Streamable {
|
||||
public class MultiTermVectorsItemResponse implements Writeable {
|
||||
|
||||
private TermVectorsResponse response;
|
||||
private MultiTermVectorsResponse.Failure failure;
|
||||
|
||||
MultiTermVectorsItemResponse() {
|
||||
|
||||
}
|
||||
private final TermVectorsResponse response;
|
||||
private final MultiTermVectorsResponse.Failure failure;
|
||||
|
||||
public MultiTermVectorsItemResponse(TermVectorsResponse response, MultiTermVectorsResponse.Failure failure) {
|
||||
assert (((response == null) && (failure != null)) || ((response != null) && (failure == null)));
|
||||
|
@ -43,6 +39,16 @@ public class MultiTermVectorsItemResponse implements Streamable {
|
|||
this.failure = failure;
|
||||
}
|
||||
|
||||
MultiTermVectorsItemResponse(StreamInput in) throws IOException {
|
||||
if (in.readBoolean()) {
|
||||
failure = new MultiTermVectorsResponse.Failure(in);
|
||||
response = null;
|
||||
} else {
|
||||
response = new TermVectorsResponse(in);
|
||||
failure = null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The index name of the document.
|
||||
*/
|
||||
|
@ -94,21 +100,6 @@ public class MultiTermVectorsItemResponse implements Streamable {
|
|||
return this.failure;
|
||||
}
|
||||
|
||||
public static MultiTermVectorsItemResponse readItemResponse(StreamInput in) throws IOException {
|
||||
MultiTermVectorsItemResponse response = new MultiTermVectorsItemResponse();
|
||||
response.readFrom(in);
|
||||
return response;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
if (in.readBoolean()) {
|
||||
failure = MultiTermVectorsResponse.Failure.readFailure(in);
|
||||
} else {
|
||||
response = new TermVectorsResponse(in);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
if (failure != null) {
|
||||
|
|
|
@ -47,6 +47,18 @@ public class MultiTermVectorsRequest extends ActionRequest
|
|||
|
||||
final Set<String> ids = new HashSet<>();
|
||||
|
||||
public MultiTermVectorsRequest(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
preference = in.readOptionalString();
|
||||
int size = in.readVInt();
|
||||
requests = new ArrayList<>(size);
|
||||
for (int i = 0; i < size; i++) {
|
||||
requests.add(new TermVectorsRequest(in));
|
||||
}
|
||||
}
|
||||
|
||||
public MultiTermVectorsRequest() {}
|
||||
|
||||
public MultiTermVectorsRequest add(TermVectorsRequest termVectorsRequest) {
|
||||
requests.add(termVectorsRequest);
|
||||
return this;
|
||||
|
@ -138,13 +150,7 @@ public class MultiTermVectorsRequest extends ActionRequest
|
|||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
super.readFrom(in);
|
||||
preference = in.readOptionalString();
|
||||
int size = in.readVInt();
|
||||
requests = new ArrayList<>(size);
|
||||
for (int i = 0; i < size; i++) {
|
||||
requests.add(new TermVectorsRequest(in));
|
||||
}
|
||||
throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -23,7 +23,7 @@ import org.elasticsearch.ElasticsearchException;
|
|||
import org.elasticsearch.action.ActionResponse;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.io.stream.Streamable;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.common.xcontent.ToXContentObject;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
|
||||
|
@ -36,15 +36,11 @@ public class MultiTermVectorsResponse extends ActionResponse implements Iterable
|
|||
/**
|
||||
* Represents a failure.
|
||||
*/
|
||||
public static class Failure implements Streamable {
|
||||
private String index;
|
||||
private String type;
|
||||
private String id;
|
||||
private Exception cause;
|
||||
|
||||
Failure() {
|
||||
|
||||
}
|
||||
public static class Failure implements Writeable {
|
||||
private final String index;
|
||||
private final String type;
|
||||
private final String id;
|
||||
private final Exception cause;
|
||||
|
||||
public Failure(String index, String type, String id, Exception cause) {
|
||||
this.index = index;
|
||||
|
@ -53,6 +49,13 @@ public class MultiTermVectorsResponse extends ActionResponse implements Iterable
|
|||
this.cause = cause;
|
||||
}
|
||||
|
||||
public Failure(StreamInput in) throws IOException {
|
||||
index = in.readString();
|
||||
type = in.readOptionalString();
|
||||
id = in.readString();
|
||||
cause = in.readException();
|
||||
}
|
||||
|
||||
/**
|
||||
* The index name of the action.
|
||||
*/
|
||||
|
@ -84,20 +87,6 @@ public class MultiTermVectorsResponse extends ActionResponse implements Iterable
|
|||
return this.cause;
|
||||
}
|
||||
|
||||
public static Failure readFailure(StreamInput in) throws IOException {
|
||||
Failure failure = new Failure();
|
||||
failure.readFrom(in);
|
||||
return failure;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
index = in.readString();
|
||||
type = in.readOptionalString();
|
||||
id = in.readString();
|
||||
cause = in.readException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
out.writeString(index);
|
||||
|
@ -107,15 +96,20 @@ public class MultiTermVectorsResponse extends ActionResponse implements Iterable
|
|||
}
|
||||
}
|
||||
|
||||
private MultiTermVectorsItemResponse[] responses;
|
||||
|
||||
MultiTermVectorsResponse() {
|
||||
}
|
||||
private final MultiTermVectorsItemResponse[] responses;
|
||||
|
||||
public MultiTermVectorsResponse(MultiTermVectorsItemResponse[] responses) {
|
||||
this.responses = responses;
|
||||
}
|
||||
|
||||
public MultiTermVectorsResponse(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
responses = new MultiTermVectorsItemResponse[in.readVInt()];
|
||||
for (int i = 0; i < responses.length; i++) {
|
||||
responses[i] = new MultiTermVectorsItemResponse(in);
|
||||
}
|
||||
}
|
||||
|
||||
public MultiTermVectorsItemResponse[] getResponses() {
|
||||
return this.responses;
|
||||
}
|
||||
|
@ -157,11 +151,7 @@ public class MultiTermVectorsResponse extends ActionResponse implements Iterable
|
|||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
super.readFrom(in);
|
||||
responses = new MultiTermVectorsItemResponse[in.readVInt()];
|
||||
for (int i = 0; i < responses.length; i++) {
|
||||
responses[i] = MultiTermVectorsItemResponse.readItemResponse(in);
|
||||
}
|
||||
throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -54,7 +54,7 @@ public class MultiTermVectorsShardResponse extends ActionResponse {
|
|||
responses.add(null);
|
||||
}
|
||||
if (in.readBoolean()) {
|
||||
failures.add(MultiTermVectorsResponse.Failure.readFailure(in));
|
||||
failures.add(new MultiTermVectorsResponse.Failure(in));
|
||||
} else {
|
||||
failures.add(null);
|
||||
}
|
||||
|
|
|
@ -48,7 +48,7 @@ public class TransportMultiTermVectorsAction extends HandledTransportAction<Mult
|
|||
public TransportMultiTermVectorsAction(TransportService transportService, ClusterService clusterService,
|
||||
TransportShardMultiTermsVectorAction shardAction, ActionFilters actionFilters,
|
||||
IndexNameExpressionResolver indexNameExpressionResolver) {
|
||||
super(MultiTermVectorsAction.NAME, transportService, MultiTermVectorsRequest::new, actionFilters);
|
||||
super(MultiTermVectorsAction.NAME, transportService, actionFilters, MultiTermVectorsRequest::new);
|
||||
this.clusterService = clusterService;
|
||||
this.shardAction = shardAction;
|
||||
this.indexNameExpressionResolver = indexNameExpressionResolver;
|
||||
|
|
|
@ -85,13 +85,12 @@ public class SyncedFlushUnitTests extends ESTestCase {
|
|||
BytesStreamOutput out = new BytesStreamOutput();
|
||||
testPlan.result.writeTo(out);
|
||||
StreamInput in = out.bytes().streamInput();
|
||||
SyncedFlushResponse readResponse = new SyncedFlushResponse();
|
||||
readResponse.readFrom(in);
|
||||
SyncedFlushResponse readResponse = new SyncedFlushResponse(in);
|
||||
assertThat(readResponse.totalShards(), equalTo(testPlan.totalCounts.total));
|
||||
assertThat(readResponse.successfulShards(), equalTo(testPlan.totalCounts.successful));
|
||||
assertThat(readResponse.failedShards(), equalTo(testPlan.totalCounts.failed));
|
||||
assertThat(readResponse.restStatus(), equalTo(testPlan.totalCounts.failed > 0 ? RestStatus.CONFLICT : RestStatus.OK));
|
||||
assertThat(readResponse.shardsResultPerIndex.size(), equalTo(testPlan.result.getShardsResultPerIndex().size()));
|
||||
assertThat(readResponse.getShardsResultPerIndex().size(), equalTo(testPlan.result.getShardsResultPerIndex().size()));
|
||||
for (Map.Entry<String, List<ShardsSyncedFlushResult>> entry : readResponse.getShardsResultPerIndex().entrySet()) {
|
||||
List<ShardsSyncedFlushResult> originalShardsResults = testPlan.result.getShardsResultPerIndex().get(entry.getKey());
|
||||
assertNotNull(originalShardsResults);
|
||||
|
|
|
@ -21,15 +21,16 @@ package org.elasticsearch.action.fieldcaps;
|
|||
|
||||
import org.elasticsearch.action.ActionRequestValidationException;
|
||||
import org.elasticsearch.action.support.IndicesOptions;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.common.util.ArrayUtils;
|
||||
import org.elasticsearch.test.AbstractStreamableTestCase;
|
||||
import org.elasticsearch.test.AbstractWireSerializingTestCase;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.function.Consumer;
|
||||
|
||||
public class FieldCapabilitiesRequestTests extends AbstractStreamableTestCase<FieldCapabilitiesRequest> {
|
||||
public class FieldCapabilitiesRequestTests extends AbstractWireSerializingTestCase<FieldCapabilitiesRequest> {
|
||||
|
||||
@Override
|
||||
protected FieldCapabilitiesRequest createTestInstance() {
|
||||
|
@ -55,8 +56,8 @@ public class FieldCapabilitiesRequestTests extends AbstractStreamableTestCase<Fi
|
|||
}
|
||||
|
||||
@Override
|
||||
protected FieldCapabilitiesRequest createBlankInstance() {
|
||||
return new FieldCapabilitiesRequest();
|
||||
protected Writeable.Reader<FieldCapabilitiesRequest> instanceReader() {
|
||||
return FieldCapabilitiesRequest::new;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -19,7 +19,8 @@
|
|||
|
||||
package org.elasticsearch.action.fieldcaps;
|
||||
|
||||
import org.elasticsearch.test.AbstractStreamableTestCase;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.test.AbstractWireSerializingTestCase;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
|
@ -29,12 +30,7 @@ import java.util.Map;
|
|||
|
||||
import static com.carrotsearch.randomizedtesting.RandomizedTest.randomAsciiLettersOfLength;
|
||||
|
||||
|
||||
public class FieldCapabilitiesResponseTests extends AbstractStreamableTestCase<FieldCapabilitiesResponse> {
|
||||
@Override
|
||||
protected FieldCapabilitiesResponse createBlankInstance() {
|
||||
return new FieldCapabilitiesResponse();
|
||||
}
|
||||
public class FieldCapabilitiesResponseTests extends AbstractWireSerializingTestCase<FieldCapabilitiesResponse> {
|
||||
|
||||
@Override
|
||||
protected FieldCapabilitiesResponse createTestInstance() {
|
||||
|
@ -47,6 +43,11 @@ public class FieldCapabilitiesResponseTests extends AbstractStreamableTestCase<F
|
|||
return new FieldCapabilitiesResponse(responses);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Writeable.Reader<FieldCapabilitiesResponse> instanceReader() {
|
||||
return FieldCapabilitiesResponse::new;
|
||||
}
|
||||
|
||||
private FieldCapabilitiesIndexResponse createRandomIndexResponse() {
|
||||
Map<String, FieldCapabilities> responses = new HashMap<>();
|
||||
|
||||
|
|
|
@ -20,12 +20,13 @@
|
|||
package org.elasticsearch.action.fieldcaps;
|
||||
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.common.xcontent.XContentType;
|
||||
import org.elasticsearch.test.AbstractStreamableXContentTestCase;
|
||||
import org.elasticsearch.test.AbstractSerializingTestCase;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
|
@ -33,18 +34,13 @@ import java.util.HashMap;
|
|||
import java.util.Map;
|
||||
import java.util.function.Predicate;
|
||||
|
||||
public class MergedFieldCapabilitiesResponseTests extends AbstractStreamableXContentTestCase<FieldCapabilitiesResponse> {
|
||||
public class MergedFieldCapabilitiesResponseTests extends AbstractSerializingTestCase<FieldCapabilitiesResponse> {
|
||||
|
||||
@Override
|
||||
protected FieldCapabilitiesResponse doParseInstance(XContentParser parser) throws IOException {
|
||||
return FieldCapabilitiesResponse.fromXContent(parser);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected FieldCapabilitiesResponse createBlankInstance() {
|
||||
return new FieldCapabilitiesResponse();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected FieldCapabilitiesResponse createTestInstance() {
|
||||
// merged responses
|
||||
|
@ -71,6 +67,11 @@ public class MergedFieldCapabilitiesResponseTests extends AbstractStreamableXCon
|
|||
return new FieldCapabilitiesResponse(indices, responses);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Writeable.Reader<FieldCapabilitiesResponse> instanceReader() {
|
||||
return FieldCapabilitiesResponse::new;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected FieldCapabilitiesResponse mutateInstance(FieldCapabilitiesResponse response) {
|
||||
Map<String, Map<String, FieldCapabilities>> mutatedResponses = new HashMap<>(response.get());
|
||||
|
|
Loading…
Reference in New Issue