|
@@ -26,8 +26,6 @@ import org.apache.logging.log4j.Logger;
|
|
|
import org.apache.logging.log4j.core.LogEvent;
|
|
|
import org.apache.logging.log4j.core.appender.AbstractAppender;
|
|
|
import org.apache.logging.log4j.core.filter.RegexFilter;
|
|
|
-import org.apache.lucene.analysis.Analyzer;
|
|
|
-import org.apache.lucene.codecs.Codec;
|
|
|
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
|
|
|
import org.apache.lucene.document.Field;
|
|
|
import org.apache.lucene.document.LongPoint;
|
|
@@ -45,7 +43,6 @@ import org.apache.lucene.index.LeafReaderContext;
|
|
|
import org.apache.lucene.index.LiveIndexWriterConfig;
|
|
|
import org.apache.lucene.index.LogByteSizeMergePolicy;
|
|
|
import org.apache.lucene.index.LogDocMergePolicy;
|
|
|
-import org.apache.lucene.index.MergePolicy;
|
|
|
import org.apache.lucene.index.NoMergePolicy;
|
|
|
import org.apache.lucene.index.NumericDocValues;
|
|
|
import org.apache.lucene.index.PointValues;
|
|
@@ -72,19 +69,16 @@ import org.elasticsearch.Version;
|
|
|
import org.elasticsearch.action.index.IndexRequest;
|
|
|
import org.elasticsearch.action.support.TransportActions;
|
|
|
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
|
|
-import org.elasticsearch.cluster.routing.AllocationId;
|
|
|
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
|
|
import org.elasticsearch.cluster.routing.ShardRouting;
|
|
|
import org.elasticsearch.cluster.routing.ShardRoutingState;
|
|
|
import org.elasticsearch.cluster.routing.TestShardRouting;
|
|
|
-import org.elasticsearch.common.Nullable;
|
|
|
import org.elasticsearch.common.Randomness;
|
|
|
import org.elasticsearch.common.Strings;
|
|
|
import org.elasticsearch.common.bytes.BytesArray;
|
|
|
import org.elasticsearch.common.bytes.BytesReference;
|
|
|
import org.elasticsearch.common.collect.Tuple;
|
|
|
import org.elasticsearch.common.logging.Loggers;
|
|
|
-import org.elasticsearch.common.lucene.Lucene;
|
|
|
import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
|
|
|
import org.elasticsearch.common.lucene.uid.Versions;
|
|
|
import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver;
|
|
@@ -95,7 +89,6 @@ import org.elasticsearch.common.util.BigArrays;
|
|
|
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
|
|
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
|
|
import org.elasticsearch.common.xcontent.XContentType;
|
|
|
-import org.elasticsearch.index.Index;
|
|
|
import org.elasticsearch.index.IndexSettings;
|
|
|
import org.elasticsearch.index.VersionType;
|
|
|
import org.elasticsearch.index.codec.CodecService;
|
|
@@ -119,20 +112,13 @@ import org.elasticsearch.index.seqno.SequenceNumbersService;
|
|
|
import org.elasticsearch.index.shard.IndexSearcherWrapper;
|
|
|
import org.elasticsearch.index.shard.ShardId;
|
|
|
import org.elasticsearch.index.shard.ShardUtils;
|
|
|
-import org.elasticsearch.index.store.DirectoryService;
|
|
|
import org.elasticsearch.index.store.DirectoryUtils;
|
|
|
import org.elasticsearch.index.store.Store;
|
|
|
import org.elasticsearch.index.translog.Translog;
|
|
|
import org.elasticsearch.index.translog.TranslogConfig;
|
|
|
-import org.elasticsearch.test.DummyShardLock;
|
|
|
-import org.elasticsearch.test.ESTestCase;
|
|
|
import org.elasticsearch.test.IndexSettingsModule;
|
|
|
-import org.elasticsearch.threadpool.TestThreadPool;
|
|
|
-import org.elasticsearch.threadpool.ThreadPool;
|
|
|
import org.hamcrest.MatcherAssert;
|
|
|
import org.hamcrest.Matchers;
|
|
|
-import org.junit.After;
|
|
|
-import org.junit.Before;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.io.UncheckedIOException;
|
|
@@ -166,14 +152,13 @@ import java.util.function.ToLongBiFunction;
|
|
|
import java.util.stream.Collectors;
|
|
|
import java.util.stream.LongStream;
|
|
|
|
|
|
-import static java.util.Collections.emptyList;
|
|
|
import static java.util.Collections.emptyMap;
|
|
|
import static java.util.Collections.shuffle;
|
|
|
import static org.elasticsearch.index.engine.Engine.Operation.Origin.LOCAL_TRANSLOG_RECOVERY;
|
|
|
import static org.elasticsearch.index.engine.Engine.Operation.Origin.PEER_RECOVERY;
|
|
|
import static org.elasticsearch.index.engine.Engine.Operation.Origin.PRIMARY;
|
|
|
import static org.elasticsearch.index.engine.Engine.Operation.Origin.REPLICA;
|
|
|
-import static org.elasticsearch.index.translog.TranslogDeletionPolicyTests.createTranslogDeletionPolicy;
|
|
|
+import static org.elasticsearch.index.translog.TranslogDeletionPolicies.createTranslogDeletionPolicy;
|
|
|
import static org.hamcrest.CoreMatchers.instanceOf;
|
|
|
import static org.hamcrest.Matchers.equalTo;
|
|
|
import static org.hamcrest.Matchers.everyItem;
|
|
@@ -185,313 +170,7 @@ import static org.hamcrest.Matchers.not;
|
|
|
import static org.hamcrest.Matchers.notNullValue;
|
|
|
import static org.hamcrest.Matchers.nullValue;
|
|
|
|
|
|
-public class InternalEngineTests extends ESTestCase {
|
|
|
-
|
|
|
- protected final ShardId shardId = new ShardId(new Index("index", "_na_"), 0);
|
|
|
- protected final AllocationId allocationId = AllocationId.newInitializing();
|
|
|
- private static final IndexSettings INDEX_SETTINGS = IndexSettingsModule.newIndexSettings("index", Settings.EMPTY);
|
|
|
-
|
|
|
- protected ThreadPool threadPool;
|
|
|
-
|
|
|
- private Store store;
|
|
|
- private Store storeReplica;
|
|
|
-
|
|
|
- protected InternalEngine engine;
|
|
|
- protected InternalEngine replicaEngine;
|
|
|
-
|
|
|
- private IndexSettings defaultSettings;
|
|
|
- private String codecName;
|
|
|
- private Path primaryTranslogDir;
|
|
|
- private Path replicaTranslogDir;
|
|
|
-
|
|
|
- @Override
|
|
|
- @Before
|
|
|
- public void setUp() throws Exception {
|
|
|
- super.setUp();
|
|
|
-
|
|
|
- CodecService codecService = new CodecService(null, logger);
|
|
|
- String name = Codec.getDefault().getName();
|
|
|
- if (Arrays.asList(codecService.availableCodecs()).contains(name)) {
|
|
|
- // some codecs are read only so we only take the ones that we have in the service and randomly
|
|
|
- // selected by lucene test case.
|
|
|
- codecName = name;
|
|
|
- } else {
|
|
|
- codecName = "default";
|
|
|
- }
|
|
|
- defaultSettings = IndexSettingsModule.newIndexSettings("test", Settings.builder()
|
|
|
- .put(IndexSettings.INDEX_GC_DELETES_SETTING.getKey(), "1h") // make sure this doesn't kick in on us
|
|
|
- .put(EngineConfig.INDEX_CODEC_SETTING.getKey(), codecName)
|
|
|
- .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
|
|
|
- .put(IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.getKey(),
|
|
|
- between(10, 10 * IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.get(Settings.EMPTY)))
|
|
|
- .build()); // TODO randomize more settings
|
|
|
- threadPool = new TestThreadPool(getClass().getName());
|
|
|
- store = createStore();
|
|
|
- storeReplica = createStore();
|
|
|
- Lucene.cleanLuceneIndex(store.directory());
|
|
|
- Lucene.cleanLuceneIndex(storeReplica.directory());
|
|
|
- primaryTranslogDir = createTempDir("translog-primary");
|
|
|
- engine = createEngine(store, primaryTranslogDir);
|
|
|
- LiveIndexWriterConfig currentIndexWriterConfig = engine.getCurrentIndexWriterConfig();
|
|
|
-
|
|
|
- assertEquals(engine.config().getCodec().getName(), codecService.codec(codecName).getName());
|
|
|
- assertEquals(currentIndexWriterConfig.getCodec().getName(), codecService.codec(codecName).getName());
|
|
|
- if (randomBoolean()) {
|
|
|
- engine.config().setEnableGcDeletes(false);
|
|
|
- }
|
|
|
- replicaTranslogDir = createTempDir("translog-replica");
|
|
|
- replicaEngine = createEngine(storeReplica, replicaTranslogDir);
|
|
|
- currentIndexWriterConfig = replicaEngine.getCurrentIndexWriterConfig();
|
|
|
-
|
|
|
- assertEquals(replicaEngine.config().getCodec().getName(), codecService.codec(codecName).getName());
|
|
|
- assertEquals(currentIndexWriterConfig.getCodec().getName(), codecService.codec(codecName).getName());
|
|
|
- if (randomBoolean()) {
|
|
|
- engine.config().setEnableGcDeletes(false);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- public EngineConfig copy(EngineConfig config, EngineConfig.OpenMode openMode) {
|
|
|
- return copy(config, openMode, config.getAnalyzer());
|
|
|
- }
|
|
|
-
|
|
|
- public EngineConfig copy(EngineConfig config, EngineConfig.OpenMode openMode, Analyzer analyzer) {
|
|
|
- return new EngineConfig(openMode, config.getShardId(), config.getAllocationId(), config.getThreadPool(), config.getIndexSettings(),
|
|
|
- config.getWarmer(), config.getStore(), config.getMergePolicy(), analyzer, config.getSimilarity(),
|
|
|
- new CodecService(null, logger), config.getEventListener(), config.getQueryCache(), config.getQueryCachingPolicy(),
|
|
|
- config.getForceNewHistoryUUID(), config.getTranslogConfig(), config.getFlushMergesAfter(), config.getRefreshListeners(),
|
|
|
- config.getIndexSort(), config.getTranslogRecoveryRunner());
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- @After
|
|
|
- public void tearDown() throws Exception {
|
|
|
- super.tearDown();
|
|
|
- if (engine != null && engine.isClosed.get() == false) {
|
|
|
- engine.getTranslog().getDeletionPolicy().assertNoOpenTranslogRefs();
|
|
|
- }
|
|
|
- if (replicaEngine != null && replicaEngine.isClosed.get() == false) {
|
|
|
- replicaEngine.getTranslog().getDeletionPolicy().assertNoOpenTranslogRefs();
|
|
|
- }
|
|
|
- IOUtils.close(
|
|
|
- replicaEngine, storeReplica,
|
|
|
- engine, store);
|
|
|
- terminate(threadPool);
|
|
|
- }
|
|
|
-
|
|
|
-
|
|
|
- private static Document testDocumentWithTextField() {
|
|
|
- return testDocumentWithTextField("test");
|
|
|
- }
|
|
|
-
|
|
|
- private static Document testDocumentWithTextField(String value) {
|
|
|
- Document document = testDocument();
|
|
|
- document.add(new TextField("value", value, Field.Store.YES));
|
|
|
- return document;
|
|
|
- }
|
|
|
-
|
|
|
-
|
|
|
- private static Document testDocument() {
|
|
|
- return new Document();
|
|
|
- }
|
|
|
-
|
|
|
- public static ParsedDocument createParsedDoc(String id, String routing) {
|
|
|
- return testParsedDocument(id, routing, testDocumentWithTextField(), new BytesArray("{ \"value\" : \"test\" }"), null);
|
|
|
- }
|
|
|
-
|
|
|
- private static ParsedDocument testParsedDocument(String id, String routing, Document document, BytesReference source, Mapping mappingUpdate) {
|
|
|
- Field uidField = new Field("_id", Uid.encodeId(id), IdFieldMapper.Defaults.FIELD_TYPE);
|
|
|
- Field versionField = new NumericDocValuesField("_version", 0);
|
|
|
- SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID();
|
|
|
- document.add(uidField);
|
|
|
- document.add(versionField);
|
|
|
- document.add(seqID.seqNo);
|
|
|
- document.add(seqID.seqNoDocValue);
|
|
|
- document.add(seqID.primaryTerm);
|
|
|
- BytesRef ref = source.toBytesRef();
|
|
|
- document.add(new StoredField(SourceFieldMapper.NAME, ref.bytes, ref.offset, ref.length));
|
|
|
- return new ParsedDocument(versionField, seqID, id, "test", routing, Arrays.asList(document), source, XContentType.JSON,
|
|
|
- mappingUpdate);
|
|
|
- }
|
|
|
-
|
|
|
- protected Store createStore() throws IOException {
|
|
|
- return createStore(newDirectory());
|
|
|
- }
|
|
|
-
|
|
|
- protected Store createStore(final Directory directory) throws IOException {
|
|
|
- return createStore(INDEX_SETTINGS, directory);
|
|
|
- }
|
|
|
-
|
|
|
- protected Store createStore(final IndexSettings indexSettings, final Directory directory) throws IOException {
|
|
|
- final DirectoryService directoryService = new DirectoryService(shardId, indexSettings) {
|
|
|
- @Override
|
|
|
- public Directory newDirectory() throws IOException {
|
|
|
- return directory;
|
|
|
- }
|
|
|
- };
|
|
|
- return new Store(shardId, indexSettings, directoryService, new DummyShardLock(shardId));
|
|
|
- }
|
|
|
-
|
|
|
- protected Translog createTranslog() throws IOException {
|
|
|
- return createTranslog(primaryTranslogDir);
|
|
|
- }
|
|
|
-
|
|
|
- protected Translog createTranslog(Path translogPath) throws IOException {
|
|
|
- TranslogConfig translogConfig = new TranslogConfig(shardId, translogPath, INDEX_SETTINGS, BigArrays.NON_RECYCLING_INSTANCE);
|
|
|
- return new Translog(translogConfig, null, createTranslogDeletionPolicy(INDEX_SETTINGS), () -> SequenceNumbers.UNASSIGNED_SEQ_NO);
|
|
|
- }
|
|
|
-
|
|
|
- protected InternalEngine createEngine(Store store, Path translogPath) throws IOException {
|
|
|
- return createEngine(defaultSettings, store, translogPath, newMergePolicy(), null);
|
|
|
- }
|
|
|
-
|
|
|
- protected InternalEngine createEngine(Store store,
|
|
|
- Path translogPath,
|
|
|
- BiFunction<EngineConfig, SeqNoStats, SequenceNumbersService> sequenceNumbersServiceSupplier) throws IOException {
|
|
|
- return createEngine(defaultSettings, store, translogPath, newMergePolicy(), null, sequenceNumbersServiceSupplier);
|
|
|
- }
|
|
|
-
|
|
|
- protected InternalEngine createEngine(Store store,
|
|
|
- Path translogPath,
|
|
|
- BiFunction<EngineConfig, SeqNoStats, SequenceNumbersService> sequenceNumbersServiceSupplier,
|
|
|
- ToLongBiFunction<Engine, Engine.Operation> seqNoForOperation) throws IOException {
|
|
|
- return createEngine(defaultSettings, store, translogPath, newMergePolicy(), null, sequenceNumbersServiceSupplier, seqNoForOperation, null);
|
|
|
- }
|
|
|
-
|
|
|
- protected InternalEngine createEngine(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy) throws IOException {
|
|
|
- return createEngine(indexSettings, store, translogPath, mergePolicy, null);
|
|
|
-
|
|
|
- }
|
|
|
-
|
|
|
- protected InternalEngine createEngine(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy,
|
|
|
- @Nullable IndexWriterFactory indexWriterFactory) throws IOException {
|
|
|
- return createEngine(indexSettings, store, translogPath, mergePolicy, indexWriterFactory, null);
|
|
|
- }
|
|
|
-
|
|
|
- protected InternalEngine createEngine(
|
|
|
- IndexSettings indexSettings,
|
|
|
- Store store,
|
|
|
- Path translogPath,
|
|
|
- MergePolicy mergePolicy,
|
|
|
- @Nullable IndexWriterFactory indexWriterFactory,
|
|
|
- @Nullable BiFunction<EngineConfig, SeqNoStats, SequenceNumbersService> sequenceNumbersServiceSupplier) throws IOException {
|
|
|
- return createEngine(indexSettings, store, translogPath, mergePolicy, indexWriterFactory, sequenceNumbersServiceSupplier, null, null);
|
|
|
- }
|
|
|
-
|
|
|
- protected InternalEngine createEngine(
|
|
|
- IndexSettings indexSettings,
|
|
|
- Store store,
|
|
|
- Path translogPath,
|
|
|
- MergePolicy mergePolicy,
|
|
|
- @Nullable IndexWriterFactory indexWriterFactory,
|
|
|
- @Nullable BiFunction<EngineConfig, SeqNoStats, SequenceNumbersService> sequenceNumbersServiceSupplier,
|
|
|
- @Nullable ToLongBiFunction<Engine, Engine.Operation> seqNoForOperation) throws IOException {
|
|
|
- return createEngine(indexSettings, store, translogPath, mergePolicy, indexWriterFactory, sequenceNumbersServiceSupplier, seqNoForOperation, null);
|
|
|
- }
|
|
|
-
|
|
|
- protected InternalEngine createEngine(
|
|
|
- IndexSettings indexSettings,
|
|
|
- Store store,
|
|
|
- Path translogPath,
|
|
|
- MergePolicy mergePolicy,
|
|
|
- @Nullable IndexWriterFactory indexWriterFactory,
|
|
|
- @Nullable BiFunction<EngineConfig, SeqNoStats, SequenceNumbersService> sequenceNumbersServiceSupplier,
|
|
|
- @Nullable ToLongBiFunction<Engine, Engine.Operation> seqNoForOperation,
|
|
|
- @Nullable Sort indexSort) throws IOException {
|
|
|
- EngineConfig config = config(indexSettings, store, translogPath, mergePolicy, null, indexSort);
|
|
|
- InternalEngine internalEngine = createInternalEngine(indexWriterFactory, sequenceNumbersServiceSupplier, seqNoForOperation, config);
|
|
|
- if (config.getOpenMode() == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG) {
|
|
|
- internalEngine.recoverFromTranslog();
|
|
|
- }
|
|
|
- return internalEngine;
|
|
|
- }
|
|
|
-
|
|
|
- @FunctionalInterface
|
|
|
- public interface IndexWriterFactory {
|
|
|
-
|
|
|
- IndexWriter createWriter(Directory directory, IndexWriterConfig iwc) throws IOException;
|
|
|
- }
|
|
|
-
|
|
|
- public static InternalEngine createInternalEngine(@Nullable final IndexWriterFactory indexWriterFactory,
|
|
|
- @Nullable final BiFunction<EngineConfig, SeqNoStats, SequenceNumbersService> sequenceNumbersServiceSupplier,
|
|
|
- @Nullable final ToLongBiFunction<Engine, Engine.Operation> seqNoForOperation,
|
|
|
- final EngineConfig config) {
|
|
|
- if (sequenceNumbersServiceSupplier == null) {
|
|
|
- return new InternalEngine(config) {
|
|
|
- @Override
|
|
|
- IndexWriter createWriter(Directory directory, IndexWriterConfig iwc) throws IOException {
|
|
|
- return (indexWriterFactory != null) ?
|
|
|
- indexWriterFactory.createWriter(directory, iwc) :
|
|
|
- super.createWriter(directory, iwc);
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- protected long doGenerateSeqNoForOperation(final Operation operation) {
|
|
|
- return seqNoForOperation != null ? seqNoForOperation.applyAsLong(this, operation) : super.doGenerateSeqNoForOperation(operation);
|
|
|
- }
|
|
|
- };
|
|
|
- } else {
|
|
|
- return new InternalEngine(config, sequenceNumbersServiceSupplier) {
|
|
|
- @Override
|
|
|
- IndexWriter createWriter(Directory directory, IndexWriterConfig iwc) throws IOException {
|
|
|
- return (indexWriterFactory != null) ?
|
|
|
- indexWriterFactory.createWriter(directory, iwc) :
|
|
|
- super.createWriter(directory, iwc);
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- protected long doGenerateSeqNoForOperation(final Operation operation) {
|
|
|
- return seqNoForOperation != null ? seqNoForOperation.applyAsLong(this, operation) : super.doGenerateSeqNoForOperation(operation);
|
|
|
- }
|
|
|
- };
|
|
|
- }
|
|
|
-
|
|
|
- }
|
|
|
-
|
|
|
- public EngineConfig config(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy,
|
|
|
- ReferenceManager.RefreshListener refreshListener) {
|
|
|
- return config(indexSettings, store, translogPath, mergePolicy, refreshListener, null);
|
|
|
- }
|
|
|
-
|
|
|
- public EngineConfig config(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy,
|
|
|
- ReferenceManager.RefreshListener refreshListener, Sort indexSort) {
|
|
|
- IndexWriterConfig iwc = newIndexWriterConfig();
|
|
|
- TranslogConfig translogConfig = new TranslogConfig(shardId, translogPath, indexSettings, BigArrays.NON_RECYCLING_INSTANCE);
|
|
|
- final EngineConfig.OpenMode openMode;
|
|
|
- try {
|
|
|
- if (Lucene.indexExists(store.directory()) == false) {
|
|
|
- openMode = EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG;
|
|
|
- } else {
|
|
|
- openMode = EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG;
|
|
|
- }
|
|
|
- } catch (IOException e) {
|
|
|
- throw new ElasticsearchException("can't find index?", e);
|
|
|
- }
|
|
|
- Engine.EventListener listener = new Engine.EventListener() {
|
|
|
- @Override
|
|
|
- public void onFailedEngine(String reason, @Nullable Exception e) {
|
|
|
- // we don't need to notify anybody in this test
|
|
|
- }
|
|
|
- };
|
|
|
- final TranslogHandler handler = new TranslogHandler(xContentRegistry(), IndexSettingsModule.newIndexSettings(shardId.getIndexName(),
|
|
|
- indexSettings.getSettings()));
|
|
|
- final List<ReferenceManager.RefreshListener> refreshListenerList =
|
|
|
- refreshListener == null ? emptyList() : Collections.singletonList(refreshListener);
|
|
|
- EngineConfig config = new EngineConfig(openMode, shardId, allocationId.getId(), threadPool, indexSettings, null, store,
|
|
|
- mergePolicy, iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(null, logger), listener,
|
|
|
- IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), false, translogConfig,
|
|
|
- TimeValue.timeValueMinutes(5), refreshListenerList, indexSort, handler);
|
|
|
-
|
|
|
- return config;
|
|
|
- }
|
|
|
-
|
|
|
- private static final BytesReference B_1 = new BytesArray(new byte[]{1});
|
|
|
- private static final BytesReference B_2 = new BytesArray(new byte[]{2});
|
|
|
- private static final BytesReference B_3 = new BytesArray(new byte[]{3});
|
|
|
- private static final BytesArray SOURCE = bytesArray("{}");
|
|
|
-
|
|
|
- private static BytesArray bytesArray(String string) {
|
|
|
- return new BytesArray(string.getBytes(Charset.defaultCharset()));
|
|
|
- }
|
|
|
+public class InternalEngineTests extends EngineTestCase {
|
|
|
|
|
|
public void testSegments() throws Exception {
|
|
|
try (Store store = createStore();
|
|
@@ -2487,29 +2166,6 @@ public class InternalEngineTests extends ESTestCase {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- protected Term newUid(String id) {
|
|
|
- return new Term("_id", Uid.encodeId(id));
|
|
|
- }
|
|
|
-
|
|
|
- protected Term newUid(ParsedDocument doc) {
|
|
|
- return newUid(doc.id());
|
|
|
- }
|
|
|
-
|
|
|
- protected Engine.Get newGet(boolean realtime, ParsedDocument doc) {
|
|
|
- return new Engine.Get(realtime, doc.type(), doc.id(), newUid(doc));
|
|
|
- }
|
|
|
-
|
|
|
- private Engine.Index indexForDoc(ParsedDocument doc) {
|
|
|
- return new Engine.Index(newUid(doc), doc);
|
|
|
- }
|
|
|
-
|
|
|
- private Engine.Index replicaIndexForDoc(ParsedDocument doc, long version, long seqNo,
|
|
|
- boolean isRetry) {
|
|
|
- return new Engine.Index(newUid(doc), doc, seqNo, 1, version, VersionType.EXTERNAL,
|
|
|
- Engine.Operation.Origin.REPLICA, System.nanoTime(),
|
|
|
- IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, isRetry);
|
|
|
- }
|
|
|
-
|
|
|
public void testExtractShardId() {
|
|
|
try (Engine.Searcher test = this.engine.acquireSearcher("test")) {
|
|
|
ShardId shardId = ShardUtils.extractShardId(test.getDirectoryReader());
|