mirror of
https://github.com/honeymoose/OpenSearch.git
synced 2025-03-24 17:09:48 +00:00
Add test that throws exceptions during search execution
Currently we only test if readers are correctly released when exceptions occur during reopen or flush. This commit adds a test that randomly throws exceptions during the search execution ie. when Terms are pulled or if a docs enum is created.
This commit is contained in:
parent
08a077ffae
commit
3113203e9e
@ -19,24 +19,36 @@
|
||||
|
||||
package org.elasticsearch.search.basic;
|
||||
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.util.English;
|
||||
import org.elasticsearch.ElasticSearchException;
|
||||
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
|
||||
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
|
||||
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
|
||||
import org.elasticsearch.action.index.IndexResponse;
|
||||
import org.elasticsearch.action.search.SearchPhaseExecutionException;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.client.Requests;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
import org.elasticsearch.common.settings.ImmutableSettings.Builder;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||
import org.elasticsearch.index.query.QueryBuilders;
|
||||
import org.elasticsearch.test.ElasticsearchIntegrationTest;
|
||||
import org.elasticsearch.test.engine.MockRobinEngine;
|
||||
import org.elasticsearch.test.engine.ThrowingAtomicReaderWrapper;
|
||||
import org.elasticsearch.test.store.MockDirectoryHelper;
|
||||
import org.elasticsearch.test.store.MockFSIndexStoreModule;
|
||||
import org.hamcrest.Matchers;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
|
||||
import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder;
|
||||
@ -44,7 +56,7 @@ import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilde
|
||||
public class SearchWithRandomExceptionsTests extends ElasticsearchIntegrationTest {
|
||||
|
||||
@Test
|
||||
public void testRandomExceptions() throws IOException, InterruptedException, ExecutionException {
|
||||
public void testRandomDirectoryIOExceptions() throws IOException, InterruptedException, ExecutionException {
|
||||
final int numShards = between(1, 5);
|
||||
String mapping = XContentFactory.jsonBuilder().
|
||||
startObject().
|
||||
@ -120,7 +132,6 @@ public class SearchWithRandomExceptionsTests extends ElasticsearchIntegrationTes
|
||||
RefreshResponse refreshResponse = client().admin().indices().prepareRefresh("test").execute().get(); // don't assert on failures here
|
||||
final boolean refreshFailed = refreshResponse.getShardFailures().length != 0 || refreshResponse.getFailedShards() != 0;
|
||||
logger.info("Refresh failed [{}] numShardsFailed: [{}], shardFailuresLength: [{}], successfulShards: [{}], totalShards: [{}] ", refreshFailed, refreshResponse.getFailedShards(), refreshResponse.getShardFailures().length, refreshResponse.getSuccessfulShards(), refreshResponse.getTotalShards());
|
||||
|
||||
final int numSearches = atLeast(10);
|
||||
// we don't check anything here really just making sure we don't leave any open files or a broken index behind.
|
||||
for (int i = 0; i < numSearches; i++) {
|
||||
@ -130,15 +141,8 @@ public class SearchWithRandomExceptionsTests extends ElasticsearchIntegrationTes
|
||||
logger.info("Searching for [test:{}]", English.intToEnglish(docToQuery));
|
||||
SearchResponse searchResponse = client().prepareSearch().setQuery(QueryBuilders.matchQuery("test", English.intToEnglish(docToQuery))).get();
|
||||
logger.info("Successful shards: [{}] numShards: [{}]", searchResponse.getSuccessfulShards(), numShards);
|
||||
if (searchResponse.getSuccessfulShards() == numShards && !refreshFailed) {
|
||||
assertThat(searchResponse.getHits().getTotalHits(), Matchers.equalTo(expectedResults));
|
||||
}
|
||||
// check match all
|
||||
searchResponse = client().prepareSearch().setQuery(QueryBuilders.matchAllQuery()).get();
|
||||
if (searchResponse.getSuccessfulShards() == numShards && !refreshFailed) {
|
||||
assertThat(searchResponse.getHits().getTotalHits(), Matchers.equalTo(numCreated));
|
||||
}
|
||||
|
||||
} catch (SearchPhaseExecutionException ex) {
|
||||
if (!expectAllShardsFailed) {
|
||||
throw ex;
|
||||
@ -149,4 +153,159 @@ public class SearchWithRandomExceptionsTests extends ElasticsearchIntegrationTes
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRandomExceptions() throws IOException, InterruptedException, ExecutionException {
|
||||
final int numShards = between(1, 5);
|
||||
String mapping = XContentFactory.jsonBuilder().
|
||||
startObject().
|
||||
startObject("type").
|
||||
startObject("properties").
|
||||
startObject("test")
|
||||
.field("type", "string")
|
||||
.field("index", "not_analyzed")
|
||||
.endObject().
|
||||
endObject().
|
||||
endObject()
|
||||
.endObject().string();
|
||||
final double lowLevelRate;
|
||||
final double topLevelRate;
|
||||
if (frequently()) {
|
||||
if (randomBoolean()) {
|
||||
if (randomBoolean()) {
|
||||
lowLevelRate = 1.0/between(2, 10);
|
||||
topLevelRate = 0.0d;
|
||||
} else {
|
||||
topLevelRate = 1.0/between(2, 10);
|
||||
lowLevelRate = 0.0d;
|
||||
}
|
||||
} else {
|
||||
lowLevelRate = 1.0/between(2, 10);
|
||||
topLevelRate = 1.0/between(2, 10);
|
||||
}
|
||||
} else {
|
||||
// rarely no exception
|
||||
topLevelRate = 0d;
|
||||
lowLevelRate = 0d;
|
||||
}
|
||||
|
||||
Builder settings = settingsBuilder()
|
||||
.put("index.number_of_shards", numShards)
|
||||
.put("index.number_of_replicas", randomIntBetween(0, 1))
|
||||
.put(MockRobinEngine.READER_WRAPPER_TYPE, RandomExceptionDirectoryReaderWrapper.class.getName())
|
||||
.put(EXCEPTION_TOP_LEVEL_RATIO_KEY, topLevelRate)
|
||||
.put(EXCEPTION_LOW_LEVEL_RATIO_KEY, lowLevelRate)
|
||||
.put(MockRobinEngine.WRAP_READER_RATIO, 1.0d);
|
||||
logger.info("creating index: [test] using settings: [{}]", settings.build().getAsMap());
|
||||
client().admin().indices().prepareCreate("test")
|
||||
.setSettings(settings)
|
||||
.addMapping("type", mapping).execute().actionGet();
|
||||
ensureSearchable();
|
||||
final int numDocs = between(10, 100);
|
||||
long numCreated = 0;
|
||||
boolean[] added = new boolean[numDocs];
|
||||
for (int i = 0; i < numDocs ; i++) {
|
||||
try {
|
||||
IndexResponse indexResponse = client().prepareIndex("test", "type", "" + i).setTimeout(TimeValue.timeValueSeconds(1)).setSource("test", English.intToEnglish(i)).get();
|
||||
if (indexResponse.isCreated()) {
|
||||
numCreated++;
|
||||
added[i] = true;
|
||||
}
|
||||
} catch (ElasticSearchException ex) {
|
||||
}
|
||||
}
|
||||
logger.info("Start Refresh");
|
||||
RefreshResponse refreshResponse = client().admin().indices().prepareRefresh("test").execute().get(); // don't assert on failures here
|
||||
final boolean refreshFailed = refreshResponse.getShardFailures().length != 0 || refreshResponse.getFailedShards() != 0;
|
||||
logger.info("Refresh failed [{}] numShardsFailed: [{}], shardFailuresLength: [{}], successfulShards: [{}], totalShards: [{}] ", refreshFailed, refreshResponse.getFailedShards(), refreshResponse.getShardFailures().length, refreshResponse.getSuccessfulShards(), refreshResponse.getTotalShards());
|
||||
|
||||
final int numSearches = atLeast(100);
|
||||
// we don't check anything here really just making sure we don't leave any open files or a broken index behind.
|
||||
for (int i = 0; i < numSearches; i++) {
|
||||
try {
|
||||
int docToQuery = between(0, numDocs-1);
|
||||
long expectedResults = added[docToQuery] ? 1 : 0;
|
||||
logger.info("Searching for [test:{}]", English.intToEnglish(docToQuery));
|
||||
SearchResponse searchResponse = client().prepareSearch().setQuery(QueryBuilders.matchQuery("test", English.intToEnglish(docToQuery))).get();
|
||||
logger.info("Successful shards: [{}] numShards: [{}]", searchResponse.getSuccessfulShards(), numShards);
|
||||
if (searchResponse.getSuccessfulShards() == numShards && !refreshFailed) {
|
||||
assertThat(searchResponse.getHits().getTotalHits(), Matchers.equalTo(expectedResults));
|
||||
}
|
||||
// check match all
|
||||
searchResponse = client().prepareSearch().setQuery(QueryBuilders.matchAllQuery()).get();
|
||||
logger.info("Match all Successful shards: [{}] numShards: [{}]", searchResponse.getSuccessfulShards(), numShards);
|
||||
if (searchResponse.getSuccessfulShards() == numShards && !refreshFailed) {
|
||||
assertThat(searchResponse.getHits().getTotalHits(), Matchers.equalTo(numCreated));
|
||||
}
|
||||
|
||||
} catch (SearchPhaseExecutionException ex) {
|
||||
logger.info("expected SearchPhaseException: [{}]", ex.getMessage());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public static final String EXCEPTION_TOP_LEVEL_RATIO_KEY = "index.engine.exception.ratio.top";
|
||||
public static final String EXCEPTION_LOW_LEVEL_RATIO_KEY = "index.engine.exception.ratio.low";
|
||||
|
||||
|
||||
public static class RandomExceptionDirectoryReaderWrapper extends MockRobinEngine.DirectoryReaderWrapper {
|
||||
private final Settings settings;
|
||||
static class ThrowingSubReaderWrapper extends SubReaderWrapper implements ThrowingAtomicReaderWrapper.Thrower {
|
||||
private final Random random;
|
||||
private final double topLevelRatio;
|
||||
private final double lowLevelRatio;
|
||||
|
||||
ThrowingSubReaderWrapper(Settings settings) {
|
||||
final long seed = settings.getAsLong(ElasticsearchIntegrationTest.INDEX_SEED_SETTING, 0l);
|
||||
this.topLevelRatio = settings.getAsDouble(EXCEPTION_TOP_LEVEL_RATIO_KEY, 0.1d);
|
||||
this.lowLevelRatio = settings.getAsDouble(EXCEPTION_LOW_LEVEL_RATIO_KEY, 0.1d);
|
||||
this.random = new Random(seed);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AtomicReader wrap(AtomicReader reader) {
|
||||
return new ThrowingAtomicReaderWrapper(reader, this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void maybeThrow(ThrowingAtomicReaderWrapper.Flags flag) throws IOException {
|
||||
switch (flag) {
|
||||
case Fields:
|
||||
case TermVectors:
|
||||
case Terms:
|
||||
case TermsEnum:
|
||||
case Intersect:
|
||||
case Norms:
|
||||
case NumericDocValues:
|
||||
case BinaryDocValues:
|
||||
case SortedDocValues:
|
||||
case SortedSetDocValues:
|
||||
if (random.nextDouble() < topLevelRatio) {
|
||||
throw new IOException("Forced top level Exception on [" + flag.name() + "]");
|
||||
}
|
||||
break;
|
||||
case DocsEnum:
|
||||
case DocsAndPositionsEnum:
|
||||
if (random.nextDouble() < lowLevelRatio) {
|
||||
throw new IOException("Forced low level Exception on [" + flag.name() + "]");
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
public RandomExceptionDirectoryReaderWrapper(DirectoryReader in, Settings settings) {
|
||||
super(in, new ThrowingSubReaderWrapper(settings));
|
||||
this.settings = settings;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) {
|
||||
return new RandomExceptionDirectoryReaderWrapper(in, settings);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -191,6 +191,7 @@ public final class TestCluster implements Iterable<Client> {
|
||||
Builder builder = ImmutableSettings.settingsBuilder()
|
||||
/* use RAM directories in 10% of the runs */
|
||||
//.put("index.store.type", random.nextInt(10) == 0 ? MockRamIndexStoreModule.class.getName() : MockFSIndexStoreModule.class.getName())
|
||||
// TODO we should run without those mock modules once in a while to make sure we don't hide any bugs in the actual impl.
|
||||
.put("index.store.type", MockFSIndexStoreModule.class.getName()) // no RAM dir for now!
|
||||
.put(IndexEngineModule.EngineSettings.ENGINE_TYPE, MockEngineModule.class.getName())
|
||||
.put("cluster.name", clusterName)
|
||||
|
@ -19,7 +19,7 @@
|
||||
|
||||
package org.elasticsearch.test.engine;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.*;
|
||||
import org.apache.lucene.search.AssertingIndexSearcher;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.SearcherManager;
|
||||
@ -46,6 +46,9 @@ import org.elasticsearch.indices.warmer.IndicesWarmer;
|
||||
import org.elasticsearch.test.ElasticsearchIntegrationTest;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Constructor;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Random;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
@ -53,7 +56,12 @@ import java.util.concurrent.ConcurrentMap;
|
||||
|
||||
public final class MockRobinEngine extends RobinEngine implements Engine {
|
||||
public static final ConcurrentMap<AssertingSearcher, RuntimeException> INFLIGHT_ENGINE_SEARCHERS = new ConcurrentHashMap<AssertingSearcher, RuntimeException>();
|
||||
public static final String WRAP_READER_RATIO = "index.engine.mock.random.wrap_reader_ratio";
|
||||
public static final String READER_WRAPPER_TYPE = "index.engine.mock.random.wrapper";
|
||||
|
||||
private final Random random;
|
||||
private final boolean wrapReader;
|
||||
private final Class<? extends FilterDirectoryReader> wrapper;
|
||||
|
||||
@Inject
|
||||
public MockRobinEngine(ShardId shardId, @IndexSettings Settings indexSettings, ThreadPool threadPool,
|
||||
@ -64,10 +72,13 @@ public final class MockRobinEngine extends RobinEngine implements Engine {
|
||||
super(shardId, indexSettings, threadPool, indexSettingsService, indexingService, warmer, store,
|
||||
deletionPolicy, translog, mergePolicyProvider, mergeScheduler, analysisService, similarityService, codecService);
|
||||
final long seed = indexSettings.getAsLong(ElasticsearchIntegrationTest.INDEX_SEED_SETTING, 0l);
|
||||
if (logger.isTraceEnabled()) {
|
||||
logger.trace("Using [{}] for shard [{}] seed: [{}]", this.getClass().getName(), shardId, seed);
|
||||
}
|
||||
random = new Random(seed);
|
||||
final double ratio = indexSettings.getAsDouble(WRAP_READER_RATIO, 0.0d); // DISABLED by default - AssertingDR is crazy slow
|
||||
wrapper = indexSettings.getAsClass(READER_WRAPPER_TYPE, AssertingDirectoryReader.class);
|
||||
wrapReader = random.nextDouble() < ratio;
|
||||
if (logger.isTraceEnabled()) {
|
||||
logger.trace("Using [{}] for shard [{}] seed: [{}] wrapReader: [{}]", this.getClass().getName(), shardId, seed, wrapReader);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -86,30 +97,67 @@ public final class MockRobinEngine extends RobinEngine implements Engine {
|
||||
|
||||
@Override
|
||||
protected Searcher newSearcher(String source, IndexSearcher searcher, SearcherManager manager) throws EngineException {
|
||||
|
||||
IndexReader reader = searcher.getIndexReader();
|
||||
IndexReader wrappedReader = reader;
|
||||
if (reader instanceof DirectoryReader && wrapReader) {
|
||||
wrappedReader = wrapReader((DirectoryReader) reader);
|
||||
}
|
||||
// this executes basic query checks and asserts that weights are normalized only once etc.
|
||||
final AssertingIndexSearcher assertingIndexSearcher = new AssertingIndexSearcher(random, searcher.getTopReaderContext());
|
||||
final AssertingIndexSearcher assertingIndexSearcher = new AssertingIndexSearcher(random, wrappedReader);
|
||||
assertingIndexSearcher.setSimilarity(searcher.getSimilarity());
|
||||
return new AssertingSearcher(super.newSearcher(source, assertingIndexSearcher, manager), shardId);
|
||||
// pass the original searcher to the super.newSearcher() method to make sure this is the searcher that will
|
||||
// be released later on. If we wrap an index reader here must not pass the wrapped version to the manager
|
||||
// on release otherwise the reader will be closed too early. - good news, stuff will fail all over the place if we don't get this right here
|
||||
return new AssertingSearcher(assertingIndexSearcher, super.newSearcher(source, searcher, manager), shardId);
|
||||
}
|
||||
|
||||
private DirectoryReader wrapReader(DirectoryReader reader) {
|
||||
try {
|
||||
Constructor<?>[] constructors = wrapper.getConstructors();
|
||||
Constructor<?> nonRandom = null;
|
||||
for (Constructor<?> constructor : constructors) {
|
||||
Class<?>[] parameterTypes = constructor.getParameterTypes();
|
||||
if (parameterTypes.length > 0 && parameterTypes[0] == DirectoryReader.class) {
|
||||
if (parameterTypes.length == 1) {
|
||||
nonRandom = constructor;
|
||||
} else if (parameterTypes.length == 2 && parameterTypes[1] == Settings.class) {
|
||||
|
||||
return (DirectoryReader) constructor.newInstance(reader, indexSettings);
|
||||
}
|
||||
}
|
||||
}
|
||||
if (nonRandom != null) {
|
||||
return (DirectoryReader) nonRandom.newInstance(reader);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new ElasticSearchException("Can not wrap reader", e);
|
||||
}
|
||||
return reader;
|
||||
}
|
||||
|
||||
public final class AssertingSearcher implements Searcher {
|
||||
private final Searcher searcher;
|
||||
private final Searcher wrappedSearcher;
|
||||
private final ShardId shardId;
|
||||
private final IndexSearcher indexSearcher;
|
||||
private RuntimeException firstReleaseStack;
|
||||
private final Object lock = new Object();
|
||||
private final int initialRefCount;
|
||||
|
||||
public AssertingSearcher(Searcher searcher, ShardId shardId) {
|
||||
this.searcher = searcher;
|
||||
public AssertingSearcher(IndexSearcher indexSearcher, Searcher wrappedSearcher, ShardId shardId) {
|
||||
// we only use the given index searcher here instead of the IS of the wrapped searcher. the IS might be a wrapped searcher
|
||||
// with a wrapped reader.
|
||||
this.wrappedSearcher = wrappedSearcher;
|
||||
this.shardId = shardId;
|
||||
initialRefCount = searcher.reader().getRefCount();
|
||||
initialRefCount = wrappedSearcher.reader().getRefCount();
|
||||
this.indexSearcher = indexSearcher;
|
||||
assert initialRefCount > 0 : "IndexReader#getRefCount() was [" + initialRefCount + "] expected a value > [0] - reader is already closed";
|
||||
INFLIGHT_ENGINE_SEARCHERS.put(this, new RuntimeException("Unreleased Searcher, source [" + searcher.source() + "]"));
|
||||
INFLIGHT_ENGINE_SEARCHERS.put(this, new RuntimeException("Unreleased Searcher, source [" + wrappedSearcher.source() + "]"));
|
||||
}
|
||||
|
||||
@Override
|
||||
public String source() {
|
||||
return searcher.source();
|
||||
return wrappedSearcher.source();
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -119,20 +167,20 @@ public final class MockRobinEngine extends RobinEngine implements Engine {
|
||||
// make sure we only get this once and store the stack of the first caller!
|
||||
if (remove == null) {
|
||||
assert firstReleaseStack != null;
|
||||
AssertionError error = new AssertionError("Released Searcher more than once, source [" + searcher.source() + "]");
|
||||
AssertionError error = new AssertionError("Released Searcher more than once, source [" + wrappedSearcher.source() + "]");
|
||||
error.initCause(firstReleaseStack);
|
||||
throw error;
|
||||
} else {
|
||||
assert firstReleaseStack == null;
|
||||
firstReleaseStack = new RuntimeException("Searcher Released first here, source [" + searcher.source() + "]");
|
||||
firstReleaseStack = new RuntimeException("Searcher Released first here, source [" + wrappedSearcher.source() + "]");
|
||||
}
|
||||
}
|
||||
final int refCount = searcher.reader().getRefCount();
|
||||
final int refCount = wrappedSearcher.reader().getRefCount();
|
||||
// this assert seems to be paranoid but given LUCENE-5362 we better add some assertions here to make sure we catch any potential
|
||||
// problems.
|
||||
assert refCount > 0 : "IndexReader#getRefCount() was [" + refCount + "] expected a value > [0] - reader is already closed. Initial refCount was: [" + initialRefCount + "]";
|
||||
try {
|
||||
return searcher.release();
|
||||
return wrappedSearcher.release();
|
||||
} catch (RuntimeException ex) {
|
||||
logger.debug("Failed to release searcher", ex);
|
||||
throw ex;
|
||||
@ -141,16 +189,36 @@ public final class MockRobinEngine extends RobinEngine implements Engine {
|
||||
|
||||
@Override
|
||||
public IndexReader reader() {
|
||||
return searcher.reader();
|
||||
return indexSearcher.getIndexReader();
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexSearcher searcher() {
|
||||
return searcher.searcher();
|
||||
return indexSearcher;
|
||||
}
|
||||
|
||||
public ShardId shardId() {
|
||||
return shardId;
|
||||
}
|
||||
}
|
||||
|
||||
public static abstract class DirectoryReaderWrapper extends FilterDirectoryReader {
|
||||
protected final SubReaderWrapper subReaderWrapper;
|
||||
|
||||
public DirectoryReaderWrapper(DirectoryReader in, SubReaderWrapper subReaderWrapper) {
|
||||
super(in, subReaderWrapper);
|
||||
this.subReaderWrapper = subReaderWrapper;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getCoreCacheKey() {
|
||||
return in.getCoreCacheKey();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getCombinedCoreAndDeletesKey() {
|
||||
return in.getCombinedCoreAndDeletesKey();
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,182 @@
|
||||
/*
|
||||
* Licensed to ElasticSearch and Shay Banon under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. ElasticSearch licenses this
|
||||
* file to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.test.engine;
|
||||
|
||||
import org.apache.lucene.index.*;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.automaton.CompiledAutomaton;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* An FilterAtomicReader that allows to throw exceptions if certain methods
|
||||
* are called on is. This allows to test parts of the system under certain
|
||||
* error conditions that would otherwise not be possible.
|
||||
*/
|
||||
public class ThrowingAtomicReaderWrapper extends FilterAtomicReader {
|
||||
|
||||
private final Thrower thrower;
|
||||
|
||||
/**
|
||||
* Flags passed to {@link Thrower#maybeThrow(org.elasticsearch.test.engine.ThrowingAtomicReaderWrapper.Flags)}
|
||||
* when the corresponding method is called.
|
||||
*/
|
||||
public enum Flags {
|
||||
TermVectors,
|
||||
Terms,
|
||||
TermsEnum,
|
||||
Intersect,
|
||||
DocsEnum,
|
||||
DocsAndPositionsEnum,
|
||||
Fields,
|
||||
Norms, NumericDocValues, BinaryDocValues, SortedDocValues, SortedSetDocValues;
|
||||
}
|
||||
|
||||
/**
|
||||
* A callback interface that allows to throw certain exceptions for
|
||||
* methods called on the IndexReader that is wrapped by {@link ThrowingAtomicReaderWrapper}
|
||||
*/
|
||||
public static interface Thrower {
|
||||
/**
|
||||
* Maybe throws an exception ;)
|
||||
*/
|
||||
public void maybeThrow(Flags flag) throws IOException;
|
||||
}
|
||||
|
||||
public ThrowingAtomicReaderWrapper(AtomicReader in, Thrower thrower) {
|
||||
super(in);
|
||||
this.thrower = thrower;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Fields fields() throws IOException {
|
||||
Fields fields = super.fields();
|
||||
thrower.maybeThrow(Flags.Fields);
|
||||
return fields == null ? null : new ThrowingFields(fields, thrower);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Fields getTermVectors(int docID) throws IOException {
|
||||
Fields fields = super.getTermVectors(docID);
|
||||
thrower.maybeThrow(Flags.TermVectors);
|
||||
return fields == null ? null : new ThrowingFields(fields, thrower);
|
||||
}
|
||||
|
||||
/**
|
||||
* Wraps a Fields but with additional asserts
|
||||
*/
|
||||
public static class ThrowingFields extends FilterFields {
|
||||
private final Thrower thrower;
|
||||
|
||||
public ThrowingFields(Fields in, Thrower thrower) {
|
||||
super(in);
|
||||
this.thrower = thrower;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Terms terms(String field) throws IOException {
|
||||
Terms terms = super.terms(field);
|
||||
thrower.maybeThrow(Flags.Terms);
|
||||
return terms == null ? null : new ThrowingTerms(terms, thrower);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Wraps a Terms but with additional asserts
|
||||
*/
|
||||
public static class ThrowingTerms extends FilterTerms {
|
||||
private final Thrower thrower;
|
||||
|
||||
public ThrowingTerms(Terms in, Thrower thrower) {
|
||||
super(in);
|
||||
this.thrower = thrower;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum intersect(CompiledAutomaton automaton, BytesRef bytes) throws IOException {
|
||||
TermsEnum termsEnum = in.intersect(automaton, bytes);
|
||||
thrower.maybeThrow(Flags.Intersect);
|
||||
return new ThrowingTermsEnum(termsEnum, thrower);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum iterator(TermsEnum reuse) throws IOException {
|
||||
TermsEnum termsEnum = super.iterator(reuse);
|
||||
thrower.maybeThrow(Flags.TermsEnum);
|
||||
return new ThrowingTermsEnum(termsEnum, thrower);
|
||||
}
|
||||
}
|
||||
|
||||
static class ThrowingTermsEnum extends FilterTermsEnum {
|
||||
private final Thrower thrower;
|
||||
|
||||
public ThrowingTermsEnum(TermsEnum in, Thrower thrower) {
|
||||
super(in);
|
||||
this.thrower = thrower;
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
|
||||
thrower.maybeThrow(Flags.DocsEnum);
|
||||
return super.docs(liveDocs, reuse, flags);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
|
||||
thrower.maybeThrow(Flags.DocsAndPositionsEnum);
|
||||
return super.docsAndPositions(liveDocs, reuse, flags);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public NumericDocValues getNumericDocValues(String field) throws IOException {
|
||||
thrower.maybeThrow(Flags.NumericDocValues);
|
||||
return super.getNumericDocValues(field);
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public BinaryDocValues getBinaryDocValues(String field) throws IOException {
|
||||
thrower.maybeThrow(Flags.BinaryDocValues);
|
||||
return super.getBinaryDocValues(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedDocValues getSortedDocValues(String field) throws IOException {
|
||||
thrower.maybeThrow(Flags.SortedDocValues);
|
||||
return super.getSortedDocValues(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
|
||||
thrower.maybeThrow(Flags.SortedSetDocValues);
|
||||
return super.getSortedSetDocValues(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public NumericDocValues getNormValues(String field) throws IOException {
|
||||
thrower.maybeThrow(Flags.Norms);
|
||||
return super.getNormValues(field);
|
||||
}
|
||||
}
|
Loading…
x
Reference in New Issue
Block a user