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:
Simon Willnauer 2013-12-28 23:58:02 +01:00
parent 08a077ffae
commit 3113203e9e
4 changed files with 437 additions and 27 deletions

View File

@ -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);
}
}
}

View File

@ -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)

View File

@ -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();
}
}
}

View File

@ -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);
}
}