Enable index-time sorting (#24055)
This change adds an index setting to define how the documents should be sorted inside each Segment. It allows any numeric, date, boolean or keyword field inside a mapping to be used to sort the index on disk. It is not allowed to use a `nested` fields inside an index that defines an index sorting since `nested` fields relies on the original sort of the index. This change does not add early termination capabilities in the search layer. This will be added in a follow up. Relates #6720
This commit is contained in:
parent
c0ac50eaa4
commit
f05af0a382
|
@ -19,6 +19,10 @@
|
||||||
|
|
||||||
package org.elasticsearch.action.admin.indices.segments;
|
package org.elasticsearch.action.admin.indices.segments;
|
||||||
|
|
||||||
|
import org.apache.lucene.search.Sort;
|
||||||
|
import org.apache.lucene.search.SortField;
|
||||||
|
import org.apache.lucene.search.SortedNumericSortField;
|
||||||
|
import org.apache.lucene.search.SortedSetSortField;
|
||||||
import org.apache.lucene.util.Accountable;
|
import org.apache.lucene.util.Accountable;
|
||||||
import org.elasticsearch.action.ShardOperationFailedException;
|
import org.elasticsearch.action.ShardOperationFailedException;
|
||||||
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
|
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
|
||||||
|
@ -37,6 +41,7 @@ import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.Locale;
|
||||||
|
|
||||||
public class IndicesSegmentResponse extends BroadcastResponse implements ToXContent {
|
public class IndicesSegmentResponse extends BroadcastResponse implements ToXContent {
|
||||||
|
|
||||||
|
@ -140,6 +145,9 @@ public class IndicesSegmentResponse extends BroadcastResponse implements ToXCont
|
||||||
if (segment.getMergeId() != null) {
|
if (segment.getMergeId() != null) {
|
||||||
builder.field(Fields.MERGE_ID, segment.getMergeId());
|
builder.field(Fields.MERGE_ID, segment.getMergeId());
|
||||||
}
|
}
|
||||||
|
if (segment.getSegmentSort() != null) {
|
||||||
|
toXContent(builder, segment.getSegmentSort());
|
||||||
|
}
|
||||||
if (segment.ramTree != null) {
|
if (segment.ramTree != null) {
|
||||||
builder.startArray(Fields.RAM_TREE);
|
builder.startArray(Fields.RAM_TREE);
|
||||||
for (Accountable child : segment.ramTree.getChildResources()) {
|
for (Accountable child : segment.ramTree.getChildResources()) {
|
||||||
|
@ -164,6 +172,25 @@ public class IndicesSegmentResponse extends BroadcastResponse implements ToXCont
|
||||||
return builder;
|
return builder;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
static void toXContent(XContentBuilder builder, Sort sort) throws IOException {
|
||||||
|
builder.startArray("sort");
|
||||||
|
for (SortField field : sort.getSort()) {
|
||||||
|
builder.startObject();
|
||||||
|
builder.field("field", field.getField());
|
||||||
|
if (field instanceof SortedNumericSortField) {
|
||||||
|
builder.field("mode", ((SortedNumericSortField) field).getSelector()
|
||||||
|
.toString().toLowerCase(Locale.ROOT));
|
||||||
|
} else if (field instanceof SortedSetSortField) {
|
||||||
|
builder.field("mode", ((SortedSetSortField) field).getSelector()
|
||||||
|
.toString().toLowerCase(Locale.ROOT));
|
||||||
|
}
|
||||||
|
builder.field("missing", field.getMissingValue());
|
||||||
|
builder.field("reverse", field.getReverse());
|
||||||
|
builder.endObject();
|
||||||
|
}
|
||||||
|
builder.endArray();
|
||||||
|
}
|
||||||
|
|
||||||
static void toXContent(XContentBuilder builder, Accountable tree) throws IOException {
|
static void toXContent(XContentBuilder builder, Accountable tree) throws IOException {
|
||||||
builder.startObject();
|
builder.startObject();
|
||||||
builder.field(Fields.DESCRIPTION, tree.toString());
|
builder.field(Fields.DESCRIPTION, tree.toString());
|
||||||
|
|
|
@ -66,6 +66,9 @@ public class ShrinkRequest extends AcknowledgedRequest<ShrinkRequest> implements
|
||||||
if (shrinkIndexRequest == null) {
|
if (shrinkIndexRequest == null) {
|
||||||
validationException = addValidationError("shrink index request is missing", validationException);
|
validationException = addValidationError("shrink index request is missing", validationException);
|
||||||
}
|
}
|
||||||
|
if (shrinkIndexRequest.settings().getByPrefix("index.sort.").isEmpty() == false) {
|
||||||
|
validationException = addValidationError("can't override index sort when shrinking index", validationException);
|
||||||
|
}
|
||||||
return validationException;
|
return validationException;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -374,9 +374,18 @@ public class MetaDataCreateIndexService extends AbstractComponent {
|
||||||
throw e;
|
throw e;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (request.shrinkFrom() == null) {
|
||||||
|
// now that the mapping is merged we can validate the index sort.
|
||||||
|
// we cannot validate for index shrinking since the mapping is empty
|
||||||
|
// at this point. The validation will take place later in the process
|
||||||
|
// (when all shards are copied in a single place).
|
||||||
|
indexService.getIndexSortSupplier().get();
|
||||||
|
}
|
||||||
|
|
||||||
// the context is only used for validation so it's fine to pass fake values for the shard id and the current
|
// the context is only used for validation so it's fine to pass fake values for the shard id and the current
|
||||||
// timestamp
|
// timestamp
|
||||||
final QueryShardContext queryShardContext = indexService.newQueryShardContext(0, null, () -> 0L);
|
final QueryShardContext queryShardContext = indexService.newQueryShardContext(0, null, () -> 0L);
|
||||||
|
|
||||||
for (Alias alias : request.aliases()) {
|
for (Alias alias : request.aliases()) {
|
||||||
if (Strings.hasLength(alias.filter())) {
|
if (Strings.hasLength(alias.filter())) {
|
||||||
aliasValidator.validateAliasFilter(alias.name(), alias.filter(), queryShardContext, xContentRegistry);
|
aliasValidator.validateAliasFilter(alias.name(), alias.filter(), queryShardContext, xContentRegistry);
|
||||||
|
@ -581,10 +590,11 @@ public class MetaDataCreateIndexService extends AbstractComponent {
|
||||||
|
|
||||||
static void prepareShrinkIndexSettings(ClusterState currentState, Set<String> mappingKeys, Settings.Builder indexSettingsBuilder, Index shrinkFromIndex, String shrinkIntoName) {
|
static void prepareShrinkIndexSettings(ClusterState currentState, Set<String> mappingKeys, Settings.Builder indexSettingsBuilder, Index shrinkFromIndex, String shrinkIntoName) {
|
||||||
final IndexMetaData sourceMetaData = currentState.metaData().index(shrinkFromIndex.getName());
|
final IndexMetaData sourceMetaData = currentState.metaData().index(shrinkFromIndex.getName());
|
||||||
|
|
||||||
final List<String> nodesToAllocateOn = validateShrinkIndex(currentState, shrinkFromIndex.getName(),
|
final List<String> nodesToAllocateOn = validateShrinkIndex(currentState, shrinkFromIndex.getName(),
|
||||||
mappingKeys, shrinkIntoName, indexSettingsBuilder.build());
|
mappingKeys, shrinkIntoName, indexSettingsBuilder.build());
|
||||||
final Predicate<String> analysisSimilarityPredicate = (s) -> s.startsWith("index.similarity.")
|
final Predicate<String> sourceSettingsPredicate = (s) -> s.startsWith("index.similarity.")
|
||||||
|| s.startsWith("index.analysis.");
|
|| s.startsWith("index.analysis.") || s.startsWith("index.sort.");
|
||||||
indexSettingsBuilder
|
indexSettingsBuilder
|
||||||
// we use "i.r.a.initial_recovery" rather than "i.r.a.require|include" since we want the replica to allocate right away
|
// we use "i.r.a.initial_recovery" rather than "i.r.a.require|include" since we want the replica to allocate right away
|
||||||
// once we are allocated.
|
// once we are allocated.
|
||||||
|
@ -592,11 +602,11 @@ public class MetaDataCreateIndexService extends AbstractComponent {
|
||||||
Strings.arrayToCommaDelimitedString(nodesToAllocateOn.toArray()))
|
Strings.arrayToCommaDelimitedString(nodesToAllocateOn.toArray()))
|
||||||
// we only try once and then give up with a shrink index
|
// we only try once and then give up with a shrink index
|
||||||
.put("index.allocation.max_retries", 1)
|
.put("index.allocation.max_retries", 1)
|
||||||
// now copy all similarity / analysis settings - this overrides all settings from the user unless they
|
// now copy all similarity / analysis / sort settings - this overrides all settings from the user unless they
|
||||||
// wanna add extra settings
|
// wanna add extra settings
|
||||||
.put(IndexMetaData.SETTING_VERSION_CREATED, sourceMetaData.getCreationVersion())
|
.put(IndexMetaData.SETTING_VERSION_CREATED, sourceMetaData.getCreationVersion())
|
||||||
.put(IndexMetaData.SETTING_VERSION_UPGRADED, sourceMetaData.getUpgradedVersion())
|
.put(IndexMetaData.SETTING_VERSION_UPGRADED, sourceMetaData.getUpgradedVersion())
|
||||||
.put(sourceMetaData.getSettings().filter(analysisSimilarityPredicate))
|
.put(sourceMetaData.getSettings().filter(sourceSettingsPredicate))
|
||||||
.put(IndexMetaData.SETTING_ROUTING_PARTITION_SIZE, sourceMetaData.getRoutingPartitionSize())
|
.put(IndexMetaData.SETTING_ROUTING_PARTITION_SIZE, sourceMetaData.getRoutingPartitionSize())
|
||||||
.put(IndexMetaData.INDEX_SHRINK_SOURCE_NAME.getKey(), shrinkFromIndex.getName())
|
.put(IndexMetaData.INDEX_SHRINK_SOURCE_NAME.getKey(), shrinkFromIndex.getName())
|
||||||
.put(IndexMetaData.INDEX_SHRINK_SOURCE_UUID.getKey(), shrinkFromIndex.getUUID());
|
.put(IndexMetaData.INDEX_SHRINK_SOURCE_UUID.getKey(), shrinkFromIndex.getUUID());
|
||||||
|
|
|
@ -18,13 +18,13 @@
|
||||||
*/
|
*/
|
||||||
package org.elasticsearch.common.settings;
|
package org.elasticsearch.common.settings;
|
||||||
|
|
||||||
|
import org.elasticsearch.index.IndexSortConfig;
|
||||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||||
import org.elasticsearch.cluster.routing.UnassignedInfo;
|
import org.elasticsearch.cluster.routing.UnassignedInfo;
|
||||||
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
|
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
|
||||||
import org.elasticsearch.cluster.routing.allocation.decider.MaxRetryAllocationDecider;
|
import org.elasticsearch.cluster.routing.allocation.decider.MaxRetryAllocationDecider;
|
||||||
import org.elasticsearch.cluster.routing.allocation.decider.ShardsLimitAllocationDecider;
|
import org.elasticsearch.cluster.routing.allocation.decider.ShardsLimitAllocationDecider;
|
||||||
import org.elasticsearch.common.settings.Setting.Property;
|
import org.elasticsearch.common.settings.Setting.Property;
|
||||||
import org.elasticsearch.gateway.PrimaryShardAllocator;
|
|
||||||
import org.elasticsearch.index.IndexModule;
|
import org.elasticsearch.index.IndexModule;
|
||||||
import org.elasticsearch.index.IndexSettings;
|
import org.elasticsearch.index.IndexSettings;
|
||||||
import org.elasticsearch.index.IndexingSlowLog;
|
import org.elasticsearch.index.IndexingSlowLog;
|
||||||
|
@ -100,6 +100,10 @@ public final class IndexScopedSettings extends AbstractScopedSettings {
|
||||||
MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGED_SEGMENT_SETTING,
|
MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGED_SEGMENT_SETTING,
|
||||||
MergePolicyConfig.INDEX_MERGE_POLICY_SEGMENTS_PER_TIER_SETTING,
|
MergePolicyConfig.INDEX_MERGE_POLICY_SEGMENTS_PER_TIER_SETTING,
|
||||||
MergePolicyConfig.INDEX_MERGE_POLICY_RECLAIM_DELETES_WEIGHT_SETTING,
|
MergePolicyConfig.INDEX_MERGE_POLICY_RECLAIM_DELETES_WEIGHT_SETTING,
|
||||||
|
IndexSortConfig.INDEX_SORT_FIELD_SETTING,
|
||||||
|
IndexSortConfig.INDEX_SORT_ORDER_SETTING,
|
||||||
|
IndexSortConfig.INDEX_SORT_MISSING_SETTING,
|
||||||
|
IndexSortConfig.INDEX_SORT_MODE_SETTING,
|
||||||
IndexSettings.INDEX_TRANSLOG_DURABILITY_SETTING,
|
IndexSettings.INDEX_TRANSLOG_DURABILITY_SETTING,
|
||||||
IndexSettings.INDEX_WARMER_ENABLED_SETTING,
|
IndexSettings.INDEX_WARMER_ENABLED_SETTING,
|
||||||
IndexSettings.INDEX_REFRESH_INTERVAL_SETTING,
|
IndexSettings.INDEX_REFRESH_INTERVAL_SETTING,
|
||||||
|
|
|
@ -20,8 +20,8 @@
|
||||||
package org.elasticsearch.index;
|
package org.elasticsearch.index;
|
||||||
|
|
||||||
import org.apache.logging.log4j.message.ParameterizedMessage;
|
import org.apache.logging.log4j.message.ParameterizedMessage;
|
||||||
import org.apache.logging.log4j.util.Supplier;
|
|
||||||
import org.apache.lucene.index.IndexReader;
|
import org.apache.lucene.index.IndexReader;
|
||||||
|
import org.apache.lucene.search.Sort;
|
||||||
import org.apache.lucene.store.AlreadyClosedException;
|
import org.apache.lucene.store.AlreadyClosedException;
|
||||||
import org.apache.lucene.util.Accountable;
|
import org.apache.lucene.util.Accountable;
|
||||||
import org.apache.lucene.util.IOUtils;
|
import org.apache.lucene.util.IOUtils;
|
||||||
|
@ -84,6 +84,7 @@ import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
import java.util.function.Consumer;
|
import java.util.function.Consumer;
|
||||||
import java.util.function.LongSupplier;
|
import java.util.function.LongSupplier;
|
||||||
|
import java.util.function.Supplier;
|
||||||
|
|
||||||
import static java.util.Collections.emptyMap;
|
import static java.util.Collections.emptyMap;
|
||||||
import static java.util.Collections.unmodifiableMap;
|
import static java.util.Collections.unmodifiableMap;
|
||||||
|
@ -119,6 +120,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
|
||||||
private final ScriptService scriptService;
|
private final ScriptService scriptService;
|
||||||
private final ClusterService clusterService;
|
private final ClusterService clusterService;
|
||||||
private final Client client;
|
private final Client client;
|
||||||
|
private Supplier<Sort> indexSortSupplier;
|
||||||
|
|
||||||
public IndexService(IndexSettings indexSettings, NodeEnvironment nodeEnv,
|
public IndexService(IndexSettings indexSettings, NodeEnvironment nodeEnv,
|
||||||
NamedXContentRegistry xContentRegistry,
|
NamedXContentRegistry xContentRegistry,
|
||||||
|
@ -153,6 +155,16 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
|
||||||
throw new IllegalArgumentException("Percolator queries are not allowed to use the current timestamp");
|
throw new IllegalArgumentException("Percolator queries are not allowed to use the current timestamp");
|
||||||
}));
|
}));
|
||||||
this.indexFieldData = new IndexFieldDataService(indexSettings, indicesFieldDataCache, circuitBreakerService, mapperService);
|
this.indexFieldData = new IndexFieldDataService(indexSettings, indicesFieldDataCache, circuitBreakerService, mapperService);
|
||||||
|
if (indexSettings.getIndexSortConfig().hasIndexSort()) {
|
||||||
|
// we delay the actual creation of the sort order for this index because the mapping has not been merged yet.
|
||||||
|
// The sort order is validated right after the merge of the mapping later in the process.
|
||||||
|
this.indexSortSupplier = () -> indexSettings.getIndexSortConfig().buildIndexSort(
|
||||||
|
mapperService::fullName,
|
||||||
|
indexFieldData::getForField
|
||||||
|
);
|
||||||
|
} else {
|
||||||
|
this.indexSortSupplier = () -> null;
|
||||||
|
}
|
||||||
this.shardStoreDeleter = shardStoreDeleter;
|
this.shardStoreDeleter = shardStoreDeleter;
|
||||||
this.bigArrays = bigArrays;
|
this.bigArrays = bigArrays;
|
||||||
this.threadPool = threadPool;
|
this.threadPool = threadPool;
|
||||||
|
@ -243,6 +255,10 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
|
||||||
return similarityService;
|
return similarityService;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Supplier<Sort> getIndexSortSupplier() {
|
||||||
|
return indexSortSupplier;
|
||||||
|
}
|
||||||
|
|
||||||
public synchronized void close(final String reason, boolean delete) throws IOException {
|
public synchronized void close(final String reason, boolean delete) throws IOException {
|
||||||
if (closed.compareAndSet(false, true)) {
|
if (closed.compareAndSet(false, true)) {
|
||||||
deleted.compareAndSet(false, delete);
|
deleted.compareAndSet(false, delete);
|
||||||
|
@ -350,10 +366,10 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
|
||||||
};
|
};
|
||||||
store = new Store(shardId, this.indexSettings, indexStore.newDirectoryService(path), lock,
|
store = new Store(shardId, this.indexSettings, indexStore.newDirectoryService(path), lock,
|
||||||
new StoreCloseListener(shardId, () -> eventListener.onStoreClosed(shardId)));
|
new StoreCloseListener(shardId, () -> eventListener.onStoreClosed(shardId)));
|
||||||
indexShard = new IndexShard(routing, this.indexSettings, path, store, indexCache, mapperService, similarityService,
|
indexShard = new IndexShard(routing, this.indexSettings, path, store, indexSortSupplier,
|
||||||
indexFieldData, engineFactory, eventListener, searcherWrapper, threadPool, bigArrays, engineWarmer,
|
indexCache, mapperService, similarityService, indexFieldData, engineFactory,
|
||||||
() -> globalCheckpointSyncer.accept(shardId),
|
eventListener, searcherWrapper, threadPool, bigArrays, engineWarmer,
|
||||||
searchOperationListeners, indexingOperationListeners);
|
() -> globalCheckpointSyncer.accept(shardId), searchOperationListeners, indexingOperationListeners);
|
||||||
eventListener.indexShardStateChanged(indexShard, null, indexShard.state(), "shard created");
|
eventListener.indexShardStateChanged(indexShard, null, indexShard.state(), "shard created");
|
||||||
eventListener.afterIndexShardCreated(indexShard);
|
eventListener.afterIndexShardCreated(indexShard);
|
||||||
shards = newMapBuilder(shards).put(shardId.id(), indexShard).immutableMap();
|
shards = newMapBuilder(shards).put(shardId.id(), indexShard).immutableMap();
|
||||||
|
@ -401,7 +417,8 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
|
||||||
final boolean flushEngine = deleted.get() == false && closed.get();
|
final boolean flushEngine = deleted.get() == false && closed.get();
|
||||||
indexShard.close(reason, flushEngine);
|
indexShard.close(reason, flushEngine);
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
logger.debug((Supplier<?>) () -> new ParameterizedMessage("[{}] failed to close index shard", shardId), e);
|
logger.debug((org.apache.logging.log4j.util.Supplier<?>)
|
||||||
|
() -> new ParameterizedMessage("[{}] failed to close index shard", shardId), e);
|
||||||
// ignore
|
// ignore
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -98,11 +98,11 @@ public final class IndexSettings {
|
||||||
Setting.intSetting("index.max_rescore_window", MAX_RESULT_WINDOW_SETTING, 1, Property.Dynamic, Property.IndexScope);
|
Setting.intSetting("index.max_rescore_window", MAX_RESULT_WINDOW_SETTING, 1, Property.Dynamic, Property.IndexScope);
|
||||||
/**
|
/**
|
||||||
* Index setting describing the maximum number of filters clauses that can be used
|
* Index setting describing the maximum number of filters clauses that can be used
|
||||||
* in an adjacency_matrix aggregation. The max number of buckets produced by
|
* in an adjacency_matrix aggregation. The max number of buckets produced by
|
||||||
* N filters is (N*N)/2 so a limit of 100 filters is imposed by default.
|
* N filters is (N*N)/2 so a limit of 100 filters is imposed by default.
|
||||||
*/
|
*/
|
||||||
public static final Setting<Integer> MAX_ADJACENCY_MATRIX_FILTERS_SETTING =
|
public static final Setting<Integer> MAX_ADJACENCY_MATRIX_FILTERS_SETTING =
|
||||||
Setting.intSetting("index.max_adjacency_matrix_filters", 100, 2, Property.Dynamic, Property.IndexScope);
|
Setting.intSetting("index.max_adjacency_matrix_filters", 100, 2, Property.Dynamic, Property.IndexScope);
|
||||||
public static final TimeValue DEFAULT_REFRESH_INTERVAL = new TimeValue(1, TimeUnit.SECONDS);
|
public static final TimeValue DEFAULT_REFRESH_INTERVAL = new TimeValue(1, TimeUnit.SECONDS);
|
||||||
public static final Setting<TimeValue> INDEX_REFRESH_INTERVAL_SETTING =
|
public static final Setting<TimeValue> INDEX_REFRESH_INTERVAL_SETTING =
|
||||||
Setting.timeSetting("index.refresh_interval", DEFAULT_REFRESH_INTERVAL, new TimeValue(-1, TimeUnit.MILLISECONDS),
|
Setting.timeSetting("index.refresh_interval", DEFAULT_REFRESH_INTERVAL, new TimeValue(-1, TimeUnit.MILLISECONDS),
|
||||||
|
@ -176,6 +176,7 @@ public final class IndexSettings {
|
||||||
private volatile ByteSizeValue generationThresholdSize;
|
private volatile ByteSizeValue generationThresholdSize;
|
||||||
private final MergeSchedulerConfig mergeSchedulerConfig;
|
private final MergeSchedulerConfig mergeSchedulerConfig;
|
||||||
private final MergePolicyConfig mergePolicyConfig;
|
private final MergePolicyConfig mergePolicyConfig;
|
||||||
|
private final IndexSortConfig indexSortConfig;
|
||||||
private final IndexScopedSettings scopedSettings;
|
private final IndexScopedSettings scopedSettings;
|
||||||
private long gcDeletesInMillis = DEFAULT_GC_DELETES.millis();
|
private long gcDeletesInMillis = DEFAULT_GC_DELETES.millis();
|
||||||
private volatile boolean warmerEnabled;
|
private volatile boolean warmerEnabled;
|
||||||
|
@ -278,6 +279,7 @@ public final class IndexSettings {
|
||||||
maxRefreshListeners = scopedSettings.get(MAX_REFRESH_LISTENERS_PER_SHARD);
|
maxRefreshListeners = scopedSettings.get(MAX_REFRESH_LISTENERS_PER_SHARD);
|
||||||
maxSlicesPerScroll = scopedSettings.get(MAX_SLICES_PER_SCROLL);
|
maxSlicesPerScroll = scopedSettings.get(MAX_SLICES_PER_SCROLL);
|
||||||
this.mergePolicyConfig = new MergePolicyConfig(logger, this);
|
this.mergePolicyConfig = new MergePolicyConfig(logger, this);
|
||||||
|
this.indexSortConfig = new IndexSortConfig(this);
|
||||||
|
|
||||||
scopedSettings.addSettingsUpdateConsumer(MergePolicyConfig.INDEX_COMPOUND_FORMAT_SETTING, mergePolicyConfig::setNoCFSRatio);
|
scopedSettings.addSettingsUpdateConsumer(MergePolicyConfig.INDEX_COMPOUND_FORMAT_SETTING, mergePolicyConfig::setNoCFSRatio);
|
||||||
scopedSettings.addSettingsUpdateConsumer(MergePolicyConfig.INDEX_MERGE_POLICY_EXPUNGE_DELETES_ALLOWED_SETTING, mergePolicyConfig::setExpungeDeletesAllowed);
|
scopedSettings.addSettingsUpdateConsumer(MergePolicyConfig.INDEX_MERGE_POLICY_EXPUNGE_DELETES_ALLOWED_SETTING, mergePolicyConfig::setExpungeDeletesAllowed);
|
||||||
|
@ -499,7 +501,7 @@ public final class IndexSettings {
|
||||||
private void setMaxResultWindow(int maxResultWindow) {
|
private void setMaxResultWindow(int maxResultWindow) {
|
||||||
this.maxResultWindow = maxResultWindow;
|
this.maxResultWindow = maxResultWindow;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the max number of filters in adjacency_matrix aggregation search requests
|
* Returns the max number of filters in adjacency_matrix aggregation search requests
|
||||||
*/
|
*/
|
||||||
|
@ -509,7 +511,7 @@ public final class IndexSettings {
|
||||||
|
|
||||||
private void setMaxAdjacencyMatrixFilters(int maxAdjacencyFilters) {
|
private void setMaxAdjacencyMatrixFilters(int maxAdjacencyFilters) {
|
||||||
this.maxAdjacencyMatrixFilters = maxAdjacencyFilters;
|
this.maxAdjacencyMatrixFilters = maxAdjacencyFilters;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the maximum rescore window for search requests.
|
* Returns the maximum rescore window for search requests.
|
||||||
|
@ -574,5 +576,12 @@ public final class IndexSettings {
|
||||||
this.maxSlicesPerScroll = value;
|
this.maxSlicesPerScroll = value;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the index sort config that should be used for this index.
|
||||||
|
*/
|
||||||
|
public IndexSortConfig getIndexSortConfig() {
|
||||||
|
return indexSortConfig;
|
||||||
|
}
|
||||||
|
|
||||||
public IndexScopedSettings getScopedSettings() { return scopedSettings;}
|
public IndexScopedSettings getScopedSettings() { return scopedSettings;}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,247 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Elasticsearch under one or more contributor
|
||||||
|
* license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright
|
||||||
|
* ownership. Elasticsearch licenses this file to you under
|
||||||
|
* the Apache License, Version 2.0 (the "License"); you may
|
||||||
|
* not use this file except in compliance with the License.
|
||||||
|
* You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.index;
|
||||||
|
|
||||||
|
import org.apache.lucene.search.Sort;
|
||||||
|
import org.apache.lucene.search.SortField;
|
||||||
|
import org.apache.lucene.search.SortedNumericSortField;
|
||||||
|
import org.apache.lucene.search.SortedSetSortField;
|
||||||
|
import org.elasticsearch.Version;
|
||||||
|
import org.elasticsearch.common.settings.Setting;
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.index.fielddata.IndexFieldData;
|
||||||
|
import org.elasticsearch.index.mapper.MappedFieldType;
|
||||||
|
import org.elasticsearch.search.MultiValueMode;
|
||||||
|
import org.elasticsearch.search.sort.SortOrder;
|
||||||
|
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.EnumSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.function.Function;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Holds all the information that is used to build the sort order of an index.
|
||||||
|
*
|
||||||
|
* The index sort settings are <b>final</b> and can be defined only at index creation.
|
||||||
|
* These settings are divided in four lists that are merged during the initialization of this class:
|
||||||
|
* <ul>
|
||||||
|
* <li>`index.sort.field`: the field or a list of field to use for the sort</li>
|
||||||
|
* <li>`index.sort.order` the {@link SortOrder} to use for the field or a list of {@link SortOrder}
|
||||||
|
* for each field defined in `index.sort.field`.
|
||||||
|
* </li>
|
||||||
|
* <li>`index.sort.mode`: the {@link MultiValueMode} to use for the field or a list of orders
|
||||||
|
* for each field defined in `index.sort.field`.
|
||||||
|
* </li>
|
||||||
|
* <li>`index.sort.missing`: the missing value to use for the field or a list of missing values
|
||||||
|
* for each field defined in `index.sort.field`
|
||||||
|
* </li>
|
||||||
|
* </ul>
|
||||||
|
*
|
||||||
|
**/
|
||||||
|
public final class IndexSortConfig {
|
||||||
|
/**
|
||||||
|
* The list of field names
|
||||||
|
*/
|
||||||
|
public static final Setting<List<String>> INDEX_SORT_FIELD_SETTING =
|
||||||
|
Setting.listSetting("index.sort.field", Collections.emptyList(),
|
||||||
|
Function.identity(), Setting.Property.IndexScope, Setting.Property.Final);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The {@link SortOrder} for each specified sort field (ie. <b>asc</b> or <b>desc</b>).
|
||||||
|
*/
|
||||||
|
public static final Setting<List<SortOrder>> INDEX_SORT_ORDER_SETTING =
|
||||||
|
Setting.listSetting("index.sort.order", Collections.emptyList(),
|
||||||
|
IndexSortConfig::parseOrderMode, Setting.Property.IndexScope, Setting.Property.Final);
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The {@link MultiValueMode} for each specified sort field (ie. <b>max</b> or <b>min</b>).
|
||||||
|
*/
|
||||||
|
public static final Setting<List<MultiValueMode>> INDEX_SORT_MODE_SETTING =
|
||||||
|
Setting.listSetting("index.sort.mode", Collections.emptyList(),
|
||||||
|
IndexSortConfig::parseMultiValueMode, Setting.Property.IndexScope, Setting.Property.Final);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The missing value for each specified sort field (ie. <b>_first</b> or <b>_last</b>)
|
||||||
|
*/
|
||||||
|
public static final Setting<List<String>> INDEX_SORT_MISSING_SETTING =
|
||||||
|
Setting.listSetting("index.sort.missing", Collections.emptyList(),
|
||||||
|
IndexSortConfig::validateMissingValue, Setting.Property.IndexScope, Setting.Property.Final);
|
||||||
|
|
||||||
|
private static String validateMissingValue(String missing) {
|
||||||
|
if ("_last".equals(missing) == false && "_first".equals(missing) == false) {
|
||||||
|
throw new IllegalArgumentException("Illegal missing value:[" + missing + "], " +
|
||||||
|
"must be one of [_last, _first]");
|
||||||
|
}
|
||||||
|
return missing;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static SortOrder parseOrderMode(String value) {
|
||||||
|
try {
|
||||||
|
return SortOrder.fromString(value);
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new IllegalArgumentException("Illegal sort order:" + value);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static MultiValueMode parseMultiValueMode(String value) {
|
||||||
|
MultiValueMode mode = MultiValueMode.fromString(value);
|
||||||
|
if (mode != MultiValueMode.MAX && mode != MultiValueMode.MIN) {
|
||||||
|
throw new IllegalArgumentException("Illegal index sort mode:[" + mode + "], " +
|
||||||
|
"must be one of [" + MultiValueMode.MAX + ", " + MultiValueMode.MIN + "]");
|
||||||
|
}
|
||||||
|
return mode;
|
||||||
|
}
|
||||||
|
|
||||||
|
// visible for tests
|
||||||
|
final FieldSortSpec[] sortSpecs;
|
||||||
|
|
||||||
|
public IndexSortConfig(IndexSettings indexSettings) {
|
||||||
|
final Settings settings = indexSettings.getSettings();
|
||||||
|
List<String> fields = INDEX_SORT_FIELD_SETTING.get(settings);
|
||||||
|
this.sortSpecs = fields.stream()
|
||||||
|
.map((name) -> new FieldSortSpec(name))
|
||||||
|
.toArray(FieldSortSpec[]::new);
|
||||||
|
|
||||||
|
if (sortSpecs.length > 0 && indexSettings.getIndexVersionCreated().before(Version.V_6_0_0_alpha1_UNRELEASED)) {
|
||||||
|
/**
|
||||||
|
* This index might be assigned to a node where the index sorting feature is not available
|
||||||
|
* (ie. versions prior to {@link Version.V_6_0_0_alpha1_UNRELEASED}) so we must fail here rather than later.
|
||||||
|
*/
|
||||||
|
throw new IllegalArgumentException("unsupported index.version.created:" + indexSettings.getIndexVersionCreated() +
|
||||||
|
", can't set index.sort on versions prior to " + Version.V_6_0_0_alpha1_UNRELEASED);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (INDEX_SORT_ORDER_SETTING.exists(settings)) {
|
||||||
|
List<SortOrder> orders = INDEX_SORT_ORDER_SETTING.get(settings);
|
||||||
|
if (orders.size() != sortSpecs.length) {
|
||||||
|
throw new IllegalArgumentException("index.sort.field:" + fields +
|
||||||
|
" index.sort.order:" + orders.toString() + ", size mismatch");
|
||||||
|
}
|
||||||
|
for (int i = 0; i < sortSpecs.length; i++) {
|
||||||
|
sortSpecs[i].order = orders.get(i);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (INDEX_SORT_MODE_SETTING.exists(settings)) {
|
||||||
|
List<MultiValueMode> modes = INDEX_SORT_MODE_SETTING.get(settings);
|
||||||
|
if (modes.size() != sortSpecs.length) {
|
||||||
|
throw new IllegalArgumentException("index.sort.field:" + fields +
|
||||||
|
" index.sort.mode:" + modes + ", size mismatch");
|
||||||
|
}
|
||||||
|
for (int i = 0; i < sortSpecs.length; i++) {
|
||||||
|
sortSpecs[i].mode = modes.get(i);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (INDEX_SORT_MISSING_SETTING.exists(settings)) {
|
||||||
|
List<String> missingValues = INDEX_SORT_MISSING_SETTING.get(settings);
|
||||||
|
if (missingValues.size() != sortSpecs.length) {
|
||||||
|
throw new IllegalArgumentException("index.sort.field:" + fields +
|
||||||
|
" index.sort.missing:" + missingValues + ", size mismatch");
|
||||||
|
}
|
||||||
|
for (int i = 0; i < sortSpecs.length; i++) {
|
||||||
|
sortSpecs[i].missingValue = missingValues.get(i);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns true if the index should be sorted
|
||||||
|
*/
|
||||||
|
public boolean hasIndexSort() {
|
||||||
|
return sortSpecs.length > 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Builds the {@link Sort} order from the settings for this index
|
||||||
|
* or returns null if this index has no sort.
|
||||||
|
*/
|
||||||
|
public Sort buildIndexSort(Function<String, MappedFieldType> fieldTypeLookup,
|
||||||
|
Function<MappedFieldType, IndexFieldData<?>> fieldDataLookup) {
|
||||||
|
if (hasIndexSort() == false) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
final SortField[] sortFields = new SortField[sortSpecs.length];
|
||||||
|
for (int i = 0; i < sortSpecs.length; i++) {
|
||||||
|
FieldSortSpec sortSpec = sortSpecs[i];
|
||||||
|
final MappedFieldType ft = fieldTypeLookup.apply(sortSpec.field);
|
||||||
|
if (ft == null) {
|
||||||
|
throw new IllegalArgumentException("unknown index sort field:[" + sortSpec.field + "]");
|
||||||
|
}
|
||||||
|
boolean reverse = sortSpec.order == null ? false : (sortSpec.order == SortOrder.DESC);
|
||||||
|
MultiValueMode mode = sortSpec.mode;
|
||||||
|
if (mode == null) {
|
||||||
|
mode = reverse ? MultiValueMode.MAX : MultiValueMode.MIN;
|
||||||
|
}
|
||||||
|
IndexFieldData<?> fieldData;
|
||||||
|
try {
|
||||||
|
fieldData = fieldDataLookup.apply(ft);
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new IllegalArgumentException("docvalues not found for index sort field:[" + sortSpec.field + "]");
|
||||||
|
}
|
||||||
|
if (fieldData == null) {
|
||||||
|
throw new IllegalArgumentException("docvalues not found for index sort field:[" + sortSpec.field + "]");
|
||||||
|
}
|
||||||
|
sortFields[i] = fieldData.sortField(sortSpec.missingValue, mode, null, reverse);
|
||||||
|
validateIndexSortField(sortFields[i]);
|
||||||
|
}
|
||||||
|
return new Sort(sortFields);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void validateIndexSortField(SortField sortField) {
|
||||||
|
SortField.Type type = getSortFieldType(sortField);
|
||||||
|
if (ALLOWED_INDEX_SORT_TYPES.contains(type) == false) {
|
||||||
|
throw new IllegalArgumentException("invalid index sort field:[" + sortField.getField() + "]");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static class FieldSortSpec {
|
||||||
|
final String field;
|
||||||
|
SortOrder order;
|
||||||
|
MultiValueMode mode;
|
||||||
|
String missingValue;
|
||||||
|
|
||||||
|
FieldSortSpec(String field) {
|
||||||
|
this.field = field;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/** We only allow index sorting on these types */
|
||||||
|
private static final EnumSet<SortField.Type> ALLOWED_INDEX_SORT_TYPES = EnumSet.of(
|
||||||
|
SortField.Type.STRING,
|
||||||
|
SortField.Type.LONG,
|
||||||
|
SortField.Type.INT,
|
||||||
|
SortField.Type.DOUBLE,
|
||||||
|
SortField.Type.FLOAT
|
||||||
|
);
|
||||||
|
|
||||||
|
static SortField.Type getSortFieldType(SortField sortField) {
|
||||||
|
if (sortField instanceof SortedSetSortField) {
|
||||||
|
return SortField.Type.STRING;
|
||||||
|
} else if (sortField instanceof SortedNumericSortField) {
|
||||||
|
return ((SortedNumericSortField) sortField).getNumericType();
|
||||||
|
} else {
|
||||||
|
return sortField.getType();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -705,6 +705,7 @@ public abstract class Engine implements Closeable {
|
||||||
}
|
}
|
||||||
final SegmentReader segmentReader = segmentReader(reader.reader());
|
final SegmentReader segmentReader = segmentReader(reader.reader());
|
||||||
segment.memoryInBytes = segmentReader.ramBytesUsed();
|
segment.memoryInBytes = segmentReader.ramBytesUsed();
|
||||||
|
segment.segmentSort = info.info.getIndexSort();
|
||||||
if (verbose) {
|
if (verbose) {
|
||||||
segment.ramTree = Accountables.namedAccountable("root", segmentReader);
|
segment.ramTree = Accountables.namedAccountable("root", segmentReader);
|
||||||
}
|
}
|
||||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.lucene.index.SnapshotDeletionPolicy;
|
||||||
import org.apache.lucene.search.QueryCache;
|
import org.apache.lucene.search.QueryCache;
|
||||||
import org.apache.lucene.search.QueryCachingPolicy;
|
import org.apache.lucene.search.QueryCachingPolicy;
|
||||||
import org.apache.lucene.search.ReferenceManager;
|
import org.apache.lucene.search.ReferenceManager;
|
||||||
|
import org.apache.lucene.search.Sort;
|
||||||
import org.apache.lucene.search.similarities.Similarity;
|
import org.apache.lucene.search.similarities.Similarity;
|
||||||
import org.elasticsearch.action.index.IndexRequest;
|
import org.elasticsearch.action.index.IndexRequest;
|
||||||
import org.elasticsearch.common.Nullable;
|
import org.elasticsearch.common.Nullable;
|
||||||
|
@ -69,6 +70,8 @@ public final class EngineConfig {
|
||||||
private final long maxUnsafeAutoIdTimestamp;
|
private final long maxUnsafeAutoIdTimestamp;
|
||||||
@Nullable
|
@Nullable
|
||||||
private final ReferenceManager.RefreshListener refreshListeners;
|
private final ReferenceManager.RefreshListener refreshListeners;
|
||||||
|
@Nullable
|
||||||
|
private final Sort indexSort;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Index setting to change the low level lucene codec used for writing new segments.
|
* Index setting to change the low level lucene codec used for writing new segments.
|
||||||
|
@ -113,7 +116,7 @@ public final class EngineConfig {
|
||||||
Similarity similarity, CodecService codecService, Engine.EventListener eventListener,
|
Similarity similarity, CodecService codecService, Engine.EventListener eventListener,
|
||||||
TranslogRecoveryPerformer translogRecoveryPerformer, QueryCache queryCache, QueryCachingPolicy queryCachingPolicy,
|
TranslogRecoveryPerformer translogRecoveryPerformer, QueryCache queryCache, QueryCachingPolicy queryCachingPolicy,
|
||||||
TranslogConfig translogConfig, TimeValue flushMergesAfter, ReferenceManager.RefreshListener refreshListeners,
|
TranslogConfig translogConfig, TimeValue flushMergesAfter, ReferenceManager.RefreshListener refreshListeners,
|
||||||
long maxUnsafeAutoIdTimestamp) {
|
long maxUnsafeAutoIdTimestamp, Sort indexSort) {
|
||||||
if (openMode == null) {
|
if (openMode == null) {
|
||||||
throw new IllegalArgumentException("openMode must not be null");
|
throw new IllegalArgumentException("openMode must not be null");
|
||||||
}
|
}
|
||||||
|
@ -143,6 +146,7 @@ public final class EngineConfig {
|
||||||
assert maxUnsafeAutoIdTimestamp >= IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP :
|
assert maxUnsafeAutoIdTimestamp >= IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP :
|
||||||
"maxUnsafeAutoIdTimestamp must be >= -1 but was " + maxUnsafeAutoIdTimestamp;
|
"maxUnsafeAutoIdTimestamp must be >= -1 but was " + maxUnsafeAutoIdTimestamp;
|
||||||
this.maxUnsafeAutoIdTimestamp = maxUnsafeAutoIdTimestamp;
|
this.maxUnsafeAutoIdTimestamp = maxUnsafeAutoIdTimestamp;
|
||||||
|
this.indexSort = indexSort;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -335,4 +339,11 @@ public final class EngineConfig {
|
||||||
public long getMaxUnsafeAutoIdTimestamp() {
|
public long getMaxUnsafeAutoIdTimestamp() {
|
||||||
return indexSettings.getValue(INDEX_OPTIMIZE_AUTO_GENERATED_IDS) ? maxUnsafeAutoIdTimestamp : Long.MAX_VALUE;
|
return indexSettings.getValue(INDEX_OPTIMIZE_AUTO_GENERATED_IDS) ? maxUnsafeAutoIdTimestamp : Long.MAX_VALUE;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the sort order of this index, or null if the index has no sort.
|
||||||
|
*/
|
||||||
|
public Sort getIndexSort() {
|
||||||
|
return indexSort;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -1610,6 +1610,9 @@ public class InternalEngine extends Engine {
|
||||||
iwc.setRAMBufferSizeMB(engineConfig.getIndexingBufferSize().getMbFrac());
|
iwc.setRAMBufferSizeMB(engineConfig.getIndexingBufferSize().getMbFrac());
|
||||||
iwc.setCodec(engineConfig.getCodec());
|
iwc.setCodec(engineConfig.getCodec());
|
||||||
iwc.setUseCompoundFile(true); // always use compound on flush - reduces # of file-handles on refresh
|
iwc.setUseCompoundFile(true); // always use compound on flush - reduces # of file-handles on refresh
|
||||||
|
if (config().getIndexSort() != null) {
|
||||||
|
iwc.setIndexSort(config().getIndexSort());
|
||||||
|
}
|
||||||
return iwc;
|
return iwc;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -19,8 +19,15 @@
|
||||||
|
|
||||||
package org.elasticsearch.index.engine;
|
package org.elasticsearch.index.engine;
|
||||||
|
|
||||||
|
import org.apache.lucene.search.Sort;
|
||||||
|
import org.apache.lucene.search.SortField;
|
||||||
|
import org.apache.lucene.search.SortedSetSortField;
|
||||||
|
import org.apache.lucene.search.SortedNumericSortField;
|
||||||
|
import org.apache.lucene.search.SortedSetSelector;
|
||||||
|
import org.apache.lucene.search.SortedNumericSelector;
|
||||||
import org.apache.lucene.util.Accountable;
|
import org.apache.lucene.util.Accountable;
|
||||||
import org.apache.lucene.util.Accountables;
|
import org.apache.lucene.util.Accountables;
|
||||||
|
import org.elasticsearch.Version;
|
||||||
import org.elasticsearch.common.Nullable;
|
import org.elasticsearch.common.Nullable;
|
||||||
import org.elasticsearch.common.io.stream.StreamInput;
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
@ -46,6 +53,7 @@ public class Segment implements Streamable {
|
||||||
public Boolean compound = null;
|
public Boolean compound = null;
|
||||||
public String mergeId;
|
public String mergeId;
|
||||||
public long memoryInBytes;
|
public long memoryInBytes;
|
||||||
|
public Sort segmentSort;
|
||||||
public Accountable ramTree = null;
|
public Accountable ramTree = null;
|
||||||
|
|
||||||
Segment() {
|
Segment() {
|
||||||
|
@ -113,6 +121,13 @@ public class Segment implements Streamable {
|
||||||
return this.memoryInBytes;
|
return this.memoryInBytes;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the sort order of this segment, or null if the segment has no sort.
|
||||||
|
*/
|
||||||
|
public Sort getSegmentSort() {
|
||||||
|
return segmentSort;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean equals(Object o) {
|
public boolean equals(Object o) {
|
||||||
if (this == o) return true;
|
if (this == o) return true;
|
||||||
|
@ -153,6 +168,11 @@ public class Segment implements Streamable {
|
||||||
// verbose mode
|
// verbose mode
|
||||||
ramTree = readRamTree(in);
|
ramTree = readRamTree(in);
|
||||||
}
|
}
|
||||||
|
if (in.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) {
|
||||||
|
segmentSort = readSegmentSort(in);
|
||||||
|
} else {
|
||||||
|
segmentSort = null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -167,12 +187,106 @@ public class Segment implements Streamable {
|
||||||
out.writeOptionalBoolean(compound);
|
out.writeOptionalBoolean(compound);
|
||||||
out.writeOptionalString(mergeId);
|
out.writeOptionalString(mergeId);
|
||||||
out.writeLong(memoryInBytes);
|
out.writeLong(memoryInBytes);
|
||||||
|
|
||||||
boolean verbose = ramTree != null;
|
boolean verbose = ramTree != null;
|
||||||
out.writeBoolean(verbose);
|
out.writeBoolean(verbose);
|
||||||
if (verbose) {
|
if (verbose) {
|
||||||
writeRamTree(out, ramTree);
|
writeRamTree(out, ramTree);
|
||||||
}
|
}
|
||||||
|
if (out.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) {
|
||||||
|
writeSegmentSort(out, segmentSort);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
Sort readSegmentSort(StreamInput in) throws IOException {
|
||||||
|
int size = in.readVInt();
|
||||||
|
if (size == 0) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
SortField[] fields = new SortField[size];
|
||||||
|
for (int i = 0; i < size; i++) {
|
||||||
|
String field = in.readString();
|
||||||
|
byte type = in.readByte();
|
||||||
|
if (type == 0) {
|
||||||
|
Boolean missingFirst = in.readOptionalBoolean();
|
||||||
|
boolean max = in.readBoolean();
|
||||||
|
boolean reverse = in.readBoolean();
|
||||||
|
fields[i] = new SortedSetSortField(field, reverse,
|
||||||
|
max ? SortedSetSelector.Type.MAX : SortedSetSelector.Type.MIN);
|
||||||
|
if (missingFirst != null) {
|
||||||
|
fields[i].setMissingValue(missingFirst ?
|
||||||
|
SortedSetSortField.STRING_FIRST : SortedSetSortField.STRING_LAST);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
Object missing = in.readGenericValue();
|
||||||
|
boolean max = in.readBoolean();
|
||||||
|
boolean reverse = in.readBoolean();
|
||||||
|
final SortField.Type numericType;
|
||||||
|
switch (type) {
|
||||||
|
case 1:
|
||||||
|
numericType = SortField.Type.INT;
|
||||||
|
break;
|
||||||
|
case 2:
|
||||||
|
numericType = SortField.Type.FLOAT;
|
||||||
|
break;
|
||||||
|
case 3:
|
||||||
|
numericType = SortField.Type.DOUBLE;
|
||||||
|
break;
|
||||||
|
case 4:
|
||||||
|
numericType = SortField.Type.LONG;
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
throw new IOException("invalid index sort type:[" + type +
|
||||||
|
"] for numeric field:[" + field + "]");
|
||||||
|
}
|
||||||
|
fields[i] = new SortedNumericSortField(field, numericType, reverse, max ?
|
||||||
|
SortedNumericSelector.Type.MAX : SortedNumericSelector.Type.MIN);
|
||||||
|
if (missing != null) {
|
||||||
|
fields[i].setMissingValue(missing);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return new Sort(fields);
|
||||||
|
}
|
||||||
|
|
||||||
|
void writeSegmentSort(StreamOutput out, Sort sort) throws IOException {
|
||||||
|
if (sort == null) {
|
||||||
|
out.writeVInt(0);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
out.writeVInt(sort.getSort().length);
|
||||||
|
for (SortField field : sort.getSort()) {
|
||||||
|
out.writeString(field.getField());
|
||||||
|
if (field instanceof SortedSetSortField) {
|
||||||
|
out.writeByte((byte) 0);
|
||||||
|
out.writeOptionalBoolean(field.getMissingValue() == null ?
|
||||||
|
null : field.getMissingValue() == SortField.STRING_FIRST);
|
||||||
|
out.writeBoolean(((SortedSetSortField) field).getSelector() == SortedSetSelector.Type.MAX);
|
||||||
|
out.writeBoolean(field.getReverse());
|
||||||
|
} else if (field instanceof SortedNumericSortField) {
|
||||||
|
switch (((SortedNumericSortField) field).getNumericType()) {
|
||||||
|
case INT:
|
||||||
|
out.writeByte((byte) 1);
|
||||||
|
break;
|
||||||
|
case FLOAT:
|
||||||
|
out.writeByte((byte) 2);
|
||||||
|
break;
|
||||||
|
case DOUBLE:
|
||||||
|
out.writeByte((byte) 3);
|
||||||
|
break;
|
||||||
|
case LONG:
|
||||||
|
out.writeByte((byte) 4);
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
throw new IOException("invalid index sort field:" + field);
|
||||||
|
}
|
||||||
|
out.writeGenericValue(field.getMissingValue());
|
||||||
|
out.writeBoolean(((SortedNumericSortField) field).getSelector() == SortedNumericSelector.Type.MAX);
|
||||||
|
out.writeBoolean(field.getReverse());
|
||||||
|
} else {
|
||||||
|
throw new IOException("invalid index sort field:" + field + "");
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
Accountable readRamTree(StreamInput in) throws IOException {
|
Accountable readRamTree(StreamInput in) throws IOException {
|
||||||
|
@ -188,7 +302,7 @@ public class Segment implements Streamable {
|
||||||
}
|
}
|
||||||
return Accountables.namedAccountable(name, children, bytes);
|
return Accountables.namedAccountable(name, children, bytes);
|
||||||
}
|
}
|
||||||
|
|
||||||
// the ram tree is written recursively since the depth is fairly low (5 or 6)
|
// the ram tree is written recursively since the depth is fairly low (5 or 6)
|
||||||
void writeRamTree(StreamOutput out, Accountable tree) throws IOException {
|
void writeRamTree(StreamOutput out, Accountable tree) throws IOException {
|
||||||
out.writeString(tree.toString());
|
out.writeString(tree.toString());
|
||||||
|
@ -214,6 +328,7 @@ public class Segment implements Streamable {
|
||||||
", compound=" + compound +
|
", compound=" + compound +
|
||||||
", mergeId='" + mergeId + '\'' +
|
", mergeId='" + mergeId + '\'' +
|
||||||
", memoryInBytes=" + memoryInBytes +
|
", memoryInBytes=" + memoryInBytes +
|
||||||
|
(segmentSort != null ? ", sort=" + segmentSort : "") +
|
||||||
'}';
|
'}';
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -38,6 +38,7 @@ import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
import org.elasticsearch.common.xcontent.XContentType;
|
import org.elasticsearch.common.xcontent.XContentType;
|
||||||
import org.elasticsearch.index.AbstractIndexComponent;
|
import org.elasticsearch.index.AbstractIndexComponent;
|
||||||
import org.elasticsearch.index.IndexSettings;
|
import org.elasticsearch.index.IndexSettings;
|
||||||
|
import org.elasticsearch.index.IndexSortConfig;
|
||||||
import org.elasticsearch.index.analysis.IndexAnalyzers;
|
import org.elasticsearch.index.analysis.IndexAnalyzers;
|
||||||
import org.elasticsearch.index.mapper.Mapper.BuilderContext;
|
import org.elasticsearch.index.mapper.Mapper.BuilderContext;
|
||||||
import org.elasticsearch.index.query.QueryShardContext;
|
import org.elasticsearch.index.query.QueryShardContext;
|
||||||
|
@ -438,6 +439,7 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
|
||||||
checkNestedFieldsLimit(fullPathObjectMappers);
|
checkNestedFieldsLimit(fullPathObjectMappers);
|
||||||
checkDepthLimit(fullPathObjectMappers.keySet());
|
checkDepthLimit(fullPathObjectMappers.keySet());
|
||||||
}
|
}
|
||||||
|
checkIndexSortCompatibility(indexSettings.getIndexSortConfig(), hasNested);
|
||||||
|
|
||||||
for (Map.Entry<String, DocumentMapper> entry : mappers.entrySet()) {
|
for (Map.Entry<String, DocumentMapper> entry : mappers.entrySet()) {
|
||||||
if (entry.getKey().equals(DEFAULT_MAPPING)) {
|
if (entry.getKey().equals(DEFAULT_MAPPING)) {
|
||||||
|
@ -619,6 +621,12 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void checkIndexSortCompatibility(IndexSortConfig sortConfig, boolean hasNested) {
|
||||||
|
if (sortConfig.hasIndexSort() && hasNested) {
|
||||||
|
throw new IllegalArgumentException("cannot have nested fields when index sort is activated");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public DocumentMapper parse(String mappingType, CompressedXContent mappingSource, boolean applyDefault) throws MapperParsingException {
|
public DocumentMapper parse(String mappingType, CompressedXContent mappingSource, boolean applyDefault) throws MapperParsingException {
|
||||||
return documentParser.parse(mappingType, mappingSource, applyDefault ? defaultMappingSource : null);
|
return documentParser.parse(mappingType, mappingSource, applyDefault ? defaultMappingSource : null);
|
||||||
}
|
}
|
||||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.lucene.index.SnapshotDeletionPolicy;
|
||||||
import org.apache.lucene.index.Term;
|
import org.apache.lucene.index.Term;
|
||||||
import org.apache.lucene.search.Query;
|
import org.apache.lucene.search.Query;
|
||||||
import org.apache.lucene.search.QueryCachingPolicy;
|
import org.apache.lucene.search.QueryCachingPolicy;
|
||||||
|
import org.apache.lucene.search.Sort;
|
||||||
import org.apache.lucene.search.UsageTrackingQueryCachingPolicy;
|
import org.apache.lucene.search.UsageTrackingQueryCachingPolicy;
|
||||||
import org.apache.lucene.store.AlreadyClosedException;
|
import org.apache.lucene.store.AlreadyClosedException;
|
||||||
import org.apache.lucene.store.Lock;
|
import org.apache.lucene.store.Lock;
|
||||||
|
@ -145,6 +146,7 @@ import java.util.concurrent.atomic.AtomicLong;
|
||||||
import java.util.concurrent.atomic.AtomicReference;
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
import java.util.function.BiConsumer;
|
import java.util.function.BiConsumer;
|
||||||
import java.util.function.Consumer;
|
import java.util.function.Consumer;
|
||||||
|
import java.util.function.Supplier;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
public class IndexShard extends AbstractIndexShardComponent implements IndicesClusterStateService.Shard {
|
public class IndexShard extends AbstractIndexShardComponent implements IndicesClusterStateService.Shard {
|
||||||
|
@ -170,7 +172,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
private final TranslogConfig translogConfig;
|
private final TranslogConfig translogConfig;
|
||||||
private final IndexEventListener indexEventListener;
|
private final IndexEventListener indexEventListener;
|
||||||
private final QueryCachingPolicy cachingPolicy;
|
private final QueryCachingPolicy cachingPolicy;
|
||||||
|
private final Supplier<Sort> indexSortSupplier;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* How many bytes we are currently moving to disk, via either IndexWriter.flush or refresh. IndexingMemoryController polls this
|
* How many bytes we are currently moving to disk, via either IndexWriter.flush or refresh. IndexingMemoryController polls this
|
||||||
|
@ -225,9 +227,9 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
*/
|
*/
|
||||||
private final RefreshListeners refreshListeners;
|
private final RefreshListeners refreshListeners;
|
||||||
|
|
||||||
public IndexShard(ShardRouting shardRouting, IndexSettings indexSettings, ShardPath path, Store store, IndexCache indexCache,
|
public IndexShard(ShardRouting shardRouting, IndexSettings indexSettings, ShardPath path, Store store,
|
||||||
MapperService mapperService, SimilarityService similarityService, IndexFieldDataService indexFieldDataService,
|
Supplier<Sort> indexSortSupplier, IndexCache indexCache, MapperService mapperService, SimilarityService similarityService,
|
||||||
@Nullable EngineFactory engineFactory,
|
IndexFieldDataService indexFieldDataService, @Nullable EngineFactory engineFactory,
|
||||||
IndexEventListener indexEventListener, IndexSearcherWrapper indexSearcherWrapper, ThreadPool threadPool, BigArrays bigArrays,
|
IndexEventListener indexEventListener, IndexSearcherWrapper indexSearcherWrapper, ThreadPool threadPool, BigArrays bigArrays,
|
||||||
Engine.Warmer warmer, Runnable globalCheckpointSyncer, List<SearchOperationListener> searchOperationListener, List<IndexingOperationListener> listeners) throws IOException {
|
Engine.Warmer warmer, Runnable globalCheckpointSyncer, List<SearchOperationListener> searchOperationListener, List<IndexingOperationListener> listeners) throws IOException {
|
||||||
super(shardRouting.shardId(), indexSettings);
|
super(shardRouting.shardId(), indexSettings);
|
||||||
|
@ -241,6 +243,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
Objects.requireNonNull(store, "Store must be provided to the index shard");
|
Objects.requireNonNull(store, "Store must be provided to the index shard");
|
||||||
this.engineFactory = engineFactory == null ? new InternalEngineFactory() : engineFactory;
|
this.engineFactory = engineFactory == null ? new InternalEngineFactory() : engineFactory;
|
||||||
this.store = store;
|
this.store = store;
|
||||||
|
this.indexSortSupplier = indexSortSupplier;
|
||||||
this.indexEventListener = indexEventListener;
|
this.indexEventListener = indexEventListener;
|
||||||
this.threadPool = threadPool;
|
this.threadPool = threadPool;
|
||||||
this.mapperService = mapperService;
|
this.mapperService = mapperService;
|
||||||
|
@ -289,6 +292,12 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
return this.store;
|
return this.store;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the sort order of this index, or null if the index has no sort.
|
||||||
|
*/
|
||||||
|
public Sort getIndexSort() {
|
||||||
|
return indexSortSupplier.get();
|
||||||
|
}
|
||||||
/**
|
/**
|
||||||
* returns true if this shard supports indexing (i.e., write) operations.
|
* returns true if this shard supports indexing (i.e., write) operations.
|
||||||
*/
|
*/
|
||||||
|
@ -1788,11 +1797,12 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
|
|
||||||
private EngineConfig newEngineConfig(EngineConfig.OpenMode openMode, long maxUnsafeAutoIdTimestamp) {
|
private EngineConfig newEngineConfig(EngineConfig.OpenMode openMode, long maxUnsafeAutoIdTimestamp) {
|
||||||
final IndexShardRecoveryPerformer translogRecoveryPerformer = new IndexShardRecoveryPerformer(shardId, mapperService, logger);
|
final IndexShardRecoveryPerformer translogRecoveryPerformer = new IndexShardRecoveryPerformer(shardId, mapperService, logger);
|
||||||
|
Sort indexSort = indexSortSupplier.get();
|
||||||
return new EngineConfig(openMode, shardId,
|
return new EngineConfig(openMode, shardId,
|
||||||
threadPool, indexSettings, warmer, store, deletionPolicy, indexSettings.getMergePolicy(),
|
threadPool, indexSettings, warmer, store, deletionPolicy, indexSettings.getMergePolicy(),
|
||||||
mapperService.indexAnalyzer(), similarityService.similarity(mapperService), codecService, shardEventListener, translogRecoveryPerformer, indexCache.query(), cachingPolicy, translogConfig,
|
mapperService.indexAnalyzer(), similarityService.similarity(mapperService), codecService, shardEventListener, translogRecoveryPerformer, indexCache.query(), cachingPolicy, translogConfig,
|
||||||
IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING.get(indexSettings.getSettings()), refreshListeners,
|
IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING.get(indexSettings.getSettings()), refreshListeners,
|
||||||
maxUnsafeAutoIdTimestamp);
|
maxUnsafeAutoIdTimestamp, indexSort);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.lucene.index.IndexWriter;
|
||||||
import org.apache.lucene.index.IndexWriterConfig;
|
import org.apache.lucene.index.IndexWriterConfig;
|
||||||
import org.apache.lucene.index.NoMergePolicy;
|
import org.apache.lucene.index.NoMergePolicy;
|
||||||
import org.apache.lucene.index.SegmentInfos;
|
import org.apache.lucene.index.SegmentInfos;
|
||||||
|
import org.apache.lucene.search.Sort;
|
||||||
import org.apache.lucene.store.Directory;
|
import org.apache.lucene.store.Directory;
|
||||||
import org.apache.lucene.store.FilterDirectory;
|
import org.apache.lucene.store.FilterDirectory;
|
||||||
import org.apache.lucene.store.IOContext;
|
import org.apache.lucene.store.IOContext;
|
||||||
|
@ -109,11 +110,14 @@ final class StoreRecovery {
|
||||||
mappingUpdateConsumer.accept(mapping.key, mapping.value);
|
mappingUpdateConsumer.accept(mapping.key, mapping.value);
|
||||||
}
|
}
|
||||||
indexShard.mapperService().merge(indexMetaData, MapperService.MergeReason.MAPPING_RECOVERY, true);
|
indexShard.mapperService().merge(indexMetaData, MapperService.MergeReason.MAPPING_RECOVERY, true);
|
||||||
|
// now that the mapping is merged we can validate the index sort configuration.
|
||||||
|
Sort indexSort = indexShard.getIndexSort();
|
||||||
return executeRecovery(indexShard, () -> {
|
return executeRecovery(indexShard, () -> {
|
||||||
logger.debug("starting recovery from local shards {}", shards);
|
logger.debug("starting recovery from local shards {}", shards);
|
||||||
try {
|
try {
|
||||||
final Directory directory = indexShard.store().directory(); // don't close this directory!!
|
final Directory directory = indexShard.store().directory(); // don't close this directory!!
|
||||||
addIndices(indexShard.recoveryState().getIndex(), directory, shards.stream().map(s -> s.getSnapshotDirectory())
|
addIndices(indexShard.recoveryState().getIndex(), directory, indexSort,
|
||||||
|
shards.stream().map(s -> s.getSnapshotDirectory())
|
||||||
.collect(Collectors.toList()).toArray(new Directory[shards.size()]));
|
.collect(Collectors.toList()).toArray(new Directory[shards.size()]));
|
||||||
internalRecoverFromStore(indexShard);
|
internalRecoverFromStore(indexShard);
|
||||||
// just trigger a merge to do housekeeping on the
|
// just trigger a merge to do housekeeping on the
|
||||||
|
@ -128,16 +132,19 @@ final class StoreRecovery {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
void addIndices(RecoveryState.Index indexRecoveryStats, Directory target, Directory... sources) throws IOException {
|
void addIndices(RecoveryState.Index indexRecoveryStats, Directory target, Sort indexSort, Directory... sources) throws IOException {
|
||||||
target = new org.apache.lucene.store.HardlinkCopyDirectoryWrapper(target);
|
target = new org.apache.lucene.store.HardlinkCopyDirectoryWrapper(target);
|
||||||
try (IndexWriter writer = new IndexWriter(new StatsDirectoryWrapper(target, indexRecoveryStats),
|
IndexWriterConfig iwc = new IndexWriterConfig(null)
|
||||||
new IndexWriterConfig(null)
|
.setCommitOnClose(false)
|
||||||
.setCommitOnClose(false)
|
// we don't want merges to happen here - we call maybe merge on the engine
|
||||||
// we don't want merges to happen here - we call maybe merge on the engine
|
// later once we stared it up otherwise we would need to wait for it here
|
||||||
// later once we stared it up otherwise we would need to wait for it here
|
// we also don't specify a codec here and merges should use the engines for this index
|
||||||
// we also don't specify a codec here and merges should use the engines for this index
|
.setMergePolicy(NoMergePolicy.INSTANCE)
|
||||||
.setMergePolicy(NoMergePolicy.INSTANCE)
|
.setOpenMode(IndexWriterConfig.OpenMode.CREATE);
|
||||||
.setOpenMode(IndexWriterConfig.OpenMode.CREATE))) {
|
if (indexSort != null) {
|
||||||
|
iwc.setIndexSort(indexSort);
|
||||||
|
}
|
||||||
|
try (IndexWriter writer = new IndexWriter(new StatsDirectoryWrapper(target, indexRecoveryStats), iwc)) {
|
||||||
writer.addIndexes(sources);
|
writer.addIndexes(sources);
|
||||||
writer.commit();
|
writer.commit();
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,9 +19,17 @@
|
||||||
|
|
||||||
package org.elasticsearch.action.admin.indices.create;
|
package org.elasticsearch.action.admin.indices.create;
|
||||||
|
|
||||||
|
import org.apache.lucene.search.Sort;
|
||||||
|
import org.apache.lucene.search.SortField;
|
||||||
|
import org.apache.lucene.search.SortedSetSelector;
|
||||||
|
import org.apache.lucene.search.SortedSetSortField;
|
||||||
import org.elasticsearch.Version;
|
import org.elasticsearch.Version;
|
||||||
import org.elasticsearch.action.admin.cluster.reroute.ClusterRerouteResponse;
|
import org.elasticsearch.action.admin.cluster.reroute.ClusterRerouteResponse;
|
||||||
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
|
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
|
||||||
|
import org.elasticsearch.action.admin.indices.segments.IndexSegments;
|
||||||
|
import org.elasticsearch.action.admin.indices.segments.IndexShardSegments;
|
||||||
|
import org.elasticsearch.action.admin.indices.segments.IndicesSegmentResponse;
|
||||||
|
import org.elasticsearch.action.admin.indices.segments.ShardSegments;
|
||||||
import org.elasticsearch.action.admin.indices.settings.get.GetSettingsResponse;
|
import org.elasticsearch.action.admin.indices.settings.get.GetSettingsResponse;
|
||||||
import org.elasticsearch.action.support.ActiveShardCount;
|
import org.elasticsearch.action.support.ActiveShardCount;
|
||||||
import org.elasticsearch.cluster.ClusterInfoService;
|
import org.elasticsearch.cluster.ClusterInfoService;
|
||||||
|
@ -33,6 +41,7 @@ import org.elasticsearch.common.Priority;
|
||||||
import org.elasticsearch.common.collect.ImmutableOpenMap;
|
import org.elasticsearch.common.collect.ImmutableOpenMap;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.xcontent.XContentType;
|
import org.elasticsearch.common.xcontent.XContentType;
|
||||||
|
import org.elasticsearch.index.engine.Segment;
|
||||||
import org.elasticsearch.index.query.TermsQueryBuilder;
|
import org.elasticsearch.index.query.TermsQueryBuilder;
|
||||||
import org.elasticsearch.plugins.Plugin;
|
import org.elasticsearch.plugins.Plugin;
|
||||||
import org.elasticsearch.test.ESIntegTestCase;
|
import org.elasticsearch.test.ESIntegTestCase;
|
||||||
|
@ -44,6 +53,7 @@ import java.util.Collection;
|
||||||
|
|
||||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
||||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
|
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
|
||||||
|
import static org.hamcrest.Matchers.containsString;
|
||||||
|
|
||||||
public class ShrinkIndexIT extends ESIntegTestCase {
|
public class ShrinkIndexIT extends ESIntegTestCase {
|
||||||
|
|
||||||
|
@ -250,4 +260,76 @@ public class ShrinkIndexIT extends ESIntegTestCase {
|
||||||
ensureGreen();
|
ensureGreen();
|
||||||
assertHitCount(client().prepareSearch("target").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), 20);
|
assertHitCount(client().prepareSearch("target").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), 20);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void testCreateShrinkWithIndexSort() throws Exception {
|
||||||
|
SortField expectedSortField = new SortedSetSortField("id", true, SortedSetSelector.Type.MAX);
|
||||||
|
expectedSortField.setMissingValue(SortedSetSortField.STRING_FIRST);
|
||||||
|
Sort expectedIndexSort = new Sort(expectedSortField);
|
||||||
|
internalCluster().ensureAtLeastNumDataNodes(2);
|
||||||
|
prepareCreate("source")
|
||||||
|
.setSettings(
|
||||||
|
Settings.builder()
|
||||||
|
.put(indexSettings())
|
||||||
|
.put("sort.field", "id")
|
||||||
|
.put("sort.order", "desc")
|
||||||
|
.put("number_of_shards", 8)
|
||||||
|
.put("number_of_replicas", 0)
|
||||||
|
)
|
||||||
|
.addMapping("t1", "id", "type=keyword,doc_values=true")
|
||||||
|
.get();
|
||||||
|
for (int i = 0; i < 20; i++) {
|
||||||
|
client().prepareIndex("source", "t1", Integer.toString(i))
|
||||||
|
.setSource("{\"foo\" : \"bar\", \"id\" : " + i + "}", XContentType.JSON).get();
|
||||||
|
}
|
||||||
|
ImmutableOpenMap<String, DiscoveryNode> dataNodes = client().admin().cluster().prepareState().get().getState().nodes()
|
||||||
|
.getDataNodes();
|
||||||
|
assertTrue("at least 2 nodes but was: " + dataNodes.size(), dataNodes.size() >= 2);
|
||||||
|
DiscoveryNode[] discoveryNodes = dataNodes.values().toArray(DiscoveryNode.class);
|
||||||
|
String mergeNode = discoveryNodes[0].getName();
|
||||||
|
// ensure all shards are allocated otherwise the ensure green below might not succeed since we require the merge node
|
||||||
|
// if we change the setting too quickly we will end up with one replica unassigned which can't be assigned anymore due
|
||||||
|
// to the require._name below.
|
||||||
|
ensureGreen();
|
||||||
|
|
||||||
|
flushAndRefresh();
|
||||||
|
assertSortedSegments("source", expectedIndexSort);
|
||||||
|
|
||||||
|
// relocate all shards to one node such that we can merge it.
|
||||||
|
client().admin().indices().prepareUpdateSettings("source")
|
||||||
|
.setSettings(Settings.builder()
|
||||||
|
.put("index.routing.allocation.require._name", mergeNode)
|
||||||
|
.put("index.blocks.write", true)).get();
|
||||||
|
ensureGreen();
|
||||||
|
|
||||||
|
// check that index sort cannot be set on the target index
|
||||||
|
IllegalArgumentException exc = expectThrows(IllegalArgumentException.class,
|
||||||
|
() -> client().admin().indices().prepareShrinkIndex("source", "target")
|
||||||
|
.setSettings(Settings.builder()
|
||||||
|
.put("index.number_of_replicas", 0)
|
||||||
|
.put("index.number_of_shards", "2")
|
||||||
|
.put("index.sort.field", "foo")
|
||||||
|
.build()).get());
|
||||||
|
assertThat(exc.getMessage(), containsString("can't override index sort when shrinking index"));
|
||||||
|
|
||||||
|
// check that the index sort order of `source` is correctly applied to the `target`
|
||||||
|
assertAcked(client().admin().indices().prepareShrinkIndex("source", "target")
|
||||||
|
.setSettings(Settings.builder()
|
||||||
|
.put("index.number_of_replicas", 0)
|
||||||
|
.put("index.number_of_shards", "2").build()).get());
|
||||||
|
ensureGreen();
|
||||||
|
flushAndRefresh();
|
||||||
|
GetSettingsResponse settingsResponse =
|
||||||
|
client().admin().indices().prepareGetSettings("target").execute().actionGet();
|
||||||
|
assertEquals(settingsResponse.getSetting("target", "index.sort.field"), "id");
|
||||||
|
assertEquals(settingsResponse.getSetting("target", "index.sort.order"), "desc");
|
||||||
|
assertSortedSegments("target", expectedIndexSort);
|
||||||
|
|
||||||
|
// ... and that the index sort is also applied to updates
|
||||||
|
for (int i = 20; i < 40; i++) {
|
||||||
|
client().prepareIndex("target", randomFrom("t1", "t2", "t3"))
|
||||||
|
.setSource("{\"foo\" : \"bar\", \"i\" : " + i + "}", XContentType.JSON).get();
|
||||||
|
}
|
||||||
|
flushAndRefresh();
|
||||||
|
assertSortedSegments("target", expectedIndexSort);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,131 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Elasticsearch under one or more contributor
|
||||||
|
* license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright
|
||||||
|
* ownership. Elasticsearch licenses this file to you under
|
||||||
|
* the Apache License, Version 2.0 (the "License"); you may
|
||||||
|
* not use this file except in compliance with the License.
|
||||||
|
* You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.index;
|
||||||
|
|
||||||
|
import org.apache.lucene.search.Sort;
|
||||||
|
import org.apache.lucene.search.SortField;
|
||||||
|
import org.apache.lucene.search.SortedNumericSortField;
|
||||||
|
import org.apache.lucene.search.SortedSetSortField;
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.test.ESIntegTestCase;
|
||||||
|
import org.junit.AfterClass;
|
||||||
|
import org.junit.BeforeClass;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||||
|
import static org.hamcrest.Matchers.containsString;
|
||||||
|
|
||||||
|
public class IndexSortIT extends ESIntegTestCase {
|
||||||
|
private static final XContentBuilder TEST_MAPPING = createTestMapping();
|
||||||
|
|
||||||
|
private static XContentBuilder createTestMapping() {
|
||||||
|
try {
|
||||||
|
return jsonBuilder()
|
||||||
|
.startObject()
|
||||||
|
.startObject("properties")
|
||||||
|
.startObject("date")
|
||||||
|
.field("type", "date")
|
||||||
|
.endObject()
|
||||||
|
.startObject("numeric")
|
||||||
|
.field("type", "integer")
|
||||||
|
.field("doc_values", false)
|
||||||
|
.endObject()
|
||||||
|
.startObject("numeric_dv")
|
||||||
|
.field("type", "integer")
|
||||||
|
.field("doc_values", true)
|
||||||
|
.endObject()
|
||||||
|
.startObject("keyword_dv")
|
||||||
|
.field("type", "keyword")
|
||||||
|
.field("doc_values", true)
|
||||||
|
.endObject()
|
||||||
|
.startObject("keyword")
|
||||||
|
.field("type", "keyword")
|
||||||
|
.field("doc_values", false)
|
||||||
|
.endObject()
|
||||||
|
.endObject().endObject();
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new IllegalStateException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testIndexSort() {
|
||||||
|
SortField dateSort = new SortedNumericSortField("date", SortField.Type.LONG, false);
|
||||||
|
dateSort.setMissingValue(Long.MAX_VALUE);
|
||||||
|
SortField numericSort = new SortedNumericSortField("numeric_dv", SortField.Type.LONG, false);
|
||||||
|
numericSort.setMissingValue(Long.MAX_VALUE);
|
||||||
|
SortField keywordSort = new SortedSetSortField("keyword_dv", false);
|
||||||
|
keywordSort.setMissingValue(SortField.STRING_LAST);
|
||||||
|
Sort indexSort = new Sort(dateSort, numericSort, keywordSort);
|
||||||
|
prepareCreate("test")
|
||||||
|
.setSettings(Settings.builder()
|
||||||
|
.put(indexSettings())
|
||||||
|
.put("index.number_of_shards", "1")
|
||||||
|
.put("index.number_of_replicas", "1")
|
||||||
|
.putArray("index.sort.field", "date", "numeric_dv", "keyword_dv")
|
||||||
|
)
|
||||||
|
.addMapping("test", TEST_MAPPING)
|
||||||
|
.get();
|
||||||
|
for (int i = 0; i < 20; i++) {
|
||||||
|
client().prepareIndex("test", "test", Integer.toString(i))
|
||||||
|
.setSource("numeric_dv", randomInt(), "keyword_dv", randomAlphaOfLengthBetween(10, 20))
|
||||||
|
.get();
|
||||||
|
}
|
||||||
|
flushAndRefresh();
|
||||||
|
ensureYellow();
|
||||||
|
assertSortedSegments("test", indexSort);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testInvalidIndexSort() {
|
||||||
|
IllegalArgumentException exc = expectThrows(IllegalArgumentException.class,
|
||||||
|
() -> prepareCreate("test")
|
||||||
|
.setSettings(Settings.builder()
|
||||||
|
.put(indexSettings())
|
||||||
|
.putArray("index.sort.field", "invalid_field")
|
||||||
|
)
|
||||||
|
.addMapping("test", TEST_MAPPING)
|
||||||
|
.get()
|
||||||
|
);
|
||||||
|
assertThat(exc.getMessage(), containsString("unknown index sort field:[invalid_field]"));
|
||||||
|
|
||||||
|
exc = expectThrows(IllegalArgumentException.class,
|
||||||
|
() -> prepareCreate("test")
|
||||||
|
.setSettings(Settings.builder()
|
||||||
|
.put(indexSettings())
|
||||||
|
.putArray("index.sort.field", "numeric")
|
||||||
|
)
|
||||||
|
.addMapping("test", TEST_MAPPING)
|
||||||
|
.get()
|
||||||
|
);
|
||||||
|
assertThat(exc.getMessage(), containsString("docvalues not found for index sort field:[numeric]"));
|
||||||
|
|
||||||
|
exc = expectThrows(IllegalArgumentException.class,
|
||||||
|
() -> prepareCreate("test")
|
||||||
|
.setSettings(Settings.builder()
|
||||||
|
.put(indexSettings())
|
||||||
|
.putArray("index.sort.field", "keyword")
|
||||||
|
)
|
||||||
|
.addMapping("test", TEST_MAPPING)
|
||||||
|
.get()
|
||||||
|
);
|
||||||
|
assertThat(exc.getMessage(), containsString("docvalues not found for index sort field:[keyword]"));
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,160 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Elasticsearch under one or more contributor
|
||||||
|
* license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright
|
||||||
|
* ownership. Elasticsearch licenses this file to you under
|
||||||
|
* the Apache License, Version 2.0 (the "License"); you may
|
||||||
|
* not use this file except in compliance with the License.
|
||||||
|
* You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.index;
|
||||||
|
|
||||||
|
import org.elasticsearch.Version;
|
||||||
|
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.search.MultiValueMode;
|
||||||
|
import org.elasticsearch.search.sort.SortOrder;
|
||||||
|
import org.elasticsearch.test.ESTestCase;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import static org.elasticsearch.common.settings.Settings.Builder.EMPTY_SETTINGS;
|
||||||
|
import static org.elasticsearch.index.IndexSettingsTests.newIndexMeta;
|
||||||
|
import static org.hamcrest.Matchers.containsString;
|
||||||
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
|
|
||||||
|
public class IndexSortSettingsTests extends ESTestCase {
|
||||||
|
private static IndexSettings indexSettings(Settings settings) {
|
||||||
|
return indexSettings(settings, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static IndexSettings indexSettings(Settings settings, Version version) {
|
||||||
|
final Settings newSettings;
|
||||||
|
if (version != null) {
|
||||||
|
newSettings = Settings.builder()
|
||||||
|
.put(settings)
|
||||||
|
.put(IndexMetaData.SETTING_VERSION_CREATED, version)
|
||||||
|
.build();
|
||||||
|
} else {
|
||||||
|
newSettings = settings;
|
||||||
|
}
|
||||||
|
return new IndexSettings(newIndexMeta("test", newSettings), Settings.EMPTY);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testNoIndexSort() throws IOException {
|
||||||
|
IndexSettings indexSettings = indexSettings(EMPTY_SETTINGS);
|
||||||
|
assertFalse(indexSettings.getIndexSortConfig().hasIndexSort());
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testSimpleIndexSort() throws IOException {
|
||||||
|
Settings settings = Settings.builder()
|
||||||
|
.put("index.sort.field", "field1")
|
||||||
|
.put("index.sort.order", "asc")
|
||||||
|
.put("index.sort.mode", "max")
|
||||||
|
.put("index.sort.missing", "_last")
|
||||||
|
.build();
|
||||||
|
IndexSettings indexSettings = indexSettings(settings);
|
||||||
|
IndexSortConfig config = indexSettings.getIndexSortConfig();
|
||||||
|
assertTrue(config.hasIndexSort());
|
||||||
|
assertThat(config.sortSpecs.length, equalTo(1));
|
||||||
|
|
||||||
|
assertThat(config.sortSpecs[0].field, equalTo("field1"));
|
||||||
|
assertThat(config.sortSpecs[0].order, equalTo(SortOrder.ASC));
|
||||||
|
assertThat(config.sortSpecs[0].missingValue, equalTo("_last"));
|
||||||
|
assertThat(config.sortSpecs[0].mode, equalTo(MultiValueMode.MAX));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testIndexSortWithArrays() throws IOException {
|
||||||
|
Settings settings = Settings.builder()
|
||||||
|
.putArray("index.sort.field", "field1", "field2")
|
||||||
|
.putArray("index.sort.order", "asc", "desc")
|
||||||
|
.putArray("index.sort.missing", "_last", "_first")
|
||||||
|
.build();
|
||||||
|
IndexSettings indexSettings = indexSettings(settings);
|
||||||
|
IndexSortConfig config = indexSettings.getIndexSortConfig();
|
||||||
|
assertTrue(config.hasIndexSort());
|
||||||
|
assertThat(config.sortSpecs.length, equalTo(2));
|
||||||
|
|
||||||
|
assertThat(config.sortSpecs[0].field, equalTo("field1"));
|
||||||
|
assertThat(config.sortSpecs[1].field, equalTo("field2"));
|
||||||
|
assertThat(config.sortSpecs[0].order, equalTo(SortOrder.ASC));
|
||||||
|
assertThat(config.sortSpecs[1].order, equalTo(SortOrder.DESC));
|
||||||
|
assertThat(config.sortSpecs[0].missingValue, equalTo("_last"));
|
||||||
|
assertThat(config.sortSpecs[1].missingValue, equalTo("_first"));
|
||||||
|
assertNull(config.sortSpecs[0].mode);
|
||||||
|
assertNull(config.sortSpecs[1].mode);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testInvalidIndexSort() throws IOException {
|
||||||
|
final Settings settings = Settings.builder()
|
||||||
|
.put("index.sort.field", "field1")
|
||||||
|
.put("index.sort.order", "asc, desc")
|
||||||
|
.build();
|
||||||
|
IllegalArgumentException exc =
|
||||||
|
expectThrows(IllegalArgumentException.class, () -> indexSettings(settings));
|
||||||
|
assertThat(exc.getMessage(), containsString("index.sort.field:[field1] index.sort.order:[asc, desc], size mismatch"));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testInvalidIndexSortWithArray() throws IOException {
|
||||||
|
final Settings settings = Settings.builder()
|
||||||
|
.put("index.sort.field", "field1")
|
||||||
|
.putArray("index.sort.order", new String[] {"asc", "desc"})
|
||||||
|
.build();
|
||||||
|
IllegalArgumentException exc =
|
||||||
|
expectThrows(IllegalArgumentException.class, () -> indexSettings(settings));
|
||||||
|
assertThat(exc.getMessage(),
|
||||||
|
containsString("index.sort.field:[field1] index.sort.order:[asc, desc], size mismatch"));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testInvalidOrder() throws IOException {
|
||||||
|
final Settings settings = Settings.builder()
|
||||||
|
.put("index.sort.field", "field1")
|
||||||
|
.put("index.sort.order", "invalid")
|
||||||
|
.build();
|
||||||
|
IllegalArgumentException exc =
|
||||||
|
expectThrows(IllegalArgumentException.class, () -> indexSettings(settings));
|
||||||
|
assertThat(exc.getMessage(), containsString("Illegal sort order:invalid"));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testInvalidMode() throws IOException {
|
||||||
|
final Settings settings = Settings.builder()
|
||||||
|
.put("index.sort.field", "field1")
|
||||||
|
.put("index.sort.mode", "invalid")
|
||||||
|
.build();
|
||||||
|
IllegalArgumentException exc =
|
||||||
|
expectThrows(IllegalArgumentException.class, () -> indexSettings(settings));
|
||||||
|
assertThat(exc.getMessage(), containsString("Illegal sort mode: invalid"));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testInvalidMissing() throws IOException {
|
||||||
|
final Settings settings = Settings.builder()
|
||||||
|
.put("index.sort.field", "field1")
|
||||||
|
.put("index.sort.missing", "default")
|
||||||
|
.build();
|
||||||
|
IllegalArgumentException exc =
|
||||||
|
expectThrows(IllegalArgumentException.class, () -> indexSettings(settings));
|
||||||
|
assertThat(exc.getMessage(), containsString("Illegal missing value:[default]," +
|
||||||
|
" must be one of [_last, _first]"));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testInvalidVersion() throws IOException {
|
||||||
|
final Settings settings = Settings.builder()
|
||||||
|
.put("index.sort.field", "field1")
|
||||||
|
.build();
|
||||||
|
IllegalArgumentException exc =
|
||||||
|
expectThrows(IllegalArgumentException.class, () -> indexSettings(settings, Version.V_5_4_0_UNRELEASED));
|
||||||
|
assertThat(exc.getMessage(),
|
||||||
|
containsString("unsupported index.version.created:5.4.0, " +
|
||||||
|
"can't set index.sort on versions prior to 6.0.0-alpha1"));
|
||||||
|
}
|
||||||
|
}
|
|
@ -58,6 +58,8 @@ import org.apache.lucene.search.MatchAllDocsQuery;
|
||||||
import org.apache.lucene.search.ReferenceManager;
|
import org.apache.lucene.search.ReferenceManager;
|
||||||
import org.apache.lucene.search.TermQuery;
|
import org.apache.lucene.search.TermQuery;
|
||||||
import org.apache.lucene.search.TopDocs;
|
import org.apache.lucene.search.TopDocs;
|
||||||
|
import org.apache.lucene.search.Sort;
|
||||||
|
import org.apache.lucene.search.SortedSetSortField;
|
||||||
import org.apache.lucene.search.TotalHitCountCollector;
|
import org.apache.lucene.search.TotalHitCountCollector;
|
||||||
import org.apache.lucene.store.AlreadyClosedException;
|
import org.apache.lucene.store.AlreadyClosedException;
|
||||||
import org.apache.lucene.store.Directory;
|
import org.apache.lucene.store.Directory;
|
||||||
|
@ -260,7 +262,7 @@ public class InternalEngineTests extends ESTestCase {
|
||||||
config.getStore(), config.getDeletionPolicy(), config.getMergePolicy(), analyzer, config.getSimilarity(),
|
config.getStore(), config.getDeletionPolicy(), config.getMergePolicy(), analyzer, config.getSimilarity(),
|
||||||
new CodecService(null, logger), config.getEventListener(), config.getTranslogRecoveryPerformer(), config.getQueryCache(),
|
new CodecService(null, logger), config.getEventListener(), config.getTranslogRecoveryPerformer(), config.getQueryCache(),
|
||||||
config.getQueryCachingPolicy(), config.getTranslogConfig(), config.getFlushMergesAfter(), config.getRefreshListeners(),
|
config.getQueryCachingPolicy(), config.getTranslogConfig(), config.getFlushMergesAfter(), config.getRefreshListeners(),
|
||||||
config.getMaxUnsafeAutoIdTimestamp());
|
config.getMaxUnsafeAutoIdTimestamp(), config.getIndexSort());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -358,7 +360,18 @@ public class InternalEngineTests extends ESTestCase {
|
||||||
MergePolicy mergePolicy,
|
MergePolicy mergePolicy,
|
||||||
@Nullable IndexWriterFactory indexWriterFactory,
|
@Nullable IndexWriterFactory indexWriterFactory,
|
||||||
@Nullable Supplier<SequenceNumbersService> sequenceNumbersServiceSupplier) throws IOException {
|
@Nullable Supplier<SequenceNumbersService> sequenceNumbersServiceSupplier) throws IOException {
|
||||||
EngineConfig config = config(indexSettings, store, translogPath, mergePolicy, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, null);
|
return createEngine(indexSettings, store, translogPath, mergePolicy, indexWriterFactory, sequenceNumbersServiceSupplier, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected InternalEngine createEngine(
|
||||||
|
IndexSettings indexSettings,
|
||||||
|
Store store,
|
||||||
|
Path translogPath,
|
||||||
|
MergePolicy mergePolicy,
|
||||||
|
@Nullable IndexWriterFactory indexWriterFactory,
|
||||||
|
@Nullable Supplier<SequenceNumbersService> sequenceNumbersServiceSupplier,
|
||||||
|
@Nullable Sort indexSort) throws IOException {
|
||||||
|
EngineConfig config = config(indexSettings, store, translogPath, mergePolicy, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, null, indexSort);
|
||||||
InternalEngine internalEngine = createInternalEngine(indexWriterFactory, sequenceNumbersServiceSupplier, config);
|
InternalEngine internalEngine = createInternalEngine(indexWriterFactory, sequenceNumbersServiceSupplier, config);
|
||||||
if (config.getOpenMode() == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG) {
|
if (config.getOpenMode() == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG) {
|
||||||
internalEngine.recoverFromTranslog();
|
internalEngine.recoverFromTranslog();
|
||||||
|
@ -393,12 +406,24 @@ public class InternalEngineTests extends ESTestCase {
|
||||||
public EngineConfig config(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy,
|
public EngineConfig config(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy,
|
||||||
long maxUnsafeAutoIdTimestamp, ReferenceManager.RefreshListener refreshListener) {
|
long maxUnsafeAutoIdTimestamp, ReferenceManager.RefreshListener refreshListener) {
|
||||||
return config(indexSettings, store, translogPath, mergePolicy, createSnapshotDeletionPolicy(),
|
return config(indexSettings, store, translogPath, mergePolicy, createSnapshotDeletionPolicy(),
|
||||||
maxUnsafeAutoIdTimestamp, refreshListener);
|
maxUnsafeAutoIdTimestamp, refreshListener, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
public EngineConfig config(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy,
|
||||||
|
long maxUnsafeAutoIdTimestamp, ReferenceManager.RefreshListener refreshListener, Sort indexSort) {
|
||||||
|
return config(indexSettings, store, translogPath, mergePolicy, createSnapshotDeletionPolicy(),
|
||||||
|
maxUnsafeAutoIdTimestamp, refreshListener, indexSort);
|
||||||
}
|
}
|
||||||
|
|
||||||
public EngineConfig config(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy,
|
public EngineConfig config(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy,
|
||||||
SnapshotDeletionPolicy deletionPolicy, long maxUnsafeAutoIdTimestamp,
|
SnapshotDeletionPolicy deletionPolicy, long maxUnsafeAutoIdTimestamp,
|
||||||
ReferenceManager.RefreshListener refreshListener) {
|
ReferenceManager.RefreshListener refreshListener) {
|
||||||
|
return config(indexSettings, store, translogPath, mergePolicy, deletionPolicy, maxUnsafeAutoIdTimestamp, refreshListener, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
public EngineConfig config(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy,
|
||||||
|
SnapshotDeletionPolicy deletionPolicy, long maxUnsafeAutoIdTimestamp,
|
||||||
|
ReferenceManager.RefreshListener refreshListener, Sort indexSort) {
|
||||||
IndexWriterConfig iwc = newIndexWriterConfig();
|
IndexWriterConfig iwc = newIndexWriterConfig();
|
||||||
TranslogConfig translogConfig = new TranslogConfig(shardId, translogPath, indexSettings, BigArrays.NON_RECYCLING_INSTANCE);
|
TranslogConfig translogConfig = new TranslogConfig(shardId, translogPath, indexSettings, BigArrays.NON_RECYCLING_INSTANCE);
|
||||||
final EngineConfig.OpenMode openMode;
|
final EngineConfig.OpenMode openMode;
|
||||||
|
@ -421,7 +446,7 @@ public class InternalEngineTests extends ESTestCase {
|
||||||
mergePolicy, iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(null, logger), listener,
|
mergePolicy, iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(null, logger), listener,
|
||||||
new TranslogHandler(xContentRegistry(), shardId.getIndexName(), logger), IndexSearcher.getDefaultQueryCache(),
|
new TranslogHandler(xContentRegistry(), shardId.getIndexName(), logger), IndexSearcher.getDefaultQueryCache(),
|
||||||
IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5), refreshListener,
|
IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5), refreshListener,
|
||||||
maxUnsafeAutoIdTimestamp);
|
maxUnsafeAutoIdTimestamp, indexSort);
|
||||||
|
|
||||||
return config;
|
return config;
|
||||||
}
|
}
|
||||||
|
@ -636,6 +661,37 @@ public class InternalEngineTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void testSegmentsWithIndexSort() throws Exception {
|
||||||
|
Sort indexSort = new Sort(new SortedSetSortField("_type", false));
|
||||||
|
try (Store store = createStore();
|
||||||
|
Engine engine = createEngine(defaultSettings, store, createTempDir(), NoMergePolicy.INSTANCE,
|
||||||
|
null, null, indexSort)) {
|
||||||
|
List<Segment> segments = engine.segments(true);
|
||||||
|
assertThat(segments.isEmpty(), equalTo(true));
|
||||||
|
|
||||||
|
ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField(), B_1, null);
|
||||||
|
engine.index(indexForDoc(doc));
|
||||||
|
engine.refresh("test");
|
||||||
|
|
||||||
|
segments = engine.segments(false);
|
||||||
|
assertThat(segments.size(), equalTo(1));
|
||||||
|
assertThat(segments.get(0).getSegmentSort(), equalTo(indexSort));
|
||||||
|
|
||||||
|
ParsedDocument doc2 = testParsedDocument("2", "test", null, testDocumentWithTextField(), B_2, null);
|
||||||
|
engine.index(indexForDoc(doc2));
|
||||||
|
engine.refresh("test");
|
||||||
|
ParsedDocument doc3 = testParsedDocument("3", "test", null, testDocumentWithTextField(), B_3, null);
|
||||||
|
engine.index(indexForDoc(doc3));
|
||||||
|
engine.refresh("test");
|
||||||
|
|
||||||
|
segments = engine.segments(true);
|
||||||
|
assertThat(segments.size(), equalTo(3));
|
||||||
|
assertThat(segments.get(0).getSegmentSort(), equalTo(indexSort));
|
||||||
|
assertThat(segments.get(1).getSegmentSort(), equalTo(indexSort));
|
||||||
|
assertThat(segments.get(2).getSegmentSort(), equalTo(indexSort));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public void testSegmentsStatsIncludingFileSizes() throws Exception {
|
public void testSegmentsStatsIncludingFileSizes() throws Exception {
|
||||||
try (Store store = createStore();
|
try (Store store = createStore();
|
||||||
Engine engine = createEngine(defaultSettings, store, createTempDir(), NoMergePolicy.INSTANCE)) {
|
Engine engine = createEngine(defaultSettings, store, createTempDir(), NoMergePolicy.INSTANCE)) {
|
||||||
|
@ -2680,7 +2736,7 @@ public class InternalEngineTests extends ESTestCase {
|
||||||
config.getIndexSettings(), null, store, createSnapshotDeletionPolicy(), newMergePolicy(), config.getAnalyzer(),
|
config.getIndexSettings(), null, store, createSnapshotDeletionPolicy(), newMergePolicy(), config.getAnalyzer(),
|
||||||
config.getSimilarity(), new CodecService(null, logger), config.getEventListener(), config.getTranslogRecoveryPerformer(),
|
config.getSimilarity(), new CodecService(null, logger), config.getEventListener(), config.getTranslogRecoveryPerformer(),
|
||||||
IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig,
|
IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig,
|
||||||
TimeValue.timeValueMinutes(5), config.getRefreshListeners(), IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP);
|
TimeValue.timeValueMinutes(5), config.getRefreshListeners(), IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, null);
|
||||||
|
|
||||||
try {
|
try {
|
||||||
InternalEngine internalEngine = new InternalEngine(brokenConfig);
|
InternalEngine internalEngine = new InternalEngine(brokenConfig);
|
||||||
|
|
|
@ -0,0 +1,114 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Elasticsearch under one or more contributor
|
||||||
|
* license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright
|
||||||
|
* ownership. Elasticsearch licenses this file to you under
|
||||||
|
* the Apache License, Version 2.0 (the "License"); you may
|
||||||
|
* not use this file except in compliance with the License.
|
||||||
|
* You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.index.engine;
|
||||||
|
|
||||||
|
import org.apache.lucene.search.Sort;
|
||||||
|
import org.apache.lucene.search.SortedNumericSelector;
|
||||||
|
import org.apache.lucene.search.SortedNumericSortField;
|
||||||
|
import org.apache.lucene.search.SortedSetSortField;
|
||||||
|
import org.apache.lucene.search.SortedSetSelector;
|
||||||
|
import org.apache.lucene.search.SortField;
|
||||||
|
import org.apache.lucene.util.Version;
|
||||||
|
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.test.ESTestCase;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
public class SegmentTests extends ESTestCase {
|
||||||
|
static SortField randomSortField() {
|
||||||
|
if (randomBoolean()) {
|
||||||
|
SortedNumericSortField field =
|
||||||
|
new SortedNumericSortField(randomAlphaOfLengthBetween(1, 10),
|
||||||
|
SortField.Type.INT,
|
||||||
|
randomBoolean(),
|
||||||
|
randomBoolean() ? SortedNumericSelector.Type.MAX : SortedNumericSelector.Type.MIN);
|
||||||
|
if (randomBoolean()) {
|
||||||
|
field.setMissingValue(randomInt());
|
||||||
|
}
|
||||||
|
return field;
|
||||||
|
} else {
|
||||||
|
SortedSetSortField field =
|
||||||
|
new SortedSetSortField(randomAlphaOfLengthBetween(1, 10),
|
||||||
|
randomBoolean(),
|
||||||
|
randomBoolean() ? SortedSetSelector.Type.MAX : SortedSetSelector.Type.MIN);
|
||||||
|
if (randomBoolean()) {
|
||||||
|
field.setMissingValue(randomBoolean() ? SortedSetSortField.STRING_FIRST : SortedSetSortField.STRING_LAST);
|
||||||
|
}
|
||||||
|
return field;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static Sort randomIndexSort() {
|
||||||
|
if (randomBoolean()) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
int size = randomIntBetween(1, 5);
|
||||||
|
SortField[] fields = new SortField[size];
|
||||||
|
for (int i = 0; i < size; i++) {
|
||||||
|
fields[i] = randomSortField();
|
||||||
|
}
|
||||||
|
return new Sort(fields);
|
||||||
|
}
|
||||||
|
|
||||||
|
static Segment randomSegment() {
|
||||||
|
Segment segment = new Segment(randomAlphaOfLength(10));
|
||||||
|
segment.committed = randomBoolean();
|
||||||
|
segment.search = randomBoolean();
|
||||||
|
segment.sizeInBytes = randomNonNegativeLong();
|
||||||
|
segment.docCount = randomIntBetween(1, Integer.MAX_VALUE);
|
||||||
|
segment.delDocCount = randomIntBetween(0, segment.docCount);
|
||||||
|
segment.version = Version.LUCENE_6_5_0;
|
||||||
|
segment.compound = randomBoolean();
|
||||||
|
segment.mergeId = randomAlphaOfLengthBetween(1, 10);
|
||||||
|
segment.memoryInBytes = randomNonNegativeLong();
|
||||||
|
segment.segmentSort = randomIndexSort();
|
||||||
|
return segment;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testSerialization() throws IOException {
|
||||||
|
for (int i = 0; i < 20; i++) {
|
||||||
|
Segment segment = randomSegment();
|
||||||
|
BytesStreamOutput output = new BytesStreamOutput();
|
||||||
|
segment.writeTo(output);
|
||||||
|
output.flush();
|
||||||
|
StreamInput input = output.bytes().streamInput();
|
||||||
|
Segment deserialized = new Segment();
|
||||||
|
deserialized.readFrom(input);
|
||||||
|
assertTrue(isSegmentEquals(deserialized, segment));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static boolean isSegmentEquals(Segment seg1, Segment seg2) {
|
||||||
|
return seg1.docCount == seg2.docCount &&
|
||||||
|
seg1.delDocCount == seg2.delDocCount &&
|
||||||
|
seg1.committed == seg2.committed &&
|
||||||
|
seg1.search == seg2.search &&
|
||||||
|
Objects.equals(seg1.version, seg2.version) &&
|
||||||
|
Objects.equals(seg1.compound, seg2.compound) &&
|
||||||
|
seg1.sizeInBytes == seg2.sizeInBytes &&
|
||||||
|
seg1.memoryInBytes == seg2.memoryInBytes &&
|
||||||
|
seg1.getGeneration() == seg2.getGeneration() &&
|
||||||
|
seg1.getName().equals(seg2.getName()) &&
|
||||||
|
seg1.getMergeId().equals(seg2.getMergeId()) &&
|
||||||
|
Objects.equals(seg1.segmentSort, seg2.segmentSort);
|
||||||
|
}
|
||||||
|
}
|
|
@ -287,4 +287,26 @@ public class MapperServiceTests extends ESSingleNodeTestCase {
|
||||||
.put("index.routing_partition_size", 2))
|
.put("index.routing_partition_size", 2))
|
||||||
.execute().actionGet().isAcknowledged());
|
.execute().actionGet().isAcknowledged());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void testIndexSortWithNestedFields() throws IOException {
|
||||||
|
Settings settings = Settings.builder()
|
||||||
|
.put("index.sort.field", "_type")
|
||||||
|
.build();
|
||||||
|
IllegalArgumentException invalidNestedException = expectThrows(IllegalArgumentException.class,
|
||||||
|
() -> createIndex("test", settings, "t", "nested_field", "type=nested"));
|
||||||
|
assertThat(invalidNestedException.getMessage(),
|
||||||
|
containsString("cannot have nested fields when index sort is activated"));
|
||||||
|
IndexService indexService = createIndex("test", settings, "t");
|
||||||
|
CompressedXContent nestedFieldMapping = new CompressedXContent(XContentFactory.jsonBuilder().startObject()
|
||||||
|
.startObject("properties")
|
||||||
|
.startObject("nested_field")
|
||||||
|
.field("type", "nested")
|
||||||
|
.endObject()
|
||||||
|
.endObject().endObject().bytes());
|
||||||
|
invalidNestedException = expectThrows(IllegalArgumentException.class,
|
||||||
|
() -> indexService.mapperService().merge("t", nestedFieldMapping,
|
||||||
|
MergeReason.MAPPING_UPDATE, true));
|
||||||
|
assertThat(invalidNestedException.getMessage(),
|
||||||
|
containsString("cannot have nested fields when index sort is activated"));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -531,7 +531,7 @@ public class IndexShardIT extends ESSingleNodeTestCase {
|
||||||
IndexingOperationListener... listeners) throws IOException {
|
IndexingOperationListener... listeners) throws IOException {
|
||||||
ShardRouting initializingShardRouting = getInitializingShardRouting(shard.routingEntry());
|
ShardRouting initializingShardRouting = getInitializingShardRouting(shard.routingEntry());
|
||||||
IndexShard newShard = new IndexShard(initializingShardRouting, indexService.getIndexSettings(), shard.shardPath(),
|
IndexShard newShard = new IndexShard(initializingShardRouting, indexService.getIndexSettings(), shard.shardPath(),
|
||||||
shard.store(), indexService.cache(), indexService.mapperService(), indexService.similarityService(),
|
shard.store(), indexService.getIndexSortSupplier(), indexService.cache(), indexService.mapperService(), indexService.similarityService(),
|
||||||
indexService.fieldData(), shard.getEngineFactory(), indexService.getIndexEventListener(), wrapper,
|
indexService.fieldData(), shard.getEngineFactory(), indexService.getIndexEventListener(), wrapper,
|
||||||
indexService.getThreadPool(), indexService.getBigArrays(), null, () -> {}, Collections.emptyList(), Arrays.asList(listeners));
|
indexService.getThreadPool(), indexService.getBigArrays(), null, () -> {}, Collections.emptyList(), Arrays.asList(listeners));
|
||||||
return newShard;
|
return newShard;
|
||||||
|
|
|
@ -123,7 +123,7 @@ public class RefreshListenersTests extends ESTestCase {
|
||||||
store, new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy()), newMergePolicy(), iwc.getAnalyzer(),
|
store, new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy()), newMergePolicy(), iwc.getAnalyzer(),
|
||||||
iwc.getSimilarity(), new CodecService(null, logger), eventListener, translogHandler,
|
iwc.getSimilarity(), new CodecService(null, logger), eventListener, translogHandler,
|
||||||
IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig,
|
IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig,
|
||||||
TimeValue.timeValueMinutes(5), listeners, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP);
|
TimeValue.timeValueMinutes(5), listeners, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, null);
|
||||||
engine = new InternalEngine(config);
|
engine = new InternalEngine(config);
|
||||||
listeners.setTranslog(engine.getTranslog());
|
listeners.setTranslog(engine.getTranslog());
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,6 +20,8 @@ package org.elasticsearch.index.shard;
|
||||||
|
|
||||||
import org.apache.lucene.codecs.CodecUtil;
|
import org.apache.lucene.codecs.CodecUtil;
|
||||||
import org.apache.lucene.document.Field;
|
import org.apache.lucene.document.Field;
|
||||||
|
import org.apache.lucene.document.SortedNumericDocValuesField;
|
||||||
|
import org.apache.lucene.document.SortedSetDocValuesField;
|
||||||
import org.apache.lucene.document.StringField;
|
import org.apache.lucene.document.StringField;
|
||||||
import org.apache.lucene.index.DirectoryReader;
|
import org.apache.lucene.index.DirectoryReader;
|
||||||
import org.apache.lucene.index.IndexWriter;
|
import org.apache.lucene.index.IndexWriter;
|
||||||
|
@ -27,6 +29,10 @@ import org.apache.lucene.index.IndexWriterConfig;
|
||||||
import org.apache.lucene.index.NoMergePolicy;
|
import org.apache.lucene.index.NoMergePolicy;
|
||||||
import org.apache.lucene.index.SegmentCommitInfo;
|
import org.apache.lucene.index.SegmentCommitInfo;
|
||||||
import org.apache.lucene.index.SegmentInfos;
|
import org.apache.lucene.index.SegmentInfos;
|
||||||
|
import org.apache.lucene.search.Sort;
|
||||||
|
import org.apache.lucene.search.SortField;
|
||||||
|
import org.apache.lucene.search.SortedNumericSortField;
|
||||||
|
import org.apache.lucene.search.SortedSetSortField;
|
||||||
import org.apache.lucene.store.Directory;
|
import org.apache.lucene.store.Directory;
|
||||||
import org.apache.lucene.store.IOContext;
|
import org.apache.lucene.store.IOContext;
|
||||||
import org.apache.lucene.store.IndexOutput;
|
import org.apache.lucene.store.IndexOutput;
|
||||||
|
@ -47,13 +53,27 @@ public class StoreRecoveryTests extends ESTestCase {
|
||||||
public void testAddIndices() throws IOException {
|
public void testAddIndices() throws IOException {
|
||||||
Directory[] dirs = new Directory[randomIntBetween(1, 10)];
|
Directory[] dirs = new Directory[randomIntBetween(1, 10)];
|
||||||
final int numDocs = randomIntBetween(50, 100);
|
final int numDocs = randomIntBetween(50, 100);
|
||||||
|
final Sort indexSort;
|
||||||
|
if (randomBoolean()) {
|
||||||
|
indexSort = new Sort(new SortedNumericSortField("num", SortField.Type.LONG, true));
|
||||||
|
} else {
|
||||||
|
indexSort = null;
|
||||||
|
}
|
||||||
int id = 0;
|
int id = 0;
|
||||||
for (int i = 0; i < dirs.length; i++) {
|
for (int i = 0; i < dirs.length; i++) {
|
||||||
dirs[i] = newFSDirectory(createTempDir());
|
dirs[i] = newFSDirectory(createTempDir());
|
||||||
IndexWriter writer = new IndexWriter(dirs[i], newIndexWriterConfig().setMergePolicy(NoMergePolicy.INSTANCE)
|
IndexWriterConfig iwc = newIndexWriterConfig()
|
||||||
.setOpenMode(IndexWriterConfig.OpenMode.CREATE));
|
.setMergePolicy(NoMergePolicy.INSTANCE)
|
||||||
|
.setOpenMode(IndexWriterConfig.OpenMode.CREATE);
|
||||||
|
if (indexSort != null) {
|
||||||
|
iwc.setIndexSort(indexSort);
|
||||||
|
}
|
||||||
|
IndexWriter writer = new IndexWriter(dirs[i], iwc);
|
||||||
for (int j = 0; j < numDocs; j++) {
|
for (int j = 0; j < numDocs; j++) {
|
||||||
writer.addDocument(Arrays.asList(new StringField("id", Integer.toString(id++), Field.Store.YES)));
|
writer.addDocument(Arrays.asList(
|
||||||
|
new StringField("id", Integer.toString(id++), Field.Store.YES),
|
||||||
|
new SortedNumericDocValuesField("num", randomLong())
|
||||||
|
));
|
||||||
}
|
}
|
||||||
|
|
||||||
writer.commit();
|
writer.commit();
|
||||||
|
@ -62,7 +82,7 @@ public class StoreRecoveryTests extends ESTestCase {
|
||||||
StoreRecovery storeRecovery = new StoreRecovery(new ShardId("foo", "bar", 1), logger);
|
StoreRecovery storeRecovery = new StoreRecovery(new ShardId("foo", "bar", 1), logger);
|
||||||
RecoveryState.Index indexStats = new RecoveryState.Index();
|
RecoveryState.Index indexStats = new RecoveryState.Index();
|
||||||
Directory target = newFSDirectory(createTempDir());
|
Directory target = newFSDirectory(createTempDir());
|
||||||
storeRecovery.addIndices(indexStats, target, dirs);
|
storeRecovery.addIndices(indexStats, target, indexSort, dirs);
|
||||||
int numFiles = 0;
|
int numFiles = 0;
|
||||||
Predicate<String> filesFilter = (f) -> f.startsWith("segments") == false && f.equals("write.lock") == false
|
Predicate<String> filesFilter = (f) -> f.startsWith("segments") == false && f.equals("write.lock") == false
|
||||||
&& f.startsWith("extra") == false;
|
&& f.startsWith("extra") == false;
|
||||||
|
@ -80,7 +100,11 @@ public class StoreRecoveryTests extends ESTestCase {
|
||||||
DirectoryReader reader = DirectoryReader.open(target);
|
DirectoryReader reader = DirectoryReader.open(target);
|
||||||
SegmentInfos segmentCommitInfos = SegmentInfos.readLatestCommit(target);
|
SegmentInfos segmentCommitInfos = SegmentInfos.readLatestCommit(target);
|
||||||
for (SegmentCommitInfo info : segmentCommitInfos) { // check that we didn't merge
|
for (SegmentCommitInfo info : segmentCommitInfos) { // check that we didn't merge
|
||||||
assertEquals("all sources must be flush", info.info.getDiagnostics().get("source"), "flush");
|
assertEquals("all sources must be flush",
|
||||||
|
info.info.getDiagnostics().get("source"), "flush");
|
||||||
|
if (indexSort != null) {
|
||||||
|
assertEquals(indexSort, info.info.getIndexSort());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
assertEquals(reader.numDeletedDocs(), 0);
|
assertEquals(reader.numDeletedDocs(), 0);
|
||||||
assertEquals(reader.numDocs(), id);
|
assertEquals(reader.numDocs(), id);
|
||||||
|
|
|
@ -151,6 +151,7 @@ public class ClusterStateChanges extends AbstractComponent {
|
||||||
when(indexService.mapperService()).thenReturn(mapperService);
|
when(indexService.mapperService()).thenReturn(mapperService);
|
||||||
when(mapperService.docMappers(anyBoolean())).thenReturn(Collections.emptyList());
|
when(mapperService.docMappers(anyBoolean())).thenReturn(Collections.emptyList());
|
||||||
when(indexService.getIndexEventListener()).thenReturn(new IndexEventListener() {});
|
when(indexService.getIndexEventListener()).thenReturn(new IndexEventListener() {});
|
||||||
|
when(indexService.getIndexSortSupplier()).thenReturn(() -> null);
|
||||||
return indexService;
|
return indexService;
|
||||||
});
|
});
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
|
|
|
@ -0,0 +1,107 @@
|
||||||
|
[[index-modules-index-sorting]]
|
||||||
|
== Index Sorting
|
||||||
|
|
||||||
|
experimental[]
|
||||||
|
|
||||||
|
When creating a new index in elasticsearch it is possible to configure how the Segments
|
||||||
|
inside each Shard will be sorted. By default Lucene does not apply any sort.
|
||||||
|
The `index.sort.*` settings define which fields should be used to sort the documents inside each Segment.
|
||||||
|
|
||||||
|
[WARNING]
|
||||||
|
nested fields are not compatible with index sorting because they rely on the assumption
|
||||||
|
that nested documents are stored in contiguous doc ids, which can be broken by index sorting.
|
||||||
|
An error will be thrown if index sorting is activated on an index that contains nested fields.
|
||||||
|
|
||||||
|
For instance the following example shows how to define a sort on a single field:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
PUT twitter
|
||||||
|
{
|
||||||
|
"settings" : {
|
||||||
|
"index" : {
|
||||||
|
"sort.field" : "date", <1>
|
||||||
|
"sort.order" : "desc" <2>
|
||||||
|
}
|
||||||
|
},
|
||||||
|
"mappings": {
|
||||||
|
"tweet": {
|
||||||
|
"properties": {
|
||||||
|
"date": {
|
||||||
|
"type": "date"
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
// CONSOLE
|
||||||
|
|
||||||
|
<1> This index is sorted by the `date` field
|
||||||
|
<2> ... in descending order.
|
||||||
|
|
||||||
|
It is also possible to sort the index by more than one field:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
PUT twitter
|
||||||
|
{
|
||||||
|
"settings" : {
|
||||||
|
"index" : {
|
||||||
|
"sort.field" : ["username", "date"], <1>
|
||||||
|
"sort.order" : ["asc", "desc"] <2>
|
||||||
|
}
|
||||||
|
},
|
||||||
|
"mappings": {
|
||||||
|
"tweet": {
|
||||||
|
"properties": {
|
||||||
|
"username": {
|
||||||
|
"type": "keyword",
|
||||||
|
"doc_values": true
|
||||||
|
},
|
||||||
|
"date": {
|
||||||
|
"type": "date"
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
// CONSOLE
|
||||||
|
|
||||||
|
<1> This index is sorted by `username` first then by `date`
|
||||||
|
<2> ... in ascending order for the `username` field and in descending order for the `date` field.
|
||||||
|
|
||||||
|
|
||||||
|
Index sorting supports the following settings:
|
||||||
|
|
||||||
|
`index.sort.field`::
|
||||||
|
|
||||||
|
The list of fields used to sort the index.
|
||||||
|
Only `boolean`, `numeric`, `date` and `keyword` fields with `doc_values` are allowed here.
|
||||||
|
|
||||||
|
`index.sort.order`::
|
||||||
|
|
||||||
|
The sort order to use for each field.
|
||||||
|
The order option can have the following values:
|
||||||
|
* `asc`: For ascending order
|
||||||
|
* `desc`: For descending order.
|
||||||
|
|
||||||
|
`index.sort.mode`::
|
||||||
|
|
||||||
|
Elasticsearch supports sorting by multi-valued fields.
|
||||||
|
The mode option controls what value is picked to sort the document.
|
||||||
|
The mode option can have the following values:
|
||||||
|
* `min`: Pick the lowest value.
|
||||||
|
* `max`: Pick the highest value.
|
||||||
|
|
||||||
|
`index.sort.missing`::
|
||||||
|
|
||||||
|
The missing parameter specifies how docs which are missing the field should be treated.
|
||||||
|
The missing value can have the following values:
|
||||||
|
* `_last`: Documents without value for the field are sorted last.
|
||||||
|
* `_first`: Documents without value for the field are sorted first.
|
||||||
|
|
||||||
|
[WARNING]
|
||||||
|
Index sorting can be defined only once at index creation. It is not allowed to add or update
|
||||||
|
a sort on an existing index.
|
|
@ -0,0 +1,75 @@
|
||||||
|
---
|
||||||
|
"Index Sort":
|
||||||
|
|
||||||
|
- skip:
|
||||||
|
version: " - 5.99.99"
|
||||||
|
reason: this uses a new feature that has been added in 6.0.0
|
||||||
|
|
||||||
|
- do:
|
||||||
|
indices.create:
|
||||||
|
index: test
|
||||||
|
body:
|
||||||
|
settings:
|
||||||
|
number_of_shards: 1
|
||||||
|
number_of_replicas: 1
|
||||||
|
index.sort.field: rank
|
||||||
|
mappings:
|
||||||
|
t:
|
||||||
|
properties:
|
||||||
|
rank:
|
||||||
|
type: integer
|
||||||
|
|
||||||
|
- do:
|
||||||
|
index:
|
||||||
|
index: test
|
||||||
|
type: test
|
||||||
|
id: "1"
|
||||||
|
body: { "rank": 4 }
|
||||||
|
|
||||||
|
- do:
|
||||||
|
index:
|
||||||
|
index: test
|
||||||
|
type: test
|
||||||
|
id: "2"
|
||||||
|
body: { "rank": 1 }
|
||||||
|
|
||||||
|
- do:
|
||||||
|
index:
|
||||||
|
index: test
|
||||||
|
type: test
|
||||||
|
id: "3"
|
||||||
|
body: { "rank": 3 }
|
||||||
|
|
||||||
|
- do:
|
||||||
|
index:
|
||||||
|
index: test
|
||||||
|
type: test
|
||||||
|
id: "4"
|
||||||
|
body: { "rank": 2 }
|
||||||
|
|
||||||
|
- do:
|
||||||
|
indices.refresh:
|
||||||
|
index: test
|
||||||
|
|
||||||
|
- do:
|
||||||
|
indices.forcemerge:
|
||||||
|
index: test
|
||||||
|
max_num_segments: 1
|
||||||
|
|
||||||
|
- do:
|
||||||
|
indices.refresh:
|
||||||
|
index: test
|
||||||
|
|
||||||
|
- do:
|
||||||
|
search:
|
||||||
|
index: test
|
||||||
|
type: test
|
||||||
|
body:
|
||||||
|
sort: _doc
|
||||||
|
|
||||||
|
- match: {hits.total: 4 }
|
||||||
|
- length: {hits.hits: 4 }
|
||||||
|
- match: {hits.hits.0._id: "2" }
|
||||||
|
- match: {hits.hits.1._id: "4" }
|
||||||
|
- match: {hits.hits.2._id: "3" }
|
||||||
|
- match: {hits.hits.3._id: "1" }
|
|
@ -279,7 +279,7 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
||||||
});
|
});
|
||||||
IndexFieldDataService indexFieldDataService = new IndexFieldDataService(indexSettings, indicesFieldDataCache,
|
IndexFieldDataService indexFieldDataService = new IndexFieldDataService(indexSettings, indicesFieldDataCache,
|
||||||
new NoneCircuitBreakerService(), mapperService);
|
new NoneCircuitBreakerService(), mapperService);
|
||||||
indexShard = new IndexShard(routing, indexSettings, shardPath, store, indexCache, mapperService, similarityService,
|
indexShard = new IndexShard(routing, indexSettings, shardPath, store, () ->null, indexCache, mapperService, similarityService,
|
||||||
indexFieldDataService, engineFactory, indexEventListener, indexSearcherWrapper, threadPool,
|
indexFieldDataService, engineFactory, indexEventListener, indexSearcherWrapper, threadPool,
|
||||||
BigArrays.NON_RECYCLING_INSTANCE, warmer, globalCheckpointSyncer, Collections.emptyList(), Arrays.asList(listeners));
|
BigArrays.NON_RECYCLING_INSTANCE, warmer, globalCheckpointSyncer, Collections.emptyList(), Arrays.asList(listeners));
|
||||||
success = true;
|
success = true;
|
||||||
|
|
|
@ -24,6 +24,7 @@ import com.carrotsearch.randomizedtesting.annotations.TestGroup;
|
||||||
import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
|
import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
|
||||||
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
|
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
|
||||||
import org.apache.http.HttpHost;
|
import org.apache.http.HttpHost;
|
||||||
|
import org.apache.lucene.search.Sort;
|
||||||
import org.apache.lucene.util.IOUtils;
|
import org.apache.lucene.util.IOUtils;
|
||||||
import org.apache.lucene.util.LuceneTestCase;
|
import org.apache.lucene.util.LuceneTestCase;
|
||||||
import org.apache.lucene.util.TestUtil;
|
import org.apache.lucene.util.TestUtil;
|
||||||
|
@ -45,7 +46,10 @@ import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse;
|
||||||
import org.elasticsearch.action.admin.indices.get.GetIndexResponse;
|
import org.elasticsearch.action.admin.indices.get.GetIndexResponse;
|
||||||
import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse;
|
import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse;
|
||||||
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
|
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
|
||||||
|
import org.elasticsearch.action.admin.indices.segments.IndexSegments;
|
||||||
|
import org.elasticsearch.action.admin.indices.segments.IndexShardSegments;
|
||||||
import org.elasticsearch.action.admin.indices.segments.IndicesSegmentResponse;
|
import org.elasticsearch.action.admin.indices.segments.IndicesSegmentResponse;
|
||||||
|
import org.elasticsearch.action.admin.indices.segments.ShardSegments;
|
||||||
import org.elasticsearch.action.admin.indices.template.put.PutIndexTemplateRequestBuilder;
|
import org.elasticsearch.action.admin.indices.template.put.PutIndexTemplateRequestBuilder;
|
||||||
import org.elasticsearch.action.bulk.BulkRequestBuilder;
|
import org.elasticsearch.action.bulk.BulkRequestBuilder;
|
||||||
import org.elasticsearch.action.bulk.BulkResponse;
|
import org.elasticsearch.action.bulk.BulkResponse;
|
||||||
|
@ -111,6 +115,7 @@ import org.elasticsearch.index.MergePolicyConfig;
|
||||||
import org.elasticsearch.index.MergeSchedulerConfig;
|
import org.elasticsearch.index.MergeSchedulerConfig;
|
||||||
import org.elasticsearch.index.MockEngineFactoryPlugin;
|
import org.elasticsearch.index.MockEngineFactoryPlugin;
|
||||||
import org.elasticsearch.index.codec.CodecService;
|
import org.elasticsearch.index.codec.CodecService;
|
||||||
|
import org.elasticsearch.index.engine.Segment;
|
||||||
import org.elasticsearch.index.mapper.DocumentMapper;
|
import org.elasticsearch.index.mapper.DocumentMapper;
|
||||||
import org.elasticsearch.index.translog.Translog;
|
import org.elasticsearch.index.translog.Translog;
|
||||||
import org.elasticsearch.indices.IndicesQueryCache;
|
import org.elasticsearch.indices.IndicesQueryCache;
|
||||||
|
@ -1996,6 +2001,23 @@ public abstract class ESIntegTestCase extends ESTestCase {
|
||||||
return nodes;
|
return nodes;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Asserts that all segments are sorted with the provided {@link Sort}.
|
||||||
|
*/
|
||||||
|
public void assertSortedSegments(String indexName, Sort expectedIndexSort) {
|
||||||
|
IndicesSegmentResponse segmentResponse =
|
||||||
|
client().admin().indices().prepareSegments(indexName).execute().actionGet();
|
||||||
|
IndexSegments indexSegments = segmentResponse.getIndices().get(indexName);
|
||||||
|
for (IndexShardSegments indexShardSegments : indexSegments.getShards().values()) {
|
||||||
|
for (ShardSegments shardSegments : indexShardSegments.getShards()) {
|
||||||
|
for (Segment segment : shardSegments) {
|
||||||
|
assertThat(expectedIndexSort, equalTo(segment.getSegmentSort()));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
protected static class NumShards {
|
protected static class NumShards {
|
||||||
public final int numPrimaries;
|
public final int numPrimaries;
|
||||||
public final int numReplicas;
|
public final int numReplicas;
|
||||||
|
|
Loading…
Reference in New Issue