mirror of
https://github.com/honeymoose/OpenSearch.git
synced 2025-03-24 17:09:48 +00:00
Ban java serialization
We had several problems with Java Serializatin in the past. At some point in the Java 1.7.x series JDKs where not compatible anymore when java serialization (ObjectStream) was used to exchange objects. In elasticsearch we used this to serialize exceptions across the wire which caused several problems with incompatible JDKs. While causing lot of trouble this essentially prevented users from moving forward and upgrade their JVMs. To prevent these kind of issues this commit removes the dependency on java serialization entirely and bans the usage of ObjectOutputStream and ObjectInputStream entirely. Yet, we can't fully serialize all exception anymore such that this commit is best effort and adds hand written serialization to all elasticsearch exceptions as well to a selected set of JDK and Lucene exceptions. (see StreamOutput#writeThrowable / StreamInput.readThrowable). Stacktraces should be preserved for all exceptions while several names might be replaced with ElasticsearchException if there is no mapping for the given exception.
This commit is contained in:
parent
f26672c184
commit
e7eb9cf4de
@ -22,17 +22,18 @@ package org.elasticsearch;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import org.elasticsearch.action.ShardOperationFailedException;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.collect.Tuple;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.IndexException;
|
||||
import org.elasticsearch.rest.HasRestHeaders;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Constructor;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.util.*;
|
||||
|
||||
/**
|
||||
@ -41,6 +42,7 @@ import java.util.*;
|
||||
public class ElasticsearchException extends RuntimeException implements ToXContent {
|
||||
|
||||
public static final String REST_EXCEPTION_SKIP_CAUSE = "rest.exception.skip_cause";
|
||||
static final Map<String, Constructor<? extends ElasticsearchException>> MAPPING;
|
||||
|
||||
/**
|
||||
* Construct a <code>ElasticsearchException</code> with the specified detail message.
|
||||
@ -62,6 +64,12 @@ public class ElasticsearchException extends RuntimeException implements ToXConte
|
||||
super(msg, cause);
|
||||
}
|
||||
|
||||
public ElasticsearchException(StreamInput in) throws IOException {
|
||||
super(in.readOptionalString(), in.readThrowable()); //TODO readOptionalThrowable
|
||||
readStackTrace(this, in);
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the rest status code associated with this exception.
|
||||
*/
|
||||
@ -152,19 +160,73 @@ public class ElasticsearchException extends RuntimeException implements ToXConte
|
||||
}
|
||||
}
|
||||
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
out.writeOptionalString(this.getMessage());
|
||||
out.writeThrowable(this.getCause());
|
||||
writeStackTraces(this, out);
|
||||
}
|
||||
|
||||
public static ElasticsearchException readException(StreamInput input) throws IOException {
|
||||
final String name = input.readString();
|
||||
Constructor<? extends ElasticsearchException> elasticsearchException = MAPPING.get(name);
|
||||
if (elasticsearchException == null) {
|
||||
throw new IllegalStateException("unknown exception with name: " + name);
|
||||
}
|
||||
try {
|
||||
return elasticsearchException.newInstance(input);
|
||||
} catch (InstantiationException|IllegalAccessException|InvocationTargetException e) {
|
||||
throw new IOException("failed to read exception: [" + name + "]", e);
|
||||
}
|
||||
}
|
||||
|
||||
public static void writeException(ElasticsearchException ex, StreamOutput output) throws IOException {
|
||||
output.writeString(ex.getClass().getName());
|
||||
ex.writeTo(output);
|
||||
}
|
||||
|
||||
/**
|
||||
* A base class for exceptions that should carry rest headers
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
public static class WithRestHeaders extends ElasticsearchException implements HasRestHeaders {
|
||||
public static class WithRestHeadersException extends ElasticsearchException implements HasRestHeaders {
|
||||
|
||||
private final ImmutableMap<String, List<String>> headers;
|
||||
|
||||
public WithRestHeaders(String msg, Tuple<String, String[]>... headers) {
|
||||
public WithRestHeadersException(String msg, Tuple<String, String[]>... headers) {
|
||||
super(msg);
|
||||
this.headers = headers(headers);
|
||||
}
|
||||
|
||||
public WithRestHeadersException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
int numKeys = in.readVInt();
|
||||
ImmutableMap.Builder<String, List<String>> builder = ImmutableMap.builder();
|
||||
for (int i = 0; i < numKeys; i++) {
|
||||
String key = in.readString();
|
||||
int numValues = in.readVInt();
|
||||
ArrayList<String> headers = new ArrayList<>(numValues);
|
||||
for (int j = 0; j < numValues; j++) {
|
||||
headers.add(in.readString());
|
||||
}
|
||||
builder.put(key, headers);
|
||||
}
|
||||
headers = builder.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeVInt(headers.size());
|
||||
for (Map.Entry<String, List<String>> entry : headers.entrySet()) {
|
||||
out.writeString(entry.getKey());
|
||||
out.writeVInt(entry.getValue().size());
|
||||
for (String v : entry.getValue()) {
|
||||
out.writeString(v);
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ImmutableMap<String, List<String>> getHeaders() {
|
||||
return headers;
|
||||
@ -290,4 +352,205 @@ public class ElasticsearchException extends RuntimeException implements ToXConte
|
||||
public String toString() {
|
||||
return ExceptionsHelper.detailedMessage(this).trim();
|
||||
}
|
||||
|
||||
public static <T extends Throwable> T readStackTrace(T throwable, StreamInput in) throws IOException {
|
||||
int stackTraceElements = in.readVInt();
|
||||
StackTraceElement[] stackTrace = new StackTraceElement[stackTraceElements];
|
||||
for (int i = 0; i < stackTraceElements; i++) {
|
||||
String declaringClasss = in.readString();
|
||||
String fielName = in.readString();
|
||||
String methodName = in.readString();
|
||||
int lineNumber = in.readVInt();
|
||||
stackTrace[i] = new StackTraceElement(declaringClasss,methodName, fielName, lineNumber);
|
||||
}
|
||||
throwable.setStackTrace(stackTrace);
|
||||
|
||||
int numSuppressed = in.readVInt();
|
||||
for (int i = 0; i < numSuppressed; i++) {
|
||||
throwable.addSuppressed(in.readThrowable());
|
||||
}
|
||||
return throwable;
|
||||
}
|
||||
|
||||
public static <T extends Throwable> T writeStackTraces(T throwable, StreamOutput out) throws IOException {
|
||||
StackTraceElement[] stackTrace = throwable.getStackTrace();
|
||||
out.writeVInt(stackTrace.length);
|
||||
for (StackTraceElement element : stackTrace) {
|
||||
out.writeString(element.getClassName());
|
||||
out.writeString(element.getFileName());
|
||||
out.writeString(element.getMethodName());
|
||||
out.writeVInt(element.getLineNumber());
|
||||
}
|
||||
Throwable[] suppressed = throwable.getSuppressed();
|
||||
out.writeVInt(suppressed.length);
|
||||
for (Throwable t : suppressed) {
|
||||
out.writeThrowable(t);
|
||||
}
|
||||
return throwable;
|
||||
}
|
||||
|
||||
static {
|
||||
Class<? extends ElasticsearchException>[] exceptions = new Class[]{
|
||||
org.elasticsearch.common.settings.SettingsException.class,
|
||||
org.elasticsearch.index.snapshots.IndexShardSnapshotFailedException.class,
|
||||
org.elasticsearch.index.engine.IndexFailedEngineException.class,
|
||||
org.elasticsearch.indices.recovery.RecoverFilesRecoveryException.class,
|
||||
org.elasticsearch.index.translog.TruncatedTranslogException.class,
|
||||
org.elasticsearch.repositories.RepositoryException.class,
|
||||
org.elasticsearch.index.shard.IndexShardException.class,
|
||||
org.elasticsearch.index.engine.DocumentSourceMissingException.class,
|
||||
org.elasticsearch.index.engine.DocumentMissingException.class,
|
||||
org.elasticsearch.common.util.concurrent.EsRejectedExecutionException.class,
|
||||
org.elasticsearch.cluster.routing.RoutingException.class,
|
||||
org.elasticsearch.common.lucene.Lucene.EarlyTerminationException.class,
|
||||
org.elasticsearch.indices.InvalidAliasNameException.class,
|
||||
org.elasticsearch.index.engine.EngineCreationFailureException.class,
|
||||
org.elasticsearch.index.snapshots.IndexShardRestoreFailedException.class,
|
||||
org.elasticsearch.script.groovy.GroovyScriptCompilationException.class,
|
||||
org.elasticsearch.cluster.routing.RoutingValidationException.class,
|
||||
org.elasticsearch.snapshots.SnapshotMissingException.class,
|
||||
org.elasticsearch.index.shard.IndexShardRecoveryException.class,
|
||||
org.elasticsearch.action.search.SearchPhaseExecutionException.class,
|
||||
org.elasticsearch.common.util.concurrent.UncategorizedExecutionException.class,
|
||||
org.elasticsearch.index.engine.SnapshotFailedEngineException.class,
|
||||
org.elasticsearch.action.search.ReduceSearchPhaseException.class,
|
||||
org.elasticsearch.action.RoutingMissingException.class,
|
||||
org.elasticsearch.index.engine.DeleteFailedEngineException.class,
|
||||
org.elasticsearch.indices.recovery.RecoveryFailedException.class,
|
||||
org.elasticsearch.search.builder.SearchSourceBuilderException.class,
|
||||
org.elasticsearch.index.engine.RefreshFailedEngineException.class,
|
||||
org.elasticsearch.index.snapshots.IndexShardSnapshotException.class,
|
||||
org.elasticsearch.search.query.QueryPhaseExecutionException.class,
|
||||
org.elasticsearch.cluster.metadata.ProcessClusterEventTimeoutException.class,
|
||||
org.elasticsearch.index.shard.IndexShardCreationException.class,
|
||||
org.elasticsearch.index.percolator.PercolatorException.class,
|
||||
org.elasticsearch.snapshots.ConcurrentSnapshotExecutionException.class,
|
||||
org.elasticsearch.indices.IndexTemplateAlreadyExistsException.class,
|
||||
org.elasticsearch.indices.InvalidIndexNameException.class,
|
||||
org.elasticsearch.index.IndexException.class,
|
||||
org.elasticsearch.indices.recovery.DelayRecoveryException.class,
|
||||
org.elasticsearch.indices.AliasFilterParsingException.class,
|
||||
org.elasticsearch.indices.InvalidIndexTemplateException.class,
|
||||
org.elasticsearch.http.HttpException.class,
|
||||
org.elasticsearch.index.shard.IndexShardNotRecoveringException.class,
|
||||
org.elasticsearch.indices.IndexPrimaryShardNotAllocatedException.class,
|
||||
org.elasticsearch.env.FailedToResolveConfigException.class,
|
||||
org.elasticsearch.action.UnavailableShardsException.class,
|
||||
org.elasticsearch.transport.ActionNotFoundTransportException.class,
|
||||
org.elasticsearch.index.shard.TranslogRecoveryPerformer.BatchOperationException.class,
|
||||
org.elasticsearch.ElasticsearchException.class,
|
||||
org.elasticsearch.index.shard.IndexShardClosedException.class,
|
||||
org.elasticsearch.client.transport.NoNodeAvailableException.class,
|
||||
org.elasticsearch.cluster.block.ClusterBlockException.class,
|
||||
org.elasticsearch.action.FailedNodeException.class,
|
||||
org.elasticsearch.indices.TypeMissingException.class,
|
||||
org.elasticsearch.index.IndexShardMissingException.class,
|
||||
org.elasticsearch.indices.InvalidTypeNameException.class,
|
||||
org.elasticsearch.transport.netty.SizeHeaderFrameDecoder.HttpOnTransportException.class,
|
||||
org.elasticsearch.common.util.CancellableThreads.ExecutionCancelledException.class,
|
||||
org.elasticsearch.snapshots.SnapshotCreationException.class,
|
||||
org.elasticsearch.script.groovy.GroovyScriptExecutionException.class,
|
||||
org.elasticsearch.indices.IndexTemplateMissingException.class,
|
||||
org.elasticsearch.transport.NodeNotConnectedException.class,
|
||||
org.elasticsearch.index.shard.IndexShardRecoveringException.class,
|
||||
org.elasticsearch.index.shard.IndexShardStartedException.class,
|
||||
org.elasticsearch.indices.IndexClosedException.class,
|
||||
org.elasticsearch.repositories.RepositoryMissingException.class,
|
||||
org.elasticsearch.search.warmer.IndexWarmerMissingException.class,
|
||||
org.elasticsearch.percolator.PercolateException.class,
|
||||
org.elasticsearch.index.engine.EngineException.class,
|
||||
org.elasticsearch.script.expression.ExpressionScriptExecutionException.class,
|
||||
org.elasticsearch.action.NoShardAvailableActionException.class,
|
||||
org.elasticsearch.transport.ReceiveTimeoutTransportException.class,
|
||||
org.elasticsearch.http.BindHttpException.class,
|
||||
org.elasticsearch.transport.RemoteTransportException.class,
|
||||
org.elasticsearch.index.shard.IndexShardRelocatedException.class,
|
||||
org.elasticsearch.snapshots.InvalidSnapshotNameException.class,
|
||||
org.elasticsearch.repositories.RepositoryVerificationException.class,
|
||||
org.elasticsearch.search.SearchException.class,
|
||||
org.elasticsearch.transport.ActionTransportException.class,
|
||||
org.elasticsearch.common.settings.NoClassSettingsException.class,
|
||||
org.elasticsearch.transport.NodeShouldNotConnectException.class,
|
||||
org.elasticsearch.index.mapper.MapperParsingException.class,
|
||||
org.elasticsearch.action.support.replication.TransportReplicationAction.RetryOnReplicaException.class,
|
||||
org.elasticsearch.search.dfs.DfsPhaseExecutionException.class,
|
||||
org.elasticsearch.index.engine.VersionConflictEngineException.class,
|
||||
org.elasticsearch.snapshots.SnapshotRestoreException.class,
|
||||
org.elasticsearch.script.Script.ScriptParseException.class,
|
||||
org.elasticsearch.ElasticsearchGenerationException.class,
|
||||
org.elasticsearch.action.TimestampParsingException.class,
|
||||
org.elasticsearch.action.NoSuchNodeException.class,
|
||||
org.elasticsearch.transport.BindTransportException.class,
|
||||
org.elasticsearch.snapshots.SnapshotException.class,
|
||||
org.elasticsearch.index.mapper.MapperException.class,
|
||||
org.elasticsearch.transport.TransportException.class,
|
||||
org.elasticsearch.search.SearchContextException.class,
|
||||
org.elasticsearch.index.translog.TranslogCorruptedException.class,
|
||||
org.elasticsearch.transport.TransportSerializationException.class,
|
||||
org.elasticsearch.cluster.IncompatibleClusterStateVersionException.class,
|
||||
org.elasticsearch.indices.IndexCreationException.class,
|
||||
org.elasticsearch.index.mapper.MergeMappingException.class,
|
||||
org.elasticsearch.transport.NotSerializableTransportException.class,
|
||||
org.elasticsearch.ElasticsearchTimeoutException.class,
|
||||
org.elasticsearch.search.SearchContextMissingException.class,
|
||||
org.elasticsearch.transport.SendRequestTransportException.class,
|
||||
org.elasticsearch.indices.IndexMissingException.class,
|
||||
org.elasticsearch.index.IndexShardAlreadyExistsException.class,
|
||||
org.elasticsearch.indices.IndexAlreadyExistsException.class,
|
||||
org.elasticsearch.index.engine.DocumentAlreadyExistsException.class,
|
||||
org.elasticsearch.transport.ConnectTransportException.class,
|
||||
org.elasticsearch.gateway.GatewayException.class,
|
||||
org.elasticsearch.script.ScriptException.class,
|
||||
org.elasticsearch.script.expression.ExpressionScriptCompilationException.class,
|
||||
org.elasticsearch.index.shard.IndexShardNotStartedException.class,
|
||||
org.elasticsearch.index.mapper.StrictDynamicMappingException.class,
|
||||
org.elasticsearch.index.engine.EngineClosedException.class,
|
||||
org.elasticsearch.rest.action.admin.indices.alias.delete.AliasesMissingException.class,
|
||||
org.elasticsearch.transport.ResponseHandlerFailureTransportException.class,
|
||||
org.elasticsearch.search.SearchParseException.class,
|
||||
org.elasticsearch.search.fetch.FetchPhaseExecutionException.class,
|
||||
org.elasticsearch.transport.NodeDisconnectedException.class,
|
||||
org.elasticsearch.common.breaker.CircuitBreakingException.class,
|
||||
org.elasticsearch.search.aggregations.AggregationInitializationException.class,
|
||||
org.elasticsearch.search.aggregations.InvalidAggregationPathException.class,
|
||||
org.elasticsearch.cluster.routing.IllegalShardRoutingStateException.class,
|
||||
org.elasticsearch.index.engine.FlushFailedEngineException.class,
|
||||
org.elasticsearch.index.AlreadyExpiredException.class,
|
||||
org.elasticsearch.index.translog.TranslogException.class,
|
||||
org.elasticsearch.index.engine.FlushNotAllowedEngineException.class,
|
||||
org.elasticsearch.index.engine.RecoveryEngineException.class,
|
||||
org.elasticsearch.common.blobstore.BlobStoreException.class,
|
||||
org.elasticsearch.index.snapshots.IndexShardRestoreException.class,
|
||||
org.elasticsearch.index.store.StoreException.class,
|
||||
org.elasticsearch.index.query.QueryParsingException.class,
|
||||
org.elasticsearch.action.support.replication.TransportReplicationAction.RetryOnPrimaryException.class,
|
||||
org.elasticsearch.index.engine.DeleteByQueryFailedEngineException.class,
|
||||
org.elasticsearch.index.engine.ForceMergeFailedEngineException.class,
|
||||
org.elasticsearch.discovery.MasterNotDiscoveredException.class,
|
||||
org.elasticsearch.action.support.broadcast.BroadcastShardOperationFailedException.class,
|
||||
org.elasticsearch.node.NodeClosedException.class,
|
||||
org.elasticsearch.search.aggregations.AggregationExecutionException.class,
|
||||
org.elasticsearch.ElasticsearchParseException.class,
|
||||
org.elasticsearch.action.PrimaryMissingActionException.class,
|
||||
org.elasticsearch.index.engine.CreateFailedEngineException.class,
|
||||
org.elasticsearch.index.shard.IllegalIndexShardStateException.class
|
||||
};
|
||||
Map<String, Constructor<? extends ElasticsearchException>> mapping = new HashMap<>();
|
||||
|
||||
for (Class<? extends ElasticsearchException> e : exceptions) {
|
||||
String name = e.getName();
|
||||
try {
|
||||
Constructor<? extends ElasticsearchException> constructor = e.getDeclaredConstructor(StreamInput.class);
|
||||
if (constructor == null) {
|
||||
throw new IllegalStateException(name + " has not StreamInput ctor");
|
||||
}
|
||||
mapping.put(name, constructor);
|
||||
} catch (NoSuchMethodException t) {
|
||||
throw new RuntimeException("failed to register [" + name + "] ", t);
|
||||
}
|
||||
}
|
||||
|
||||
MAPPING = Collections.unmodifiableMap(mapping);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -19,6 +19,10 @@
|
||||
|
||||
package org.elasticsearch;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* A generic exception indicating failure to generate.
|
||||
*
|
||||
@ -33,4 +37,8 @@ public class ElasticsearchGenerationException extends ElasticsearchException {
|
||||
public ElasticsearchGenerationException(String msg, Throwable cause) {
|
||||
super(msg, cause);
|
||||
}
|
||||
|
||||
public ElasticsearchGenerationException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -1,38 +0,0 @@
|
||||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class ElasticsearchNullPointerException extends ElasticsearchException {
|
||||
|
||||
public ElasticsearchNullPointerException() {
|
||||
super(null);
|
||||
}
|
||||
|
||||
public ElasticsearchNullPointerException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
|
||||
public ElasticsearchNullPointerException(String msg, Throwable cause) {
|
||||
super(msg, cause);
|
||||
}
|
||||
}
|
@ -19,8 +19,11 @@
|
||||
|
||||
package org.elasticsearch;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -34,6 +37,9 @@ public class ElasticsearchParseException extends ElasticsearchException {
|
||||
super(msg, cause);
|
||||
}
|
||||
|
||||
public ElasticsearchParseException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
}
|
||||
@Override
|
||||
public RestStatus status() {
|
||||
return RestStatus.BAD_REQUEST;
|
||||
|
@ -19,12 +19,19 @@
|
||||
|
||||
package org.elasticsearch;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* The same as {@link java.util.concurrent.TimeoutException} simply a runtime one.
|
||||
*
|
||||
*
|
||||
*/
|
||||
public class ElasticsearchTimeoutException extends ElasticsearchException {
|
||||
public ElasticsearchTimeoutException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
}
|
||||
|
||||
public ElasticsearchTimeoutException(String message) {
|
||||
super(message);
|
||||
|
@ -20,6 +20,10 @@
|
||||
package org.elasticsearch.action;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
@ -36,4 +40,15 @@ public class FailedNodeException extends ElasticsearchException {
|
||||
public String nodeId() {
|
||||
return this.nodeId;
|
||||
}
|
||||
|
||||
public FailedNodeException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
nodeId = in.readOptionalString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeOptionalString(nodeId);
|
||||
}
|
||||
}
|
||||
|
@ -19,10 +19,13 @@
|
||||
|
||||
package org.elasticsearch.action;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.shard.IndexShardException;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -44,4 +47,8 @@ public class NoShardAvailableActionException extends IndexShardException {
|
||||
public RestStatus status() {
|
||||
return RestStatus.SERVICE_UNAVAILABLE;
|
||||
}
|
||||
|
||||
public NoShardAvailableActionException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
@ -19,6 +19,11 @@
|
||||
|
||||
package org.elasticsearch.action;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -27,4 +32,8 @@ public class NoSuchNodeException extends FailedNodeException {
|
||||
public NoSuchNodeException(String nodeId) {
|
||||
super(nodeId, "No such node [" + nodeId + "]", null);
|
||||
}
|
||||
|
||||
public NoSuchNodeException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -20,6 +20,9 @@
|
||||
package org.elasticsearch.action;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
@ -29,4 +32,8 @@ public class PrimaryMissingActionException extends ElasticsearchException {
|
||||
public PrimaryMissingActionException(String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
public PrimaryMissingActionException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -20,8 +20,12 @@
|
||||
package org.elasticsearch.action;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -56,4 +60,19 @@ public class RoutingMissingException extends ElasticsearchException {
|
||||
public RestStatus status() {
|
||||
return RestStatus.BAD_REQUEST;
|
||||
}
|
||||
|
||||
public RoutingMissingException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
index = in.readString();
|
||||
type = in.readString();
|
||||
id = in.readString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeString(index);
|
||||
out.writeString(type);
|
||||
out.writeString(id);
|
||||
}
|
||||
}
|
||||
|
@ -20,6 +20,10 @@
|
||||
package org.elasticsearch.action;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*/
|
||||
@ -40,4 +44,15 @@ public class TimestampParsingException extends ElasticsearchException {
|
||||
public String timestamp() {
|
||||
return timestamp;
|
||||
}
|
||||
|
||||
public TimestampParsingException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
this.timestamp = in.readOptionalString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeOptionalString(timestamp);
|
||||
}
|
||||
}
|
@ -21,9 +21,12 @@ package org.elasticsearch.action;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -40,6 +43,10 @@ public class UnavailableShardsException extends ElasticsearchException {
|
||||
return "[" + shardId.index().name() + "][" + shardId.id() + "] " + message;
|
||||
}
|
||||
|
||||
public UnavailableShardsException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RestStatus status() {
|
||||
return RestStatus.SERVICE_UNAVAILABLE;
|
||||
|
@ -19,6 +19,10 @@
|
||||
|
||||
package org.elasticsearch.action.search;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* A failure during a reduce phase (when receiving results from several shards, and reducing them
|
||||
* into one or more results and possible actions).
|
||||
@ -27,11 +31,11 @@ package org.elasticsearch.action.search;
|
||||
*/
|
||||
public class ReduceSearchPhaseException extends SearchPhaseExecutionException {
|
||||
|
||||
public ReduceSearchPhaseException(String phaseName, String msg, ShardSearchFailure[] shardFailures) {
|
||||
super(phaseName, "[reduce] " + msg, shardFailures);
|
||||
}
|
||||
|
||||
public ReduceSearchPhaseException(String phaseName, String msg, Throwable cause, ShardSearchFailure[] shardFailures) {
|
||||
super(phaseName, "[reduce] " + msg, cause, shardFailures);
|
||||
}
|
||||
|
||||
public ReduceSearchPhaseException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -22,6 +22,8 @@ package org.elasticsearch.action.search;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.action.ShardOperationFailedException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.IndexException;
|
||||
@ -36,7 +38,7 @@ import java.util.*;
|
||||
public class SearchPhaseExecutionException extends ElasticsearchException {
|
||||
private final String phaseName;
|
||||
|
||||
private ShardSearchFailure[] shardFailures;
|
||||
private final ShardSearchFailure[] shardFailures;
|
||||
|
||||
public SearchPhaseExecutionException(String phaseName, String msg, ShardSearchFailure[] shardFailures) {
|
||||
super(msg);
|
||||
@ -50,6 +52,28 @@ public class SearchPhaseExecutionException extends ElasticsearchException {
|
||||
this.shardFailures = shardFailures;
|
||||
}
|
||||
|
||||
public SearchPhaseExecutionException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
phaseName = in.readOptionalString();
|
||||
int numFailures = in.readVInt();
|
||||
shardFailures = new ShardSearchFailure[numFailures];
|
||||
for (int i = 0; i < numFailures; i++) {
|
||||
shardFailures[i] = ShardSearchFailure.readShardSearchFailure(in);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeOptionalString(phaseName);
|
||||
out.writeVInt(shardFailures == null ? 0 : shardFailures.length);
|
||||
if (shardFailures != null) {
|
||||
for (ShardSearchFailure failure : shardFailures) {
|
||||
failure.writeTo(out);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public RestStatus status() {
|
||||
if (shardFailures.length == 0) {
|
||||
|
@ -56,8 +56,8 @@ public abstract class HandledTransportAction<Request extends ActionRequest, Resp
|
||||
try {
|
||||
channel.sendResponse(e);
|
||||
} catch (Exception e1) {
|
||||
logger.warn("Failed to send error response for action [{}] and request [{}]",
|
||||
actionName, request, e1);
|
||||
logger.warn("Failed to send error response for action [{}] and request [{}]", e1,
|
||||
actionName, request);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
@ -20,9 +20,12 @@
|
||||
package org.elasticsearch.action.support.broadcast;
|
||||
|
||||
import org.elasticsearch.ElasticsearchWrapperException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.shard.IndexShardException;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* An exception indicating that a failure occurred performing an operation on the shard.
|
||||
*
|
||||
@ -41,4 +44,8 @@ public class BroadcastShardOperationFailedException extends IndexShardException
|
||||
public BroadcastShardOperationFailedException(ShardId shardId, String msg, Throwable cause) {
|
||||
super(shardId, msg, cause);
|
||||
}
|
||||
|
||||
public BroadcastShardOperationFailedException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
@ -1,44 +0,0 @@
|
||||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.action.support.replication;
|
||||
|
||||
import org.elasticsearch.ElasticsearchWrapperException;
|
||||
import org.elasticsearch.index.shard.IndexShardException;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
/**
|
||||
* An exception indicating that a failure occurred performing an operation on the shard.
|
||||
*
|
||||
*
|
||||
*/
|
||||
public class ReplicationShardOperationFailedException extends IndexShardException implements ElasticsearchWrapperException {
|
||||
|
||||
public ReplicationShardOperationFailedException(ShardId shardId, String msg) {
|
||||
super(shardId, msg, null);
|
||||
}
|
||||
|
||||
public ReplicationShardOperationFailedException(ShardId shardId, Throwable cause) {
|
||||
super(shardId, "", cause);
|
||||
}
|
||||
|
||||
public ReplicationShardOperationFailedException(ShardId shardId, String msg, Throwable cause) {
|
||||
super(shardId, msg, cause);
|
||||
}
|
||||
}
|
@ -43,6 +43,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.routing.*;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.collect.Tuple;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.lease.Releasable;
|
||||
import org.elasticsearch.common.lease.Releasables;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
@ -239,14 +240,14 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
|
||||
}
|
||||
}
|
||||
|
||||
protected static class RetryOnReplicaException extends IndexShardException {
|
||||
public static class RetryOnReplicaException extends IndexShardException {
|
||||
|
||||
public RetryOnReplicaException(ShardId shardId, String msg) {
|
||||
super(shardId, msg);
|
||||
}
|
||||
|
||||
public RetryOnReplicaException(ShardId shardId, String msg, Throwable cause) {
|
||||
super(shardId, msg, cause);
|
||||
public RetryOnReplicaException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
||||
@ -322,14 +323,13 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
|
||||
}
|
||||
}
|
||||
|
||||
protected static class RetryOnPrimaryException extends IndexShardException {
|
||||
|
||||
public static class RetryOnPrimaryException extends IndexShardException {
|
||||
public RetryOnPrimaryException(ShardId shardId, String msg) {
|
||||
super(shardId, msg);
|
||||
}
|
||||
|
||||
public RetryOnPrimaryException(ShardId shardId, String msg, Throwable cause) {
|
||||
super(shardId, msg, cause);
|
||||
public RetryOnPrimaryException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -20,8 +20,11 @@
|
||||
package org.elasticsearch.client.transport;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* An exception indicating no node is available to perform the operation.
|
||||
*/
|
||||
@ -35,6 +38,10 @@ public class NoNodeAvailableException extends ElasticsearchException {
|
||||
super(message, t);
|
||||
}
|
||||
|
||||
public NoNodeAvailableException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RestStatus status() {
|
||||
return RestStatus.SERVICE_UNAVAILABLE;
|
||||
|
@ -20,6 +20,9 @@
|
||||
package org.elasticsearch.cluster;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Thrown by {@link Diffable#readDiffAndApply(org.elasticsearch.common.io.stream.StreamInput)} method
|
||||
@ -32,4 +35,8 @@ public class IncompatibleClusterStateVersionException extends ElasticsearchExcep
|
||||
public IncompatibleClusterStateVersionException(long expectedVersion, String expectedUuid, long receivedVersion, String receivedUuid) {
|
||||
super("Expected diff for version " + expectedVersion + " with uuid " + expectedUuid + " got version " + receivedVersion + " and uuid " + receivedUuid);
|
||||
}
|
||||
|
||||
public IncompatibleClusterStateVersionException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -21,8 +21,13 @@ package org.elasticsearch.cluster.block;
|
||||
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -35,6 +40,29 @@ public class ClusterBlockException extends ElasticsearchException {
|
||||
this.blocks = blocks;
|
||||
}
|
||||
|
||||
public ClusterBlockException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
int num = in.readVInt();
|
||||
ImmutableSet.Builder<ClusterBlock> builder = ImmutableSet.builder();
|
||||
for (int i = 0; i < num; i++) {
|
||||
builder.add(ClusterBlock.readClusterBlock(in));
|
||||
}
|
||||
blocks = builder.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
if (blocks != null) {
|
||||
out.writeVInt(blocks.size());
|
||||
for (ClusterBlock block : blocks) {
|
||||
block.writeTo(out);
|
||||
}
|
||||
} else {
|
||||
out.writeVInt(0);
|
||||
}
|
||||
}
|
||||
|
||||
public boolean retryable() {
|
||||
for (ClusterBlock block : blocks) {
|
||||
if (!block.retryable()) {
|
||||
|
@ -20,9 +20,12 @@
|
||||
package org.elasticsearch.cluster.metadata;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ProcessClusterEventTimeoutException extends ElasticsearchException {
|
||||
@ -31,6 +34,10 @@ public class ProcessClusterEventTimeoutException extends ElasticsearchException
|
||||
super("failed to process cluster event (" + source + ") within " + timeValue);
|
||||
}
|
||||
|
||||
public ProcessClusterEventTimeoutException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RestStatus status() {
|
||||
return RestStatus.SERVICE_UNAVAILABLE;
|
||||
|
@ -39,7 +39,7 @@ import static org.elasticsearch.common.transport.TransportAddressSerializers.add
|
||||
/**
|
||||
* A discovery node represents a node that is part of the cluster.
|
||||
*/
|
||||
public class DiscoveryNode implements Streamable, Serializable {
|
||||
public class DiscoveryNode implements Streamable {
|
||||
|
||||
/**
|
||||
* Minimum version of a node to communicate with. This version corresponds to the minimum compatibility version
|
||||
@ -373,19 +373,4 @@ public class DiscoveryNode implements Streamable, Serializable {
|
||||
}
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
// we need this custom serialization logic because Version is not serializable (because org.apache.lucene.util.Version is not serializable)
|
||||
private void writeObject(java.io.ObjectOutputStream out)
|
||||
throws IOException {
|
||||
StreamOutput streamOutput = new OutputStreamStreamOutput(out);
|
||||
streamOutput.setVersion(Version.CURRENT.minimumCompatibilityVersion());
|
||||
this.writeTo(streamOutput);
|
||||
}
|
||||
|
||||
private void readObject(java.io.ObjectInputStream in)
|
||||
throws IOException, ClassNotFoundException {
|
||||
StreamInput streamInput = new InputStreamStreamInput(in);
|
||||
streamInput.setVersion(Version.CURRENT.minimumCompatibilityVersion());
|
||||
this.readFrom(streamInput);
|
||||
}
|
||||
}
|
||||
|
@ -19,6 +19,11 @@
|
||||
|
||||
package org.elasticsearch.cluster.routing;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* This exception defines illegal states of shard routing
|
||||
*/
|
||||
@ -35,6 +40,17 @@ public class IllegalShardRoutingStateException extends RoutingException {
|
||||
this.shard = shard;
|
||||
}
|
||||
|
||||
public IllegalShardRoutingStateException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
shard = ShardRouting.readShardRoutingEntry(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
shard.writeTo(out);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the shard instance referenced by this exception
|
||||
* @return shard instance referenced by this exception
|
||||
|
@ -20,6 +20,9 @@
|
||||
package org.elasticsearch.cluster.routing;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* A base {@link Exception}s for all exceptions thrown by routing related operations.
|
||||
@ -33,4 +36,8 @@ public class RoutingException extends ElasticsearchException {
|
||||
public RoutingException(String message, Throwable cause) {
|
||||
super(message, cause);
|
||||
}
|
||||
|
||||
public RoutingException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
@ -19,6 +19,11 @@
|
||||
|
||||
package org.elasticsearch.cluster.routing;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* This class defines {@link RoutingException}s related to
|
||||
* the validation of routing
|
||||
@ -32,6 +37,17 @@ public class RoutingValidationException extends RoutingException {
|
||||
this.validation = validation;
|
||||
}
|
||||
|
||||
public RoutingValidationException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
validation = in.readOptionalStreamable(new RoutingTableValidation());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeOptionalStreamable(validation);
|
||||
}
|
||||
|
||||
public RoutingTableValidation validation() {
|
||||
return this.validation;
|
||||
}
|
||||
|
@ -632,116 +632,6 @@ public class Base64 {
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Serializes an object and returns the Base64-encoded
|
||||
* version of that serialized object.
|
||||
* <p/>
|
||||
* <p>As of v 2.3, if the object
|
||||
* cannot be serialized or there is another error,
|
||||
* the method will throw an java.io.IOException. <b>This is new to v2.3!</b>
|
||||
* In earlier versions, it just returned a null value, but
|
||||
* in retrospect that's a pretty poor way to handle it.</p>
|
||||
* <p/>
|
||||
* The object is not GZip-compressed before being encoded.
|
||||
*
|
||||
* @param serializableObject The object to encode
|
||||
* @return The Base64-encoded object
|
||||
* @throws java.io.IOException if there is an error
|
||||
* @throws NullPointerException if serializedObject is null
|
||||
* @since 1.4
|
||||
*/
|
||||
public static String encodeObject(java.io.Serializable serializableObject)
|
||||
throws java.io.IOException {
|
||||
return encodeObject(serializableObject, NO_OPTIONS);
|
||||
} // end encodeObject
|
||||
|
||||
|
||||
/**
|
||||
* Serializes an object and returns the Base64-encoded
|
||||
* version of that serialized object.
|
||||
* <p/>
|
||||
* <p>As of v 2.3, if the object
|
||||
* cannot be serialized or there is another error,
|
||||
* the method will throw an java.io.IOException. <b>This is new to v2.3!</b>
|
||||
* In earlier versions, it just returned a null value, but
|
||||
* in retrospect that's a pretty poor way to handle it.</p>
|
||||
* <p/>
|
||||
* The object is not GZip-compressed before being encoded.
|
||||
* <p/>
|
||||
* Example options:<pre>
|
||||
* GZIP: gzip-compresses object before encoding it.
|
||||
* DO_BREAK_LINES: break lines at 76 characters
|
||||
* </pre>
|
||||
* <p/>
|
||||
* Example: <code>encodeObject( myObj, Base64.GZIP )</code> or
|
||||
* <p/>
|
||||
* Example: <code>encodeObject( myObj, Base64.GZIP | Base64.DO_BREAK_LINES )</code>
|
||||
*
|
||||
* @param serializableObject The object to encode
|
||||
* @param options Specified options
|
||||
* @return The Base64-encoded object
|
||||
* @throws java.io.IOException if there is an error
|
||||
* @see Base64#GZIP
|
||||
* @see Base64#DO_BREAK_LINES
|
||||
* @since 2.0
|
||||
*/
|
||||
public static String encodeObject(java.io.Serializable serializableObject, int options)
|
||||
throws java.io.IOException {
|
||||
|
||||
if (serializableObject == null) {
|
||||
throw new NullPointerException("Cannot serialize a null object.");
|
||||
} // end if: null
|
||||
|
||||
// Streams
|
||||
java.io.ByteArrayOutputStream baos = null;
|
||||
java.io.OutputStream b64os = null;
|
||||
java.util.zip.GZIPOutputStream gzos = null;
|
||||
java.io.ObjectOutputStream oos = null;
|
||||
|
||||
|
||||
try {
|
||||
// ObjectOutputStream -> (GZIP) -> Base64 -> ByteArrayOutputStream
|
||||
baos = new java.io.ByteArrayOutputStream();
|
||||
b64os = new Base64.OutputStream(baos, ENCODE | options);
|
||||
if ((options & GZIP) != 0) {
|
||||
// Gzip
|
||||
gzos = new java.util.zip.GZIPOutputStream(b64os);
|
||||
oos = new java.io.ObjectOutputStream(gzos);
|
||||
} else {
|
||||
// Not gzipped
|
||||
oos = new java.io.ObjectOutputStream(b64os);
|
||||
}
|
||||
oos.writeObject(serializableObject);
|
||||
} // 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 {
|
||||
oos.close();
|
||||
} catch (Exception e) {
|
||||
}
|
||||
try {
|
||||
gzos.close();
|
||||
} catch (Exception e) {
|
||||
}
|
||||
try {
|
||||
b64os.close();
|
||||
} catch (Exception e) {
|
||||
}
|
||||
try {
|
||||
baos.close();
|
||||
} catch (Exception e) {
|
||||
}
|
||||
} // end finally
|
||||
|
||||
// Return value according to relevant encoding.
|
||||
return new String(baos.toByteArray(), PREFERRED_ENCODING);
|
||||
|
||||
} // end encode
|
||||
|
||||
|
||||
/**
|
||||
* Encodes a byte array into Base64 notation.
|
||||
@ -799,7 +689,6 @@ public class Base64 {
|
||||
return encodeBytes(source, 0, source.length, options);
|
||||
} // end encodeBytes
|
||||
|
||||
|
||||
/**
|
||||
* Encodes a byte array into Base64 notation.
|
||||
* Does not GZip-compress data.
|
||||
|
@ -20,6 +20,9 @@
|
||||
package org.elasticsearch.common.blobstore;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
@ -33,4 +36,8 @@ public class BlobStoreException extends ElasticsearchException {
|
||||
public BlobStoreException(String msg, Throwable cause) {
|
||||
super(msg, cause);
|
||||
}
|
||||
|
||||
public BlobStoreException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -19,6 +19,10 @@
|
||||
package org.elasticsearch.common.breaker;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Exception thrown when the circuit breaker trips
|
||||
@ -34,12 +38,25 @@ public class CircuitBreakingException extends ElasticsearchException {
|
||||
this.byteLimit = 0;
|
||||
}
|
||||
|
||||
public CircuitBreakingException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
byteLimit = in.readLong();
|
||||
bytesWanted = in.readLong();
|
||||
}
|
||||
|
||||
public CircuitBreakingException(String message, long bytesWanted, long byteLimit) {
|
||||
super(message);
|
||||
this.bytesWanted = bytesWanted;
|
||||
this.byteLimit = byteLimit;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeLong(byteLimit);
|
||||
out.writeLong(bytesWanted);
|
||||
}
|
||||
|
||||
public long getBytesWanted() {
|
||||
return this.bytesWanted;
|
||||
}
|
||||
|
@ -1,99 +0,0 @@
|
||||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.common.io;
|
||||
|
||||
import org.elasticsearch.common.Classes;
|
||||
|
||||
import java.io.*;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class ThrowableObjectInputStream extends ObjectInputStream {
|
||||
|
||||
private final ClassLoader classLoader;
|
||||
|
||||
public ThrowableObjectInputStream(InputStream in) throws IOException {
|
||||
this(in, null);
|
||||
}
|
||||
|
||||
public ThrowableObjectInputStream(InputStream in, ClassLoader classLoader) throws IOException {
|
||||
super(in);
|
||||
this.classLoader = classLoader;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void readStreamHeader() throws IOException, StreamCorruptedException {
|
||||
int version = readByte() & 0xFF;
|
||||
if (version != STREAM_VERSION) {
|
||||
throw new StreamCorruptedException(
|
||||
"Unsupported version: " + version);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ObjectStreamClass readClassDescriptor()
|
||||
throws IOException, ClassNotFoundException {
|
||||
int type = read();
|
||||
if (type < 0) {
|
||||
throw new EOFException();
|
||||
}
|
||||
switch (type) {
|
||||
case ThrowableObjectOutputStream.TYPE_EXCEPTION:
|
||||
return ObjectStreamClass.lookup(Exception.class);
|
||||
case ThrowableObjectOutputStream.TYPE_STACKTRACEELEMENT:
|
||||
return ObjectStreamClass.lookup(StackTraceElement.class);
|
||||
case ThrowableObjectOutputStream.TYPE_FAT_DESCRIPTOR:
|
||||
return super.readClassDescriptor();
|
||||
case ThrowableObjectOutputStream.TYPE_THIN_DESCRIPTOR:
|
||||
String className = readUTF();
|
||||
Class<?> clazz = loadClass(className);
|
||||
return ObjectStreamClass.lookup(clazz);
|
||||
default:
|
||||
throw new StreamCorruptedException(
|
||||
"Unexpected class descriptor type: " + type);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Class<?> resolveClass(ObjectStreamClass desc) throws IOException, ClassNotFoundException {
|
||||
String className = desc.getName();
|
||||
try {
|
||||
return loadClass(className);
|
||||
} catch (ClassNotFoundException ex) {
|
||||
return super.resolveClass(desc);
|
||||
}
|
||||
}
|
||||
|
||||
protected Class<?> loadClass(String className) throws ClassNotFoundException {
|
||||
Class<?> clazz;
|
||||
ClassLoader classLoader = this.classLoader;
|
||||
if (classLoader == null) {
|
||||
classLoader = Classes.getDefaultClassLoader();
|
||||
}
|
||||
|
||||
if (classLoader != null) {
|
||||
clazz = classLoader.loadClass(className);
|
||||
} else {
|
||||
clazz = Class.forName(className);
|
||||
}
|
||||
return clazz;
|
||||
}
|
||||
}
|
@ -1,68 +0,0 @@
|
||||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.common.io;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.ObjectOutputStream;
|
||||
import java.io.ObjectStreamClass;
|
||||
import java.io.OutputStream;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class ThrowableObjectOutputStream extends ObjectOutputStream {
|
||||
|
||||
static final int TYPE_FAT_DESCRIPTOR = 0;
|
||||
static final int TYPE_THIN_DESCRIPTOR = 1;
|
||||
|
||||
private static final String EXCEPTION_CLASSNAME = Exception.class.getName();
|
||||
static final int TYPE_EXCEPTION = 2;
|
||||
|
||||
private static final String STACKTRACEELEMENT_CLASSNAME = StackTraceElement.class.getName();
|
||||
static final int TYPE_STACKTRACEELEMENT = 3;
|
||||
|
||||
|
||||
public ThrowableObjectOutputStream(OutputStream out) throws IOException {
|
||||
super(out);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void writeStreamHeader() throws IOException {
|
||||
writeByte(STREAM_VERSION);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void writeClassDescriptor(ObjectStreamClass desc) throws IOException {
|
||||
if (desc.getName().equals(EXCEPTION_CLASSNAME)) {
|
||||
write(TYPE_EXCEPTION);
|
||||
} else if (desc.getName().equals(STACKTRACEELEMENT_CLASSNAME)) {
|
||||
write(TYPE_STACKTRACEELEMENT);
|
||||
} else {
|
||||
Class<?> clazz = desc.forClass();
|
||||
if (clazz.isPrimitive() || clazz.isArray()) {
|
||||
write(TYPE_FAT_DESCRIPTOR);
|
||||
super.writeClassDescriptor(desc);
|
||||
} else {
|
||||
write(TYPE_THIN_DESCRIPTOR);
|
||||
writeUTF(desc.getName());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -19,8 +19,12 @@
|
||||
|
||||
package org.elasticsearch.common.io.stream;
|
||||
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.IndexFormatTooNewException;
|
||||
import org.apache.lucene.index.IndexFormatTooOldException;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CharsRefBuilder;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.Strings;
|
||||
@ -31,11 +35,9 @@ import org.elasticsearch.common.text.Text;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.DateTimeZone;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.ObjectInputStream;
|
||||
import java.io.*;
|
||||
import java.util.*;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
/**
|
||||
*
|
||||
@ -480,12 +482,51 @@ public abstract class StreamInput extends InputStream {
|
||||
}
|
||||
|
||||
public <T extends Throwable> T readThrowable() throws IOException {
|
||||
try {
|
||||
ObjectInputStream oin = new ObjectInputStream(this);
|
||||
return (T) oin.readObject();
|
||||
} catch (ClassNotFoundException e) {
|
||||
throw new IOException("failed to deserialize exception", e);
|
||||
if (readBoolean()) {
|
||||
int key = readVInt();
|
||||
switch (key) {
|
||||
case 0:
|
||||
return (T) ElasticsearchException.readException(this);
|
||||
case 1:
|
||||
// nocommit - this sucks it would be nice to have a better way to construct those?
|
||||
String msg = readOptionalString();
|
||||
final int idx = msg.indexOf(" (resource=");
|
||||
String resource = msg.substring(idx + " (resource=".length(), msg.length()-1);
|
||||
msg = msg.substring(0, idx);
|
||||
return (T) ElasticsearchException.readStackTrace(new CorruptIndexException(msg, resource, readThrowable()), this); // TODO add a string throwable ctor to this?
|
||||
case 2:
|
||||
String itnMessage = readOptionalString();
|
||||
readThrowable();
|
||||
return (T) ElasticsearchException.readStackTrace(new IndexFormatTooNewException(itnMessage, -1, -1, -1), this);
|
||||
case 3:
|
||||
String itoMessage = readOptionalString();
|
||||
readThrowable();
|
||||
return (T) ElasticsearchException.readStackTrace(new IndexFormatTooOldException(itoMessage, -1, -1, -1), this);
|
||||
case 4:
|
||||
String npeMessage = readOptionalString();
|
||||
readThrowable();
|
||||
return (T) ElasticsearchException.readStackTrace(new NullPointerException(npeMessage), this);
|
||||
case 5:
|
||||
String nfeMessage = readOptionalString();
|
||||
readThrowable();
|
||||
return (T) ElasticsearchException.readStackTrace(new NumberFormatException(nfeMessage), this);
|
||||
case 6:
|
||||
return (T) ElasticsearchException.readStackTrace(new IllegalArgumentException(readOptionalString(), readThrowable()), this);
|
||||
case 7:
|
||||
return (T) ElasticsearchException.readStackTrace(new IllegalStateException(readOptionalString(), readThrowable()), this);
|
||||
case 8:
|
||||
String eofMessage = readOptionalString();
|
||||
readThrowable();
|
||||
return (T) ElasticsearchException.readStackTrace(new EOFException(eofMessage), this);
|
||||
case 9:
|
||||
return (T) ElasticsearchException.readStackTrace(new SecurityException(readOptionalString(), readThrowable()), this);
|
||||
case 10: // unknown -- // nocommit - should we use a dedicated exception
|
||||
return (T) ElasticsearchException.readStackTrace(new ElasticsearchException(readOptionalString(), readThrowable()), this);
|
||||
default:
|
||||
assert false : "no such exception for id: " + key;
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
public static StreamInput wrap(BytesReference reference) {
|
||||
@ -502,4 +543,5 @@ public abstract class StreamInput extends InputStream {
|
||||
public static StreamInput wrap(byte[] bytes, int offset, int length) {
|
||||
return new InputStreamStreamInput(new ByteArrayInputStream(bytes, offset, length));
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -19,14 +19,20 @@
|
||||
|
||||
package org.elasticsearch.common.io.stream;
|
||||
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.IndexFormatTooNewException;
|
||||
import org.apache.lucene.index.IndexFormatTooOldException;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.bootstrap.Elasticsearch;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.text.Text;
|
||||
import org.joda.time.ReadableInstant;
|
||||
|
||||
import java.io.EOFException;
|
||||
import java.io.IOException;
|
||||
import java.io.ObjectOutputStream;
|
||||
import java.io.OutputStream;
|
||||
@ -445,8 +451,38 @@ public abstract class StreamOutput extends OutputStream {
|
||||
}
|
||||
|
||||
public void writeThrowable(Throwable throwable) throws IOException {
|
||||
ObjectOutputStream out = new ObjectOutputStream(this);
|
||||
out.writeObject(throwable);
|
||||
out.flush();
|
||||
if (throwable == null) {
|
||||
writeBoolean(false);
|
||||
} else {
|
||||
writeBoolean(true);
|
||||
if (throwable instanceof ElasticsearchException) {
|
||||
writeVInt(0);
|
||||
ElasticsearchException.writeException((ElasticsearchException) throwable, this);
|
||||
return;
|
||||
} else if (throwable instanceof CorruptIndexException) {
|
||||
writeVInt(1);
|
||||
} else if (throwable instanceof IndexFormatTooNewException) {
|
||||
writeVInt(2);
|
||||
} else if (throwable instanceof IndexFormatTooOldException) {
|
||||
writeVInt(3);
|
||||
} else if (throwable instanceof NullPointerException) {
|
||||
writeVInt(4);
|
||||
} else if (throwable instanceof NumberFormatException) {
|
||||
writeVInt(5);
|
||||
} else if (throwable instanceof IllegalArgumentException) {
|
||||
writeVInt(6);
|
||||
} else if (throwable instanceof IllegalStateException) {
|
||||
writeVInt(7);
|
||||
} else if (throwable instanceof EOFException) {
|
||||
writeVInt(8);
|
||||
} else if (throwable instanceof SecurityException) {
|
||||
writeVInt(9);
|
||||
} else {
|
||||
writeVInt(10); // unknown
|
||||
}
|
||||
writeOptionalString(throwable.getMessage());
|
||||
writeThrowable(throwable.getCause());
|
||||
ElasticsearchException.writeStackTraces(throwable, this);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -604,6 +604,10 @@ public class Lucene {
|
||||
public EarlyTerminationException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
|
||||
public EarlyTerminationException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -19,6 +19,10 @@
|
||||
|
||||
package org.elasticsearch.common.settings;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* A specific type of {@link SettingsException} indicating failure to load a class
|
||||
* based on a settings value.
|
||||
@ -27,11 +31,11 @@ package org.elasticsearch.common.settings;
|
||||
*/
|
||||
public class NoClassSettingsException extends SettingsException {
|
||||
|
||||
public NoClassSettingsException(String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
public NoClassSettingsException(String message, Throwable cause) {
|
||||
super(message, cause);
|
||||
}
|
||||
|
||||
public NoClassSettingsException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -20,6 +20,9 @@
|
||||
package org.elasticsearch.common.settings;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* A generic failure to handle settings.
|
||||
@ -35,4 +38,8 @@ public class SettingsException extends ElasticsearchException {
|
||||
public SettingsException(String message, Throwable cause) {
|
||||
super(message, cause);
|
||||
}
|
||||
|
||||
public SettingsException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -60,12 +60,6 @@ public class InetSocketTransportAddress implements TransportAddress {
|
||||
this.address = address;
|
||||
}
|
||||
|
||||
public static InetSocketTransportAddress readInetSocketTransportAddress(StreamInput in) throws IOException {
|
||||
InetSocketTransportAddress address = new InetSocketTransportAddress();
|
||||
address.readFrom(in);
|
||||
return address;
|
||||
}
|
||||
|
||||
@Override
|
||||
public short uniqueAddressTypeId() {
|
||||
return 1;
|
||||
|
@ -26,7 +26,7 @@ import java.io.Serializable;
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public interface TransportAddress extends Streamable, Serializable {
|
||||
public interface TransportAddress extends Streamable {
|
||||
|
||||
short uniqueAddressTypeId();
|
||||
|
||||
|
@ -20,7 +20,9 @@ package org.elasticsearch.common.util;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
@ -132,14 +134,14 @@ public class CancellableThreads {
|
||||
public void run() throws InterruptedException;
|
||||
}
|
||||
|
||||
public class ExecutionCancelledException extends ElasticsearchException {
|
||||
public static class ExecutionCancelledException extends ElasticsearchException {
|
||||
|
||||
public ExecutionCancelledException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
|
||||
public ExecutionCancelledException(String msg, Throwable cause) {
|
||||
super(msg, cause);
|
||||
public ExecutionCancelledException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -20,8 +20,11 @@
|
||||
package org.elasticsearch.common.util.concurrent;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class EsRejectedExecutionException extends ElasticsearchException {
|
||||
@ -31,7 +34,7 @@ public class EsRejectedExecutionException extends ElasticsearchException {
|
||||
}
|
||||
|
||||
public EsRejectedExecutionException() {
|
||||
super(null);
|
||||
super((String)null);
|
||||
}
|
||||
|
||||
public EsRejectedExecutionException(Throwable e) {
|
||||
@ -42,4 +45,8 @@ public class EsRejectedExecutionException extends ElasticsearchException {
|
||||
public RestStatus status() {
|
||||
return RestStatus.TOO_MANY_REQUESTS;
|
||||
}
|
||||
|
||||
public EsRejectedExecutionException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -20,6 +20,9 @@
|
||||
package org.elasticsearch.common.util.concurrent;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
@ -29,4 +32,8 @@ public class UncategorizedExecutionException extends ElasticsearchException {
|
||||
public UncategorizedExecutionException(String msg, Throwable cause) {
|
||||
super(msg, cause);
|
||||
}
|
||||
|
||||
public UncategorizedExecutionException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -20,8 +20,11 @@
|
||||
package org.elasticsearch.discovery;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -39,4 +42,8 @@ public class MasterNotDiscoveredException extends ElasticsearchException {
|
||||
public RestStatus status() {
|
||||
return RestStatus.SERVICE_UNAVAILABLE;
|
||||
}
|
||||
|
||||
public MasterNotDiscoveredException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -20,6 +20,9 @@
|
||||
package org.elasticsearch.env;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
@ -33,4 +36,8 @@ public class FailedToResolveConfigException extends ElasticsearchException {
|
||||
public FailedToResolveConfigException(String msg, Throwable cause) {
|
||||
super(msg, cause);
|
||||
}
|
||||
|
||||
public FailedToResolveConfigException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -20,6 +20,9 @@
|
||||
package org.elasticsearch.gateway;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
@ -33,4 +36,8 @@ public class GatewayException extends ElasticsearchException {
|
||||
public GatewayException(String msg, Throwable cause) {
|
||||
super(msg, cause);
|
||||
}
|
||||
|
||||
public GatewayException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -19,16 +19,20 @@
|
||||
|
||||
package org.elasticsearch.http;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class BindHttpException extends HttpException {
|
||||
|
||||
public BindHttpException(String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
public BindHttpException(String message, Throwable cause) {
|
||||
super(message, cause);
|
||||
}
|
||||
|
||||
public BindHttpException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
}
|
||||
}
|
@ -20,6 +20,9 @@
|
||||
package org.elasticsearch.http;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
@ -33,4 +36,8 @@ public class HttpException extends ElasticsearchException {
|
||||
public HttpException(String message, Throwable cause) {
|
||||
super(message, cause);
|
||||
}
|
||||
|
||||
public HttpException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
}
|
||||
}
|
@ -20,8 +20,12 @@
|
||||
package org.elasticsearch.index;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.index.engine.IgnoreOnRecoveryEngineException;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class AlreadyExpiredException extends ElasticsearchException implements IgnoreOnRecoveryEngineException {
|
||||
private String index;
|
||||
private String type;
|
||||
@ -63,4 +67,25 @@ public class AlreadyExpiredException extends ElasticsearchException implements I
|
||||
public long now() {
|
||||
return now;
|
||||
}
|
||||
|
||||
public AlreadyExpiredException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
index = in.readOptionalString();
|
||||
type = in.readOptionalString();
|
||||
id = in.readOptionalString();
|
||||
timestamp = in.readLong();
|
||||
ttl = in.readLong();
|
||||
now = in.readLong();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeOptionalString(index);
|
||||
out.writeOptionalString(type);
|
||||
out.writeOptionalString(id);
|
||||
out.writeLong(timestamp);
|
||||
out.writeLong(ttl);
|
||||
out.writeLong(now);
|
||||
}
|
||||
}
|
@ -20,6 +20,8 @@
|
||||
package org.elasticsearch.index;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
|
||||
import java.io.IOException;
|
||||
@ -56,4 +58,16 @@ public class IndexException extends ElasticsearchException {
|
||||
public String toString() {
|
||||
return "[" + (index == null ? "_na" : index.name()) + "] " + getMessage();
|
||||
}
|
||||
|
||||
|
||||
public IndexException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
index = in.readBoolean() ? Index.readIndexName(in) : null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeOptionalStreamable(index);
|
||||
}
|
||||
}
|
||||
|
@ -20,6 +20,9 @@
|
||||
package org.elasticsearch.index;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
@ -29,4 +32,8 @@ public class IndexShardAlreadyExistsException extends ElasticsearchException {
|
||||
public IndexShardAlreadyExistsException(String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
public IndexShardAlreadyExistsException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
}
|
||||
}
|
@ -19,10 +19,13 @@
|
||||
|
||||
package org.elasticsearch.index;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.shard.IndexShardException;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -32,6 +35,10 @@ public class IndexShardMissingException extends IndexShardException {
|
||||
super(shardId, "missing");
|
||||
}
|
||||
|
||||
public IndexShardMissingException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RestStatus status() {
|
||||
return RestStatus.NOT_FOUND;
|
||||
|
@ -1,38 +0,0 @@
|
||||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.index.engine;
|
||||
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
/**
|
||||
* An exception indicating that an {@link org.elasticsearch.index.engine.Engine} close failed.
|
||||
*
|
||||
*
|
||||
*/
|
||||
public class CloseEngineException extends EngineException {
|
||||
|
||||
public CloseEngineException(ShardId shardId, String msg) {
|
||||
super(shardId, msg);
|
||||
}
|
||||
|
||||
public CloseEngineException(ShardId shardId, String msg, Throwable cause) {
|
||||
super(shardId, msg, cause);
|
||||
}
|
||||
}
|
@ -19,8 +19,12 @@
|
||||
|
||||
package org.elasticsearch.index.engine;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -36,6 +40,12 @@ public class CreateFailedEngineException extends EngineException {
|
||||
this.id = create.id();
|
||||
}
|
||||
|
||||
public CreateFailedEngineException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
type = in.readString();
|
||||
id = in.readString();
|
||||
}
|
||||
|
||||
public String type() {
|
||||
return this.type;
|
||||
}
|
||||
@ -43,4 +53,11 @@ public class CreateFailedEngineException extends EngineException {
|
||||
public String id() {
|
||||
return this.id;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeString(type);
|
||||
out.writeString(id);
|
||||
}
|
||||
}
|
||||
|
@ -19,8 +19,11 @@
|
||||
|
||||
package org.elasticsearch.index.engine;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/** @deprecated Delete-by-query is removed in 2.0, but we keep this so translog can replay on upgrade. */
|
||||
@Deprecated
|
||||
public class DeleteByQueryFailedEngineException extends EngineException {
|
||||
@ -28,4 +31,8 @@ public class DeleteByQueryFailedEngineException extends EngineException {
|
||||
public DeleteByQueryFailedEngineException(ShardId shardId, Engine.DeleteByQuery deleteByQuery, Throwable cause) {
|
||||
super(shardId, "Delete by query failed for [" + deleteByQuery.query() + "]", cause);
|
||||
}
|
||||
|
||||
public DeleteByQueryFailedEngineException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -19,8 +19,11 @@
|
||||
|
||||
package org.elasticsearch.index.engine;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -29,4 +32,8 @@ public class DeleteFailedEngineException extends EngineException {
|
||||
public DeleteFailedEngineException(ShardId shardId, Engine.Delete delete, Throwable cause) {
|
||||
super(shardId, "Delete failed for [" + delete.uid().text() + "]", cause);
|
||||
}
|
||||
|
||||
public DeleteFailedEngineException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
@ -18,9 +18,12 @@
|
||||
*/
|
||||
package org.elasticsearch.index.engine;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -30,6 +33,10 @@ public class DocumentAlreadyExistsException extends EngineException {
|
||||
super(shardId, "[" + type + "][" + id + "]: document already exists");
|
||||
}
|
||||
|
||||
public DocumentAlreadyExistsException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RestStatus status() {
|
||||
return RestStatus.CONFLICT;
|
||||
|
@ -18,9 +18,12 @@
|
||||
*/
|
||||
package org.elasticsearch.index.engine;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -30,6 +33,10 @@ public class DocumentMissingException extends EngineException {
|
||||
super(shardId, "[" + type + "][" + id + "]: document missing");
|
||||
}
|
||||
|
||||
public DocumentMissingException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RestStatus status() {
|
||||
return RestStatus.NOT_FOUND;
|
||||
|
@ -18,9 +18,12 @@
|
||||
*/
|
||||
package org.elasticsearch.index.engine;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -30,6 +33,10 @@ public class DocumentSourceMissingException extends EngineException {
|
||||
super(shardId, "[" + type + "][" + id + "]: document source missing");
|
||||
}
|
||||
|
||||
public DocumentSourceMissingException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RestStatus status() {
|
||||
return RestStatus.BAD_REQUEST;
|
||||
|
@ -1,32 +0,0 @@
|
||||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.index.engine;
|
||||
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class EngineAlreadyStartedException extends EngineException {
|
||||
|
||||
public EngineAlreadyStartedException(ShardId shardId) {
|
||||
super(shardId, "Already started");
|
||||
}
|
||||
}
|
@ -19,9 +19,12 @@
|
||||
|
||||
package org.elasticsearch.index.engine;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.shard.IndexShardClosedException;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* An engine is already closed.
|
||||
* <p/>
|
||||
@ -39,4 +42,8 @@ public class EngineClosedException extends IndexShardClosedException {
|
||||
public EngineClosedException(ShardId shardId, Throwable t) {
|
||||
super(shardId, t);
|
||||
}
|
||||
|
||||
public EngineClosedException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -19,8 +19,11 @@
|
||||
|
||||
package org.elasticsearch.index.engine;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* An exception indicating that an {@link Engine} creation failed.
|
||||
*
|
||||
@ -32,4 +35,8 @@ public class EngineCreationFailureException extends EngineException {
|
||||
super(shardId, msg, cause);
|
||||
}
|
||||
|
||||
public EngineCreationFailureException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
|
||||
}
|
@ -19,9 +19,12 @@
|
||||
|
||||
package org.elasticsearch.index.engine;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.shard.IndexShardException;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -34,4 +37,8 @@ public class EngineException extends IndexShardException {
|
||||
public EngineException(ShardId shardId, String msg, Throwable cause) {
|
||||
super(shardId, msg, cause);
|
||||
}
|
||||
|
||||
public EngineException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
@ -19,8 +19,11 @@
|
||||
|
||||
package org.elasticsearch.index.engine;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -30,7 +33,7 @@ public class FlushFailedEngineException extends EngineException {
|
||||
super(shardId, "Flush failed", t);
|
||||
}
|
||||
|
||||
public FlushFailedEngineException(ShardId shardId, String message, Throwable t) {
|
||||
super(shardId, "Flush failed [" + message + "]", t);
|
||||
public FlushFailedEngineException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
@ -19,9 +19,12 @@
|
||||
|
||||
package org.elasticsearch.index.engine;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -31,6 +34,10 @@ public class FlushNotAllowedEngineException extends EngineException {
|
||||
super(shardId, msg);
|
||||
}
|
||||
|
||||
public FlushNotAllowedEngineException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RestStatus status() {
|
||||
return RestStatus.SERVICE_UNAVAILABLE;
|
||||
|
@ -19,8 +19,11 @@
|
||||
|
||||
package org.elasticsearch.index.engine;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -29,4 +32,8 @@ public class ForceMergeFailedEngineException extends EngineException {
|
||||
public ForceMergeFailedEngineException(ShardId shardId, Throwable t) {
|
||||
super(shardId, "force merge failed", t);
|
||||
}
|
||||
|
||||
public ForceMergeFailedEngineException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
@ -19,8 +19,12 @@
|
||||
|
||||
package org.elasticsearch.index.engine;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -36,6 +40,19 @@ public class IndexFailedEngineException extends EngineException {
|
||||
this.id = index.id();
|
||||
}
|
||||
|
||||
public IndexFailedEngineException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
type = in.readString();
|
||||
id = in.readString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeString(type);
|
||||
out.writeString(id);
|
||||
}
|
||||
|
||||
public String type() {
|
||||
return this.type;
|
||||
}
|
||||
|
@ -19,8 +19,12 @@
|
||||
|
||||
package org.elasticsearch.index.engine;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -33,6 +37,17 @@ public class RecoveryEngineException extends EngineException {
|
||||
this.phase = phase;
|
||||
}
|
||||
|
||||
public RecoveryEngineException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
phase = in.readByte();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeInt(phase);
|
||||
}
|
||||
|
||||
public int phase() {
|
||||
return phase;
|
||||
}
|
||||
|
@ -19,8 +19,11 @@
|
||||
|
||||
package org.elasticsearch.index.engine;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -29,4 +32,8 @@ public class RefreshFailedEngineException extends EngineException {
|
||||
public RefreshFailedEngineException(ShardId shardId, Throwable t) {
|
||||
super(shardId, "Refresh failed", t);
|
||||
}
|
||||
|
||||
public RefreshFailedEngineException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
@ -1,32 +0,0 @@
|
||||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.index.engine;
|
||||
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class RollbackFailedEngineException extends EngineException {
|
||||
|
||||
public RollbackFailedEngineException(ShardId shardId, Throwable t) {
|
||||
super(shardId, "Rollback failed", t);
|
||||
}
|
||||
}
|
@ -1,32 +0,0 @@
|
||||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.index.engine;
|
||||
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class RollbackNotAllowedEngineException extends EngineException {
|
||||
|
||||
public RollbackNotAllowedEngineException(ShardId shardId, String msg) {
|
||||
super(shardId, msg);
|
||||
}
|
||||
}
|
@ -19,8 +19,11 @@
|
||||
|
||||
package org.elasticsearch.index.engine;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -30,4 +33,7 @@ public class SnapshotFailedEngineException extends EngineException {
|
||||
super(shardId, "Snapshot failed", cause);
|
||||
}
|
||||
|
||||
public SnapshotFailedEngineException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
@ -18,22 +18,19 @@
|
||||
*/
|
||||
package org.elasticsearch.index.engine;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class VersionConflictEngineException extends EngineException {
|
||||
|
||||
private final long current;
|
||||
|
||||
private final long provided;
|
||||
|
||||
public VersionConflictEngineException(ShardId shardId, String type, String id, long current, long provided) {
|
||||
super(shardId, "[" + type + "][" + id + "]: version conflict, current [" + current + "], provided [" + provided + "]");
|
||||
this.current = current;
|
||||
this.provided = provided;
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -41,11 +38,7 @@ public class VersionConflictEngineException extends EngineException {
|
||||
return RestStatus.CONFLICT;
|
||||
}
|
||||
|
||||
public long getCurrentVersion() {
|
||||
return this.current;
|
||||
}
|
||||
|
||||
public long getProvidedVersion() {
|
||||
return this.provided;
|
||||
public VersionConflictEngineException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -1,34 +0,0 @@
|
||||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.index.mapper;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class MapperCompressionException extends MapperException {
|
||||
|
||||
public MapperCompressionException(String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
public MapperCompressionException(String message, Throwable cause) {
|
||||
super(message, cause);
|
||||
}
|
||||
}
|
@ -20,11 +20,17 @@
|
||||
package org.elasticsearch.index.mapper;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class MapperException extends ElasticsearchException {
|
||||
public MapperException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
}
|
||||
|
||||
public MapperException(String message) {
|
||||
super(message);
|
||||
|
@ -19,13 +19,20 @@
|
||||
|
||||
package org.elasticsearch.index.mapper;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class MapperParsingException extends MapperException {
|
||||
|
||||
public MapperParsingException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
}
|
||||
|
||||
public MapperParsingException(String message) {
|
||||
super(message);
|
||||
}
|
||||
|
@ -19,8 +19,11 @@
|
||||
|
||||
package org.elasticsearch.index.mapper;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
|
||||
/**
|
||||
@ -35,6 +38,17 @@ public class MergeMappingException extends MapperException {
|
||||
this.failures = failures;
|
||||
}
|
||||
|
||||
public MergeMappingException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
failures = in.readStringArray();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeStringArray(failures);
|
||||
}
|
||||
|
||||
public String[] failures() {
|
||||
return failures;
|
||||
}
|
||||
|
@ -18,8 +18,11 @@
|
||||
*/
|
||||
package org.elasticsearch.index.mapper;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class StrictDynamicMappingException extends MapperParsingException {
|
||||
@ -28,6 +31,10 @@ public class StrictDynamicMappingException extends MapperParsingException {
|
||||
super("mapping set to strict, dynamic introduction of [" + fieldName + "] within [" + path + "] is not allowed");
|
||||
}
|
||||
|
||||
public StrictDynamicMappingException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RestStatus status() {
|
||||
return RestStatus.BAD_REQUEST;
|
||||
|
@ -18,9 +18,12 @@
|
||||
*/
|
||||
package org.elasticsearch.index.percolator;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.IndexException;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Exception during indexing a percolator query.
|
||||
*/
|
||||
@ -29,4 +32,8 @@ public class PercolatorException extends IndexException {
|
||||
public PercolatorException(Index index, String msg, Throwable cause) {
|
||||
super(index, msg, cause);
|
||||
}
|
||||
|
||||
public PercolatorException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -19,6 +19,8 @@
|
||||
|
||||
package org.elasticsearch.index.query;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentLocation;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
@ -34,8 +36,8 @@ import java.io.IOException;
|
||||
public class QueryParsingException extends IndexException {
|
||||
|
||||
static final int UNKNOWN_POSITION = -1;
|
||||
private int lineNumber = UNKNOWN_POSITION;
|
||||
private int columnNumber = UNKNOWN_POSITION;
|
||||
private final int lineNumber;
|
||||
private final int columnNumber;
|
||||
|
||||
public QueryParsingException(QueryParseContext parseContext, String msg) {
|
||||
this(parseContext, msg, null);
|
||||
@ -43,7 +45,8 @@ public class QueryParsingException extends IndexException {
|
||||
|
||||
public QueryParsingException(QueryParseContext parseContext, String msg, Throwable cause) {
|
||||
super(parseContext.index(), msg, cause);
|
||||
|
||||
int lineNumber = UNKNOWN_POSITION;
|
||||
int columnNumber = UNKNOWN_POSITION;
|
||||
XContentParser parser = parseContext.parser();
|
||||
if (parser != null) {
|
||||
XContentLocation location = parser.getTokenLocation();
|
||||
@ -52,13 +55,15 @@ public class QueryParsingException extends IndexException {
|
||||
columnNumber = location.columnNumber;
|
||||
}
|
||||
}
|
||||
this.columnNumber = columnNumber;
|
||||
this.lineNumber = lineNumber;
|
||||
}
|
||||
|
||||
/**
|
||||
* This constructor is provided for use in unit tests where a
|
||||
* {@link QueryParseContext} may not be available
|
||||
*/
|
||||
QueryParsingException(Index index, int line, int col, String msg, Throwable cause) {
|
||||
public QueryParsingException(Index index, int line, int col, String msg, Throwable cause) {
|
||||
super(index, msg, cause);
|
||||
this.lineNumber = line;
|
||||
this.columnNumber = col;
|
||||
@ -96,4 +101,17 @@ public class QueryParsingException extends IndexException {
|
||||
super.innerToXContent(builder, params);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeInt(lineNumber);
|
||||
out.writeInt(columnNumber);
|
||||
}
|
||||
|
||||
public QueryParsingException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
lineNumber = in.readInt();
|
||||
columnNumber = in.readInt();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -19,8 +19,12 @@
|
||||
|
||||
package org.elasticsearch.index.shard;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -46,4 +50,15 @@ public class IllegalIndexShardStateException extends IndexShardException {
|
||||
public RestStatus status() {
|
||||
return RestStatus.NOT_FOUND;
|
||||
}
|
||||
|
||||
public IllegalIndexShardStateException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
currentState = IndexShardState.fromId(in.readByte());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeByte(currentState.id());
|
||||
}
|
||||
}
|
||||
|
@ -19,6 +19,10 @@
|
||||
|
||||
package org.elasticsearch.index.shard;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -34,4 +38,8 @@ public class IndexShardClosedException extends IllegalIndexShardStateException {
|
||||
public IndexShardClosedException(ShardId shardId, String message) {
|
||||
super(shardId, IndexShardState.CLOSED, message);
|
||||
}
|
||||
|
||||
public IndexShardClosedException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -19,6 +19,10 @@
|
||||
|
||||
package org.elasticsearch.index.shard;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class IndexShardCreationException extends IndexShardException {
|
||||
@ -26,4 +30,8 @@ public class IndexShardCreationException extends IndexShardException {
|
||||
public IndexShardCreationException(ShardId shardId, Throwable cause) {
|
||||
super(shardId, "failed to create shard", cause);
|
||||
}
|
||||
|
||||
public IndexShardCreationException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -19,6 +19,8 @@
|
||||
|
||||
package org.elasticsearch.index.shard;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.index.IndexException;
|
||||
|
||||
@ -56,4 +58,19 @@ public class IndexShardException extends IndexException {
|
||||
}
|
||||
super.innerToXContent(builder, params);
|
||||
}
|
||||
|
||||
public IndexShardException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
if (in.readBoolean()) {
|
||||
shardId = ShardId.readShardId(in);
|
||||
} else {
|
||||
shardId = null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeOptionalStreamable(shardId);
|
||||
}
|
||||
}
|
||||
|
@ -19,6 +19,10 @@
|
||||
|
||||
package org.elasticsearch.index.shard;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -27,4 +31,8 @@ public class IndexShardNotRecoveringException extends IllegalIndexShardStateExce
|
||||
public IndexShardNotRecoveringException(ShardId shardId, IndexShardState currentState) {
|
||||
super(shardId, currentState, "Shard not in recovering state");
|
||||
}
|
||||
|
||||
public IndexShardNotRecoveringException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
@ -19,6 +19,10 @@
|
||||
|
||||
package org.elasticsearch.index.shard;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -27,4 +31,8 @@ public class IndexShardNotStartedException extends IllegalIndexShardStateExcepti
|
||||
public IndexShardNotStartedException(ShardId shardId, IndexShardState currentState) {
|
||||
super(shardId, currentState, "Shard not started");
|
||||
}
|
||||
|
||||
public IndexShardNotStartedException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
@ -19,6 +19,10 @@
|
||||
|
||||
package org.elasticsearch.index.shard;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -27,4 +31,8 @@ public class IndexShardRecoveringException extends IllegalIndexShardStateExcepti
|
||||
public IndexShardRecoveringException(ShardId shardId) {
|
||||
super(shardId, IndexShardState.RECOVERING, "Already recovering");
|
||||
}
|
||||
|
||||
public IndexShardRecoveringException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -19,9 +19,12 @@
|
||||
|
||||
package org.elasticsearch.index.shard;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.shard.IndexShardException;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -29,4 +32,8 @@ public class IndexShardRecoveryException extends IndexShardException {
|
||||
public IndexShardRecoveryException(ShardId shardId, String msg, Throwable cause) {
|
||||
super(shardId, msg, cause);
|
||||
}
|
||||
|
||||
public IndexShardRecoveryException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -19,6 +19,10 @@
|
||||
|
||||
package org.elasticsearch.index.shard;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -27,4 +31,8 @@ public class IndexShardRelocatedException extends IllegalIndexShardStateExceptio
|
||||
public IndexShardRelocatedException(ShardId shardId) {
|
||||
super(shardId, IndexShardState.RELOCATED, "Already relocated");
|
||||
}
|
||||
|
||||
public IndexShardRelocatedException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
@ -19,6 +19,10 @@
|
||||
|
||||
package org.elasticsearch.index.shard;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -27,4 +31,8 @@ public class IndexShardStartedException extends IllegalIndexShardStateException
|
||||
public IndexShardStartedException(ShardId shardId) {
|
||||
super(shardId, IndexShardState.STARTED, "Already started");
|
||||
}
|
||||
|
||||
public IndexShardStartedException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -26,6 +26,8 @@ import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.collect.Tuple;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.lucene.search.Queries;
|
||||
import org.elasticsearch.common.xcontent.XContentHelper;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
@ -39,6 +41,7 @@ import org.elasticsearch.index.query.ParsedQuery;
|
||||
import org.elasticsearch.index.query.QueryParsingException;
|
||||
import org.elasticsearch.index.translog.Translog;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
@ -96,6 +99,16 @@ public class TranslogRecoveryPerformer {
|
||||
this.completedOperations = completedOperations;
|
||||
}
|
||||
|
||||
public BatchOperationException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
completedOperations = in.readInt();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeInt(completedOperations);
|
||||
}
|
||||
|
||||
/** the number of succesful operations performed before the exception was thrown */
|
||||
public int completedOperations() {
|
||||
|
@ -19,9 +19,12 @@
|
||||
|
||||
package org.elasticsearch.index.snapshots;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.shard.IndexShardException;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Generic shard restore exception
|
||||
*/
|
||||
@ -33,4 +36,8 @@ public class IndexShardRestoreException extends IndexShardException {
|
||||
public IndexShardRestoreException(ShardId shardId, String msg, Throwable cause) {
|
||||
super(shardId, msg, cause);
|
||||
}
|
||||
|
||||
public IndexShardRestoreException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -19,8 +19,11 @@
|
||||
|
||||
package org.elasticsearch.index.snapshots;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Thrown when restore of a shard fails
|
||||
*/
|
||||
@ -32,4 +35,8 @@ public class IndexShardRestoreFailedException extends IndexShardRestoreException
|
||||
public IndexShardRestoreFailedException(ShardId shardId, String msg, Throwable cause) {
|
||||
super(shardId, msg, cause);
|
||||
}
|
||||
|
||||
public IndexShardRestoreFailedException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -19,9 +19,12 @@
|
||||
|
||||
package org.elasticsearch.index.snapshots;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.shard.IndexShardException;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Generic shard snapshot exception
|
||||
*/
|
||||
@ -33,4 +36,8 @@ public class IndexShardSnapshotException extends IndexShardException {
|
||||
public IndexShardSnapshotException(ShardId shardId, String msg, Throwable cause) {
|
||||
super(shardId, msg, cause);
|
||||
}
|
||||
|
||||
public IndexShardSnapshotException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -19,8 +19,11 @@
|
||||
|
||||
package org.elasticsearch.index.snapshots;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Thrown when snapshot process is failed on a shard level
|
||||
*/
|
||||
@ -32,4 +35,8 @@ public class IndexShardSnapshotFailedException extends IndexShardSnapshotExcepti
|
||||
public IndexShardSnapshotFailedException(ShardId shardId, String msg, Throwable cause) {
|
||||
super(shardId, msg, cause);
|
||||
}
|
||||
|
||||
public IndexShardSnapshotFailedException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -19,19 +19,22 @@
|
||||
|
||||
package org.elasticsearch.index.store;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.shard.IndexShardException;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class StoreException extends IndexShardException {
|
||||
|
||||
public StoreException(ShardId shardId, String msg) {
|
||||
super(shardId, msg);
|
||||
}
|
||||
|
||||
public StoreException(ShardId shardId, String msg, Throwable cause) {
|
||||
super(shardId, msg, cause);
|
||||
}
|
||||
|
||||
public StoreException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
@ -20,6 +20,9 @@
|
||||
package org.elasticsearch.index.translog;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class TranslogCorruptedException extends ElasticsearchException {
|
||||
public TranslogCorruptedException(String msg) {
|
||||
@ -29,4 +32,8 @@ public class TranslogCorruptedException extends ElasticsearchException {
|
||||
public TranslogCorruptedException(String msg, Throwable cause) {
|
||||
super(msg, cause);
|
||||
}
|
||||
|
||||
public TranslogCorruptedException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -19,9 +19,12 @@
|
||||
|
||||
package org.elasticsearch.index.translog;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.shard.IndexShardException;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -34,4 +37,8 @@ public class TranslogException extends IndexShardException {
|
||||
public TranslogException(ShardId shardId, String msg, Throwable cause) {
|
||||
super(shardId, msg, cause);
|
||||
}
|
||||
|
||||
public TranslogException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
@ -19,9 +19,17 @@
|
||||
|
||||
package org.elasticsearch.index.translog;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class TruncatedTranslogException extends TranslogCorruptedException {
|
||||
|
||||
public TruncatedTranslogException(String msg, Throwable cause) {
|
||||
super(msg, cause);
|
||||
}
|
||||
|
||||
public TruncatedTranslogException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
@ -19,9 +19,12 @@
|
||||
|
||||
package org.elasticsearch.indices;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.IndexException;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -32,4 +35,7 @@ public class AliasFilterParsingException extends IndexException {
|
||||
}
|
||||
|
||||
|
||||
public AliasFilterParsingException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
}
|
@ -19,10 +19,13 @@
|
||||
|
||||
package org.elasticsearch.indices;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.IndexException;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@ -36,6 +39,10 @@ public class IndexAlreadyExistsException extends IndexException {
|
||||
super(index, message);
|
||||
}
|
||||
|
||||
public IndexAlreadyExistsException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RestStatus status() {
|
||||
return RestStatus.BAD_REQUEST;
|
||||
|
@ -19,10 +19,13 @@
|
||||
|
||||
package org.elasticsearch.indices;
|
||||
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.IndexException;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Exception indicating that one or more requested indices are closed.
|
||||
*/
|
||||
@ -32,6 +35,10 @@ public class IndexClosedException extends IndexException {
|
||||
super(index, "closed");
|
||||
}
|
||||
|
||||
public IndexClosedException(StreamInput in) throws IOException{
|
||||
super(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RestStatus status() {
|
||||
return RestStatus.FORBIDDEN;
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
x
Reference in New Issue
Block a user