Skip to content

Commit

Permalink
Rename RobinEngine and friends to InternalEngine
Browse files Browse the repository at this point in the history
  • Loading branch information
s1monw authored and brusic committed Jan 19, 2014
1 parent 7c10e7f commit 2ac191a
Show file tree
Hide file tree
Showing 26 changed files with 106 additions and 106 deletions.
2 changes: 1 addition & 1 deletion docs/reference/indices/refresh.asciidoc
Expand Up @@ -4,7 +4,7 @@
The refresh API allows to explicitly refresh one or more index, making
all operations performed since the last refresh available for search.
The (near) real-time capabilities depend on the index engine used. For
example, the robin one requires refresh to be called, but by default a
example, the internal one requires refresh to be called, but by default a
refresh is scheduled periodically.

[source,js]
Expand Down
Expand Up @@ -27,7 +27,7 @@

/**
* A refresh request making all operations performed since the last refresh available for search. The (near) real-time
* capabilities depends on the index engine used. For example, the robin one requires refresh to be called, but by
* capabilities depends on the index engine used. For example, the internal one requires refresh to be called, but by
* default a refresh is scheduled periodically.
*
* @see org.elasticsearch.client.Requests#refreshRequest(String...)
Expand Down
Expand Up @@ -26,7 +26,7 @@

/**
* A refresh request making all operations performed since the last refresh available for search. The (near) real-time
* capabilities depends on the index engine used. For example, the robin one requires refresh to be called, but by
* capabilities depends on the index engine used. For example, the internal one requires refresh to be called, but by
* default a refresh is scheduled periodically.
*/
public class RefreshRequestBuilder extends BroadcastOperationRequestBuilder<RefreshRequest, RefreshResponse, RefreshRequestBuilder> {
Expand Down
Expand Up @@ -136,7 +136,7 @@ public SnapshotIndexCommit snapshot() throws IOException {
@Override
public IndexDeletionPolicy clone() {
// Lucene IW makes a clone internally but since we hold on to this instance
// the clone will just be the identity. See RobinEngine recovery why we need this.
// the clone will just be the identity. See InternalEngine recovery why we need this.
return this;
}

Expand Down
Expand Up @@ -24,8 +24,8 @@
import org.elasticsearch.common.inject.Module;
import org.elasticsearch.common.inject.SpawnModules;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.engine.robin.RobinEngineModule;
import org.elasticsearch.index.engine.robin.RobinIndexEngineModule;
import org.elasticsearch.index.engine.internal.InternalEngineModule;
import org.elasticsearch.index.engine.internal.InternalIndexEngineModule;

import static org.elasticsearch.common.inject.Modules.createModule;

Expand All @@ -37,8 +37,8 @@ public class IndexEngineModule extends AbstractModule implements SpawnModules {
public static final class EngineSettings {
public static final String ENGINE_TYPE = "index.engine.type";
public static final String INDEX_ENGINE_TYPE = "index.index_engine.type";
public static final Class<? extends Module> DEFAULT_INDEX_ENGINE = RobinIndexEngineModule.class;
public static final Class<? extends Module> DEFAULT_ENGINE = RobinEngineModule.class;
public static final Class<? extends Module> DEFAULT_INDEX_ENGINE = InternalIndexEngineModule.class;
public static final Class<? extends Module> DEFAULT_ENGINE = InternalEngineModule.class;
}

private final Settings settings;
Expand Down
Expand Up @@ -17,7 +17,7 @@
* under the License.
*/

package org.elasticsearch.index.engine.robin;
package org.elasticsearch.index.engine.internal;

import com.google.common.collect.Lists;
import org.apache.lucene.index.*;
Expand Down Expand Up @@ -89,7 +89,7 @@
/**
*
*/
public class RobinEngine extends AbstractIndexShardComponent implements Engine {
public class InternalEngine extends AbstractIndexShardComponent implements Engine {

private volatile ByteSizeValue indexingBufferSize;
private volatile int indexConcurrency;
Expand Down Expand Up @@ -119,7 +119,7 @@ public class RobinEngine extends AbstractIndexShardComponent implements Engine {

private volatile IndexWriter indexWriter;

private final SearcherFactory searcherFactory = new RobinSearchFactory();
private final SearcherFactory searcherFactory = new SearchFactory();
private volatile SearcherManager searcherManager;

private volatile boolean closed = false;
Expand Down Expand Up @@ -159,11 +159,11 @@ public class RobinEngine extends AbstractIndexShardComponent implements Engine {
private SegmentInfos lastCommittedSegmentInfos;

@Inject
public RobinEngine(ShardId shardId, @IndexSettings Settings indexSettings, ThreadPool threadPool,
IndexSettingsService indexSettingsService, ShardIndexingService indexingService, @Nullable IndicesWarmer warmer,
Store store, SnapshotDeletionPolicy deletionPolicy, Translog translog,
MergePolicyProvider mergePolicyProvider, MergeSchedulerProvider mergeScheduler,
AnalysisService analysisService, SimilarityService similarityService, CodecService codecService) throws EngineException {
public InternalEngine(ShardId shardId, @IndexSettings Settings indexSettings, ThreadPool threadPool,
IndexSettingsService indexSettingsService, ShardIndexingService indexingService, @Nullable IndicesWarmer warmer,
Store store, SnapshotDeletionPolicy deletionPolicy, Translog translog,
MergePolicyProvider mergePolicyProvider, MergeSchedulerProvider mergeScheduler,
AnalysisService analysisService, SimilarityService similarityService, CodecService codecService) throws EngineException {
super(shardId, indexSettings);
Preconditions.checkNotNull(store, "Store must be provided to the engine");
Preconditions.checkNotNull(deletionPolicy, "Snapshot deletion policy must be provided to the engine");
Expand Down Expand Up @@ -694,7 +694,7 @@ public final Searcher acquireSearcher(String source) throws EngineException {
}

protected Searcher newSearcher(String source, IndexSearcher searcher, SearcherManager manager) {
return new RobinSearcher(source, searcher, manager);
return new EngineSearcher(source, searcher, manager);
}

@Override
Expand Down Expand Up @@ -1409,45 +1409,45 @@ class ApplySettings implements IndexSettingsService.Listener {

@Override
public void onRefreshSettings(Settings settings) {
long gcDeletesInMillis = settings.getAsTime(INDEX_GC_DELETES, TimeValue.timeValueMillis(RobinEngine.this.gcDeletesInMillis)).millis();
if (gcDeletesInMillis != RobinEngine.this.gcDeletesInMillis) {
logger.info("updating index.gc_deletes from [{}] to [{}]", TimeValue.timeValueMillis(RobinEngine.this.gcDeletesInMillis), TimeValue.timeValueMillis(gcDeletesInMillis));
RobinEngine.this.gcDeletesInMillis = gcDeletesInMillis;
long gcDeletesInMillis = settings.getAsTime(INDEX_GC_DELETES, TimeValue.timeValueMillis(InternalEngine.this.gcDeletesInMillis)).millis();
if (gcDeletesInMillis != InternalEngine.this.gcDeletesInMillis) {
logger.info("updating index.gc_deletes from [{}] to [{}]", TimeValue.timeValueMillis(InternalEngine.this.gcDeletesInMillis), TimeValue.timeValueMillis(gcDeletesInMillis));
InternalEngine.this.gcDeletesInMillis = gcDeletesInMillis;
}

final boolean compoundOnFlush = settings.getAsBoolean(INDEX_COMPOUND_ON_FLUSH, RobinEngine.this.compoundOnFlush);
if (compoundOnFlush != RobinEngine.this.compoundOnFlush) {
logger.info("updating {} from [{}] to [{}]", RobinEngine.INDEX_COMPOUND_ON_FLUSH, RobinEngine.this.compoundOnFlush, compoundOnFlush);
RobinEngine.this.compoundOnFlush = compoundOnFlush;
final boolean compoundOnFlush = settings.getAsBoolean(INDEX_COMPOUND_ON_FLUSH, InternalEngine.this.compoundOnFlush);
if (compoundOnFlush != InternalEngine.this.compoundOnFlush) {
logger.info("updating {} from [{}] to [{}]", InternalEngine.INDEX_COMPOUND_ON_FLUSH, InternalEngine.this.compoundOnFlush, compoundOnFlush);
InternalEngine.this.compoundOnFlush = compoundOnFlush;
indexWriter.getConfig().setUseCompoundFile(compoundOnFlush);
}

int indexConcurrency = settings.getAsInt(INDEX_INDEX_CONCURRENCY, RobinEngine.this.indexConcurrency);
boolean failOnMergeFailure = settings.getAsBoolean(INDEX_FAIL_ON_MERGE_FAILURE, RobinEngine.this.failOnMergeFailure);
String codecName = settings.get(INDEX_CODEC, RobinEngine.this.codecName);
int indexConcurrency = settings.getAsInt(INDEX_INDEX_CONCURRENCY, InternalEngine.this.indexConcurrency);
boolean failOnMergeFailure = settings.getAsBoolean(INDEX_FAIL_ON_MERGE_FAILURE, InternalEngine.this.failOnMergeFailure);
String codecName = settings.get(INDEX_CODEC, InternalEngine.this.codecName);
final boolean codecBloomLoad = settings.getAsBoolean(CodecService.INDEX_CODEC_BLOOM_LOAD, codecService.isLoadBloomFilter());
boolean requiresFlushing = false;
if (indexConcurrency != RobinEngine.this.indexConcurrency ||
!codecName.equals(RobinEngine.this.codecName) ||
failOnMergeFailure != RobinEngine.this.failOnMergeFailure ||
if (indexConcurrency != InternalEngine.this.indexConcurrency ||
!codecName.equals(InternalEngine.this.codecName) ||
failOnMergeFailure != InternalEngine.this.failOnMergeFailure ||
codecBloomLoad != codecService.isLoadBloomFilter()) {
rwl.readLock().lock();
try {
if (indexConcurrency != RobinEngine.this.indexConcurrency) {
logger.info("updating index.index_concurrency from [{}] to [{}]", RobinEngine.this.indexConcurrency, indexConcurrency);
RobinEngine.this.indexConcurrency = indexConcurrency;
if (indexConcurrency != InternalEngine.this.indexConcurrency) {
logger.info("updating index.index_concurrency from [{}] to [{}]", InternalEngine.this.indexConcurrency, indexConcurrency);
InternalEngine.this.indexConcurrency = indexConcurrency;
// we have to flush in this case, since it only applies on a new index writer
requiresFlushing = true;
}
if (!codecName.equals(RobinEngine.this.codecName)) {
logger.info("updating index.codec from [{}] to [{}]", RobinEngine.this.codecName, codecName);
RobinEngine.this.codecName = codecName;
if (!codecName.equals(InternalEngine.this.codecName)) {
logger.info("updating index.codec from [{}] to [{}]", InternalEngine.this.codecName, codecName);
InternalEngine.this.codecName = codecName;
// we want to flush in this case, so the new codec will be reflected right away...
requiresFlushing = true;
}
if (failOnMergeFailure != RobinEngine.this.failOnMergeFailure) {
logger.info("updating {} from [{}] to [{}]", RobinEngine.INDEX_FAIL_ON_MERGE_FAILURE, RobinEngine.this.failOnMergeFailure, failOnMergeFailure);
RobinEngine.this.failOnMergeFailure = failOnMergeFailure;
if (failOnMergeFailure != InternalEngine.this.failOnMergeFailure) {
logger.info("updating {} from [{}] to [{}]", InternalEngine.INDEX_FAIL_ON_MERGE_FAILURE, InternalEngine.this.failOnMergeFailure, failOnMergeFailure);
InternalEngine.this.failOnMergeFailure = failOnMergeFailure;
}
if (codecBloomLoad != codecService.isLoadBloomFilter()) {
logger.info("updating {} from [{}] to [{}]", CodecService.INDEX_CODEC_BLOOM_LOAD, codecService.isLoadBloomFilter(), codecBloomLoad);
Expand All @@ -1469,13 +1469,13 @@ private SearcherManager buildSearchManager(IndexWriter indexWriter) throws IOExc
return new SearcherManager(indexWriter, true, searcherFactory);
}

static class RobinSearcher implements Searcher {
static class EngineSearcher implements Searcher {

private final String source;
private final IndexSearcher searcher;
private final SearcherManager manager;

private RobinSearcher(String source, IndexSearcher searcher, SearcherManager manager) {
private EngineSearcher(String source, IndexSearcher searcher, SearcherManager manager) {
this.source = source;
this.searcher = searcher;
this.manager = manager;
Expand Down Expand Up @@ -1542,7 +1542,7 @@ public Translog.Location translogLocation() {
}
}

class RobinSearchFactory extends SearcherFactory {
class SearchFactory extends SearcherFactory {

@Override
public IndexSearcher newSearcher(IndexReader reader) throws IOException {
Expand Down
Expand Up @@ -17,18 +17,18 @@
* under the License.
*/

package org.elasticsearch.index.engine.robin;
package org.elasticsearch.index.engine.internal;

import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.index.engine.Engine;

/**
*
*/
public class RobinEngineModule extends AbstractModule {
public class InternalEngineModule extends AbstractModule {

@Override
protected void configure() {
bind(Engine.class).to(RobinEngine.class).asEagerSingleton();
bind(Engine.class).to(InternalEngine.class).asEagerSingleton();
}
}
Expand Up @@ -17,7 +17,7 @@
* under the License.
*/

package org.elasticsearch.index.engine.robin;
package org.elasticsearch.index.engine.internal;

import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
Expand All @@ -31,14 +31,14 @@
/**
*
*/
public class RobinIndexEngine extends AbstractIndexComponent implements IndexEngine {
public class InternalIndexEngine extends AbstractIndexComponent implements IndexEngine {

public RobinIndexEngine(Index index) {
public InternalIndexEngine(Index index) {
this(index, EMPTY_SETTINGS);
}

@Inject
public RobinIndexEngine(Index index, @IndexSettings Settings indexSettings) {
public InternalIndexEngine(Index index, @IndexSettings Settings indexSettings) {
super(index, indexSettings);
}

Expand Down
Expand Up @@ -17,18 +17,18 @@
* under the License.
*/

package org.elasticsearch.index.engine.robin;
package org.elasticsearch.index.engine.internal;

import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.index.engine.IndexEngine;

/**
*
*/
public class RobinIndexEngineModule extends AbstractModule {
public class InternalIndexEngineModule extends AbstractModule {

@Override
protected void configure() {
bind(IndexEngine.class).to(RobinIndexEngine.class).asEagerSingleton();
bind(IndexEngine.class).to(InternalIndexEngine.class).asEagerSingleton();
}
}
Expand Up @@ -29,7 +29,7 @@
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.gateway.local.LocalGatewayAllocator;
import org.elasticsearch.index.codec.CodecService;
import org.elasticsearch.index.engine.robin.RobinEngine;
import org.elasticsearch.index.engine.internal.InternalEngine;
import org.elasticsearch.index.gateway.IndexShardGatewayService;
import org.elasticsearch.index.indexing.slowlog.ShardSlowLogIndexingService;
import org.elasticsearch.index.merge.policy.LogByteSizeMergePolicyProvider;
Expand Down Expand Up @@ -80,12 +80,12 @@ public IndexDynamicSettingsModule() {
indexDynamicSettings.addDynamicSetting(LogDocMergePolicyProvider.INDEX_MERGE_POLICY_MAX_MERGE_DOCS, Validator.POSITIVE_INTEGER);
indexDynamicSettings.addDynamicSetting(LogDocMergePolicyProvider.INDEX_MERGE_POLICY_MERGE_FACTOR, Validator.INTEGER_GTE_2);
indexDynamicSettings.addDynamicSetting(LogDocMergePolicyProvider.INDEX_COMPOUND_FORMAT);
indexDynamicSettings.addDynamicSetting(RobinEngine.INDEX_INDEX_CONCURRENCY, Validator.NON_NEGATIVE_INTEGER);
indexDynamicSettings.addDynamicSetting(RobinEngine.INDEX_COMPOUND_ON_FLUSH, Validator.BOOLEAN);
indexDynamicSettings.addDynamicSetting(InternalEngine.INDEX_INDEX_CONCURRENCY, Validator.NON_NEGATIVE_INTEGER);
indexDynamicSettings.addDynamicSetting(InternalEngine.INDEX_COMPOUND_ON_FLUSH, Validator.BOOLEAN);
indexDynamicSettings.addDynamicSetting(CodecService.INDEX_CODEC_BLOOM_LOAD, Validator.BOOLEAN);
indexDynamicSettings.addDynamicSetting(RobinEngine.INDEX_GC_DELETES, Validator.TIME);
indexDynamicSettings.addDynamicSetting(RobinEngine.INDEX_CODEC);
indexDynamicSettings.addDynamicSetting(RobinEngine.INDEX_FAIL_ON_MERGE_FAILURE);
indexDynamicSettings.addDynamicSetting(InternalEngine.INDEX_GC_DELETES, Validator.TIME);
indexDynamicSettings.addDynamicSetting(InternalEngine.INDEX_CODEC);
indexDynamicSettings.addDynamicSetting(InternalEngine.INDEX_FAIL_ON_MERGE_FAILURE);
indexDynamicSettings.addDynamicSetting(ShardSlowLogIndexingService.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_WARN, Validator.TIME);
indexDynamicSettings.addDynamicSetting(ShardSlowLogIndexingService.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_INFO, Validator.TIME);
indexDynamicSettings.addDynamicSetting(ShardSlowLogIndexingService.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_DEBUG, Validator.TIME);
Expand Down
Expand Up @@ -35,7 +35,7 @@ public interface IndexShardRepository {
/**
* Creates a snapshot of the shard based on the index commit point.
* <p/>
* The index commit point can be obtained by using {@link org.elasticsearch.index.engine.robin.RobinEngine#snapshotIndex()} method.
* The index commit point can be obtained by using {@link org.elasticsearch.index.engine.internal.InternalEngine#snapshotIndex()} method.
* IndexShardRepository implementations shouldn't release the snapshot index commit point. It is done by the method caller.
* <p/>
* As snapshot process progresses, implementation of this method should update {@link IndexShardSnapshotStatus} object and check
Expand Down
Expand Up @@ -44,7 +44,7 @@ public static enum Stage {
}

/** The wrapped {@link IndexSearcher}. The reason why we sometimes prefer delegating to this searcher instead of <tt>super</tt> is that
* this instance may have more assertions, for example if it comes from MockRobinEngine which wraps the IndexSearcher into an
* this instance may have more assertions, for example if it comes from MockInternalEngine which wraps the IndexSearcher into an
* AssertingIndexSearcher. */
private final IndexSearcher in;

Expand Down
Expand Up @@ -56,7 +56,7 @@ public class HistogramAggregationSearchBenchmark {

public static void main(String[] args) throws Exception {
Settings settings = settingsBuilder()
.put("index.engine.robin.refreshInterval", "-1")
.put("refresh_interval", "-1")
.put("gateway.type", "local")
.put(SETTING_NUMBER_OF_SHARDS, 2)
.put(SETTING_NUMBER_OF_REPLICAS, 1)
Expand Down
Expand Up @@ -59,7 +59,7 @@ public class ChildSearchAndIndexingBenchmark {

public static void main(String[] args) throws Exception {
Settings settings = settingsBuilder()
.put("index.engine.robin.refreshInterval", "-1")
.put("refresh_interval", "-1")
.put("gateway.type", "local")
.put(SETTING_NUMBER_OF_SHARDS, 1)
.put(SETTING_NUMBER_OF_REPLICAS, 0)
Expand Down
Expand Up @@ -51,7 +51,7 @@ public class ChildSearchBenchmark {

public static void main(String[] args) throws Exception {
Settings settings = settingsBuilder()
.put("index.engine.robin.refreshInterval", "-1")
.put("index.refresh_interval", "-1")
.put("gateway.type", "local")
.put(SETTING_NUMBER_OF_SHARDS, 1)
.put(SETTING_NUMBER_OF_REPLICAS, 0)
Expand Down

0 comments on commit 2ac191a

Please sign in to comment.