Merge remote-tracking branch 'es/master' into feature/ingest

This commit is contained in:
Martijn van Groningen 2016-01-21 11:46:27 +01:00
commit 44465c94f8
113 changed files with 1278 additions and 745 deletions

View File

@ -372,6 +372,7 @@ class BuildPlugin implements Plugin<Project> {
systemProperty 'tests.artifact', project.name
systemProperty 'tests.task', path
systemProperty 'tests.security.manager', 'true'
systemProperty 'jna.nosys', 'true'
// default test sysprop values
systemProperty 'tests.ifNoTests', 'fail'
systemProperty 'es.logger.level', 'WARN'

View File

@ -41,6 +41,7 @@ public class StandaloneTestPlugin implements Plugin<Project> {
]
RandomizedTestingTask test = project.tasks.create(testOptions)
test.configure(BuildPlugin.commonTestConfig(project))
BuildPlugin.configureCompile(project)
test.classpath = project.sourceSets.test.runtimeClasspath
test.testClassesDir project.sourceSets.test.output.classesDir
test.mustRunAfter(project.precommit)

View File

@ -1,5 +1,5 @@
elasticsearch = 3.0.0-SNAPSHOT
lucene = 5.5.0-snapshot-1721183
lucene = 5.5.0-snapshot-1725675
# optional dependencies
spatial4j = 0.5

View File

@ -40,13 +40,15 @@ public interface ActionFilter {
* Enables filtering the execution of an action on the request side, either by sending a response through the
* {@link ActionListener} or by continuing the execution through the given {@link ActionFilterChain chain}
*/
void apply(Task task, String action, ActionRequest<?> request, ActionListener<?> listener, ActionFilterChain chain);
<Request extends ActionRequest<Request>, Response extends ActionResponse> void apply(Task task, String action, Request request,
ActionListener<Response> listener, ActionFilterChain<Request, Response> chain);
/**
* Enables filtering the execution of an action on the response side, either by sending a response through the
* {@link ActionListener} or by continuing the execution through the given {@link ActionFilterChain chain}
*/
void apply(String action, ActionResponse response, ActionListener<?> listener, ActionFilterChain chain);
<Response extends ActionResponse> void apply(String action, Response response, ActionListener<Response> listener,
ActionFilterChain<?, Response> chain);
/**
* A simple base class for injectable action filters that spares the implementation from handling the
@ -60,7 +62,8 @@ public interface ActionFilter {
}
@Override
public final void apply(Task task, String action, ActionRequest<?> request, ActionListener<?> listener, ActionFilterChain chain) {
public final <Request extends ActionRequest<Request>, Response extends ActionResponse> void apply(Task task, String action, Request request,
ActionListener<Response> listener, ActionFilterChain<Request, Response> chain) {
if (apply(action, request, listener)) {
chain.proceed(task, action, request, listener);
}
@ -73,7 +76,8 @@ public interface ActionFilter {
protected abstract boolean apply(String action, ActionRequest<?> request, ActionListener<?> listener);
@Override
public final void apply(String action, ActionResponse response, ActionListener<?> listener, ActionFilterChain chain) {
public final <Response extends ActionResponse> void apply(String action, Response response, ActionListener<Response> listener,
ActionFilterChain<?, Response> chain) {
if (apply(action, response, listener)) {
chain.proceed(action, response, listener);
}

View File

@ -27,17 +27,17 @@ import org.elasticsearch.tasks.Task;
/**
* A filter chain allowing to continue and process the transport action request
*/
public interface ActionFilterChain {
public interface ActionFilterChain<Request extends ActionRequest<Request>, Response extends ActionResponse> {
/**
* Continue processing the request. Should only be called if a response has not been sent through
* the given {@link ActionListener listener}
*/
void proceed(Task task, final String action, final ActionRequest request, final ActionListener listener);
void proceed(Task task, final String action, final Request request, final ActionListener<Response> listener);
/**
* Continue processing the response. Should only be called if a response has not been sent through
* the given {@link ActionListener listener}
*/
void proceed(final String action, final ActionResponse response, final ActionListener listener);
void proceed(final String action, final Response response, final ActionListener<Response> listener);
}

View File

@ -104,7 +104,7 @@ public abstract class TransportAction<Request extends ActionRequest<Request>, Re
listener.onFailure(t);
}
} else {
RequestFilterChain requestFilterChain = new RequestFilterChain<>(this, logger);
RequestFilterChain<Request, Response> requestFilterChain = new RequestFilterChain<>(this, logger);
requestFilterChain.proceed(task, actionName, request, listener);
}
}
@ -115,7 +115,8 @@ public abstract class TransportAction<Request extends ActionRequest<Request>, Re
protected abstract void doExecute(Request request, ActionListener<Response> listener);
private static class RequestFilterChain<Request extends ActionRequest<Request>, Response extends ActionResponse> implements ActionFilterChain {
private static class RequestFilterChain<Request extends ActionRequest<Request>, Response extends ActionResponse>
implements ActionFilterChain<Request, Response> {
private final TransportAction<Request, Response> action;
private final AtomicInteger index = new AtomicInteger();
@ -126,14 +127,15 @@ public abstract class TransportAction<Request extends ActionRequest<Request>, Re
this.logger = logger;
}
@Override @SuppressWarnings("unchecked")
public void proceed(Task task, String actionName, ActionRequest request, ActionListener listener) {
@Override
public void proceed(Task task, String actionName, Request request, ActionListener<Response> listener) {
int i = index.getAndIncrement();
try {
if (i < this.action.filters.length) {
this.action.filters[i].apply(task, actionName, request, listener, this);
} else if (i == this.action.filters.length) {
this.action.doExecute(task, (Request) request, new FilteredActionListener<Response>(actionName, listener, new ResponseFilterChain(this.action.filters, logger)));
this.action.doExecute(task, request, new FilteredActionListener<Response>(actionName, listener,
new ResponseFilterChain<>(this.action.filters, logger)));
} else {
listener.onFailure(new IllegalStateException("proceed was called too many times"));
}
@ -144,12 +146,13 @@ public abstract class TransportAction<Request extends ActionRequest<Request>, Re
}
@Override
public void proceed(String action, ActionResponse response, ActionListener listener) {
public void proceed(String action, Response response, ActionListener<Response> listener) {
assert false : "request filter chain should never be called on the response side";
}
}
private static class ResponseFilterChain implements ActionFilterChain {
private static class ResponseFilterChain<Request extends ActionRequest<Request>, Response extends ActionResponse>
implements ActionFilterChain<Request, Response> {
private final ActionFilter[] filters;
private final AtomicInteger index;
@ -162,12 +165,12 @@ public abstract class TransportAction<Request extends ActionRequest<Request>, Re
}
@Override
public void proceed(Task task, String action, ActionRequest request, ActionListener listener) {
public void proceed(Task task, String action, Request request, ActionListener<Response> listener) {
assert false : "response filter chain should never be called on the request side";
}
@Override @SuppressWarnings("unchecked")
public void proceed(String action, ActionResponse response, ActionListener listener) {
@Override
public void proceed(String action, Response response, ActionListener<Response> listener) {
int i = index.decrementAndGet();
try {
if (i >= 0) {
@ -187,10 +190,10 @@ public abstract class TransportAction<Request extends ActionRequest<Request>, Re
private static class FilteredActionListener<Response extends ActionResponse> implements ActionListener<Response> {
private final String actionName;
private final ActionListener listener;
private final ResponseFilterChain chain;
private final ActionListener<Response> listener;
private final ResponseFilterChain<?, Response> chain;
private FilteredActionListener(String actionName, ActionListener listener, ResponseFilterChain chain) {
private FilteredActionListener(String actionName, ActionListener<Response> listener, ResponseFilterChain<?, Response> chain) {
this.actionName = actionName;
this.listener = listener;
this.chain = chain;

View File

@ -30,9 +30,8 @@ final class AutoExpandReplicas {
// the value we recognize in the "max" position to mean all the nodes
private static final String ALL_NODES_VALUE = "all";
public static final Setting<AutoExpandReplicas> SETTING = new Setting<>(IndexMetaData.SETTING_AUTO_EXPAND_REPLICAS, "false", (value) -> {
// TODO change the following back to be final, https://github.com/elastic/elasticsearch/issues/16097
int min;
int max;
final int min;
final int max;
if (Booleans.parseBoolean(value, true) == false) {
return new AutoExpandReplicas(0, 0, false);
}

View File

@ -31,8 +31,10 @@ import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.function.BiConsumer;
import java.util.function.Consumer;
import java.util.regex.Pattern;
/**
* A basic setting service that can be used for per-index and per-cluster settings.
@ -40,38 +42,54 @@ import java.util.function.Consumer;
*/
public abstract class AbstractScopedSettings extends AbstractComponent {
private Settings lastSettingsApplied = Settings.EMPTY;
private final List<SettingUpdater<?>> settingUpdaters = new ArrayList<>();
private final Map<String, Setting<?>> complexMatchers = new HashMap<>();
private final Map<String, Setting<?>> keySettings = new HashMap<>();
private final List<SettingUpdater<?>> settingUpdaters = new CopyOnWriteArrayList<>();
private final Map<String, Setting<?>> complexMatchers;
private final Map<String, Setting<?>> keySettings;
private final Setting.Scope scope;
private static final Pattern KEY_PATTERN = Pattern.compile("^(?:[-\\w]+[.])*[-\\w]+$");
private static final Pattern GROUP_KEY_PATTERN = Pattern.compile("^(?:[-\\w]+[.])+$");
protected AbstractScopedSettings(Settings settings, Set<Setting<?>> settingsSet, Setting.Scope scope) {
super(settings);
this.lastSettingsApplied = Settings.EMPTY;
this.scope = scope;
for (Setting<?> entry : settingsSet) {
addSetting(entry);
Map<String, Setting<?>> complexMatchers = new HashMap<>();
Map<String, Setting<?>> keySettings = new HashMap<>();
for (Setting<?> setting : settingsSet) {
if (setting.getScope() != scope) {
throw new IllegalArgumentException("Setting must be a " + scope + " setting but was: " + setting.getScope());
}
if (isValidKey(setting.getKey()) == false && (setting.isGroupSetting() && isValidGroupKey(setting.getKey())) == false) {
throw new IllegalArgumentException("illegal settings key: [" + setting.getKey() + "]");
}
if (setting.hasComplexMatcher()) {
complexMatchers.putIfAbsent(setting.getKey(), setting);
} else {
keySettings.putIfAbsent(setting.getKey(), setting);
}
}
this.complexMatchers = Collections.unmodifiableMap(complexMatchers);
this.keySettings = Collections.unmodifiableMap(keySettings);
}
protected AbstractScopedSettings(Settings nodeSettings, Settings scopeSettings, AbstractScopedSettings other) {
super(nodeSettings);
this.lastSettingsApplied = scopeSettings;
this.scope = other.scope;
complexMatchers.putAll(other.complexMatchers);
keySettings.putAll(other.keySettings);
complexMatchers = other.complexMatchers;
keySettings = other.keySettings;
settingUpdaters.addAll(other.settingUpdaters);
}
protected final void addSetting(Setting<?> setting) {
if (setting.getScope() != scope) {
throw new IllegalArgumentException("Setting must be a " + scope + " setting but was: " + setting.getScope());
}
if (setting.hasComplexMatcher()) {
complexMatchers.putIfAbsent(setting.getKey(), setting);
} else {
keySettings.putIfAbsent(setting.getKey(), setting);
}
/**
* Returns <code>true</code> iff the given key is a valid settings key otherwise <code>false</code>
*/
public static boolean isValidKey(String key) {
return KEY_PATTERN.matcher(key).matches();
}
private static boolean isValidGroupKey(String key) {
return GROUP_KEY_PATTERN.matcher(key).matches();
}
public Setting.Scope getScope() {

View File

@ -61,6 +61,8 @@ public class SettingsModule extends AbstractModule {
for (Map.Entry<String, String> entry : settings.filter(IndexScopedSettings.INDEX_SETTINGS_KEY_PREDICATE.negate()).getAsMap().entrySet()) {
if (clusterSettings.get(entry.getKey()) != null) {
clusterSettings.validate(entry.getKey(), settings);
} else if (AbstractScopedSettings.isValidKey(entry.getKey()) == false) {
throw new IllegalArgumentException("illegal settings key: [" + entry.getKey() + "]");
}
}

View File

@ -38,6 +38,7 @@ import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -90,8 +91,8 @@ public abstract class PrimaryShardAllocator extends AbstractComponent {
}
final IndexMetaData indexMetaData = metaData.index(shard.getIndex());
final IndexSettings indexSettings = new IndexSettings(indexMetaData, settings);
// don't go wild here and create a new IndexSetting object for every shard this could cause a lot of garbage
// on cluster restart if we allocate a boat load of shards
if (shard.allocatedPostIndexCreate(indexMetaData) == false) {
// when we create a fresh index
continue;
@ -107,13 +108,13 @@ public abstract class PrimaryShardAllocator extends AbstractComponent {
final Set<String> lastActiveAllocationIds = indexMetaData.activeAllocationIds(shard.id());
final boolean snapshotRestore = shard.restoreSource() != null;
final boolean recoverOnAnyNode = recoverOnAnyNode(indexSettings);
final boolean recoverOnAnyNode = recoverOnAnyNode(indexMetaData);
final NodesAndVersions nodesAndVersions;
final boolean enoughAllocationsFound;
if (lastActiveAllocationIds.isEmpty()) {
assert indexSettings.getIndexVersionCreated().before(Version.V_3_0_0) : "trying to allocated a primary with an empty allocation id set, but index is new";
assert Version.indexCreated(indexMetaData.getSettings()).before(Version.V_3_0_0) : "trying to allocated a primary with an empty allocation id set, but index is new";
// when we load an old index (after upgrading cluster) or restore a snapshot of an old index
// fall back to old version-based allocation mode
// Note that once the shard has been active, lastActiveAllocationIds will be non-empty
@ -175,8 +176,8 @@ public abstract class PrimaryShardAllocator extends AbstractComponent {
*/
protected NodesAndVersions buildAllocationIdBasedNodes(ShardRouting shard, boolean matchAnyShard, Set<String> ignoreNodes,
Set<String> lastActiveAllocationIds, AsyncShardFetch.FetchResult<TransportNodesListGatewayStartedShards.NodeGatewayStartedShards> shardState) {
List<DiscoveryNode> matchingNodes = new ArrayList<>();
List<DiscoveryNode> nonMatchingNodes = new ArrayList<>();
LinkedList<DiscoveryNode> matchingNodes = new LinkedList<>();
LinkedList<DiscoveryNode> nonMatchingNodes = new LinkedList<>();
long highestVersion = -1;
for (TransportNodesListGatewayStartedShards.NodeGatewayStartedShards nodeShardState : shardState.getData().values()) {
DiscoveryNode node = nodeShardState.getNode();
@ -200,10 +201,18 @@ public abstract class PrimaryShardAllocator extends AbstractComponent {
if (allocationId != null) {
if (lastActiveAllocationIds.contains(allocationId)) {
matchingNodes.add(node);
if (nodeShardState.primary()) {
matchingNodes.addFirst(node);
} else {
matchingNodes.addLast(node);
}
highestVersion = Math.max(highestVersion, nodeShardState.version());
} else if (matchAnyShard) {
nonMatchingNodes.add(node);
if (nodeShardState.primary()) {
nonMatchingNodes.addFirst(node);
} else {
nonMatchingNodes.addLast(node);
}
highestVersion = Math.max(highestVersion, nodeShardState.version());
}
}
@ -347,9 +356,9 @@ public abstract class PrimaryShardAllocator extends AbstractComponent {
* Return {@code true} if the index is configured to allow shards to be
* recovered on any node
*/
private boolean recoverOnAnyNode(IndexSettings indexSettings) {
return indexSettings.isOnSharedFilesystem()
&& IndexMetaData.INDEX_SHARED_FS_ALLOW_RECOVERY_ON_ANY_NODE_SETTING.get(indexSettings.getSettings());
private boolean recoverOnAnyNode(IndexMetaData metaData) {
return (IndexMetaData.isOnSharedFilesystem(metaData.getSettings()) || IndexMetaData.isOnSharedFilesystem(this.settings))
&& IndexMetaData.INDEX_SHARED_FS_ALLOW_RECOVERY_ON_ANY_NODE_SETTING.get(metaData.getSettings(), this.settings);
}
protected abstract AsyncShardFetch.FetchResult<TransportNodesListGatewayStartedShards.NodeGatewayStartedShards> fetchData(ShardRouting shard, RoutingAllocation allocation);

View File

@ -139,7 +139,7 @@ public class TransportNodesListGatewayStartedShards extends TransportNodesAction
} catch (Exception exception) {
logger.trace("{} can't open index for shard [{}] in path [{}]", exception, shardId, shardStateMetaData, (shardPath != null) ? shardPath.resolveIndex() : "");
String allocationId = shardStateMetaData.allocationId != null ? shardStateMetaData.allocationId.getId() : null;
return new NodeGatewayStartedShards(clusterService.localNode(), shardStateMetaData.version, allocationId, exception);
return new NodeGatewayStartedShards(clusterService.localNode(), shardStateMetaData.version, allocationId, shardStateMetaData.primary, exception);
}
}
// old shard metadata doesn't have the actual index UUID so we need to check if the actual uuid in the metadata
@ -150,11 +150,11 @@ public class TransportNodesListGatewayStartedShards extends TransportNodesAction
} else {
logger.debug("{} shard state info found: [{}]", shardId, shardStateMetaData);
String allocationId = shardStateMetaData.allocationId != null ? shardStateMetaData.allocationId.getId() : null;
return new NodeGatewayStartedShards(clusterService.localNode(), shardStateMetaData.version, allocationId);
return new NodeGatewayStartedShards(clusterService.localNode(), shardStateMetaData.version, allocationId, shardStateMetaData.primary);
}
}
logger.trace("{} no local shard info found", shardId);
return new NodeGatewayStartedShards(clusterService.localNode(), -1, null);
return new NodeGatewayStartedShards(clusterService.localNode(), -1, null, false);
} catch (Exception e) {
throw new ElasticsearchException("failed to load started shards", e);
}
@ -279,18 +279,20 @@ public class TransportNodesListGatewayStartedShards extends TransportNodesAction
private long version = -1;
private String allocationId = null;
private boolean primary = false;
private Throwable storeException = null;
public NodeGatewayStartedShards() {
}
public NodeGatewayStartedShards(DiscoveryNode node, long version, String allocationId) {
this(node, version, allocationId, null);
public NodeGatewayStartedShards(DiscoveryNode node, long version, String allocationId, boolean primary) {
this(node, version, allocationId, primary, null);
}
public NodeGatewayStartedShards(DiscoveryNode node, long version, String allocationId, Throwable storeException) {
public NodeGatewayStartedShards(DiscoveryNode node, long version, String allocationId, boolean primary, Throwable storeException) {
super(node);
this.version = version;
this.allocationId = allocationId;
this.primary = primary;
this.storeException = storeException;
}
@ -302,6 +304,10 @@ public class TransportNodesListGatewayStartedShards extends TransportNodesAction
return this.allocationId;
}
public boolean primary() {
return this.primary;
}
public Throwable storeException() {
return this.storeException;
}
@ -311,6 +317,7 @@ public class TransportNodesListGatewayStartedShards extends TransportNodesAction
super.readFrom(in);
version = in.readLong();
allocationId = in.readOptionalString();
primary = in.readBoolean();
if (in.readBoolean()) {
storeException = in.readThrowable();
}
@ -321,6 +328,7 @@ public class TransportNodesListGatewayStartedShards extends TransportNodesAction
super.writeTo(out);
out.writeLong(version);
out.writeOptionalString(allocationId);
out.writeBoolean(primary);
if (storeException != null) {
out.writeBoolean(true);
out.writeThrowable(storeException);

View File

@ -258,8 +258,8 @@ public final class IndexModule {
throw new IllegalStateException("store must not be null");
}
}
indexSettings.getScopedSettings().addSettingsUpdateConsumer(IndexStore.INDEX_STORE_THROTTLE_MAX_BYTES_PER_SEC_SETTING, store::setMaxRate);
indexSettings.getScopedSettings().addSettingsUpdateConsumer(IndexStore.INDEX_STORE_THROTTLE_TYPE_SETTING, store::setType);
indexSettings.getScopedSettings().addSettingsUpdateConsumer(IndexStore.INDEX_STORE_THROTTLE_MAX_BYTES_PER_SEC_SETTING, store::setMaxRate);
final String queryCacheType = indexSettings.getValue(INDEX_QUERY_CACHE_TYPE_SETTING);
final BiFunction<IndexSettings, IndicesQueryCache, QueryCache> queryCacheProvider = queryCaches.get(queryCacheType);
final QueryCache queryCache = queryCacheProvider.apply(indexSettings, servicesProvider.getIndicesQueryCache());

View File

@ -635,7 +635,9 @@ public final class IndexService extends AbstractIndexComponent implements IndexC
case STARTED:
case RELOCATED:
try {
shard.refresh("schedule");
if (shard.isRefreshNeeded()) {
shard.refresh("schedule");
}
} catch (EngineClosedException | AlreadyClosedException ex) {
// fine - continue;
}

View File

@ -36,6 +36,7 @@ import org.elasticsearch.index.translog.Translog;
import java.util.Locale;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Consumer;
import java.util.function.Function;
import java.util.function.Predicate;
@ -167,10 +168,6 @@ public final class IndexSettings {
this(indexMetaData, nodeSettings, (index) -> Regex.simpleMatch(index, indexMetaData.getIndex()), IndexScopedSettings.DEFAULT_SCOPED_SETTINGS);
}
IndexSettings(final IndexMetaData indexMetaData, final Settings nodeSettings, IndexScopedSettings indexScopedSettings) {
this(indexMetaData, nodeSettings, (index) -> Regex.simpleMatch(index, indexMetaData.getIndex()), indexScopedSettings);
}
/**
* Creates a new {@link IndexSettings} instance. The given node settings will be merged with the settings in the metadata
* while index level settings will overwrite node settings.
@ -200,24 +197,35 @@ public final class IndexSettings {
this.defaultAllowUnmappedFields = scopedSettings.get(ALLOW_UNMAPPED);
this.indexNameMatcher = indexNameMatcher;
this.durability = scopedSettings.get(INDEX_TRANSLOG_DURABILITY_SETTING);
scopedSettings.addSettingsUpdateConsumer(INDEX_TRANSLOG_DURABILITY_SETTING, this::setTranslogDurability);
syncInterval = INDEX_TRANSLOG_SYNC_INTERVAL_SETTING.get(settings);
refreshInterval = scopedSettings.get(INDEX_REFRESH_INTERVAL_SETTING);
scopedSettings.addSettingsUpdateConsumer(INDEX_REFRESH_INTERVAL_SETTING, this::setRefreshInterval);
flushThresholdSize = scopedSettings.get(INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTTING);
scopedSettings.addSettingsUpdateConsumer(INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTTING, this::setTranslogFlushThresholdSize);
mergeSchedulerConfig = new MergeSchedulerConfig(this);
scopedSettings.addSettingsUpdateConsumer(INDEX_GC_DELETES_SETTING, this::setGCDeletes);
gcDeletesInMillis = scopedSettings.get(INDEX_GC_DELETES_SETTING).getMillis();
warmerEnabled = scopedSettings.get(INDEX_WARMER_ENABLED_SETTING);
scopedSettings.addSettingsUpdateConsumer(INDEX_WARMER_ENABLED_SETTING, this::setEnableWarmer);
maxResultWindow = scopedSettings.get(MAX_RESULT_WINDOW_SETTING);
scopedSettings.addSettingsUpdateConsumer(MAX_RESULT_WINDOW_SETTING, this::setMaxResultWindow);
TTLPurgeDisabled = scopedSettings.get(INDEX_TTL_DISABLE_PURGE_SETTING);
scopedSettings.addSettingsUpdateConsumer(INDEX_TTL_DISABLE_PURGE_SETTING, this::setTTLPurgeDisabled);
this.mergePolicyConfig = new MergePolicyConfig(logger, this);
assert indexNameMatcher.test(indexMetaData.getIndex());
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_FLOOR_SEGMENT_SETTING, mergePolicyConfig::setFloorSegmentSetting);
scopedSettings.addSettingsUpdateConsumer(MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGE_AT_ONCE_SETTING, mergePolicyConfig::setMaxMergesAtOnce);
scopedSettings.addSettingsUpdateConsumer(MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGE_AT_ONCE_EXPLICIT_SETTING, mergePolicyConfig::setMaxMergesAtOnceExplicit);
scopedSettings.addSettingsUpdateConsumer(MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGED_SEGMENT_SETTING, mergePolicyConfig::setMaxMergedSegment);
scopedSettings.addSettingsUpdateConsumer(MergePolicyConfig.INDEX_MERGE_POLICY_SEGMENTS_PER_TIER_SETTING, mergePolicyConfig::setSegmentsPerTier);
scopedSettings.addSettingsUpdateConsumer(MergePolicyConfig.INDEX_MERGE_POLICY_RECLAIM_DELETES_WEIGHT_SETTING, mergePolicyConfig::setReclaimDeletesWeight);
scopedSettings.addSettingsUpdateConsumer(MergeSchedulerConfig.MAX_THREAD_COUNT_SETTING, mergeSchedulerConfig::setMaxThreadCount);
scopedSettings.addSettingsUpdateConsumer(MergeSchedulerConfig.MAX_MERGE_COUNT_SETTING, mergeSchedulerConfig::setMaxMergeCount);
scopedSettings.addSettingsUpdateConsumer(MergeSchedulerConfig.AUTO_THROTTLE_SETTING, mergeSchedulerConfig::setAutoThrottle);
scopedSettings.addSettingsUpdateConsumer(INDEX_TRANSLOG_DURABILITY_SETTING, this::setTranslogDurability);
scopedSettings.addSettingsUpdateConsumer(INDEX_TTL_DISABLE_PURGE_SETTING, this::setTTLPurgeDisabled);
scopedSettings.addSettingsUpdateConsumer(MAX_RESULT_WINDOW_SETTING, this::setMaxResultWindow);
scopedSettings.addSettingsUpdateConsumer(INDEX_WARMER_ENABLED_SETTING, this::setEnableWarmer);
scopedSettings.addSettingsUpdateConsumer(INDEX_GC_DELETES_SETTING, this::setGCDeletes);
scopedSettings.addSettingsUpdateConsumer(INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTTING, this::setTranslogFlushThresholdSize);
scopedSettings.addSettingsUpdateConsumer(INDEX_REFRESH_INTERVAL_SETTING, this::setRefreshInterval);
}
private void setTranslogFlushThresholdSize(ByteSizeValue byteSizeValue) {
@ -457,5 +465,5 @@ public final class IndexSettings {
}
public IndexScopedSettings getScopedSettings() { return scopedSettings;}
IndexScopedSettings getScopedSettings() { return scopedSettings;}
}

View File

@ -23,6 +23,7 @@ import org.apache.lucene.index.MergePolicy;
import org.apache.lucene.index.NoMergePolicy;
import org.apache.lucene.index.TieredMergePolicy;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.settings.IndexScopedSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
@ -137,16 +138,9 @@ public final class MergePolicyConfig {
public static final String INDEX_MERGE_ENABLED = "index.merge.enabled"; // don't convert to Setting<> and register... we only set this in tests and register via a plugin
MergePolicyConfig(ESLogger logger, IndexSettings indexSettings) {
MergePolicyConfig(ESLogger logger, IndexSettings indexSettings) {
this.logger = logger;
indexSettings.getScopedSettings().addSettingsUpdateConsumer(INDEX_COMPOUND_FORMAT_SETTING, this::setNoCFSRatio);
indexSettings.getScopedSettings().addSettingsUpdateConsumer(INDEX_MERGE_POLICY_EXPUNGE_DELETES_ALLOWED_SETTING, this::expungeDeletesAllowed);
indexSettings.getScopedSettings().addSettingsUpdateConsumer(INDEX_MERGE_POLICY_FLOOR_SEGMENT_SETTING, this::floorSegmentSetting);
indexSettings.getScopedSettings().addSettingsUpdateConsumer(INDEX_MERGE_POLICY_MAX_MERGE_AT_ONCE_SETTING, this::maxMergesAtOnce);
indexSettings.getScopedSettings().addSettingsUpdateConsumer(INDEX_MERGE_POLICY_MAX_MERGE_AT_ONCE_EXPLICIT_SETTING, this::maxMergesAtOnceExplicit);
indexSettings.getScopedSettings().addSettingsUpdateConsumer(INDEX_MERGE_POLICY_MAX_MERGED_SEGMENT_SETTING, this::maxMergedSegment);
indexSettings.getScopedSettings().addSettingsUpdateConsumer(INDEX_MERGE_POLICY_SEGMENTS_PER_TIER_SETTING, this::segmentsPerTier);
indexSettings.getScopedSettings().addSettingsUpdateConsumer(INDEX_MERGE_POLICY_RECLAIM_DELETES_WEIGHT_SETTING, this::reclaimDeletesWeight);
IndexScopedSettings scopedSettings = indexSettings.getScopedSettings();
double forceMergeDeletesPctAllowed = indexSettings.getValue(INDEX_MERGE_POLICY_EXPUNGE_DELETES_ALLOWED_SETTING); // percentage
ByteSizeValue floorSegment = indexSettings.getValue(INDEX_MERGE_POLICY_FLOOR_SEGMENT_SETTING);
int maxMergeAtOnce = indexSettings.getValue(INDEX_MERGE_POLICY_MAX_MERGE_AT_ONCE_SETTING);
@ -168,39 +162,41 @@ public final class MergePolicyConfig {
mergePolicy.setMaxMergedSegmentMB(maxMergedSegment.mbFrac());
mergePolicy.setSegmentsPerTier(segmentsPerTier);
mergePolicy.setReclaimDeletesWeight(reclaimDeletesWeight);
logger.debug("using [tiered] merge mergePolicy with expunge_deletes_allowed[{}], floor_segment[{}], max_merge_at_once[{}], max_merge_at_once_explicit[{}], max_merged_segment[{}], segments_per_tier[{}], reclaim_deletes_weight[{}]",
if (logger.isTraceEnabled()) {
logger.trace("using [tiered] merge mergePolicy with expunge_deletes_allowed[{}], floor_segment[{}], max_merge_at_once[{}], max_merge_at_once_explicit[{}], max_merged_segment[{}], segments_per_tier[{}], reclaim_deletes_weight[{}]",
forceMergeDeletesPctAllowed, floorSegment, maxMergeAtOnce, maxMergeAtOnceExplicit, maxMergedSegment, segmentsPerTier, reclaimDeletesWeight);
}
}
private void reclaimDeletesWeight(Double reclaimDeletesWeight) {
void setReclaimDeletesWeight(Double reclaimDeletesWeight) {
mergePolicy.setReclaimDeletesWeight(reclaimDeletesWeight);
}
private void segmentsPerTier(Double segmentsPerTier) {
void setSegmentsPerTier(Double segmentsPerTier) {
mergePolicy.setSegmentsPerTier(segmentsPerTier);
}
private void maxMergedSegment(ByteSizeValue maxMergedSegment) {
void setMaxMergedSegment(ByteSizeValue maxMergedSegment) {
mergePolicy.setMaxMergedSegmentMB(maxMergedSegment.mbFrac());
}
private void maxMergesAtOnceExplicit(Integer maxMergeAtOnceExplicit) {
void setMaxMergesAtOnceExplicit(Integer maxMergeAtOnceExplicit) {
mergePolicy.setMaxMergeAtOnceExplicit(maxMergeAtOnceExplicit);
}
private void maxMergesAtOnce(Integer maxMergeAtOnce) {
void setMaxMergesAtOnce(Integer maxMergeAtOnce) {
mergePolicy.setMaxMergeAtOnce(maxMergeAtOnce);
}
private void floorSegmentSetting(ByteSizeValue floorSegementSetting) {
void setFloorSegmentSetting(ByteSizeValue floorSegementSetting) {
mergePolicy.setFloorSegmentMB(floorSegementSetting.mbFrac());
}
private void expungeDeletesAllowed(Double value) {
void setExpungeDeletesAllowed(Double value) {
mergePolicy.setForceMergeDeletesPctAllowed(value);
}
private void setNoCFSRatio(Double noCFSRatio) {
void setNoCFSRatio(Double noCFSRatio) {
mergePolicy.setNoCFSRatio(noCFSRatio);
}

View File

@ -21,9 +21,7 @@ package org.elasticsearch.index;
import org.apache.lucene.index.ConcurrentMergeScheduler;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.index.IndexSettings;
/**
* The merge scheduler (<code>ConcurrentMergeScheduler</code>) controls the execution of
@ -62,9 +60,6 @@ public final class MergeSchedulerConfig {
private volatile int maxMergeCount;
MergeSchedulerConfig(IndexSettings indexSettings) {
indexSettings.getScopedSettings().addSettingsUpdateConsumer(MAX_THREAD_COUNT_SETTING, this::setMaxThreadCount);
indexSettings.getScopedSettings().addSettingsUpdateConsumer(MAX_MERGE_COUNT_SETTING, this::setMaxMergeCount);
indexSettings.getScopedSettings().addSettingsUpdateConsumer(AUTO_THROTTLE_SETTING, this::setAutoThrottle);
maxThreadCount = indexSettings.getValue(MAX_THREAD_COUNT_SETTING);
maxMergeCount = indexSettings.getValue(MAX_MERGE_COUNT_SETTING);
this.autoThrottle = indexSettings.getValue(AUTO_THROTTLE_SETTING);

View File

@ -19,7 +19,9 @@
package org.elasticsearch.index.get;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.Term;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.bytes.BytesReference;
@ -50,7 +52,10 @@ import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import org.elasticsearch.index.shard.AbstractIndexShardComponent;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.search.SearchHitField;
import org.elasticsearch.search.fetch.parent.ParentFieldSubFetchPhase;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.internal.InternalSearchHitField;
import org.elasticsearch.search.lookup.LeafSearchLookup;
import org.elasticsearch.search.lookup.SearchLookup;
@ -350,6 +355,14 @@ public final class ShardGetService extends AbstractIndexShardComponent {
}
}
if (docMapper.parentFieldMapper().active()) {
String parentId = ParentFieldSubFetchPhase.getParentId(docMapper.parentFieldMapper(), docIdAndVersion.context.reader(), docIdAndVersion.docId);
if (fields == null) {
fields = new HashMap<>(1);
}
fields.put(ParentFieldMapper.NAME, new GetField(ParentFieldMapper.NAME, Collections.singletonList(parentId)));
}
// now, go and do the script thingy if needed
if (gFields != null && gFields.length > 0) {

View File

@ -1,33 +0,0 @@
/*
* 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.mapper;
/**
*/
public interface DocumentTypeListener {
/**
* Invoked just before a new document type has been created.
*
* @param mapper The new document mapper of the type being added
*/
void beforeCreate(DocumentMapper mapper);
}

View File

@ -70,6 +70,7 @@ public abstract class FieldMapper extends Mapper implements Cloneable {
this.fieldType = fieldType.clone();
this.defaultFieldType = defaultFieldType.clone();
this.defaultOptions = fieldType.indexOptions(); // we have to store it the fieldType is mutable
this.docValuesSet = fieldType.hasDocValues();
multiFieldsBuilder = new MultiFields.Builder();
}

View File

@ -125,8 +125,6 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
private final MapperAnalyzerWrapper searchAnalyzer;
private final MapperAnalyzerWrapper searchQuoteAnalyzer;
private final List<DocumentTypeListener> typeListeners = new CopyOnWriteArrayList<>();
private volatile Map<String, MappedFieldType> unmappedFieldTypes = emptyMap();
private volatile Set<String> parentTypes = emptySet();
@ -212,14 +210,6 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
return this.documentParser;
}
public void addTypeListener(DocumentTypeListener listener) {
typeListeners.add(listener);
}
public void removeTypeListener(DocumentTypeListener listener) {
typeListeners.remove(listener);
}
public DocumentMapper merge(String type, CompressedXContent mappingSource, MergeReason reason, boolean updateAllTypes) {
if (DEFAULT_MAPPING.equals(type)) {
// verify we can parse it
@ -335,14 +325,6 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
this.fullPathObjectMappers = fullPathObjectMappers;
this.parentTypes = parentTypes;
// 5. send notifications about the change
if (oldMapper == null) {
// means the mapping was created
for (DocumentTypeListener typeListener : typeListeners) {
typeListener.beforeCreate(mapper);
}
}
assert assertSerialization(newMapper);
assert assertMappersShareSameFieldType();
@ -481,105 +463,6 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
return new DocumentMapperForType(mapper, mapper.mapping());
}
/**
* A filter for search. If a filter is required, will return it, otherwise, will return <tt>null</tt>.
*/
@Nullable
public Query searchFilter(String... types) {
boolean filterPercolateType = hasMapping(PercolatorService.TYPE_NAME);
if (types != null && filterPercolateType) {
for (String type : types) {
if (PercolatorService.TYPE_NAME.equals(type)) {
filterPercolateType = false;
break;
}
}
}
Query percolatorType = null;
if (filterPercolateType) {
percolatorType = documentMapper(PercolatorService.TYPE_NAME).typeFilter();
}
if (types == null || types.length == 0) {
if (hasNested && filterPercolateType) {
BooleanQuery.Builder bq = new BooleanQuery.Builder();
bq.add(percolatorType, Occur.MUST_NOT);
bq.add(Queries.newNonNestedFilter(), Occur.MUST);
return new ConstantScoreQuery(bq.build());
} else if (hasNested) {
return Queries.newNonNestedFilter();
} else if (filterPercolateType) {
return new ConstantScoreQuery(Queries.not(percolatorType));
} else {
return null;
}
}
// if we filter by types, we don't need to filter by non nested docs
// since they have different types (starting with __)
if (types.length == 1) {
DocumentMapper docMapper = documentMapper(types[0]);
Query filter = docMapper != null ? docMapper.typeFilter() : new TermQuery(new Term(TypeFieldMapper.NAME, types[0]));
if (filterPercolateType) {
BooleanQuery.Builder bq = new BooleanQuery.Builder();
bq.add(percolatorType, Occur.MUST_NOT);
bq.add(filter, Occur.MUST);
return new ConstantScoreQuery(bq.build());
} else {
return filter;
}
}
// see if we can use terms filter
boolean useTermsFilter = true;
for (String type : types) {
DocumentMapper docMapper = documentMapper(type);
if (docMapper == null) {
useTermsFilter = false;
break;
}
if (docMapper.typeMapper().fieldType().indexOptions() == IndexOptions.NONE) {
useTermsFilter = false;
break;
}
}
// We only use terms filter is there is a type filter, this means we don't need to check for hasNested here
if (useTermsFilter) {
BytesRef[] typesBytes = new BytesRef[types.length];
for (int i = 0; i < typesBytes.length; i++) {
typesBytes[i] = new BytesRef(types[i]);
}
TermsQuery termsFilter = new TermsQuery(TypeFieldMapper.NAME, typesBytes);
if (filterPercolateType) {
BooleanQuery.Builder bq = new BooleanQuery.Builder();
bq.add(percolatorType, Occur.MUST_NOT);
bq.add(termsFilter, Occur.MUST);
return new ConstantScoreQuery(bq.build());
} else {
return termsFilter;
}
} else {
BooleanQuery.Builder typesBool = new BooleanQuery.Builder();
for (String type : types) {
DocumentMapper docMapper = documentMapper(type);
if (docMapper == null) {
typesBool.add(new TermQuery(new Term(TypeFieldMapper.NAME, type)), BooleanClause.Occur.SHOULD);
} else {
typesBool.add(docMapper.typeFilter(), BooleanClause.Occur.SHOULD);
}
}
BooleanQuery.Builder bool = new BooleanQuery.Builder();
bool.add(typesBool.build(), Occur.MUST);
if (filterPercolateType) {
bool.add(percolatorType, BooleanClause.Occur.MUST_NOT);
}
if (hasNested) {
bool.add(Queries.newNonNestedFilter(), BooleanClause.Occur.MUST);
}
return new ConstantScoreQuery(bool.build());
}
}
/**
* Returns the {@link MappedFieldType} for the give fullName.
*
@ -642,33 +525,6 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
return this.searchQuoteAnalyzer;
}
/**
* Resolves the closest inherited {@link ObjectMapper} that is nested.
*/
public ObjectMapper resolveClosestNestedObjectMapper(String fieldName) {
int indexOf = fieldName.lastIndexOf('.');
if (indexOf == -1) {
return null;
} else {
do {
String objectPath = fieldName.substring(0, indexOf);
ObjectMapper objectMapper = fullPathObjectMappers.get(objectPath);
if (objectMapper == null) {
indexOf = objectPath.lastIndexOf('.');
continue;
}
if (objectMapper.nested().isNested()) {
return objectMapper;
}
indexOf = objectPath.lastIndexOf('.');
} while (indexOf != -1);
}
return null;
}
public Set<String> getParentTypes() {
return parentTypes;
}

View File

@ -22,25 +22,28 @@ import org.apache.lucene.document.Field;
import org.apache.lucene.document.SortedDocValuesField;
import org.apache.lucene.index.DocValuesType;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.queries.TermsQuery;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.DocValuesTermsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.Version;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.lucene.BytesRefs;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.loader.SettingsLoader;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.ContentPath;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperBuilders;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.Uid;
import org.elasticsearch.index.mapper.core.StringFieldMapper;
import org.elasticsearch.index.query.QueryShardContext;
import java.io.IOException;
@ -65,22 +68,13 @@ public class ParentFieldMapper extends MetadataFieldMapper {
public static class Defaults {
public static final String NAME = ParentFieldMapper.NAME;
public static final MappedFieldType FIELD_TYPE = new ParentFieldType();
public static final MappedFieldType JOIN_FIELD_TYPE = new ParentFieldType();
public static final ParentFieldType FIELD_TYPE = new ParentFieldType();
static {
FIELD_TYPE.setIndexOptions(IndexOptions.DOCS);
FIELD_TYPE.setTokenized(false);
FIELD_TYPE.setStored(true);
FIELD_TYPE.setOmitNorms(true);
FIELD_TYPE.setIndexAnalyzer(Lucene.KEYWORD_ANALYZER);
FIELD_TYPE.setSearchAnalyzer(Lucene.KEYWORD_ANALYZER);
FIELD_TYPE.setName(NAME);
FIELD_TYPE.setIndexOptions(IndexOptions.NONE);
FIELD_TYPE.setHasDocValues(true);
FIELD_TYPE.setDocValuesType(DocValuesType.SORTED);
FIELD_TYPE.freeze();
JOIN_FIELD_TYPE.setHasDocValues(true);
JOIN_FIELD_TYPE.setDocValuesType(DocValuesType.SORTED);
JOIN_FIELD_TYPE.freeze();
}
}
@ -88,17 +82,10 @@ public class ParentFieldMapper extends MetadataFieldMapper {
private String parentType;
protected String indexName;
private final String documentType;
private final MappedFieldType parentJoinFieldType = Defaults.JOIN_FIELD_TYPE.clone();
private final MappedFieldType childJoinFieldType = Defaults.JOIN_FIELD_TYPE.clone();
public Builder(String documentType) {
super(Defaults.NAME, Defaults.FIELD_TYPE, Defaults.FIELD_TYPE);
this.indexName = name;
super(Defaults.NAME, new ParentFieldType(Defaults.FIELD_TYPE, documentType), Defaults.FIELD_TYPE);
this.documentType = documentType;
builder = this;
}
@ -108,22 +95,14 @@ public class ParentFieldMapper extends MetadataFieldMapper {
return builder;
}
@Override
public Builder fieldDataSettings(Settings fieldDataSettings) {
Settings settings = Settings.builder().put(childJoinFieldType.fieldDataType().getSettings()).put(fieldDataSettings).build();
childJoinFieldType.setFieldDataType(new FieldDataType(childJoinFieldType.fieldDataType().getType(), settings));
return this;
}
@Override
public ParentFieldMapper build(BuilderContext context) {
if (parentType == null) {
throw new MapperParsingException("[_parent] field mapping must contain the [type] option");
}
parentJoinFieldType.setName(joinField(documentType));
parentJoinFieldType.setFieldDataType(null);
childJoinFieldType.setName(joinField(parentType));
return new ParentFieldMapper(fieldType, parentJoinFieldType, childJoinFieldType, parentType, context.indexSettings());
name = joinField(parentType);
setupFieldType(context);
return new ParentFieldMapper(createParentJoinFieldMapper(documentType, context), fieldType, parentType, context.indexSettings());
}
}
@ -152,19 +131,40 @@ public class ParentFieldMapper extends MetadataFieldMapper {
}
@Override
public MetadataFieldMapper getDefault(Settings indexSettings, MappedFieldType fieldType, String parentType) {
return new ParentFieldMapper(indexSettings, fieldType, parentType);
public MetadataFieldMapper getDefault(Settings indexSettings, MappedFieldType fieldType, String typeName) {
StringFieldMapper parentJoinField = createParentJoinFieldMapper(typeName, new BuilderContext(indexSettings, new ContentPath(0)));
MappedFieldType childJoinFieldType = Defaults.FIELD_TYPE.clone();
childJoinFieldType.setName(joinField(null));
return new ParentFieldMapper(parentJoinField, childJoinFieldType, null, indexSettings);
}
}
static StringFieldMapper createParentJoinFieldMapper(String docType, BuilderContext context) {
StringFieldMapper.Builder parentJoinField = MapperBuilders.stringField(joinField(docType));
parentJoinField.indexOptions(IndexOptions.NONE);
parentJoinField.docValues(true);
parentJoinField.fieldType().setDocValuesType(DocValuesType.SORTED);
parentJoinField.fieldType().setFieldDataType(null);
return parentJoinField.build(context);
}
static final class ParentFieldType extends MappedFieldType {
final String documentType;
public ParentFieldType() {
setFieldDataType(new FieldDataType("_parent", settingsBuilder().put(MappedFieldType.Loading.KEY, Loading.EAGER_VALUE)));
setFieldDataType(new FieldDataType(NAME, settingsBuilder().put(MappedFieldType.Loading.KEY, Loading.EAGER_VALUE)));
documentType = null;
}
protected ParentFieldType(ParentFieldType ref) {
ParentFieldType(ParentFieldType ref, String documentType) {
super(ref);
this.documentType = documentType;
}
private ParentFieldType(ParentFieldType ref) {
super(ref);
this.documentType = ref.documentType;
}
@Override
@ -177,30 +177,6 @@ public class ParentFieldMapper extends MetadataFieldMapper {
return CONTENT_TYPE;
}
@Override
public Uid value(Object value) {
if (value == null) {
return null;
}
return Uid.createUid(value.toString());
}
@Override
public Object valueForSearch(Object value) {
if (value == null) {
return null;
}
String sValue = value.toString();
if (sValue == null) {
return null;
}
int index = sValue.indexOf(Uid.DELIMITER);
if (index == -1) {
return sValue;
}
return sValue.substring(index + 1);
}
/**
* We don't need to analyzer the text, and we need to convert it to UID...
*/
@ -216,67 +192,30 @@ public class ParentFieldMapper extends MetadataFieldMapper {
@Override
public Query termsQuery(List values, @Nullable QueryShardContext context) {
if (context == null) {
return super.termsQuery(values, context);
BytesRef[] ids = new BytesRef[values.size()];
for (int i = 0; i < ids.length; i++) {
ids[i] = indexedValueForSearch(values.get(i));
}
List<String> types = new ArrayList<>(context.getMapperService().types().size());
for (DocumentMapper documentMapper : context.getMapperService().docMappers(false)) {
if (!documentMapper.parentFieldMapper().active()) {
types.add(documentMapper.type());
}
}
List<BytesRef> bValues = new ArrayList<>(values.size());
for (Object value : values) {
BytesRef bValue = BytesRefs.toBytesRef(value);
if (Uid.hasDelimiter(bValue)) {
bValues.add(bValue);
} else {
// we use all non child types, cause we don't know if its exact or not...
for (String type : types) {
bValues.add(Uid.createUidAsBytes(type, bValue));
}
}
}
return new TermsQuery(name(), bValues);
BooleanQuery.Builder query = new BooleanQuery.Builder();
query.add(new DocValuesTermsQuery(name(), ids), BooleanClause.Occur.MUST);
query.add(new TermQuery(new Term(TypeFieldMapper.NAME, documentType)), BooleanClause.Occur.FILTER);
return query.build();
}
}
private final String parentType;
// determines the field data settings
private MappedFieldType childJoinFieldType;
// has no impact of field data settings, is just here for creating a join field, the parent field mapper in the child type pointing to this type determines the field data settings for this join field
private final MappedFieldType parentJoinFieldType;
// has no impact of field data settings, is just here for creating a join field,
// the parent field mapper in the child type pointing to this type determines the field data settings for this join field
private final StringFieldMapper parentJoinField;
private ParentFieldMapper(MappedFieldType fieldType, MappedFieldType parentJoinFieldType, MappedFieldType childJoinFieldType, String parentType, Settings indexSettings) {
super(NAME, fieldType, Defaults.FIELD_TYPE, indexSettings);
private ParentFieldMapper(StringFieldMapper parentJoinField, MappedFieldType childJoinFieldType, String parentType, Settings indexSettings) {
super(NAME, childJoinFieldType, Defaults.FIELD_TYPE, indexSettings);
this.parentType = parentType;
this.parentJoinFieldType = parentJoinFieldType;
this.parentJoinFieldType.freeze();
this.childJoinFieldType = childJoinFieldType;
if (childJoinFieldType != null) {
this.childJoinFieldType.freeze();
}
}
private ParentFieldMapper(Settings indexSettings, MappedFieldType existing, String parentType) {
this(existing == null ? Defaults.FIELD_TYPE.clone() : existing.clone(), joinFieldTypeForParentType(parentType, indexSettings), null, null, indexSettings);
}
private static MappedFieldType joinFieldTypeForParentType(String parentType, Settings indexSettings) {
MappedFieldType parentJoinFieldType = Defaults.JOIN_FIELD_TYPE.clone();
parentJoinFieldType.setName(joinField(parentType));
parentJoinFieldType.freeze();
return parentJoinFieldType;
this.parentJoinField = parentJoinField;
}
public MappedFieldType getParentJoinFieldType() {
return parentJoinFieldType;
}
public MappedFieldType getChildJoinFieldType() {
return childJoinFieldType;
return parentJoinField.fieldType();
}
public String type() {
@ -298,7 +237,7 @@ public class ParentFieldMapper extends MetadataFieldMapper {
protected void parseCreateField(ParseContext context, List<Field> fields) throws IOException {
boolean parent = context.docMapper().isParent(context.type());
if (parent) {
addJoinFieldIfNeeded(fields, parentJoinFieldType, context.id());
fields.add(new SortedDocValuesField(parentJoinField.fieldType().name(), new BytesRef(context.id())));
}
if (!active()) {
@ -309,8 +248,7 @@ public class ParentFieldMapper extends MetadataFieldMapper {
// we are in the parsing of _parent phase
String parentId = context.parser().text();
context.sourceToParse().parent(parentId);
fields.add(new Field(fieldType().name(), Uid.createUid(context.stringBuilder(), parentType, parentId), fieldType()));
addJoinFieldIfNeeded(fields, childJoinFieldType, parentId);
fields.add(new SortedDocValuesField(fieldType.name(), new BytesRef(parentId)));
} else {
// otherwise, we are running it post processing of the xcontent
String parsedParentId = context.doc().get(Defaults.NAME);
@ -321,8 +259,7 @@ public class ParentFieldMapper extends MetadataFieldMapper {
throw new MapperParsingException("No parent id provided, not within the document, and not externally");
}
// we did not add it in the parsing phase, add it now
fields.add(new Field(fieldType().name(), Uid.createUid(context.stringBuilder(), parentType, parentId), fieldType()));
addJoinFieldIfNeeded(fields, childJoinFieldType, parentId);
fields.add(new SortedDocValuesField(fieldType.name(), new BytesRef(parentId)));
} else if (parentId != null && !parsedParentId.equals(Uid.createUid(context.stringBuilder(), parentType, parentId))) {
throw new MapperParsingException("Parent id mismatch, document value is [" + Uid.createUid(parsedParentId).id() + "], while external value is [" + parentId + "]");
}
@ -331,12 +268,6 @@ public class ParentFieldMapper extends MetadataFieldMapper {
// we have parent mapping, yet no value was set, ignore it...
}
private void addJoinFieldIfNeeded(List<Field> fields, MappedFieldType fieldType, String id) {
if (fieldType.hasDocValues()) {
fields.add(new SortedDocValuesField(fieldType.name(), new BytesRef(id)));
}
}
public static String joinField(String parentType) {
return ParentFieldMapper.NAME + "#" + parentType;
}
@ -346,8 +277,9 @@ public class ParentFieldMapper extends MetadataFieldMapper {
return CONTENT_TYPE;
}
private boolean joinFieldHasCustomFieldDataSettings() {
return childJoinFieldType != null && childJoinFieldType.fieldDataType() != null && childJoinFieldType.fieldDataType().equals(Defaults.JOIN_FIELD_TYPE.fieldDataType()) == false;
@Override
public Iterator<Mapper> iterator() {
return Collections.<Mapper>singleton(parentJoinField).iterator();
}
@Override
@ -360,12 +292,16 @@ public class ParentFieldMapper extends MetadataFieldMapper {
builder.startObject(CONTENT_TYPE);
builder.field("type", parentType);
if (includeDefaults || joinFieldHasCustomFieldDataSettings()) {
builder.field("fielddata", (Map) childJoinFieldType.fieldDataType().getSettings().getAsMap());
builder.field("fielddata", (Map) fieldType().fieldDataType().getSettings().getAsMap());
}
builder.endObject();
return builder;
}
private boolean joinFieldHasCustomFieldDataSettings() {
return fieldType != null && fieldType.fieldDataType() != null && fieldType.fieldDataType().equals(Defaults.FIELD_TYPE.fieldDataType()) == false;
}
@Override
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
super.doMerge(mergeWith, updateAllTypes);
@ -375,18 +311,13 @@ public class ParentFieldMapper extends MetadataFieldMapper {
}
List<String> conflicts = new ArrayList<>();
fieldType().checkCompatibility(fieldMergeWith.fieldType(), conflicts, true); // always strict, this cannot change
parentJoinFieldType.checkCompatibility(fieldMergeWith.parentJoinFieldType, conflicts, true); // same here
if (childJoinFieldType != null) {
// TODO: this can be set to false when the old parent/child impl is removed, we can do eager global ordinals loading per type.
childJoinFieldType.checkCompatibility(fieldMergeWith.childJoinFieldType, conflicts, updateAllTypes == false);
}
fieldType().checkCompatibility(fieldMergeWith.fieldType, conflicts, true);
if (conflicts.isEmpty() == false) {
throw new IllegalArgumentException("Merge conflicts: " + conflicts);
}
if (active()) {
childJoinFieldType = fieldMergeWith.childJoinFieldType.clone();
fieldType = fieldMergeWith.fieldType.clone();
}
}

View File

@ -0,0 +1,104 @@
/*
* 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.query;
import org.apache.lucene.search.DocValuesTermsQuery;
import org.apache.lucene.search.Query;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.internal.ParentFieldMapper;
import java.io.IOException;
import java.util.Objects;
public final class ParentIdQueryBuilder extends AbstractQueryBuilder<ParentIdQueryBuilder> {
public static final String NAME = "parent_id";
static final ParentIdQueryBuilder PROTO = new ParentIdQueryBuilder(null, null);
private final String type;
private final String id;
public ParentIdQueryBuilder(String type, String id) {
this.type = type;
this.id = id;
}
public String getType() {
return type;
}
public String getId() {
return id;
}
@Override
protected void doXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(NAME);
builder.field(ParentIdQueryParser.TYPE_FIELD.getPreferredName(), type);
builder.field(ParentIdQueryParser.ID_FIELD.getPreferredName(), id);
printBoostAndQueryName(builder);
builder.endObject();
}
@Override
protected Query doToQuery(QueryShardContext context) throws IOException {
DocumentMapper childDocMapper = context.getMapperService().documentMapper(type);
if (childDocMapper == null) {
throw new QueryShardException(context, "[" + NAME + "] no mapping found for type [" + type + "]");
}
ParentFieldMapper parentFieldMapper = childDocMapper.parentFieldMapper();
if (parentFieldMapper.active() == false) {
throw new QueryShardException(context, "[" + NAME + "] _parent field has no parent type configured");
}
String fieldName = ParentFieldMapper.joinField(parentFieldMapper.type());
return new DocValuesTermsQuery(fieldName, id);
}
@Override
protected ParentIdQueryBuilder doReadFrom(StreamInput in) throws IOException {
String type = in.readString();
String id = in.readString();
return new ParentIdQueryBuilder(type, id);
}
@Override
protected void doWriteTo(StreamOutput out) throws IOException {
out.writeString(type);
out.writeString(id);
}
@Override
protected boolean doEquals(ParentIdQueryBuilder that) {
return Objects.equals(type, that.type) && Objects.equals(id, that.id);
}
@Override
protected int doHashCode() {
return Objects.hash(type, id);
}
@Override
public String getWriteableName() {
return NAME;
}
}

View File

@ -0,0 +1,76 @@
/*
* 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.query;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.XContentParser;
import java.io.IOException;
public final class ParentIdQueryParser implements QueryParser<ParentIdQueryBuilder> {
public static final ParseField ID_FIELD = new ParseField("id");
public static final ParseField TYPE_FIELD = new ParseField("type", "child_type");
@Override
public String[] names() {
return new String[]{ParentIdQueryBuilder.NAME};
}
@Override
public ParentIdQueryBuilder fromXContent(QueryParseContext parseContext) throws IOException {
XContentParser parser = parseContext.parser();
float boost = AbstractQueryBuilder.DEFAULT_BOOST;
String type = null;
String id = null;
String queryName = null;
String currentFieldName = null;
XContentParser.Token token;
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (token.isValue()) {
if (parseContext.parseFieldMatcher().match(currentFieldName, TYPE_FIELD)) {
type = parser.text();
} else if (parseContext.parseFieldMatcher().match(currentFieldName, ID_FIELD)) {
id = parser.text();
} else if (parseContext.parseFieldMatcher().match(currentFieldName, AbstractQueryBuilder.BOOST_FIELD)) {
boost = parser.floatValue();
} else if (parseContext.parseFieldMatcher().match(currentFieldName, AbstractQueryBuilder.NAME_FIELD)) {
queryName = parser.text();
} else {
throw new ParsingException(parser.getTokenLocation(), "[parent_id] query does not support [" + currentFieldName + "]");
}
} else {
throw new ParsingException(parser.getTokenLocation(), "[parent_id] query does not support [" + currentFieldName + "]");
}
}
ParentIdQueryBuilder queryBuilder = new ParentIdQueryBuilder(type, id);
queryBuilder.queryName(queryName);
queryBuilder.boost(boost);
return queryBuilder;
}
@Override
public ParentIdQueryBuilder getBuilderPrototype() {
return ParentIdQueryBuilder.PROTO;
}
}

View File

@ -490,6 +490,14 @@ public abstract class QueryBuilders {
return new HasParentQueryBuilder(type, query);
}
/**
* Constructs a new parent id query that returns all child documents of the specified type that
* point to the specified id.
*/
public static ParentIdQueryBuilder parentId(String type, String id) {
return new ParentIdQueryBuilder(type, id);
}
public static NestedQueryBuilder nestedQuery(String path, QueryBuilder query) {
return new NestedQueryBuilder(path, query);
}

View File

@ -541,25 +541,23 @@ public class IndexShard extends AbstractIndexShardComponent {
/** Writes all indexing changes to disk and opens a new searcher reflecting all changes. This can throw {@link EngineClosedException}. */
public void refresh(String source) {
verifyNotClosed();
if (getEngine().refreshNeeded()) {
if (canIndex()) {
long bytes = getEngine().getIndexBufferRAMBytesUsed();
writingBytes.addAndGet(bytes);
try {
logger.debug("refresh with source [{}] indexBufferRAMBytesUsed [{}]", source, new ByteSizeValue(bytes));
long time = System.nanoTime();
getEngine().refresh(source);
refreshMetric.inc(System.nanoTime() - time);
} finally {
logger.debug("remove [{}] writing bytes for shard [{}]", new ByteSizeValue(bytes), shardId());
writingBytes.addAndGet(-bytes);
}
} else {
logger.debug("refresh with source [{}]", source);
if (canIndex()) {
long bytes = getEngine().getIndexBufferRAMBytesUsed();
writingBytes.addAndGet(bytes);
try {
logger.debug("refresh with source [{}] indexBufferRAMBytesUsed [{}]", source, new ByteSizeValue(bytes));
long time = System.nanoTime();
getEngine().refresh(source);
refreshMetric.inc(System.nanoTime() - time);
} finally {
logger.debug("remove [{}] writing bytes for shard [{}]", new ByteSizeValue(bytes), shardId());
writingBytes.addAndGet(-bytes);
}
} else {
logger.debug("refresh with source [{}]", source);
long time = System.nanoTime();
getEngine().refresh(source);
refreshMetric.inc(System.nanoTime() - time);
}
}
@ -1514,4 +1512,15 @@ public class IndexShard extends AbstractIndexShardComponent {
return engineFactory;
}
/**
* Returns <code>true</code> iff one or more changes to the engine are not visible to via the current searcher.
* Otherwise <code>false</code>.
*
* @throws EngineClosedException if the engine is already closed
* @throws AlreadyClosedException if the internal indexwriter in the engine is already closed
*/
public boolean isRefreshNeeded() {
return getEngine().refreshNeeded();
}
}

View File

@ -0,0 +1,51 @@
/*
* 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.similarity;
import org.apache.lucene.search.similarities.DFISimilarity;
import org.apache.lucene.search.similarities.Similarity;
import org.elasticsearch.common.settings.Settings;
/**
* {@link SimilarityProvider} for the {@link DFISimilarity}.
* <p>
* Configuration options available:
* <ul>
* <li>discount_overlaps</li>
* </ul>
* @see DFISimilarity For more information about configuration
*/
public class DFISimilarityProvider extends AbstractSimilarityProvider {
private final DFISimilarity similarity;
public DFISimilarityProvider(String name, Settings settings) {
super(name);
boolean discountOverlaps = settings.getAsBoolean("discount_overlaps", true);
this.similarity = new DFISimilarity();
this.similarity.setDiscountOverlaps(discountOverlaps);
}
@Override
public Similarity get() {
return similarity;
}
}

View File

@ -52,6 +52,7 @@ public final class SimilarityService extends AbstractIndexComponent {
buildIn.put("IB", IBSimilarityProvider::new);
buildIn.put("LMDirichlet", LMDirichletSimilarityProvider::new);
buildIn.put("LMJelinekMercer", LMJelinekMercerSimilarityProvider::new);
buildIn.put("DFI", DFISimilarityProvider::new);
DEFAULTS = Collections.unmodifiableMap(defaults);
BUILT_IN = Collections.unmodifiableMap(buildIn);
}

View File

@ -21,7 +21,6 @@ package org.elasticsearch.index.store;
import org.apache.lucene.store.StoreRateLimiting;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.index.AbstractIndexComponent;
import org.elasticsearch.index.IndexSettings;
@ -30,16 +29,17 @@ import org.elasticsearch.index.shard.ShardPath;
*
*/
public class IndexStore extends AbstractIndexComponent {
public static final Setting<StoreRateLimiting.Type> INDEX_STORE_THROTTLE_TYPE_SETTING = new Setting<>("index.store.throttle.type", "none", StoreRateLimiting.Type::fromString, true, Setting.Scope.INDEX) ;
public static final Setting<IndexRateLimitingType> INDEX_STORE_THROTTLE_TYPE_SETTING = new Setting<>("index.store.throttle.type", "none", IndexRateLimitingType::fromString, true, Setting.Scope.INDEX) ;
public static final Setting<ByteSizeValue> INDEX_STORE_THROTTLE_MAX_BYTES_PER_SEC_SETTING = Setting.byteSizeSetting("index.store.throttle.max_bytes_per_sec", new ByteSizeValue(0), true, Setting.Scope.INDEX);
protected final IndexStoreConfig indexStoreConfig;
private final StoreRateLimiting rateLimiting = new StoreRateLimiting();
private volatile IndexRateLimitingType type;
public IndexStore(IndexSettings indexSettings, IndexStoreConfig indexStoreConfig) {
super(indexSettings);
this.indexStoreConfig = indexStoreConfig;
rateLimiting.setType(indexSettings.getValue(INDEX_STORE_THROTTLE_TYPE_SETTING));
setType(indexSettings.getValue(INDEX_STORE_THROTTLE_TYPE_SETTING));
rateLimiting.setMaxRate(indexSettings.getValue(INDEX_STORE_THROTTLE_MAX_BYTES_PER_SEC_SETTING));
logger.debug("using index.store.throttle.type [{}], with index.store.throttle.max_bytes_per_sec [{}]", rateLimiting.getType(), rateLimiting.getRateLimiter());
}
@ -49,7 +49,7 @@ public class IndexStore extends AbstractIndexComponent {
* the node level one (defaults to the node level one).
*/
public StoreRateLimiting rateLimiting() {
return rateLimiting.getType() == StoreRateLimiting.Type.NONE ? indexStoreConfig.getNodeRateLimiter() : this.rateLimiting;
return type.useStoreLimiter() ? indexStoreConfig.getNodeRateLimiter() : this.rateLimiting;
}
/**
@ -59,11 +59,44 @@ public class IndexStore extends AbstractIndexComponent {
return new FsDirectoryService(indexSettings, this, path);
}
public void setType(StoreRateLimiting.Type type) {
rateLimiting.setType(type);
public void setType(IndexRateLimitingType type) {
this.type = type;
if (type.useStoreLimiter() == false) {
rateLimiting.setType(type.type);
}
}
public void setMaxRate(ByteSizeValue rate) {
rateLimiting.setMaxRate(rate);
}
/**
* On an index level we can configure all of {@link org.apache.lucene.store.StoreRateLimiting.Type} as well as
* <tt>node</tt> which will then use a global rate limiter that has it's own configuration. The global one is
* configured in {@link IndexStoreConfig} which is managed by the per-node {@link org.elasticsearch.indices.IndicesService}
*/
public static final class IndexRateLimitingType {
private final StoreRateLimiting.Type type;
private IndexRateLimitingType(StoreRateLimiting.Type type) {
this.type = type;
}
private boolean useStoreLimiter() {
return type == null;
}
static IndexRateLimitingType fromString(String type) {
if ("node".equalsIgnoreCase(type)) {
return new IndexRateLimitingType(null);
} else {
try {
return new IndexRateLimitingType(StoreRateLimiting.Type.fromString(type));
} catch (IllegalArgumentException ex) {
throw new IllegalArgumentException("rate limiting type [" + type + "] not valid, can be one of [all|merge|none|node]");
}
}
}
}
}

View File

@ -73,7 +73,6 @@ import org.elasticsearch.snapshots.RestoreService;
import org.elasticsearch.threadpool.ThreadPool;
import java.util.Arrays;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;

View File

@ -111,11 +111,12 @@ public class IndicesStore extends AbstractComponent implements ClusterStateListe
}
for (IndexRoutingTable indexRoutingTable : event.state().routingTable()) {
IndexSettings indexSettings = new IndexSettings(event.state().getMetaData().index(indexRoutingTable.index()), settings);
// Note, closed indices will not have any routing information, so won't be deleted
for (IndexShardRoutingTable indexShardRoutingTable : indexRoutingTable) {
if (shardCanBeDeleted(event.state(), indexShardRoutingTable)) {
ShardId shardId = indexShardRoutingTable.shardId();
IndexService indexService = indicesService.indexService(indexRoutingTable.getIndex());
IndexSettings indexSettings = indexService != null ? indexService.getIndexSettings() : new IndexSettings(event.state().getMetaData().index(indexRoutingTable.index()), settings);
if (indicesService.canDeleteShardContent(shardId, indexSettings)) {
deleteShardIfExistElseWhere(event.state(), indexShardRoutingTable);
}

View File

@ -68,6 +68,7 @@ import org.elasticsearch.index.query.MatchQueryParser;
import org.elasticsearch.index.query.MoreLikeThisQueryParser;
import org.elasticsearch.index.query.MultiMatchQueryParser;
import org.elasticsearch.index.query.NestedQueryParser;
import org.elasticsearch.index.query.ParentIdQueryParser;
import org.elasticsearch.index.query.PrefixQueryParser;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryParser;
@ -218,6 +219,7 @@ import org.elasticsearch.search.fetch.explain.ExplainFetchSubPhase;
import org.elasticsearch.search.fetch.fielddata.FieldDataFieldsFetchSubPhase;
import org.elasticsearch.search.fetch.innerhits.InnerHitsFetchSubPhase;
import org.elasticsearch.search.fetch.matchedqueries.MatchedQueriesFetchSubPhase;
import org.elasticsearch.search.fetch.parent.ParentFieldSubFetchPhase;
import org.elasticsearch.search.fetch.script.ScriptFieldsFetchSubPhase;
import org.elasticsearch.search.fetch.source.FetchSourceSubPhase;
import org.elasticsearch.search.fetch.version.VersionFetchSubPhase;
@ -336,6 +338,7 @@ public class SearchModule extends AbstractModule {
fetchSubPhaseMultibinder.addBinding().to(VersionFetchSubPhase.class);
fetchSubPhaseMultibinder.addBinding().to(MatchedQueriesFetchSubPhase.class);
fetchSubPhaseMultibinder.addBinding().to(HighlightPhase.class);
fetchSubPhaseMultibinder.addBinding().to(ParentFieldSubFetchPhase.class);
for (Class<? extends FetchSubPhase> clazz : fetchSubPhases) {
fetchSubPhaseMultibinder.addBinding().to(clazz);
}
@ -523,6 +526,7 @@ public class SearchModule extends AbstractModule {
registerQueryParser(GeoPolygonQueryParser::new);
registerQueryParser(ExistsQueryParser::new);
registerQueryParser(MatchNoneQueryParser::new);
registerQueryParser(ParentIdQueryParser::new);
if (ShapesAvailability.JTS_AVAILABLE && ShapesAvailability.SPATIAL4J_AVAILABLE) {
registerQueryParser(GeoShapeQueryParser::new);
}

View File

@ -1033,22 +1033,8 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
final Map<String, MappedFieldType> warmUp = new HashMap<>();
for (DocumentMapper docMapper : mapperService.docMappers(false)) {
for (FieldMapper fieldMapper : docMapper.mappers()) {
final FieldDataType fieldDataType;
final String indexName;
if (fieldMapper instanceof ParentFieldMapper) {
MappedFieldType joinFieldType = ((ParentFieldMapper) fieldMapper).getChildJoinFieldType();
if (joinFieldType == null) {
continue;
}
fieldDataType = joinFieldType.fieldDataType();
// TODO: this can be removed in 3.0 when the old parent/child impl is removed:
// related to: https://github.com/elastic/elasticsearch/pull/12418
indexName = fieldMapper.fieldType().name();
} else {
fieldDataType = fieldMapper.fieldType().fieldDataType();
indexName = fieldMapper.fieldType().name();
}
final FieldDataType fieldDataType = fieldMapper.fieldType().fieldDataType();
final String indexName = fieldMapper.fieldType().name();
if (fieldDataType == null) {
continue;
}
@ -1101,21 +1087,8 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
final Map<String, MappedFieldType> warmUpGlobalOrdinals = new HashMap<>();
for (DocumentMapper docMapper : mapperService.docMappers(false)) {
for (FieldMapper fieldMapper : docMapper.mappers()) {
final FieldDataType fieldDataType;
final String indexName;
if (fieldMapper instanceof ParentFieldMapper) {
MappedFieldType joinFieldType = ((ParentFieldMapper) fieldMapper).getChildJoinFieldType();
if (joinFieldType == null) {
continue;
}
fieldDataType = joinFieldType.fieldDataType();
// TODO: this can be removed in 3.0 when the old parent/child impl is removed:
// related to: https://github.com/elastic/elasticsearch/pull/12418
indexName = fieldMapper.fieldType().name();
} else {
fieldDataType = fieldMapper.fieldType().fieldDataType();
indexName = fieldMapper.fieldType().name();
}
final FieldDataType fieldDataType = fieldMapper.fieldType().fieldDataType();
final String indexName = fieldMapper.fieldType().name();
if (fieldDataType == null) {
continue;
}

View File

@ -27,6 +27,7 @@ import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.ConstantScoreScorer;
import org.apache.lucene.search.ConstantScoreWeight;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.DocValuesTermsQuery;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.Scorer;
@ -284,20 +285,18 @@ public final class InnerHitsContext {
@Override
public TopDocs topDocs(SearchContext context, FetchSubPhase.HitContext hitContext) throws IOException {
final String field;
final String term;
final Query hitQuery;
if (isParentHit(hitContext.hit())) {
field = ParentFieldMapper.NAME;
term = Uid.createUid(hitContext.hit().type(), hitContext.hit().id());
String field = ParentFieldMapper.joinField(hitContext.hit().type());
hitQuery = new DocValuesTermsQuery(field, hitContext.hit().id());
} else if (isChildHit(hitContext.hit())) {
DocumentMapper hitDocumentMapper = mapperService.documentMapper(hitContext.hit().type());
final String parentType = hitDocumentMapper.parentFieldMapper().type();
field = UidFieldMapper.NAME;
SearchHitField parentField = hitContext.hit().field(ParentFieldMapper.NAME);
if (parentField == null) {
throw new IllegalStateException("All children must have a _parent");
}
term = Uid.createUid(parentType, (String) parentField.getValue());
hitQuery = new TermQuery(new Term(UidFieldMapper.NAME, Uid.createUid(parentType, parentField.getValue())));
} else {
return Lucene.EMPTY_TOP_DOCS;
}
@ -305,9 +304,9 @@ public final class InnerHitsContext {
BooleanQuery q = new BooleanQuery.Builder()
.add(query.query(), Occur.MUST)
// Only include docs that have the current hit as parent
.add(new TermQuery(new Term(field, term)), Occur.MUST)
.add(hitQuery, Occur.FILTER)
// Only include docs that have this inner hits type
.add(documentMapper.typeFilter(), Occur.MUST)
.add(documentMapper.typeFilter(), Occur.FILTER)
.build();
if (size() == 0) {
final int count = context.searcher().count(q);

View File

@ -0,0 +1,88 @@
/*
* 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.search.fetch.parent;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.index.mapper.internal.ParentFieldMapper;
import org.elasticsearch.search.SearchHitField;
import org.elasticsearch.search.SearchParseElement;
import org.elasticsearch.search.fetch.FetchSubPhase;
import org.elasticsearch.search.fetch.innerhits.InnerHitsContext;
import org.elasticsearch.search.internal.InternalSearchHit;
import org.elasticsearch.search.internal.InternalSearchHitField;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
public class ParentFieldSubFetchPhase implements FetchSubPhase {
@Override
public Map<String, ? extends SearchParseElement> parseElements() {
return Collections.emptyMap();
}
@Override
public boolean hitExecutionNeeded(SearchContext context) {
return true;
}
@Override
public void hitExecute(SearchContext context, HitContext hitContext) {
ParentFieldMapper parentFieldMapper = context.mapperService().documentMapper(hitContext.hit().type()).parentFieldMapper();
if (parentFieldMapper.active() == false) {
return;
}
String parentId = getParentId(parentFieldMapper, hitContext.reader(), hitContext.docId());
Map<String, SearchHitField> fields = hitContext.hit().fieldsOrNull();
if (fields == null) {
fields = new HashMap<>();
hitContext.hit().fields(fields);
}
fields.put(ParentFieldMapper.NAME, new InternalSearchHitField(ParentFieldMapper.NAME, Collections.singletonList(parentId)));
}
@Override
public boolean hitsExecutionNeeded(SearchContext context) {
return false;
}
@Override
public void hitsExecute(SearchContext context, InternalSearchHit[] hits) {
}
public static String getParentId(ParentFieldMapper fieldMapper, LeafReader reader, int docId) {
try {
SortedDocValues docValues = reader.getSortedDocValues(fieldMapper.name());
BytesRef parentId = docValues.get(docId);
assert parentId.length > 0;
return parentId.utf8ToString();
} catch (IOException e) {
throw ExceptionsHelper.convertToElastic(e);
}
}
}

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.internal;
import org.apache.lucene.queries.TermsQuery;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.BoostQuery;
@ -26,6 +27,7 @@ import org.apache.lucene.search.Collector;
import org.apache.lucene.search.ConstantScoreQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.Sort;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter;
import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.cache.recycler.PageCacheRecycler;
@ -45,6 +47,7 @@ import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.fielddata.IndexFieldDataService;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.internal.TypeFieldMapper;
import org.elasticsearch.index.mapper.object.ObjectMapper;
import org.elasticsearch.index.query.AbstractQueryBuilder;
import org.elasticsearch.index.query.ParsedQuery;
@ -238,19 +241,37 @@ public class DefaultSearchContext extends SearchContext {
}
@Override
@Nullable
public Query searchFilter(String[] types) {
Query filter = mapperService().searchFilter(types);
if (filter == null && aliasFilter == null) {
return createSearchFilter(types, aliasFilter, mapperService().hasNested());
}
// extracted to static helper method to make writing unit tests easier:
static Query createSearchFilter(String[] types, Query aliasFilter, boolean hasNestedFields) {
Query typesFilter = null;
if (types != null && types.length >= 1) {
BytesRef[] typesBytes = new BytesRef[types.length];
for (int i = 0; i < typesBytes.length; i++) {
typesBytes[i] = new BytesRef(types[i]);
}
typesFilter = new TermsQuery(TypeFieldMapper.NAME, typesBytes);
}
if (typesFilter == null && aliasFilter == null && hasNestedFields == false) {
return null;
}
BooleanQuery.Builder bq = new BooleanQuery.Builder();
if (filter != null) {
bq.add(filter, Occur.MUST);
if (typesFilter != null) {
bq.add(typesFilter, Occur.FILTER);
} else if (hasNestedFields) {
bq.add(Queries.newNonNestedFilter(), Occur.FILTER);
}
if (aliasFilter != null) {
bq.add(aliasFilter, Occur.MUST);
bq.add(aliasFilter, Occur.FILTER);
}
return new ConstantScoreQuery(bq.build());
return bq.build();
}
@Override

View File

@ -210,15 +210,20 @@ public final class SuggestUtils {
public static final ParseField MIN_WORD_LENGTH = new ParseField("min_word_length", "min_word_len");
public static final ParseField MIN_DOC_FREQ = new ParseField("min_doc_freq");
public static final ParseField SHARD_SIZE = new ParseField("shard_size");
public static final ParseField ANALYZER = new ParseField("analyzer");
public static final ParseField FIELD = new ParseField("field");
public static final ParseField SIZE = new ParseField("size");
public static final ParseField SORT = new ParseField("sort");
public static final ParseField ACCURACY = new ParseField("accuracy");
}
public static boolean parseDirectSpellcheckerSettings(XContentParser parser, String fieldName,
DirectSpellcheckerSettings suggestion, ParseFieldMatcher parseFieldMatcher) throws IOException {
if ("accuracy".equals(fieldName)) {
if (parseFieldMatcher.match(fieldName, Fields.ACCURACY)) {
suggestion.accuracy(parser.floatValue());
} else if (parseFieldMatcher.match(fieldName, Fields.SUGGEST_MODE)) {
suggestion.suggestMode(SuggestUtils.resolveSuggestMode(parser.text()));
} else if ("sort".equals(fieldName)) {
} else if (parseFieldMatcher.match(fieldName, Fields.SORT)) {
suggestion.sort(SuggestUtils.resolveSort(parser.text()));
} else if (parseFieldMatcher.match(fieldName, Fields.STRING_DISTANCE)) {
suggestion.stringDistance(SuggestUtils.resolveDistance(parser.text()));
@ -246,16 +251,16 @@ public final class SuggestUtils {
public static boolean parseSuggestContext(XContentParser parser, MapperService mapperService, String fieldName,
SuggestionSearchContext.SuggestionContext suggestion, ParseFieldMatcher parseFieldMatcher) throws IOException {
if ("analyzer".equals(fieldName)) {
if (parseFieldMatcher.match(fieldName, Fields.ANALYZER)) {
String analyzerName = parser.text();
Analyzer analyzer = mapperService.analysisService().analyzer(analyzerName);
if (analyzer == null) {
throw new IllegalArgumentException("Analyzer [" + analyzerName + "] doesn't exists");
}
suggestion.setAnalyzer(analyzer);
} else if ("field".equals(fieldName)) {
} else if (parseFieldMatcher.match(fieldName, Fields.FIELD)) {
suggestion.setField(parser.text());
} else if ("size".equals(fieldName)) {
} else if (parseFieldMatcher.match(fieldName, Fields.SIZE)) {
suggestion.setSize(parser.intValue());
} else if (parseFieldMatcher.match(fieldName, Fields.SHARD_SIZE)) {
suggestion.setShardSize(parser.intValue());

View File

@ -21,7 +21,6 @@ package org.elasticsearch.search.suggest.completion;
import org.apache.lucene.analysis.Analyzer;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.HasContextAndHeaders;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.unit.Fuzziness;
import org.elasticsearch.common.xcontent.ObjectParser;
@ -34,7 +33,10 @@ import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.core.CompletionFieldMapper;
import org.elasticsearch.index.query.RegexpFlag;
import org.elasticsearch.search.suggest.SuggestContextParser;
import org.elasticsearch.search.suggest.SuggestUtils.Fields;
import org.elasticsearch.search.suggest.SuggestionSearchContext;
import org.elasticsearch.search.suggest.completion.CompletionSuggestionBuilder.FuzzyOptionsBuilder;
import org.elasticsearch.search.suggest.completion.CompletionSuggestionBuilder.RegexOptionsBuilder;
import org.elasticsearch.search.suggest.completion.context.ContextMapping;
import org.elasticsearch.search.suggest.completion.context.ContextMappings;
@ -74,29 +76,29 @@ import java.util.Map;
*/
public class CompletionSuggestParser implements SuggestContextParser {
private static ObjectParser<CompletionSuggestionContext, ContextAndSuggest> TLP_PARSER = new ObjectParser<>("completion", null);
private static ObjectParser<CompletionSuggestionBuilder.RegexOptionsBuilder, ContextAndSuggest> REGEXP_PARSER = new ObjectParser<>("regexp", CompletionSuggestionBuilder.RegexOptionsBuilder::new);
private static ObjectParser<CompletionSuggestionBuilder.FuzzyOptionsBuilder, ContextAndSuggest> FUZZY_PARSER = new ObjectParser<>("fuzzy", CompletionSuggestionBuilder.FuzzyOptionsBuilder::new);
private static ObjectParser<CompletionSuggestionContext, ContextAndSuggest> TLP_PARSER = new ObjectParser<>(CompletionSuggestionBuilder.SUGGESTION_NAME, null);
private static ObjectParser<CompletionSuggestionBuilder.RegexOptionsBuilder, ContextAndSuggest> REGEXP_PARSER = new ObjectParser<>(RegexOptionsBuilder.REGEX_OPTIONS.getPreferredName(), CompletionSuggestionBuilder.RegexOptionsBuilder::new);
private static ObjectParser<CompletionSuggestionBuilder.FuzzyOptionsBuilder, ContextAndSuggest> FUZZY_PARSER = new ObjectParser<>(FuzzyOptionsBuilder.FUZZY_OPTIONS.getPreferredName(), CompletionSuggestionBuilder.FuzzyOptionsBuilder::new);
static {
FUZZY_PARSER.declareInt(CompletionSuggestionBuilder.FuzzyOptionsBuilder::setFuzzyMinLength, new ParseField("min_length"));
FUZZY_PARSER.declareInt(CompletionSuggestionBuilder.FuzzyOptionsBuilder::setMaxDeterminizedStates, new ParseField("max_determinized_states"));
FUZZY_PARSER.declareBoolean(CompletionSuggestionBuilder.FuzzyOptionsBuilder::setUnicodeAware, new ParseField("unicode_aware"));
FUZZY_PARSER.declareInt(CompletionSuggestionBuilder.FuzzyOptionsBuilder::setFuzzyPrefixLength, new ParseField("prefix_length"));
FUZZY_PARSER.declareBoolean(CompletionSuggestionBuilder.FuzzyOptionsBuilder::setTranspositions, new ParseField("transpositions"));
FUZZY_PARSER.declareInt(CompletionSuggestionBuilder.FuzzyOptionsBuilder::setFuzzyMinLength, FuzzyOptionsBuilder.MIN_LENGTH_FIELD);
FUZZY_PARSER.declareInt(CompletionSuggestionBuilder.FuzzyOptionsBuilder::setMaxDeterminizedStates, FuzzyOptionsBuilder.MAX_DETERMINIZED_STATES_FIELD);
FUZZY_PARSER.declareBoolean(CompletionSuggestionBuilder.FuzzyOptionsBuilder::setUnicodeAware, FuzzyOptionsBuilder.UNICODE_AWARE_FIELD);
FUZZY_PARSER.declareInt(CompletionSuggestionBuilder.FuzzyOptionsBuilder::setFuzzyPrefixLength, FuzzyOptionsBuilder.PREFIX_LENGTH_FIELD);
FUZZY_PARSER.declareBoolean(CompletionSuggestionBuilder.FuzzyOptionsBuilder::setTranspositions, FuzzyOptionsBuilder.TRANSPOSITION_FIELD);
FUZZY_PARSER.declareValue((a, b) -> {
try {
a.setFuzziness(Fuzziness.parse(b).asDistance());
} catch (IOException e) {
throw new ElasticsearchException(e);
}
}, new ParseField("fuzziness"));
REGEXP_PARSER.declareInt(CompletionSuggestionBuilder.RegexOptionsBuilder::setMaxDeterminizedStates, new ParseField("max_determinized_states"));
REGEXP_PARSER.declareStringOrNull(CompletionSuggestionBuilder.RegexOptionsBuilder::setFlags, new ParseField("flags"));
}, Fuzziness.FIELD);
REGEXP_PARSER.declareInt(CompletionSuggestionBuilder.RegexOptionsBuilder::setMaxDeterminizedStates, RegexOptionsBuilder.MAX_DETERMINIZED_STATES);
REGEXP_PARSER.declareStringOrNull(CompletionSuggestionBuilder.RegexOptionsBuilder::setFlags, RegexOptionsBuilder.FLAGS_VALUE);
TLP_PARSER.declareStringArray(CompletionSuggestionContext::setPayloadFields, new ParseField("payload"));
TLP_PARSER.declareObjectOrDefault(CompletionSuggestionContext::setFuzzyOptionsBuilder, FUZZY_PARSER, CompletionSuggestionBuilder.FuzzyOptionsBuilder::new, new ParseField("fuzzy"));
TLP_PARSER.declareObject(CompletionSuggestionContext::setRegexOptionsBuilder, REGEXP_PARSER, new ParseField("regexp"));
TLP_PARSER.declareString(SuggestionSearchContext.SuggestionContext::setField, new ParseField("field"));
TLP_PARSER.declareStringArray(CompletionSuggestionContext::setPayloadFields, CompletionSuggestionBuilder.PAYLOAD_FIELD);
TLP_PARSER.declareObjectOrDefault(CompletionSuggestionContext::setFuzzyOptionsBuilder, FUZZY_PARSER, CompletionSuggestionBuilder.FuzzyOptionsBuilder::new, FuzzyOptionsBuilder.FUZZY_OPTIONS);
TLP_PARSER.declareObject(CompletionSuggestionContext::setRegexOptionsBuilder, REGEXP_PARSER, RegexOptionsBuilder.REGEX_OPTIONS);
TLP_PARSER.declareString(SuggestionSearchContext.SuggestionContext::setField, Fields.FIELD);
TLP_PARSER.declareField((p, v, c) -> {
String analyzerName = p.text();
Analyzer analyzer = c.mapperService.analysisService().analyzer(analyzerName);
@ -104,10 +106,9 @@ public class CompletionSuggestParser implements SuggestContextParser {
throw new IllegalArgumentException("Analyzer [" + analyzerName + "] doesn't exists");
}
v.setAnalyzer(analyzer);
}, new ParseField("analyzer"), ObjectParser.ValueType.STRING);
TLP_PARSER.declareString(SuggestionSearchContext.SuggestionContext::setField, new ParseField("analyzer"));
TLP_PARSER.declareInt(SuggestionSearchContext.SuggestionContext::setSize, new ParseField("size"));
TLP_PARSER.declareInt(SuggestionSearchContext.SuggestionContext::setShardSize, new ParseField("size"));
}, Fields.ANALYZER, ObjectParser.ValueType.STRING);
TLP_PARSER.declareInt(SuggestionSearchContext.SuggestionContext::setSize, Fields.SIZE);
TLP_PARSER.declareInt(SuggestionSearchContext.SuggestionContext::setShardSize, Fields.SHARD_SIZE);
TLP_PARSER.declareField((p, v, c) -> {
// Copy the current structure. We will parse, once the mapping is provided
XContentBuilder builder = XContentFactory.contentBuilder(p.contentType());
@ -115,7 +116,7 @@ public class CompletionSuggestParser implements SuggestContextParser {
BytesReference bytes = builder.bytes();
c.contextParser = XContentFactory.xContent(bytes).createParser(bytes);
p.skipChildren();
}, new ParseField("contexts", "context"), ObjectParser.ValueType.OBJECT); // context is deprecated
}, CompletionSuggestionBuilder.CONTEXTS_FIELD, ObjectParser.ValueType.OBJECT); // context is deprecated
}
private static class ContextAndSuggest {

View File

@ -21,6 +21,7 @@ package org.elasticsearch.search.suggest.completion;
import org.apache.lucene.search.suggest.document.FuzzyCompletionQuery;
import org.apache.lucene.util.automaton.Operations;
import org.apache.lucene.util.automaton.RegExp;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.unit.Fuzziness;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -45,19 +46,30 @@ import java.util.Set;
* indexing.
*/
public class CompletionSuggestionBuilder extends SuggestBuilder.SuggestionBuilder<CompletionSuggestionBuilder> {
final static String SUGGESTION_NAME = "completion";
static final ParseField PAYLOAD_FIELD = new ParseField("payload");
static final ParseField CONTEXTS_FIELD = new ParseField("contexts", "context");
private FuzzyOptionsBuilder fuzzyOptionsBuilder;
private RegexOptionsBuilder regexOptionsBuilder;
private final Map<String, List<ToXContent>> queryContexts = new HashMap<>();
private final Set<String> payloadFields = new HashSet<>();
public CompletionSuggestionBuilder(String name) {
super(name, "completion");
super(name, SUGGESTION_NAME);
}
/**
* Options for fuzzy queries
*/
public static class FuzzyOptionsBuilder implements ToXContent {
static final ParseField FUZZY_OPTIONS = new ParseField("fuzzy");
static final ParseField TRANSPOSITION_FIELD = new ParseField("transpositions");
static final ParseField MIN_LENGTH_FIELD = new ParseField("min_length");
static final ParseField PREFIX_LENGTH_FIELD = new ParseField("prefix_length");
static final ParseField UNICODE_AWARE_FIELD = new ParseField("unicode_aware");
static final ParseField MAX_DETERMINIZED_STATES_FIELD = new ParseField("max_determinized_states");
private int editDistance = FuzzyCompletionQuery.DEFAULT_MAX_EDITS;
private boolean transpositions = FuzzyCompletionQuery.DEFAULT_TRANSPOSITIONS;
private int fuzzyMinLength = FuzzyCompletionQuery.DEFAULT_MIN_FUZZY_LENGTH;
@ -179,13 +191,13 @@ public class CompletionSuggestionBuilder extends SuggestBuilder.SuggestionBuilde
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject("fuzzy");
builder.startObject(FUZZY_OPTIONS.getPreferredName());
builder.field(Fuzziness.FIELD.getPreferredName(), editDistance);
builder.field("transpositions", transpositions);
builder.field("min_length", fuzzyMinLength);
builder.field("prefix_length", fuzzyPrefixLength);
builder.field("unicode_aware", unicodeAware);
builder.field("max_determinized_states", maxDeterminizedStates);
builder.field(TRANSPOSITION_FIELD.getPreferredName(), transpositions);
builder.field(MIN_LENGTH_FIELD.getPreferredName(), fuzzyMinLength);
builder.field(PREFIX_LENGTH_FIELD.getPreferredName(), fuzzyPrefixLength);
builder.field(UNICODE_AWARE_FIELD.getPreferredName(), unicodeAware);
builder.field(MAX_DETERMINIZED_STATES_FIELD.getPreferredName(), maxDeterminizedStates);
builder.endObject();
return builder;
}
@ -195,6 +207,9 @@ public class CompletionSuggestionBuilder extends SuggestBuilder.SuggestionBuilde
* Options for regular expression queries
*/
public static class RegexOptionsBuilder implements ToXContent {
static final ParseField REGEX_OPTIONS = new ParseField("regex");
static final ParseField FLAGS_VALUE = new ParseField("flags", "flags_value");
static final ParseField MAX_DETERMINIZED_STATES = new ParseField("max_determinized_states");
private int flagsValue = RegExp.ALL;
private int maxDeterminizedStates = Operations.DEFAULT_MAX_DETERMINIZED_STATES;
@ -228,9 +243,9 @@ public class CompletionSuggestionBuilder extends SuggestBuilder.SuggestionBuilde
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject("regex");
builder.field("flags_value", flagsValue);
builder.field("max_determinized_states", maxDeterminizedStates);
builder.startObject(REGEX_OPTIONS.getPreferredName());
builder.field(FLAGS_VALUE.getPreferredName(), flagsValue);
builder.field(MAX_DETERMINIZED_STATES.getPreferredName(), maxDeterminizedStates);
builder.endObject();
return builder;
}
@ -322,7 +337,7 @@ public class CompletionSuggestionBuilder extends SuggestBuilder.SuggestionBuilde
@Override
protected XContentBuilder innerToXContent(XContentBuilder builder, Params params) throws IOException {
if (payloadFields != null) {
builder.startArray("payload");
builder.startArray(PAYLOAD_FIELD.getPreferredName());
for (String field : payloadFields) {
builder.value(field);
}
@ -335,7 +350,7 @@ public class CompletionSuggestionBuilder extends SuggestBuilder.SuggestionBuilde
regexOptionsBuilder.toXContent(builder, params);
}
if (queryContexts.isEmpty() == false) {
builder.startObject("contexts");
builder.startObject(CONTEXTS_FIELD.getPreferredName());
for (Map.Entry<String, List<ToXContent>> entry : this.queryContexts.entrySet()) {
builder.startArray(entry.getKey());
for (ToXContent queryContext : entry.getValue()) {

View File

@ -31,7 +31,7 @@ grant codeBase "${codebase.securesm-1.0.jar}" {
//// Very special jar permissions:
//// These are dangerous permissions that we don't want to grant to everything.
grant codeBase "${codebase.lucene-core-5.5.0-snapshot-1721183.jar}" {
grant codeBase "${codebase.lucene-core-5.5.0-snapshot-1725675.jar}" {
// needed to allow MMapDirectory's "unmap hack"
permission java.lang.RuntimePermission "accessClassInPackage.sun.misc";
permission java.lang.reflect.ReflectPermission "suppressAccessChecks";

View File

@ -31,7 +31,7 @@ grant codeBase "${codebase.securemock-1.2.jar}" {
permission java.lang.reflect.ReflectPermission "suppressAccessChecks";
};
grant codeBase "${codebase.lucene-test-framework-5.5.0-snapshot-1721183.jar}" {
grant codeBase "${codebase.lucene-test-framework-5.5.0-snapshot-1725675.jar}" {
// needed by RamUsageTester
permission java.lang.reflect.ReflectPermission "suppressAccessChecks";
};

View File

@ -220,9 +220,10 @@ public class TransportActionFilterChainTests extends ESTestCase {
RequestTestFilter testFilter = new RequestTestFilter(randomInt(), new RequestCallback() {
@Override
public void execute(Task task, final String action, final ActionRequest actionRequest, final ActionListener actionListener, final ActionFilterChain actionFilterChain) {
public <Request extends ActionRequest<Request>, Response extends ActionResponse> void execute(Task task, String action, Request request,
ActionListener<Response> listener, ActionFilterChain<Request, Response> actionFilterChain) {
for (int i = 0; i <= additionalContinueCount; i++) {
actionFilterChain.proceed(task, action, actionRequest, actionListener);
actionFilterChain.proceed(task, action, request, listener);
}
}
});
@ -276,7 +277,8 @@ public class TransportActionFilterChainTests extends ESTestCase {
ResponseTestFilter testFilter = new ResponseTestFilter(randomInt(), new ResponseCallback() {
@Override
public void execute(String action, ActionResponse response, ActionListener listener, ActionFilterChain chain) {
public <Response extends ActionResponse> void execute(String action, Response response, ActionListener<Response> listener,
ActionFilterChain<?, Response> chain) {
for (int i = 0; i <= additionalContinueCount; i++) {
chain.proceed(action, response, listener);
}
@ -344,17 +346,18 @@ public class TransportActionFilterChainTests extends ESTestCase {
return order;
}
@SuppressWarnings("unchecked")
@Override
public void apply(Task task, String action, ActionRequest actionRequest, ActionListener actionListener, ActionFilterChain actionFilterChain) {
public <Request extends ActionRequest<Request>, Response extends ActionResponse> void apply(Task task, String action, Request request,
ActionListener<Response> listener, ActionFilterChain<Request, Response> chain) {
this.runs.incrementAndGet();
this.lastActionName = action;
this.executionToken = counter.incrementAndGet();
this.callback.execute(task, action, actionRequest, actionListener, actionFilterChain);
this.callback.execute(task, action, request, listener, chain);
}
@Override
public void apply(String action, ActionResponse response, ActionListener listener, ActionFilterChain chain) {
public <Response extends ActionResponse> void apply(String action, Response response, ActionListener<Response> listener,
ActionFilterChain<?, Response> chain) {
chain.proceed(action, response, listener);
}
}
@ -377,12 +380,14 @@ public class TransportActionFilterChainTests extends ESTestCase {
}
@Override
public void apply(Task task, String action, ActionRequest request, ActionListener listener, ActionFilterChain chain) {
public <Request extends ActionRequest<Request>, Response extends ActionResponse> void apply(Task task, String action, Request request,
ActionListener<Response> listener, ActionFilterChain<Request, Response> chain) {
chain.proceed(task, action, request, listener);
}
@Override
public void apply(String action, ActionResponse response, ActionListener listener, ActionFilterChain chain) {
public <Response extends ActionResponse> void apply(String action, Response response, ActionListener<Response> listener,
ActionFilterChain<?, Response> chain) {
this.runs.incrementAndGet();
this.lastActionName = action;
this.executionToken = counter.incrementAndGet();
@ -393,21 +398,24 @@ public class TransportActionFilterChainTests extends ESTestCase {
private static enum RequestOperation implements RequestCallback {
CONTINUE_PROCESSING {
@Override
public void execute(Task task, String action, ActionRequest actionRequest, ActionListener actionListener, ActionFilterChain actionFilterChain) {
actionFilterChain.proceed(task, action, actionRequest, actionListener);
public <Request extends ActionRequest<Request>, Response extends ActionResponse> void execute(Task task, String action, Request request,
ActionListener<Response> listener, ActionFilterChain<Request, Response> actionFilterChain) {
actionFilterChain.proceed(task, action, request, listener);
}
},
LISTENER_RESPONSE {
@Override
@SuppressWarnings("unchecked")
public void execute(Task task, String action, ActionRequest actionRequest, ActionListener actionListener, ActionFilterChain actionFilterChain) {
actionListener.onResponse(new TestResponse());
@SuppressWarnings("unchecked") // Safe because its all we test with
public <Request extends ActionRequest<Request>, Response extends ActionResponse> void execute(Task task, String action, Request request,
ActionListener<Response> listener, ActionFilterChain<Request, Response> actionFilterChain) {
((ActionListener<TestResponse>) listener).onResponse(new TestResponse());
}
},
LISTENER_FAILURE {
@Override
public void execute(Task task, String action, ActionRequest actionRequest, ActionListener actionListener, ActionFilterChain actionFilterChain) {
actionListener.onFailure(new ElasticsearchTimeoutException(""));
public <Request extends ActionRequest<Request>, Response extends ActionResponse> void execute(Task task, String action, Request request,
ActionListener<Response> listener, ActionFilterChain<Request, Response> actionFilterChain) {
listener.onFailure(new ElasticsearchTimeoutException(""));
}
}
}
@ -415,31 +423,36 @@ public class TransportActionFilterChainTests extends ESTestCase {
private static enum ResponseOperation implements ResponseCallback {
CONTINUE_PROCESSING {
@Override
public void execute(String action, ActionResponse response, ActionListener listener, ActionFilterChain chain) {
public <Response extends ActionResponse> void execute(String action, Response response, ActionListener<Response> listener,
ActionFilterChain<?, Response> chain) {
chain.proceed(action, response, listener);
}
},
LISTENER_RESPONSE {
@Override
@SuppressWarnings("unchecked")
public void execute(String action, ActionResponse response, ActionListener listener, ActionFilterChain chain) {
listener.onResponse(new TestResponse());
@SuppressWarnings("unchecked") // Safe because its all we test with
public <Response extends ActionResponse> void execute(String action, Response response, ActionListener<Response> listener,
ActionFilterChain<?, Response> chain) {
((ActionListener<TestResponse>) listener).onResponse(new TestResponse());
}
},
LISTENER_FAILURE {
@Override
public void execute(String action, ActionResponse response, ActionListener listener, ActionFilterChain chain) {
public <Response extends ActionResponse> void execute(String action, Response response, ActionListener<Response> listener,
ActionFilterChain<?, Response> chain) {
listener.onFailure(new ElasticsearchTimeoutException(""));
}
}
}
private static interface RequestCallback {
void execute(Task task, String action, ActionRequest actionRequest, ActionListener actionListener, ActionFilterChain actionFilterChain);
<Request extends ActionRequest<Request>, Response extends ActionResponse> void execute(Task task, String action, Request request,
ActionListener<Response> listener, ActionFilterChain<Request, Response> actionFilterChain);
}
private static interface ResponseCallback {
void execute(String action, ActionResponse response, ActionListener listener, ActionFilterChain chain);
<Response extends ActionResponse> void execute(String action, Response response, ActionListener<Response> listener,
ActionFilterChain<?, Response> chain);
}
public static class TestRequest extends ActionRequest<TestRequest> {

View File

@ -20,6 +20,7 @@
package org.elasticsearch.cluster;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionModule;
@ -100,7 +101,7 @@ public class ClusterInfoServiceIT extends ESIntegTestCase {
}
@Override
protected boolean apply(String action, ActionRequest request, ActionListener listener) {
protected boolean apply(String action, ActionRequest<?> request, ActionListener<?> listener) {
if (blockedActions.contains(action)) {
throw new ElasticsearchException("force exception on [" + action + "]");
}
@ -108,7 +109,7 @@ public class ClusterInfoServiceIT extends ESIntegTestCase {
}
@Override
protected boolean apply(String action, ActionResponse response, ActionListener listener) {
protected boolean apply(String action, ActionResponse response, ActionListener<?> listener) {
return true;
}

View File

@ -162,8 +162,8 @@ public class ClusterRerouteIT extends ESIntegTestCase {
public void testDelayWithALargeAmountOfShards() throws Exception {
Settings commonSettings = settingsBuilder()
.put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_INCOMING_RECOVERIES_SETTING, 1)
.put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_OUTGOING_RECOVERIES_SETTING, 1)
.put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_INCOMING_RECOVERIES_SETTING.getKey(), 1)
.put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_OUTGOING_RECOVERIES_SETTING.getKey(), 1)
.build();
logger.info("--> starting 4 nodes");
String node_1 = internalCluster().startNode(commonSettings);

View File

@ -225,4 +225,33 @@ public class ScopedSettingsTests extends ESTestCase {
return metaData;
}
public void testKeyPattern() {
assertTrue(AbstractScopedSettings.isValidKey("a.b.c-b.d"));
assertTrue(AbstractScopedSettings.isValidKey("a.b.c.d"));
assertTrue(AbstractScopedSettings.isValidKey("a.b_012.c_b.d"));
assertTrue(AbstractScopedSettings.isValidKey("a"));
assertFalse(AbstractScopedSettings.isValidKey("a b"));
assertFalse(AbstractScopedSettings.isValidKey(""));
assertFalse(AbstractScopedSettings.isValidKey("\""));
try {
new IndexScopedSettings(
Settings.EMPTY, Collections.singleton(Setting.groupSetting("boo .", false, Setting.Scope.INDEX)));
fail();
} catch (IllegalArgumentException e) {
assertEquals("illegal settings key: [boo .]", e.getMessage());
}
new IndexScopedSettings(
Settings.EMPTY, Collections.singleton(Setting.groupSetting("boo.", false, Setting.Scope.INDEX)));
try {
new IndexScopedSettings(
Settings.EMPTY, Collections.singleton(Setting.boolSetting("boo.", true, false, Setting.Scope.INDEX)));
fail();
} catch (IllegalArgumentException e) {
assertEquals("illegal settings key: [boo.]", e.getMessage());
}
new IndexScopedSettings(
Settings.EMPTY, Collections.singleton(Setting.boolSetting("boo", true, false, Setting.Scope.INDEX)));
}
}

View File

@ -35,15 +35,15 @@ import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.test.ESAllocationTestCase;
import org.junit.Before;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import static org.hamcrest.Matchers.anyOf;
@ -104,7 +104,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
} else {
allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, Version.V_2_1_0);
}
testAllocator.addData(node1, -1, null);
testAllocator.addData(node1, -1, null, randomBoolean());
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(false));
assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1));
@ -116,7 +116,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
*/
public void testNoMatchingAllocationIdFound() {
RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, Version.CURRENT, "id2");
testAllocator.addData(node1, 1, "id1");
testAllocator.addData(node1, 1, "id1", randomBoolean());
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(false));
assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1));
@ -129,7 +129,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
*/
public void testNoActiveAllocationIds() {
RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, Version.V_2_1_1);
testAllocator.addData(node1, 1, null);
testAllocator.addData(node1, 1, null, randomBoolean());
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(true));
assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true));
@ -144,10 +144,10 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
final RoutingAllocation allocation;
if (randomBoolean()) {
allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, randomFrom(Version.V_2_0_0, Version.CURRENT), "allocId1");
testAllocator.addData(node1, 1, "allocId1", new CorruptIndexException("test", "test"));
testAllocator.addData(node1, 1, "allocId1", randomBoolean(), new CorruptIndexException("test", "test"));
} else {
allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, Version.V_2_1_1);
testAllocator.addData(node1, 3, null, new CorruptIndexException("test", "test"));
testAllocator.addData(node1, 3, null, randomBoolean(), new CorruptIndexException("test", "test"));
}
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(false));
@ -162,10 +162,10 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
final RoutingAllocation allocation;
if (randomBoolean()) {
allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, randomFrom(Version.V_2_0_0, Version.CURRENT), "allocId1");
testAllocator.addData(node1, 1, "allocId1");
testAllocator.addData(node1, 1, "allocId1", randomBoolean());
} else {
allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, Version.V_2_2_0);
testAllocator.addData(node1, 3, null);
testAllocator.addData(node1, 3, null, randomBoolean());
}
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(true));
@ -174,6 +174,24 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), equalTo(node1.id()));
}
/**
* Tests that when there was a node that previously had the primary, it will be allocated to that same node again.
*/
public void testPreferAllocatingPreviousPrimary() {
String primaryAllocId = Strings.randomBase64UUID();
String replicaAllocId = Strings.randomBase64UUID();
RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, randomFrom(Version.V_2_0_0, Version.CURRENT), primaryAllocId, replicaAllocId);
boolean node1HasPrimaryShard = randomBoolean();
testAllocator.addData(node1, 1, node1HasPrimaryShard ? primaryAllocId : replicaAllocId, node1HasPrimaryShard);
testAllocator.addData(node2, 1, node1HasPrimaryShard ? replicaAllocId : primaryAllocId, !node1HasPrimaryShard);
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(true));
assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true));
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1));
DiscoveryNode allocatedNode = node1HasPrimaryShard ? node1 : node2;
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), equalTo(allocatedNode.id()));
}
/**
* Tests that when there is a node to allocate to, but it is throttling (and it is the only one),
* it will be moved to ignore unassigned until it can be allocated to.
@ -182,10 +200,10 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
final RoutingAllocation allocation;
if (randomBoolean()) {
allocation = routingAllocationWithOnePrimaryNoReplicas(throttleAllocationDeciders(), false, randomFrom(Version.V_2_0_0, Version.CURRENT), "allocId1");
testAllocator.addData(node1, 1, "allocId1");
testAllocator.addData(node1, 1, "allocId1", randomBoolean());
} else {
allocation = routingAllocationWithOnePrimaryNoReplicas(throttleAllocationDeciders(), false, Version.V_2_2_0);
testAllocator.addData(node1, 3, null);
testAllocator.addData(node1, 3, null, randomBoolean());
}
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(false));
@ -201,10 +219,10 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
final RoutingAllocation allocation;
if (randomBoolean()) {
allocation = routingAllocationWithOnePrimaryNoReplicas(noAllocationDeciders(), false, randomFrom(Version.V_2_0_0, Version.CURRENT), "allocId1");
testAllocator.addData(node1, 1, "allocId1");
testAllocator.addData(node1, 1, "allocId1", randomBoolean());
} else {
allocation = routingAllocationWithOnePrimaryNoReplicas(noAllocationDeciders(), false, Version.V_2_0_0);
testAllocator.addData(node1, 3, null);
testAllocator.addData(node1, 3, null, randomBoolean());
}
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(true));
@ -218,7 +236,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
*/
public void testAllocateToTheHighestVersionOnLegacyIndex() {
RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, Version.V_2_0_0);
testAllocator.addData(node1, 10, null).addData(node2, 12, null);
testAllocator.addData(node1, 10, null, randomBoolean()).addData(node2, 12, null, randomBoolean());
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(true));
assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true));
@ -232,7 +250,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
*/
public void testRestore() {
RoutingAllocation allocation = getRestoreRoutingAllocation(yesAllocationDeciders());
testAllocator.addData(node1, 1, randomFrom(null, "allocId"));
testAllocator.addData(node1, 1, randomFrom(null, "allocId"), randomBoolean());
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(true));
assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true));
@ -245,7 +263,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
*/
public void testRestoreThrottle() {
RoutingAllocation allocation = getRestoreRoutingAllocation(throttleAllocationDeciders());
testAllocator.addData(node1, 1, randomFrom(null, "allocId"));
testAllocator.addData(node1, 1, randomFrom(null, "allocId"), randomBoolean());
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(false));
assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(false));
@ -257,7 +275,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
*/
public void testRestoreForcesAllocateIfShardAvailable() {
RoutingAllocation allocation = getRestoreRoutingAllocation(noAllocationDeciders());
testAllocator.addData(node1, 1, randomFrom(null, "some allocId"));
testAllocator.addData(node1, 1, randomFrom(null, "some allocId"), randomBoolean());
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(true));
assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true));
@ -270,7 +288,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
*/
public void testRestoreDoesNotAssignIfNoShardAvailable() {
RoutingAllocation allocation = getRestoreRoutingAllocation(yesAllocationDeciders());
testAllocator.addData(node1, -1, null);
testAllocator.addData(node1, -1, null, false);
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(false));
assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true));
@ -281,7 +299,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
Version version = randomFrom(Version.CURRENT, Version.V_2_0_0);
MetaData metaData = MetaData.builder()
.put(IndexMetaData.builder(shardId.getIndex()).settings(settings(version)).numberOfShards(1).numberOfReplicas(0)
.putActiveAllocationIds(0, version == Version.CURRENT ? new HashSet<>(Arrays.asList("allocId")) : Collections.emptySet()))
.putActiveAllocationIds(0, version == Version.CURRENT ? Sets.newHashSet("allocId") : Collections.emptySet()))
.build();
RoutingTable routingTable = RoutingTable.builder()
@ -300,7 +318,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
*/
public void testRecoverOnAnyNode() {
RoutingAllocation allocation = getRecoverOnAnyNodeRoutingAllocation(yesAllocationDeciders());
testAllocator.addData(node1, 1, randomFrom(null, "allocId"));
testAllocator.addData(node1, 1, randomFrom(null, "allocId"), randomBoolean());
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(true));
assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true));
@ -313,7 +331,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
*/
public void testRecoverOnAnyNodeThrottle() {
RoutingAllocation allocation = getRestoreRoutingAllocation(throttleAllocationDeciders());
testAllocator.addData(node1, 1, randomFrom(null, "allocId"));
testAllocator.addData(node1, 1, randomFrom(null, "allocId"), randomBoolean());
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(false));
assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(false));
@ -325,7 +343,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
*/
public void testRecoverOnAnyNodeForcesAllocateIfShardAvailable() {
RoutingAllocation allocation = getRecoverOnAnyNodeRoutingAllocation(noAllocationDeciders());
testAllocator.addData(node1, 1, randomFrom(null, "allocId"));
testAllocator.addData(node1, 1, randomFrom(null, "allocId"), randomBoolean());
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(true));
assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true));
@ -338,7 +356,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
*/
public void testRecoverOnAnyNodeDoesNotAssignIfNoShardAvailable() {
RoutingAllocation allocation = getRecoverOnAnyNodeRoutingAllocation(yesAllocationDeciders());
testAllocator.addData(node1, -1, null);
testAllocator.addData(node1, -1, null, randomBoolean());
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(false));
assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true));
@ -351,7 +369,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
.put(IndexMetaData.builder(shardId.getIndex()).settings(settings(version)
.put(IndexMetaData.SETTING_SHARED_FILESYSTEM, true)
.put(IndexMetaData.SETTING_SHARED_FS_ALLOW_RECOVERY_ON_ANY_NODE, true))
.numberOfShards(1).numberOfReplicas(0).putActiveAllocationIds(0, version == Version.CURRENT ? new HashSet<>(Arrays.asList("allocId")) : Collections.emptySet()))
.numberOfShards(1).numberOfReplicas(0).putActiveAllocationIds(0, version == Version.CURRENT ? Sets.newHashSet("allocId") : Collections.emptySet()))
.build();
RoutingTable routingTable = RoutingTable.builder()
@ -387,7 +405,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId));
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(2)); // replicas
testAllocator.addData(node1, 1, null);
testAllocator.addData(node1, 1, null, randomBoolean());
allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state.nodes(), null, System.nanoTime());
changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(false));
@ -395,7 +413,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId));
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(2)); // replicas
testAllocator.addData(node2, 1, null);
testAllocator.addData(node2, 1, null, randomBoolean());
allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state.nodes(), null, System.nanoTime());
changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(true));
@ -428,7 +446,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId));
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(2)); // replicas
testAllocator.addData(node1, 1, null);
testAllocator.addData(node1, 1, null, randomBoolean());
allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state.nodes(), null, System.nanoTime());
changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(false));
@ -436,7 +454,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId));
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(2)); // replicas
testAllocator.addData(node2, 2, null);
testAllocator.addData(node2, 2, null, randomBoolean());
allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state.nodes(), null, System.nanoTime());
changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(true));
@ -449,7 +467,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
private RoutingAllocation routingAllocationWithOnePrimaryNoReplicas(AllocationDeciders deciders, boolean asNew, Version version, String... activeAllocationIds) {
MetaData metaData = MetaData.builder()
.put(IndexMetaData.builder(shardId.getIndex()).settings(settings(version))
.numberOfShards(1).numberOfReplicas(0).putActiveAllocationIds(0, new HashSet<>(Arrays.asList(activeAllocationIds))))
.numberOfShards(1).numberOfReplicas(0).putActiveAllocationIds(0, Sets.newHashSet(activeAllocationIds)))
.build();
RoutingTable.Builder routingTableBuilder = RoutingTable.builder();
if (asNew) {
@ -477,15 +495,15 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
return this;
}
public TestAllocator addData(DiscoveryNode node, long version, String allocationId) {
return addData(node, version, allocationId, null);
public TestAllocator addData(DiscoveryNode node, long version, String allocationId, boolean primary) {
return addData(node, version, allocationId, primary, null);
}
public TestAllocator addData(DiscoveryNode node, long version, String allocationId, @Nullable Throwable storeException) {
public TestAllocator addData(DiscoveryNode node, long version, String allocationId, boolean primary, @Nullable Throwable storeException) {
if (data == null) {
data = new HashMap<>();
}
data.put(node, new TransportNodesListGatewayStartedShards.NodeGatewayStartedShards(node, version, allocationId, storeException));
data.put(node, new TransportNodesListGatewayStartedShards.NodeGatewayStartedShards(node, version, allocationId, primary, storeException));
return this;
}

View File

@ -331,10 +331,10 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase {
public void testReusePeerRecovery() throws Exception {
final Settings settings = settingsBuilder()
.put("action.admin.cluster.node.shutdown.delay", "10ms")
.put(MockFSIndexStore.INDEX_CHECK_INDEX_ON_CLOSE_SETTING, false)
.put(MockFSIndexStore.INDEX_CHECK_INDEX_ON_CLOSE_SETTING.getKey(), false)
.put("gateway.recover_after_nodes", 4)
.put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_INCOMING_RECOVERIES_SETTING, 4)
.put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_OUTGOING_RECOVERIES_SETTING, 4)
.put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_INCOMING_RECOVERIES_SETTING.getKey(), 4)
.put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_OUTGOING_RECOVERIES_SETTING.getKey(), 4)
.put(MockFSDirectoryService.CRASH_INDEX_SETTING.getKey(), false).build();
internalCluster().startNodesAsync(4, settings).get();

View File

@ -43,6 +43,7 @@ import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.store.Store;
@ -51,11 +52,9 @@ import org.elasticsearch.indices.store.TransportNodesListShardStoreMetaData;
import org.elasticsearch.test.ESAllocationTestCase;
import org.junit.Before;
import java.util.Arrays;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
@ -289,7 +288,7 @@ public class ReplicaShardAllocatorTests extends ESAllocationTestCase {
MetaData metaData = MetaData.builder()
.put(IndexMetaData.builder(shardId.getIndex()).settings(settings(Version.CURRENT).put(settings))
.numberOfShards(1).numberOfReplicas(1)
.putActiveAllocationIds(0, new HashSet<>(Arrays.asList(primaryShard.allocationId().getId()))))
.putActiveAllocationIds(0, Sets.newHashSet(primaryShard.allocationId().getId())))
.build();
RoutingTable routingTable = RoutingTable.builder()
.add(IndexRoutingTable.builder(shardId.getIndex())
@ -311,7 +310,7 @@ public class ReplicaShardAllocatorTests extends ESAllocationTestCase {
MetaData metaData = MetaData.builder()
.put(IndexMetaData.builder(shardId.getIndex()).settings(settings(Version.CURRENT))
.numberOfShards(1).numberOfReplicas(1)
.putActiveAllocationIds(0, new HashSet<>(Arrays.asList(primaryShard.allocationId().getId()))))
.putActiveAllocationIds(0, Sets.newHashSet(primaryShard.allocationId().getId())))
.build();
RoutingTable routingTable = RoutingTable.builder()
.add(IndexRoutingTable.builder(shardId.getIndex())

View File

@ -179,6 +179,8 @@ public class AnalysisFactoryTests extends ESTestCase {
put("typeaspayload", Void.class);
// fingerprint
put("fingerprint", Void.class);
// for tee-sinks
put("daterecognizer", Void.class);
}};
public void testTokenFilters() {

View File

@ -164,11 +164,11 @@ public class ParentChildFieldDataTests extends AbstractFieldDataTestCase {
}
public void testSorting() throws Exception {
IndexFieldData indexFieldData = getForField(childType);
IndexFieldData indexFieldData = getForField(parentType);
IndexSearcher searcher = new IndexSearcher(DirectoryReader.open(writer, true));
IndexFieldData.XFieldComparatorSource comparator = indexFieldData.comparatorSource("_last", MultiValueMode.MIN, null);
TopFieldDocs topDocs = searcher.search(new MatchAllDocsQuery(), 10, new Sort(new SortField(ParentFieldMapper.NAME, comparator, false)));
TopFieldDocs topDocs = searcher.search(new MatchAllDocsQuery(), 10, new Sort(new SortField(ParentFieldMapper.joinField(parentType), comparator, false)));
assertThat(topDocs.totalHits, equalTo(8));
assertThat(topDocs.scoreDocs.length, equalTo(8));
assertThat(topDocs.scoreDocs[0].doc, equalTo(0));
@ -188,7 +188,7 @@ public class ParentChildFieldDataTests extends AbstractFieldDataTestCase {
assertThat(topDocs.scoreDocs[7].doc, equalTo(7));
assertThat(((BytesRef) ((FieldDoc) topDocs.scoreDocs[7]).fields[0]), equalTo(null));
topDocs = searcher.search(new MatchAllDocsQuery(), 10, new Sort(new SortField(ParentFieldMapper.NAME, comparator, true)));
topDocs = searcher.search(new MatchAllDocsQuery(), 10, new Sort(new SortField(ParentFieldMapper.joinField(parentType), comparator, true)));
assertThat(topDocs.totalHits, equalTo(8));
assertThat(topDocs.scoreDocs.length, equalTo(8));
assertThat(topDocs.scoreDocs[0].doc, equalTo(3));

View File

@ -20,11 +20,13 @@
package org.elasticsearch.index.mapper;
import org.apache.lucene.index.Term;
import org.apache.lucene.queries.TermsQuery;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.ConstantScoreQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.common.lucene.search.Queries;
@ -40,6 +42,7 @@ import java.util.HashSet;
import java.util.concurrent.ExecutionException;
import static org.hamcrest.CoreMatchers.containsString;
import static org.hamcrest.CoreMatchers.nullValue;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.hasToString;
@ -103,7 +106,7 @@ public class MapperServiceTests extends ESSingleNodeTestCase {
fail();
} catch (Throwable t) {
if (t instanceof ExecutionException) {
t = ((ExecutionException) t).getCause();
t = t.getCause();
}
final Throwable throwable = ExceptionsHelper.unwrapCause(t);
if (throwable instanceof IllegalArgumentException) {
@ -120,7 +123,7 @@ public class MapperServiceTests extends ESSingleNodeTestCase {
fail();
} catch (Throwable t) {
if (t instanceof ExecutionException) {
t = ((ExecutionException) t).getCause();
t = t.getCause();
}
final Throwable throwable = ExceptionsHelper.unwrapCause(t);
if (throwable instanceof IllegalArgumentException) {
@ -132,21 +135,4 @@ public class MapperServiceTests extends ESSingleNodeTestCase {
assertFalse(indexService.mapperService().hasMapping(MapperService.DEFAULT_MAPPING));
}
public void testSearchFilter() {
IndexService indexService = createIndex("index1", client().admin().indices().prepareCreate("index1")
.addMapping("type1", "field1", "type=nested")
.addMapping("type2", new Object[0])
);
Query searchFilter = indexService.mapperService().searchFilter("type1", "type3");
Query expectedQuery = new BooleanQuery.Builder()
.add(new BooleanQuery.Builder()
.add(new ConstantScoreQuery(new TermQuery(new Term(TypeFieldMapper.NAME, "type1"))), BooleanClause.Occur.SHOULD)
.add(new TermQuery(new Term(TypeFieldMapper.NAME, "type3")), BooleanClause.Occur.SHOULD)
.build(), BooleanClause.Occur.MUST
)
.add(Queries.newNonNestedFilter(), BooleanClause.Occur.MUST)
.build();
assertThat(searchFilter, equalTo(new ConstantScoreQuery(expectedQuery)));
}
}

View File

@ -47,10 +47,10 @@ public class ParentFieldMapperTests extends ESTestCase {
assertThat(parentFieldMapper.getParentJoinFieldType().hasDocValues(), is(true));
assertThat(parentFieldMapper.getParentJoinFieldType().docValuesType(), equalTo(DocValuesType.SORTED));
assertThat(parentFieldMapper.getChildJoinFieldType().name(), equalTo("_parent#parent"));
assertThat(parentFieldMapper.getChildJoinFieldType().fieldDataType().getLoading(), equalTo(Loading.LAZY));
assertThat(parentFieldMapper.getChildJoinFieldType().hasDocValues(), is(true));
assertThat(parentFieldMapper.getChildJoinFieldType().docValuesType(), equalTo(DocValuesType.SORTED));
assertThat(parentFieldMapper.fieldType().name(), equalTo("_parent#parent"));
assertThat(parentFieldMapper.fieldType().fieldDataType().getLoading(), equalTo(Loading.LAZY));
assertThat(parentFieldMapper.fieldType().hasDocValues(), is(true));
assertThat(parentFieldMapper.fieldType().docValuesType(), equalTo(DocValuesType.SORTED));
}
public void testPost2Dot0EagerLoading() {
@ -65,10 +65,10 @@ public class ParentFieldMapperTests extends ESTestCase {
assertThat(parentFieldMapper.getParentJoinFieldType().hasDocValues(), is(true));
assertThat(parentFieldMapper.getParentJoinFieldType().docValuesType(), equalTo(DocValuesType.SORTED));
assertThat(parentFieldMapper.getChildJoinFieldType().name(), equalTo("_parent#parent"));
assertThat(parentFieldMapper.getChildJoinFieldType().fieldDataType().getLoading(), equalTo(Loading.EAGER));
assertThat(parentFieldMapper.getChildJoinFieldType().hasDocValues(), is(true));
assertThat(parentFieldMapper.getChildJoinFieldType().docValuesType(), equalTo(DocValuesType.SORTED));
assertThat(parentFieldMapper.fieldType().name(), equalTo("_parent#parent"));
assertThat(parentFieldMapper.fieldType().fieldDataType().getLoading(), equalTo(Loading.EAGER));
assertThat(parentFieldMapper.fieldType().hasDocValues(), is(true));
assertThat(parentFieldMapper.fieldType().docValuesType(), equalTo(DocValuesType.SORTED));
}
public void testPost2Dot0EagerGlobalOrdinalsLoading() {
@ -83,10 +83,10 @@ public class ParentFieldMapperTests extends ESTestCase {
assertThat(parentFieldMapper.getParentJoinFieldType().hasDocValues(), is(true));
assertThat(parentFieldMapper.getParentJoinFieldType().docValuesType(), equalTo(DocValuesType.SORTED));
assertThat(parentFieldMapper.getChildJoinFieldType().name(), equalTo("_parent#parent"));
assertThat(parentFieldMapper.getChildJoinFieldType().fieldDataType().getLoading(), equalTo(Loading.EAGER_GLOBAL_ORDINALS));
assertThat(parentFieldMapper.getChildJoinFieldType().hasDocValues(), is(true));
assertThat(parentFieldMapper.getChildJoinFieldType().docValuesType(), equalTo(DocValuesType.SORTED));
assertThat(parentFieldMapper.fieldType().name(), equalTo("_parent#parent"));
assertThat(parentFieldMapper.fieldType().fieldDataType().getLoading(), equalTo(Loading.EAGER_GLOBAL_ORDINALS));
assertThat(parentFieldMapper.fieldType().hasDocValues(), is(true));
assertThat(parentFieldMapper.fieldType().docValuesType(), equalTo(DocValuesType.SORTED));
}
private static Settings post2Dot0IndexSettings() {

View File

@ -55,6 +55,6 @@ public class ParentMappingTests extends ESSingleNodeTestCase {
.endObject()
.bytes()).type("type").id("1").parent("1122"));
assertEquals(Uid.createUid("p_type", "1122"), doc.rootDoc().get("_parent"));
assertEquals("1122", doc.rootDoc().getBinaryValue("_parent#p_type").utf8ToString());
}
}

View File

@ -0,0 +1,127 @@
/*
* 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.query;
import org.apache.lucene.search.DocValuesTermsQuery;
import org.apache.lucene.search.Query;
import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest;
import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.index.fielddata.IndexFieldDataService;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.search.fetch.innerhits.InnerHitsContext;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.test.TestSearchContext;
import org.hamcrest.Matchers;
import java.io.IOException;
public class ParentIdQueryBuilderTests extends AbstractQueryTestCase<ParentIdQueryBuilder> {
protected static final String PARENT_TYPE = "parent";
protected static final String CHILD_TYPE = "child";
@Override
public void setUp() throws Exception {
super.setUp();
MapperService mapperService = queryShardContext().getMapperService();
mapperService.merge(PARENT_TYPE, new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef(PARENT_TYPE,
STRING_FIELD_NAME, "type=string",
INT_FIELD_NAME, "type=integer",
DOUBLE_FIELD_NAME, "type=double",
BOOLEAN_FIELD_NAME, "type=boolean",
DATE_FIELD_NAME, "type=date",
OBJECT_FIELD_NAME, "type=object"
).string()), MapperService.MergeReason.MAPPING_UPDATE, false);
mapperService.merge(CHILD_TYPE, new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef(CHILD_TYPE,
"_parent", "type=" + PARENT_TYPE,
STRING_FIELD_NAME, "type=string",
INT_FIELD_NAME, "type=integer",
DOUBLE_FIELD_NAME, "type=double",
BOOLEAN_FIELD_NAME, "type=boolean",
DATE_FIELD_NAME, "type=date",
OBJECT_FIELD_NAME, "type=object"
).string()), MapperService.MergeReason.MAPPING_UPDATE, false);
}
@Override
protected void setSearchContext(String[] types) {
final MapperService mapperService = queryShardContext().getMapperService();
final IndexFieldDataService fieldData = indexFieldDataService();
TestSearchContext testSearchContext = new TestSearchContext() {
private InnerHitsContext context;
@Override
public void innerHits(InnerHitsContext innerHitsContext) {
context = innerHitsContext;
}
@Override
public InnerHitsContext innerHits() {
return context;
}
@Override
public MapperService mapperService() {
return mapperService; // need to build / parse inner hits sort fields
}
@Override
public IndexFieldDataService fieldData() {
return fieldData; // need to build / parse inner hits sort fields
}
};
testSearchContext.setTypes(types);
SearchContext.setCurrent(testSearchContext);
}
@Override
protected ParentIdQueryBuilder doCreateTestQueryBuilder() {
return new ParentIdQueryBuilder(CHILD_TYPE, randomAsciiOfLength(4));
}
@Override
protected void doAssertLuceneQuery(ParentIdQueryBuilder queryBuilder, Query query, QueryShardContext context) throws IOException {
assertThat(query, Matchers.instanceOf(DocValuesTermsQuery.class));
DocValuesTermsQuery termsQuery = (DocValuesTermsQuery) query;
// there are no getters to get the field and terms on DocValuesTermsQuery, so lets validate by creating a
// new query based on the builder:
assertThat(termsQuery, Matchers.equalTo(new DocValuesTermsQuery("_parent#" + PARENT_TYPE, queryBuilder.getId())));
}
public void testFromJson() throws IOException {
String query =
"{\n" +
" \"parent_id\" : {\n" +
" \"type\" : \"child\",\n" +
" \"id\" : \"123\",\n" +
" \"boost\" : 3.0,\n" +
" \"_name\" : \"name\"" +
" }\n" +
"}";
ParentIdQueryBuilder queryBuilder = (ParentIdQueryBuilder) parseQuery(query);
checkGeneratedJson(query, queryBuilder);
assertThat(queryBuilder.getType(), Matchers.equalTo("child"));
assertThat(queryBuilder.getId(), Matchers.equalTo("123"));
assertThat(queryBuilder.boost(), Matchers.equalTo(3f));
assertThat(queryBuilder.queryName(), Matchers.equalTo("name"));
}
}

View File

@ -20,6 +20,7 @@
package org.elasticsearch.index.similarity;
import org.apache.lucene.search.similarities.ClassicSimilarity;
import org.apache.lucene.search.similarities.DFISimilarity;
import org.apache.lucene.search.similarities.AfterEffectL;
import org.apache.lucene.search.similarities.BM25Similarity;
import org.apache.lucene.search.similarities.BasicModelG;
@ -38,6 +39,7 @@ import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.DocumentMapperParser;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.ESSingleNodeTestCase;
@ -93,7 +95,7 @@ public class SimilarityTests extends ESSingleNodeTestCase {
Settings indexSettings = Settings.settingsBuilder()
.put("index.similarity.my_similarity.type", "BM25")
.put("index.similarity.my_similarity.k1", 2.0f)
.put("index.similarity.my_similarity.b", 1.5f)
.put("index.similarity.my_similarity.b", 0.5f)
.put("index.similarity.my_similarity.discount_overlaps", false)
.build();
IndexService indexService = createIndex("foo", indexSettings);
@ -102,7 +104,7 @@ public class SimilarityTests extends ESSingleNodeTestCase {
BM25Similarity similarity = (BM25Similarity) documentMapper.mappers().getMapper("field1").fieldType().similarity().get();
assertThat(similarity.getK1(), equalTo(2.0f));
assertThat(similarity.getB(), equalTo(1.5f));
assertThat(similarity.getB(), equalTo(0.5f));
assertThat(similarity.getDiscountOverlaps(), equalTo(false));
}
@ -156,6 +158,23 @@ public class SimilarityTests extends ESSingleNodeTestCase {
assertThat(((NormalizationH2) similarity.getNormalization()).getC(), equalTo(3f));
}
public void testResolveSimilaritiesFromMapping_DFI() throws IOException {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties")
.startObject("field1").field("type", "string").field("similarity", "my_similarity").endObject()
.endObject()
.endObject().endObject().string();
Settings indexSettings = Settings.settingsBuilder()
.put("index.similarity.my_similarity.type", "DFI")
.build();
IndexService indexService = createIndex("foo", indexSettings);
DocumentMapper documentMapper = indexService.mapperService().documentMapperParser().parse("type", new CompressedXContent(mapping));
MappedFieldType fieldType = documentMapper.mappers().getMapper("field1").fieldType();
assertThat(fieldType.similarity(), instanceOf(DFISimilarityProvider.class));
assertThat(fieldType.similarity().get(), instanceOf(DFISimilarity.class));
}
public void testResolveSimilaritiesFromMapping_LMDirichlet() throws IOException {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties")

View File

@ -25,6 +25,7 @@ import org.apache.lucene.store.MMapDirectory;
import org.apache.lucene.store.NIOFSDirectory;
import org.apache.lucene.store.NoLockFactory;
import org.apache.lucene.store.SimpleFSDirectory;
import org.apache.lucene.store.StoreRateLimiting;
import org.apache.lucene.util.Constants;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.IndexMetaData;
@ -95,4 +96,24 @@ public class IndexStoreTests extends ESTestCase {
}
}
}
public void testUpdateThrottleType() throws IOException {
Settings settings = Settings.settingsBuilder().put(IndexStoreConfig.INDICES_STORE_THROTTLE_TYPE_SETTING.getKey(), "all")
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(new Index("foo"), settings);
IndexStoreConfig indexStoreConfig = new IndexStoreConfig(settings);
IndexStore store = new IndexStore(indexSettings, indexStoreConfig);
assertEquals(StoreRateLimiting.Type.NONE, store.rateLimiting().getType());
assertEquals(StoreRateLimiting.Type.ALL, indexStoreConfig.getNodeRateLimiter().getType());
assertNotSame(indexStoreConfig.getNodeRateLimiter(), store.rateLimiting());
store.setType(IndexStore.IndexRateLimitingType.fromString("NODE"));
assertEquals(StoreRateLimiting.Type.ALL, store.rateLimiting().getType());
assertSame(indexStoreConfig.getNodeRateLimiter(), store.rateLimiting());
store.setType(IndexStore.IndexRateLimitingType.fromString("merge"));
assertEquals(StoreRateLimiting.Type.MERGE, store.rateLimiting().getType());
assertNotSame(indexStoreConfig.getNodeRateLimiter(), store.rateLimiting());
assertEquals(StoreRateLimiting.Type.ALL, indexStoreConfig.getNodeRateLimiter().getType());
}
}

View File

@ -163,12 +163,6 @@ public class PercolatorIT extends ESIntegTestCase {
assertThat(response.getMatches(), arrayWithSize(1));
assertThat(convertFromTextArray(response.getMatches(), "test"), arrayContaining("4"));
logger.info("--> Search dummy doc, percolate queries must not be included");
SearchResponse searchResponse = client().prepareSearch("test", "test").execute().actionGet();
assertThat(searchResponse.getHits().totalHits(), equalTo(1L));
assertThat(searchResponse.getHits().getAt(0).type(), equalTo("type"));
assertThat(searchResponse.getHits().getAt(0).id(), equalTo("1"));
logger.info("--> Percolate non existing doc");
try {
client().preparePercolate()
@ -657,14 +651,6 @@ public class PercolatorIT extends ESIntegTestCase {
assertMatchCount(response, 1l);
assertThat(response.getMatches(), arrayWithSize(1));
assertThat(convertFromTextArray(response.getMatches(), "test"), arrayContaining("4"));
logger.info("--> Search normals docs, percolate queries must not be included");
SearchResponse searchResponse = client().prepareSearch("test").execute().actionGet();
assertThat(searchResponse.getHits().totalHits(), equalTo(4L));
assertThat(searchResponse.getHits().getAt(0).type(), equalTo("type"));
assertThat(searchResponse.getHits().getAt(1).type(), equalTo("type"));
assertThat(searchResponse.getHits().getAt(2).type(), equalTo("type"));
assertThat(searchResponse.getHits().getAt(3).type(), equalTo("type"));
}
public void testPercolatingExistingDocs_routing() throws Exception {

View File

@ -47,11 +47,11 @@ public class ParentIdAggIT extends ESIntegTestCase {
refresh();
ensureGreen("testidx");
SearchResponse searchResponse = client().prepareSearch("testidx").setTypes("childtype").setQuery(matchAllQuery()).addAggregation(AggregationBuilders.terms("children").field("_parent")).get();
SearchResponse searchResponse = client().prepareSearch("testidx").setTypes("childtype").setQuery(matchAllQuery()).addAggregation(AggregationBuilders.terms("children").field("_parent#parenttype")).get();
assertThat(searchResponse.getHits().getTotalHits(), equalTo(2l));
assertSearchResponse(searchResponse);
assertThat(searchResponse.getAggregations().getAsMap().get("children"), instanceOf(Terms.class));
Terms terms = (Terms) searchResponse.getAggregations().getAsMap().get("children");
assertThat(terms.getBuckets().iterator().next().getDocCount(), equalTo(2l));
}
}
}

View File

@ -67,6 +67,7 @@ import static org.elasticsearch.index.query.QueryBuilders.idsQuery;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.index.query.QueryBuilders.matchQuery;
import static org.elasticsearch.index.query.QueryBuilders.multiMatchQuery;
import static org.elasticsearch.index.query.QueryBuilders.parentId;
import static org.elasticsearch.index.query.QueryBuilders.prefixQuery;
import static org.elasticsearch.index.query.QueryBuilders.queryStringQuery;
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
@ -208,7 +209,7 @@ public class ChildQuerySearchIT extends ESIntegTestCase {
assertThat(searchResponse.getHits().getAt(0).field("_parent").value().toString(), equalTo("p1"));
// TEST matching on parent
searchResponse = client().prepareSearch("test").setQuery(termQuery("_parent", "p1")).fields("_parent").get();
searchResponse = client().prepareSearch("test").setQuery(termQuery("_parent#parent", "p1")).fields("_parent").get();
assertNoFailures(searchResponse);
assertThat(searchResponse.getHits().totalHits(), equalTo(2l));
assertThat(searchResponse.getHits().getAt(0).id(), anyOf(equalTo("c1"), equalTo("c2")));
@ -216,7 +217,7 @@ public class ChildQuerySearchIT extends ESIntegTestCase {
assertThat(searchResponse.getHits().getAt(1).id(), anyOf(equalTo("c1"), equalTo("c2")));
assertThat(searchResponse.getHits().getAt(1).field("_parent").value().toString(), equalTo("p1"));
searchResponse = client().prepareSearch("test").setQuery(queryStringQuery("_parent:p1")).fields("_parent").get();
searchResponse = client().prepareSearch("test").setQuery(queryStringQuery("_parent#parent:p1")).fields("_parent").get();
assertNoFailures(searchResponse);
assertThat(searchResponse.getHits().totalHits(), equalTo(2l));
assertThat(searchResponse.getHits().getAt(0).id(), anyOf(equalTo("c1"), equalTo("c2")));
@ -953,70 +954,63 @@ public class ChildQuerySearchIT extends ESIntegTestCase {
assertThat(searchResponse.getHits().getAt(0).score(), equalTo(3.0f));
}
public void testParentFieldFilter() throws Exception {
public void testParentFieldQuery() throws Exception {
assertAcked(prepareCreate("test")
.setSettings(settingsBuilder().put(indexSettings())
.put("index.refresh_interval", -1))
.addMapping("parent")
.addMapping("child", "_parent", "type=parent")
.addMapping("child2", "_parent", "type=parent"));
.addMapping("child", "_parent", "type=parent"));
ensureGreen();
// test term filter
SearchResponse response = client().prepareSearch("test").setQuery(boolQuery().must(matchAllQuery()).filter(termQuery("_parent", "p1")))
SearchResponse response = client().prepareSearch("test").setQuery(termQuery("_parent", "p1"))
.get();
assertHitCount(response, 0l);
assertHitCount(response, 0L);
client().prepareIndex("test", "some_type", "1").setSource("field", "value").get();
client().prepareIndex("test", "parent", "p1").setSource("p_field", "value").get();
client().prepareIndex("test", "child", "c1").setSource("c_field", "value").setParent("p1").get();
response = client().prepareSearch("test").setQuery(boolQuery().must(matchAllQuery()).filter(termQuery("_parent", "p1"))).execute()
.actionGet();
assertHitCount(response, 0l);
client().prepareIndex("test", "child", "c1").setSource("{}").setParent("p1").get();
refresh();
response = client().prepareSearch("test").setQuery(boolQuery().must(matchAllQuery()).filter(termQuery("_parent", "p1"))).execute()
.actionGet();
assertHitCount(response, 1l);
response = client().prepareSearch("test").setQuery(termQuery("_parent#parent", "p1")).get();
assertHitCount(response, 1L);
response = client().prepareSearch("test").setQuery(boolQuery().must(matchAllQuery()).filter(termQuery("_parent", "parent#p1"))).execute()
.actionGet();
assertHitCount(response, 1l);
client().prepareIndex("test", "parent2", "p1").setSource("p_field", "value").setRefresh(true).get();
response = client().prepareSearch("test").setQuery(boolQuery().must(matchAllQuery()).filter(termQuery("_parent", "p1"))).execute()
.actionGet();
assertHitCount(response, 1l);
response = client().prepareSearch("test").setQuery(boolQuery().must(matchAllQuery()).filter(termQuery("_parent", "parent#p1"))).execute()
.actionGet();
assertHitCount(response, 1l);
// test terms filter
client().prepareIndex("test", "child2", "c1").setSource("c_field", "value").setParent("p1").get();
response = client().prepareSearch("test").setQuery(boolQuery().must(matchAllQuery()).filter(termsQuery("_parent", "p1"))).execute()
.actionGet();
assertHitCount(response, 1l);
response = client().prepareSearch("test").setQuery(boolQuery().must(matchAllQuery()).filter(termsQuery("_parent", "parent#p1"))).execute()
.actionGet();
assertHitCount(response, 1l);
response = client().prepareSearch("test").setQuery(queryStringQuery("_parent#parent:p1")).get();
assertHitCount(response, 1L);
client().prepareIndex("test", "child", "c2").setSource("{}").setParent("p2").get();
refresh();
response = client().prepareSearch("test").setQuery(boolQuery().must(matchAllQuery()).filter(termsQuery("_parent", "p1"))).execute()
.actionGet();
assertHitCount(response, 2l);
refresh();
response = client().prepareSearch("test").setQuery(boolQuery().must(matchAllQuery()).filter(termsQuery("_parent", "p1", "p1"))).execute()
.actionGet();
assertHitCount(response, 2l);
response = client().prepareSearch("test").setQuery(termsQuery("_parent#parent", "p1", "p2")).get();
assertHitCount(response, 2L);
response = client().prepareSearch("test")
.setQuery(boolQuery().must(matchAllQuery()).filter(termsQuery("_parent", "parent#p1", "parent2#p1"))).get();
assertHitCount(response, 2l);
.setQuery(boolQuery()
.should(termQuery("_parent#parent", "p1"))
.should(termQuery("_parent#parent", "p2"))
).get();
assertHitCount(response, 2L);
}
public void testParentIdQuery() throws Exception {
assertAcked(prepareCreate("test")
.setSettings(settingsBuilder().put(indexSettings())
.put("index.refresh_interval", -1))
.addMapping("parent")
.addMapping("child", "_parent", "type=parent"));
ensureGreen();
client().prepareIndex("test", "child", "c1").setSource("{}").setParent("p1").get();
refresh();
SearchResponse response = client().prepareSearch("test").setQuery(parentId("child", "p1")).get();
assertHitCount(response, 1L);
client().prepareIndex("test", "child", "c2").setSource("{}").setParent("p2").get();
refresh();
response = client().prepareSearch("test")
.setQuery(boolQuery()
.should(parentId("child", "p1"))
.should(parentId("child", "p2"))
).get();
assertHitCount(response, 2L);
}
public void testHasChildNotBeingCached() throws IOException {
@ -1459,7 +1453,7 @@ public class ChildQuerySearchIT extends ESIntegTestCase {
refresh();
SearchResponse response = client().prepareSearch("test")
.setQuery(multiMatchQuery("1", "_parent"))
.setQuery(multiMatchQuery("1", "_parent#type1"))
.get();
assertThat(response.getHits().totalHits(), equalTo(1l));

View File

@ -157,7 +157,7 @@ public class ParentFieldLoadingIT extends ESIntegTestCase {
MapperService mapperService = indexService.mapperService();
DocumentMapper documentMapper = mapperService.documentMapper("child");
if (documentMapper != null) {
verified = documentMapper.parentFieldMapper().getChildJoinFieldType().fieldDataType().getLoading() == MappedFieldType.Loading.EAGER_GLOBAL_ORDINALS;
verified = documentMapper.parentFieldMapper().fieldType().fieldDataType().getLoading() == MappedFieldType.Loading.EAGER_GLOBAL_ORDINALS;
}
}
assertTrue(verified);

View File

@ -0,0 +1,72 @@
/*
* 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.search.internal;
import org.apache.lucene.queries.TermsQuery;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.index.mapper.internal.TypeFieldMapper;
import org.elasticsearch.test.ESTestCase;
import static org.apache.lucene.search.BooleanClause.Occur.FILTER;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.nullValue;
public class DefaultSearchContextTests extends ESTestCase {
public void testCreateSearchFilter() {
Query searchFilter = DefaultSearchContext.createSearchFilter(new String[]{"type1", "type2"}, null, randomBoolean());
Query expectedQuery = new BooleanQuery.Builder()
.add(new TermsQuery(TypeFieldMapper.NAME, new BytesRef("type1"), new BytesRef("type2")), FILTER)
.build();
assertThat(searchFilter, equalTo(expectedQuery));
searchFilter = DefaultSearchContext.createSearchFilter(new String[]{"type1", "type2"}, new MatchAllDocsQuery(), randomBoolean());
expectedQuery = new BooleanQuery.Builder()
.add(new TermsQuery(TypeFieldMapper.NAME, new BytesRef("type1"), new BytesRef("type2")), FILTER)
.add(new MatchAllDocsQuery(), FILTER)
.build();
assertThat(searchFilter, equalTo(expectedQuery));
searchFilter = DefaultSearchContext.createSearchFilter(null, null, false);
assertThat(searchFilter, nullValue());
searchFilter = DefaultSearchContext.createSearchFilter(null, null, true);
expectedQuery = new BooleanQuery.Builder().add(Queries.newNonNestedFilter(), FILTER).build();
assertThat(searchFilter, equalTo(expectedQuery));
searchFilter = DefaultSearchContext.createSearchFilter(null, new MatchAllDocsQuery(), true);
expectedQuery = new BooleanQuery.Builder()
.add(new MatchAllDocsQuery(), FILTER)
.add(Queries.newNonNestedFilter(), FILTER)
.build();
assertThat(searchFilter, equalTo(expectedQuery));
searchFilter = DefaultSearchContext.createSearchFilter(null, new MatchAllDocsQuery(), false);
expectedQuery = new BooleanQuery.Builder()
.add(new MatchAllDocsQuery(), FILTER)
.build();
assertThat(searchFilter, equalTo(expectedQuery));
}
}

View File

@ -55,7 +55,7 @@ public class SimilarityIT extends ESIntegTestCase {
.put("index.number_of_replicas", 0)
.put("similarity.custom.type", "BM25")
.put("similarity.custom.k1", 2.0f)
.put("similarity.custom.b", 1.5f)
.put("similarity.custom.b", 0.5f)
).execute().actionGet();
client().prepareIndex("test", "type1", "1").setSource("field1", "the quick brown fox jumped over the lazy dog",

View File

@ -1540,7 +1540,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
// Update settings to back to normal
assertAcked(client.admin().indices().prepareUpdateSettings("test-idx").setSettings(Settings.builder()
.put(IndexStore.INDEX_STORE_THROTTLE_TYPE_SETTING.getKey(), "none")
.put(IndexStore.INDEX_STORE_THROTTLE_TYPE_SETTING.getKey(), "node")
));
logger.info("--> wait for snapshot to complete");

View File

@ -51,6 +51,7 @@ import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Map;
import java.util.function.Function;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
@ -93,7 +94,7 @@ public class TribeIT extends ESIntegTestCase {
};
cluster2 = new InternalTestCluster(InternalTestCluster.configuredNodeMode(), randomLong(), createTempDir(), 2, 2,
Strings.randomBase64UUID(getRandom()), nodeConfigurationSource, 0, false, SECOND_CLUSTER_NODE_PREFIX, Collections.emptyList());
Strings.randomBase64UUID(getRandom()), nodeConfigurationSource, 0, false, SECOND_CLUSTER_NODE_PREFIX, Collections.emptyList(), Function.identity());
cluster2.beforeTest(getRandom(), 0.1);
cluster2.ensureAtLeastNumDataNodes(2);

View File

@ -1 +0,0 @@
69e187ef1d2d9c9570363eb4186821e0341df5b8

View File

@ -0,0 +1 @@
528a695bb8882dbc3d9866335ac1bb3905cba4e3

View File

@ -1 +0,0 @@
0fa00a45ff9bc6a4df44db81f2e4e44ea94bf88e

View File

@ -0,0 +1 @@
3fb1bcc1001a10b74ae91848c8558572891c1409

View File

@ -1 +0,0 @@
f6854c65c7f4c6d9de583f4daa4fd3ae8a3800f1

View File

@ -0,0 +1 @@
9eff7f186877882f8b68f031f610bd7ab8c5c1fb

View File

@ -1 +0,0 @@
e996e6c723eb415ba2cfa7f5e98bbf194a4918dd

View File

@ -0,0 +1 @@
6e6253936522f27b35ba4d8485806f517ef2df45

View File

@ -1 +0,0 @@
3b7a5d97b10885f16eb53deb15d64c942b9f9fdb

View File

@ -0,0 +1 @@
8a313aa34b0070d3f7d48005e7677b680db1b09d

View File

@ -1 +0,0 @@
e4dda3eeb76e340aa4713a3b20d68c4a1504e505

View File

@ -0,0 +1 @@
bf4c5a17cfb265d321ef4cfb0f3d7c1a6a6651de

View File

@ -1 +0,0 @@
800442a5d7612ce4c8748831871b4d436a50554e

View File

@ -0,0 +1 @@
2713a319d0aa696c65a32a36fda830bc482a5880

View File

@ -1 +0,0 @@
bdf184de9b5773c7af3ae908af78eeb1e512470c

View File

@ -0,0 +1 @@
88251ecdbf877c15a94d4013aa5157f5b5ce4cea

View File

@ -1 +0,0 @@
fc59de52bd2c7e420edfd235723cb8b0dd44e92d

View File

@ -0,0 +1 @@
bf9e522244c7c4eee6c3bcc3212ff057f7b88000

View File

@ -1 +0,0 @@
1d341e6a4f11f3170773ccffdbe6815b45967e3d

View File

@ -0,0 +1 @@
12d71cf10a4b79231dc488af16d723dfca5ab64b

View File

@ -1 +0,0 @@
a1b02c2b595ac92f45f0d2be03841a3a7fcae1f1

View File

@ -0,0 +1 @@
f903d67d042904527a7e2e8a75c55afe36a04251

View File

@ -1 +0,0 @@
e3ea422b56734329fb6974e9cf9f66478adb5793

View File

@ -0,0 +1 @@
2f5758bbcf97048ab62d2d4ae73867d06f1ed03f

View File

@ -1 +0,0 @@
5eadbd4e63120b59ab6445e39489205f98420471

View File

@ -0,0 +1 @@
2cc29e4658be151658fac6e5ed7915982b6de861

View File

@ -1 +0,0 @@
a336287e65d082535f02a8427666dbe46b1b9b74

View File

@ -0,0 +1 @@
f490a09ca056aba42e8751a469ef114df64aae0d

View File

@ -107,6 +107,13 @@ All options but the first option need a normalization value.
Type name: `DFR`
[float]
[[dfi]]
==== DFI similarity
Similarity that implements the http://trec.nist.gov/pubs/trec21/papers/irra.web.nb.pdf[divergence from independence]
model (normalized chi-squared distance)
[float]
[[ib]]
==== IB similarity.

View File

@ -8,7 +8,7 @@ algorithm other than the default TF/IDF, such as `BM25`.
Similarities are mostly useful for <<string,`string`>> fields, especially
`analyzed` string fields, but can also apply to other field types.
Custom similarites can be configured by tuning the parameters of the built-in
Custom similarities can be configured by tuning the parameters of the built-in
similarities. For more details about this expert options, see the
<<index-modules-similarity,similarity module>>.

View File

@ -15,6 +15,7 @@ your application to Elasticsearch 3.0.
* <<breaking_30_non_loopback>>
* <<breaking_30_thread_pool>>
* <<breaking_30_allocation>>
* <<breaking_30_percolator>>
[[breaking_30_search_changes]]
=== Warmers
@ -181,6 +182,12 @@ When `max_children` was set to `0` on the `has_child` query then there was no up
are allowed to match. This has changed and `0` now really means to zero child documents are allowed. If no upper limit
is needed then the `max_children` option shouldn't be defined at all on the `has_child` query.
==== `_parent` field no longer indexed
The join between parent and child documents no longer relies on indexed fields and therefor from `3.0.0` onwards
the `_parent` indexed field won't be indexed. In order to find documents that referrer to a specific parent id
the new `parent_id` query can be used. The get response and hits inside the search response remain to include
the parent id under the `_parent` key.
[[breaking_30_settings_changes]]
=== Settings changes
@ -619,6 +626,7 @@ in the case where shard copies can be found. Previously, a node not holding the
holding shard copies were satisfying the allocation deciders. Now, the shard will be assigned to a node having a shard copy,
even if none of the nodes holding a shard copy satisfy the allocation deciders.
[[breaking_30_percolator]]
=== Percolator
Adding percolator queries and modifications to existing percolator queries are no longer visible in immediately
@ -634,3 +642,5 @@ The percolate api can no longer accept documents that have fields that don't exi
When percolating an existing document then specifying a document in the source of the percolate request is not allowed
any more.
Percolator documents are no longer excluded from the search response.

View File

@ -29,4 +29,6 @@ include::has-child-query.asciidoc[]
include::has-parent-query.asciidoc[]
include::parent-id-query.asciidoc[]

View File

@ -0,0 +1,31 @@
[[query-dsl-parent-id-query]]
=== Parent Id Query
added[3.0.0]
The `parent_id` query can be used to find a child document pointing to a particular parent id.
The actual underlying Lucene field that is used to store to what parent id a child document is referring to
is determined by the child type's `_parent` field. This query helps by selecting the right field based
on the specified child type. Example:
[source,js]
--------------------------------------------------
{
"parent_id" : {
"type" : "blog_tag",
"id" : "1"
}
}
--------------------------------------------------
==== Parameters
This query has two required parameters:
[horizontal]
`type`::
The child type. This must be a type with `_parent` field.
`id`::
The required parent id select documents must referrer to.

View File

@ -99,7 +99,7 @@ PUT place/shops/1
...
"suggest": {
"input": ["timmy's", "starbucks", "dunkin donuts"],
"context": {
"contexts": {
"place_type": ["cafe", "food"] <1>
}
}
@ -273,7 +273,7 @@ PUT place/shops/1
{
"suggest": {
"input": "timmy's",
"context": [
"contexts": [
"location": [
{
"lat": 43.6624803,
@ -305,7 +305,7 @@ POST place/_suggest
"completion" : {
"field" : "suggest",
"size": 10,
"context": {
"contexts": {
"location": {
"lat": 43.662,
"lon": -79.380

View File

@ -1 +0,0 @@
60e056d2dd04a81440482b047af0737bc41593d9

View File

@ -0,0 +1 @@
31db8e49e4089772eae8ab2db0ac59bab6fbcd2f

View File

@ -1 +0,0 @@
1fce4e9b5c4482bb95e8b275c825d112640d6f1e

Some files were not shown because too many files have changed in this diff Show More