Removed writeTo from TransportResponse and ActionResponse (#44092)

The base classes for transport requests and responses currently
implement Streamable and Writeable. The writeTo method on these base
classes is implemented with an empty implementation. Not only does this
complicate subclasses to think they need to call super.writeTo, but it
also can lead to not implementing writeTo when it should have been
implemented, or extendiong one of these classes when not necessary,
since there is nothing to actually implement.

This commit removes the empty writeTo from these base classes, and fixes
subclasses to not call super and in some cases implement an empty
writeTo themselves.

relates #34389
This commit is contained in:
Ryan Ernst 2019-07-09 00:15:39 -07:00 committed by Ryan Ernst
parent bde5802ad6
commit fb77d8f461
174 changed files with 94 additions and 194 deletions

View File

@ -73,5 +73,4 @@ public class TaskSubmissionResponse {
public static TaskSubmissionResponse fromXContent(XContentParser parser) throws IOException {
return PARSER.parse(parser, null);
}
}

View File

@ -94,7 +94,6 @@ public class GrokProcessorGetAction extends StreamableResponseActionType<GrokPro
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeMap(grokPatterns, StreamOutput::writeString, StreamOutput::writeString);
}
}

View File

@ -153,7 +153,6 @@ public class MultiSearchTemplateResponse extends ActionResponse implements Itera
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(items.length);
for (Item item : items) {
item.writeTo(out);

View File

@ -81,7 +81,6 @@ public class SearchTemplateResponse extends ActionResponse implements StatusToXC
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeOptionalBytesReference(source);
out.writeOptionalStreamable(response);
}

View File

@ -146,7 +146,6 @@ public class PainlessContextAction extends ActionType<PainlessContextAction.Resp
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeStringCollection(scriptContextNames);
out.writeOptionalWriteable(painlessContextInfo);
}

View File

@ -406,7 +406,6 @@ public class PainlessExecuteAction extends ActionType<PainlessExecuteAction.Resp
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeGenericValue(result);
}

View File

@ -84,7 +84,6 @@ public class RankEvalResponse extends ActionResponse implements ToXContentObject
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeDouble(metricScore);
out.writeVInt(details.size());
for (String queryId : details.keySet()) {

View File

@ -20,7 +20,6 @@
package org.elasticsearch.action;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.transport.TransportResponse;
import java.io.IOException;
@ -41,9 +40,4 @@ public abstract class ActionResponse extends TransportResponse {
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
}
}

View File

@ -54,7 +54,6 @@ public class ClusterAllocationExplainResponse extends ActionResponse {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
cae.writeTo(out);
}
}

View File

@ -45,9 +45,7 @@ public class AddVotingConfigExclusionsResponse extends ActionResponse implements
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
}
public void writeTo(StreamOutput out) throws IOException {}
@Override
public XContentBuilder toXContent(final XContentBuilder builder, final Params params) throws IOException {

View File

@ -44,9 +44,7 @@ public class ClearVotingConfigExclusionsResponse extends ActionResponse implemen
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
}
public void writeTo(StreamOutput out) throws IOException {}
@Override
public XContentBuilder toXContent(final XContentBuilder builder, final Params params) throws IOException {

View File

@ -297,7 +297,6 @@ public class ClusterHealthResponse extends ActionResponse implements StatusToXCo
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeString(clusterName);
out.writeByte(clusterHealthStatus.value());
clusterStateHealth.writeTo(out);

View File

@ -53,7 +53,6 @@ public final class LivenessResponse extends ActionResponse {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
clusterName.writeTo(out);
out.writeOptionalWriteable(node);
}

View File

@ -52,7 +52,6 @@ public class GetTaskResponse extends ActionResponse implements ToXContentObject
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeOptionalWriteable(task);
}

View File

@ -49,7 +49,6 @@ public final class RemoteInfoResponse extends ActionResponse implements ToXConte
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeList(infos);
}

View File

@ -134,7 +134,6 @@ public class VerifyRepositoryResponse extends ActionResponse implements ToXConte
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeList(nodes);
}

View File

@ -22,6 +22,7 @@ package org.elasticsearch.action.admin.cluster.settings;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.ToXContentObject;
@ -161,4 +162,7 @@ public class ClusterGetSettingsResponse extends ActionResponse implements ToXCon
public String toString() {
return Strings.toString(this);
}
@Override
public void writeTo(StreamOutput out) throws IOException {}
}

View File

@ -64,7 +64,6 @@ public class ClusterSearchShardsResponse extends ActionResponse implements ToXCo
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(groups.length);
for (ClusterSearchShardsGroup response : groups) {
response.writeTo(out);

View File

@ -79,7 +79,6 @@ public class CreateSnapshotResponse extends ActionResponse implements ToXContent
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeOptionalWriteable(snapshotInfo);
}

View File

@ -83,7 +83,6 @@ public class GetSnapshotsResponse extends ActionResponse implements ToXContentOb
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(snapshots.size());
for (SnapshotInfo snapshotInfo : snapshots) {
snapshotInfo.writeTo(out);

View File

@ -69,7 +69,6 @@ public class RestoreSnapshotResponse extends ActionResponse implements ToXConten
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeOptionalStreamable(restoreInfo);
}

View File

@ -71,7 +71,6 @@ public class SnapshotsStatusResponse extends ActionResponse implements ToXConten
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(snapshots.size());
for (SnapshotStatus snapshotInfo : snapshots) {
snapshotInfo.writeTo(out);

View File

@ -92,7 +92,6 @@ public class ClusterStateResponse extends ActionResponse {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
clusterName.writeTo(out);
if (out.getVersion().onOrAfter(Version.V_6_6_0)) {
out.writeOptionalWriteable(clusterState);

View File

@ -126,8 +126,6 @@ public class GetStoredScriptResponse extends ActionResponse implements StatusToX
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
if (source == null) {
out.writeBoolean(false);
} else {

View File

@ -114,7 +114,6 @@ public class PendingClusterTasksResponse extends ActionResponse implements Itera
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(pendingTasks.size());
for (PendingClusterTask task : pendingTasks) {
task.writeTo(out);

View File

@ -52,7 +52,6 @@ public class AliasesExistResponse extends ActionResponse {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeBoolean(exists);
}
}

View File

@ -66,7 +66,6 @@ public class GetAliasesResponse extends ActionResponse {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(aliases.size());
for (ObjectObjectCursor<String, List<AliasMetaData>> entry : aliases) {
out.writeString(entry.key);

View File

@ -346,7 +346,6 @@ public class AnalyzeAction extends ActionType<AnalyzeAction.Response> {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
if (tokens != null) {
out.writeVInt(tokens.size());
for (AnalyzeToken token : tokens) {

View File

@ -48,7 +48,6 @@ public class IndicesExistsResponse extends ActionResponse {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeBoolean(exists);
}
}

View File

@ -50,7 +50,6 @@ public class TypesExistsResponse extends ActionResponse {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeBoolean(exists);
}
}

View File

@ -211,7 +211,6 @@ public class SyncedFlushResponse extends ActionResponse implements ToXContentFra
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
shardCounts.writeTo(out);
out.writeInt(shardsResultPerIndex.size());
for (Map.Entry<String, List<ShardsSyncedFlushResult>> entry : shardsResultPerIndex.entrySet()) {

View File

@ -204,7 +204,6 @@ public class GetIndexResponse extends ActionResponse implements ToXContentObject
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeStringArray(indices);
out.writeVInt(mappings.size());
for (ObjectObjectCursor<String, ImmutableOpenMap<String, MappingMetaData>> indexEntry : mappings) {

View File

@ -298,7 +298,6 @@ public class GetFieldMappingsResponse extends ActionResponse implements ToXConte
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(mappings.size());
for (Map.Entry<String, Map<String, Map<String, FieldMappingMetaData>>> indexEntry : mappings.entrySet()) {
out.writeString(indexEntry.getKey());

View File

@ -77,7 +77,6 @@ public class GetMappingsResponse extends ActionResponse implements ToXContentFra
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(mappings.size());
for (ObjectObjectCursor<String, ImmutableOpenMap<String, MappingMetaData>> indexEntry : mappings) {
out.writeString(indexEntry.key);

View File

@ -122,7 +122,6 @@ public class GetSettingsResponse extends ActionResponse implements ToXContentObj
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(indexToSettings.size());
for (ObjectObjectCursor<String, Settings> cursor : indexToSettings) {
out.writeString(cursor.key);

View File

@ -337,7 +337,6 @@ public class IndicesShardStoresResponse extends ActionResponse implements ToXCon
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(storeStatuses.size());
for (ObjectObjectCursor<String, ImmutableOpenIntMap<List<StoreStatus>>> indexShards : storeStatuses) {
out.writeString(indexShards.key);

View File

@ -63,7 +63,6 @@ public class GetIndexTemplatesResponse extends ActionResponse implements ToXCont
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(indexTemplates.size());
for (IndexTemplateMetaData indexTemplate : indexTemplates) {
indexTemplate.writeTo(out);

View File

@ -142,7 +142,6 @@ public class BulkResponse extends ActionResponse implements Iterable<BulkItemRes
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(responses.length);
for (BulkItemResponse response : responses) {
response.writeTo(out);

View File

@ -143,7 +143,6 @@ public class ExplainResponse extends ActionResponse implements StatusToXContentO
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeString(index);
out.writeString(type);
out.writeString(id);

View File

@ -80,7 +80,6 @@ public class FieldCapabilitiesIndexResponse extends ActionResponse implements Wr
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeString(indexName);
out.writeMap(responseMap,
StreamOutput::writeString, (valueOut, fc) -> fc.writeTo(valueOut));

View File

@ -123,7 +123,6 @@ public class FieldCapabilitiesResponse extends ActionResponse implements ToXCont
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
if (out.getVersion().onOrAfter(Version.V_7_2_0)) {
out.writeStringArray(indices);
}

View File

@ -210,7 +210,6 @@ public class GetResponse extends ActionResponse implements Iterable<DocumentFiel
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
getResult.writeTo(out);
}

View File

@ -254,7 +254,6 @@ public class MultiGetResponse extends ActionResponse implements Iterable<MultiGe
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(responses.length);
for (MultiGetItemResponse response : responses) {
response.writeTo(out);

View File

@ -80,7 +80,6 @@ public class MultiGetShardResponse extends ActionResponse {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(locations.size());
for (int i = 0; i < locations.size(); i++) {
out.writeVInt(locations.get(i));

View File

@ -72,7 +72,6 @@ public class GetPipelineResponse extends ActionResponse implements StatusToXCont
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(pipelines.size());
for (PipelineConfiguration pipeline : pipelines) {
pipeline.writeTo(out);

View File

@ -127,7 +127,6 @@ public class SimulatePipelineResponse extends ActionResponse implements ToXConte
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeOptionalString(pipelineId);
out.writeBoolean(verbose);
out.writeVInt(results.size());

View File

@ -76,7 +76,6 @@ public class MainResponse extends ActionResponse implements ToXContentObject {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeString(nodeName);
Version.writeVersion(version, out);
clusterName.writeTo(out);

View File

@ -104,7 +104,6 @@ public class ClearScrollResponse extends ActionResponse implements StatusToXCont
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeBoolean(succeeded);
out.writeVInt(numFreed);
}

View File

@ -165,7 +165,6 @@ public class MultiSearchResponse extends ActionResponse implements Iterable<Mult
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(items.length);
for (Item item : items) {
item.writeTo(out);

View File

@ -372,7 +372,6 @@ public class SearchResponse extends ActionResponse implements StatusToXContentOb
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
internalResponse.writeTo(out);
out.writeVInt(totalShards);
out.writeVInt(successfulShards);

View File

@ -297,7 +297,6 @@ public class SearchTransportService {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeBoolean(freed);
}
}

View File

@ -137,7 +137,6 @@ public class BroadcastResponse extends ActionResponse implements ToXContentObjec
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(totalShards);
out.writeVInt(successfulShards);
out.writeVInt(failedShards);

View File

@ -54,7 +54,6 @@ public abstract class BroadcastShardResponse extends TransportResponse {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
shardId.writeTo(out);
}
}

View File

@ -549,7 +549,6 @@ public abstract class TransportBroadcastByNodeAction<Request extends BroadcastRe
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeString(nodeId);
out.writeVInt(totalShards);
out.writeVInt(results.size());

View File

@ -78,7 +78,6 @@ public class AcknowledgedResponse extends ActionResponse implements ToXContentOb
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeBoolean(acknowledged);
}

View File

@ -56,7 +56,6 @@ public abstract class BaseNodeResponse extends TransportResponse {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
node.writeTo(out);
}
}

View File

@ -110,7 +110,6 @@ public abstract class BaseNodesResponse<TNodeResponse extends BaseNodeResponse>
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
clusterName.writeTo(out);
writeNodesTo(out, nodes);
out.writeList(failures);

View File

@ -58,7 +58,6 @@ public class ReplicationResponse extends ActionResponse {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
shardInfo.writeTo(out);
}

View File

@ -988,7 +988,6 @@ public abstract class TransportReplicationAction<
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeZLong(localCheckpoint);
out.writeZLong(globalCheckpoint);
}

View File

@ -73,7 +73,6 @@ public class BaseTasksResponse extends ActionResponse {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(taskFailures.size());
for (TaskOperationFailure exp : taskFailures) {
exp.writeTo(out);

View File

@ -410,7 +410,6 @@ public abstract class TransportTasksAction<
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeString(nodeId);
out.writeVInt(results.size());
for (TaskResponse result : results) {

View File

@ -166,7 +166,6 @@ public class MultiTermVectorsResponse extends ActionResponse implements Iterable
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(responses.length);
for (MultiTermVectorsItemResponse response : responses) {
response.writeTo(out);

View File

@ -80,7 +80,6 @@ public class MultiTermVectorsShardResponse extends ActionResponse {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(locations.size());
for (int i = 0; i < locations.size(); i++) {
out.writeVInt(locations.get(i));

View File

@ -50,7 +50,6 @@ public class PeersResponse extends TransportResponse {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeOptionalWriteable(masterNode.orElse(null));
out.writeList(knownPeers);
out.writeLong(term);

View File

@ -47,7 +47,6 @@ public class PreVoteResponse extends TransportResponse {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeLong(currentTerm);
out.writeLong(lastAcceptedTerm);
out.writeLong(lastAcceptedVersion);

View File

@ -45,7 +45,6 @@ public class PublishWithJoinResponse extends TransportResponse {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
publishResponse.writeTo(out);
out.writeOptionalWriteable(optionalJoin.orElse(null));
}

View File

@ -440,5 +440,8 @@ public class MasterFaultDetection extends FaultDetection {
public MasterPingResponseResponse(StreamInput in) throws IOException {
super(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {}
}
}

View File

@ -366,5 +366,8 @@ public class NodesFaultDetection extends FaultDetection {
public PingResponse(StreamInput in) throws IOException {
super(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {}
}
}

View File

@ -572,7 +572,6 @@ public class UnicastZenPing implements ZenPing {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeInt(id);
out.writeVInt(pingResponses.length);
for (PingResponse pingResponse : pingResponses) {

View File

@ -260,7 +260,6 @@ public class LocalAllocateDangledIndices {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeBoolean(ack);
}
}

View File

@ -186,7 +186,6 @@ public class BulkByScrollResponse extends ActionResponse implements ToXContentFr
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeTimeValue(took);
status.writeTo(out);
out.writeList(bulkFailures);

View File

@ -392,6 +392,8 @@ public class RetentionLeaseActions {
super(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {}
}
}

View File

@ -618,7 +618,6 @@ public class SyncedFlushService implements IndexEventListener {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
commitId.writeTo(out);
if (includeNumDocs(out.getVersion())) {
out.writeInt(numDocs);
@ -707,7 +706,6 @@ public class SyncedFlushService implements IndexEventListener {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeOptionalString(failureReason);
}
@ -793,7 +791,6 @@ public class SyncedFlushService implements IndexEventListener {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(opCount);
}

View File

@ -75,7 +75,6 @@ public final class RecoveryResponse extends TransportResponse {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeStringCollection(phase1FileNames);
out.writeCollection(phase1FileSizes, StreamOutput::writeVLong);
out.writeStringCollection(phase1ExistingFileNames);

View File

@ -47,7 +47,6 @@ final class RecoveryTranslogOperationsResponse extends TransportResponse {
@Override
public void writeTo(final StreamOutput out) throws IOException {
super.writeTo(out);
// before 6.0.0 we responded with an empty response so we have to maintain that
if (out.getVersion().onOrAfter(Version.V_6_0_0_alpha1)) {
out.writeZLong(localCheckpoint);

View File

@ -443,7 +443,6 @@ public class IndicesStore implements ClusterStateListener, Closeable {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeBoolean(shardActive);
node.writeTo(out);
}

View File

@ -48,7 +48,6 @@ public class PersistentTaskResponse extends ActionResponse {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeOptionalWriteable(task);
}

View File

@ -20,6 +20,7 @@
package org.elasticsearch.search;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.search.fetch.FetchSearchResult;
import org.elasticsearch.search.query.QuerySearchResult;
import org.elasticsearch.transport.TransportResponse;
@ -93,4 +94,9 @@ public abstract class SearchPhaseResult extends TransportResponse {
public final void readFrom(StreamInput in) {
throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable");
}
@Override
public void writeTo(StreamOutput out) throws IOException {
// TODO: this seems wrong, SearchPhaseResult should have a writeTo?
}
}

View File

@ -1089,7 +1089,6 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeBoolean(canMatch);
}

View File

@ -100,7 +100,6 @@ public class DfsSearchResult extends SearchPhaseResult {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeLong(requestId);
out.writeVInt(terms.length);
for (Term term : terms) {

View File

@ -86,7 +86,6 @@ public final class FetchSearchResult extends SearchPhaseResult {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeLong(requestId);
hits.writeTo(out);
}

View File

@ -79,7 +79,6 @@ public final class QueryFetchSearchResult extends SearchPhaseResult {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
queryResult.writeTo(out);
fetchResult.writeTo(out);
}

View File

@ -71,7 +71,6 @@ public final class ScrollQueryFetchSearchResult extends SearchPhaseResult {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
getSearchShardTarget().writeTo(out);
result.writeTo(out);
}

View File

@ -305,7 +305,6 @@ public final class QuerySearchResult extends SearchPhaseResult {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeLong(requestId);
writeToNoId(out);
}

View File

@ -61,7 +61,6 @@ public final class ScrollQuerySearchResult extends SearchPhaseResult {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
getSearchShardTarget().writeTo(out);
result.writeTo(out);
}

View File

@ -609,7 +609,8 @@ public class SnapshotShardsService extends AbstractLifecycleComponent implements
}
static class UpdateIndexShardSnapshotStatusResponse extends ActionResponse {
@Override
public void writeTo(StreamOutput out) throws IOException {}
}
private class UpdateSnapshotStatusAction

View File

@ -224,7 +224,6 @@ final class TransportHandshaker {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
assert responseVersion != null;
Version.writeVersion(responseVersion, out);
}

View File

@ -20,7 +20,6 @@
package org.elasticsearch.transport;
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.transport.TransportAddress;
@ -56,9 +55,4 @@ public abstract class TransportMessage implements Streamable, Writeable {
public void readFrom(StreamInput in) throws IOException {
}
@Override
public void writeTo(StreamOutput out) throws IOException {
}
}

View File

@ -67,7 +67,6 @@ public abstract class TransportRequest extends TransportMessage implements TaskA
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
parentTaskId.writeTo(out);
}
}

View File

@ -20,6 +20,7 @@
package org.elasticsearch.transport;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import java.io.IOException;
@ -47,5 +48,8 @@ public abstract class TransportResponse extends TransportMessage {
public String toString() {
return "Empty{}";
}
@Override
public void writeTo(StreamOutput out) throws IOException {}
}
}

View File

@ -509,7 +509,6 @@ public class TransportService extends AbstractLifecycleComponent implements Tran
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeOptionalWriteable(discoveryNode);
clusterName.writeTo(out);
Version.writeVersion(version, out);

View File

@ -25,6 +25,7 @@ import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.LatchedActionListener;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.node.Node;
import org.elasticsearch.tasks.Task;
@ -34,6 +35,7 @@ import org.elasticsearch.threadpool.ThreadPool;
import org.junit.After;
import org.junit.Before;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
@ -261,6 +263,7 @@ public class TransportActionFilterChainTests extends ESTestCase {
}
private static class TestResponse extends ActionResponse {
@Override
public void writeTo(StreamOutput out) throws IOException {}
}
}

View File

@ -149,7 +149,6 @@ public class TransportMasterNodeActionTests extends ESTestCase {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeLong(identity);
}

View File

@ -36,6 +36,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.rest.RestStatus;
@ -51,6 +52,7 @@ import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
@ -82,6 +84,9 @@ public class TransportInstanceSingleOperationActionTests extends ESTestCase {
public static class Response extends ActionResponse {
public Response() {
}
@Override
public void writeTo(StreamOutput out) throws IOException {}
}
class TestTransportInstanceSingleOperationAction extends TransportInstanceSingleOperationAction<Request, Response> {

View File

@ -33,6 +33,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.common.unit.TimeValue;
@ -437,5 +438,8 @@ public class TransportClientNodesServiceTests extends ESTestCase {
private TestResponse() {}
private TestResponse(StreamInput in) {}
@Override
public void writeTo(StreamOutput out) throws IOException {}
}
}

View File

@ -111,7 +111,8 @@ public class InternalOrPrivateSettingsPlugin extends Plugin implements ActionPlu
}
static class Response extends ActionResponse {
@Override
public void writeTo(StreamOutput out) throws IOException {}
}
@Override

View File

@ -202,7 +202,6 @@ public class InboundHandlerTests extends ESTestCase {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeString(value);
}
}

View File

@ -241,7 +241,6 @@ public class TransportActionProxyTests extends ESTestCase {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeString(targetNode);
}
}

View File

@ -1172,7 +1172,6 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeString(message);
}
}
@ -1236,7 +1235,6 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeInt(value1);
}
}
@ -1690,7 +1688,6 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeOptionalString(info);
}

View File

@ -292,7 +292,6 @@ public class ShardChangesAction extends ActionType<ShardChangesAction.Response>
@Override
public void writeTo(final StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVLong(mappingVersion);
out.writeVLong(settingsVersion);
if (out.getVersion().onOrAfter(Version.V_7_3_0)) {

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