diff --git a/core/pom.xml b/core/pom.xml index 300cf5fddb7b2..d958fcee41ac9 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -1024,6 +1024,50 @@ + + + org.apache.maven.plugins + maven-antrun-plugin + + + + integ-setup + pre-integration-test + + run + + + + + + + + + + integ-teardown + post-integration-test + + run + + + + + + + + + + + org.apache.maven.plugins + maven-failsafe-plugin + + + 127.0.0.1:9300 + + + diff --git a/core/src/main/java/org/apache/lucene/queryparser/classic/MapperQueryParser.java b/core/src/main/java/org/apache/lucene/queryparser/classic/MapperQueryParser.java index 237fe67b36e20..db4164f9f9eca 100644 --- a/core/src/main/java/org/apache/lucene/queryparser/classic/MapperQueryParser.java +++ b/core/src/main/java/org/apache/lucene/queryparser/classic/MapperQueryParser.java @@ -428,8 +428,7 @@ private Query getFuzzyQuerySingle(String field, String termStr, String minSimila currentFieldType = parseContext.fieldMapper(field); if (currentFieldType != null) { try { - //LUCENE 4 UPGRADE I disabled transpositions here by default - maybe this needs to be changed - return currentFieldType.fuzzyQuery(termStr, Fuzziness.build(minSimilarity), fuzzyPrefixLength, settings.fuzzyMaxExpansions(), false); + return currentFieldType.fuzzyQuery(termStr, Fuzziness.build(minSimilarity), fuzzyPrefixLength, settings.fuzzyMaxExpansions(), FuzzyQuery.defaultTranspositions); } catch (RuntimeException e) { if (settings.lenient()) { return null; @@ -444,8 +443,7 @@ private Query getFuzzyQuerySingle(String field, String termStr, String minSimila protected Query newFuzzyQuery(Term term, float minimumSimilarity, int prefixLength) { String text = term.text(); int numEdits = FuzzyQuery.floatToEdits(minimumSimilarity, text.codePointCount(0, text.length())); - //LUCENE 4 UPGRADE I disabled transpositions here by default - maybe this needs to be changed - FuzzyQuery query = new FuzzyQuery(term, numEdits, prefixLength, settings.fuzzyMaxExpansions(), false); + FuzzyQuery query = new FuzzyQuery(term, numEdits, prefixLength, settings.fuzzyMaxExpansions(), FuzzyQuery.defaultTranspositions); QueryParsers.setRewriteMethod(query, settings.fuzzyRewriteMethod()); return query; } diff --git a/core/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodeInfo.java b/core/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodeInfo.java index f3c5eb7e1fc16..5ac72d70b9cf3 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodeInfo.java +++ b/core/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodeInfo.java @@ -30,7 +30,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.http.HttpInfo; import org.elasticsearch.monitor.jvm.JvmInfo; -import org.elasticsearch.monitor.network.NetworkInfo; import org.elasticsearch.monitor.os.OsInfo; import org.elasticsearch.monitor.process.ProcessInfo; import org.elasticsearch.threadpool.ThreadPoolInfo; @@ -65,9 +64,6 @@ public class NodeInfo extends BaseNodeResponse { @Nullable private ThreadPoolInfo threadPool; - @Nullable - private NetworkInfo network; - @Nullable private TransportInfo transport; @@ -81,7 +77,7 @@ public class NodeInfo extends BaseNodeResponse { } public NodeInfo(Version version, Build build, DiscoveryNode node, @Nullable ImmutableMap serviceAttributes, @Nullable Settings settings, - @Nullable OsInfo os, @Nullable ProcessInfo process, @Nullable JvmInfo jvm, @Nullable ThreadPoolInfo threadPool, @Nullable NetworkInfo network, + @Nullable OsInfo os, @Nullable ProcessInfo process, @Nullable JvmInfo jvm, @Nullable ThreadPoolInfo threadPool, @Nullable TransportInfo transport, @Nullable HttpInfo http, @Nullable PluginsInfo plugins) { super(node); this.version = version; @@ -92,7 +88,6 @@ public NodeInfo(Version version, Build build, DiscoveryNode node, @Nullable Immu this.process = process; this.jvm = jvm; this.threadPool = threadPool; - this.network = network; this.transport = transport; this.http = http; this.plugins = plugins; @@ -165,14 +160,6 @@ public ThreadPoolInfo getThreadPool() { return this.threadPool; } - /** - * Network level information. - */ - @Nullable - public NetworkInfo getNetwork() { - return network; - } - @Nullable public TransportInfo getTransport() { return transport; @@ -222,9 +209,6 @@ public void readFrom(StreamInput in) throws IOException { if (in.readBoolean()) { threadPool = ThreadPoolInfo.readThreadPoolInfo(in); } - if (in.readBoolean()) { - network = NetworkInfo.readNetworkInfo(in); - } if (in.readBoolean()) { transport = TransportInfo.readTransportInfo(in); } @@ -281,12 +265,6 @@ public void writeTo(StreamOutput out) throws IOException { out.writeBoolean(true); threadPool.writeTo(out); } - if (network == null) { - out.writeBoolean(false); - } else { - out.writeBoolean(true); - network.writeTo(out); - } if (transport == null) { out.writeBoolean(false); } else { @@ -306,5 +284,4 @@ public void writeTo(StreamOutput out) throws IOException { plugins.writeTo(out); } } - } diff --git a/core/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoResponse.java b/core/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoResponse.java index e872be90142a3..65033f37b1581 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoResponse.java +++ b/core/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoResponse.java @@ -111,9 +111,6 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (nodeInfo.getThreadPool() != null) { nodeInfo.getThreadPool().toXContent(builder, params); } - if (nodeInfo.getNetwork() != null) { - nodeInfo.getNetwork().toXContent(builder, params); - } if (nodeInfo.getTransport() != null) { nodeInfo.getTransport().toXContent(builder, params); } diff --git a/core/src/main/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStats.java b/core/src/main/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStats.java index c7a6f4615a33c..ea8c3bc106f67 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStats.java +++ b/core/src/main/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStats.java @@ -29,9 +29,8 @@ import org.elasticsearch.http.HttpStats; import org.elasticsearch.indices.NodeIndicesStats; import org.elasticsearch.indices.breaker.AllCircuitBreakerStats; -import org.elasticsearch.monitor.fs.FsStats; +import org.elasticsearch.monitor.fs.FsInfo; import org.elasticsearch.monitor.jvm.JvmStats; -import org.elasticsearch.monitor.network.NetworkStats; import org.elasticsearch.monitor.os.OsStats; import org.elasticsearch.monitor.process.ProcessStats; import org.elasticsearch.threadpool.ThreadPoolStats; @@ -63,10 +62,7 @@ public class NodeStats extends BaseNodeResponse implements ToXContent { private ThreadPoolStats threadPool; @Nullable - private NetworkStats network; - - @Nullable - private FsStats fs; + private FsInfo fs; @Nullable private TransportStats transport; @@ -82,7 +78,7 @@ public class NodeStats extends BaseNodeResponse implements ToXContent { public NodeStats(DiscoveryNode node, long timestamp, @Nullable NodeIndicesStats indices, @Nullable OsStats os, @Nullable ProcessStats process, @Nullable JvmStats jvm, @Nullable ThreadPoolStats threadPool, - @Nullable NetworkStats network, @Nullable FsStats fs, @Nullable TransportStats transport, @Nullable HttpStats http, + @Nullable FsInfo fs, @Nullable TransportStats transport, @Nullable HttpStats http, @Nullable AllCircuitBreakerStats breaker) { super(node); this.timestamp = timestamp; @@ -91,7 +87,6 @@ public NodeStats(DiscoveryNode node, long timestamp, @Nullable NodeIndicesStats this.process = process; this.jvm = jvm; this.threadPool = threadPool; - this.network = network; this.fs = fs; this.transport = transport; this.http = http; @@ -147,19 +142,11 @@ public ThreadPoolStats getThreadPool() { return this.threadPool; } - /** - * Network level statistics. - */ - @Nullable - public NetworkStats getNetwork() { - return network; - } - /** * File system level stats. */ @Nullable - public FsStats getFs() { + public FsInfo getFs() { return fs; } @@ -204,10 +191,7 @@ public void readFrom(StreamInput in) throws IOException { threadPool = ThreadPoolStats.readThreadPoolStats(in); } if (in.readBoolean()) { - network = NetworkStats.readNetworkStats(in); - } - if (in.readBoolean()) { - fs = FsStats.readFsStats(in); + fs = FsInfo.readFsInfo(in); } if (in.readBoolean()) { transport = TransportStats.readTransportStats(in); @@ -253,12 +237,6 @@ public void writeTo(StreamOutput out) throws IOException { out.writeBoolean(true); threadPool.writeTo(out); } - if (network == null) { - out.writeBoolean(false); - } else { - out.writeBoolean(true); - network.writeTo(out); - } if (fs == null) { out.writeBoolean(false); } else { @@ -313,9 +291,6 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (getThreadPool() != null) { getThreadPool().toXContent(builder, params); } - if (getNetwork() != null) { - getNetwork().toXContent(builder, params); - } if (getFs() != null) { getFs().toXContent(builder, params); } diff --git a/core/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java b/core/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java index 61b927d9ad703..7a512fd75ea8f 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java +++ b/core/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java @@ -36,7 +36,7 @@ import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilderString; -import org.elasticsearch.monitor.fs.FsStats; +import org.elasticsearch.monitor.fs.FsInfo; import org.elasticsearch.monitor.jvm.JvmInfo; import org.elasticsearch.monitor.os.OsInfo; @@ -52,7 +52,7 @@ public class ClusterStatsNodes implements ToXContent, Streamable { private OsStats os; private ProcessStats process; private JvmStats jvm; - private FsStats.Info fs; + private FsInfo.Path fs; private Set plugins; private ClusterStatsNodes() { @@ -63,7 +63,7 @@ public ClusterStatsNodes(ClusterStatsNodeResponse[] nodeResponses) { this.versions = new HashSet<>(); this.os = new OsStats(); this.jvm = new JvmStats(); - this.fs = new FsStats.Info(); + this.fs = new FsInfo.Path(); this.plugins = new HashSet<>(); this.process = new ProcessStats(); @@ -116,7 +116,7 @@ public JvmStats getJvm() { return jvm; } - public FsStats.Info getFs() { + public FsInfo.Path getFs() { return fs; } @@ -138,7 +138,7 @@ public void readFrom(StreamInput in) throws IOException { os = OsStats.readOsStats(in); process = ProcessStats.readStats(in); jvm = JvmStats.readJvmStats(in); - fs = FsStats.Info.readInfoFrom(in); + fs = FsInfo.Path.readInfoFrom(in); size = in.readVInt(); plugins = new HashSet<>(size); diff --git a/core/src/main/java/org/elasticsearch/action/bulk/BulkItemResponse.java b/core/src/main/java/org/elasticsearch/action/bulk/BulkItemResponse.java index 3b9d2e03ba094..80e86eaaf1719 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/BulkItemResponse.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/BulkItemResponse.java @@ -19,7 +19,6 @@ package org.elasticsearch.action.bulk; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionWriteResponse; import org.elasticsearch.action.delete.DeleteResponse; @@ -28,7 +27,6 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Streamable; -import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.rest.RestStatus; import java.io.IOException; diff --git a/core/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java b/core/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java index e97af0f3da3cb..783db0c33db53 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java @@ -168,13 +168,13 @@ private boolean setResponseFailureIfIndexMatches(AtomicArray r } else if (request instanceof DeleteRequest) { DeleteRequest deleteRequest = (DeleteRequest) request; if (index.equals(deleteRequest.index())) { - responses.set(idx, new BulkItemResponse(idx, "index", new BulkItemResponse.Failure(deleteRequest.index(), deleteRequest.type(), deleteRequest.id(), e))); + responses.set(idx, new BulkItemResponse(idx, "delete", new BulkItemResponse.Failure(deleteRequest.index(), deleteRequest.type(), deleteRequest.id(), e))); return true; } } else if (request instanceof UpdateRequest) { UpdateRequest updateRequest = (UpdateRequest) request; if (index.equals(updateRequest.index())) { - responses.set(idx, new BulkItemResponse(idx, "index", new BulkItemResponse.Failure(updateRequest.index(), updateRequest.type(), updateRequest.id(), e))); + responses.set(idx, new BulkItemResponse(idx, "update", new BulkItemResponse.Failure(updateRequest.index(), updateRequest.type(), updateRequest.id(), e))); return true; } } else { @@ -379,7 +379,15 @@ private boolean addFailureIfIndexIsUnavailable(DocumentRequest request, BulkRequ if (unavailableException != null) { BulkItemResponse.Failure failure = new BulkItemResponse.Failure(request.index(), request.type(), request.id(), unavailableException); - BulkItemResponse bulkItemResponse = new BulkItemResponse(idx, "index", failure); + String operationType = "unknown"; + if (request instanceof IndexRequest) { + operationType = "index"; + } else if (request instanceof DeleteRequest) { + operationType = "delete"; + } else if (request instanceof UpdateRequest) { + operationType = "update"; + } + BulkItemResponse bulkItemResponse = new BulkItemResponse(idx, operationType, failure); responses.set(idx, bulkItemResponse); // make sure the request gets never processed again bulkRequest.requests.set(idx, null); diff --git a/core/src/main/java/org/elasticsearch/action/search/type/TransportSearchScrollScanAction.java b/core/src/main/java/org/elasticsearch/action/search/type/TransportSearchScrollScanAction.java index 93042815e002a..0be39abba14c3 100644 --- a/core/src/main/java/org/elasticsearch/action/search/type/TransportSearchScrollScanAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/type/TransportSearchScrollScanAction.java @@ -41,7 +41,9 @@ import org.elasticsearch.search.internal.InternalSearchResponse; import java.io.IOException; +import java.util.ArrayList; import java.util.List; +import java.util.Objects; import java.util.concurrent.atomic.AtomicInteger; import static org.elasticsearch.action.search.type.TransportSearchHelper.internalScrollSearchRequest; @@ -159,7 +161,9 @@ void executePhase(final int shardIndex, DiscoveryNode node, final long searchId) searchService.sendExecuteScan(node, internalScrollSearchRequest(searchId, request), new ActionListener() { @Override public void onResponse(ScrollQueryFetchSearchResult result) { - queryFetchResults.set(shardIndex, result.result()); + QueryFetchSearchResult shardResult = result.result(); + Objects.requireNonNull(shardResult, "QueryFetchSearchResult can't be null"); + queryFetchResults.setOnce(shardIndex, shardResult); if (counter.decrementAndGet() == 0) { finishHim(); } @@ -197,25 +201,27 @@ private void finishHim() { private void innerFinishHim() throws IOException { int numberOfHits = 0; - for (AtomicArray.Entry entry : queryFetchResults.asList()) { + List> entries = queryFetchResults.asList(); + for (AtomicArray.Entry entry : entries) { numberOfHits += entry.value.queryResult().topDocs().scoreDocs.length; } - ScoreDoc[] docs = new ScoreDoc[numberOfHits]; - int counter = 0; - for (AtomicArray.Entry entry : queryFetchResults.asList()) { + List docs = new ArrayList<>(numberOfHits); + for (AtomicArray.Entry entry : entries) { ScoreDoc[] scoreDocs = entry.value.queryResult().topDocs().scoreDocs; for (ScoreDoc scoreDoc : scoreDocs) { scoreDoc.shardIndex = entry.index; - docs[counter++] = scoreDoc; + docs.add(scoreDoc); } } - final InternalSearchResponse internalResponse = searchPhaseController.merge(docs, queryFetchResults, queryFetchResults); + final InternalSearchResponse internalResponse = searchPhaseController.merge(docs.toArray(new ScoreDoc[0]), queryFetchResults, queryFetchResults); ((InternalSearchHits) internalResponse.hits()).totalHits = Long.parseLong(this.scrollId.getAttributes().get("total_hits")); - for (AtomicArray.Entry entry : queryFetchResults.asList()) { + for (AtomicArray.Entry entry : entries) { if (entry.value.queryResult().topDocs().scoreDocs.length < entry.value.queryResult().size()) { - // we found more than we want for this round, remove this from our scrolling + // we found more than we want for this round, remove this from our scrolling, so we don't go back to + // this shard, since all hits have been processed. + // The SearchContext already gets freed on the node holding the shard, via a similar check. queryFetchResults.set(entry.index, null); } } diff --git a/core/src/main/java/org/elasticsearch/action/update/UpdateHelper.java b/core/src/main/java/org/elasticsearch/action/update/UpdateHelper.java index df67a6d417d12..c9401ca5392ec 100644 --- a/core/src/main/java/org/elasticsearch/action/update/UpdateHelper.java +++ b/core/src/main/java/org/elasticsearch/action/update/UpdateHelper.java @@ -73,39 +73,41 @@ public UpdateHelper(Settings settings, ScriptService scriptService) { */ @SuppressWarnings("unchecked") public Result prepare(UpdateRequest request, IndexShard indexShard) { - long getDateNS = System.nanoTime(); final GetResult getResult = indexShard.getService().get(request.type(), request.id(), new String[]{RoutingFieldMapper.NAME, ParentFieldMapper.NAME, TTLFieldMapper.NAME, TimestampFieldMapper.NAME}, true, request.version(), request.versionType(), FetchSourceContext.FETCH_SOURCE, false); + return prepare(request, getResult); + } + /** + * Prepares an update request by converting it into an index or delete request or an update response (no action). + */ + @SuppressWarnings("unchecked") + protected Result prepare(UpdateRequest request, final GetResult getResult) { + long getDateNS = System.nanoTime(); if (!getResult.isExists()) { if (request.upsertRequest() == null && !request.docAsUpsert()) { - throw new DocumentMissingException(new ShardId(indexShard.indexService().index().name(), request.shardId()), request.type(), request.id()); + throw new DocumentMissingException(new ShardId(request.index(), request.shardId()), request.type(), request.id()); } - Long ttl = null; IndexRequest indexRequest = request.docAsUpsert() ? request.doc() : request.upsertRequest(); - if (request.scriptedUpsert() && (request.script() != null)) { + Long ttl = indexRequest.ttl(); + if (request.scriptedUpsert() && request.script() != null) { // Run the script to perform the create logic - IndexRequest upsert = request.upsertRequest(); + IndexRequest upsert = request.upsertRequest(); Map upsertDoc = upsert.sourceAsMap(); Map ctx = new HashMap<>(2); // Tell the script that this is a create and not an update ctx.put("op", "create"); ctx.put("_source", upsertDoc); - try { - ExecutableScript script = scriptService.executable(request.script, ScriptContext.Standard.UPDATE); - script.setNextVar("ctx", ctx); - script.run(); - // we need to unwrap the ctx... - ctx = (Map) script.unwrap(ctx); - } catch (Exception e) { - throw new IllegalArgumentException("failed to execute script", e); - } + ctx = executeScript(request, ctx); //Allow the script to set TTL using ctx._ttl - ttl = getTTLFromScriptContext(ctx); + if (ttl < 0) { + ttl = getTTLFromScriptContext(ctx); + } + //Allow the script to abort the create by setting "op" to "none" String scriptOpChoice = (String) ctx.get("op"); - + // Only valid options for an upsert script are "create" // (the default) or "none", meaning abort upsert if (!"create".equals(scriptOpChoice)) { @@ -123,8 +125,8 @@ public Result prepare(UpdateRequest request, IndexShard indexShard) { indexRequest.index(request.index()).type(request.type()).id(request.id()) // it has to be a "create!" - .create(true) - .ttl(ttl) + .create(true) + .ttl(ttl == null || ttl < 0 ? null : ttl) .refresh(request.refresh()) .routing(request.routing()) .parent(request.parent()) @@ -146,7 +148,7 @@ public Result prepare(UpdateRequest request, IndexShard indexShard) { if (getResult.internalSourceRef() == null) { // no source, we can't do nothing, through a failure... - throw new DocumentSourceMissingException(new ShardId(indexShard.indexService().index().name(), request.shardId()), request.type(), request.id()); + throw new DocumentSourceMissingException(new ShardId(request.index(), request.shardId()), request.type(), request.id()); } Tuple> sourceAndContent = XContentHelper.convertToMap(getResult.internalSourceRef(), true); @@ -192,15 +194,7 @@ public Result prepare(UpdateRequest request, IndexShard indexShard) { ctx.put("_ttl", originalTtl); ctx.put("_source", sourceAndContent.v2()); - try { - ExecutableScript script = scriptService.executable(request.script, ScriptContext.Standard.UPDATE); - script.setNextVar("ctx", ctx); - script.run(); - // we need to unwrap the ctx... - ctx = (Map) script.unwrap(ctx); - } catch (Exception e) { - throw new IllegalArgumentException("failed to execute script", e); - } + ctx = executeScript(request, ctx); operation = (String) ctx.get("op"); @@ -213,7 +207,7 @@ public Result prepare(UpdateRequest request, IndexShard indexShard) { } ttl = getTTLFromScriptContext(ctx); - + updatedSourceAsMap = (Map) ctx.get("_source"); } @@ -243,7 +237,7 @@ public Result prepare(UpdateRequest request, IndexShard indexShard) { return new Result(deleteRequest, Operation.DELETE, updatedSourceAsMap, updateSourceContentType); } else if ("none".equals(operation)) { UpdateResponse update = new UpdateResponse(getResult.getIndex(), getResult.getType(), getResult.getId(), getResult.getVersion(), false); - update.setGetResult(extractGetResult(request, indexShard.indexService().index().name(), getResult.getVersion(), updatedSourceAsMap, updateSourceContentType, getResult.internalSourceRef())); + update.setGetResult(extractGetResult(request, request.index(), getResult.getVersion(), updatedSourceAsMap, updateSourceContentType, getResult.internalSourceRef())); return new Result(update, Operation.NONE, updatedSourceAsMap, updateSourceContentType); } else { logger.warn("Used update operation [{}] for script [{}], doing nothing...", operation, request.script.getScript()); @@ -252,6 +246,21 @@ public Result prepare(UpdateRequest request, IndexShard indexShard) { } } + private Map executeScript(UpdateRequest request, Map ctx) { + try { + if (scriptService != null) { + ExecutableScript script = scriptService.executable(request.script, ScriptContext.Standard.UPDATE); + script.setNextVar("ctx", ctx); + script.run(); + // we need to unwrap the ctx... + ctx = (Map) script.unwrap(ctx); + } + } catch (Exception e) { + throw new IllegalArgumentException("failed to execute script", e); + } + return ctx; + } + private Long getTTLFromScriptContext(Map ctx) { Long ttl = null; Object fetchedTTL = ctx.get("_ttl"); diff --git a/core/src/main/java/org/elasticsearch/action/update/UpdateRequest.java b/core/src/main/java/org/elasticsearch/action/update/UpdateRequest.java index 090acf1694aa0..cc7fea15b1031 100644 --- a/core/src/main/java/org/elasticsearch/action/update/UpdateRequest.java +++ b/core/src/main/java/org/elasticsearch/action/update/UpdateRequest.java @@ -690,16 +690,18 @@ public boolean docAsUpsert() { return this.docAsUpsert; } - public void docAsUpsert(boolean shouldUpsertDoc) { + public UpdateRequest docAsUpsert(boolean shouldUpsertDoc) { this.docAsUpsert = shouldUpsertDoc; + return this; } public boolean scriptedUpsert(){ return this.scriptedUpsert; } - public void scriptedUpsert(boolean scriptedUpsert) { + public UpdateRequest scriptedUpsert(boolean scriptedUpsert) { this.scriptedUpsert = scriptedUpsert; + return this; } diff --git a/core/src/main/java/org/elasticsearch/bootstrap/JarHell.java b/core/src/main/java/org/elasticsearch/bootstrap/JarHell.java index dac6241554946..c27d81abbeb21 100644 --- a/core/src/main/java/org/elasticsearch/bootstrap/JarHell.java +++ b/core/src/main/java/org/elasticsearch/bootstrap/JarHell.java @@ -81,10 +81,10 @@ public static void checkJarHell(URL urls[]) throws Exception { // a "list" at all. So just exclude any elements underneath the java home String javaHome = System.getProperty("java.home"); logger.debug("java.home: {}", javaHome); - final Map clazzes = new HashMap<>(32768); - Set seenJars = new HashSet<>(); + final Map clazzes = new HashMap<>(32768); + Set seenJars = new HashSet<>(); for (final URL url : urls) { - String path = URLDecoder.decode(url.getPath(), "UTF-8"); + final Path path = PathUtils.get(url.toURI()); // exclude system resources if (path.startsWith(javaHome)) { logger.debug("excluding system resource: {}", path); @@ -96,7 +96,7 @@ public static void checkJarHell(URL urls[]) throws Exception { continue; // we can't fail because of sheistiness with joda-time } logger.debug("examining jar: {}", path); - try (JarFile file = new JarFile(path)) { + try (JarFile file = new JarFile(path.toString())) { Manifest manifest = file.getManifest(); if (manifest != null) { // inspect Manifest: give a nice error if jar requires a newer java version @@ -124,7 +124,7 @@ public static void checkJarHell(URL urls[]) throws Exception { if (entry.endsWith(".class")) { // for jar format, the separator is defined as / entry = entry.replace('/', '.').substring(0, entry.length() - 6); - checkClass(clazzes, entry, url); + checkClass(clazzes, entry, path); } } } @@ -140,7 +140,7 @@ public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IO if (entry.endsWith(".class")) { // normalize with the os separator entry = entry.replace(sep, ".").substring(0, entry.length() - 6); - checkClass(clazzes, entry, url); + checkClass(clazzes, entry, path); } return super.visitFile(file, attrs); } @@ -148,21 +148,32 @@ public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IO } } } - - @SuppressForbidden(reason = "proper use of URL to reduce noise") - static void checkClass(Map clazzes, String clazz, URL url) { - if (clazz.startsWith("org.apache.log4j")) { - return; // go figure, jar hell for what should be System.out.println... - } - if (clazz.equals("org.joda.time.base.BaseDateTime")) { - return; // apparently this is intentional... clean this up - } - URL previous = clazzes.put(clazz, url); + + static void checkClass(Map clazzes, String clazz, Path jarpath) { + Path previous = clazzes.put(clazz, jarpath); if (previous != null) { - throw new IllegalStateException("jar hell!" + System.lineSeparator() + - "class: " + clazz + System.lineSeparator() + - "jar1: " + previous.getPath() + System.lineSeparator() + - "jar2: " + url.getPath()); + if (previous.equals(jarpath)) { + if (clazz.startsWith("org.apache.xmlbeans")) { + return; // https://issues.apache.org/jira/browse/XMLBEANS-499 + } + // throw a better exception in this ridiculous case. + // unfortunately the zip file format allows this buggy possibility + // UweSays: It can, but should be considered as bug :-) + throw new IllegalStateException("jar hell!" + System.lineSeparator() + + "class: " + clazz + System.lineSeparator() + + "exists multiple times in jar: " + jarpath + " !!!!!!!!!"); + } else { + if (clazz.startsWith("org.apache.log4j")) { + return; // go figure, jar hell for what should be System.out.println... + } + if (clazz.equals("org.joda.time.base.BaseDateTime")) { + return; // apparently this is intentional... clean this up + } + throw new IllegalStateException("jar hell!" + System.lineSeparator() + + "class: " + clazz + System.lineSeparator() + + "jar1: " + previous + System.lineSeparator() + + "jar2: " + jarpath); + } } } } diff --git a/core/src/main/java/org/elasticsearch/bootstrap/Security.java b/core/src/main/java/org/elasticsearch/bootstrap/Security.java index 464bb040304cb..4d91341296abc 100644 --- a/core/src/main/java/org/elasticsearch/bootstrap/Security.java +++ b/core/src/main/java/org/elasticsearch/bootstrap/Security.java @@ -80,6 +80,7 @@ public void checkExit(int status) { m.put(Pattern.compile(".*lucene-core-.*\\.jar$"), "es.security.jar.lucene.core"); m.put(Pattern.compile(".*jsr166e-.*\\.jar$"), "es.security.jar.twitter.jsr166e"); m.put(Pattern.compile(".*securemock-.*\\.jar$"), "es.security.jar.elasticsearch.securemock"); + m.put(Pattern.compile(".*bcprov-.*\\.jar$"), "es.security.jar.bouncycastle.bcprov"); SPECIAL_JARS = Collections.unmodifiableMap(m); } @@ -96,11 +97,9 @@ static void setCodebaseProperties() { for (Map.Entry e : SPECIAL_JARS.entrySet()) { if (e.getKey().matcher(url.getPath()).matches()) { String prop = e.getValue(); - // TODO: we need to fix plugins to not include duplicate e.g. lucene-core jars, - // to add back this safety check! see https://github.com/elastic/elasticsearch/issues/11647 - // if (System.getProperty(prop) != null) { - // throw new IllegalStateException("property: " + prop + " is unexpectedly set: " + System.getProperty(prop)); - //} + if (System.getProperty(prop) != null) { + throw new IllegalStateException("property: " + prop + " is unexpectedly set: " + System.getProperty(prop)); + } System.setProperty(prop, url.toString()); } } diff --git a/core/src/main/java/org/elasticsearch/cluster/InternalClusterInfoService.java b/core/src/main/java/org/elasticsearch/cluster/InternalClusterInfoService.java index 3cac23d87a5a5..87f6e57616f41 100644 --- a/core/src/main/java/org/elasticsearch/cluster/InternalClusterInfoService.java +++ b/core/src/main/java/org/elasticsearch/cluster/InternalClusterInfoService.java @@ -39,7 +39,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; -import org.elasticsearch.monitor.fs.FsStats; +import org.elasticsearch.monitor.fs.FsInfo; import org.elasticsearch.node.settings.NodeSettingsService; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.ReceiveTimeoutTransportException; @@ -320,7 +320,7 @@ public void onResponse(NodesStatsResponse nodeStatses) { long available = 0; long total = 0; - for (FsStats.Info info : nodeStats.getFs()) { + for (FsInfo.Path info : nodeStats.getFs()) { available += info.getAvailable().bytes(); total += info.getTotal().bytes(); } diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java b/core/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java index 9634736cbeb3b..4e7998ca381be 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java @@ -167,6 +167,7 @@ public static State fromString(String state) { public static final String SETTING_VERSION_UPGRADED_STRING = "index.version.upgraded_string"; public static final String SETTING_VERSION_MINIMUM_COMPATIBLE = "index.version.minimum_compatible"; public static final String SETTING_CREATION_DATE = "index.creation_date"; + public static final String SETTING_PRIORITY = "index.priority"; public static final String SETTING_CREATION_DATE_STRING = "index.creation_date_string"; public static final String SETTING_UUID = "index.uuid"; public static final String SETTING_LEGACY_ROUTING_HASH_FUNCTION = "index.legacy.routing.hash.type"; diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java b/core/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java index 899e2633eba97..a947e9527c235 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java @@ -23,6 +23,7 @@ import com.carrotsearch.hppc.cursors.ObjectCursor; import com.google.common.base.Predicate; import com.google.common.collect.*; +import org.apache.lucene.util.CollectionUtil; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.block.ClusterBlocks; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -578,6 +579,10 @@ public void addAll(Collection mutableShardRoutings) { } } + public void sort(Comparator comparator) { + CollectionUtil.timSort(unassigned, comparator); + } + public int size() { return unassigned.size(); } diff --git a/core/src/main/java/org/elasticsearch/common/joda/Joda.java b/core/src/main/java/org/elasticsearch/common/joda/Joda.java index a457b85d80364..5f1ffb4207b50 100644 --- a/core/src/main/java/org/elasticsearch/common/joda/Joda.java +++ b/core/src/main/java/org/elasticsearch/common/joda/Joda.java @@ -118,6 +118,8 @@ public static FormatDateTimeFormatter forPattern(String input, Locale locale) { formatter = ISODateTimeFormat.ordinalDateTimeNoMillis(); } else if ("time".equals(input)) { formatter = ISODateTimeFormat.time(); + } else if ("timeNoMillis".equals(input) || "time_no_millis".equals(input)) { + formatter = ISODateTimeFormat.timeNoMillis(); } else if ("tTime".equals(input) || "t_time".equals(input)) { formatter = ISODateTimeFormat.tTime(); } else if ("tTimeNoMillis".equals(input) || "t_time_no_millis".equals(input)) { @@ -126,10 +128,14 @@ public static FormatDateTimeFormatter forPattern(String input, Locale locale) { formatter = ISODateTimeFormat.weekDate(); } else if ("weekDateTime".equals(input) || "week_date_time".equals(input)) { formatter = ISODateTimeFormat.weekDateTime(); + } else if ("weekDateTimeNoMillis".equals(input) || "week_date_time_no_millis".equals(input)) { + formatter = ISODateTimeFormat.weekDateTimeNoMillis(); } else if ("weekyear".equals(input) || "week_year".equals(input)) { formatter = ISODateTimeFormat.weekyear(); - } else if ("weekyearWeek".equals(input)) { + } else if ("weekyearWeek".equals(input) || "weekyear_week".equals(input)) { formatter = ISODateTimeFormat.weekyearWeek(); + } else if ("weekyearWeekDay".equals(input) || "weekyear_week_day".equals(input)) { + formatter = ISODateTimeFormat.weekyearWeekDay(); } else if ("year".equals(input)) { formatter = ISODateTimeFormat.year(); } else if ("yearMonth".equals(input) || "year_month".equals(input)) { @@ -140,6 +146,77 @@ public static FormatDateTimeFormatter forPattern(String input, Locale locale) { formatter = new DateTimeFormatterBuilder().append(new EpochTimePrinter(false), new EpochTimeParser(false)).toFormatter(); } else if ("epoch_millis".equals(input)) { formatter = new DateTimeFormatterBuilder().append(new EpochTimePrinter(true), new EpochTimeParser(true)).toFormatter(); + // strict date formats here, must be at least 4 digits for year and two for months and two for day + } else if ("strictBasicWeekDate".equals(input) || "strict_basic_week_date".equals(input)) { + formatter = StrictISODateTimeFormat.basicWeekDate(); + } else if ("strictBasicWeekDateTime".equals(input) || "strict_basic_week_date_time".equals(input)) { + formatter = StrictISODateTimeFormat.basicWeekDateTime(); + } else if ("strictBasicWeekDateTimeNoMillis".equals(input) || "strict_basic_week_date_time_no_millis".equals(input)) { + formatter = StrictISODateTimeFormat.basicWeekDateTimeNoMillis(); + } else if ("strictDate".equals(input) || "strict_date".equals(input)) { + formatter = StrictISODateTimeFormat.date(); + } else if ("strictDateHour".equals(input) || "strict_date_hour".equals(input)) { + formatter = StrictISODateTimeFormat.dateHour(); + } else if ("strictDateHourMinute".equals(input) || "strict_date_hour_minute".equals(input)) { + formatter = StrictISODateTimeFormat.dateHourMinute(); + } else if ("strictDateHourMinuteSecond".equals(input) || "strict_date_hour_minute_second".equals(input)) { + formatter = StrictISODateTimeFormat.dateHourMinuteSecond(); + } else if ("strictDateHourMinuteSecondFraction".equals(input) || "strict_date_hour_minute_second_fraction".equals(input)) { + formatter = StrictISODateTimeFormat.dateHourMinuteSecondFraction(); + } else if ("strictDateHourMinuteSecondMillis".equals(input) || "strict_date_hour_minute_second_millis".equals(input)) { + formatter = StrictISODateTimeFormat.dateHourMinuteSecondMillis(); + } else if ("strictDateOptionalTime".equals(input) || "strict_date_optional_time".equals(input)) { + // in this case, we have a separate parser and printer since the dataOptionalTimeParser can't print + // this sucks we should use the root local by default and not be dependent on the node + return new FormatDateTimeFormatter(input, + StrictISODateTimeFormat.dateOptionalTimeParser().withZone(DateTimeZone.UTC), + StrictISODateTimeFormat.dateTime().withZone(DateTimeZone.UTC), locale); + } else if ("strictDateTime".equals(input) || "strict_date_time".equals(input)) { + formatter = StrictISODateTimeFormat.dateTime(); + } else if ("strictDateTimeNoMillis".equals(input) || "strict_date_time_no_millis".equals(input)) { + formatter = StrictISODateTimeFormat.dateTimeNoMillis(); + } else if ("strictHour".equals(input) || "strict_hour".equals(input)) { + formatter = StrictISODateTimeFormat.hour(); + } else if ("strictHourMinute".equals(input) || "strict_hour_minute".equals(input)) { + formatter = StrictISODateTimeFormat.hourMinute(); + } else if ("strictHourMinuteSecond".equals(input) || "strict_hour_minute_second".equals(input)) { + formatter = StrictISODateTimeFormat.hourMinuteSecond(); + } else if ("strictHourMinuteSecondFraction".equals(input) || "strict_hour_minute_second_fraction".equals(input)) { + formatter = StrictISODateTimeFormat.hourMinuteSecondFraction(); + } else if ("strictHourMinuteSecondMillis".equals(input) || "strict_hour_minute_second_millis".equals(input)) { + formatter = StrictISODateTimeFormat.hourMinuteSecondMillis(); + } else if ("strictOrdinalDate".equals(input) || "strict_ordinal_date".equals(input)) { + formatter = StrictISODateTimeFormat.ordinalDate(); + } else if ("strictOrdinalDateTime".equals(input) || "strict_ordinal_date_time".equals(input)) { + formatter = StrictISODateTimeFormat.ordinalDateTime(); + } else if ("strictOrdinalDateTimeNoMillis".equals(input) || "strict_ordinal_date_time_no_millis".equals(input)) { + formatter = StrictISODateTimeFormat.ordinalDateTimeNoMillis(); + } else if ("strictTime".equals(input) || "strict_time".equals(input)) { + formatter = StrictISODateTimeFormat.time(); + } else if ("strictTimeNoMillis".equals(input) || "strict_time_no_millis".equals(input)) { + formatter = StrictISODateTimeFormat.timeNoMillis(); + } else if ("strictTTime".equals(input) || "strict_t_time".equals(input)) { + formatter = StrictISODateTimeFormat.tTime(); + } else if ("strictTTimeNoMillis".equals(input) || "strict_t_time_no_millis".equals(input)) { + formatter = StrictISODateTimeFormat.tTimeNoMillis(); + } else if ("strictWeekDate".equals(input) || "strict_week_date".equals(input)) { + formatter = StrictISODateTimeFormat.weekDate(); + } else if ("strictWeekDateTime".equals(input) || "strict_week_date_time".equals(input)) { + formatter = StrictISODateTimeFormat.weekDateTime(); + } else if ("strictWeekDateTimeNoMillis".equals(input) || "strict_week_date_time_no_millis".equals(input)) { + formatter = StrictISODateTimeFormat.weekDateTimeNoMillis(); + } else if ("strictWeekyear".equals(input) || "strict_weekyear".equals(input)) { + formatter = StrictISODateTimeFormat.weekyear(); + } else if ("strictWeekyearWeek".equals(input) || "strict_weekyear_week".equals(input)) { + formatter = StrictISODateTimeFormat.weekyearWeek(); + } else if ("strictWeekyearWeekDay".equals(input) || "strict_weekyear_week_day".equals(input)) { + formatter = StrictISODateTimeFormat.weekyearWeekDay(); + } else if ("strictYear".equals(input) || "strict_year".equals(input)) { + formatter = StrictISODateTimeFormat.year(); + } else if ("strictYearMonth".equals(input) || "strict_year_month".equals(input)) { + formatter = StrictISODateTimeFormat.yearMonth(); + } else if ("strictYearMonthDay".equals(input) || "strict_year_month_day".equals(input)) { + formatter = StrictISODateTimeFormat.yearMonthDay(); } else if (Strings.hasLength(input) && input.contains("||")) { String[] formats = Strings.delimitedListToStringArray(input, "||"); DateTimeParser[] parsers = new DateTimeParser[formats.length]; @@ -171,6 +248,38 @@ public static FormatDateTimeFormatter forPattern(String input, Locale locale) { return new FormatDateTimeFormatter(input, formatter.withZone(DateTimeZone.UTC), locale); } + public static FormatDateTimeFormatter getStrictStandardDateFormatter() { + // 2014/10/10 + DateTimeFormatter shortFormatter = new DateTimeFormatterBuilder() + .appendFixedDecimal(DateTimeFieldType.year(), 4) + .appendLiteral('/') + .appendFixedDecimal(DateTimeFieldType.monthOfYear(), 2) + .appendLiteral('/') + .appendFixedDecimal(DateTimeFieldType.dayOfMonth(), 2) + .toFormatter() + .withZoneUTC(); + + // 2014/10/10 12:12:12 + DateTimeFormatter longFormatter = new DateTimeFormatterBuilder() + .appendFixedDecimal(DateTimeFieldType.year(), 4) + .appendLiteral('/') + .appendFixedDecimal(DateTimeFieldType.monthOfYear(), 2) + .appendLiteral('/') + .appendFixedDecimal(DateTimeFieldType.dayOfMonth(), 2) + .appendLiteral(' ') + .appendFixedSignedDecimal(DateTimeFieldType.hourOfDay(), 2) + .appendLiteral(':') + .appendFixedSignedDecimal(DateTimeFieldType.minuteOfHour(), 2) + .appendLiteral(':') + .appendFixedSignedDecimal(DateTimeFieldType.secondOfMinute(), 2) + .toFormatter() + .withZoneUTC(); + + DateTimeFormatterBuilder builder = new DateTimeFormatterBuilder().append(longFormatter.withZone(DateTimeZone.UTC).getPrinter(), new DateTimeParser[] {longFormatter.getParser(), shortFormatter.getParser()}); + + return new FormatDateTimeFormatter("yyyy/MM/dd HH:mm:ss||yyyy/MM/dd", builder.toFormatter().withZone(DateTimeZone.UTC), Locale.ROOT); + } + public static final DurationFieldType Quarters = new DurationFieldType("quarters") { private static final long serialVersionUID = -8167713675442491871L; diff --git a/core/src/main/java/org/elasticsearch/common/lucene/search/XMoreLikeThis.java b/core/src/main/java/org/elasticsearch/common/lucene/search/XMoreLikeThis.java index b3aeb34a19854..4999f2a7cf439 100644 --- a/core/src/main/java/org/elasticsearch/common/lucene/search/XMoreLikeThis.java +++ b/core/src/main/java/org/elasticsearch/common/lucene/search/XMoreLikeThis.java @@ -39,10 +39,7 @@ import org.apache.lucene.analysis.tokenattributes.CharTermAttribute; import org.apache.lucene.document.Document; import org.apache.lucene.index.*; -import org.apache.lucene.search.BooleanClause; -import org.apache.lucene.search.BooleanQuery; -import org.apache.lucene.search.Query; -import org.apache.lucene.search.TermQuery; +import org.apache.lucene.search.*; import org.apache.lucene.search.similarities.DefaultSimilarity; import org.apache.lucene.search.similarities.TFIDFSimilarity; import org.apache.lucene.util.BytesRef; @@ -855,9 +852,12 @@ private void addTermFrequencies(Map termFreqMap, Terms vector, @Nul continue; } - PostingsEnum docs = termsEnum.postings(null, null); - final int freq = docs.freq(); - + final PostingsEnum docs = termsEnum.postings(null, null); + int freq = 0; + while(docs != null && docs.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) { + freq += docs.freq(); + } + // increment frequency Int cnt = termFreqMap.get(term); if (cnt == null) { diff --git a/core/src/main/java/org/elasticsearch/common/settings/Settings.java b/core/src/main/java/org/elasticsearch/common/settings/Settings.java index 24743afe980ec..b3a55fd324570 100644 --- a/core/src/main/java/org/elasticsearch/common/settings/Settings.java +++ b/core/src/main/java/org/elasticsearch/common/settings/Settings.java @@ -22,6 +22,7 @@ import com.google.common.base.Charsets; import com.google.common.base.Predicate; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSortedMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -81,7 +82,9 @@ public static boolean getSettingsRequireUnits() { private transient ClassLoader classLoader; Settings(Map settings, ClassLoader classLoader) { - this.settings = ImmutableMap.copyOf(settings); + // we use a sorted map for consistent serialization when using getAsMap() + // TODO: use Collections.unmodifiableMap with a TreeMap + this.settings = ImmutableSortedMap.copyOf(settings); Map forcedUnderscoreSettings = null; for (Map.Entry entry : settings.entrySet()) { String toUnderscoreCase = Strings.toUnderscoreCase(entry.getKey()); diff --git a/core/src/main/java/org/elasticsearch/common/util/concurrent/AtomicArray.java b/core/src/main/java/org/elasticsearch/common/util/concurrent/AtomicArray.java index 93644d0d80ff1..38953c51b02cc 100644 --- a/core/src/main/java/org/elasticsearch/common/util/concurrent/AtomicArray.java +++ b/core/src/main/java/org/elasticsearch/common/util/concurrent/AtomicArray.java @@ -67,6 +67,15 @@ public void set(int i, E value) { } } + public final void setOnce(int i, E value) { + if (array.compareAndSet(i, null, value) == false) { + throw new IllegalStateException("index [" + i + "] has already been set"); + } + if (nonNullList != null) { // read first, lighter, and most times it will be null... + nonNullList = null; + } + } + /** * Gets the current value at position {@code i}. * diff --git a/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java b/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java index 44090eff69d39..00c2c2437a2aa 100644 --- a/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java +++ b/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java @@ -41,8 +41,8 @@ import org.elasticsearch.index.settings.IndexSettings; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.FsDirectoryService; -import org.elasticsearch.monitor.fs.FsStats; -import org.elasticsearch.monitor.fs.JmxFsProbe; +import org.elasticsearch.monitor.fs.FsInfo; +import org.elasticsearch.monitor.fs.FsProbe; import java.io.Closeable; import java.io.IOException; @@ -225,38 +225,37 @@ private void maybeLogPathDetails() throws IOException { spinsDesc = "no"; } - FsStats.Info fsInfo = JmxFsProbe.getFSInfo(nodePath); + FsInfo.Path fsPath = FsProbe.getFSInfo(nodePath); sb.append(", free_space [") - .append(fsInfo.getFree()) + .append(fsPath.getFree()) .append("], usable_space [") - .append(fsInfo.getAvailable()) + .append(fsPath.getAvailable()) .append("], total_space [") - .append(fsInfo.getTotal()) + .append(fsPath.getTotal()) .append("], spins? [") .append(spinsDesc) .append("], mount [") - .append(fsInfo.getMount()) + .append(fsPath.getMount()) .append("], type [") - .append(fsInfo.getType()) + .append(fsPath.getType()) .append(']'); } logger.debug(sb.toString()); } else if (logger.isInfoEnabled()) { - FsStats.Info totFSInfo = new FsStats.Info(); + FsInfo.Path totFSPath = new FsInfo.Path(); Set allTypes = new HashSet<>(); Set allSpins = new HashSet<>(); Set allMounts = new HashSet<>(); for (NodePath nodePath : nodePaths) { - // TODO: can/should I use the chosen FsProbe instead (i.e. sigar if it's available)? - FsStats.Info fsInfo = JmxFsProbe.getFSInfo(nodePath); - String mount = fsInfo.getMount(); + FsInfo.Path fsPath = FsProbe.getFSInfo(nodePath); + String mount = fsPath.getMount(); if (allMounts.contains(mount) == false) { allMounts.add(mount); - String type = fsInfo.getType(); + String type = fsPath.getType(); if (type != null) { allTypes.add(type); } - Boolean spins = fsInfo.getSpins(); + Boolean spins = fsPath.getSpins(); if (spins == null) { allSpins.add("unknown"); } else if (spins.booleanValue()) { @@ -264,7 +263,7 @@ private void maybeLogPathDetails() throws IOException { } else { allSpins.add("no"); } - totFSInfo.add(fsInfo); + totFSPath.add(fsPath); } } @@ -273,8 +272,8 @@ private void maybeLogPathDetails() throws IOException { "using [%d] data paths, mounts [%s], net usable_space [%s], net total_space [%s], spins? [%s], types [%s]", nodePaths.length, allMounts, - totFSInfo.getAvailable(), - totFSInfo.getTotal(), + totFSPath.getAvailable(), + totFSPath.getTotal(), toString(allSpins), toString(allTypes))); } diff --git a/core/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java b/core/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java index e24a83bac5aa7..e71ebc88e9ff0 100644 --- a/core/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java +++ b/core/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java @@ -147,8 +147,17 @@ public boolean allocateUnassigned(RoutingAllocation allocation) { RoutingNodes routingNodes = allocation.routingNodes(); // First, handle primaries, they must find a place to be allocated on here - MetaData metaData = routingNodes.metaData(); - Iterator unassignedIterator = routingNodes.unassigned().iterator(); + final MetaData metaData = routingNodes.metaData(); + RoutingNodes.UnassignedShards unassigned = routingNodes.unassigned(); + unassigned.sort(new PriorityComparator() { + + @Override + protected Settings getIndexSettings(String index) { + IndexMetaData indexMetaData = metaData.index(index); + return indexMetaData.getSettings(); + } + }); // sort for priority ordering + Iterator unassignedIterator = unassigned.iterator(); while (unassignedIterator.hasNext()) { ShardRouting shard = unassignedIterator.next(); @@ -368,7 +377,7 @@ public int compare(DiscoveryNode o1, DiscoveryNode o2) { } // Now, handle replicas, try to assign them to nodes that are similar to the one the primary was allocated on - unassignedIterator = routingNodes.unassigned().iterator(); + unassignedIterator = unassigned.iterator(); while (unassignedIterator.hasNext()) { ShardRouting shard = unassignedIterator.next(); if (shard.primary()) { @@ -542,4 +551,5 @@ protected void reroute(ShardId shardId, String reason) { routingService.reroute("async_shard_fetch"); } } + } diff --git a/core/src/main/java/org/elasticsearch/gateway/PriorityComparator.java b/core/src/main/java/org/elasticsearch/gateway/PriorityComparator.java new file mode 100644 index 0000000000000..2176a70c74b08 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/gateway/PriorityComparator.java @@ -0,0 +1,66 @@ +/* + * 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.gateway; + +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.common.settings.Settings; + +import java.util.Comparator; + +/** + * A comparator that compares ShardRouting based on it's indexes priority (index.priority), + * it's creation date (index.creation_date), or eventually by it's index name in reverse order. + * We try to recover first shards from an index with the highest priority, if that's the same + * we try to compare the timestamp the index is created and pick the newer first (time-based indices, + * here the newer indices matter more). If even that is the same, we compare the index name which is useful + * if the date is baked into the index name. ie logstash-2015.05.03. + */ +abstract class PriorityComparator implements Comparator { + + @Override + public final int compare(ShardRouting o1, ShardRouting o2) { + final String o1Index = o1.index(); + final String o2Index = o2.index(); + int cmp = 0; + if (o1Index.equals(o2Index) == false) { + final Settings settingsO1 = getIndexSettings(o1Index); + final Settings settingsO2 = getIndexSettings(o2Index); + cmp = Long.compare(priority(settingsO2), priority(settingsO1)); + if (cmp == 0) { + cmp = Long.compare(timeCreated(settingsO2), timeCreated(settingsO1)); + if (cmp == 0) { + cmp = o2Index.compareTo(o1Index); + } + } + } + return cmp; + } + + private int priority(Settings settings) { + return settings.getAsInt(IndexMetaData.SETTING_PRIORITY, 1); + } + + private long timeCreated(Settings settings) { + return settings.getAsLong(IndexMetaData.SETTING_CREATION_DATE, -1l); + } + + protected abstract Settings getIndexSettings(String index); +} diff --git a/core/src/main/java/org/elasticsearch/index/engine/Engine.java b/core/src/main/java/org/elasticsearch/index/engine/Engine.java index 0114833996671..f56afd4e07208 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -510,34 +510,36 @@ public void forceMerge(boolean flush) { */ public abstract SnapshotIndexCommit snapshotIndex(boolean flushFirst) throws EngineException; - /** fail engine due to some error. the engine will also be closed. */ - public void failEngine(String reason, Throwable failure) { - assert failure != null; + /** + * fail engine due to some error. the engine will also be closed. + * The underlying store is marked corrupted iff failure is caused by index corruption + */ + public void failEngine(String reason, @Nullable Throwable failure) { if (failEngineLock.tryLock()) { store.incRef(); try { try { // we just go and close this engine - no way to recover closeNoLock("engine failed on: [" + reason + "]"); + } finally { + if (failedEngine != null) { + logger.debug("tried to fail engine but engine is already failed. ignoring. [{}]", reason, failure); + return; + } + logger.warn("failed engine [{}]", failure, reason); + // we must set a failure exception, generate one if not supplied + failedEngine = (failure != null) ? failure : new IllegalStateException(reason); // we first mark the store as corrupted before we notify any listeners // this must happen first otherwise we might try to reallocate so quickly // on the same node that we don't see the corrupted marker file when // the shard is initializing if (Lucene.isCorruptionException(failure)) { try { - store.markStoreCorrupted(ExceptionsHelper.unwrapCorruption(failure)); + store.markStoreCorrupted(new IOException("failed engine (reason: [" + reason + "])", ExceptionsHelper.unwrapCorruption(failure))); } catch (IOException e) { - logger.warn("Couldn't marks store corrupted", e); + logger.warn("Couldn't mark store corrupted", e); } } - } finally { - if (failedEngine != null) { - logger.debug("tried to fail engine but engine is already failed. ignoring. [{}]", reason, failure); - return; - } - logger.warn("failed engine [{}]", failure, reason); - // we must set a failure exception, generate one if not supplied - failedEngine = failure; failedEngineListener.onFailedEngine(shardId, reason, failure); } } catch (Throwable t) { @@ -554,10 +556,10 @@ public void failEngine(String reason, Throwable failure) { /** Check whether the engine should be failed */ protected boolean maybeFailEngine(String source, Throwable t) { if (Lucene.isCorruptionException(t)) { - failEngine("corrupt file detected source: [" + source + "]", t); + failEngine("corrupt file (source: [" + source + "])", t); return true; } else if (ExceptionsHelper.isOOM(t)) { - failEngine("out of memory", t); + failEngine("out of memory (source: [" + source + "])", t); return true; } return false; diff --git a/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java index de201941ed7f7..1a847997e8429 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java @@ -399,10 +399,10 @@ public boolean isParent(String type) { return mapperService.getParentTypes().contains(type); } - private void addMappers(Collection objectMappers, Collection fieldMappers) { + private void addMappers(Collection objectMappers, Collection fieldMappers, boolean updateAllTypes) { assert mappingLock.isWriteLockedByCurrentThread(); // first ensure we don't have any incompatible new fields - mapperService.checkNewMappersCompatibility(objectMappers, fieldMappers, true); + mapperService.checkNewMappersCompatibility(objectMappers, fieldMappers, updateAllTypes); // update mappers for this document type MapBuilder builder = MapBuilder.newMapBuilder(this.objectMappers); @@ -424,7 +424,7 @@ public MergeResult merge(Mapping mapping, boolean simulate, boolean updateAllTyp final MergeResult mergeResult = new MergeResult(simulate, updateAllTypes); this.mapping.merge(mapping, mergeResult); if (simulate == false) { - addMappers(mergeResult.getNewObjectMappers(), mergeResult.getNewFieldMappers()); + addMappers(mergeResult.getNewObjectMappers(), mergeResult.getNewFieldMappers(), updateAllTypes); refreshSource(); } return mergeResult; diff --git a/core/src/main/java/org/elasticsearch/index/mapper/FieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/FieldMapper.java index 0a95e01bee2fb..78302804b1d7f 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/FieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/FieldMapper.java @@ -19,45 +19,725 @@ package org.elasticsearch.index.mapper; -import org.elasticsearch.index.mapper.core.AbstractFieldMapper; +import com.carrotsearch.hppc.cursors.ObjectCursor; +import com.carrotsearch.hppc.cursors.ObjectObjectCursor; +import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterators; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.FieldType; +import org.apache.lucene.index.IndexOptions; +import org.elasticsearch.Version; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.lucene.Lucene; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.index.analysis.NamedAnalyzer; +import org.elasticsearch.index.fielddata.FieldDataType; +import org.elasticsearch.index.mapper.core.TypeParsers; +import org.elasticsearch.index.mapper.internal.AllFieldMapper; +import org.elasticsearch.index.similarity.SimilarityLookupService; +import org.elasticsearch.index.similarity.SimilarityProvider; import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Locale; public abstract class FieldMapper extends Mapper { - public FieldMapper(String simpleName) { + public abstract static class Builder extends Mapper.Builder { + + protected final MappedFieldType fieldType; + protected final MappedFieldType defaultFieldType; + private final IndexOptions defaultOptions; + protected boolean omitNormsSet = false; + protected String indexName; + protected Boolean includeInAll; + protected boolean indexOptionsSet = false; + protected boolean docValuesSet = false; + @Nullable + protected Settings fieldDataSettings; + protected final MultiFields.Builder multiFieldsBuilder; + protected CopyTo copyTo; + + protected Builder(String name, MappedFieldType fieldType) { + super(name); + this.fieldType = fieldType.clone(); + this.defaultFieldType = fieldType.clone(); + this.defaultOptions = fieldType.indexOptions(); // we have to store it the fieldType is mutable + multiFieldsBuilder = new MultiFields.Builder(); + } + + public MappedFieldType fieldType() { + return fieldType; + } + + public T index(boolean index) { + if (index) { + if (fieldType.indexOptions() == IndexOptions.NONE) { + /* + * the logic here is to reset to the default options only if we are not indexed ie. options are null + * if the fieldType has a non-null option we are all good it might have been set through a different + * call. + */ + final IndexOptions options = getDefaultIndexOption(); + assert options != IndexOptions.NONE : "default IndexOptions is NONE can't enable indexing"; + fieldType.setIndexOptions(options); + } + } else { + fieldType.setIndexOptions(IndexOptions.NONE); + } + return builder; + } + + protected IndexOptions getDefaultIndexOption() { + return defaultOptions; + } + + public T store(boolean store) { + this.fieldType.setStored(store); + return builder; + } + + public T docValues(boolean docValues) { + this.fieldType.setHasDocValues(docValues); + this.docValuesSet = true; + return builder; + } + + public T storeTermVectors(boolean termVectors) { + if (termVectors != this.fieldType.storeTermVectors()) { + this.fieldType.setStoreTermVectors(termVectors); + } // don't set it to false, it is default and might be flipped by a more specific option + return builder; + } + + public T storeTermVectorOffsets(boolean termVectorOffsets) { + if (termVectorOffsets) { + this.fieldType.setStoreTermVectors(termVectorOffsets); + } + this.fieldType.setStoreTermVectorOffsets(termVectorOffsets); + return builder; + } + + public T storeTermVectorPositions(boolean termVectorPositions) { + if (termVectorPositions) { + this.fieldType.setStoreTermVectors(termVectorPositions); + } + this.fieldType.setStoreTermVectorPositions(termVectorPositions); + return builder; + } + + public T storeTermVectorPayloads(boolean termVectorPayloads) { + if (termVectorPayloads) { + this.fieldType.setStoreTermVectors(termVectorPayloads); + } + this.fieldType.setStoreTermVectorPayloads(termVectorPayloads); + return builder; + } + + public T tokenized(boolean tokenized) { + this.fieldType.setTokenized(tokenized); + return builder; + } + + public T boost(float boost) { + this.fieldType.setBoost(boost); + return builder; + } + + public T omitNorms(boolean omitNorms) { + this.fieldType.setOmitNorms(omitNorms); + this.omitNormsSet = true; + return builder; + } + + public T indexOptions(IndexOptions indexOptions) { + this.fieldType.setIndexOptions(indexOptions); + this.indexOptionsSet = true; + return builder; + } + + public T indexName(String indexName) { + this.indexName = indexName; + return builder; + } + + public T indexAnalyzer(NamedAnalyzer indexAnalyzer) { + this.fieldType.setIndexAnalyzer(indexAnalyzer); + return builder; + } + + public T searchAnalyzer(NamedAnalyzer searchAnalyzer) { + this.fieldType.setSearchAnalyzer(searchAnalyzer); + return builder; + } + + public T includeInAll(Boolean includeInAll) { + this.includeInAll = includeInAll; + return builder; + } + + public T similarity(SimilarityProvider similarity) { + this.fieldType.setSimilarity(similarity); + return builder; + } + + public T normsLoading(MappedFieldType.Loading normsLoading) { + this.fieldType.setNormsLoading(normsLoading); + return builder; + } + + public T fieldDataSettings(Settings settings) { + this.fieldDataSettings = settings; + return builder; + } + + public Builder nullValue(Object nullValue) { + this.fieldType.setNullValue(nullValue); + return this; + } + + public T multiFieldPathType(ContentPath.Type pathType) { + multiFieldsBuilder.pathType(pathType); + return builder; + } + + public T addMultiField(Mapper.Builder mapperBuilder) { + multiFieldsBuilder.add(mapperBuilder); + return builder; + } + + public T copyTo(CopyTo copyTo) { + this.copyTo = copyTo; + return builder; + } + + protected MappedFieldType.Names buildNames(BuilderContext context) { + return new MappedFieldType.Names(buildIndexName(context), buildIndexNameClean(context), buildFullName(context)); + } + + protected String buildIndexName(BuilderContext context) { + if (context.indexCreatedVersion().onOrAfter(Version.V_2_0_0)) { + return buildFullName(context); + } + String actualIndexName = indexName == null ? name : indexName; + return context.path().pathAsText(actualIndexName); + } + + protected String buildIndexNameClean(BuilderContext context) { + if (context.indexCreatedVersion().onOrAfter(Version.V_2_0_0)) { + return buildFullName(context); + } + return indexName == null ? name : indexName; + } + + protected String buildFullName(BuilderContext context) { + return context.path().fullPathAsText(name); + } + + protected void setupFieldType(BuilderContext context) { + fieldType.setNames(buildNames(context)); + if (fieldType.indexAnalyzer() == null && fieldType.tokenized() == false && fieldType.indexOptions() != IndexOptions.NONE) { + fieldType.setIndexAnalyzer(Lucene.KEYWORD_ANALYZER); + fieldType.setSearchAnalyzer(Lucene.KEYWORD_ANALYZER); + } + if (fieldDataSettings != null) { + Settings settings = Settings.builder().put(fieldType.fieldDataType().getSettings()).put(fieldDataSettings).build(); + fieldType.setFieldDataType(new FieldDataType(fieldType.fieldDataType().getType(), settings)); + } + boolean defaultDocValues = false; // pre 2.0 + if (context.indexCreatedVersion().onOrAfter(Version.V_2_0_0)) { + defaultDocValues = fieldType.tokenized() == false && fieldType.indexOptions() != IndexOptions.NONE; + } + // backcompat for "fielddata: format: docvalues" for now... + boolean fieldDataDocValues = fieldType.fieldDataType() != null + && FieldDataType.DOC_VALUES_FORMAT_VALUE.equals(fieldType.fieldDataType().getFormat(context.indexSettings())); + if (fieldDataDocValues && docValuesSet && fieldType.hasDocValues() == false) { + // this forces the doc_values setting to be written, so fielddata does not mask the original setting + defaultDocValues = true; + } + defaultFieldType.setHasDocValues(defaultDocValues); + if (docValuesSet == false) { + fieldType.setHasDocValues(defaultDocValues || fieldDataDocValues); + } + } + } + + protected MappedFieldTypeReference fieldTypeRef; + protected final MappedFieldType defaultFieldType; + protected final MultiFields multiFields; + protected CopyTo copyTo; + protected final boolean indexCreatedBefore2x; + + protected FieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { super(simpleName); + assert indexSettings != null; + this.indexCreatedBefore2x = Version.indexCreated(indexSettings).before(Version.V_2_0_0); + this.fieldTypeRef = new MappedFieldTypeReference(fieldType); // the reference ctor freezes the field type + defaultFieldType.freeze(); + this.defaultFieldType = defaultFieldType; + this.multiFields = multiFields; + this.copyTo = copyTo; + } + + @Override + public String name() { + return fieldType().names().fullName(); } - public abstract MappedFieldType fieldType(); + public MappedFieldType fieldType() { + return fieldTypeRef.get(); + } /** Returns a reference to the MappedFieldType for this mapper. */ - public abstract MappedFieldTypeReference fieldTypeReference(); + public MappedFieldTypeReference fieldTypeReference() { + return fieldTypeRef; + } /** * Updates the reference to this field's MappedFieldType. * Implementations should assert equality of the underlying field type */ - public abstract void setFieldTypeReference(MappedFieldTypeReference ref); + public void setFieldTypeReference(MappedFieldTypeReference ref) { + if (ref.get().equals(fieldType()) == false) { + throw new IllegalStateException("Cannot overwrite field type reference to unequal reference"); + } + ref.incrementAssociatedMappers(); + this.fieldTypeRef = ref; + } /** * List of fields where this field should be copied to */ - public abstract AbstractFieldMapper.CopyTo copyTo(); + public CopyTo copyTo() { + return copyTo; + } + + /** + * Parse using the provided {@link ParseContext} and return a mapping + * update if dynamic mappings modified the mappings, or {@code null} if + * mappings were not modified. + */ + public Mapper parse(ParseContext context) throws IOException { + final List fields = new ArrayList<>(2); + try { + parseCreateField(context, fields); + for (Field field : fields) { + if (!customBoost()) { + field.setBoost(fieldType().boost()); + } + context.doc().add(field); + } + } catch (Exception e) { + throw new MapperParsingException("failed to parse [" + fieldType().names().fullName() + "]", e); + } + multiFields.parse(this, context); + return null; + } + + /** + * Parse the field value and populate fields. + */ + protected abstract void parseCreateField(ParseContext context, List fields) throws IOException; + + /** + * Derived classes can override it to specify that boost value is set by derived classes. + */ + protected boolean customBoost() { + return false; + } + + public Iterator iterator() { + if (multiFields == null) { + return Collections.emptyIterator(); + } + return multiFields.iterator(); + } + + @Override + public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException { + if (!this.getClass().equals(mergeWith.getClass())) { + String mergedType = mergeWith.getClass().getSimpleName(); + if (mergeWith instanceof FieldMapper) { + mergedType = ((FieldMapper) mergeWith).contentType(); + } + mergeResult.addConflict("mapper [" + fieldType().names().fullName() + "] of different type, current_type [" + contentType() + "], merged_type [" + mergedType + "]"); + // different types, return + return; + } + FieldMapper fieldMergeWith = (FieldMapper) mergeWith; + List subConflicts = new ArrayList<>(); // TODO: just expose list from MergeResult? + fieldType().checkTypeName(fieldMergeWith.fieldType(), subConflicts); + if (subConflicts.isEmpty() == false) { + // return early if field types don't match + assert subConflicts.size() == 1; + mergeResult.addConflict(subConflicts.get(0)); + return; + } + + boolean strict = this.fieldTypeRef.getNumAssociatedMappers() > 1 && mergeResult.updateAllTypes() == false; + fieldType().checkCompatibility(fieldMergeWith.fieldType(), subConflicts, strict); + for (String conflict : subConflicts) { + mergeResult.addConflict(conflict); + } + multiFields.merge(mergeWith, mergeResult); + + if (mergeResult.simulate() == false && mergeResult.hasConflicts() == false) { + // apply changeable values + MappedFieldType fieldType = fieldMergeWith.fieldType().clone(); + fieldType.freeze(); + fieldTypeRef.set(fieldType); + this.copyTo = fieldMergeWith.copyTo; + } + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(simpleName()); + boolean includeDefaults = params.paramAsBoolean("include_defaults", false); + doXContentBody(builder, includeDefaults, params); + return builder.endObject(); + } + + protected void doXContentBody(XContentBuilder builder, boolean includeDefaults, Params params) throws IOException { + + builder.field("type", contentType()); + if (indexCreatedBefore2x && (includeDefaults || !simpleName().equals(fieldType().names().originalIndexName()))) { + builder.field("index_name", fieldType().names().originalIndexName()); + } + + if (includeDefaults || fieldType().boost() != 1.0f) { + builder.field("boost", fieldType().boost()); + } + + boolean indexed = fieldType().indexOptions() != IndexOptions.NONE; + boolean defaultIndexed = defaultFieldType.indexOptions() != IndexOptions.NONE; + if (includeDefaults || indexed != defaultIndexed || + fieldType().tokenized() != defaultFieldType.tokenized()) { + builder.field("index", indexTokenizeOptionToString(indexed, fieldType().tokenized())); + } + if (includeDefaults || fieldType().stored() != defaultFieldType.stored()) { + builder.field("store", fieldType().stored()); + } + doXContentDocValues(builder, includeDefaults); + if (includeDefaults || fieldType().storeTermVectors() != defaultFieldType.storeTermVectors()) { + builder.field("term_vector", termVectorOptionsToString(fieldType())); + } + if (includeDefaults || fieldType().omitNorms() != defaultFieldType.omitNorms() || fieldType().normsLoading() != null) { + builder.startObject("norms"); + if (includeDefaults || fieldType().omitNorms() != defaultFieldType.omitNorms()) { + builder.field("enabled", !fieldType().omitNorms()); + } + if (fieldType().normsLoading() != null) { + builder.field(MappedFieldType.Loading.KEY, fieldType().normsLoading()); + } + builder.endObject(); + } + if (indexed && (includeDefaults || fieldType().indexOptions() != defaultFieldType.indexOptions())) { + builder.field("index_options", indexOptionToString(fieldType().indexOptions())); + } + + doXContentAnalyzers(builder, includeDefaults); + + if (fieldType().similarity() != null) { + builder.field("similarity", fieldType().similarity().name()); + } else if (includeDefaults) { + builder.field("similarity", SimilarityLookupService.DEFAULT_SIMILARITY); + } + + if (includeDefaults || hasCustomFieldDataSettings()) { + builder.field("fielddata", fieldType().fieldDataType().getSettings().getAsMap()); + } + multiFields.toXContent(builder, params); + + if (copyTo != null) { + copyTo.toXContent(builder, params); + } + } + + protected void doXContentAnalyzers(XContentBuilder builder, boolean includeDefaults) throws IOException { + if (fieldType().indexAnalyzer() == null) { + if (includeDefaults) { + builder.field("analyzer", "default"); + } + } else if (includeDefaults || fieldType().indexAnalyzer().name().startsWith("_") == false && fieldType().indexAnalyzer().name().equals("default") == false) { + builder.field("analyzer", fieldType().indexAnalyzer().name()); + if (fieldType().searchAnalyzer().name().equals(fieldType().indexAnalyzer().name()) == false) { + builder.field("search_analyzer", fieldType().searchAnalyzer().name()); + } + } + } + + protected void doXContentDocValues(XContentBuilder builder, boolean includeDefaults) throws IOException { + if (includeDefaults || defaultFieldType.hasDocValues() != fieldType().hasDocValues()) { + builder.field("doc_values", fieldType().hasDocValues()); + } + } + + protected static String indexOptionToString(IndexOptions indexOption) { + switch (indexOption) { + case DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS: + return TypeParsers.INDEX_OPTIONS_OFFSETS; + case DOCS_AND_FREQS: + return TypeParsers.INDEX_OPTIONS_FREQS; + case DOCS_AND_FREQS_AND_POSITIONS: + return TypeParsers.INDEX_OPTIONS_POSITIONS; + case DOCS: + return TypeParsers.INDEX_OPTIONS_DOCS; + default: + throw new IllegalArgumentException("Unknown IndexOptions [" + indexOption + "]"); + } + } + + public static String termVectorOptionsToString(FieldType fieldType) { + if (!fieldType.storeTermVectors()) { + return "no"; + } else if (!fieldType.storeTermVectorOffsets() && !fieldType.storeTermVectorPositions()) { + return "yes"; + } else if (fieldType.storeTermVectorOffsets() && !fieldType.storeTermVectorPositions()) { + return "with_offsets"; + } else { + StringBuilder builder = new StringBuilder("with"); + if (fieldType.storeTermVectorPositions()) { + builder.append("_positions"); + } + if (fieldType.storeTermVectorOffsets()) { + builder.append("_offsets"); + } + if (fieldType.storeTermVectorPayloads()) { + builder.append("_payloads"); + } + return builder.toString(); + } + } + + protected static String indexTokenizeOptionToString(boolean indexed, boolean tokenized) { + if (!indexed) { + return "no"; + } else if (tokenized) { + return "analyzed"; + } else { + return "not_analyzed"; + } + } + + protected boolean hasCustomFieldDataSettings() { + return fieldType().fieldDataType() != null && fieldType().fieldDataType().equals(defaultFieldType.fieldDataType()) == false; + } + + protected abstract String contentType(); + + public static class MultiFields { + + public static MultiFields empty() { + return new MultiFields(ContentPath.Type.FULL, ImmutableOpenMap.of()); + } + + public static class Builder { + + private final ImmutableOpenMap.Builder mapperBuilders = ImmutableOpenMap.builder(); + private ContentPath.Type pathType = ContentPath.Type.FULL; + + public Builder pathType(ContentPath.Type pathType) { + this.pathType = pathType; + return this; + } + + public Builder add(Mapper.Builder builder) { + mapperBuilders.put(builder.name(), builder); + return this; + } + + @SuppressWarnings("unchecked") + public MultiFields build(FieldMapper.Builder mainFieldBuilder, BuilderContext context) { + if (pathType == ContentPath.Type.FULL && mapperBuilders.isEmpty()) { + return empty(); + } else if (mapperBuilders.isEmpty()) { + return new MultiFields(pathType, ImmutableOpenMap.of()); + } else { + ContentPath.Type origPathType = context.path().pathType(); + context.path().pathType(pathType); + context.path().add(mainFieldBuilder.name()); + ImmutableOpenMap.Builder mapperBuilders = this.mapperBuilders; + for (ObjectObjectCursor cursor : this.mapperBuilders) { + String key = cursor.key; + Mapper.Builder value = cursor.value; + Mapper mapper = value.build(context); + assert mapper instanceof FieldMapper; + mapperBuilders.put(key, mapper); + } + context.path().remove(); + context.path().pathType(origPathType); + ImmutableOpenMap.Builder mappers = mapperBuilders.cast(); + return new MultiFields(pathType, mappers.build()); + } + } + } + + private final ContentPath.Type pathType; + private volatile ImmutableOpenMap mappers; + + public MultiFields(ContentPath.Type pathType, ImmutableOpenMap mappers) { + this.pathType = pathType; + this.mappers = mappers; + // we disable the all in multi-field mappers + for (ObjectCursor cursor : mappers.values()) { + FieldMapper mapper = cursor.value; + if (mapper instanceof AllFieldMapper.IncludeInAll) { + ((AllFieldMapper.IncludeInAll) mapper).unsetIncludeInAll(); + } + } + } + + public void parse(FieldMapper mainField, ParseContext context) throws IOException { + // TODO: multi fields are really just copy fields, we just need to expose "sub fields" or something that can be part of the mappings + if (mappers.isEmpty()) { + return; + } + + context = context.createMultiFieldContext(); + + ContentPath.Type origPathType = context.path().pathType(); + context.path().pathType(pathType); + + context.path().add(mainField.simpleName()); + for (ObjectCursor cursor : mappers.values()) { + cursor.value.parse(context); + } + context.path().remove(); + context.path().pathType(origPathType); + } + + // No need for locking, because locking is taken care of in ObjectMapper#merge and DocumentMapper#merge + public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException { + FieldMapper mergeWithMultiField = (FieldMapper) mergeWith; + + List newFieldMappers = null; + ImmutableOpenMap.Builder newMappersBuilder = null; + + for (ObjectCursor cursor : mergeWithMultiField.multiFields.mappers.values()) { + FieldMapper mergeWithMapper = cursor.value; + Mapper mergeIntoMapper = mappers.get(mergeWithMapper.simpleName()); + if (mergeIntoMapper == null) { + // no mapping, simply add it if not simulating + if (!mergeResult.simulate()) { + // we disable the all in multi-field mappers + if (mergeWithMapper instanceof AllFieldMapper.IncludeInAll) { + ((AllFieldMapper.IncludeInAll) mergeWithMapper).unsetIncludeInAll(); + } + if (newMappersBuilder == null) { + newMappersBuilder = ImmutableOpenMap.builder(mappers); + } + newMappersBuilder.put(mergeWithMapper.simpleName(), mergeWithMapper); + if (mergeWithMapper instanceof FieldMapper) { + if (newFieldMappers == null) { + newFieldMappers = new ArrayList<>(2); + } + newFieldMappers.add(mergeWithMapper); + } + } + } else { + mergeIntoMapper.merge(mergeWithMapper, mergeResult); + } + } + + // first add all field mappers + if (newFieldMappers != null) { + mergeResult.addFieldMappers(newFieldMappers); + } + // now publish mappers + if (newMappersBuilder != null) { + mappers = newMappersBuilder.build(); + } + } + + public Iterator iterator() { + return Iterators.transform(mappers.values().iterator(), new Function, Mapper>() { + @Override + public Mapper apply(@Nullable ObjectCursor cursor) { + return cursor.value; + } + }); + } + + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + if (pathType != ContentPath.Type.FULL) { + builder.field("path", pathType.name().toLowerCase(Locale.ROOT)); + } + if (!mappers.isEmpty()) { + // sort the mappers so we get consistent serialization format + Mapper[] sortedMappers = mappers.values().toArray(Mapper.class); + Arrays.sort(sortedMappers, new Comparator() { + @Override + public int compare(Mapper o1, Mapper o2) { + return o1.name().compareTo(o2.name()); + } + }); + builder.startObject("fields"); + for (Mapper mapper : sortedMappers) { + mapper.toXContent(builder, params); + } + builder.endObject(); + } + return builder; + } + } + + /** + * Represents a list of fields with optional boost factor where the current field should be copied to + */ + public static class CopyTo { + + private final ImmutableList copyToFields; + + private CopyTo(ImmutableList copyToFields) { + this.copyToFields = copyToFields; + } + + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + if (!copyToFields.isEmpty()) { + builder.startArray("copy_to"); + for (String field : copyToFields) { + builder.value(field); + } + builder.endArray(); + } + return builder; + } + + public static class Builder { + private final ImmutableList.Builder copyToBuilders = ImmutableList.builder(); + + public Builder add(String field) { + copyToBuilders.add(field); + return this; + } + + public CopyTo build() { + return new CopyTo(copyToBuilders.build()); + } + } + + public List copyToFields() { + return copyToFields; + } + } /** * Fields might not be available before indexing, for example _all, token_count,... * When get is called and these fields are requested, this case needs special treatment. * * @return If the field is available before indexing or not. - * */ - public abstract boolean isGenerated(); - - /** - * Parse using the provided {@link ParseContext} and return a mapping - * update if dynamic mappings modified the mappings, or {@code null} if - * mappings were not modified. */ - public abstract Mapper parse(ParseContext context) throws IOException; + public boolean isGenerated() { + return false; + } } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java b/core/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java index f0d0483de1605..152bdca757537 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java @@ -25,18 +25,12 @@ import org.apache.lucene.index.Term; import org.apache.lucene.index.Terms; import org.apache.lucene.queries.TermsQuery; -import org.apache.lucene.search.ConstantScoreQuery; -import org.apache.lucene.search.FuzzyQuery; -import org.apache.lucene.search.MultiTermQuery; -import org.apache.lucene.search.PrefixQuery; -import org.apache.lucene.search.Query; -import org.apache.lucene.search.RegexpQuery; -import org.apache.lucene.search.TermQuery; -import org.apache.lucene.search.TermRangeQuery; +import org.apache.lucene.search.*; import org.apache.lucene.util.BytesRef; import org.elasticsearch.action.fieldstats.FieldStats; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.lucene.BytesRefs; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.Fuzziness; import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.fielddata.FieldDataType; @@ -189,6 +183,7 @@ public MappedFieldType() { setOmitNorms(false); setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS); setBoost(1.0f); + fieldDataType = new FieldDataType(typeName()); } public abstract MappedFieldType clone(); @@ -461,8 +456,8 @@ public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower includeLower, includeUpper); } - public Query fuzzyQuery(String value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions) { - return new FuzzyQuery(createTerm(value), fuzziness.asDistance(value), prefixLength, maxExpansions, transpositions); + public Query fuzzyQuery(Object value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions) { + return new FuzzyQuery(createTerm(value), fuzziness.asDistance(BytesRefs.toString(value)), prefixLength, maxExpansions, transpositions); } public Query prefixQuery(Object value, @Nullable MultiTermQuery.RewriteMethod method, @Nullable QueryParseContext context) { diff --git a/core/src/main/java/org/elasticsearch/index/mapper/Mapper.java b/core/src/main/java/org/elasticsearch/index/mapper/Mapper.java index a183f8a67db01..eaeea7ae396c3 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/Mapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/Mapper.java @@ -139,7 +139,7 @@ public Mapper(String simpleName) { } /** Returns the simple name, which identifies this mapper against other mappers at the same level in the mappers hierarchy - * TODO: make this protected once Mapper, FieldMapper and AbstractFieldMapper are merged together */ + * TODO: make this protected once Mapper and FieldMapper are merged together */ public final String simpleName() { return simpleName; } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/MetadataFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/MetadataFieldMapper.java index 541b4fdf645c1..34e35131ba319 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/MetadataFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/MetadataFieldMapper.java @@ -19,9 +19,7 @@ package org.elasticsearch.index.mapper; -import org.elasticsearch.common.Nullable; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.index.mapper.core.AbstractFieldMapper; import org.elasticsearch.index.mapper.object.RootObjectMapper; import java.io.IOException; @@ -30,16 +28,16 @@ /** * A mapper for a builtin field containing metadata about a document. */ -public abstract class MetadataFieldMapper extends AbstractFieldMapper { +public abstract class MetadataFieldMapper extends FieldMapper { - public abstract static class Builder extends AbstractFieldMapper.Builder { + public abstract static class Builder extends FieldMapper.Builder { public Builder(String name, MappedFieldType fieldType) { super(name, fieldType); } } - protected MetadataFieldMapper(String simpleName, MappedFieldType fieldType, Boolean docValues, @Nullable Settings fieldDataSettings, Settings indexSettings) { - super(simpleName, fieldType, docValues, fieldDataSettings, indexSettings, MultiFields.empty(), null); + protected MetadataFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Settings indexSettings) { + super(simpleName, fieldType, defaultFieldType, indexSettings, MultiFields.empty(), null); } /** diff --git a/core/src/main/java/org/elasticsearch/index/mapper/core/AbstractFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/core/AbstractFieldMapper.java deleted file mode 100644 index cc62dc412cb69..0000000000000 --- a/core/src/main/java/org/elasticsearch/index/mapper/core/AbstractFieldMapper.java +++ /dev/null @@ -1,770 +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.core; - -import com.carrotsearch.hppc.cursors.ObjectCursor; -import com.carrotsearch.hppc.cursors.ObjectObjectCursor; -import com.google.common.base.Function; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterators; -import org.apache.lucene.document.Field; -import org.apache.lucene.document.FieldType; -import org.apache.lucene.index.IndexOptions; -import org.elasticsearch.Version; -import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.collect.ImmutableOpenMap; -import org.elasticsearch.common.lucene.Lucene; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.index.analysis.NamedAnalyzer; -import org.elasticsearch.index.fielddata.FieldDataType; -import org.elasticsearch.index.mapper.ContentPath; -import org.elasticsearch.index.mapper.FieldMapper; -import org.elasticsearch.index.mapper.MappedFieldType; -import org.elasticsearch.index.mapper.MappedFieldTypeReference; -import org.elasticsearch.index.mapper.Mapper; -import org.elasticsearch.index.mapper.MapperParsingException; -import org.elasticsearch.index.mapper.MergeMappingException; -import org.elasticsearch.index.mapper.MergeResult; -import org.elasticsearch.index.mapper.ParseContext; -import org.elasticsearch.index.mapper.internal.AllFieldMapper; -import org.elasticsearch.index.similarity.SimilarityLookupService; -import org.elasticsearch.index.similarity.SimilarityProvider; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; -import java.util.Locale; -import java.util.TreeMap; - -import static org.elasticsearch.index.mapper.core.TypeParsers.DOC_VALUES; - -public abstract class AbstractFieldMapper extends FieldMapper { - - public static class Defaults { - public static final float BOOST = 1.0f; - public static final ContentPath.Type PATH_TYPE = ContentPath.Type.FULL; - } - - public abstract static class Builder extends Mapper.Builder { - - protected final MappedFieldType fieldType; - private final IndexOptions defaultOptions; - protected Boolean docValues; - protected boolean omitNormsSet = false; - protected String indexName; - protected Boolean includeInAll; - protected boolean indexOptionsSet = false; - @Nullable - protected Settings fieldDataSettings; - protected final MultiFields.Builder multiFieldsBuilder; - protected CopyTo copyTo; - - protected Builder(String name, MappedFieldType fieldType) { - super(name); - this.fieldType = fieldType.clone(); - this.defaultOptions = fieldType.indexOptions(); // we have to store it the fieldType is mutable - multiFieldsBuilder = new MultiFields.Builder(); - } - - public T index(boolean index) { - if (index) { - if (fieldType.indexOptions() == IndexOptions.NONE) { - /* - * the logic here is to reset to the default options only if we are not indexed ie. options are null - * if the fieldType has a non-null option we are all good it might have been set through a different - * call. - */ - final IndexOptions options = getDefaultIndexOption(); - assert options != IndexOptions.NONE : "default IndexOptions is NONE can't enable indexing"; - fieldType.setIndexOptions(options); - } - } else { - fieldType.setIndexOptions(IndexOptions.NONE); - } - return builder; - } - - protected IndexOptions getDefaultIndexOption() { - return defaultOptions; - } - - public T store(boolean store) { - this.fieldType.setStored(store); - return builder; - } - - public T docValues(boolean docValues) { - this.docValues = docValues; - return builder; - } - - public T storeTermVectors(boolean termVectors) { - if (termVectors != this.fieldType.storeTermVectors()) { - this.fieldType.setStoreTermVectors(termVectors); - } // don't set it to false, it is default and might be flipped by a more specific option - return builder; - } - - public T storeTermVectorOffsets(boolean termVectorOffsets) { - if (termVectorOffsets) { - this.fieldType.setStoreTermVectors(termVectorOffsets); - } - this.fieldType.setStoreTermVectorOffsets(termVectorOffsets); - return builder; - } - - public T storeTermVectorPositions(boolean termVectorPositions) { - if (termVectorPositions) { - this.fieldType.setStoreTermVectors(termVectorPositions); - } - this.fieldType.setStoreTermVectorPositions(termVectorPositions); - return builder; - } - - public T storeTermVectorPayloads(boolean termVectorPayloads) { - if (termVectorPayloads) { - this.fieldType.setStoreTermVectors(termVectorPayloads); - } - this.fieldType.setStoreTermVectorPayloads(termVectorPayloads); - return builder; - } - - public T tokenized(boolean tokenized) { - this.fieldType.setTokenized(tokenized); - return builder; - } - - public T boost(float boost) { - this.fieldType.setBoost(boost); - return builder; - } - - public T omitNorms(boolean omitNorms) { - this.fieldType.setOmitNorms(omitNorms); - this.omitNormsSet = true; - return builder; - } - - public T indexOptions(IndexOptions indexOptions) { - this.fieldType.setIndexOptions(indexOptions); - this.indexOptionsSet = true; - return builder; - } - - public T indexName(String indexName) { - this.indexName = indexName; - return builder; - } - - public T indexAnalyzer(NamedAnalyzer indexAnalyzer) { - this.fieldType.setIndexAnalyzer(indexAnalyzer); - return builder; - } - - public T searchAnalyzer(NamedAnalyzer searchAnalyzer) { - this.fieldType.setSearchAnalyzer(searchAnalyzer); - return builder; - } - - public T includeInAll(Boolean includeInAll) { - this.includeInAll = includeInAll; - return builder; - } - - public T similarity(SimilarityProvider similarity) { - this.fieldType.setSimilarity(similarity); - return builder; - } - - public T normsLoading(MappedFieldType.Loading normsLoading) { - this.fieldType.setNormsLoading(normsLoading); - return builder; - } - - public T fieldDataSettings(Settings settings) { - this.fieldDataSettings = settings; - return builder; - } - - public Builder nullValue(Object nullValue) { - this.fieldType.setNullValue(nullValue); - return this; - } - - public T multiFieldPathType(ContentPath.Type pathType) { - multiFieldsBuilder.pathType(pathType); - return builder; - } - - public T addMultiField(Mapper.Builder mapperBuilder) { - multiFieldsBuilder.add(mapperBuilder); - return builder; - } - - public T copyTo(CopyTo copyTo) { - this.copyTo = copyTo; - return builder; - } - - protected MappedFieldType.Names buildNames(BuilderContext context) { - return new MappedFieldType.Names(buildIndexName(context), buildIndexNameClean(context), buildFullName(context)); - } - - protected String buildIndexName(BuilderContext context) { - if (context.indexCreatedVersion().onOrAfter(Version.V_2_0_0)) { - return buildFullName(context); - } - String actualIndexName = indexName == null ? name : indexName; - return context.path().pathAsText(actualIndexName); - } - - protected String buildIndexNameClean(BuilderContext context) { - if (context.indexCreatedVersion().onOrAfter(Version.V_2_0_0)) { - return buildFullName(context); - } - return indexName == null ? name : indexName; - } - - protected String buildFullName(BuilderContext context) { - return context.path().fullPathAsText(name); - } - - protected void setupFieldType(BuilderContext context) { - fieldType.setNames(buildNames(context)); - } - } - - protected MappedFieldTypeReference fieldTypeRef; - protected final boolean hasDefaultDocValues; - protected Settings customFieldDataSettings; - protected final MultiFields multiFields; - protected CopyTo copyTo; - protected final boolean indexCreatedBefore2x; - - protected AbstractFieldMapper(String simpleName, MappedFieldType fieldType, Boolean docValues, @Nullable Settings fieldDataSettings, Settings indexSettings) { - this(simpleName, fieldType, docValues, fieldDataSettings, indexSettings, MultiFields.empty(), null); - } - - protected AbstractFieldMapper(String simpleName, MappedFieldType fieldType, Boolean docValues, @Nullable Settings fieldDataSettings, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { - super(simpleName); - assert indexSettings != null; - this.indexCreatedBefore2x = Version.indexCreated(indexSettings).before(Version.V_2_0_0); - this.customFieldDataSettings = fieldDataSettings; - FieldDataType fieldDataType; - if (fieldDataSettings == null) { - fieldDataType = defaultFieldDataType(); - } else { - // create a new field data type, with the default settings as well as the "new ones" - fieldDataType = new FieldDataType(defaultFieldDataType().getType(), - Settings.builder().put(defaultFieldDataType().getSettings()).put(fieldDataSettings) - ); - } - - // TODO: hasDocValues should just be set directly on the field type by callers of this ctor, but - // then we need to eliminate defaultDocValues() (only needed by geo, which needs to be fixed with passing - // doc values setting down to lat/lon) and get rid of specifying doc values in fielddata (which - // complicates whether we can just compare to the default value to know whether to write the setting) - if (docValues == null && fieldDataType != null && FieldDataType.DOC_VALUES_FORMAT_VALUE.equals(fieldDataType.getFormat(indexSettings))) { - docValues = true; - } - hasDefaultDocValues = docValues == null; - - this.fieldTypeRef = new MappedFieldTypeReference(fieldType); // must init first so defaultDocValues() can be called - fieldType = fieldType.clone(); - if (fieldType.indexAnalyzer() == null && fieldType.tokenized() == false && fieldType.indexOptions() != IndexOptions.NONE) { - fieldType.setIndexAnalyzer(Lucene.KEYWORD_ANALYZER); - fieldType.setSearchAnalyzer(Lucene.KEYWORD_ANALYZER); - } - fieldType.setHasDocValues(docValues == null ? defaultDocValues() : docValues); - fieldType.setFieldDataType(fieldDataType); - fieldType.freeze(); - this.fieldTypeRef.set(fieldType); // now reset ref once extra settings have been initialized - - this.multiFields = multiFields; - this.copyTo = copyTo; - } - - protected boolean defaultDocValues() { - if (indexCreatedBefore2x) { - return false; - } else { - return fieldType().tokenized() == false && fieldType().indexOptions() != IndexOptions.NONE; - } - } - - @Override - public String name() { - return fieldType().names().fullName(); - } - - public abstract MappedFieldType defaultFieldType(); - - public abstract FieldDataType defaultFieldDataType(); - - @Override - public MappedFieldType fieldType() { - return fieldTypeRef.get(); - } - - @Override - public MappedFieldTypeReference fieldTypeReference() { - return fieldTypeRef; - } - - @Override - public void setFieldTypeReference(MappedFieldTypeReference ref) { - if (ref.get().equals(fieldType()) == false) { - throw new IllegalStateException("Cannot overwrite field type reference to unequal reference"); - } - ref.incrementAssociatedMappers(); - this.fieldTypeRef = ref; - } - - @Override - public CopyTo copyTo() { - return copyTo; - } - - @Override - public Mapper parse(ParseContext context) throws IOException { - final List fields = new ArrayList<>(2); - try { - parseCreateField(context, fields); - for (Field field : fields) { - if (!customBoost()) { - field.setBoost(fieldType().boost()); - } - context.doc().add(field); - } - } catch (Exception e) { - throw new MapperParsingException("failed to parse [" + fieldType().names().fullName() + "]", e); - } - multiFields.parse(this, context); - return null; - } - - /** - * Parse the field value and populate fields. - */ - protected abstract void parseCreateField(ParseContext context, List fields) throws IOException; - - /** - * Derived classes can override it to specify that boost value is set by derived classes. - */ - protected boolean customBoost() { - return false; - } - - public Iterator iterator() { - if (multiFields == null) { - return Collections.emptyIterator(); - } - return multiFields.iterator(); - } - - @Override - public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException { - if (!this.getClass().equals(mergeWith.getClass())) { - String mergedType = mergeWith.getClass().getSimpleName(); - if (mergeWith instanceof AbstractFieldMapper) { - mergedType = ((AbstractFieldMapper) mergeWith).contentType(); - } - mergeResult.addConflict("mapper [" + fieldType().names().fullName() + "] of different type, current_type [" + contentType() + "], merged_type [" + mergedType + "]"); - // different types, return - return; - } - AbstractFieldMapper fieldMergeWith = (AbstractFieldMapper) mergeWith; - List subConflicts = new ArrayList<>(); // TODO: just expose list from MergeResult? - fieldType().checkTypeName(fieldMergeWith.fieldType(), subConflicts); - if (subConflicts.isEmpty() == false) { - // return early if field types don't match - assert subConflicts.size() == 1; - mergeResult.addConflict(subConflicts.get(0)); - return; - } - - boolean strict = this.fieldTypeRef.getNumAssociatedMappers() > 1 && mergeResult.updateAllTypes() == false; - fieldType().checkCompatibility(fieldMergeWith.fieldType(), subConflicts, strict); - for (String conflict : subConflicts) { - mergeResult.addConflict(conflict); - } - multiFields.merge(mergeWith, mergeResult); - - if (mergeResult.simulate() == false && mergeResult.hasConflicts() == false) { - // apply changeable values - MappedFieldType fieldType = fieldMergeWith.fieldType().clone(); - fieldType.freeze(); - fieldTypeRef.set(fieldType); - this.customFieldDataSettings = fieldMergeWith.customFieldDataSettings; - this.copyTo = fieldMergeWith.copyTo; - } - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(simpleName()); - boolean includeDefaults = params.paramAsBoolean("include_defaults", false); - doXContentBody(builder, includeDefaults, params); - return builder.endObject(); - } - - protected void doXContentBody(XContentBuilder builder, boolean includeDefaults, Params params) throws IOException { - - builder.field("type", contentType()); - if (indexCreatedBefore2x && (includeDefaults || !simpleName().equals(fieldType().names().originalIndexName()))) { - builder.field("index_name", fieldType().names().originalIndexName()); - } - - if (includeDefaults || fieldType().boost() != 1.0f) { - builder.field("boost", fieldType().boost()); - } - - FieldType defaultFieldType = defaultFieldType(); - boolean indexed = fieldType().indexOptions() != IndexOptions.NONE; - boolean defaultIndexed = defaultFieldType.indexOptions() != IndexOptions.NONE; - if (includeDefaults || indexed != defaultIndexed || - fieldType().tokenized() != defaultFieldType.tokenized()) { - builder.field("index", indexTokenizeOptionToString(indexed, fieldType().tokenized())); - } - if (includeDefaults || fieldType().stored() != defaultFieldType.stored()) { - builder.field("store", fieldType().stored()); - } - doXContentDocValues(builder, includeDefaults); - if (includeDefaults || fieldType().storeTermVectors() != defaultFieldType.storeTermVectors()) { - builder.field("term_vector", termVectorOptionsToString(fieldType())); - } - if (includeDefaults || fieldType().omitNorms() != defaultFieldType.omitNorms() || fieldType().normsLoading() != null) { - builder.startObject("norms"); - if (includeDefaults || fieldType().omitNorms() != defaultFieldType.omitNorms()) { - builder.field("enabled", !fieldType().omitNorms()); - } - if (fieldType().normsLoading() != null) { - builder.field(MappedFieldType.Loading.KEY, fieldType().normsLoading()); - } - builder.endObject(); - } - if (indexed && (includeDefaults || fieldType().indexOptions() != defaultFieldType.indexOptions())) { - builder.field("index_options", indexOptionToString(fieldType().indexOptions())); - } - - doXContentAnalyzers(builder, includeDefaults); - - if (fieldType().similarity() != null) { - builder.field("similarity", fieldType().similarity().name()); - } else if (includeDefaults) { - builder.field("similarity", SimilarityLookupService.DEFAULT_SIMILARITY); - } - - TreeMap orderedFielddataSettings = new TreeMap<>(); - if (hasCustomFieldDataSettings()) { - orderedFielddataSettings.putAll(customFieldDataSettings.getAsMap()); - builder.field("fielddata", orderedFielddataSettings); - } else if (includeDefaults) { - orderedFielddataSettings.putAll(fieldType().fieldDataType().getSettings().getAsMap()); - builder.field("fielddata", orderedFielddataSettings); - } - multiFields.toXContent(builder, params); - - if (copyTo != null) { - copyTo.toXContent(builder, params); - } - } - - protected void doXContentAnalyzers(XContentBuilder builder, boolean includeDefaults) throws IOException { - if (fieldType().indexAnalyzer() == null) { - if (includeDefaults) { - builder.field("analyzer", "default"); - } - } else if (includeDefaults || fieldType().indexAnalyzer().name().startsWith("_") == false && fieldType().indexAnalyzer().name().equals("default") == false) { - builder.field("analyzer", fieldType().indexAnalyzer().name()); - if (fieldType().searchAnalyzer().name().equals(fieldType().indexAnalyzer().name()) == false) { - builder.field("search_analyzer", fieldType().searchAnalyzer().name()); - } - } - } - - protected void doXContentDocValues(XContentBuilder builder, boolean includeDefaults) throws IOException { - if (includeDefaults || hasDefaultDocValues == false) { - builder.field(DOC_VALUES, fieldType().hasDocValues()); - } - } - - protected static String indexOptionToString(IndexOptions indexOption) { - switch (indexOption) { - case DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS: - return TypeParsers.INDEX_OPTIONS_OFFSETS; - case DOCS_AND_FREQS: - return TypeParsers.INDEX_OPTIONS_FREQS; - case DOCS_AND_FREQS_AND_POSITIONS: - return TypeParsers.INDEX_OPTIONS_POSITIONS; - case DOCS: - return TypeParsers.INDEX_OPTIONS_DOCS; - default: - throw new IllegalArgumentException("Unknown IndexOptions [" + indexOption + "]"); - } - } - - public static String termVectorOptionsToString(FieldType fieldType) { - if (!fieldType.storeTermVectors()) { - return "no"; - } else if (!fieldType.storeTermVectorOffsets() && !fieldType.storeTermVectorPositions()) { - return "yes"; - } else if (fieldType.storeTermVectorOffsets() && !fieldType.storeTermVectorPositions()) { - return "with_offsets"; - } else { - StringBuilder builder = new StringBuilder("with"); - if (fieldType.storeTermVectorPositions()) { - builder.append("_positions"); - } - if (fieldType.storeTermVectorOffsets()) { - builder.append("_offsets"); - } - if (fieldType.storeTermVectorPayloads()) { - builder.append("_payloads"); - } - return builder.toString(); - } - } - - protected static String indexTokenizeOptionToString(boolean indexed, boolean tokenized) { - if (!indexed) { - return "no"; - } else if (tokenized) { - return "analyzed"; - } else { - return "not_analyzed"; - } - } - - protected boolean hasCustomFieldDataSettings() { - return customFieldDataSettings != null && customFieldDataSettings.equals(Settings.EMPTY) == false; - } - - protected abstract String contentType(); - - public static class MultiFields { - - public static MultiFields empty() { - return new MultiFields(Defaults.PATH_TYPE, ImmutableOpenMap.of()); - } - - public static class Builder { - - private final ImmutableOpenMap.Builder mapperBuilders = ImmutableOpenMap.builder(); - private ContentPath.Type pathType = Defaults.PATH_TYPE; - - public Builder pathType(ContentPath.Type pathType) { - this.pathType = pathType; - return this; - } - - public Builder add(Mapper.Builder builder) { - mapperBuilders.put(builder.name(), builder); - return this; - } - - @SuppressWarnings("unchecked") - public MultiFields build(AbstractFieldMapper.Builder mainFieldBuilder, BuilderContext context) { - if (pathType == Defaults.PATH_TYPE && mapperBuilders.isEmpty()) { - return empty(); - } else if (mapperBuilders.isEmpty()) { - return new MultiFields(pathType, ImmutableOpenMap.of()); - } else { - ContentPath.Type origPathType = context.path().pathType(); - context.path().pathType(pathType); - context.path().add(mainFieldBuilder.name()); - ImmutableOpenMap.Builder mapperBuilders = this.mapperBuilders; - for (ObjectObjectCursor cursor : this.mapperBuilders) { - String key = cursor.key; - Mapper.Builder value = cursor.value; - Mapper mapper = value.build(context); - assert mapper instanceof FieldMapper; - mapperBuilders.put(key, mapper); - } - context.path().remove(); - context.path().pathType(origPathType); - ImmutableOpenMap.Builder mappers = mapperBuilders.cast(); - return new MultiFields(pathType, mappers.build()); - } - } - } - - private final ContentPath.Type pathType; - private volatile ImmutableOpenMap mappers; - - public MultiFields(ContentPath.Type pathType, ImmutableOpenMap mappers) { - this.pathType = pathType; - this.mappers = mappers; - // we disable the all in multi-field mappers - for (ObjectCursor cursor : mappers.values()) { - FieldMapper mapper = cursor.value; - if (mapper instanceof AllFieldMapper.IncludeInAll) { - ((AllFieldMapper.IncludeInAll) mapper).unsetIncludeInAll(); - } - } - } - - public void parse(AbstractFieldMapper mainField, ParseContext context) throws IOException { - // TODO: multi fields are really just copy fields, we just need to expose "sub fields" or something that can be part of the mappings - if (mappers.isEmpty()) { - return; - } - - context = context.createMultiFieldContext(); - - ContentPath.Type origPathType = context.path().pathType(); - context.path().pathType(pathType); - - context.path().add(mainField.simpleName()); - for (ObjectCursor cursor : mappers.values()) { - cursor.value.parse(context); - } - context.path().remove(); - context.path().pathType(origPathType); - } - - // No need for locking, because locking is taken care of in ObjectMapper#merge and DocumentMapper#merge - public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException { - AbstractFieldMapper mergeWithMultiField = (AbstractFieldMapper) mergeWith; - - List newFieldMappers = null; - ImmutableOpenMap.Builder newMappersBuilder = null; - - for (ObjectCursor cursor : mergeWithMultiField.multiFields.mappers.values()) { - FieldMapper mergeWithMapper = cursor.value; - Mapper mergeIntoMapper = mappers.get(mergeWithMapper.simpleName()); - if (mergeIntoMapper == null) { - // no mapping, simply add it if not simulating - if (!mergeResult.simulate()) { - // we disable the all in multi-field mappers - if (mergeWithMapper instanceof AllFieldMapper.IncludeInAll) { - ((AllFieldMapper.IncludeInAll) mergeWithMapper).unsetIncludeInAll(); - } - if (newMappersBuilder == null) { - newMappersBuilder = ImmutableOpenMap.builder(mappers); - } - newMappersBuilder.put(mergeWithMapper.simpleName(), mergeWithMapper); - if (mergeWithMapper instanceof AbstractFieldMapper) { - if (newFieldMappers == null) { - newFieldMappers = new ArrayList<>(2); - } - newFieldMappers.add(mergeWithMapper); - } - } - } else { - mergeIntoMapper.merge(mergeWithMapper, mergeResult); - } - } - - // first add all field mappers - if (newFieldMappers != null) { - mergeResult.addFieldMappers(newFieldMappers); - } - // now publish mappers - if (newMappersBuilder != null) { - mappers = newMappersBuilder.build(); - } - } - - public Iterator iterator() { - return Iterators.transform(mappers.values().iterator(), new Function, Mapper>() { - @Override - public Mapper apply(@Nullable ObjectCursor cursor) { - return cursor.value; - } - }); - } - - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - if (pathType != Defaults.PATH_TYPE) { - builder.field("path", pathType.name().toLowerCase(Locale.ROOT)); - } - if (!mappers.isEmpty()) { - // sort the mappers so we get consistent serialization format - Mapper[] sortedMappers = mappers.values().toArray(Mapper.class); - Arrays.sort(sortedMappers, new Comparator() { - @Override - public int compare(Mapper o1, Mapper o2) { - return o1.name().compareTo(o2.name()); - } - }); - builder.startObject("fields"); - for (Mapper mapper : sortedMappers) { - mapper.toXContent(builder, params); - } - builder.endObject(); - } - return builder; - } - } - - /** - * Represents a list of fields with optional boost factor where the current field should be copied to - */ - public static class CopyTo { - - private final ImmutableList copyToFields; - - private CopyTo(ImmutableList copyToFields) { - this.copyToFields = copyToFields; - } - - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - if (!copyToFields.isEmpty()) { - builder.startArray("copy_to"); - for (String field : copyToFields) { - builder.value(field); - } - builder.endArray(); - } - return builder; - } - - public static class Builder { - private final ImmutableList.Builder copyToBuilders = ImmutableList.builder(); - - public Builder add(String field) { - copyToBuilders.add(field); - return this; - } - - public CopyTo build() { - return new CopyTo(copyToBuilders.build()); - } - } - - public List copyToFields() { - return copyToFields; - } - } - - /** - * Returns if this field is only generated when indexing. For example, the field of type token_count - */ - @Override - public boolean isGenerated() { - return false; - } -} diff --git a/core/src/main/java/org/elasticsearch/index/mapper/core/BinaryFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/core/BinaryFieldMapper.java index 13c1c8b37d910..8f2fe5cb639d5 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/core/BinaryFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/core/BinaryFieldMapper.java @@ -28,7 +28,6 @@ import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.Version; import org.elasticsearch.common.Base64; -import org.elasticsearch.common.Nullable; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; @@ -36,7 +35,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.CollectionUtils; import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.index.fielddata.FieldDataType; +import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperParsingException; @@ -54,14 +53,14 @@ /** * */ -public class BinaryFieldMapper extends AbstractFieldMapper { +public class BinaryFieldMapper extends FieldMapper { public static final String CONTENT_TYPE = "binary"; private static final ParseField COMPRESS = new ParseField("compress").withAllDeprecated("no replacement, implemented at the codec level"); private static final ParseField COMPRESS_THRESHOLD = new ParseField("compress_threshold").withAllDeprecated("no replacement"); - public static class Defaults extends AbstractFieldMapper.Defaults { + public static class Defaults { public static final MappedFieldType FIELD_TYPE = new BinaryFieldType(); static { @@ -70,7 +69,7 @@ public static class Defaults extends AbstractFieldMapper.Defaults { } } - public static class Builder extends AbstractFieldMapper.Builder { + public static class Builder extends FieldMapper.Builder { public Builder(String name) { super(name, Defaults.FIELD_TYPE); @@ -81,8 +80,8 @@ public Builder(String name) { public BinaryFieldMapper build(BuilderContext context) { setupFieldType(context); ((BinaryFieldType)fieldType).setTryUncompressing(context.indexCreatedVersion().before(Version.V_2_0_0)); - return new BinaryFieldMapper(name, fieldType, docValues, - fieldDataSettings, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); + return new BinaryFieldMapper(name, fieldType, defaultFieldType, + context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); } } @@ -181,19 +180,9 @@ public Object valueForSearch(Object value) { } } - protected BinaryFieldMapper(String simpleName, MappedFieldType fieldType, Boolean docValues, - @Nullable Settings fieldDataSettings, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { - super(simpleName, fieldType, docValues, fieldDataSettings, indexSettings, multiFields, copyTo); - } - - @Override - public MappedFieldType defaultFieldType() { - return Defaults.FIELD_TYPE; - } - - @Override - public FieldDataType defaultFieldDataType() { - return new FieldDataType("binary"); + protected BinaryFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, + Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { + super(simpleName, fieldType, defaultFieldType, indexSettings, multiFields, copyTo); } @Override diff --git a/core/src/main/java/org/elasticsearch/index/mapper/core/BooleanFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/core/BooleanFieldMapper.java index 7bae3be7e5677..cd76fdbb047e8 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/core/BooleanFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/core/BooleanFieldMapper.java @@ -24,18 +24,15 @@ import org.apache.lucene.index.IndexOptions; import org.apache.lucene.util.BytesRef; import org.elasticsearch.common.Booleans; -import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.index.fielddata.FieldDataType; +import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperParsingException; -import org.elasticsearch.index.mapper.MergeMappingException; -import org.elasticsearch.index.mapper.MergeResult; import org.elasticsearch.index.mapper.ParseContext; import java.io.IOException; @@ -50,11 +47,11 @@ /** * A field mapper for boolean fields. */ -public class BooleanFieldMapper extends AbstractFieldMapper { +public class BooleanFieldMapper extends FieldMapper { public static final String CONTENT_TYPE = "boolean"; - public static class Defaults extends AbstractFieldMapper.Defaults { + public static class Defaults { public static final MappedFieldType FIELD_TYPE = new BooleanFieldType(); static { @@ -72,7 +69,7 @@ public static class Values { public final static BytesRef FALSE = new BytesRef("F"); } - public static class Builder extends AbstractFieldMapper.Builder { + public static class Builder extends FieldMapper.Builder { public Builder(String name) { super(name, Defaults.FIELD_TYPE); @@ -90,8 +87,8 @@ public Builder tokenized(boolean tokenized) { @Override public BooleanFieldMapper build(BuilderContext context) { setupFieldType(context); - return new BooleanFieldMapper(name, fieldType, docValues, - fieldDataSettings, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); + return new BooleanFieldMapper(name, fieldType, defaultFieldType, + context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); } } @@ -194,9 +191,9 @@ public boolean useTermQueryWithQueryString() { } } - protected BooleanFieldMapper(String simpleName, MappedFieldType fieldType, Boolean docValues, - @Nullable Settings fieldDataSettings, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { - super(simpleName, fieldType, docValues, fieldDataSettings, indexSettings, multiFields, copyTo); + protected BooleanFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, + Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { + super(simpleName, fieldType, defaultFieldType, indexSettings, multiFields, copyTo); } @Override @@ -204,17 +201,6 @@ public BooleanFieldType fieldType() { return (BooleanFieldType) super.fieldType(); } - @Override - public MappedFieldType defaultFieldType() { - return Defaults.FIELD_TYPE; - } - - @Override - public FieldDataType defaultFieldDataType() { - // TODO have a special boolean type? - return new FieldDataType(CONTENT_TYPE); - } - @Override protected void parseCreateField(ParseContext context, List fields) throws IOException { if (fieldType().indexOptions() == IndexOptions.NONE && !fieldType().stored() && !fieldType().hasDocValues()) { diff --git a/core/src/main/java/org/elasticsearch/index/mapper/core/ByteFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/core/ByteFieldMapper.java index b3192a31d829d..92944f7e8fd15 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/core/ByteFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/core/ByteFieldMapper.java @@ -38,12 +38,9 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.analysis.NumericIntegerAnalyzer; -import org.elasticsearch.index.fielddata.FieldDataType; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperParsingException; -import org.elasticsearch.index.mapper.MergeMappingException; -import org.elasticsearch.index.mapper.MergeResult; import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.query.QueryParseContext; @@ -81,8 +78,8 @@ public Builder(String name) { @Override public ByteFieldMapper build(BuilderContext context) { setupFieldType(context); - ByteFieldMapper fieldMapper = new ByteFieldMapper(name, fieldType, docValues, ignoreMalformed(context), - coerce(context), fieldDataSettings, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); + ByteFieldMapper fieldMapper = new ByteFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), + coerce(context), context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); fieldMapper.includeInAll(includeInAll); return fieldMapper; } @@ -174,8 +171,8 @@ public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower } @Override - public Query fuzzyQuery(String value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions) { - byte iValue = Byte.parseByte(value); + public Query fuzzyQuery(Object value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions) { + byte iValue = parseValue(value); byte iSim = fuzziness.asByte(); return NumericRangeQuery.newIntRange(names().indexName(), numericPrecisionStep(), iValue - iSim, @@ -193,10 +190,10 @@ public FieldStats stats(Terms terms, int maxDoc) throws IOException { } } - protected ByteFieldMapper(String simpleName, MappedFieldType fieldType, Boolean docValues, + protected ByteFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Explicit ignoreMalformed, Explicit coerce, - @Nullable Settings fieldDataSettings, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { - super(simpleName, fieldType, docValues, ignoreMalformed, coerce, fieldDataSettings, indexSettings, multiFields, copyTo); + Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { + super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, indexSettings, multiFields, copyTo); } @Override @@ -204,16 +201,6 @@ public ByteFieldType fieldType() { return (ByteFieldType) super.fieldType(); } - @Override - public MappedFieldType defaultFieldType() { - return Defaults.FIELD_TYPE; - } - - @Override - public FieldDataType defaultFieldDataType() { - return new FieldDataType("byte"); - } - private static byte parseValue(Object value) { if (value instanceof Number) { return ((Number) value).byteValue(); diff --git a/core/src/main/java/org/elasticsearch/index/mapper/core/CompletionFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/core/CompletionFieldMapper.java index 37e60c4bae009..5cf925c9e6bcf 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/core/CompletionFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/core/CompletionFieldMapper.java @@ -37,7 +37,7 @@ import org.elasticsearch.common.xcontent.XContentParser.NumberType; import org.elasticsearch.common.xcontent.XContentParser.Token; import org.elasticsearch.index.analysis.NamedAnalyzer; -import org.elasticsearch.index.fielddata.FieldDataType; +import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperException; @@ -66,11 +66,11 @@ /** * */ -public class CompletionFieldMapper extends AbstractFieldMapper { +public class CompletionFieldMapper extends FieldMapper { public static final String CONTENT_TYPE = "completion"; - public static class Defaults extends AbstractFieldMapper.Defaults { + public static class Defaults { public static final CompletionFieldType FIELD_TYPE = new CompletionFieldType(); static { @@ -104,7 +104,7 @@ public static class Fields { public static final Set ALLOWED_CONTENT_FIELD_NAMES = Sets.newHashSet(Fields.CONTENT_FIELD_NAME_INPUT, Fields.CONTENT_FIELD_NAME_OUTPUT, Fields.CONTENT_FIELD_NAME_PAYLOAD, Fields.CONTENT_FIELD_NAME_WEIGHT, Fields.CONTEXT); - public static class Builder extends AbstractFieldMapper.Builder { + public static class Builder extends FieldMapper.Builder { private boolean preserveSeparators = Defaults.DEFAULT_PRESERVE_SEPARATORS; private boolean payloads = Defaults.DEFAULT_HAS_PAYLOADS; @@ -226,7 +226,9 @@ public static final class CompletionFieldType extends MappedFieldType { private AnalyzingCompletionLookupProvider analyzingSuggestLookupProvider; private SortedMap contextMapping = ContextMapping.EMPTY_MAPPING; - public CompletionFieldType() {} + public CompletionFieldType() { + setFieldDataType(null); + } protected CompletionFieldType(CompletionFieldType ref) { super(ref); @@ -312,7 +314,7 @@ public boolean isSortable() { private int maxInputLength; public CompletionFieldMapper(String simpleName, MappedFieldType fieldType, int maxInputLength, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { - super(simpleName, fieldType, false, null, indexSettings, multiFields, copyTo); + super(simpleName, fieldType, Defaults.FIELD_TYPE, indexSettings, multiFields, copyTo); this.maxInputLength = maxInputLength; } @@ -538,16 +540,6 @@ protected String contentType() { return CONTENT_TYPE; } - @Override - public MappedFieldType defaultFieldType() { - return Defaults.FIELD_TYPE; - } - - @Override - public FieldDataType defaultFieldDataType() { - return null; - } - public boolean isStoringPayloads() { return fieldType().analyzingSuggestLookupProvider.hasPayloads(); } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/core/DateFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/core/DateFieldMapper.java index 3a6850e8d50b6..eccf9035a56db 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/core/DateFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/core/DateFieldMapper.java @@ -69,7 +69,8 @@ public class DateFieldMapper extends NumberFieldMapper { public static final String CONTENT_TYPE = "date"; public static class Defaults extends NumberFieldMapper.Defaults { - public static final FormatDateTimeFormatter DATE_TIME_FORMATTER = Joda.forPattern("dateOptionalTime||epoch_millis", Locale.ROOT); + public static final FormatDateTimeFormatter DATE_TIME_FORMATTER = Joda.forPattern("strictDateOptionalTime||epoch_millis", Locale.ROOT); + public static final FormatDateTimeFormatter DATE_TIME_FORMATTER_BEFORE_2_0 = Joda.forPattern("dateOptionalTime", Locale.ROOT); public static final TimeUnit TIME_UNIT = TimeUnit.MILLISECONDS; public static final DateFieldType FIELD_TYPE = new DateFieldType(); @@ -93,7 +94,8 @@ public Builder(String name) { locale = Locale.ROOT; } - DateFieldType fieldType() { + @Override + public DateFieldType fieldType() { return (DateFieldType)fieldType; } @@ -116,23 +118,20 @@ public Builder dateTimeFormatter(FormatDateTimeFormatter dateTimeFormatter) { public DateFieldMapper build(BuilderContext context) { setupFieldType(context); fieldType.setNullValue(nullValue); - DateFieldMapper fieldMapper = new DateFieldMapper(name, fieldType, - docValues, ignoreMalformed(context), coerce(context), - fieldDataSettings, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); + DateFieldMapper fieldMapper = new DateFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), + coerce(context), context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); fieldMapper.includeInAll(includeInAll); return fieldMapper; } protected void setupFieldType(BuilderContext context) { - FormatDateTimeFormatter dateTimeFormatter = fieldType().dateTimeFormatter; - // TODO MOVE ME OUTSIDE OF THIS SPACE? - if (Version.indexCreated(context.indexSettings()).before(Version.V_2_0_0)) { - boolean includesEpochFormatter = dateTimeFormatter.format().contains("epoch_"); - if (!includesEpochFormatter) { - String format = fieldType().timeUnit().equals(TimeUnit.SECONDS) ? "epoch_second" : "epoch_millis"; - fieldType().setDateTimeFormatter(Joda.forPattern(format + "||" + dateTimeFormatter.format())); - } + if (Version.indexCreated(context.indexSettings()).before(Version.V_2_0_0) && + !fieldType().dateTimeFormatter().format().contains("epoch_")) { + String format = fieldType().timeUnit().equals(TimeUnit.SECONDS) ? "epoch_second" : "epoch_millis"; + fieldType().setDateTimeFormatter(Joda.forPattern(format + "||" + fieldType().dateTimeFormatter().format())); } + + FormatDateTimeFormatter dateTimeFormatter = fieldType().dateTimeFormatter; if (!locale.equals(dateTimeFormatter.locale())) { fieldType().setDateTimeFormatter(new FormatDateTimeFormatter(dateTimeFormatter.format(), dateTimeFormatter.parser(), dateTimeFormatter.printer(), locale)); } @@ -160,6 +159,7 @@ public static class TypeParser implements Mapper.TypeParser { public Mapper.Builder parse(String name, Map node, ParserContext parserContext) throws MapperParsingException { DateFieldMapper.Builder builder = dateField(name); parseNumberField(builder, name, node, parserContext); + boolean configuredFormat = false; for (Iterator> iterator = node.entrySet().iterator(); iterator.hasNext();) { Map.Entry entry = iterator.next(); String propName = Strings.toUnderscoreCase(entry.getKey()); @@ -172,6 +172,7 @@ public static class TypeParser implements Mapper.TypeParser { iterator.remove(); } else if (propName.equals("format")) { builder.dateTimeFormatter(parseDateTimeFormatter(propNode)); + configuredFormat = true; iterator.remove(); } else if (propName.equals("numeric_resolution")) { builder.timeUnit(TimeUnit.valueOf(propNode.toString().toUpperCase(Locale.ROOT))); @@ -181,6 +182,13 @@ public static class TypeParser implements Mapper.TypeParser { iterator.remove(); } } + if (!configuredFormat) { + if (parserContext.indexVersionCreated().onOrAfter(Version.V_2_0_0)) { + builder.dateTimeFormatter(Defaults.DATE_TIME_FORMATTER); + } else { + builder.dateTimeFormatter(Defaults.DATE_TIME_FORMATTER_BEFORE_2_0); + } + } return builder; } } @@ -259,6 +267,7 @@ public String toString(String s) { public DateFieldType() { super(NumericType.LONG); + setFieldDataType(new FieldDataType("long")); } protected DateFieldType(DateFieldType ref) { @@ -386,8 +395,8 @@ public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower } @Override - public Query fuzzyQuery(String value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions) { - long iValue = dateMathParser().parse(value, now()); + public Query fuzzyQuery(Object value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions) { + long iValue = parseValue(value); long iSim; try { iSim = fuzziness.asTimeValue().millis(); @@ -436,9 +445,9 @@ public long parseToMilliseconds(Object value, boolean inclusive, @Nullable DateT } } - protected DateFieldMapper(String simpleName, MappedFieldType fieldType, Boolean docValues, Explicit ignoreMalformed,Explicit coerce, - @Nullable Settings fieldDataSettings, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { - super(simpleName, fieldType, docValues, ignoreMalformed, coerce, fieldDataSettings, indexSettings, multiFields, copyTo); + protected DateFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Explicit ignoreMalformed,Explicit coerce, + Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { + super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, indexSettings, multiFields, copyTo); } @Override @@ -446,16 +455,6 @@ public DateFieldType fieldType() { return (DateFieldType) super.fieldType(); } - @Override - public MappedFieldType defaultFieldType() { - return Defaults.FIELD_TYPE; - } - - @Override - public FieldDataType defaultFieldDataType() { - return new FieldDataType("long"); - } - private static Callable now() { return new Callable() { @Override diff --git a/core/src/main/java/org/elasticsearch/index/mapper/core/DoubleFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/core/DoubleFieldMapper.java index a25c1e096c02e..49ea3c2e25a82 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/core/DoubleFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/core/DoubleFieldMapper.java @@ -20,11 +20,9 @@ package org.elasticsearch.index.mapper.core; import com.carrotsearch.hppc.DoubleArrayList; - import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.document.Field; -import org.apache.lucene.document.FieldType.NumericType; import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.Terms; import org.apache.lucene.search.NumericRangeQuery; @@ -44,12 +42,9 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.analysis.NumericDoubleAnalyzer; -import org.elasticsearch.index.fielddata.FieldDataType; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperParsingException; -import org.elasticsearch.index.mapper.MergeMappingException; -import org.elasticsearch.index.mapper.MergeResult; import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.query.QueryParseContext; @@ -88,8 +83,8 @@ public Builder(String name) { @Override public DoubleFieldMapper build(BuilderContext context) { setupFieldType(context); - DoubleFieldMapper fieldMapper = new DoubleFieldMapper(name, fieldType, docValues, ignoreMalformed(context), coerce(context), - fieldDataSettings, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); + DoubleFieldMapper fieldMapper = new DoubleFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), coerce(context), + context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); fieldMapper.includeInAll(includeInAll); return fieldMapper; } @@ -182,8 +177,8 @@ public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower } @Override - public Query fuzzyQuery(String value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions) { - double iValue = Double.parseDouble(value); + public Query fuzzyQuery(Object value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions) { + double iValue = parseDoubleValue(value); double iSim = fuzziness.asDouble(); return NumericRangeQuery.newDoubleRange(names().indexName(), numericPrecisionStep(), iValue - iSim, @@ -201,9 +196,9 @@ public FieldStats stats(Terms terms, int maxDoc) throws IOException { } } - protected DoubleFieldMapper(String simpleName, MappedFieldType fieldType, Boolean docValues, Explicit ignoreMalformed, Explicit coerce, - @Nullable Settings fieldDataSettings, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { - super(simpleName, fieldType, docValues, ignoreMalformed, coerce, fieldDataSettings, indexSettings, multiFields, copyTo); + protected DoubleFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Explicit ignoreMalformed, + Explicit coerce, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { + super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, indexSettings, multiFields, copyTo); } @Override @@ -211,16 +206,6 @@ public DoubleFieldType fieldType() { return (DoubleFieldType) super.fieldType(); } - @Override - public MappedFieldType defaultFieldType() { - return Defaults.FIELD_TYPE; - } - - @Override - public FieldDataType defaultFieldDataType() { - return new FieldDataType("double"); - } - @Override protected boolean customBoost() { return true; diff --git a/core/src/main/java/org/elasticsearch/index/mapper/core/FloatFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/core/FloatFieldMapper.java index 0870a7329aab5..8012672c36946 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/core/FloatFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/core/FloatFieldMapper.java @@ -20,11 +20,9 @@ package org.elasticsearch.index.mapper.core; import com.carrotsearch.hppc.FloatArrayList; - import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.document.Field; -import org.apache.lucene.document.FieldType.NumericType; import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.Terms; import org.apache.lucene.search.NumericRangeQuery; @@ -45,12 +43,9 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.analysis.NumericFloatAnalyzer; -import org.elasticsearch.index.fielddata.FieldDataType; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperParsingException; -import org.elasticsearch.index.mapper.MergeMappingException; -import org.elasticsearch.index.mapper.MergeResult; import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.query.QueryParseContext; @@ -89,8 +84,8 @@ public Builder(String name) { @Override public FloatFieldMapper build(BuilderContext context) { setupFieldType(context); - FloatFieldMapper fieldMapper = new FloatFieldMapper(name, fieldType, docValues, ignoreMalformed(context), coerce(context), - fieldDataSettings, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); + FloatFieldMapper fieldMapper = new FloatFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), coerce(context), + context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); fieldMapper.includeInAll(includeInAll); return fieldMapper; } @@ -183,8 +178,8 @@ public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower } @Override - public Query fuzzyQuery(String value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions) { - float iValue = Float.parseFloat(value); + public Query fuzzyQuery(Object value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions) { + float iValue = parseValue(value); final float iSim = fuzziness.asFloat(); return NumericRangeQuery.newFloatRange(names().indexName(), numericPrecisionStep(), iValue - iSim, @@ -202,10 +197,10 @@ public FieldStats stats(Terms terms, int maxDoc) throws IOException { } } - protected FloatFieldMapper(String simpleName, MappedFieldType fieldType, Boolean docValues, + protected FloatFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Explicit ignoreMalformed, Explicit coerce, - @Nullable Settings fieldDataSettings, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { - super(simpleName, fieldType, docValues, ignoreMalformed, coerce, fieldDataSettings, indexSettings, multiFields, copyTo); + Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { + super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, indexSettings, multiFields, copyTo); } @Override @@ -213,16 +208,6 @@ public FloatFieldType fieldType() { return (FloatFieldType) super.fieldType(); } - @Override - public MappedFieldType defaultFieldType() { - return Defaults.FIELD_TYPE; - } - - @Override - public FieldDataType defaultFieldDataType() { - return new FieldDataType("float"); - } - private static float parseValue(Object value) { if (value instanceof Number) { return ((Number) value).floatValue(); diff --git a/core/src/main/java/org/elasticsearch/index/mapper/core/IntegerFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/core/IntegerFieldMapper.java index f934acc10e1ec..1271de9808bc9 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/core/IntegerFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/core/IntegerFieldMapper.java @@ -40,13 +40,9 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.analysis.NumericIntegerAnalyzer; -import org.elasticsearch.index.fielddata.FieldDataType; -import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperParsingException; -import org.elasticsearch.index.mapper.MergeMappingException; -import org.elasticsearch.index.mapper.MergeResult; import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.query.QueryParseContext; @@ -89,8 +85,8 @@ public Builder nullValue(int nullValue) { @Override public IntegerFieldMapper build(BuilderContext context) { setupFieldType(context); - IntegerFieldMapper fieldMapper = new IntegerFieldMapper(name, fieldType, docValues, - ignoreMalformed(context), coerce(context), fieldDataSettings, + IntegerFieldMapper fieldMapper = new IntegerFieldMapper(name, fieldType, defaultFieldType, + ignoreMalformed(context), coerce(context), context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); fieldMapper.includeInAll(includeInAll); return fieldMapper; @@ -145,7 +141,8 @@ public NumberFieldType clone() { @Override public String typeName() { - return CONTENT_TYPE; + // TODO: this should be the same as the mapper type name, except fielddata expects int... + return "int"; } @Override @@ -183,8 +180,8 @@ public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower } @Override - public Query fuzzyQuery(String value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions) { - int iValue = Integer.parseInt(value); + public Query fuzzyQuery(Object value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions) { + int iValue = parseValue(value); int iSim = fuzziness.asInt(); return NumericRangeQuery.newIntRange(names().indexName(), numericPrecisionStep(), iValue - iSim, @@ -202,11 +199,10 @@ public FieldStats stats(Terms terms, int maxDoc) throws IOException { } } - protected IntegerFieldMapper(String simpleName, MappedFieldType fieldType, Boolean docValues, + protected IntegerFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Explicit ignoreMalformed, Explicit coerce, - @Nullable Settings fieldDataSettings, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { - super(simpleName, fieldType, docValues, ignoreMalformed, coerce, fieldDataSettings, indexSettings, multiFields, copyTo); + super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, indexSettings, multiFields, copyTo); } @Override @@ -214,16 +210,6 @@ public IntegerFieldType fieldType() { return (IntegerFieldType) super.fieldType(); } - @Override - public MappedFieldType defaultFieldType() { - return Defaults.FIELD_TYPE; - } - - @Override - public FieldDataType defaultFieldDataType() { - return new FieldDataType("int"); - } - private static int parseValue(Object value) { if (value instanceof Number) { return ((Number) value).intValue(); diff --git a/core/src/main/java/org/elasticsearch/index/mapper/core/LongFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/core/LongFieldMapper.java index fecab71269e4f..9542b508a1984 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/core/LongFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/core/LongFieldMapper.java @@ -22,7 +22,6 @@ import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.document.Field; -import org.apache.lucene.document.FieldType.NumericType; import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.Terms; import org.apache.lucene.search.NumericRangeQuery; @@ -41,12 +40,9 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.analysis.NumericLongAnalyzer; -import org.elasticsearch.index.fielddata.FieldDataType; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperParsingException; -import org.elasticsearch.index.mapper.MergeMappingException; -import org.elasticsearch.index.mapper.MergeResult; import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.query.QueryParseContext; @@ -89,8 +85,8 @@ public Builder nullValue(long nullValue) { @Override public LongFieldMapper build(BuilderContext context) { setupFieldType(context); - LongFieldMapper fieldMapper = new LongFieldMapper(name, fieldType, docValues, - ignoreMalformed(context), coerce(context), fieldDataSettings, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); + LongFieldMapper fieldMapper = new LongFieldMapper(name, fieldType, defaultFieldType, + ignoreMalformed(context), coerce(context), context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); fieldMapper.includeInAll(includeInAll); return fieldMapper; } @@ -182,8 +178,8 @@ public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower } @Override - public Query fuzzyQuery(String value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions) { - long iValue = Long.parseLong(value); + public Query fuzzyQuery(Object value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions) { + long iValue = parseLongValue(value); final long iSim = fuzziness.asLong(); return NumericRangeQuery.newLongRange(names().indexName(), numericPrecisionStep(), iValue - iSim, @@ -201,11 +197,10 @@ public FieldStats stats(Terms terms, int maxDoc) throws IOException { } } - protected LongFieldMapper(String simpleName, MappedFieldType fieldType, Boolean docValues, + protected LongFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Explicit ignoreMalformed, Explicit coerce, - @Nullable Settings fieldDataSettings, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { - super(simpleName, fieldType, docValues, ignoreMalformed, coerce, fieldDataSettings, indexSettings, multiFields, copyTo); + super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, indexSettings, multiFields, copyTo); } @Override @@ -213,16 +208,6 @@ public LongFieldType fieldType() { return (LongFieldType) super.fieldType(); } - @Override - public MappedFieldType defaultFieldType() { - return Defaults.FIELD_TYPE; - } - - @Override - public FieldDataType defaultFieldDataType() { - return new FieldDataType("long"); - } - @Override protected boolean customBoost() { return true; diff --git a/core/src/main/java/org/elasticsearch/index/mapper/core/Murmur3FieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/core/Murmur3FieldMapper.java index 6f91db0185497..cac410ff7d3f1 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/core/Murmur3FieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/core/Murmur3FieldMapper.java @@ -23,17 +23,14 @@ import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; import org.elasticsearch.common.Explicit; -import org.elasticsearch.common.Nullable; import org.elasticsearch.common.hash.MurmurHash3; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.analysis.NamedAnalyzer; -import org.elasticsearch.index.analysis.NumericDateAnalyzer; import org.elasticsearch.index.analysis.NumericLongAnalyzer; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.ParseContext; -import org.elasticsearch.index.similarity.SimilarityProvider; import java.io.IOException; import java.util.List; @@ -61,9 +58,9 @@ public Builder(String name) { @Override public Murmur3FieldMapper build(BuilderContext context) { setupFieldType(context); - Murmur3FieldMapper fieldMapper = new Murmur3FieldMapper(name, fieldType, docValues, + Murmur3FieldMapper fieldMapper = new Murmur3FieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), coerce(context), - fieldDataSettings, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); + context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); fieldMapper.includeInAll(includeInAll); return fieldMapper; } @@ -119,12 +116,10 @@ public Murmur3FieldType clone() { } } - protected Murmur3FieldMapper(String simpleName, MappedFieldType fieldType, Boolean docValues, + protected Murmur3FieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Explicit ignoreMalformed, Explicit coerce, - @Nullable Settings fieldDataSettings, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { - super(simpleName, fieldType, docValues, ignoreMalformed, coerce, - fieldDataSettings, indexSettings, multiFields, copyTo); + super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, indexSettings, multiFields, copyTo); } @Override diff --git a/core/src/main/java/org/elasticsearch/index/mapper/core/NumberFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/core/NumberFieldMapper.java index 94f2296a65b91..78406c2afbca8 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/core/NumberFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/core/NumberFieldMapper.java @@ -20,7 +20,6 @@ package org.elasticsearch.index.mapper.core; import com.carrotsearch.hppc.LongArrayList; - import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.NumericTokenStream; import org.apache.lucene.analysis.TokenStream; @@ -36,19 +35,13 @@ import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; import org.elasticsearch.common.Explicit; -import org.elasticsearch.common.Nullable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.Fuzziness; import org.elasticsearch.common.util.ByteUtils; import org.elasticsearch.common.util.CollectionUtils; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.index.analysis.NamedAnalyzer; -import org.elasticsearch.index.mapper.MappedFieldType; -import org.elasticsearch.index.mapper.Mapper; -import org.elasticsearch.index.mapper.MapperParsingException; -import org.elasticsearch.index.mapper.MergeMappingException; -import org.elasticsearch.index.mapper.MergeResult; -import org.elasticsearch.index.mapper.ParseContext; +import org.elasticsearch.index.mapper.*; import org.elasticsearch.index.mapper.internal.AllFieldMapper; import java.io.IOException; @@ -58,9 +51,9 @@ /** * */ -public abstract class NumberFieldMapper extends AbstractFieldMapper implements AllFieldMapper.IncludeInAll { +public abstract class NumberFieldMapper extends FieldMapper implements AllFieldMapper.IncludeInAll { - public static class Defaults extends AbstractFieldMapper.Defaults { + public static class Defaults { public static final int PRECISION_STEP_8_BIT = Integer.MAX_VALUE; // 1tpv: 256 terms at most, not useful public static final int PRECISION_STEP_16_BIT = 8; // 2tpv @@ -71,7 +64,7 @@ public static class Defaults extends AbstractFieldMapper.Defaults { public static final Explicit COERCE = new Explicit<>(true, false); } - public abstract static class Builder extends AbstractFieldMapper.Builder { + public abstract static class Builder extends FieldMapper.Builder { private Boolean ignoreMalformed; @@ -158,7 +151,7 @@ public Object valueForSearch(Object value) { } @Override - public abstract Query fuzzyQuery(String value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions); + public abstract Query fuzzyQuery(Object value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions); @Override public boolean useTermQueryWithQueryString() { @@ -185,11 +178,10 @@ public boolean isNumeric() { */ protected final boolean useSortedNumericDocValues; - protected NumberFieldMapper(String simpleName, MappedFieldType fieldType, Boolean docValues, - Explicit ignoreMalformed, Explicit coerce, @Nullable Settings fieldDataSettings, Settings indexSettings, + protected NumberFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, + Explicit ignoreMalformed, Explicit coerce, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { - // LUCENE 4 UPGRADE: Since we can't do anything before the super call, we have to push the boost check down to subclasses - super(simpleName, fieldType, docValues, fieldDataSettings, indexSettings, multiFields, copyTo); + super(simpleName, fieldType, defaultFieldType, indexSettings, multiFields, copyTo); this.ignoreMalformed = ignoreMalformed; this.coerce = coerce; this.useSortedNumericDocValues = Version.indexCreated(indexSettings).onOrAfter(Version.V_1_4_0_Beta1); diff --git a/core/src/main/java/org/elasticsearch/index/mapper/core/ShortFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/core/ShortFieldMapper.java index ee53f98e296e7..b40d570e69dc9 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/core/ShortFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/core/ShortFieldMapper.java @@ -22,7 +22,6 @@ import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.document.Field; -import org.apache.lucene.document.FieldType.NumericType; import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.Terms; import org.apache.lucene.search.NumericRangeQuery; @@ -41,12 +40,9 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.analysis.NumericIntegerAnalyzer; -import org.elasticsearch.index.fielddata.FieldDataType; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperParsingException; -import org.elasticsearch.index.mapper.MergeMappingException; -import org.elasticsearch.index.mapper.MergeResult; import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.query.QueryParseContext; @@ -85,8 +81,8 @@ public Builder(String name) { @Override public ShortFieldMapper build(BuilderContext context) { setupFieldType(context); - ShortFieldMapper fieldMapper = new ShortFieldMapper(name, fieldType, docValues, - ignoreMalformed(context), coerce(context), fieldDataSettings, + ShortFieldMapper fieldMapper = new ShortFieldMapper(name, fieldType, defaultFieldType, + ignoreMalformed(context), coerce(context), context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); fieldMapper.includeInAll(includeInAll); return fieldMapper; @@ -180,8 +176,8 @@ public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower } @Override - public Query fuzzyQuery(String value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions) { - short iValue = Short.parseShort(value); + public Query fuzzyQuery(Object value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions) { + short iValue = parseValue(value); short iSim = fuzziness.asShort(); return NumericRangeQuery.newIntRange(names().indexName(), numericPrecisionStep(), iValue - iSim, @@ -199,12 +195,10 @@ public FieldStats stats(Terms terms, int maxDoc) throws IOException { } } - protected ShortFieldMapper(String simpleName, MappedFieldType fieldType, Boolean docValues, + protected ShortFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Explicit ignoreMalformed, Explicit coerce, - @Nullable Settings fieldDataSettings, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { - super(simpleName, fieldType, docValues, ignoreMalformed, coerce, - fieldDataSettings, indexSettings, multiFields, copyTo); + super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, indexSettings, multiFields, copyTo); } @Override @@ -212,16 +206,6 @@ public ShortFieldType fieldType() { return (ShortFieldType) super.fieldType(); } - @Override - public MappedFieldType defaultFieldType() { - return Defaults.FIELD_TYPE; - } - - @Override - public FieldDataType defaultFieldDataType() { - return new FieldDataType("short"); - } - private static short parseValue(Object value) { if (value instanceof Number) { return ((Number) value).shortValue(); diff --git a/core/src/main/java/org/elasticsearch/index/mapper/core/StringFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/core/StringFieldMapper.java index 1650256725bc4..a255a2f106f8b 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/core/StringFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/core/StringFieldMapper.java @@ -24,14 +24,13 @@ import org.apache.lucene.index.IndexOptions; import org.apache.lucene.search.Query; import org.apache.lucene.util.BytesRef; -import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.support.XContentMapValues; import org.elasticsearch.index.analysis.NamedAnalyzer; -import org.elasticsearch.index.fielddata.FieldDataType; +import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperParsingException; @@ -50,14 +49,11 @@ import static org.elasticsearch.index.mapper.core.TypeParsers.parseField; import static org.elasticsearch.index.mapper.core.TypeParsers.parseMultiField; -/** - * - */ -public class StringFieldMapper extends AbstractFieldMapper implements AllFieldMapper.IncludeInAll { +public class StringFieldMapper extends FieldMapper implements AllFieldMapper.IncludeInAll { public static final String CONTENT_TYPE = "string"; - public static class Defaults extends AbstractFieldMapper.Defaults { + public static class Defaults { public static final MappedFieldType FIELD_TYPE = new StringFieldType(); static { @@ -70,7 +66,7 @@ public static class Defaults extends AbstractFieldMapper.Defaults { public static final int IGNORE_ABOVE = -1; } - public static class Builder extends AbstractFieldMapper.Builder { + public static class Builder extends FieldMapper.Builder { protected String nullValue = Defaults.NULL_VALUE; @@ -116,22 +112,20 @@ public StringFieldMapper build(BuilderContext context) { // if they are set explicitly, we will use those values // we also change the values on the default field type so that toXContent emits what // differs from the defaults - MappedFieldType defaultFieldType = Defaults.FIELD_TYPE.clone(); if (fieldType.indexOptions() != IndexOptions.NONE && !fieldType.tokenized()) { defaultFieldType.setOmitNorms(true); defaultFieldType.setIndexOptions(IndexOptions.DOCS); - if (!omitNormsSet && fieldType.boost() == Defaults.BOOST) { + if (!omitNormsSet && fieldType.boost() == 1.0f) { fieldType.setOmitNorms(true); } if (!indexOptionsSet) { fieldType.setIndexOptions(IndexOptions.DOCS); } } - defaultFieldType.freeze(); setupFieldType(context); StringFieldMapper fieldMapper = new StringFieldMapper( - name, fieldType, defaultFieldType, docValues, positionOffsetGap, ignoreAbove, - fieldDataSettings, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); + name, fieldType, defaultFieldType, positionOffsetGap, ignoreAbove, + context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); fieldMapper.includeInAll(includeInAll); return fieldMapper; } @@ -163,14 +157,14 @@ public Mapper.Builder parse(String name, Map node, ParserContext builder.positionOffsetGap(XContentMapValues.nodeIntegerValue(propNode, -1)); // we need to update to actual analyzers if they are not set in this case... // so we can inject the position offset gap... - if (builder.fieldType.indexAnalyzer() == null) { - builder.fieldType.setIndexAnalyzer(parserContext.analysisService().defaultIndexAnalyzer()); + if (builder.fieldType().indexAnalyzer() == null) { + builder.fieldType().setIndexAnalyzer(parserContext.analysisService().defaultIndexAnalyzer()); } - if (builder.fieldType.searchAnalyzer() == null) { - builder.fieldType.setSearchAnalyzer(parserContext.analysisService().defaultSearchAnalyzer()); + if (builder.fieldType().searchAnalyzer() == null) { + builder.fieldType().setSearchAnalyzer(parserContext.analysisService().defaultSearchAnalyzer()); } - if (builder.fieldType.searchQuoteAnalyzer() == null) { - builder.fieldType.setSearchQuoteAnalyzer(parserContext.analysisService().defaultSearchQuoteAnalyzer()); + if (builder.fieldType().searchQuoteAnalyzer() == null) { + builder.fieldType().setSearchQuoteAnalyzer(parserContext.analysisService().defaultSearchQuoteAnalyzer()); } iterator.remove(); } else if (propName.equals("ignore_above")) { @@ -221,30 +215,18 @@ public Query nullValueQuery() { private Boolean includeInAll; private int positionOffsetGap; private int ignoreAbove; - private final MappedFieldType defaultFieldType; - protected StringFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Boolean docValues, - int positionOffsetGap, int ignoreAbove, @Nullable Settings fieldDataSettings, + protected StringFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, + int positionOffsetGap, int ignoreAbove, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { - super(simpleName, fieldType, docValues, fieldDataSettings, indexSettings, multiFields, copyTo); + super(simpleName, fieldType, defaultFieldType, indexSettings, multiFields, copyTo); if (fieldType.tokenized() && fieldType.indexOptions() != NONE && fieldType().hasDocValues()) { throw new MapperParsingException("Field [" + fieldType.names().fullName() + "] cannot be analyzed and have doc values"); } - this.defaultFieldType = defaultFieldType; this.positionOffsetGap = positionOffsetGap; this.ignoreAbove = ignoreAbove; } - @Override - public MappedFieldType defaultFieldType() { - return defaultFieldType; - } - - @Override - public FieldDataType defaultFieldDataType() { - return new FieldDataType("string"); - } - @Override public void includeInAll(Boolean includeInAll) { if (includeInAll != null) { diff --git a/core/src/main/java/org/elasticsearch/index/mapper/core/TokenCountFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/core/TokenCountFieldMapper.java index 371b69bf114c4..a148d940bbeee 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/core/TokenCountFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/core/TokenCountFieldMapper.java @@ -78,8 +78,8 @@ public NamedAnalyzer analyzer() { @Override public TokenCountFieldMapper build(BuilderContext context) { setupFieldType(context); - TokenCountFieldMapper fieldMapper = new TokenCountFieldMapper(name, fieldType, docValues, - ignoreMalformed(context), coerce(context), fieldDataSettings, context.indexSettings(), + TokenCountFieldMapper fieldMapper = new TokenCountFieldMapper(name, fieldType, defaultFieldType, + ignoreMalformed(context), coerce(context), context.indexSettings(), analyzer, multiFieldsBuilder.build(this, context), copyTo); fieldMapper.includeInAll(includeInAll); return fieldMapper; @@ -127,10 +127,9 @@ public Mapper.Builder parse(String name, Map node, ParserContext private NamedAnalyzer analyzer; - protected TokenCountFieldMapper(String simpleName, MappedFieldType fieldType, Boolean docValues, Explicit ignoreMalformed, - Explicit coerce, Settings fieldDataSettings, Settings indexSettings, - NamedAnalyzer analyzer, MultiFields multiFields, CopyTo copyTo) { - super(simpleName, fieldType, docValues, ignoreMalformed, coerce, fieldDataSettings, indexSettings, multiFields, copyTo); + protected TokenCountFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Explicit ignoreMalformed, + Explicit coerce, Settings indexSettings, NamedAnalyzer analyzer, MultiFields multiFields, CopyTo copyTo) { + super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, indexSettings, multiFields, copyTo); this.analyzer = analyzer; } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/core/TypeParsers.java b/core/src/main/java/org/elasticsearch/index/mapper/core/TypeParsers.java index a905687bd7899..91c877c0c9888 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/core/TypeParsers.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/core/TypeParsers.java @@ -30,6 +30,7 @@ import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.mapper.ContentPath; import org.elasticsearch.index.mapper.DocumentMapperParser; +import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.MappedFieldType.Loading; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperParsingException; @@ -60,8 +61,8 @@ public class TypeParsers { @Override public Mapper.Builder parse(String name, Map node, ParserContext parserContext) throws MapperParsingException { ContentPath.Type pathType = null; - AbstractFieldMapper.Builder mainFieldBuilder = null; - List fields = null; + FieldMapper.Builder mainFieldBuilder = null; + List fields = null; String firstType = null; for (Iterator> iterator = node.entrySet().iterator(); iterator.hasNext();) { @@ -94,13 +95,13 @@ public class TypeParsers { throw new MapperParsingException("no handler for type [" + type + "] declared on field [" + fieldName + "]"); } if (propName.equals(name)) { - mainFieldBuilder = (AbstractFieldMapper.Builder) typeParser.parse(propName, propNode, parserContext); + mainFieldBuilder = (FieldMapper.Builder) typeParser.parse(propName, propNode, parserContext); fieldsIterator.remove(); } else { if (fields == null) { fields = new ArrayList<>(2); } - fields.add((AbstractFieldMapper.Builder) typeParser.parse(propName, propNode, parserContext)); + fields.add((FieldMapper.Builder) typeParser.parse(propName, propNode, parserContext)); fieldsIterator.remove(); } } @@ -121,8 +122,8 @@ public class TypeParsers { mainFieldBuilder = new StringFieldMapper.Builder(name).index(false); } else { Mapper.Builder substitute = typeParser.parse(name, Collections.emptyMap(), parserContext); - if (substitute instanceof AbstractFieldMapper.Builder) { - mainFieldBuilder = ((AbstractFieldMapper.Builder) substitute).index(false); + if (substitute instanceof FieldMapper.Builder) { + mainFieldBuilder = ((FieldMapper.Builder) substitute).index(false); } else { // The first multi isn't a core field type mainFieldBuilder = new StringFieldMapper.Builder(name).index(false); @@ -180,9 +181,9 @@ public static void parseNumberField(NumberFieldMapper.Builder builder, String na } } - public static void parseField(AbstractFieldMapper.Builder builder, String name, Map fieldNode, Mapper.TypeParser.ParserContext parserContext) { - NamedAnalyzer indexAnalyzer = builder.fieldType.indexAnalyzer(); - NamedAnalyzer searchAnalyzer = builder.fieldType.searchAnalyzer(); + public static void parseField(FieldMapper.Builder builder, String name, Map fieldNode, Mapper.TypeParser.ParserContext parserContext) { + NamedAnalyzer indexAnalyzer = builder.fieldType().indexAnalyzer(); + NamedAnalyzer searchAnalyzer = builder.fieldType().searchAnalyzer(); for (Iterator> iterator = fieldNode.entrySet().iterator(); iterator.hasNext();) { Map.Entry entry = iterator.next(); final String propName = Strings.toUnderscoreCase(entry.getKey()); @@ -299,7 +300,7 @@ public static void parseField(AbstractFieldMapper.Builder builder, String name, builder.searchAnalyzer(searchAnalyzer); } - public static boolean parseMultiField(AbstractFieldMapper.Builder builder, String name, Mapper.TypeParser.ParserContext parserContext, String propName, Object propNode) { + public static boolean parseMultiField(FieldMapper.Builder builder, String name, Mapper.TypeParser.ParserContext parserContext, String propName, Object propNode) { if (propName.equals("path") && parserContext.indexVersionCreated().before(Version.V_2_0_0)) { builder.multiFieldPathType(parsePathType(name, propNode.toString())); return true; @@ -367,7 +368,7 @@ public static FormatDateTimeFormatter parseDateTimeFormatter(Object node) { return Joda.forPattern(node.toString()); } - public static void parseTermVector(String fieldName, String termVector, AbstractFieldMapper.Builder builder) throws MapperParsingException { + public static void parseTermVector(String fieldName, String termVector, FieldMapper.Builder builder) throws MapperParsingException { termVector = Strings.toUnderscoreCase(termVector); if ("no".equals(termVector)) { builder.storeTermVectors(false); @@ -392,7 +393,7 @@ public static void parseTermVector(String fieldName, String termVector, Abstract } } - public static void parseIndex(String fieldName, String index, AbstractFieldMapper.Builder builder) throws MapperParsingException { + public static void parseIndex(String fieldName, String index, FieldMapper.Builder builder) throws MapperParsingException { index = Strings.toUnderscoreCase(index); if ("no".equals(index)) { builder.index(false); @@ -429,8 +430,8 @@ public static ContentPath.Type parsePathType(String name, String path) throws Ma } @SuppressWarnings("unchecked") - public static void parseCopyFields(Object propNode, AbstractFieldMapper.Builder builder) { - AbstractFieldMapper.CopyTo.Builder copyToBuilder = new AbstractFieldMapper.CopyTo.Builder(); + public static void parseCopyFields(Object propNode, FieldMapper.Builder builder) { + FieldMapper.CopyTo.Builder copyToBuilder = new FieldMapper.CopyTo.Builder(); if (isArray(propNode)) { for(Object node : (List) propNode) { copyToBuilder.add(nodeStringValue(node, null)); diff --git a/core/src/main/java/org/elasticsearch/index/mapper/geo/GeoPointFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/geo/GeoPointFieldMapper.java index e178e0d5249d6..dabc109f2e1cb 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/geo/GeoPointFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/geo/GeoPointFieldMapper.java @@ -27,7 +27,6 @@ import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.NumericUtils; import org.elasticsearch.Version; -import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.geo.GeoDistance; import org.elasticsearch.common.geo.GeoHashUtils; @@ -39,13 +38,12 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.support.XContentMapValues; -import org.elasticsearch.index.fielddata.FieldDataType; import org.elasticsearch.index.mapper.ContentPath; +import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.ParseContext; -import org.elasticsearch.index.mapper.core.AbstractFieldMapper; import org.elasticsearch.index.mapper.core.DoubleFieldMapper; import org.elasticsearch.index.mapper.core.NumberFieldMapper; import org.elasticsearch.index.mapper.core.NumberFieldMapper.CustomNumericDocValuesField; @@ -76,7 +74,7 @@ * "lon" : 2.1 * } */ -public class GeoPointFieldMapper extends AbstractFieldMapper implements ArrayValueMapperParser { +public class GeoPointFieldMapper extends FieldMapper implements ArrayValueMapperParser { public static final String CONTENT_TYPE = "geo_point"; @@ -110,7 +108,7 @@ public static class Defaults { } } - public static class Builder extends AbstractFieldMapper.Builder { + public static class Builder extends FieldMapper.Builder { private ContentPath.Type pathType = Defaults.PATH_TYPE; @@ -129,7 +127,8 @@ public Builder(String name) { this.builder = this; } - GeoPointFieldType fieldType() { + @Override + public GeoPointFieldType fieldType() { return (GeoPointFieldType)fieldType; } @@ -204,10 +203,10 @@ public GeoPointFieldMapper build(BuilderContext context) { // this is important: even if geo points feel like they need to be tokenized to distinguish lat from lon, we actually want to // store them as a single token. fieldType.setTokenized(false); - fieldType.setHasDocValues(false); setupFieldType(context); - - return new GeoPointFieldMapper(name, fieldType, docValues, fieldDataSettings, context.indexSettings(), origPathType, + fieldType.setHasDocValues(false); + defaultFieldType.setHasDocValues(false); + return new GeoPointFieldMapper(name, fieldType, defaultFieldType, context.indexSettings(), origPathType, latMapper, lonMapper, geohashMapper, multiFieldsBuilder.build(this, context)); } } @@ -586,9 +585,9 @@ public GeoPoint decode(long latBits, long lonBits, GeoPoint out) { private final StringFieldMapper geohashMapper; - public GeoPointFieldMapper(String simpleName, MappedFieldType fieldType, Boolean docValues, @Nullable Settings fieldDataSettings, Settings indexSettings, + public GeoPointFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Settings indexSettings, ContentPath.Type pathType, DoubleFieldMapper latMapper, DoubleFieldMapper lonMapper, StringFieldMapper geohashMapper,MultiFields multiFields) { - super(simpleName, fieldType, docValues, fieldDataSettings, indexSettings, multiFields, null); + super(simpleName, fieldType, defaultFieldType, indexSettings, multiFields, null); this.pathType = pathType; this.latMapper = latMapper; this.lonMapper = lonMapper; @@ -605,21 +604,6 @@ public GeoPointFieldType fieldType() { return (GeoPointFieldType) super.fieldType(); } - @Override - public MappedFieldType defaultFieldType() { - return Defaults.FIELD_TYPE; - } - - @Override - public FieldDataType defaultFieldDataType() { - return new FieldDataType("geo_point"); - } - - @Override - protected boolean defaultDocValues() { - return false; - } - @Override protected void parseCreateField(ParseContext context, List fields) throws IOException { throw new UnsupportedOperationException("Parsing is implemented in parse(), this method should NEVER be called"); diff --git a/core/src/main/java/org/elasticsearch/index/mapper/geo/GeoShapeFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/geo/GeoShapeFieldMapper.java index 9e113374ac82f..a462140d3387c 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/geo/GeoShapeFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/geo/GeoShapeFieldMapper.java @@ -37,12 +37,11 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.DistanceUnit; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.index.fielddata.FieldDataType; +import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.ParseContext; -import org.elasticsearch.index.mapper.core.AbstractFieldMapper; import java.io.IOException; import java.util.Iterator; @@ -69,7 +68,7 @@ * ] * } */ -public class GeoShapeFieldMapper extends AbstractFieldMapper { +public class GeoShapeFieldMapper extends FieldMapper { public static final String CONTENT_TYPE = "geo_shape"; @@ -107,7 +106,7 @@ public static class Defaults { } } - public static class Builder extends AbstractFieldMapper.Builder { + public static class Builder extends FieldMapper.Builder { public Builder(String name) { super(name, Defaults.FIELD_TYPE); @@ -359,7 +358,7 @@ public String value(Object value) { } public GeoShapeFieldMapper(String simpleName, MappedFieldType fieldType, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { - super(simpleName, fieldType, false, null, indexSettings, multiFields, copyTo); + super(simpleName, fieldType, Defaults.FIELD_TYPE, indexSettings, multiFields, copyTo); } @Override @@ -367,16 +366,6 @@ public GeoShapeFieldType fieldType() { return (GeoShapeFieldType) super.fieldType(); } - @Override - public MappedFieldType defaultFieldType() { - return Defaults.FIELD_TYPE; - } - - @Override - public FieldDataType defaultFieldDataType() { - return null; - } - @Override public Mapper parse(ParseContext context) throws IOException { try { diff --git a/core/src/main/java/org/elasticsearch/index/mapper/internal/AllFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/internal/AllFieldMapper.java index a53b556f66b7f..1ceee372a518e 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/internal/AllFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/internal/AllFieldMapper.java @@ -25,7 +25,6 @@ import org.apache.lucene.index.Term; import org.apache.lucene.search.Query; import org.elasticsearch.Version; -import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.lucene.Lucene; @@ -39,9 +38,8 @@ import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.MergeMappingException; import org.elasticsearch.index.mapper.MergeResult; -import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.MetadataFieldMapper; -import org.elasticsearch.index.mapper.core.AbstractFieldMapper; +import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.query.QueryParseContext; import org.elasticsearch.index.similarity.SimilarityLookupService; @@ -72,7 +70,7 @@ public interface IncludeInAll { public static final String CONTENT_TYPE = "_all"; - public static class Defaults extends AbstractFieldMapper.Defaults { + public static class Defaults { public static final String NAME = AllFieldMapper.NAME; public static final String INDEX_NAME = AllFieldMapper.NAME; public static final EnabledAttributeMapper ENABLED = EnabledAttributeMapper.UNSET_ENABLED; @@ -111,7 +109,7 @@ public AllFieldMapper build(BuilderContext context) { } fieldType.setTokenized(true); - return new AllFieldMapper(fieldType, enabled, fieldDataSettings, context.indexSettings()); + return new AllFieldMapper(fieldType, enabled, context.indexSettings()); } } @@ -156,7 +154,9 @@ public Mapper.Builder parse(String name, Map node, ParserContext static final class AllFieldType extends MappedFieldType { - public AllFieldType() {} + public AllFieldType() { + setFieldDataType(new FieldDataType("string")); + } protected AllFieldType(AllFieldType ref) { super(ref); @@ -194,15 +194,11 @@ public Query termQuery(Object value, QueryParseContext context) { private EnabledAttributeMapper enabledState; public AllFieldMapper(Settings indexSettings, MappedFieldType existing) { - this(existing == null ? Defaults.FIELD_TYPE.clone() : existing.clone(), - Defaults.ENABLED, - existing == null ? null : (existing.fieldDataType() == null ? null : existing.fieldDataType().getSettings()), - indexSettings); + this(existing == null ? Defaults.FIELD_TYPE.clone() : existing.clone(), Defaults.ENABLED, indexSettings); } - protected AllFieldMapper(MappedFieldType fieldType, EnabledAttributeMapper enabled, - @Nullable Settings fieldDataSettings, Settings indexSettings) { - super(NAME, fieldType, false, fieldDataSettings, indexSettings); + protected AllFieldMapper(MappedFieldType fieldType, EnabledAttributeMapper enabled, Settings indexSettings) { + super(NAME, fieldType, Defaults.FIELD_TYPE, indexSettings); this.enabledState = enabled; } @@ -211,16 +207,6 @@ public boolean enabled() { return this.enabledState.enabled; } - @Override - public MappedFieldType defaultFieldType() { - return Defaults.FIELD_TYPE; - } - - @Override - public FieldDataType defaultFieldDataType() { - return new FieldDataType("string"); - } - @Override public void preParse(ParseContext context) throws IOException { } @@ -316,12 +302,6 @@ private void innerToXContent(XContentBuilder builder, boolean includeDefaults) t } else if (includeDefaults) { builder.field("similarity", SimilarityLookupService.DEFAULT_SIMILARITY); } - - if (hasCustomFieldDataSettings()) { - builder.field("fielddata", (Map) customFieldDataSettings.getAsMap()); - } else if (includeDefaults) { - builder.field("fielddata", (Map) fieldType().fieldDataType().getSettings().getAsMap()); - } } @Override diff --git a/core/src/main/java/org/elasticsearch/index/mapper/internal/FieldNamesFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/internal/FieldNamesFieldMapper.java index e03a5a96de8b1..26414c4aaca99 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/internal/FieldNamesFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/internal/FieldNamesFieldMapper.java @@ -24,7 +24,6 @@ import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexableField; import org.elasticsearch.Version; -import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.settings.Settings; @@ -33,9 +32,8 @@ import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperParsingException; -import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.MetadataFieldMapper; -import org.elasticsearch.index.mapper.core.AbstractFieldMapper; +import org.elasticsearch.index.mapper.ParseContext; import java.io.IOException; import java.util.ArrayList; @@ -59,7 +57,7 @@ public class FieldNamesFieldMapper extends MetadataFieldMapper { public static final String CONTENT_TYPE = "_field_names"; - public static class Defaults extends AbstractFieldMapper.Defaults { + public static class Defaults { public static final String NAME = FieldNamesFieldMapper.NAME; public static final boolean ENABLED = true; @@ -100,9 +98,10 @@ public Builder enabled(boolean enabled) { @Override public FieldNamesFieldMapper build(BuilderContext context) { setupFieldType(context); + fieldType.setHasDocValues(false); FieldNamesFieldType fieldNamesFieldType = (FieldNamesFieldType)fieldType; fieldNamesFieldType.setEnabled(enabled); - return new FieldNamesFieldMapper(fieldType, fieldDataSettings, context.indexSettings()); + return new FieldNamesFieldMapper(fieldType, context.indexSettings()); } } @@ -135,7 +134,9 @@ public static final class FieldNamesFieldType extends MappedFieldType { private boolean enabled = Defaults.ENABLED; - public FieldNamesFieldType() {} + public FieldNamesFieldType() { + setFieldDataType(new FieldDataType("string")); + } protected FieldNamesFieldType(FieldNamesFieldType ref) { super(ref); @@ -197,18 +198,14 @@ public boolean useTermQueryWithQueryString() { } } - private final MappedFieldType defaultFieldType; private final boolean pre13Index; // if the index was created before 1.3, _field_names is always disabled public FieldNamesFieldMapper(Settings indexSettings, MappedFieldType existing) { - this(existing == null ? Defaults.FIELD_TYPE.clone() : existing.clone(), - existing == null ? null : (existing.fieldDataType() == null ? null : existing.fieldDataType().getSettings()), - indexSettings); + this(existing == null ? Defaults.FIELD_TYPE.clone() : existing.clone(), indexSettings); } - public FieldNamesFieldMapper(MappedFieldType fieldType, @Nullable Settings fieldDataSettings, Settings indexSettings) { - super(NAME, fieldType, false, fieldDataSettings, indexSettings); - this.defaultFieldType = Defaults.FIELD_TYPE; + public FieldNamesFieldMapper(MappedFieldType fieldType, Settings indexSettings) { + super(NAME, fieldType, Defaults.FIELD_TYPE, indexSettings); this.pre13Index = Version.indexCreated(indexSettings).before(Version.V_1_3_0); if (this.pre13Index) { FieldNamesFieldType newFieldType = fieldType().clone(); @@ -223,16 +220,6 @@ public FieldNamesFieldType fieldType() { return (FieldNamesFieldType) super.fieldType(); } - @Override - public MappedFieldType defaultFieldType() { - return defaultFieldType; - } - - @Override - public FieldDataType defaultFieldDataType() { - return new FieldDataType("string"); - } - @Override public void preParse(ParseContext context) throws IOException { } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/internal/IdFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/internal/IdFieldMapper.java index 63426b5f258d0..463d2bdd2b46b 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/internal/IdFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/internal/IdFieldMapper.java @@ -46,10 +46,9 @@ import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.MergeMappingException; import org.elasticsearch.index.mapper.MergeResult; -import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.MetadataFieldMapper; +import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.Uid; -import org.elasticsearch.index.mapper.core.AbstractFieldMapper; import org.elasticsearch.index.query.QueryParseContext; import java.io.IOException; @@ -69,7 +68,7 @@ public class IdFieldMapper extends MetadataFieldMapper { public static final String CONTENT_TYPE = "_id"; - public static class Defaults extends AbstractFieldMapper.Defaults { + public static class Defaults { public static final String NAME = IdFieldMapper.NAME; public static final MappedFieldType FIELD_TYPE = new IdFieldType(); @@ -108,8 +107,8 @@ protected IndexOptions getDefaultIndexOption() { @Override public IdFieldMapper build(BuilderContext context) { - fieldType.setNames(new MappedFieldType.Names(indexName, indexName, name)); - return new IdFieldMapper(fieldType, docValues, path, fieldDataSettings, context.indexSettings()); + setupFieldType(context); + return new IdFieldMapper(fieldType, path, context.indexSettings()); } } @@ -136,7 +135,9 @@ public Mapper.Builder parse(String name, Map node, ParserContext static final class IdFieldType extends MappedFieldType { - public IdFieldType() {} + public IdFieldType() { + setFieldDataType(new FieldDataType("string")); + } protected IdFieldType(IdFieldType ref) { super(ref); @@ -228,14 +229,11 @@ public Query regexpQuery(Object value, int flags, int maxDeterminizedStates, @Nu private final String path; public IdFieldMapper(Settings indexSettings, MappedFieldType existing) { - this(idFieldType(indexSettings, existing), null, Defaults.PATH, - existing == null ? null : (existing.fieldDataType() == null ? null : existing.fieldDataType().getSettings()), - indexSettings); + this(idFieldType(indexSettings, existing), Defaults.PATH, indexSettings); } - protected IdFieldMapper(MappedFieldType fieldType, Boolean docValues, String path, - @Nullable Settings fieldDataSettings, Settings indexSettings) { - super(NAME, fieldType, docValues, fieldDataSettings, indexSettings); + protected IdFieldMapper(MappedFieldType fieldType, String path, Settings indexSettings) { + super(NAME, fieldType, Defaults.FIELD_TYPE, indexSettings); this.path = path; } @@ -255,16 +253,6 @@ public String path() { return this.path; } - @Override - public MappedFieldType defaultFieldType() { - return Defaults.FIELD_TYPE; - } - - @Override - public FieldDataType defaultFieldDataType() { - return new FieldDataType("string"); - } - @Override public void preParse(ParseContext context) throws IOException { if (context.sourceToParse().id() != null) { @@ -331,9 +319,7 @@ && hasCustomFieldDataSettings() == false) { builder.field("path", path); } - if (hasCustomFieldDataSettings()) { - builder.field("fielddata", (Map) customFieldDataSettings.getAsMap()); - } else if (includeDefaults) { + if (includeDefaults || hasCustomFieldDataSettings()) { builder.field("fielddata", (Map) fieldType().fieldDataType().getSettings().getAsMap()); } builder.endObject(); diff --git a/core/src/main/java/org/elasticsearch/index/mapper/internal/IndexFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/internal/IndexFieldMapper.java index aa9d9669d56bd..abf93b6ae3c4c 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/internal/IndexFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/internal/IndexFieldMapper.java @@ -31,7 +31,6 @@ import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.index.fielddata.FieldDataType; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperParsingException; @@ -39,7 +38,6 @@ import org.elasticsearch.index.mapper.MergeResult; import org.elasticsearch.index.mapper.MetadataFieldMapper; import org.elasticsearch.index.mapper.ParseContext; -import org.elasticsearch.index.mapper.core.AbstractFieldMapper; import org.elasticsearch.index.query.QueryParseContext; import java.io.IOException; @@ -59,7 +57,7 @@ public class IndexFieldMapper extends MetadataFieldMapper { public static final String CONTENT_TYPE = "_index"; - public static class Defaults extends AbstractFieldMapper.Defaults { + public static class Defaults { public static final String NAME = IndexFieldMapper.NAME; public static final MappedFieldType FIELD_TYPE = new IndexFieldType(); @@ -94,8 +92,9 @@ public Builder enabled(EnabledAttributeMapper enabledState) { @Override public IndexFieldMapper build(BuilderContext context) { - fieldType.setNames(new MappedFieldType.Names(indexName, indexName, name)); - return new IndexFieldMapper(fieldType, enabledState, fieldDataSettings, context.indexSettings()); + setupFieldType(context); + fieldType.setHasDocValues(false); + return new IndexFieldMapper(fieldType, enabledState, context.indexSettings()); } } @@ -207,14 +206,11 @@ public String value(Object value) { private EnabledAttributeMapper enabledState; public IndexFieldMapper(Settings indexSettings, MappedFieldType existing) { - this(existing == null ? Defaults.FIELD_TYPE.clone() : existing, - Defaults.ENABLED_STATE, - existing == null ? null : (existing.fieldDataType() == null ? null : existing.fieldDataType().getSettings()), indexSettings); + this(existing == null ? Defaults.FIELD_TYPE.clone() : existing, Defaults.ENABLED_STATE, indexSettings); } - public IndexFieldMapper(MappedFieldType fieldType, EnabledAttributeMapper enabledState, - @Nullable Settings fieldDataSettings, Settings indexSettings) { - super(NAME, fieldType, false, fieldDataSettings, indexSettings); + public IndexFieldMapper(MappedFieldType fieldType, EnabledAttributeMapper enabledState, Settings indexSettings) { + super(NAME, fieldType, Defaults.FIELD_TYPE, indexSettings); this.enabledState = enabledState; } @@ -222,16 +218,6 @@ public boolean enabled() { return this.enabledState.enabled; } - @Override - public MappedFieldType defaultFieldType() { - return Defaults.FIELD_TYPE; - } - - @Override - public FieldDataType defaultFieldDataType() { - return new FieldDataType(IndexFieldMapper.NAME); - } - public String value(Document document) { Field field = (Field) document.getField(fieldType().names().indexName()); return field == null ? null : (String)fieldType().value(field); @@ -280,13 +266,8 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (includeDefaults || enabledState != Defaults.ENABLED_STATE) { builder.field("enabled", enabledState.enabled); } - - if (indexCreatedBefore2x) { - if (hasCustomFieldDataSettings()) { - builder.field("fielddata", (Map) customFieldDataSettings.getAsMap()); - } else if (includeDefaults) { - builder.field("fielddata", (Map) fieldType().fieldDataType().getSettings().getAsMap()); - } + if (indexCreatedBefore2x && (includeDefaults || hasCustomFieldDataSettings())) { + builder.field("fielddata", (Map) fieldType().fieldDataType().getSettings().getAsMap()); } builder.endObject(); return builder; diff --git a/core/src/main/java/org/elasticsearch/index/mapper/internal/ParentFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/internal/ParentFieldMapper.java index e9407ab56cd14..ad6a45ab59408 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/internal/ParentFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/internal/ParentFieldMapper.java @@ -40,10 +40,9 @@ import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.MergeMappingException; import org.elasticsearch.index.mapper.MergeResult; -import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.MetadataFieldMapper; +import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.Uid; -import org.elasticsearch.index.mapper.core.AbstractFieldMapper; import org.elasticsearch.index.query.QueryParseContext; import java.io.IOException; @@ -64,7 +63,7 @@ public class ParentFieldMapper extends MetadataFieldMapper { public static final String NAME = "_parent"; public static final String CONTENT_TYPE = "_parent"; - public static class Defaults extends AbstractFieldMapper.Defaults { + public static class Defaults { public static final String NAME = ParentFieldMapper.NAME; public static final MappedFieldType FIELD_TYPE = new ParentFieldType(); @@ -77,7 +76,6 @@ public static class Defaults extends AbstractFieldMapper.Defaults { FIELD_TYPE.setIndexAnalyzer(Lucene.KEYWORD_ANALYZER); FIELD_TYPE.setSearchAnalyzer(Lucene.KEYWORD_ANALYZER); FIELD_TYPE.setNames(new MappedFieldType.Names(NAME)); - FIELD_TYPE.setFieldDataType(new FieldDataType("_parent", settingsBuilder().put(MappedFieldType.Loading.KEY, MappedFieldType.Loading.LAZY_VALUE))); FIELD_TYPE.freeze(); } } @@ -87,7 +85,6 @@ public static class Builder extends MetadataFieldMapper.Builder node, ParserContext static final class ParentFieldType extends MappedFieldType { - public ParentFieldType() {} + public ParentFieldType() { + setFieldDataType(new FieldDataType("_parent", settingsBuilder().put(MappedFieldType.Loading.KEY, Loading.EAGER_VALUE))); + } protected ParentFieldType(ParentFieldType ref) { super(ref); @@ -229,30 +224,23 @@ public Query termsQuery(List values, @Nullable QueryParseContext context) { private final String type; - protected ParentFieldMapper(MappedFieldType fieldType, String type, @Nullable Settings fieldDataSettings, Settings indexSettings) { - super(NAME, fieldType, Version.indexCreated(indexSettings).onOrAfter(Version.V_2_0_0), fieldDataSettings, indexSettings); + protected ParentFieldMapper(MappedFieldType fieldType, String type, Settings indexSettings) { + super(NAME, setupDocValues(indexSettings, fieldType), setupDocValues(indexSettings, Defaults.FIELD_TYPE), indexSettings); this.type = type; } public ParentFieldMapper(Settings indexSettings, MappedFieldType existing) { - this(existing == null ? Defaults.FIELD_TYPE.clone() : existing.clone(), - null, - existing == null ? null : (existing.fieldDataType() == null ? null : existing.fieldDataType().getSettings()), - indexSettings); - } - - public String type() { - return type; + this(existing == null ? Defaults.FIELD_TYPE.clone() : existing.clone(), null, indexSettings); } - @Override - public MappedFieldType defaultFieldType() { - return Defaults.FIELD_TYPE; + static MappedFieldType setupDocValues(Settings indexSettings, MappedFieldType fieldType) { + fieldType = fieldType.clone(); + fieldType.setHasDocValues(Version.indexCreated(indexSettings).onOrAfter(Version.V_2_0_0)); + return fieldType; } - @Override - public FieldDataType defaultFieldDataType() { - return new FieldDataType("_parent", settingsBuilder().put(MappedFieldType.Loading.KEY, MappedFieldType.Loading.EAGER_VALUE)); + public String type() { + return type; } @Override @@ -328,9 +316,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.startObject(CONTENT_TYPE); builder.field("type", type); - if (hasCustomFieldDataSettings()) { - builder.field("fielddata", (Map) customFieldDataSettings.getAsMap()); - } else if (includeDefaults) { + if (includeDefaults || hasCustomFieldDataSettings()) { builder.field("fielddata", (Map) fieldType().fieldDataType().getSettings().getAsMap()); } builder.endObject(); diff --git a/core/src/main/java/org/elasticsearch/index/mapper/internal/RoutingFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/internal/RoutingFieldMapper.java index 492f50a94042a..261a368ac45e3 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/internal/RoutingFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/internal/RoutingFieldMapper.java @@ -33,9 +33,8 @@ import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.MergeMappingException; import org.elasticsearch.index.mapper.MergeResult; -import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.MetadataFieldMapper; -import org.elasticsearch.index.mapper.core.AbstractFieldMapper; +import org.elasticsearch.index.mapper.ParseContext; import java.io.IOException; import java.util.Iterator; @@ -53,7 +52,7 @@ public class RoutingFieldMapper extends MetadataFieldMapper { public static final String NAME = "_routing"; public static final String CONTENT_TYPE = "_routing"; - public static class Defaults extends AbstractFieldMapper.Defaults { + public static class Defaults { public static final String NAME = "_routing"; public static final MappedFieldType FIELD_TYPE = new RoutingFieldType(); @@ -124,7 +123,9 @@ public Mapper.Builder parse(String name, Map node, ParserContext static final class RoutingFieldType extends MappedFieldType { - public RoutingFieldType() {} + public RoutingFieldType() { + setFieldDataType(new FieldDataType("string")); + } protected RoutingFieldType(RoutingFieldType ref) { super(ref); @@ -157,21 +158,11 @@ public RoutingFieldMapper(Settings indexSettings, MappedFieldType existing) { } protected RoutingFieldMapper(MappedFieldType fieldType, boolean required, String path, Settings indexSettings) { - super(NAME, fieldType, false, null, indexSettings); + super(NAME, fieldType, Defaults.FIELD_TYPE, indexSettings); this.required = required; this.path = path; } - @Override - public MappedFieldType defaultFieldType() { - return Defaults.FIELD_TYPE; - } - - @Override - public FieldDataType defaultFieldDataType() { - return new FieldDataType("string"); - } - public void markAsRequired() { this.required = true; } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/internal/SizeFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/internal/SizeFieldMapper.java index 224cb32d8dbc9..1bf0e2df209f8 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/internal/SizeFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/internal/SizeFieldMapper.java @@ -24,19 +24,15 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.analysis.NumericIntegerAnalyzer; -import org.elasticsearch.index.fielddata.FieldDataType; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.MergeMappingException; import org.elasticsearch.index.mapper.MergeResult; -import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.MetadataFieldMapper; -import org.elasticsearch.index.mapper.core.AbstractFieldMapper; +import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.core.IntegerFieldMapper; -import org.elasticsearch.index.mapper.core.NumberFieldMapper; import java.io.IOException; import java.util.Iterator; @@ -83,6 +79,7 @@ public Builder enabled(EnabledAttributeMapper enabled) { @Override public SizeFieldMapper build(BuilderContext context) { setupFieldType(context); + fieldType.setHasDocValues(false); return new SizeFieldMapper(enabledState, fieldType, context.indexSettings()); } } @@ -114,7 +111,7 @@ public SizeFieldMapper(Settings indexSettings, MappedFieldType existing) { } public SizeFieldMapper(EnabledAttributeMapper enabled, MappedFieldType fieldType, Settings indexSettings) { - super(NAME, fieldType, false, null, indexSettings); + super(NAME, fieldType, Defaults.SIZE_FIELD_TYPE, indexSettings); this.enabledState = enabled; } @@ -138,16 +135,6 @@ public void postParse(ParseContext context) throws IOException { super.parse(context); } - @Override - public MappedFieldType defaultFieldType() { - return Defaults.SIZE_FIELD_TYPE; - } - - @Override - public FieldDataType defaultFieldDataType() { - return new FieldDataType("int"); - } - @Override public Mapper parse(ParseContext context) throws IOException { // nothing to do here, we call the parent in postParse diff --git a/core/src/main/java/org/elasticsearch/index/mapper/internal/SourceFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/internal/SourceFieldMapper.java index 55274a17687d0..37e17008f13f6 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/internal/SourceFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/internal/SourceFieldMapper.java @@ -42,7 +42,6 @@ import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.support.XContentMapValues; -import org.elasticsearch.index.fielddata.FieldDataType; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperParsingException; @@ -50,7 +49,6 @@ import org.elasticsearch.index.mapper.MergeResult; import org.elasticsearch.index.mapper.MetadataFieldMapper; import org.elasticsearch.index.mapper.ParseContext; -import org.elasticsearch.index.mapper.core.AbstractFieldMapper; import java.io.BufferedInputStream; import java.io.IOException; @@ -72,7 +70,7 @@ public class SourceFieldMapper extends MetadataFieldMapper { public static final String CONTENT_TYPE = "_source"; - public static class Defaults extends AbstractFieldMapper.Defaults { + public static class Defaults { public static final String NAME = SourceFieldMapper.NAME; public static final boolean ENABLED = true; public static final long COMPRESS_THRESHOLD = -1; @@ -256,7 +254,7 @@ public SourceFieldMapper(Settings indexSettings) { protected SourceFieldMapper(boolean enabled, String format, Boolean compress, long compressThreshold, String[] includes, String[] excludes, Settings indexSettings) { - super(NAME, Defaults.FIELD_TYPE.clone(), false, null, indexSettings); // Only stored. + super(NAME, Defaults.FIELD_TYPE.clone(), Defaults.FIELD_TYPE, indexSettings); // Only stored. this.enabled = enabled; this.compress = compress; this.compressThreshold = compressThreshold; @@ -284,16 +282,6 @@ public boolean isComplete() { return complete; } - @Override - public MappedFieldType defaultFieldType() { - return Defaults.FIELD_TYPE; - } - - @Override - public FieldDataType defaultFieldDataType() { - return null; - } - @Override public void preParse(ParseContext context) throws IOException { super.parse(context); diff --git a/core/src/main/java/org/elasticsearch/index/mapper/internal/TTLFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/internal/TTLFieldMapper.java index 63c6f28e700d8..96ed142029c7c 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/internal/TTLFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/internal/TTLFieldMapper.java @@ -21,7 +21,6 @@ import org.apache.lucene.document.Field; import org.apache.lucene.index.IndexOptions; -import org.elasticsearch.common.Explicit; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; @@ -29,20 +28,16 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.AlreadyExpiredException; -import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.analysis.NumericLongAnalyzer; -import org.elasticsearch.index.fielddata.FieldDataType; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.MergeMappingException; import org.elasticsearch.index.mapper.MergeResult; -import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.MetadataFieldMapper; +import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.SourceToParse; -import org.elasticsearch.index.mapper.core.AbstractFieldMapper; import org.elasticsearch.index.mapper.core.LongFieldMapper; -import org.elasticsearch.index.mapper.core.NumberFieldMapper; import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; @@ -101,6 +96,7 @@ public Builder defaultTTL(long defaultTTL) { @Override public TTLFieldMapper build(BuilderContext context) { setupFieldType(context); + fieldType.setHasDocValues(false); return new TTLFieldMapper(fieldType, enabledState, defaultTTL, fieldDataSettings, context.indexSettings()); } } @@ -167,7 +163,7 @@ public TTLFieldMapper(Settings indexSettings) { protected TTLFieldMapper(MappedFieldType fieldType, EnabledAttributeMapper enabled, long defaultTTL, @Nullable Settings fieldDataSettings, Settings indexSettings) { - super(NAME, fieldType, false, fieldDataSettings, indexSettings); + super(NAME, fieldType, Defaults.TTL_FIELD_TYPE, indexSettings); this.enabledState = enabled; this.defaultTTL = defaultTTL; } @@ -194,16 +190,6 @@ public void postParse(ParseContext context) throws IOException { super.parse(context); } - @Override - public MappedFieldType defaultFieldType() { - return Defaults.TTL_FIELD_TYPE; - } - - @Override - public FieldDataType defaultFieldDataType() { - return new FieldDataType("long"); - } - @Override public Mapper parse(ParseContext context) throws IOException, MapperParsingException { if (context.sourceToParse().ttl() < 0) { // no ttl has been provided externally diff --git a/core/src/main/java/org/elasticsearch/index/mapper/internal/TimestampFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/internal/TimestampFieldMapper.java index 4104a2bcb14aa..ea25981f70e58 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/internal/TimestampFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/internal/TimestampFieldMapper.java @@ -24,16 +24,12 @@ import org.apache.lucene.index.IndexOptions; import org.elasticsearch.Version; import org.elasticsearch.action.TimestampParsingException; -import org.elasticsearch.common.Explicit; -import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.joda.FormatDateTimeFormatter; import org.elasticsearch.common.joda.Joda; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.analysis.NumericDateAnalyzer; -import org.elasticsearch.index.fielddata.FieldDataType; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperParsingException; @@ -41,10 +37,8 @@ import org.elasticsearch.index.mapper.MergeResult; import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.MetadataFieldMapper; -import org.elasticsearch.index.mapper.core.AbstractFieldMapper; import org.elasticsearch.index.mapper.core.DateFieldMapper; import org.elasticsearch.index.mapper.core.LongFieldMapper; -import org.elasticsearch.index.mapper.core.NumberFieldMapper; import java.io.IOException; import java.util.Iterator; @@ -59,15 +53,16 @@ public class TimestampFieldMapper extends MetadataFieldMapper { public static final String NAME = "_timestamp"; public static final String CONTENT_TYPE = "_timestamp"; - public static final String DEFAULT_DATE_TIME_FORMAT = "epoch_millis||dateOptionalTime"; + public static final String DEFAULT_DATE_TIME_FORMAT = "epoch_millis||strictDateOptionalTime"; public static class Defaults extends DateFieldMapper.Defaults { public static final String NAME = "_timestamp"; // TODO: this should be removed - public static final MappedFieldType PRE_20_FIELD_TYPE; - public static final FormatDateTimeFormatter DATE_TIME_FORMATTER = Joda.forPattern(DEFAULT_DATE_TIME_FORMAT); + public static final TimestampFieldType PRE_20_FIELD_TYPE; public static final TimestampFieldType FIELD_TYPE = new TimestampFieldType(); + public static final FormatDateTimeFormatter DATE_TIME_FORMATTER = Joda.forPattern(DEFAULT_DATE_TIME_FORMAT); + public static final FormatDateTimeFormatter DATE_TIME_FORMATTER_BEFORE_2_0 = Joda.forPattern("epoch_millis||dateOptionalTime"); static { FIELD_TYPE.setStored(true); @@ -77,9 +72,14 @@ public static class Defaults extends DateFieldMapper.Defaults { FIELD_TYPE.setDateTimeFormatter(DATE_TIME_FORMATTER); FIELD_TYPE.setIndexAnalyzer(NumericDateAnalyzer.buildNamedAnalyzer(DATE_TIME_FORMATTER, Defaults.PRECISION_STEP_64_BIT)); FIELD_TYPE.setSearchAnalyzer(NumericDateAnalyzer.buildNamedAnalyzer(DATE_TIME_FORMATTER, Integer.MAX_VALUE)); + FIELD_TYPE.setHasDocValues(true); FIELD_TYPE.freeze(); PRE_20_FIELD_TYPE = FIELD_TYPE.clone(); PRE_20_FIELD_TYPE.setStored(false); + PRE_20_FIELD_TYPE.setHasDocValues(false); + PRE_20_FIELD_TYPE.setDateTimeFormatter(DATE_TIME_FORMATTER_BEFORE_2_0); + PRE_20_FIELD_TYPE.setIndexAnalyzer(NumericDateAnalyzer.buildNamedAnalyzer(DATE_TIME_FORMATTER_BEFORE_2_0, Defaults.PRECISION_STEP_64_BIT)); + PRE_20_FIELD_TYPE.setSearchAnalyzer(NumericDateAnalyzer.buildNamedAnalyzer(DATE_TIME_FORMATTER_BEFORE_2_0, Integer.MAX_VALUE)); PRE_20_FIELD_TYPE.freeze(); } @@ -104,7 +104,8 @@ public Builder(MappedFieldType existing) { } } - DateFieldMapper.DateFieldType fieldType() { + @Override + public DateFieldMapper.DateFieldType fieldType() { return (DateFieldMapper.DateFieldType)fieldType; } @@ -144,9 +145,23 @@ public TimestampFieldMapper build(BuilderContext context) { if (explicitStore == false && context.indexCreatedVersion().before(Version.V_2_0_0)) { fieldType.setStored(false); } + + if (fieldType().dateTimeFormatter().equals(Defaults.DATE_TIME_FORMATTER)) { + fieldType().setDateTimeFormatter(getDateTimeFormatter(context.indexSettings())); + } + setupFieldType(context); - return new TimestampFieldMapper(fieldType, docValues, enabledState, path, defaultTimestamp, - ignoreMissing, fieldDataSettings, context.indexSettings()); + return new TimestampFieldMapper(fieldType, defaultFieldType, enabledState, path, defaultTimestamp, + ignoreMissing, context.indexSettings()); + } + } + + private static FormatDateTimeFormatter getDateTimeFormatter(Settings indexSettings) { + Version indexCreated = Version.indexCreated(indexSettings); + if (indexCreated.onOrAfter(Version.V_2_0_0)) { + return Defaults.DATE_TIME_FORMATTER; + } else { + return Defaults.DATE_TIME_FORMATTER_BEFORE_2_0; } } @@ -227,7 +242,7 @@ public Object valueForSearch(Object value) { } } - private static MappedFieldType defaultFieldType(Settings settings, MappedFieldType existing) { + private static MappedFieldType chooseFieldType(Settings settings, MappedFieldType existing) { if (existing != null) { return existing; } @@ -238,22 +253,18 @@ private static MappedFieldType defaultFieldType(Settings settings, MappedFieldTy private final String path; private final String defaultTimestamp; - private final MappedFieldType defaultFieldType; private final Boolean ignoreMissing; public TimestampFieldMapper(Settings indexSettings, MappedFieldType existing) { - this(defaultFieldType(indexSettings, existing).clone(), null, Defaults.ENABLED, Defaults.PATH, Defaults.DEFAULT_TIMESTAMP, null, - existing == null ? null : (existing.fieldDataType() == null ? null : existing.fieldDataType().getSettings()), - indexSettings); + this(chooseFieldType(indexSettings, existing).clone(), chooseFieldType(indexSettings, null), Defaults.ENABLED, Defaults.PATH, Defaults.DEFAULT_TIMESTAMP, null, indexSettings); } - protected TimestampFieldMapper(MappedFieldType fieldType, Boolean docValues, EnabledAttributeMapper enabledState, String path, - String defaultTimestamp, Boolean ignoreMissing, @Nullable Settings fieldDataSettings, Settings indexSettings) { - super(NAME, fieldType, docValues, fieldDataSettings, indexSettings); + protected TimestampFieldMapper(MappedFieldType fieldType, MappedFieldType defaultFieldType, EnabledAttributeMapper enabledState, String path, + String defaultTimestamp, Boolean ignoreMissing, Settings indexSettings) { + super(NAME, fieldType, defaultFieldType, indexSettings); this.enabledState = enabledState; this.path = path; this.defaultTimestamp = defaultTimestamp; - this.defaultFieldType = defaultFieldType(indexSettings, null); this.ignoreMissing = ignoreMissing; } @@ -262,16 +273,6 @@ public TimestampFieldType fieldType() { return (TimestampFieldType)super.fieldType(); } - @Override - public MappedFieldType defaultFieldType() { - return defaultFieldType; - } - - @Override - public FieldDataType defaultFieldDataType() { - return new FieldDataType("long"); - } - public boolean enabled() { return this.enabledState.enabled; } @@ -335,7 +336,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws fieldType().stored() == Defaults.FIELD_TYPE.stored() && enabledState == Defaults.ENABLED && path == Defaults.PATH && fieldType().dateTimeFormatter().format().equals(Defaults.DATE_TIME_FORMATTER.format()) && Defaults.DEFAULT_TIMESTAMP.equals(defaultTimestamp) - && defaultDocValues() == fieldType().hasDocValues()) { + && defaultFieldType.hasDocValues() == fieldType().hasDocValues()) { return builder; } builder.startObject(CONTENT_TYPE); @@ -354,7 +355,9 @@ && defaultDocValues() == fieldType().hasDocValues()) { if (indexCreatedBefore2x && (includeDefaults || path != Defaults.PATH)) { builder.field("path", path); } - if (includeDefaults || !fieldType().dateTimeFormatter().format().equals(Defaults.DATE_TIME_FORMATTER.format())) { + // different format handling depending on index version + String defaultDateFormat = indexCreatedBefore2x ? Defaults.DATE_TIME_FORMATTER_BEFORE_2_0.format() : Defaults.DATE_TIME_FORMATTER.format(); + if (includeDefaults || !fieldType().dateTimeFormatter().format().equals(defaultDateFormat)) { builder.field("format", fieldType().dateTimeFormatter().format()); } if (includeDefaults || !Defaults.DEFAULT_TIMESTAMP.equals(defaultTimestamp)) { @@ -363,12 +366,8 @@ && defaultDocValues() == fieldType().hasDocValues()) { if (includeDefaults || ignoreMissing != null) { builder.field("ignore_missing", ignoreMissing); } - if (indexCreatedBefore2x) { - if (hasCustomFieldDataSettings()) { - builder.field("fielddata", (Map) customFieldDataSettings.getAsMap()); - } else if (includeDefaults) { - builder.field("fielddata", (Map) fieldType().fieldDataType().getSettings().getAsMap()); - } + if (indexCreatedBefore2x && (includeDefaults || hasCustomFieldDataSettings())) { + builder.field("fielddata", fieldType().fieldDataType().getSettings().getAsMap()); } builder.endObject(); diff --git a/core/src/main/java/org/elasticsearch/index/mapper/internal/TypeFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/internal/TypeFieldMapper.java index 970638f371881..f5d4817ca1c86 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/internal/TypeFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/internal/TypeFieldMapper.java @@ -40,10 +40,9 @@ import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.MergeMappingException; import org.elasticsearch.index.mapper.MergeResult; -import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.MetadataFieldMapper; +import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.Uid; -import org.elasticsearch.index.mapper.core.AbstractFieldMapper; import org.elasticsearch.index.query.QueryParseContext; import java.io.IOException; @@ -61,7 +60,7 @@ public class TypeFieldMapper extends MetadataFieldMapper { public static final String CONTENT_TYPE = "_type"; - public static class Defaults extends AbstractFieldMapper.Defaults { + public static class Defaults { public static final String NAME = TypeFieldMapper.NAME; public static final MappedFieldType FIELD_TYPE = new TypeFieldType(); @@ -106,7 +105,9 @@ public Mapper.Builder parse(String name, Map node, ParserContext static final class TypeFieldType extends MappedFieldType { - public TypeFieldType() {} + public TypeFieldType() { + setFieldDataType(new FieldDataType("string")); + } protected TypeFieldType(TypeFieldType ref) { super(ref); @@ -150,20 +151,9 @@ public TypeFieldMapper(Settings indexSettings, MappedFieldType existing) { } public TypeFieldMapper(MappedFieldType fieldType, Settings indexSettings) { - super(NAME, fieldType, false, null, indexSettings); - } - - @Override - public MappedFieldType defaultFieldType() { - return Defaults.FIELD_TYPE; + super(NAME, fieldType, Defaults.FIELD_TYPE, indexSettings); } - @Override - public FieldDataType defaultFieldDataType() { - return new FieldDataType("string"); - } - - @Override public void preParse(ParseContext context) throws IOException { super.parse(context); diff --git a/core/src/main/java/org/elasticsearch/index/mapper/internal/UidFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/internal/UidFieldMapper.java index ebbd8d7a7813c..75d01407a64d2 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/internal/UidFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/internal/UidFieldMapper.java @@ -26,7 +26,6 @@ import org.apache.lucene.index.Term; import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; -import org.elasticsearch.common.Nullable; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -36,11 +35,10 @@ import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.MergeMappingException; import org.elasticsearch.index.mapper.MergeResult; +import org.elasticsearch.index.mapper.MetadataFieldMapper; import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.ParseContext.Document; -import org.elasticsearch.index.mapper.MetadataFieldMapper; import org.elasticsearch.index.mapper.Uid; -import org.elasticsearch.index.mapper.core.AbstractFieldMapper; import java.io.IOException; import java.util.List; @@ -57,7 +55,7 @@ public class UidFieldMapper extends MetadataFieldMapper { public static final String CONTENT_TYPE = "_uid"; - public static class Defaults extends AbstractFieldMapper.Defaults { + public static class Defaults { public static final String NAME = UidFieldMapper.NAME; public static final MappedFieldType FIELD_TYPE = new UidFieldType(); @@ -88,8 +86,9 @@ public Builder(MappedFieldType existing) { @Override public UidFieldMapper build(BuilderContext context) { - fieldType.setNames(new MappedFieldType.Names(indexName, indexName, name)); - return new UidFieldMapper(fieldType, docValues, fieldDataSettings, context.indexSettings()); + setupFieldType(context); + fieldType.setHasDocValues(context.indexCreatedVersion().before(Version.V_2_0_0)); + return new UidFieldMapper(fieldType, defaultFieldType, context.indexSettings()); } } @@ -107,7 +106,9 @@ public static class TypeParser implements Mapper.TypeParser { static final class UidFieldType extends MappedFieldType { - public UidFieldType() {} + public UidFieldType() { + setFieldDataType(new FieldDataType("string")); + } protected UidFieldType(UidFieldType ref) { super(ref); @@ -133,30 +134,11 @@ public Uid value(Object value) { } public UidFieldMapper(Settings indexSettings, MappedFieldType existing) { - this(existing == null ? Defaults.FIELD_TYPE.clone() : existing, null, - existing == null ? null : (existing.fieldDataType() == null ? null : existing.fieldDataType().getSettings()), - indexSettings); + this(existing == null ? Defaults.FIELD_TYPE.clone() : existing, Defaults.FIELD_TYPE, indexSettings); } - protected UidFieldMapper(MappedFieldType fieldType, Boolean docValues, @Nullable Settings fieldDataSettings, Settings indexSettings) { - super(NAME, fieldType, docValuesEnabled(docValues, indexSettings), fieldDataSettings, indexSettings); - } - - static Boolean docValuesEnabled(Boolean docValues, Settings indexSettings) { - if (Version.indexCreated(indexSettings).onOrAfter(Version.V_2_0_0)) { - return false; // explicitly disable doc values for 2.0+, for now - } - return docValues; - } - - @Override - public MappedFieldType defaultFieldType() { - return Defaults.FIELD_TYPE; - } - - @Override - public FieldDataType defaultFieldDataType() { - return new FieldDataType("string"); + protected UidFieldMapper(MappedFieldType fieldType, MappedFieldType defaultFieldType, Settings indexSettings) { + super(NAME, fieldType, defaultFieldType, indexSettings); } @Override @@ -230,9 +212,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.startObject(CONTENT_TYPE); - if (hasCustomFieldDataSettings()) { - builder.field("fielddata", (Map) customFieldDataSettings.getAsMap()); - } else if (includeDefaults) { + if (includeDefaults || hasCustomFieldDataSettings()) { builder.field("fielddata", (Map) fieldType().fieldDataType().getSettings().getAsMap()); } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/internal/VersionFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/internal/VersionFieldMapper.java index 09d8e9e115fa3..043890f8d882c 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/internal/VersionFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/internal/VersionFieldMapper.java @@ -55,6 +55,7 @@ public static class Defaults { static { FIELD_TYPE.setNames(new MappedFieldType.Names(NAME)); FIELD_TYPE.setDocValuesType(DocValuesType.NUMERIC); + FIELD_TYPE.setHasDocValues(true); FIELD_TYPE.freeze(); } } @@ -89,7 +90,9 @@ public static class TypeParser implements Mapper.TypeParser { static final class VersionFieldType extends MappedFieldType { - public VersionFieldType() {} + public VersionFieldType() { + setFieldDataType(new FieldDataType("long")); + } protected VersionFieldType(VersionFieldType ref) { super(ref); @@ -116,7 +119,7 @@ public Long value(Object value) { } public VersionFieldMapper(Settings indexSettings) { - super(NAME, Defaults.FIELD_TYPE, true, null, indexSettings); + super(NAME, Defaults.FIELD_TYPE, Defaults.FIELD_TYPE, indexSettings); } @Override @@ -148,16 +151,6 @@ public void postParse(ParseContext context) throws IOException { } } - @Override - public MappedFieldType defaultFieldType() { - return Defaults.FIELD_TYPE; - } - - @Override - public FieldDataType defaultFieldDataType() { - return new FieldDataType("long"); - } - @Override protected String contentType() { return CONTENT_TYPE; diff --git a/core/src/main/java/org/elasticsearch/index/mapper/ip/IpFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/ip/IpFieldMapper.java index b80f5051b897f..8e4e7c48783a7 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/ip/IpFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/ip/IpFieldMapper.java @@ -20,10 +20,8 @@ package org.elasticsearch.index.mapper.ip; import com.google.common.net.InetAddresses; - import org.apache.lucene.analysis.NumericTokenStream; import org.apache.lucene.document.Field; -import org.apache.lucene.document.FieldType.NumericType; import org.apache.lucene.index.IndexOptions; import org.apache.lucene.search.NumericRangeQuery; import org.apache.lucene.search.Query; @@ -45,9 +43,8 @@ import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperParsingException; -import org.elasticsearch.index.mapper.MergeMappingException; -import org.elasticsearch.index.mapper.MergeResult; import org.elasticsearch.index.mapper.ParseContext; +import org.elasticsearch.index.mapper.core.LongFieldMapper; import org.elasticsearch.index.mapper.core.LongFieldMapper.CustomLongNumericField; import org.elasticsearch.index.mapper.core.NumberFieldMapper; import org.elasticsearch.index.query.QueryParseContext; @@ -119,8 +116,8 @@ public Builder(String name) { @Override public IpFieldMapper build(BuilderContext context) { setupFieldType(context); - IpFieldMapper fieldMapper = new IpFieldMapper(name, fieldType, docValues, ignoreMalformed(context), coerce(context), - fieldDataSettings, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); + IpFieldMapper fieldMapper = new IpFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), coerce(context), + context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); fieldMapper.includeInAll(includeInAll); return fieldMapper; } @@ -158,10 +155,10 @@ public Mapper.Builder parse(String name, Map node, ParserContext } } - public static final class IpFieldType extends NumberFieldType { + public static final class IpFieldType extends LongFieldMapper.LongFieldType { public IpFieldType() { - super(NumericType.LONG); + setFieldDataType(new FieldDataType("long")); } protected IpFieldType(IpFieldType ref) { @@ -220,8 +217,8 @@ public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower } @Override - public Query fuzzyQuery(String value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions) { - long iValue = ipToLong(value); + public Query fuzzyQuery(Object value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions) { + long iValue = parseValue(value); long iSim; try { iSim = ipToLong(fuzziness.asString()); @@ -235,22 +232,10 @@ public Query fuzzyQuery(String value, Fuzziness fuzziness, int prefixLength, int } } - protected IpFieldMapper(String simpleName, MappedFieldType fieldType, Boolean docValues, + protected IpFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Explicit ignoreMalformed, Explicit coerce, - @Nullable Settings fieldDataSettings, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { - super(simpleName, fieldType, docValues, ignoreMalformed, coerce, - fieldDataSettings, indexSettings, multiFields, copyTo); - } - - @Override - public MappedFieldType defaultFieldType() { - return Defaults.FIELD_TYPE; - } - - @Override - public FieldDataType defaultFieldDataType() { - return new FieldDataType("long"); + super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, indexSettings, multiFields, copyTo); } private static long parseValue(Object value) { diff --git a/core/src/main/java/org/elasticsearch/index/mapper/object/ObjectMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/object/ObjectMapper.java index 12f0319f32a57..30592f4e9fe23 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/object/ObjectMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/object/ObjectMapper.java @@ -278,7 +278,10 @@ protected static void parseProperties(ObjectMapper.Builder objBuilder, Map> iterator = propsNode.entrySet().iterator(); while (iterator.hasNext()) { Map.Entry entry = iterator.next(); - String propName = entry.getKey(); + String fieldName = entry.getKey(); + if (fieldName.contains(".")) { + throw new MapperParsingException("Field name [" + fieldName + "] cannot contain '.'"); + } // Should accept empty arrays, as a work around for when the // user can't provide an empty Map. (PHP for example) boolean isEmptyList = entry.getValue() instanceof List && ((List) entry.getValue()).isEmpty(); @@ -301,23 +304,23 @@ protected static void parseProperties(ObjectMapper.Builder objBuilder, Map i static final FuzzyQueryBuilder PROTOTYPE = new FuzzyQueryBuilder(null, null); /** - * Constructs a new term query. + * Constructs a new fuzzy query. * * @param name The name of the field - * @param value The value of the term + * @param value The value of the text */ public FuzzyQueryBuilder(String name, Object value) { this.name = name; this.value = value; } + /** + * Constructs a new fuzzy query. + * + * @param name The name of the field + * @param value The value of the text + */ + public FuzzyQueryBuilder(String name, String value) { + this(name, (Object) value); + } + + /** + * Constructs a new fuzzy query. + * + * @param name The name of the field + * @param value The value of the text + */ + public FuzzyQueryBuilder(String name, int value) { + this(name, (Object) value); + } + + /** + * Constructs a new fuzzy query. + * + * @param name The name of the field + * @param value The value of the text + */ + public FuzzyQueryBuilder(String name, long value) { + this(name, (Object) value); + } + + /** + * Constructs a new fuzzy query. + * + * @param name The name of the field + * @param value The value of the text + */ + public FuzzyQueryBuilder(String name, float value) { + this(name, (Object) value); + } + + /** + * Constructs a new fuzzy query. + * + * @param name The name of the field + * @param value The value of the text + */ + public FuzzyQueryBuilder(String name, double value) { + this(name, (Object) value); + } + + // NO COMMIT: not sure we should also allow boolean? + /** + * Constructs a new fuzzy query. + * + * @param name The name of the field + * @param value The value of the text + */ + public FuzzyQueryBuilder(String name, boolean value) { + this(name, (Object) value); + } + + /** + * Sets the boost for this query. Documents matching this query will (in addition to the normal + * weightings) have their score multiplied by the boost provided. + */ + @Override + public FuzzyQueryBuilder boost(float boost) { + this.boost = boost; + return this; + } + public FuzzyQueryBuilder fuzziness(Fuzziness fuzziness) { this.fuzziness = fuzziness; return this; diff --git a/core/src/main/java/org/elasticsearch/index/query/FuzzyQueryParser.java b/core/src/main/java/org/elasticsearch/index/query/FuzzyQueryParser.java index 0b5e3d043e880..5152554734ae2 100644 --- a/core/src/main/java/org/elasticsearch/index/query/FuzzyQueryParser.java +++ b/core/src/main/java/org/elasticsearch/index/query/FuzzyQueryParser.java @@ -25,6 +25,7 @@ import org.apache.lucene.search.Query; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.lucene.BytesRefs; import org.elasticsearch.common.unit.Fuzziness; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.mapper.MappedFieldType; @@ -60,12 +61,12 @@ public Query parse(QueryParseContext parseContext) throws IOException, QueryPars } String fieldName = parser.currentName(); - String value = null; + Object value = null; float boost = AbstractQueryBuilder.DEFAULT_BOOST; Fuzziness fuzziness = DEFAULT_FUZZINESS; int prefixLength = FuzzyQuery.defaultPrefixLength; int maxExpansions = FuzzyQuery.defaultMaxExpansions; - boolean transpositions = false; + boolean transpositions = FuzzyQuery.defaultTranspositions; String queryName = null; MultiTermQuery.RewriteMethod rewriteMethod = null; if (parseContext.isFilter()) { @@ -79,9 +80,9 @@ public Query parse(QueryParseContext parseContext) throws IOException, QueryPars currentFieldName = parser.currentName(); } else { if ("term".equals(currentFieldName)) { - value = parser.text(); + value = parser.objectBytes(); } else if ("value".equals(currentFieldName)) { - value = parser.text(); + value = parser.objectBytes(); } else if ("boost".equals(currentFieldName)) { boost = parser.floatValue(); } else if (parseContext.parseFieldMatcher().match(currentFieldName, FUZZINESS)) { @@ -103,7 +104,7 @@ public Query parse(QueryParseContext parseContext) throws IOException, QueryPars } parser.nextToken(); } else { - value = parser.text(); + value = parser.objectBytes(); // move to the next token parser.nextToken(); } @@ -111,14 +112,15 @@ public Query parse(QueryParseContext parseContext) throws IOException, QueryPars if (value == null) { throw new QueryParsingException(parseContext, "No value specified for fuzzy query"); } - + Query query = null; MappedFieldType fieldType = parseContext.fieldMapper(fieldName); if (fieldType != null) { query = fieldType.fuzzyQuery(value, fuzziness, prefixLength, maxExpansions, transpositions); } if (query == null) { - query = new FuzzyQuery(new Term(fieldName, value), fuzziness.asDistance(value), prefixLength, maxExpansions, transpositions); + int maxEdits = fuzziness.asDistance(BytesRefs.toString(value)); + query = new FuzzyQuery(new Term(fieldName, BytesRefs.toBytesRef(value)), maxEdits, prefixLength, maxExpansions, transpositions); } if (query instanceof MultiTermQuery) { QueryParsers.setRewriteMethod((MultiTermQuery) query, rewriteMethod); diff --git a/core/src/main/java/org/elasticsearch/index/query/RegexpQueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/RegexpQueryBuilder.java index 91b740769b5f3..109e0e6d84b2f 100644 --- a/core/src/main/java/org/elasticsearch/index/query/RegexpQueryBuilder.java +++ b/core/src/main/java/org/elasticsearch/index/query/RegexpQueryBuilder.java @@ -33,7 +33,8 @@ public class RegexpQueryBuilder extends AbstractQueryBuilder private final String name; private final String regexp; - private int flags = -1; + private int flags = RegexpQueryParser.DEFAULT_FLAGS_VALUE; + private String rewrite; private int maxDeterminizedStates = Operations.DEFAULT_MAX_DETERMINIZED_STATES; private boolean maxDetermizedStatesSet; diff --git a/core/src/main/java/org/elasticsearch/index/query/RegexpQueryParser.java b/core/src/main/java/org/elasticsearch/index/query/RegexpQueryParser.java index 5843b59eec32a..c776e959689e5 100644 --- a/core/src/main/java/org/elasticsearch/index/query/RegexpQueryParser.java +++ b/core/src/main/java/org/elasticsearch/index/query/RegexpQueryParser.java @@ -37,6 +37,8 @@ */ public class RegexpQueryParser extends BaseQueryParserTemp { + public static final int DEFAULT_FLAGS_VALUE = RegexpFlag.ALL.value(); + @Inject public RegexpQueryParser() { } @@ -55,7 +57,7 @@ public Query parse(QueryParseContext parseContext) throws IOException, QueryPars Object value = null; float boost = AbstractQueryBuilder.DEFAULT_BOOST; - int flagsValue = -1; + int flagsValue = DEFAULT_FLAGS_VALUE; int maxDeterminizedStates = Operations.DEFAULT_MAX_DETERMINIZED_STATES; String queryName = null; String currentFieldName = null; diff --git a/core/src/main/java/org/elasticsearch/index/search/MatchQuery.java b/core/src/main/java/org/elasticsearch/index/search/MatchQuery.java index 5b4eb724c0ef8..49d6d4e560e11 100644 --- a/core/src/main/java/org/elasticsearch/index/search/MatchQuery.java +++ b/core/src/main/java/org/elasticsearch/index/search/MatchQuery.java @@ -65,9 +65,8 @@ public static enum ZeroTermsQuery { protected int fuzzyPrefixLength = FuzzyQuery.defaultPrefixLength; protected int maxExpansions = FuzzyQuery.defaultMaxExpansions; - - //LUCENE 4 UPGRADE we need a default value for this! - protected boolean transpositions = false; + + protected boolean transpositions = FuzzyQuery.defaultTranspositions; protected MultiTermQuery.RewriteMethod rewriteMethod; diff --git a/core/src/main/java/org/elasticsearch/index/search/stats/SearchStats.java b/core/src/main/java/org/elasticsearch/index/search/stats/SearchStats.java index 44d932304013c..2d3ee81d0ce4b 100644 --- a/core/src/main/java/org/elasticsearch/index/search/stats/SearchStats.java +++ b/core/src/main/java/org/elasticsearch/index/search/stats/SearchStats.java @@ -47,21 +47,38 @@ public static class Stats implements Streamable, ToXContent { private long fetchTimeInMillis; private long fetchCurrent; + private long scrollCount; + private long scrollTimeInMillis; + private long scrollCurrent; + Stats() { } - public Stats(long queryCount, long queryTimeInMillis, long queryCurrent, long fetchCount, long fetchTimeInMillis, long fetchCurrent) { + public Stats( + long queryCount, long queryTimeInMillis, long queryCurrent, + long fetchCount, long fetchTimeInMillis, long fetchCurrent, + long scrollCount, long scrollTimeInMillis, long scrollCurrent + ) { this.queryCount = queryCount; this.queryTimeInMillis = queryTimeInMillis; this.queryCurrent = queryCurrent; + this.fetchCount = fetchCount; this.fetchTimeInMillis = fetchTimeInMillis; this.fetchCurrent = fetchCurrent; + + this.scrollCount = scrollCount; + this.scrollTimeInMillis = scrollTimeInMillis; + this.scrollCurrent = scrollCurrent; } public Stats(Stats stats) { - this(stats.queryCount, stats.queryTimeInMillis, stats.queryCurrent, stats.fetchCount, stats.fetchTimeInMillis, stats.fetchCurrent); + this( + stats.queryCount, stats.queryTimeInMillis, stats.queryCurrent, + stats.fetchCount, stats.fetchTimeInMillis, stats.fetchCurrent, + stats.scrollCount, stats.scrollTimeInMillis, stats.scrollCurrent + ); } public void add(Stats stats) { @@ -72,6 +89,10 @@ public void add(Stats stats) { fetchCount += stats.fetchCount; fetchTimeInMillis += stats.fetchTimeInMillis; fetchCurrent += stats.fetchCurrent; + + scrollCount += stats.scrollCount; + scrollTimeInMillis += stats.scrollTimeInMillis; + scrollCurrent += stats.scrollCurrent; } public long getQueryCount() { @@ -106,6 +127,21 @@ public long getFetchCurrent() { return fetchCurrent; } + public long getScrollCount() { + return scrollCount; + } + + public TimeValue getScrollTime() { + return new TimeValue(scrollTimeInMillis); + } + + public long getScrollTimeInMillis() { + return scrollTimeInMillis; + } + + public long getScrollCurrent() { + return scrollCurrent; + } public static Stats readStats(StreamInput in) throws IOException { Stats stats = new Stats(); @@ -122,6 +158,10 @@ public void readFrom(StreamInput in) throws IOException { fetchCount = in.readVLong(); fetchTimeInMillis = in.readVLong(); fetchCurrent = in.readVLong(); + + scrollCount = in.readVLong(); + scrollTimeInMillis = in.readVLong(); + scrollCurrent = in.readVLong(); } @Override @@ -133,6 +173,10 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVLong(fetchCount); out.writeVLong(fetchTimeInMillis); out.writeVLong(fetchCurrent); + + out.writeVLong(scrollCount); + out.writeVLong(scrollTimeInMillis); + out.writeVLong(scrollCurrent); } @Override @@ -145,6 +189,10 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.timeValueField(Fields.FETCH_TIME_IN_MILLIS, Fields.FETCH_TIME, fetchTimeInMillis); builder.field(Fields.FETCH_CURRENT, fetchCurrent); + builder.field(Fields.SCROLL_TOTAL, scrollCount); + builder.timeValueField(Fields.SCROLL_TIME_IN_MILLIS, Fields.SCROLL_TIME, scrollTimeInMillis); + builder.field(Fields.SCROLL_CURRENT, scrollCurrent); + return builder; } } @@ -233,6 +281,10 @@ static final class Fields { static final XContentBuilderString FETCH_TIME = new XContentBuilderString("fetch_time"); static final XContentBuilderString FETCH_TIME_IN_MILLIS = new XContentBuilderString("fetch_time_in_millis"); static final XContentBuilderString FETCH_CURRENT = new XContentBuilderString("fetch_current"); + static final XContentBuilderString SCROLL_TOTAL = new XContentBuilderString("scroll_total"); + static final XContentBuilderString SCROLL_TIME = new XContentBuilderString("scroll_time"); + static final XContentBuilderString SCROLL_TIME_IN_MILLIS = new XContentBuilderString("scroll_time_in_millis"); + static final XContentBuilderString SCROLL_CURRENT = new XContentBuilderString("scroll_current"); } public static SearchStats readSearchStats(StreamInput in) throws IOException { diff --git a/core/src/main/java/org/elasticsearch/index/search/stats/ShardSearchStats.java b/core/src/main/java/org/elasticsearch/index/search/stats/ShardSearchStats.java index ade73676709d9..53b986b96b397 100644 --- a/core/src/main/java/org/elasticsearch/index/search/stats/ShardSearchStats.java +++ b/core/src/main/java/org/elasticsearch/index/search/stats/ShardSearchStats.java @@ -169,6 +169,15 @@ public void onFreeContext(SearchContext context) { openContexts.dec(); } + public void onNewScrollContext(SearchContext context) { + totalStats.scrollCurrent.inc(); + } + + public void onFreeScrollContext(SearchContext context) { + totalStats.scrollCurrent.dec(); + totalStats.scrollMetric.inc(TimeUnit.MILLISECONDS.toNanos(System.currentTimeMillis() - context.nowInMillis())); + } + public void onRefreshSettings(Settings settings) { slowLogSearchService.onRefreshSettings(settings); } @@ -176,21 +185,27 @@ public void onRefreshSettings(Settings settings) { final static class StatsHolder { public final MeanMetric queryMetric = new MeanMetric(); public final MeanMetric fetchMetric = new MeanMetric(); + public final MeanMetric scrollMetric = new MeanMetric(); public final CounterMetric queryCurrent = new CounterMetric(); public final CounterMetric fetchCurrent = new CounterMetric(); + public final CounterMetric scrollCurrent = new CounterMetric(); public SearchStats.Stats stats() { - return new SearchStats.Stats(queryMetric.count(), TimeUnit.NANOSECONDS.toMillis(queryMetric.sum()), queryCurrent.count(), - fetchMetric.count(), TimeUnit.NANOSECONDS.toMillis(fetchMetric.sum()), fetchCurrent.count()); + return new SearchStats.Stats( + queryMetric.count(), TimeUnit.NANOSECONDS.toMillis(queryMetric.sum()), queryCurrent.count(), + fetchMetric.count(), TimeUnit.NANOSECONDS.toMillis(fetchMetric.sum()), fetchCurrent.count(), + scrollMetric.count(), TimeUnit.NANOSECONDS.toMillis(scrollMetric.sum()), scrollCurrent.count() + ); } public long totalCurrent() { - return queryCurrent.count() + fetchCurrent.count(); + return queryCurrent.count() + fetchCurrent.count() + scrollCurrent.count(); } public void clear() { queryMetric.clear(); fetchMetric.clear(); + scrollMetric.clear(); } } } diff --git a/core/src/main/java/org/elasticsearch/index/settings/IndexDynamicSettingsModule.java b/core/src/main/java/org/elasticsearch/index/settings/IndexDynamicSettingsModule.java index 518e86b309d51..77a71f908a122 100644 --- a/core/src/main/java/org/elasticsearch/index/settings/IndexDynamicSettingsModule.java +++ b/core/src/main/java/org/elasticsearch/index/settings/IndexDynamicSettingsModule.java @@ -71,6 +71,7 @@ public IndexDynamicSettingsModule() { indexDynamicSettings.addDynamicSetting(IndexMetaData.SETTING_BLOCKS_WRITE); indexDynamicSettings.addDynamicSetting(IndexMetaData.SETTING_BLOCKS_METADATA); indexDynamicSettings.addDynamicSetting(IndexMetaData.SETTING_SHARED_FS_ALLOW_RECOVERY_ON_ANY_NODE); + indexDynamicSettings.addDynamicSetting(IndexMetaData.SETTING_PRIORITY, Validator.NON_NEGATIVE_INTEGER); indexDynamicSettings.addDynamicSetting(IndicesTTLService.INDEX_TTL_DISABLE_PURGE); indexDynamicSettings.addDynamicSetting(IndexShard.INDEX_REFRESH_INTERVAL, Validator.TIME); indexDynamicSettings.addDynamicSetting(GatewayAllocator.INDEX_RECOVERY_INITIAL_SHARDS); diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 7c3caf09c7d89..4605ce3d749e2 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -741,7 +741,11 @@ public SnapshotIndexCommit snapshotIndex(boolean flushFirst) throws EngineExcept } } - public void failShard(String reason, Throwable e) { + /** + * Fails the shard and marks the shard store as corrupted if + * e is caused by index corruption + */ + public void failShard(String reason, @Nullable Throwable e) { // fail the engine. This will cause this shard to also be removed from the node's index service. engine().failEngine(reason, e); } @@ -1271,18 +1275,11 @@ class ShardEngineFailListener implements Engine.FailedEngineListener { // called by the current engine @Override public void onFailedEngine(ShardId shardId, String reason, @Nullable Throwable failure) { - try { - // mark as corrupted, so opening the store will fail - store.markStoreCorrupted(new IOException("failed engine (reason: [" + reason + "])", failure)); - } catch (IOException e) { - logger.warn("failed to mark shard store as corrupted", e); - } finally { - for (Engine.FailedEngineListener listener : delegates) { - try { - listener.onFailedEngine(shardId, reason, failure); - } catch (Exception e) { - logger.warn("exception while notifying engine failure", e); - } + for (Engine.FailedEngineListener listener : delegates) { + try { + listener.onFailedEngine(shardId, reason, failure); + } catch (Exception e) { + logger.warn("exception while notifying engine failure", e); } } } diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java b/core/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java index 2b691f558e17c..a466147e71c12 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java @@ -68,7 +68,8 @@ public RecoveryResponse recoverToTarget() { // that case, fail the shard to reallocate a new IndexShard and // create a new IndexWriter logger.info("recovery failed for primary shadow shard, failing shard"); - shard.failShard("primary relocation failed on shared filesystem", t); + // pass the failure as null, as we want to ensure the store is not marked as corrupted + shard.failShard("primary relocation failed on shared filesystem caused by: [" + t.getMessage() + "]", null); } else { logger.info("recovery failed on shared filesystem", t); } diff --git a/core/src/main/java/org/elasticsearch/monitor/MonitorModule.java b/core/src/main/java/org/elasticsearch/monitor/MonitorModule.java index a5b9118e9a5f6..db876d9189325 100644 --- a/core/src/main/java/org/elasticsearch/monitor/MonitorModule.java +++ b/core/src/main/java/org/elasticsearch/monitor/MonitorModule.java @@ -23,12 +23,8 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.monitor.fs.FsProbe; import org.elasticsearch.monitor.fs.FsService; -import org.elasticsearch.monitor.fs.JmxFsProbe; import org.elasticsearch.monitor.jvm.JvmMonitorService; import org.elasticsearch.monitor.jvm.JvmService; -import org.elasticsearch.monitor.network.JmxNetworkProbe; -import org.elasticsearch.monitor.network.NetworkProbe; -import org.elasticsearch.monitor.network.NetworkService; import org.elasticsearch.monitor.os.JmxOsProbe; import org.elasticsearch.monitor.os.OsProbe; import org.elasticsearch.monitor.os.OsService; @@ -56,13 +52,11 @@ protected void configure() { // bind default implementations bind(ProcessProbe.class).to(JmxProcessProbe.class).asEagerSingleton(); bind(OsProbe.class).to(JmxOsProbe.class).asEagerSingleton(); - bind(NetworkProbe.class).to(JmxNetworkProbe.class).asEagerSingleton(); - bind(FsProbe.class).to(JmxFsProbe.class).asEagerSingleton(); + bind(FsProbe.class).asEagerSingleton(); // bind other services bind(ProcessService.class).asEagerSingleton(); bind(OsService.class).asEagerSingleton(); - bind(NetworkService.class).asEagerSingleton(); bind(JvmService.class).asEagerSingleton(); bind(FsService.class).asEagerSingleton(); diff --git a/core/src/main/java/org/elasticsearch/monitor/MonitorService.java b/core/src/main/java/org/elasticsearch/monitor/MonitorService.java index 8307e7fd4e57b..d7ab517617ae4 100644 --- a/core/src/main/java/org/elasticsearch/monitor/MonitorService.java +++ b/core/src/main/java/org/elasticsearch/monitor/MonitorService.java @@ -25,7 +25,6 @@ import org.elasticsearch.monitor.fs.FsService; import org.elasticsearch.monitor.jvm.JvmMonitorService; import org.elasticsearch.monitor.jvm.JvmService; -import org.elasticsearch.monitor.network.NetworkService; import org.elasticsearch.monitor.os.OsService; import org.elasticsearch.monitor.process.ProcessService; @@ -42,20 +41,17 @@ public class MonitorService extends AbstractLifecycleComponent { private final JvmService jvmService; - private final NetworkService networkService; - private final FsService fsService; @Inject public MonitorService(Settings settings, JvmMonitorService jvmMonitorService, - OsService osService, ProcessService processService, JvmService jvmService, NetworkService networkService, + OsService osService, ProcessService processService, JvmService jvmService, FsService fsService) { super(settings); this.jvmMonitorService = jvmMonitorService; this.osService = osService; this.processService = processService; this.jvmService = jvmService; - this.networkService = networkService; this.fsService = fsService; } @@ -71,10 +67,6 @@ public JvmService jvmService() { return this.jvmService; } - public NetworkService networkService() { - return this.networkService; - } - public FsService fsService() { return this.fsService; } diff --git a/core/src/main/java/org/elasticsearch/monitor/fs/FsStats.java b/core/src/main/java/org/elasticsearch/monitor/fs/FsInfo.java similarity index 52% rename from core/src/main/java/org/elasticsearch/monitor/fs/FsStats.java rename to core/src/main/java/org/elasticsearch/monitor/fs/FsInfo.java index fcb37ec6a6d90..6fcbec5ee9070 100644 --- a/core/src/main/java/org/elasticsearch/monitor/fs/FsStats.java +++ b/core/src/main/java/org/elasticsearch/monitor/fs/FsInfo.java @@ -21,7 +21,6 @@ import com.google.common.collect.Iterators; import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Streamable; @@ -35,54 +34,37 @@ import java.util.Iterator; import java.util.Set; -/** - */ -public class FsStats implements Iterable, Streamable, ToXContent { +public class FsInfo implements Iterable, Streamable, ToXContent { - public static class Info implements Streamable, ToXContent { + public static class Path implements Streamable, ToXContent { String path; @Nullable String mount; - @Nullable - String dev; /** File system type from {@code java.nio.file.FileStore type()}, if available. */ @Nullable String type; long total = -1; long free = -1; long available = -1; - long diskReads = -1; - long diskWrites = -1; - long diskReadBytes = -1; - long diskWriteBytes = -1; - double diskQueue = -1; - double diskServiceTime = -1; + /** Uses Lucene's {@code IOUtils.spins} method to try to determine if the device backed by spinning media. * This is null if we could not determine it, true if it possibly spins, else false. */ Boolean spins = null; - public Info() { + public Path() { } - public Info(String path, @Nullable String mount, @Nullable String dev, long total, long free, long available, long diskReads, - long diskWrites, long diskReadBytes, long diskWriteBytes, double diskQueue, double diskServiceTime) { + public Path(String path, @Nullable String mount, long total, long free, long available) { this.path = path; this.mount = mount; - this.dev = dev; this.total = total; this.free = free; this.available = available; - this.diskReads = diskReads; - this.diskWrites = diskWrites; - this.diskReadBytes = diskReadBytes; - this.diskWriteBytes = diskWriteBytes; - this.diskQueue = diskQueue; - this.diskServiceTime = diskServiceTime; } - static public Info readInfoFrom(StreamInput in) throws IOException { - Info i = new Info(); + static public Path readInfoFrom(StreamInput in) throws IOException { + Path i = new Path(); i.readFrom(in); return i; } @@ -91,17 +73,10 @@ static public Info readInfoFrom(StreamInput in) throws IOException { public void readFrom(StreamInput in) throws IOException { path = in.readOptionalString(); mount = in.readOptionalString(); - dev = in.readOptionalString(); type = in.readOptionalString(); total = in.readLong(); free = in.readLong(); available = in.readLong(); - diskReads = in.readLong(); - diskWrites = in.readLong(); - diskReadBytes = in.readLong(); - diskWriteBytes = in.readLong(); - diskQueue = in.readDouble(); - diskServiceTime = in.readDouble(); spins = in.readOptionalBoolean(); } @@ -109,17 +84,10 @@ public void readFrom(StreamInput in) throws IOException { public void writeTo(StreamOutput out) throws IOException { out.writeOptionalString(path); // total aggregates do not have a path out.writeOptionalString(mount); - out.writeOptionalString(dev); out.writeOptionalString(type); out.writeLong(total); out.writeLong(free); out.writeLong(available); - out.writeLong(diskReads); - out.writeLong(diskWrites); - out.writeLong(diskReadBytes); - out.writeLong(diskWriteBytes); - out.writeDouble(diskQueue); - out.writeDouble(diskServiceTime); out.writeOptionalBoolean(spins); } @@ -131,10 +99,6 @@ public String getMount() { return mount; } - public String getDev() { - return dev; - } - public String getType() { return type; } @@ -151,38 +115,6 @@ public ByteSizeValue getAvailable() { return new ByteSizeValue(available); } - public long getDiskReads() { - return this.diskReads; - } - - public long getDiskWrites() { - return this.diskWrites; - } - - public long getDiskReadSizeInBytes() { - return diskReadBytes; - } - - public ByteSizeValue getDiskReadSizeSize() { - return new ByteSizeValue(diskReadBytes); - } - - public long getDiskWriteSizeInBytes() { - return diskWriteBytes; - } - - public ByteSizeValue getDiskWriteSizeSize() { - return new ByteSizeValue(diskWriteBytes); - } - - public double getDiskQueue() { - return diskQueue; - } - - public double getDiskServiceTime() { - return diskServiceTime; - } - public Boolean getSpins() { return spins; } @@ -207,17 +139,11 @@ private double addDouble(double current, double other) { return current + other; } - public void add(Info info) { - total = addLong(total, info.total); - free = addLong(free, info.free); - available = addLong(available, info.available); - diskReads = addLong(diskReads, info.diskReads); - diskWrites = addLong(diskWrites, info.diskWrites); - diskReadBytes = addLong(diskReadBytes, info.diskReadBytes); - diskWriteBytes = addLong(diskWriteBytes, info.diskWriteBytes); - diskQueue = addDouble(diskQueue, info.diskQueue); - diskServiceTime = addDouble(diskServiceTime, info.diskServiceTime); - if (info.spins != null && info.spins.booleanValue()) { + public void add(Path path) { + total = addLong(total, path.total); + free = addLong(free, path.free); + available = addLong(available, path.available); + if (path.spins != null && path.spins.booleanValue()) { // Spinning is contagious! spins = Boolean.TRUE; } @@ -226,7 +152,6 @@ public void add(Info info) { static final class Fields { static final XContentBuilderString PATH = new XContentBuilderString("path"); static final XContentBuilderString MOUNT = new XContentBuilderString("mount"); - static final XContentBuilderString DEV = new XContentBuilderString("dev"); static final XContentBuilderString TYPE = new XContentBuilderString("type"); static final XContentBuilderString TOTAL = new XContentBuilderString("total"); static final XContentBuilderString TOTAL_IN_BYTES = new XContentBuilderString("total_in_bytes"); @@ -234,17 +159,6 @@ static final class Fields { static final XContentBuilderString FREE_IN_BYTES = new XContentBuilderString("free_in_bytes"); static final XContentBuilderString AVAILABLE = new XContentBuilderString("available"); static final XContentBuilderString AVAILABLE_IN_BYTES = new XContentBuilderString("available_in_bytes"); - static final XContentBuilderString DISK_READS = new XContentBuilderString("disk_reads"); - static final XContentBuilderString DISK_WRITES = new XContentBuilderString("disk_writes"); - static final XContentBuilderString DISK_IO_OP = new XContentBuilderString("disk_io_op"); - static final XContentBuilderString DISK_READ_SIZE = new XContentBuilderString("disk_read_size"); - static final XContentBuilderString DISK_READ_SIZE_IN_BYTES = new XContentBuilderString("disk_read_size_in_bytes"); - static final XContentBuilderString DISK_WRITE_SIZE = new XContentBuilderString("disk_write_size"); - static final XContentBuilderString DISK_WRITE_SIZE_IN_BYTES = new XContentBuilderString("disk_write_size_in_bytes"); - static final XContentBuilderString DISK_IO_SIZE = new XContentBuilderString("disk_io_size"); - static final XContentBuilderString DISK_IO_IN_BYTES = new XContentBuilderString("disk_io_size_in_bytes"); - static final XContentBuilderString DISK_QUEUE = new XContentBuilderString("disk_queue"); - static final XContentBuilderString DISK_SERVICE_TIME = new XContentBuilderString("disk_service_time"); static final XContentBuilderString SPINS = new XContentBuilderString("spins"); } @@ -257,9 +171,6 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (mount != null) { builder.field(Fields.MOUNT, mount, XContentBuilder.FieldCaseConversion.NONE); } - if (dev != null) { - builder.field(Fields.DEV, dev, XContentBuilder.FieldCaseConversion.NONE); - } if (type != null) { builder.field(Fields.TYPE, type, XContentBuilder.FieldCaseConversion.NONE); } @@ -273,51 +184,6 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (available != -1) { builder.byteSizeField(Fields.AVAILABLE_IN_BYTES, Fields.AVAILABLE, available); } - - long iop = -1; - - if (diskReads != -1) { - iop = diskReads; - builder.field(Fields.DISK_READS, diskReads); - } - if (diskWrites != -1) { - if (iop != -1) { - iop += diskWrites; - } else { - iop = diskWrites; - } - builder.field(Fields.DISK_WRITES, diskWrites); - } - - if (iop != -1) { - builder.field(Fields.DISK_IO_OP, iop); - } - - long ioBytes = -1; - - if (diskReadBytes != -1) { - ioBytes = diskReadBytes; - builder.byteSizeField(Fields.DISK_READ_SIZE_IN_BYTES, Fields.DISK_READ_SIZE, diskReadBytes); - } - if (diskWriteBytes != -1) { - if (ioBytes != -1) { - ioBytes += diskWriteBytes; - } else { - ioBytes = diskWriteBytes; - } - builder.byteSizeField(Fields.DISK_WRITE_SIZE_IN_BYTES, Fields.DISK_WRITE_SIZE, diskWriteBytes); - } - - if (ioBytes != -1) { - builder.byteSizeField(Fields.DISK_IO_IN_BYTES, Fields.DISK_IO_SIZE, ioBytes); - } - - if (diskQueue != -1) { - builder.field(Fields.DISK_QUEUE, Strings.format1Decimals(diskQueue, "")); - } - if (diskServiceTime != -1) { - builder.field(Fields.DISK_SERVICE_TIME, Strings.format1Decimals(diskServiceTime, "")); - } if (spins != null) { builder.field(Fields.SPINS, spins.toString()); } @@ -328,36 +194,36 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws } long timestamp; - Info total; - Info[] infos; + Path total; + Path[] paths; - FsStats() { + FsInfo() { } - public FsStats(long timestamp, Info[] infos) { + public FsInfo(long timestamp, Path[] paths) { this.timestamp = timestamp; - this.infos = infos; + this.paths = paths; this.total = null; } - public Info getTotal() { + public Path getTotal() { return total(); } - public Info total() { + public Path total() { if (total != null) { return total; } - Info res = new Info(); - Set seenDevices = new HashSet<>(infos.length); - for (Info subInfo : infos) { - if (subInfo.dev != null) { - if (!seenDevices.add(subInfo.dev)) { + Path res = new Path(); + Set seenDevices = new HashSet<>(paths.length); + for (Path subPath : paths) { + if (subPath.path != null) { + if (!seenDevices.add(subPath.path)) { continue; // already added numbers for this device; } } - res.add(subInfo); + res.add(subPath); } total = res; return res; @@ -368,12 +234,12 @@ public long getTimestamp() { } @Override - public Iterator iterator() { - return Iterators.forArray(infos); + public Iterator iterator() { + return Iterators.forArray(paths); } - public static FsStats readFsStats(StreamInput in) throws IOException { - FsStats stats = new FsStats(); + public static FsInfo readFsInfo(StreamInput in) throws IOException { + FsInfo stats = new FsInfo(); stats.readFrom(in); return stats; } @@ -381,18 +247,18 @@ public static FsStats readFsStats(StreamInput in) throws IOException { @Override public void readFrom(StreamInput in) throws IOException { timestamp = in.readVLong(); - infos = new Info[in.readVInt()]; - for (int i = 0; i < infos.length; i++) { - infos[i] = Info.readInfoFrom(in); + paths = new Path[in.readVInt()]; + for (int i = 0; i < paths.length; i++) { + paths[i] = Path.readInfoFrom(in); } } @Override public void writeTo(StreamOutput out) throws IOException { out.writeVLong(timestamp); - out.writeVInt(infos.length); - for (Info info : infos) { - info.writeTo(out); + out.writeVInt(paths.length); + for (Path path : paths) { + path.writeTo(out); } } @@ -410,8 +276,8 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(Fields.TOTAL); total().toXContent(builder, params); builder.startArray(Fields.DATA); - for (Info info : infos) { - info.toXContent(builder, params); + for (Path path : paths) { + path.toXContent(builder, params); } builder.endArray(); builder.endObject(); diff --git a/core/src/main/java/org/elasticsearch/monitor/fs/FsProbe.java b/core/src/main/java/org/elasticsearch/monitor/fs/FsProbe.java index 38e15ddec8fa7..56bc352a5bc6d 100644 --- a/core/src/main/java/org/elasticsearch/monitor/fs/FsProbe.java +++ b/core/src/main/java/org/elasticsearch/monitor/fs/FsProbe.java @@ -19,11 +19,49 @@ package org.elasticsearch.monitor.fs; +import org.elasticsearch.common.component.AbstractComponent; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.env.NodeEnvironment; +import org.elasticsearch.env.NodeEnvironment.NodePath; + import java.io.IOException; -/** - */ -public interface FsProbe { +public class FsProbe extends AbstractComponent { + + private final NodeEnvironment nodeEnv; + + @Inject + public FsProbe(Settings settings, NodeEnvironment nodeEnv) { + super(settings); + this.nodeEnv = nodeEnv; + } + + public FsInfo stats() throws IOException { + if (!nodeEnv.hasNodeFile()) { + return new FsInfo(System.currentTimeMillis(), new FsInfo.Path[0]); + } + NodePath[] dataLocations = nodeEnv.nodePaths(); + FsInfo.Path[] paths = new FsInfo.Path[dataLocations.length]; + for (int i = 0; i < dataLocations.length; i++) { + paths[i] = getFSInfo(dataLocations[i]); + } + return new FsInfo(System.currentTimeMillis(), paths); + } + + public static FsInfo.Path getFSInfo(NodePath nodePath) throws IOException { + FsInfo.Path fsPath = new FsInfo.Path(); + fsPath.path = nodePath.path.toAbsolutePath().toString(); - FsStats stats() throws IOException; + // NOTE: we use already cached (on node startup) FileStore and spins + // since recomputing these once per second (default) could be costly, + // and they should not change: + fsPath.total = nodePath.fileStore.getTotalSpace(); + fsPath.free = nodePath.fileStore.getUnallocatedSpace(); + fsPath.available = nodePath.fileStore.getUsableSpace(); + fsPath.type = nodePath.fileStore.type(); + fsPath.mount = nodePath.fileStore.toString(); + fsPath.spins = nodePath.spins; + return fsPath; + } } diff --git a/core/src/main/java/org/elasticsearch/monitor/fs/FsService.java b/core/src/main/java/org/elasticsearch/monitor/fs/FsService.java index 506b42e549b2e..c95a7bf8b3aba 100644 --- a/core/src/main/java/org/elasticsearch/monitor/fs/FsService.java +++ b/core/src/main/java/org/elasticsearch/monitor/fs/FsService.java @@ -33,33 +33,33 @@ public class FsService extends AbstractComponent { private final FsProbe probe; - private final SingleObjectCache fsStatsCache; + private final SingleObjectCache fsStatsCache; @Inject public FsService(Settings settings, FsProbe probe) throws IOException { super(settings); this.probe = probe; TimeValue refreshInterval = settings.getAsTime("monitor.fs.refresh_interval", TimeValue.timeValueSeconds(1)); - fsStatsCache = new FsStatsCache(refreshInterval, probe.stats()); + fsStatsCache = new FsInfoCache(refreshInterval, probe.stats()); logger.debug("Using probe [{}] with refresh_interval [{}]", probe, refreshInterval); } - public FsStats stats() { + public FsInfo stats() { return fsStatsCache.getOrRefresh(); } - private class FsStatsCache extends SingleObjectCache { - public FsStatsCache(TimeValue interval, FsStats initValue) { + private class FsInfoCache extends SingleObjectCache { + public FsInfoCache(TimeValue interval, FsInfo initValue) { super(interval, initValue); } @Override - protected FsStats refresh() { + protected FsInfo refresh() { try { return probe.stats(); } catch (IOException ex) { logger.warn("Failed to fetch fs stats - returning empty instance"); - return new FsStats(); + return new FsInfo(); } } } diff --git a/core/src/main/java/org/elasticsearch/monitor/fs/JmxFsProbe.java b/core/src/main/java/org/elasticsearch/monitor/fs/JmxFsProbe.java deleted file mode 100644 index b38f4dbf071f4..0000000000000 --- a/core/src/main/java/org/elasticsearch/monitor/fs/JmxFsProbe.java +++ /dev/null @@ -1,68 +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.monitor.fs; - -import org.elasticsearch.common.component.AbstractComponent; -import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.env.NodeEnvironment; -import org.elasticsearch.env.NodeEnvironment.NodePath; - -import java.io.IOException; - -public class JmxFsProbe extends AbstractComponent implements FsProbe { - - private final NodeEnvironment nodeEnv; - - @Inject - public JmxFsProbe(Settings settings, NodeEnvironment nodeEnv) { - super(settings); - this.nodeEnv = nodeEnv; - } - - @Override - public FsStats stats() throws IOException { - if (!nodeEnv.hasNodeFile()) { - return new FsStats(System.currentTimeMillis(), new FsStats.Info[0]); - } - NodePath[] dataLocations = nodeEnv.nodePaths(); - FsStats.Info[] infos = new FsStats.Info[dataLocations.length]; - for (int i = 0; i < dataLocations.length; i++) { - infos[i] = getFSInfo(dataLocations[i]); - } - return new FsStats(System.currentTimeMillis(), infos); - } - - public static FsStats.Info getFSInfo(NodePath nodePath) throws IOException { - FsStats.Info info = new FsStats.Info(); - info.path = nodePath.path.toAbsolutePath().toString(); - - // NOTE: we use already cached (on node startup) FileStore and spins - // since recomputing these once per second (default) could be costly, - // and they should not change: - info.total = nodePath.fileStore.getTotalSpace(); - info.free = nodePath.fileStore.getUnallocatedSpace(); - info.available = nodePath.fileStore.getUsableSpace(); - info.type = nodePath.fileStore.type(); - info.mount = nodePath.fileStore.toString(); - info.spins = nodePath.spins; - return info; - } -} diff --git a/core/src/main/java/org/elasticsearch/monitor/network/NetworkInfo.java b/core/src/main/java/org/elasticsearch/monitor/network/NetworkInfo.java deleted file mode 100644 index 77c9c9460ba10..0000000000000 --- a/core/src/main/java/org/elasticsearch/monitor/network/NetworkInfo.java +++ /dev/null @@ -1,161 +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.monitor.network; - -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.io.stream.Streamable; -import org.elasticsearch.common.xcontent.ToXContent; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentBuilderString; - -import java.io.IOException; - -/** - * - */ -public class NetworkInfo implements Streamable, ToXContent { - - public static final Interface NA_INTERFACE = new Interface(); - - long refreshInterval; - - Interface primary = NA_INTERFACE; - - public long refreshInterval() { - return this.refreshInterval; - } - - public long getRefreshInterval() { - return this.refreshInterval; - } - - public Interface primaryInterface() { - return primary; - } - - public Interface getPrimaryInterface() { - return primaryInterface(); - } - - static final class Fields { - static final XContentBuilderString NETWORK = new XContentBuilderString("network"); - static final XContentBuilderString REFRESH_INTERVAL = new XContentBuilderString("refresh_interval"); - static final XContentBuilderString REFRESH_INTERVAL_IN_MILLIS = new XContentBuilderString("refresh_interval_in_millis"); - static final XContentBuilderString PRIMARY_INTERFACE = new XContentBuilderString("primary_interface"); - static final XContentBuilderString ADDRESS = new XContentBuilderString("address"); - static final XContentBuilderString NAME = new XContentBuilderString("name"); - static final XContentBuilderString MAC_ADDRESS = new XContentBuilderString("mac_address"); - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(Fields.NETWORK); - builder.timeValueField(Fields.REFRESH_INTERVAL_IN_MILLIS, Fields.REFRESH_INTERVAL, refreshInterval); - if (primary != NA_INTERFACE) { - builder.startObject(Fields.PRIMARY_INTERFACE); - builder.field(Fields.ADDRESS, primary.address()); - builder.field(Fields.NAME, primary.name()); - builder.field(Fields.MAC_ADDRESS, primary.macAddress()); - builder.endObject(); - } - builder.endObject(); - return builder; - } - - public static NetworkInfo readNetworkInfo(StreamInput in) throws IOException { - NetworkInfo info = new NetworkInfo(); - info.readFrom(in); - return info; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - refreshInterval = in.readLong(); - primary = Interface.readNetworkInterface(in); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeLong(refreshInterval); - primary.writeTo(out); - } - - public static class Interface implements Streamable { - - private String name = ""; - private String address = ""; - private String macAddress = ""; - - private Interface() { - } - - public Interface(String name, String address, String macAddress) { - this.name = name; - this.address = address; - this.macAddress = macAddress; - } - - public String name() { - return name; - } - - public String getName() { - return name(); - } - - public String address() { - return address; - } - - public String getAddress() { - return address(); - } - - public String macAddress() { - return macAddress; - } - - public String getMacAddress() { - return macAddress(); - } - - public static Interface readNetworkInterface(StreamInput in) throws IOException { - Interface inf = new Interface(); - inf.readFrom(in); - return inf; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - name = in.readString(); - address = in.readString(); - macAddress = in.readString(); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeString(name); - out.writeString(address); - out.writeString(macAddress); - } - - } -} diff --git a/core/src/main/java/org/elasticsearch/monitor/network/NetworkService.java b/core/src/main/java/org/elasticsearch/monitor/network/NetworkService.java deleted file mode 100644 index 8e239a30231e1..0000000000000 --- a/core/src/main/java/org/elasticsearch/monitor/network/NetworkService.java +++ /dev/null @@ -1,130 +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.monitor.network; - -import org.elasticsearch.common.component.AbstractComponent; -import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.SingleObjectCache; - -import java.net.InetAddress; -import java.net.NetworkInterface; -import java.util.Enumeration; - -/** - * - */ -public final class NetworkService extends AbstractComponent { - - private final NetworkProbe probe; - - private final NetworkInfo info; - - private final SingleObjectCache networkStatsCache; - - @Inject - public NetworkService(Settings settings, NetworkProbe probe) { - super(settings); - this.probe = probe; - - TimeValue refreshInterval = settings.getAsTime("monitor.network.refresh_interval", TimeValue.timeValueSeconds(5)); - - logger.debug("Using probe [{}] with refresh_interval [{}]", probe, refreshInterval); - - this.info = probe.networkInfo(); - this.info.refreshInterval = refreshInterval.millis(); - networkStatsCache = new NetworkStatsCache(refreshInterval, probe.networkStats()); - if (logger.isDebugEnabled()) { - StringBuilder netDebug = new StringBuilder("net_info"); - try { - Enumeration interfaces = NetworkInterface.getNetworkInterfaces(); - String hostName = InetAddress.getLocalHost().getHostName(); - netDebug.append("\nhost [").append(hostName).append("]\n"); - while (interfaces.hasMoreElements()) { - NetworkInterface net = interfaces.nextElement(); - - netDebug.append(net.getName()).append('\t').append("display_name [").append(net.getDisplayName()).append("]\n"); - Enumeration addresses = net.getInetAddresses(); - netDebug.append("\t\taddress "); - while (addresses.hasMoreElements()) { - netDebug.append("[").append(addresses.nextElement()).append("] "); - } - netDebug.append('\n'); - netDebug.append("\t\tmtu [").append(net.getMTU()).append("] multicast [").append(net.supportsMulticast()).append("] ptp [").append(net.isPointToPoint()) - .append("] loopback [").append(net.isLoopback()).append("] up [").append(net.isUp()).append("] virtual [").append(net.isVirtual()).append("]") - .append('\n'); - - Enumeration subInterfaces = net.getSubInterfaces(); - if (subInterfaces != null && subInterfaces.hasMoreElements()) { - netDebug.append("\t\t\tsub interfaces:\n"); - - while (subInterfaces.hasMoreElements()) { - - net = subInterfaces.nextElement(); - - netDebug.append("\t\t\t").append(net.getName()).append("\t").append("display_name [").append(net.getDisplayName()).append("]\n"); - addresses = net.getInetAddresses(); - netDebug.append("\t\t\t\t\taddress "); - while (addresses.hasMoreElements()) { - netDebug.append("[").append(addresses.nextElement()).append("] "); - } - netDebug.append('\n'); - netDebug.append("\t\t\t\t\tmtu [").append(net.getMTU()).append("] multicast [").append(net.supportsMulticast()).append("] ptp [").append(net.isPointToPoint()) - .append("] loopback [").append(net.isLoopback()).append("] up [").append(net.isUp()).append("] virtual [").append(net.isVirtual()).append("]") - .append('\n'); - } - } - } - } catch (Exception ex) { - netDebug.append("failed to get Network Interface Info [" + ex.getMessage() + "]"); - } - logger.debug(netDebug.toString()); - } - - if (logger.isTraceEnabled()) { - logger.trace("ifconfig\n\n" + ifconfig()); - } - stats(); // pull the stats one time - } - - public NetworkInfo info() { - return this.info; - } - - public NetworkStats stats() { - return networkStatsCache.getOrRefresh(); - } - - private class NetworkStatsCache extends SingleObjectCache { - public NetworkStatsCache(TimeValue interval, NetworkStats initValue) { - super(interval, initValue); - } - - @Override - protected NetworkStats refresh() { - return probe.networkStats(); - } - } - - public String ifconfig() { - return probe.ifconfig(); - } -} diff --git a/core/src/main/java/org/elasticsearch/monitor/network/NetworkStats.java b/core/src/main/java/org/elasticsearch/monitor/network/NetworkStats.java deleted file mode 100644 index a5d4671ef8143..0000000000000 --- a/core/src/main/java/org/elasticsearch/monitor/network/NetworkStats.java +++ /dev/null @@ -1,248 +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.monitor.network; - -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.io.stream.Streamable; -import org.elasticsearch.common.xcontent.ToXContent; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentBuilderString; - -import java.io.IOException; - -/** - * - */ -public class NetworkStats implements Streamable, ToXContent { - - long timestamp; - - Tcp tcp = null; - - NetworkStats() { - - } - - static final class Fields { - static final XContentBuilderString NETWORK = new XContentBuilderString("network"); - static final XContentBuilderString TCP = new XContentBuilderString("tcp"); - static final XContentBuilderString ACTIVE_OPENS = new XContentBuilderString("active_opens"); - static final XContentBuilderString PASSIVE_OPENS = new XContentBuilderString("passive_opens"); - static final XContentBuilderString CURR_ESTAB = new XContentBuilderString("curr_estab"); - static final XContentBuilderString IN_SEGS = new XContentBuilderString("in_segs"); - static final XContentBuilderString OUT_SEGS = new XContentBuilderString("out_segs"); - static final XContentBuilderString RETRANS_SEGS = new XContentBuilderString("retrans_segs"); - static final XContentBuilderString ESTAB_RESETS = new XContentBuilderString("estab_resets"); - static final XContentBuilderString ATTEMPT_FAILS = new XContentBuilderString("attempt_fails"); - static final XContentBuilderString IN_ERRS = new XContentBuilderString("in_errs"); - static final XContentBuilderString OUT_RSTS = new XContentBuilderString("out_rsts"); - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(Fields.NETWORK); - if (tcp != null) { - builder.startObject(Fields.TCP); - builder.field(Fields.ACTIVE_OPENS, tcp.getActiveOpens()); - builder.field(Fields.PASSIVE_OPENS, tcp.getPassiveOpens()); - builder.field(Fields.CURR_ESTAB, tcp.getCurrEstab()); - builder.field(Fields.IN_SEGS, tcp.getInSegs()); - builder.field(Fields.OUT_SEGS, tcp.getOutSegs()); - builder.field(Fields.RETRANS_SEGS, tcp.getRetransSegs()); - builder.field(Fields.ESTAB_RESETS, tcp.getEstabResets()); - builder.field(Fields.ATTEMPT_FAILS, tcp.getAttemptFails()); - builder.field(Fields.IN_ERRS, tcp.getInErrs()); - builder.field(Fields.OUT_RSTS, tcp.getOutRsts()); - builder.endObject(); - } - builder.endObject(); - return builder; - } - - public static NetworkStats readNetworkStats(StreamInput in) throws IOException { - NetworkStats stats = new NetworkStats(); - stats.readFrom(in); - return stats; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - timestamp = in.readVLong(); - if (in.readBoolean()) { - tcp = Tcp.readNetworkTcp(in); - } - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeVLong(timestamp); - if (tcp == null) { - out.writeBoolean(false); - } else { - out.writeBoolean(true); - tcp.writeTo(out); - } - } - - public long timestamp() { - return timestamp; - } - - public long getTimestamp() { - return timestamp(); - } - - public Tcp tcp() { - return tcp; - } - - public Tcp getTcp() { - return tcp(); - } - - public static class Tcp implements Streamable { - - long activeOpens; - long passiveOpens; - long attemptFails; - long estabResets; - long currEstab; - long inSegs; - long outSegs; - long retransSegs; - long inErrs; - long outRsts; - - public static Tcp readNetworkTcp(StreamInput in) throws IOException { - Tcp tcp = new Tcp(); - tcp.readFrom(in); - return tcp; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - activeOpens = in.readLong(); - passiveOpens = in.readLong(); - attemptFails = in.readLong(); - estabResets = in.readLong(); - currEstab = in.readLong(); - inSegs = in.readLong(); - outSegs = in.readLong(); - retransSegs = in.readLong(); - inErrs = in.readLong(); - outRsts = in.readLong(); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeLong(activeOpens); - out.writeLong(passiveOpens); - out.writeLong(attemptFails); - out.writeLong(estabResets); - out.writeLong(currEstab); - out.writeLong(inSegs); - out.writeLong(outSegs); - out.writeLong(retransSegs); - out.writeLong(inErrs); - out.writeLong(outRsts); - } - - public long activeOpens() { - return this.activeOpens; - } - - public long getActiveOpens() { - return activeOpens(); - } - - public long passiveOpens() { - return passiveOpens; - } - - public long getPassiveOpens() { - return passiveOpens(); - } - - public long attemptFails() { - return attemptFails; - } - - public long getAttemptFails() { - return attemptFails(); - } - - public long estabResets() { - return estabResets; - } - - public long getEstabResets() { - return estabResets(); - } - - public long currEstab() { - return currEstab; - } - - public long getCurrEstab() { - return currEstab(); - } - - public long inSegs() { - return inSegs; - } - - public long getInSegs() { - return inSegs(); - } - - public long outSegs() { - return outSegs; - } - - public long getOutSegs() { - return outSegs(); - } - - public long retransSegs() { - return retransSegs; - } - - public long getRetransSegs() { - return retransSegs(); - } - - public long inErrs() { - return inErrs; - } - - public long getInErrs() { - return inErrs(); - } - - public long outRsts() { - return outRsts; - } - - public long getOutRsts() { - return outRsts(); - } - } -} diff --git a/core/src/main/java/org/elasticsearch/node/service/NodeService.java b/core/src/main/java/org/elasticsearch/node/service/NodeService.java index 81dd3c109e771..e50fd3e7ee241 100644 --- a/core/src/main/java/org/elasticsearch/node/service/NodeService.java +++ b/core/src/main/java/org/elasticsearch/node/service/NodeService.java @@ -102,7 +102,6 @@ public NodeInfo info() { monitorService.processService().info(), monitorService.jvmService().info(), threadPool.info(), - monitorService.networkService().info(), transportService.info(), httpServer == null ? null : httpServer.info(), pluginService == null ? null : pluginService.info() @@ -117,7 +116,6 @@ public NodeInfo info(boolean settings, boolean os, boolean process, boolean jvm, process ? monitorService.processService().info() : null, jvm ? monitorService.jvmService().info() : null, threadPool ? this.threadPool.info() : null, - network ? monitorService.networkService().info() : null, transport ? transportService.info() : null, http ? (httpServer == null ? null : httpServer.info()) : null, plugin ? (pluginService == null ? null : pluginService.info()) : null @@ -133,7 +131,6 @@ public NodeStats stats() throws IOException { monitorService.processService().stats(), monitorService.jvmService().stats(), threadPool.stats(), - monitorService.networkService().stats(), monitorService.fsService().stats(), transportService.stats(), httpServer == null ? null : httpServer.stats(), @@ -151,7 +148,6 @@ public NodeStats stats(CommonStatsFlags indices, boolean os, boolean process, bo process ? monitorService.processService().stats() : null, jvm ? monitorService.jvmService().stats() : null, threadPool ? this.threadPool.stats() : null, - network ? monitorService.networkService().stats() : null, fs ? monitorService.fsService().stats() : null, transport ? transportService.stats() : null, http ? (httpServer == null ? null : httpServer.stats()) : null, diff --git a/core/src/main/java/org/elasticsearch/plugins/PluginManager.java b/core/src/main/java/org/elasticsearch/plugins/PluginManager.java index 9b28a16b9786e..19649cfbb468c 100644 --- a/core/src/main/java/org/elasticsearch/plugins/PluginManager.java +++ b/core/src/main/java/org/elasticsearch/plugins/PluginManager.java @@ -31,6 +31,7 @@ import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.http.client.HttpDownloadHelper; import org.elasticsearch.common.io.FileSystemUtils; +import org.elasticsearch.common.logging.log4j.LogConfigurator; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.env.Environment; @@ -72,7 +73,7 @@ public enum OutputMode { // By default timeout is 0 which means no timeout public static final TimeValue DEFAULT_TIMEOUT = TimeValue.timeValueMillis(0); - private static final ImmutableSet BLACKLIST = ImmutableSet.builder() + private static final ImmutableSet BLACKLIST = ImmutableSet.builder() .add("elasticsearch", "elasticsearch.bat", "elasticsearch.in.sh", @@ -80,6 +81,21 @@ public enum OutputMode { "plugin.bat", "service.bat").build(); + private static final ImmutableSet OFFICIAL_PLUGINS = ImmutableSet.builder() + .add( + "elasticsearch-analysis-icu", + "elasticsearch-analysis-kuromoji", + "elasticsearch-analysis-phonetic", + "elasticsearch-analysis-smartcn", + "elasticsearch-analysis-stempel", + "elasticsearch-cloud-aws", + "elasticsearch-cloud-azure", + "elasticsearch-cloud-gce", + "elasticsearch-delete-by-query", + "elasticsearch-lang-javascript", + "elasticsearch-lang-python" + ).build(); + private final Environment environment; private String url; private OutputMode outputMode; @@ -132,6 +148,10 @@ public void downloadAndExtract(String name) throws IOException { // ignore log("Failed: " + ExceptionsHelper.detailedMessage(e)); } + } else { + if (PluginHandle.isOfficialPlugin(pluginHandle.repo, pluginHandle.user, pluginHandle.version)) { + checkForOfficialPlugins(pluginHandle.name); + } } if (!downloaded) { @@ -383,6 +403,15 @@ private static void checkForForbiddenName(String name) { } } + protected static void checkForOfficialPlugins(String name) { + // We make sure that users can use only new short naming for official plugins only + if (!OFFICIAL_PLUGINS.contains(name)) { + throw new IllegalArgumentException(name + + " is not an official plugin so you should install it using elasticsearch/" + + name + "/latest naming form."); + } + } + public Path[] getListInstalledPlugins() throws IOException { try (DirectoryStream stream = Files.newDirectoryStream(environment.pluginsFile())) { return Iterators.toArray(stream.iterator(), Path.class); @@ -408,6 +437,7 @@ public void listInstalledPlugins() throws IOException { public static void main(String[] args) { Tuple initialSettings = InternalSettingsPreparer.prepareSettings(EMPTY_SETTINGS, true, Terminal.DEFAULT); + LogConfigurator.configure(initialSettings.v1()); try { Files.createDirectories(initialSettings.v2().pluginsFile()); @@ -595,9 +625,15 @@ private static void displayHelp(String message) { SysOut.println(" -h, --help : Prints this help message"); SysOut.newline(); SysOut.println(" [*] Plugin name could be:"); - SysOut.println(" elasticsearch/plugin/version for official elasticsearch plugins (download from download.elasticsearch.org)"); + SysOut.println(" elasticsearch-plugin-name for Elasticsearch 2.0 Core plugin (download from download.elastic.co)"); + SysOut.println(" elasticsearch/plugin/version for elasticsearch commercial plugins (download from download.elastic.co)"); SysOut.println(" groupId/artifactId/version for community plugins (download from maven central or oss sonatype)"); SysOut.println(" username/repository for site plugins (download from github master)"); + SysOut.newline(); + SysOut.println("Elasticsearch Core plugins:"); + for (String o : OFFICIAL_PLUGINS) { + SysOut.println(" - " + o); + } if (message != null) { SysOut.newline(); @@ -650,17 +686,26 @@ static class PluginHandle { List urls() { List urls = new ArrayList<>(); if (version != null) { - // Elasticsearch download service - addUrl(urls, "http://download.elasticsearch.org/" + user + "/" + repo + "/" + repo + "-" + version + ".zip"); - // Maven central repository - addUrl(urls, "http://search.maven.org/remotecontent?filepath=" + user.replace('.', '/') + "/" + repo + "/" + version + "/" + repo + "-" + version + ".zip"); - // Sonatype repository - addUrl(urls, "https://oss.sonatype.org/service/local/repositories/releases/content/" + user.replace('.', '/') + "/" + repo + "/" + version + "/" + repo + "-" + version + ".zip"); - // Github repository - addUrl(urls, "https://github.com/" + user + "/" + repo + "/archive/" + version + ".zip"); + // Elasticsearch new download service uses groupId org.elasticsearch.plugins from 2.0.0 + if (user == null) { + // TODO Update to https + addUrl(urls, String.format(Locale.ROOT, "http://download.elastic.co/org.elasticsearch.plugins/%1$s/%1$s-%2$s.zip", repo, version)); + } else { + // Elasticsearch old download service + // TODO Update to https + addUrl(urls, String.format(Locale.ROOT, "http://download.elastic.co/%1$s/%2$s/%2$s-%3$s.zip", user, repo, version)); + // Maven central repository + addUrl(urls, String.format(Locale.ROOT, "http://search.maven.org/remotecontent?filepath=%1$s/%2$s/%3$s/%2$s-%3$s.zip", user.replace('.', '/'), repo, version)); + // Sonatype repository + addUrl(urls, String.format(Locale.ROOT, "https://oss.sonatype.org/service/local/repositories/releases/content/%1$s/%2$s/%3$s/%2$s-%3$s.zip", user.replace('.', '/'), repo, version)); + // Github repository + addUrl(urls, String.format(Locale.ROOT, "https://github.com/%1$s/%2$s/archive/%3$s.zip", user, repo, version)); + } + } + if (user != null) { + // Github repository for master branch (assume site) + addUrl(urls, String.format(Locale.ROOT, "https://github.com/%1$s/%2$s/archive/master.zip", user, repo)); } - // Github repository for master branch (assume site) - addUrl(urls, "https://github.com/" + user + "/" + repo + "/archive/master.zip"); return urls; } @@ -706,6 +751,10 @@ static PluginHandle parse(String name) { } } + if (isOfficialPlugin(repo, user, version)) { + return new PluginHandle(repo, Version.CURRENT.number(), null, repo); + } + if (repo.startsWith("elasticsearch-")) { // remove elasticsearch- prefix String endname = repo.substring("elasticsearch-".length()); @@ -720,6 +769,10 @@ static PluginHandle parse(String name) { return new PluginHandle(repo, version, user, repo); } + + static boolean isOfficialPlugin(String repo, String user, String version) { + return version == null && user == null && !Strings.isNullOrEmpty(repo); + } } } diff --git a/core/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java b/core/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java index 6ce84bf728320..87fc5378fd14c 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java @@ -48,7 +48,7 @@ import org.elasticsearch.index.search.stats.SearchStats; import org.elasticsearch.index.suggest.stats.SuggestStats; import org.elasticsearch.indices.NodeIndicesStats; -import org.elasticsearch.monitor.fs.FsStats; +import org.elasticsearch.monitor.fs.FsInfo; import org.elasticsearch.monitor.jvm.JvmInfo; import org.elasticsearch.monitor.jvm.JvmStats; import org.elasticsearch.monitor.os.OsInfo; @@ -224,7 +224,7 @@ private Table buildTable(RestRequest req, ClusterStateResponse state, NodesInfoR ProcessInfo processInfo = info == null ? null : info.getProcess(); JvmStats jvmStats = stats == null ? null : stats.getJvm(); - FsStats fsStats = stats == null ? null : stats.getFs(); + FsInfo fsInfo = stats == null ? null : stats.getFs(); OsStats osStats = stats == null ? null : stats.getOs(); ProcessStats processStats = stats == null ? null : stats.getProcess(); NodeIndicesStats indicesStats = stats == null ? null : stats.getIndices(); @@ -244,7 +244,7 @@ private Table buildTable(RestRequest req, ClusterStateResponse state, NodesInfoR table.addCell(node.getVersion().number()); table.addCell(info == null ? null : info.getBuild().hashShort()); table.addCell(jvmInfo == null ? null : jvmInfo.version()); - table.addCell(fsStats == null ? null : fsStats.getTotal().getAvailable()); + table.addCell(fsInfo == null ? null : fsInfo.getTotal().getAvailable()); table.addCell(jvmStats == null ? null : jvmStats.getMem().getHeapUsed()); table.addCell(jvmStats == null ? null : jvmStats.getMem().getHeapUsedPercent()); table.addCell(jvmInfo == null ? null : jvmInfo.getMem().getHeapMax()); diff --git a/core/src/main/java/org/elasticsearch/script/ScriptService.java b/core/src/main/java/org/elasticsearch/script/ScriptService.java index df119b284b190..b41be3250c4c3 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptService.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptService.java @@ -95,9 +95,9 @@ public class ScriptService extends AbstractComponent implements Closeable { private final ImmutableMap scriptEnginesByLang; private final ImmutableMap scriptEnginesByExt; - private final ConcurrentMap staticCache = ConcurrentCollections.newConcurrentMap(); + private final ConcurrentMap staticCache = ConcurrentCollections.newConcurrentMap(); - private final Cache cache; + private final Cache cache; private final Path scriptsDirectory; private final ScriptModes scriptModes; @@ -266,7 +266,7 @@ public CompiledScript compileInternal(Script script) { } ScriptEngineService scriptEngineService = getScriptEngineServiceForLang(lang); - CacheKey cacheKey = newCacheKey(scriptEngineService, script.getScript()); + String cacheKey = getCacheKey(scriptEngineService, script.getScript()); if (script.getType() == ScriptType.FILE) { CompiledScript compiled = staticCache.get(cacheKey); //On disk scripts will be loaded into the staticCache by the listener @@ -281,7 +281,7 @@ public CompiledScript compileInternal(Script script) { if (script.getType() == ScriptType.INDEXED) { final IndexedScript indexedScript = new IndexedScript(lang, script.getScript()); code = getScriptFromIndex(indexedScript.lang, indexedScript.id); - cacheKey = newCacheKey(scriptEngineService, code); + cacheKey = getCacheKey(scriptEngineService, code); } CompiledScript compiled = cache.getIfPresent(cacheKey); @@ -462,10 +462,10 @@ private boolean canExecuteScript(String lang, ScriptEngineService scriptEngineSe * {@code ScriptEngineService}'s {@code scriptRemoved} method when the * script has been removed from the cache */ - private class ScriptCacheRemovalListener implements RemovalListener { + private class ScriptCacheRemovalListener implements RemovalListener { @Override - public void onRemoval(RemovalNotification notification) { + public void onRemoval(RemovalNotification notification) { if (logger.isDebugEnabled()) { logger.debug("notifying script services of script removal due to: [{}]", notification.getCause()); } @@ -513,7 +513,7 @@ public void onFileInit(Path file) { logger.info("compiling script file [{}]", file.toAbsolutePath()); try(InputStreamReader reader = new InputStreamReader(Files.newInputStream(file), Charsets.UTF_8)) { String script = Streams.copyToString(reader); - CacheKey cacheKey = newCacheKey(engineService, scriptNameExt.v1()); + String cacheKey = getCacheKey(engineService, scriptNameExt.v1()); staticCache.put(cacheKey, new CompiledScript(engineService.types()[0], engineService.compile(script))); } } else { @@ -538,7 +538,7 @@ public void onFileDeleted(Path file) { ScriptEngineService engineService = getScriptEngineServiceForFileExt(scriptNameExt.v2()); assert engineService != null; logger.info("removing script file [{}]", file.toAbsolutePath()); - staticCache.remove(newCacheKey(engineService, scriptNameExt.v1())); + staticCache.remove(getCacheKey(engineService, scriptNameExt.v1())); } } @@ -598,32 +598,9 @@ public String toString() { } } - private static CacheKey newCacheKey(ScriptEngineService engineService, String script) { - return new CacheKey(engineService.types()[0], script); - } - - private static class CacheKey { - public final String lang; - public final String script; - - public CacheKey(String lang, String script) { - this.lang = lang; - this.script = script; - } - - @Override - public boolean equals(Object o) { - if (! (o instanceof CacheKey)) { - return false; - } - CacheKey other = (CacheKey) o; - return lang.equals(other.lang) && script.equals(other.script); - } - - @Override - public int hashCode() { - return lang.hashCode() + 31 * script.hashCode(); - } + private static String getCacheKey(ScriptEngineService scriptEngineService, String script) { + String lang = scriptEngineService.types()[0]; + return lang + ":" + script; } private static class IndexedScript { diff --git a/core/src/main/java/org/elasticsearch/search/SearchService.java b/core/src/main/java/org/elasticsearch/search/SearchService.java index 41b8e15475121..7b7f764ad932b 100644 --- a/core/src/main/java/org/elasticsearch/search/SearchService.java +++ b/core/src/main/java/org/elasticsearch/search/SearchService.java @@ -267,22 +267,39 @@ public QuerySearchResult executeScan(ShardSearchRequest request) { public ScrollQueryFetchSearchResult executeScan(InternalScrollSearchRequest request) { final SearchContext context = findContext(request.id()); + ShardSearchStats shardSearchStats = context.indexShard().searchService(); contextProcessing(context); try { processScroll(request, context); - if (context.searchType() == SearchType.QUERY_THEN_FETCH) { - // first scanning, reset the from to 0 - context.searchType(SearchType.SCAN); - context.from(0); + shardSearchStats.onPreQueryPhase(context); + long time = System.nanoTime(); + try { + if (context.searchType() == SearchType.QUERY_THEN_FETCH) { + // first scanning, reset the from to 0 + context.searchType(SearchType.SCAN); + context.from(0); + } + queryPhase.execute(context); + } catch (Throwable e) { + shardSearchStats.onFailedQueryPhase(context); + throw ExceptionsHelper.convertToRuntime(e); } - queryPhase.execute(context); - shortcutDocIdsToLoadForScanning(context); - fetchPhase.execute(context); - if (context.scroll() == null || context.fetchResult().hits().hits().length < context.size()) { - freeContext(request.id()); - } else { - contextProcessedSuccessfully(context); + long queryFinishTime = System.nanoTime(); + shardSearchStats.onQueryPhase(context, queryFinishTime - time); + shardSearchStats.onPreFetchPhase(context); + try { + shortcutDocIdsToLoadForScanning(context); + fetchPhase.execute(context); + if (context.scroll() == null || context.fetchResult().hits().hits().length < context.size()) { + freeContext(request.id()); + } else { + contextProcessedSuccessfully(context); + } + } catch (Throwable e) { + shardSearchStats.onFailedFetchPhase(context); + throw ExceptionsHelper.convertToRuntime(e); } + shardSearchStats.onFetchPhase(context, System.nanoTime() - queryFinishTime); return new ScrollQueryFetchSearchResult(new QueryFetchSearchResult(context.queryResult(), context.fetchResult()), context.shardTarget()); } catch (Throwable e) { logger.trace("Scan phase failed", e); @@ -569,6 +586,9 @@ final SearchContext createAndPutContext(ShardSearchRequest request) { boolean success = false; try { putContext(context); + if (request.scroll() != null) { + context.indexShard().searchService().onNewScrollContext(context); + } context.indexShard().searchService().onNewContext(context); success = true; return context; @@ -643,6 +663,9 @@ public boolean freeContext(long id) { if (context != null) { try { context.indexShard().searchService().onFreeContext(context); + if (context.scroll() != null) { + context.indexShard().searchService().onFreeScrollContext(context); + } } finally { context.close(); } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/AggregationModule.java b/core/src/main/java/org/elasticsearch/search/aggregations/AggregationModule.java index 6efb9f72f398c..f90cdbc343809 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/AggregationModule.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/AggregationModule.java @@ -64,6 +64,7 @@ import org.elasticsearch.search.aggregations.pipeline.cumulativesum.CumulativeSumParser; import org.elasticsearch.search.aggregations.pipeline.bucketscript.BucketScriptParser; import org.elasticsearch.search.aggregations.pipeline.derivative.DerivativeParser; +import org.elasticsearch.search.aggregations.pipeline.having.BucketSelectorParser; import org.elasticsearch.search.aggregations.pipeline.movavg.MovAvgParser; import org.elasticsearch.search.aggregations.pipeline.movavg.models.MovAvgModelModule; @@ -118,6 +119,7 @@ public AggregationModule() { pipelineAggParsers.add(MovAvgParser.class); pipelineAggParsers.add(CumulativeSumParser.class); pipelineAggParsers.add(BucketScriptParser.class); + pipelineAggParsers.add(BucketSelectorParser.class); } /** diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/TransportAggregationModule.java b/core/src/main/java/org/elasticsearch/search/aggregations/TransportAggregationModule.java index daab25eeaeea5..ef152ab11a28f 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/TransportAggregationModule.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/TransportAggregationModule.java @@ -69,6 +69,7 @@ import org.elasticsearch.search.aggregations.pipeline.bucketscript.BucketScriptPipelineAggregator; import org.elasticsearch.search.aggregations.pipeline.derivative.DerivativePipelineAggregator; import org.elasticsearch.search.aggregations.pipeline.derivative.InternalDerivative; +import org.elasticsearch.search.aggregations.pipeline.having.BucketSelectorPipelineAggregator; import org.elasticsearch.search.aggregations.pipeline.movavg.MovAvgPipelineAggregator; import org.elasticsearch.search.aggregations.pipeline.movavg.models.TransportMovAvgModelModule; @@ -131,6 +132,7 @@ protected void configure() { MovAvgPipelineAggregator.registerStreams(); CumulativeSumPipelineAggregator.registerStreams(); BucketScriptPipelineAggregator.registerStreams(); + BucketSelectorPipelineAggregator.registerStreams(); } @Override diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/PipelineAggregatorBuilders.java b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/PipelineAggregatorBuilders.java index 851b750e77816..a800dff3ee66f 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/PipelineAggregatorBuilders.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/PipelineAggregatorBuilders.java @@ -26,6 +26,7 @@ import org.elasticsearch.search.aggregations.pipeline.cumulativesum.CumulativeSumBuilder; import org.elasticsearch.search.aggregations.pipeline.bucketscript.BucketScriptBuilder; import org.elasticsearch.search.aggregations.pipeline.derivative.DerivativeBuilder; +import org.elasticsearch.search.aggregations.pipeline.having.BucketSelectorBuilder; import org.elasticsearch.search.aggregations.pipeline.movavg.MovAvgBuilder; public final class PipelineAggregatorBuilders { @@ -61,6 +62,10 @@ public static final BucketScriptBuilder bucketScript(String name) { return new BucketScriptBuilder(name); } + public static final BucketSelectorBuilder having(String name) { + return new BucketSelectorBuilder(name); + } + public static final CumulativeSumBuilder cumulativeSum(String name) { return new CumulativeSumBuilder(name); } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/having/BucketSelectorBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/having/BucketSelectorBuilder.java new file mode 100644 index 0000000000000..c291c63201651 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/having/BucketSelectorBuilder.java @@ -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.search.aggregations.pipeline.having; + +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.script.Script; +import org.elasticsearch.script.Script.ScriptField; +import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilder; + +import java.io.IOException; +import java.util.Map; + +public class BucketSelectorBuilder extends PipelineAggregatorBuilder { + + private GapPolicy gapPolicy; + private Script script; + private Map bucketsPathsMap; + + public BucketSelectorBuilder(String name) { + super(name, BucketSelectorPipelineAggregator.TYPE.name()); + } + + public BucketSelectorBuilder script(Script script) { + this.script = script; + return this; + } + + public BucketSelectorBuilder gapPolicy(GapPolicy gapPolicy) { + this.gapPolicy = gapPolicy; + return this; + } + + /** + * Sets the paths to the buckets to use for this pipeline aggregator. The + * map given to this method must contain script variable name as keys with + * bucket paths values to the metrics to use for each variable. + */ + public BucketSelectorBuilder setBucketsPathsMap(Map bucketsPathsMap) { + this.bucketsPathsMap = bucketsPathsMap; + return this; + } + + @Override + protected XContentBuilder internalXContent(XContentBuilder builder, Params builderParams) throws IOException { + if (script != null) { + builder.field(ScriptField.SCRIPT.getPreferredName(), script); + } + if (gapPolicy != null) { + builder.field(BucketSelectorParser.GAP_POLICY.getPreferredName(), gapPolicy.getName()); + } + if (bucketsPathsMap != null) { + builder.field(PipelineAggregator.Parser.BUCKETS_PATH.getPreferredName(), bucketsPathsMap); + } + return builder; + } + +} diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/having/BucketSelectorParser.java b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/having/BucketSelectorParser.java new file mode 100644 index 0000000000000..e2623b5236474 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/having/BucketSelectorParser.java @@ -0,0 +1,119 @@ +/* + * 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.aggregations.pipeline.having; + +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.script.Script; +import org.elasticsearch.script.Script.ScriptField; +import org.elasticsearch.search.SearchParseException; +import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorFactory; +import org.elasticsearch.search.internal.SearchContext; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class BucketSelectorParser implements PipelineAggregator.Parser { + + public static final ParseField FORMAT = new ParseField("format"); + public static final ParseField GAP_POLICY = new ParseField("gap_policy"); + public static final ParseField PARAMS_FIELD = new ParseField("params"); + + @Override + public String type() { + return BucketSelectorPipelineAggregator.TYPE.name(); + } + + @Override + public PipelineAggregatorFactory parse(String reducerName, XContentParser parser, SearchContext context) throws IOException { + XContentParser.Token token; + Script script = null; + String currentFieldName = null; + Map bucketsPathsMap = null; + GapPolicy gapPolicy = GapPolicy.SKIP; + + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + currentFieldName = parser.currentName(); + } else if (token == XContentParser.Token.VALUE_STRING) { + if (context.parseFieldMatcher().match(currentFieldName, BUCKETS_PATH)) { + bucketsPathsMap = new HashMap<>(); + bucketsPathsMap.put("_value", parser.text()); + } else if (context.parseFieldMatcher().match(currentFieldName, GAP_POLICY)) { + gapPolicy = GapPolicy.parse(context, parser.text(), parser.getTokenLocation()); + } else if (context.parseFieldMatcher().match(currentFieldName, ScriptField.SCRIPT)) { + script = Script.parse(parser, context.parseFieldMatcher()); + } else { + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + reducerName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); + } + } else if (token == XContentParser.Token.START_ARRAY) { + if (context.parseFieldMatcher().match(currentFieldName, BUCKETS_PATH)) { + List paths = new ArrayList<>(); + while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { + String path = parser.text(); + paths.add(path); + } + bucketsPathsMap = new HashMap<>(); + for (int i = 0; i < paths.size(); i++) { + bucketsPathsMap.put("_value" + i, paths.get(i)); + } + } else { + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + reducerName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); + } + } else if (token == XContentParser.Token.START_OBJECT) { + if (context.parseFieldMatcher().match(currentFieldName, ScriptField.SCRIPT)) { + script = Script.parse(parser, context.parseFieldMatcher()); + } else if (context.parseFieldMatcher().match(currentFieldName, BUCKETS_PATH)) { + Map map = parser.map(); + bucketsPathsMap = new HashMap<>(); + for (Map.Entry entry : map.entrySet()) { + bucketsPathsMap.put(entry.getKey(), String.valueOf(entry.getValue())); + } + } else { + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + reducerName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); + } + } else { + throw new SearchParseException(context, "Unexpected token " + token + " in [" + reducerName + "].", + parser.getTokenLocation()); + } + } + + if (bucketsPathsMap == null) { + throw new SearchParseException(context, "Missing required field [" + BUCKETS_PATH.getPreferredName() + + "] for bucket_selector aggregation [" + reducerName + "]", parser.getTokenLocation()); + } + + if (script == null) { + throw new SearchParseException(context, "Missing required field [" + ScriptField.SCRIPT.getPreferredName() + + "] for bucket_selector aggregation [" + reducerName + "]", parser.getTokenLocation()); + } + + return new BucketSelectorPipelineAggregator.Factory(reducerName, bucketsPathsMap, script, gapPolicy); + } + +} diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/having/BucketSelectorPipelineAggregator.java b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/having/BucketSelectorPipelineAggregator.java new file mode 100644 index 0000000000000..154a729d04656 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/having/BucketSelectorPipelineAggregator.java @@ -0,0 +1,164 @@ +/* + * 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.aggregations.pipeline.having; + +import com.google.common.base.Function; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.script.CompiledScript; +import org.elasticsearch.script.ExecutableScript; +import org.elasticsearch.script.Script; +import org.elasticsearch.script.ScriptContext; +import org.elasticsearch.script.expression.ExpressionScriptEngineService; +import org.elasticsearch.search.aggregations.Aggregation; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.InternalAggregation.ReduceContext; +import org.elasticsearch.search.aggregations.InternalAggregation.Type; +import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation; +import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation.Bucket; +import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorFactory; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorStreams; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.elasticsearch.search.aggregations.pipeline.BucketHelpers.resolveBucketValue; + +public class BucketSelectorPipelineAggregator extends PipelineAggregator { + + public final static Type TYPE = new Type("bucket_selector"); + + public final static PipelineAggregatorStreams.Stream STREAM = new PipelineAggregatorStreams.Stream() { + @Override + public BucketSelectorPipelineAggregator readResult(StreamInput in) throws IOException { + BucketSelectorPipelineAggregator result = new BucketSelectorPipelineAggregator(); + result.readFrom(in); + return result; + } + }; + + public static void registerStreams() { + PipelineAggregatorStreams.registerStream(STREAM, TYPE.stream()); + } + + private static final Function FUNCTION = new Function() { + @Override + public InternalAggregation apply(Aggregation input) { + return (InternalAggregation) input; + } + }; + + private GapPolicy gapPolicy; + + private Script script; + + private Map bucketsPathsMap; + + public BucketSelectorPipelineAggregator() { + } + + public BucketSelectorPipelineAggregator(String name, Map bucketsPathsMap, Script script, GapPolicy gapPolicy, + Map metadata) { + super(name, bucketsPathsMap.values().toArray(new String[bucketsPathsMap.size()]), metadata); + this.bucketsPathsMap = bucketsPathsMap; + this.script = script; + this.gapPolicy = gapPolicy; + } + + @Override + public Type type() { + return TYPE; + } + + @Override + public InternalAggregation reduce(InternalAggregation aggregation, ReduceContext reduceContext) { + InternalMultiBucketAggregation originalAgg = (InternalMultiBucketAggregation) aggregation; + List buckets = originalAgg.getBuckets(); + + CompiledScript compiledScript = reduceContext.scriptService().compile(script, ScriptContext.Standard.AGGS); + List newBuckets = new ArrayList<>(); + for (Bucket bucket : buckets) { + Map vars = new HashMap<>(); + if (script.getParams() != null) { + vars.putAll(script.getParams()); + } + for (Map.Entry entry : bucketsPathsMap.entrySet()) { + String varName = entry.getKey(); + String bucketsPath = entry.getValue(); + Double value = resolveBucketValue(originalAgg, bucket, bucketsPath, gapPolicy); + vars.put(varName, value); + } + ExecutableScript executableScript = reduceContext.scriptService().executable(compiledScript, vars); + Object scriptReturnValue = executableScript.run(); + final boolean keepBucket; + if (ExpressionScriptEngineService.NAME.equals(script.getLang())) { + double scriptDoubleValue = (double) scriptReturnValue; + keepBucket = scriptDoubleValue == 1.0; + } else { + keepBucket = (boolean) scriptReturnValue; + } + if (keepBucket) { + newBuckets.add(bucket); + } + } + return originalAgg.create(newBuckets); + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException { + script.writeTo(out); + gapPolicy.writeTo(out); + out.writeGenericValue(bucketsPathsMap); + } + + @SuppressWarnings("unchecked") + @Override + protected void doReadFrom(StreamInput in) throws IOException { + script = Script.readScript(in); + gapPolicy = GapPolicy.readFrom(in); + bucketsPathsMap = (Map) in.readGenericValue(); + } + + public static class Factory extends PipelineAggregatorFactory { + + private Script script; + private GapPolicy gapPolicy; + private Map bucketsPathsMap; + + public Factory(String name, Map bucketsPathsMap, Script script, GapPolicy gapPolicy) { + super(name, TYPE.name(), bucketsPathsMap.values().toArray(new String[bucketsPathsMap.size()])); + this.bucketsPathsMap = bucketsPathsMap; + this.script = script; + this.gapPolicy = gapPolicy; + } + + @Override + protected PipelineAggregator createInternal(Map metaData) throws IOException { + return new BucketSelectorPipelineAggregator(name, bucketsPathsMap, script, gapPolicy, metaData); + } + } + +} diff --git a/core/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java b/core/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java index f07acb0acb5f8..e862862cd3753 100644 --- a/core/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java +++ b/core/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java @@ -38,7 +38,7 @@ */ public class SnapshotInfo implements ToXContent, Streamable { - private static final FormatDateTimeFormatter DATE_TIME_FORMATTER = Joda.forPattern("dateOptionalTime"); + private static final FormatDateTimeFormatter DATE_TIME_FORMATTER = Joda.forPattern("strictDateOptionalTime"); private String name; diff --git a/core/src/main/java/org/joda/time/format/StrictISODateTimeFormat.java b/core/src/main/java/org/joda/time/format/StrictISODateTimeFormat.java new file mode 100644 index 0000000000000..9768b9a8559b1 --- /dev/null +++ b/core/src/main/java/org/joda/time/format/StrictISODateTimeFormat.java @@ -0,0 +1,2028 @@ +package org.joda.time.format; + +/* + * Copyright 2001-2009 Stephen Colebourne + * + * Licensed 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. + */ + +import org.joda.time.DateTimeFieldType; + +import java.util.Collection; +import java.util.HashSet; +import java.util.Set; + +/* + * Elasticsearch Note: This class has been copied almost identically from joda, where the + * class is named ISODatetimeFormat + * + * However there has been done one huge modification in several methods, which forces the date + * year to be at least n digits, so that a year like "5" is invalid and must be "0005" + * + * All methods have been marked with an "// ES change" commentary + * + * In case you compare this with the original ISODateTimeFormat, make sure you use a diff + * call, that ignores whitespaces/tabs/indendetations like 'diff -b' + */ +/** + * Factory that creates instances of DateTimeFormatter based on the ISO8601 standard. + *

+ * Date-time formatting is performed by the {@link DateTimeFormatter} class. + * Three classes provide factory methods to create formatters, and this is one. + * The others are {@link DateTimeFormat} and {@link DateTimeFormatterBuilder}. + *

+ * ISO8601 is the international standard for data interchange. It defines a + * framework, rather than an absolute standard. As a result this provider has a + * number of methods that represent common uses of the framework. The most common + * formats are {@link #date() date}, {@link #time() time}, and {@link #dateTime() dateTime}. + *

+ * For example, to format a date time in ISO format: + *

+ * DateTime dt = new DateTime();
+ * DateTimeFormatter fmt = ISODateTimeFormat.dateTime();
+ * String str = fmt.print(dt);
+ * 
+ *

+ * Note that these formatters mostly follow the ISO8601 standard for printing. + * For parsing, the formatters are more lenient and allow formats that are not + * in strict compliance with the standard. + *

+ * It is important to understand that these formatters are not linked to + * the ISOChronology. These formatters may be used with any + * chronology, however there may be certain side effects with more unusual + * chronologies. For example, the ISO formatters rely on dayOfWeek being + * single digit, dayOfMonth being two digit and dayOfYear being three digit. + * A chronology with a ten day week would thus cause issues. However, in + * general, it is safe to use these formatters with other chronologies. + *

+ * ISODateTimeFormat is thread-safe and immutable, and the formatters it + * returns are as well. + * + * @author Brian S O'Neill + * @since 1.0 + * @see DateTimeFormat + * @see DateTimeFormatterBuilder + */ +public class StrictISODateTimeFormat { + + /** + * Constructor. + * + * @since 1.1 (previously private) + */ + protected StrictISODateTimeFormat() { + super(); + } + + //----------------------------------------------------------------------- + /** + * Returns a formatter that outputs only those fields specified. + *

+ * This method examines the fields provided and returns an ISO-style + * formatter that best fits. This can be useful for outputting + * less-common ISO styles, such as YearMonth (YYYY-MM) or MonthDay (--MM-DD). + *

+ * The list provided may have overlapping fields, such as dayOfWeek and + * dayOfMonth. In this case, the style is chosen based on the following + * list, thus in the example, the calendar style is chosen as dayOfMonth + * is higher in priority than dayOfWeek: + *

    + *
  • monthOfYear - calendar date style + *
  • dayOfYear - ordinal date style + *
  • weekOfWeekYear - week date style + *
  • dayOfMonth - calendar date style + *
  • dayOfWeek - week date style + *
  • year + *
  • weekyear + *
+ * The supported formats are: + *
+     * Extended      Basic       Fields
+     * 2005-03-25    20050325    year/monthOfYear/dayOfMonth
+     * 2005-03       2005-03     year/monthOfYear
+     * 2005--25      2005--25    year/dayOfMonth *
+     * 2005          2005        year
+     * --03-25       --0325      monthOfYear/dayOfMonth
+     * --03          --03        monthOfYear
+     * ---03         ---03       dayOfMonth
+     * 2005-084      2005084     year/dayOfYear
+     * -084          -084        dayOfYear
+     * 2005-W12-5    2005W125    weekyear/weekOfWeekyear/dayOfWeek
+     * 2005-W-5      2005W-5     weekyear/dayOfWeek *
+     * 2005-W12      2005W12     weekyear/weekOfWeekyear
+     * -W12-5        -W125       weekOfWeekyear/dayOfWeek
+     * -W12          -W12        weekOfWeekyear
+     * -W-5          -W-5        dayOfWeek
+     * 10:20:30.040  102030.040  hour/minute/second/milli
+     * 10:20:30      102030      hour/minute/second
+     * 10:20         1020        hour/minute
+     * 10            10          hour
+     * -20:30.040    -2030.040   minute/second/milli
+     * -20:30        -2030       minute/second
+     * -20           -20         minute
+     * --30.040      --30.040    second/milli
+     * --30          --30        second
+     * ---.040       ---.040     milli *
+     * 10-30.040     10-30.040   hour/second/milli *
+     * 10:20-.040    1020-.040   hour/minute/milli *
+     * 10-30         10-30       hour/second *
+     * 10--.040      10--.040    hour/milli *
+     * -20-.040      -20-.040    minute/milli *
+     *   plus datetime formats like {date}T{time}
+     * 
+ * * indiates that this is not an official ISO format and can be excluded + * by passing in strictISO as true. + *

+ * This method can side effect the input collection of fields. + * If the input collection is modifiable, then each field that was added to + * the formatter will be removed from the collection, including any duplicates. + * If the input collection is unmodifiable then no side effect occurs. + *

+ * This side effect processing is useful if you need to know whether all + * the fields were converted into the formatter or not. To achieve this, + * pass in a modifiable list, and check that it is empty on exit. + * + * @param fields the fields to get a formatter for, not null, + * updated by the method call unless unmodifiable, + * removing those fields built in the formatter + * @param extended true to use the extended format (with separators) + * @param strictISO true to stick exactly to ISO8601, false to include additional formats + * @return a suitable formatter + * @throws IllegalArgumentException if there is no format for the fields + * @since 1.1 + */ + public static DateTimeFormatter forFields( + Collection fields, + boolean extended, + boolean strictISO) { + + if (fields == null || fields.size() == 0) { + throw new IllegalArgumentException("The fields must not be null or empty"); + } + Set workingFields = new HashSet(fields); + int inputSize = workingFields.size(); + boolean reducedPrec = false; + DateTimeFormatterBuilder bld = new DateTimeFormatterBuilder(); + // date + if (workingFields.contains(DateTimeFieldType.monthOfYear())) { + reducedPrec = dateByMonth(bld, workingFields, extended, strictISO); + } else if (workingFields.contains(DateTimeFieldType.dayOfYear())) { + reducedPrec = dateByOrdinal(bld, workingFields, extended, strictISO); + } else if (workingFields.contains(DateTimeFieldType.weekOfWeekyear())) { + reducedPrec = dateByWeek(bld, workingFields, extended, strictISO); + } else if (workingFields.contains(DateTimeFieldType.dayOfMonth())) { + reducedPrec = dateByMonth(bld, workingFields, extended, strictISO); + } else if (workingFields.contains(DateTimeFieldType.dayOfWeek())) { + reducedPrec = dateByWeek(bld, workingFields, extended, strictISO); + } else if (workingFields.remove(DateTimeFieldType.year())) { + bld.append(Constants.ye); + reducedPrec = true; + } else if (workingFields.remove(DateTimeFieldType.weekyear())) { + bld.append(Constants.we); + reducedPrec = true; + } + boolean datePresent = (workingFields.size() < inputSize); + + // time + time(bld, workingFields, extended, strictISO, reducedPrec, datePresent); + + // result + if (bld.canBuildFormatter() == false) { + throw new IllegalArgumentException("No valid format for fields: " + fields); + } + + // side effect the input collection to indicate the processed fields + // handling unmodifiable collections with no side effect + try { + fields.retainAll(workingFields); + } catch (UnsupportedOperationException ex) { + // ignore, so we can handle unmodifiable collections + } + return bld.toFormatter(); + } + + //----------------------------------------------------------------------- + /** + * Creates a date using the calendar date format. + * Specification reference: 5.2.1. + * + * @param bld the builder + * @param fields the fields + * @param extended true to use extended format + * @param strictISO true to only allow ISO formats + * @return true if reduced precision + * @since 1.1 + */ + private static boolean dateByMonth( + DateTimeFormatterBuilder bld, + Collection fields, + boolean extended, + boolean strictISO) { + + boolean reducedPrec = false; + if (fields.remove(DateTimeFieldType.year())) { + bld.append(Constants.ye); + if (fields.remove(DateTimeFieldType.monthOfYear())) { + if (fields.remove(DateTimeFieldType.dayOfMonth())) { + // YYYY-MM-DD/YYYYMMDD + appendSeparator(bld, extended); + bld.appendMonthOfYear(2); + appendSeparator(bld, extended); + bld.appendDayOfMonth(2); + } else { + // YYYY-MM/YYYY-MM + bld.appendLiteral('-'); + bld.appendMonthOfYear(2); + reducedPrec = true; + } + } else { + if (fields.remove(DateTimeFieldType.dayOfMonth())) { + // YYYY--DD/YYYY--DD (non-iso) + checkNotStrictISO(fields, strictISO); + bld.appendLiteral('-'); + bld.appendLiteral('-'); + bld.appendDayOfMonth(2); + } else { + // YYYY/YYYY + reducedPrec = true; + } + } + + } else if (fields.remove(DateTimeFieldType.monthOfYear())) { + bld.appendLiteral('-'); + bld.appendLiteral('-'); + bld.appendMonthOfYear(2); + if (fields.remove(DateTimeFieldType.dayOfMonth())) { + // --MM-DD/--MMDD + appendSeparator(bld, extended); + bld.appendDayOfMonth(2); + } else { + // --MM/--MM + reducedPrec = true; + } + } else if (fields.remove(DateTimeFieldType.dayOfMonth())) { + // ---DD/---DD + bld.appendLiteral('-'); + bld.appendLiteral('-'); + bld.appendLiteral('-'); + bld.appendDayOfMonth(2); + } + return reducedPrec; + } + + //----------------------------------------------------------------------- + /** + * Creates a date using the ordinal date format. + * Specification reference: 5.2.2. + * + * @param bld the builder + * @param fields the fields + * @param extended true to use extended format + * @param strictISO true to only allow ISO formats + * @since 1.1 + */ + private static boolean dateByOrdinal( + DateTimeFormatterBuilder bld, + Collection fields, + boolean extended, + boolean strictISO) { + + boolean reducedPrec = false; + if (fields.remove(DateTimeFieldType.year())) { + bld.append(Constants.ye); + if (fields.remove(DateTimeFieldType.dayOfYear())) { + // YYYY-DDD/YYYYDDD + appendSeparator(bld, extended); + bld.appendDayOfYear(3); + } else { + // YYYY/YYYY + reducedPrec = true; + } + + } else if (fields.remove(DateTimeFieldType.dayOfYear())) { + // -DDD/-DDD + bld.appendLiteral('-'); + bld.appendDayOfYear(3); + } + return reducedPrec; + } + + //----------------------------------------------------------------------- + /** + * Creates a date using the calendar date format. + * Specification reference: 5.2.3. + * + * @param bld the builder + * @param fields the fields + * @param extended true to use extended format + * @param strictISO true to only allow ISO formats + * @since 1.1 + */ + private static boolean dateByWeek( + DateTimeFormatterBuilder bld, + Collection fields, + boolean extended, + boolean strictISO) { + + boolean reducedPrec = false; + if (fields.remove(DateTimeFieldType.weekyear())) { + bld.append(Constants.we); + if (fields.remove(DateTimeFieldType.weekOfWeekyear())) { + appendSeparator(bld, extended); + bld.appendLiteral('W'); + bld.appendWeekOfWeekyear(2); + if (fields.remove(DateTimeFieldType.dayOfWeek())) { + // YYYY-WWW-D/YYYYWWWD + appendSeparator(bld, extended); + bld.appendDayOfWeek(1); + } else { + // YYYY-WWW/YYYY-WWW + reducedPrec = true; + } + } else { + if (fields.remove(DateTimeFieldType.dayOfWeek())) { + // YYYY-W-D/YYYYW-D (non-iso) + checkNotStrictISO(fields, strictISO); + appendSeparator(bld, extended); + bld.appendLiteral('W'); + bld.appendLiteral('-'); + bld.appendDayOfWeek(1); + } else { + // YYYY/YYYY + reducedPrec = true; + } + } + + } else if (fields.remove(DateTimeFieldType.weekOfWeekyear())) { + bld.appendLiteral('-'); + bld.appendLiteral('W'); + bld.appendWeekOfWeekyear(2); + if (fields.remove(DateTimeFieldType.dayOfWeek())) { + // -WWW-D/-WWWD + appendSeparator(bld, extended); + bld.appendDayOfWeek(1); + } else { + // -WWW/-WWW + reducedPrec = true; + } + } else if (fields.remove(DateTimeFieldType.dayOfWeek())) { + // -W-D/-W-D + bld.appendLiteral('-'); + bld.appendLiteral('W'); + bld.appendLiteral('-'); + bld.appendDayOfWeek(1); + } + return reducedPrec; + } + + //----------------------------------------------------------------------- + /** + * Adds the time fields to the builder. + * Specification reference: 5.3.1. + * + * @param bld the builder + * @param fields the fields + * @param extended whether to use the extended format + * @param strictISO whether to be strict + * @param reducedPrec whether the date was reduced precision + * @param datePresent whether there was a date + * @since 1.1 + */ + private static void time( + DateTimeFormatterBuilder bld, + Collection fields, + boolean extended, + boolean strictISO, + boolean reducedPrec, + boolean datePresent) { + + boolean hour = fields.remove(DateTimeFieldType.hourOfDay()); + boolean minute = fields.remove(DateTimeFieldType.minuteOfHour()); + boolean second = fields.remove(DateTimeFieldType.secondOfMinute()); + boolean milli = fields.remove(DateTimeFieldType.millisOfSecond()); + if (!hour && !minute && !second && !milli) { + return; + } + if (hour || minute || second || milli) { + if (strictISO && reducedPrec) { + throw new IllegalArgumentException("No valid ISO8601 format for fields because Date was reduced precision: " + fields); + } + if (datePresent) { + bld.appendLiteral('T'); + } + } + if (hour && minute && second || (hour && !second && !milli)) { + // OK - HMSm/HMS/HM/H - valid in combination with date + } else { + if (strictISO && datePresent) { + throw new IllegalArgumentException("No valid ISO8601 format for fields because Time was truncated: " + fields); + } + if (!hour && (minute && second || (minute && !milli) || second)) { + // OK - MSm/MS/M/Sm/S - valid ISO formats + } else { + if (strictISO) { + throw new IllegalArgumentException("No valid ISO8601 format for fields: " + fields); + } + } + } + if (hour) { + bld.appendHourOfDay(2); + } else if (minute || second || milli) { + bld.appendLiteral('-'); + } + if (extended && hour && minute) { + bld.appendLiteral(':'); + } + if (minute) { + bld.appendMinuteOfHour(2); + } else if (second || milli) { + bld.appendLiteral('-'); + } + if (extended && minute && second) { + bld.appendLiteral(':'); + } + if (second) { + bld.appendSecondOfMinute(2); + } else if (milli) { + bld.appendLiteral('-'); + } + if (milli) { + bld.appendLiteral('.'); + bld.appendMillisOfSecond(3); + } + } + + //----------------------------------------------------------------------- + /** + * Checks that the iso only flag is not set, throwing an exception if it is. + * + * @param fields the fields + * @param strictISO true if only ISO formats allowed + * @since 1.1 + */ + private static void checkNotStrictISO(Collection fields, boolean strictISO) { + if (strictISO) { + throw new IllegalArgumentException("No valid ISO8601 format for fields: " + fields); + } + } + + /** + * Appends the separator if necessary. + * + * @param bld the builder + * @param extended whether to append the separator + * @since 1.1 + */ + private static void appendSeparator(DateTimeFormatterBuilder bld, boolean extended) { + if (extended) { + bld.appendLiteral('-'); + } + } + + //----------------------------------------------------------------------- + /** + * Returns a generic ISO date parser for parsing dates with a possible zone. + *

+ * The returned formatter can only be used for parsing, printing is unsupported. + *

+ * It accepts formats described by the following syntax: + *

+     * date              = date-element ['T' offset]
+     * date-element      = std-date-element | ord-date-element | week-date-element
+     * std-date-element  = yyyy ['-' MM ['-' dd]]
+     * ord-date-element  = yyyy ['-' DDD]
+     * week-date-element = xxxx '-W' ww ['-' e]
+     * offset            = 'Z' | (('+' | '-') HH [':' mm [':' ss [('.' | ',') SSS]]])
+     * 
+ */ + public static DateTimeFormatter dateParser() { + return Constants.dp; + } + + /** + * Returns a generic ISO date parser for parsing local dates. + *

+ * The returned formatter can only be used for parsing, printing is unsupported. + *

+ * This parser is initialised with the local (UTC) time zone. + *

+ * It accepts formats described by the following syntax: + *

+     * date-element      = std-date-element | ord-date-element | week-date-element
+     * std-date-element  = yyyy ['-' MM ['-' dd]]
+     * ord-date-element  = yyyy ['-' DDD]
+     * week-date-element = xxxx '-W' ww ['-' e]
+     * 
+ * @since 1.3 + */ + public static DateTimeFormatter localDateParser() { + return Constants.ldp; + } + + /** + * Returns a generic ISO date parser for parsing dates. + *

+ * The returned formatter can only be used for parsing, printing is unsupported. + *

+ * It accepts formats described by the following syntax: + *

+     * date-element      = std-date-element | ord-date-element | week-date-element
+     * std-date-element  = yyyy ['-' MM ['-' dd]]
+     * ord-date-element  = yyyy ['-' DDD]
+     * week-date-element = xxxx '-W' ww ['-' e]
+     * 
+ */ + public static DateTimeFormatter dateElementParser() { + return Constants.dpe; + } + + /** + * Returns a generic ISO time parser for parsing times with a possible zone. + *

+ * The returned formatter can only be used for parsing, printing is unsupported. + *

+ * The parser is strict by default, thus time string {@code 24:00} cannot be parsed. + *

+ * It accepts formats described by the following syntax: + *

+     * time           = ['T'] time-element [offset]
+     * time-element   = HH [minute-element] | [fraction]
+     * minute-element = ':' mm [second-element] | [fraction]
+     * second-element = ':' ss [fraction]
+     * fraction       = ('.' | ',') digit+
+     * offset         = 'Z' | (('+' | '-') HH [':' mm [':' ss [('.' | ',') SSS]]])
+     * 
+ */ + public static DateTimeFormatter timeParser() { + return Constants.tp; + } + + /** + * Returns a generic ISO time parser for parsing local times. + *

+ * The returned formatter can only be used for parsing, printing is unsupported. + *

+ * This parser is initialised with the local (UTC) time zone. + * The parser is strict by default, thus time string {@code 24:00} cannot be parsed. + *

+ * It accepts formats described by the following syntax: + *

+     * time           = ['T'] time-element
+     * time-element   = HH [minute-element] | [fraction]
+     * minute-element = ':' mm [second-element] | [fraction]
+     * second-element = ':' ss [fraction]
+     * fraction       = ('.' | ',') digit+
+     * 
+ * @since 1.3 + */ + public static DateTimeFormatter localTimeParser() { + return Constants.ltp; + } + + /** + * Returns a generic ISO time parser. + *

+ * The returned formatter can only be used for parsing, printing is unsupported. + *

+ * The parser is strict by default, thus time string {@code 24:00} cannot be parsed. + *

+ * It accepts formats described by the following syntax: + *

+     * time-element   = HH [minute-element] | [fraction]
+     * minute-element = ':' mm [second-element] | [fraction]
+     * second-element = ':' ss [fraction]
+     * fraction       = ('.' | ',') digit+
+     * 
+ */ + public static DateTimeFormatter timeElementParser() { + return Constants.tpe; + } + + /** + * Returns a generic ISO datetime parser which parses either a date or a time or both. + *

+ * The returned formatter can only be used for parsing, printing is unsupported. + *

+ * The parser is strict by default, thus time string {@code 24:00} cannot be parsed. + *

+ * It accepts formats described by the following syntax: + *

+     * datetime          = time | date-opt-time
+     * time              = 'T' time-element [offset]
+     * date-opt-time     = date-element ['T' [time-element] [offset]]
+     * date-element      = std-date-element | ord-date-element | week-date-element
+     * std-date-element  = yyyy ['-' MM ['-' dd]]
+     * ord-date-element  = yyyy ['-' DDD]
+     * week-date-element = xxxx '-W' ww ['-' e]
+     * time-element      = HH [minute-element] | [fraction]
+     * minute-element    = ':' mm [second-element] | [fraction]
+     * second-element    = ':' ss [fraction]
+     * fraction          = ('.' | ',') digit+
+     * offset            = 'Z' | (('+' | '-') HH [':' mm [':' ss [('.' | ',') SSS]]])
+     * 
+ */ + public static DateTimeFormatter dateTimeParser() { + return Constants.dtp; + } + + /** + * Returns a generic ISO datetime parser where the date is mandatory and the time is optional. + *

+ * The returned formatter can only be used for parsing, printing is unsupported. + *

+ * This parser can parse zoned datetimes. + * The parser is strict by default, thus time string {@code 24:00} cannot be parsed. + *

+ * It accepts formats described by the following syntax: + *

+     * date-opt-time     = date-element ['T' [time-element] [offset]]
+     * date-element      = std-date-element | ord-date-element | week-date-element
+     * std-date-element  = yyyy ['-' MM ['-' dd]]
+     * ord-date-element  = yyyy ['-' DDD]
+     * week-date-element = xxxx '-W' ww ['-' e]
+     * time-element      = HH [minute-element] | [fraction]
+     * minute-element    = ':' mm [second-element] | [fraction]
+     * second-element    = ':' ss [fraction]
+     * fraction          = ('.' | ',') digit+
+     * 
+ * @since 1.3 + */ + public static DateTimeFormatter dateOptionalTimeParser() { + return Constants.dotp; + } + + /** + * Returns a generic ISO datetime parser where the date is mandatory and the time is optional. + *

+ * The returned formatter can only be used for parsing, printing is unsupported. + *

+ * This parser only parses local datetimes. + * This parser is initialised with the local (UTC) time zone. + * The parser is strict by default, thus time string {@code 24:00} cannot be parsed. + *

+ * It accepts formats described by the following syntax: + *

+     * datetime          = date-element ['T' time-element]
+     * date-element      = std-date-element | ord-date-element | week-date-element
+     * std-date-element  = yyyy ['-' MM ['-' dd]]
+     * ord-date-element  = yyyy ['-' DDD]
+     * week-date-element = xxxx '-W' ww ['-' e]
+     * time-element      = HH [minute-element] | [fraction]
+     * minute-element    = ':' mm [second-element] | [fraction]
+     * second-element    = ':' ss [fraction]
+     * fraction          = ('.' | ',') digit+
+     * 
+ * @since 1.3 + */ + public static DateTimeFormatter localDateOptionalTimeParser() { + return Constants.ldotp; + } + + //----------------------------------------------------------------------- + /** + * Returns a formatter for a full date as four digit year, two digit month + * of year, and two digit day of month (yyyy-MM-dd). + *

+ * The returned formatter prints and parses only this format. + * See {@link #dateParser()} for a more flexible parser that accepts different formats. + * + * @return a formatter for yyyy-MM-dd + */ + public static DateTimeFormatter date() { + return yearMonthDay(); + } + + /** + * Returns a formatter for a two digit hour of day, two digit minute of + * hour, two digit second of minute, three digit fraction of second, and + * time zone offset (HH:mm:ss.SSSZZ). + *

+ * The time zone offset is 'Z' for zero, and of the form '\u00b1HH:mm' for non-zero. + * The parser is strict by default, thus time string {@code 24:00} cannot be parsed. + *

+ * The returned formatter prints and parses only this format, which includes milliseconds. + * See {@link #timeParser()} for a more flexible parser that accepts different formats. + * + * @return a formatter for HH:mm:ss.SSSZZ + */ + public static DateTimeFormatter time() { + return Constants.t; + } + + /** + * Returns a formatter for a two digit hour of day, two digit minute of + * hour, two digit second of minute, and time zone offset (HH:mm:ssZZ). + *

+ * The time zone offset is 'Z' for zero, and of the form '\u00b1HH:mm' for non-zero. + * The parser is strict by default, thus time string {@code 24:00} cannot be parsed. + *

+ * The returned formatter prints and parses only this format, which excludes milliseconds. + * See {@link #timeParser()} for a more flexible parser that accepts different formats. + * + * @return a formatter for HH:mm:ssZZ + */ + public static DateTimeFormatter timeNoMillis() { + return Constants.tx; + } + + /** + * Returns a formatter for a two digit hour of day, two digit minute of + * hour, two digit second of minute, three digit fraction of second, and + * time zone offset prefixed by 'T' ('T'HH:mm:ss.SSSZZ). + *

+ * The time zone offset is 'Z' for zero, and of the form '\u00b1HH:mm' for non-zero. + * The parser is strict by default, thus time string {@code 24:00} cannot be parsed. + *

+ * The returned formatter prints and parses only this format, which includes milliseconds. + * See {@link #timeParser()} for a more flexible parser that accepts different formats. + * + * @return a formatter for 'T'HH:mm:ss.SSSZZ + */ + public static DateTimeFormatter tTime() { + return Constants.tt; + } + + /** + * Returns a formatter for a two digit hour of day, two digit minute of + * hour, two digit second of minute, and time zone offset prefixed + * by 'T' ('T'HH:mm:ssZZ). + *

+ * The time zone offset is 'Z' for zero, and of the form '\u00b1HH:mm' for non-zero. + * The parser is strict by default, thus time string {@code 24:00} cannot be parsed. + *

+ * The returned formatter prints and parses only this format, which excludes milliseconds. + * See {@link #timeParser()} for a more flexible parser that accepts different formats. + * + * @return a formatter for 'T'HH:mm:ssZZ + */ + public static DateTimeFormatter tTimeNoMillis() { + return Constants.ttx; + } + + /** + * Returns a formatter that combines a full date and time, separated by a 'T' + * (yyyy-MM-dd'T'HH:mm:ss.SSSZZ). + *

+ * The time zone offset is 'Z' for zero, and of the form '\u00b1HH:mm' for non-zero. + * The parser is strict by default, thus time string {@code 24:00} cannot be parsed. + *

+ * The returned formatter prints and parses only this format, which includes milliseconds. + * See {@link #dateTimeParser()} for a more flexible parser that accepts different formats. + * + * @return a formatter for yyyy-MM-dd'T'HH:mm:ss.SSSZZ + */ + public static DateTimeFormatter dateTime() { + return Constants.dt; + } + + /** + * Returns a formatter that combines a full date and time without millis, + * separated by a 'T' (yyyy-MM-dd'T'HH:mm:ssZZ). + *

+ * The time zone offset is 'Z' for zero, and of the form '\u00b1HH:mm' for non-zero. + * The parser is strict by default, thus time string {@code 24:00} cannot be parsed. + *

+ * The returned formatter prints and parses only this format, which excludes milliseconds. + * See {@link #dateTimeParser()} for a more flexible parser that accepts different formats. + * + * @return a formatter for yyyy-MM-dd'T'HH:mm:ssZZ + */ + public static DateTimeFormatter dateTimeNoMillis() { + return Constants.dtx; + } + + /** + * Returns a formatter for a full ordinal date, using a four + * digit year and three digit dayOfYear (yyyy-DDD). + *

+ * The returned formatter prints and parses only this format. + * See {@link #dateParser()} for a more flexible parser that accepts different formats. + * + * @return a formatter for yyyy-DDD + * @since 1.1 + */ + public static DateTimeFormatter ordinalDate() { + return Constants.od; + } + + /** + * Returns a formatter for a full ordinal date and time, using a four + * digit year and three digit dayOfYear (yyyy-DDD'T'HH:mm:ss.SSSZZ). + *

+ * The time zone offset is 'Z' for zero, and of the form '\u00b1HH:mm' for non-zero. + * The parser is strict by default, thus time string {@code 24:00} cannot be parsed. + *

+ * The returned formatter prints and parses only this format, which includes milliseconds. + * See {@link #dateTimeParser()} for a more flexible parser that accepts different formats. + * + * @return a formatter for yyyy-DDD'T'HH:mm:ss.SSSZZ + * @since 1.1 + */ + public static DateTimeFormatter ordinalDateTime() { + return Constants.odt; + } + + /** + * Returns a formatter for a full ordinal date and time without millis, + * using a four digit year and three digit dayOfYear (yyyy-DDD'T'HH:mm:ssZZ). + *

+ * The time zone offset is 'Z' for zero, and of the form '\u00b1HH:mm' for non-zero. + * The parser is strict by default, thus time string {@code 24:00} cannot be parsed. + *

+ * The returned formatter prints and parses only this format, which excludes milliseconds. + * See {@link #dateTimeParser()} for a more flexible parser that accepts different formats. + * + * @return a formatter for yyyy-DDD'T'HH:mm:ssZZ + * @since 1.1 + */ + public static DateTimeFormatter ordinalDateTimeNoMillis() { + return Constants.odtx; + } + + /** + * Returns a formatter for a full date as four digit weekyear, two digit + * week of weekyear, and one digit day of week (xxxx-'W'ww-e). + *

+ * The returned formatter prints and parses only this format. + * See {@link #dateParser()} for a more flexible parser that accepts different formats. + * + * @return a formatter for xxxx-'W'ww-e + */ + public static DateTimeFormatter weekDate() { + return Constants.wwd; + } + + /** + * Returns a formatter that combines a full weekyear date and time, + * separated by a 'T' (xxxx-'W'ww-e'T'HH:mm:ss.SSSZZ). + *

+ * The time zone offset is 'Z' for zero, and of the form '\u00b1HH:mm' for non-zero. + * The parser is strict by default, thus time string {@code 24:00} cannot be parsed. + *

+ * The returned formatter prints and parses only this format, which includes milliseconds. + * See {@link #dateTimeParser()} for a more flexible parser that accepts different formats. + * + * @return a formatter for xxxx-'W'ww-e'T'HH:mm:ss.SSSZZ + */ + public static DateTimeFormatter weekDateTime() { + return Constants.wdt; + } + + /** + * Returns a formatter that combines a full weekyear date and time without millis, + * separated by a 'T' (xxxx-'W'ww-e'T'HH:mm:ssZZ). + *

+ * The time zone offset is 'Z' for zero, and of the form '\u00b1HH:mm' for non-zero. + * The parser is strict by default, thus time string {@code 24:00} cannot be parsed. + *

+ * The returned formatter prints and parses only this format, which excludes milliseconds. + * See {@link #dateTimeParser()} for a more flexible parser that accepts different formats. + * + * @return a formatter for xxxx-'W'ww-e'T'HH:mm:ssZZ + */ + public static DateTimeFormatter weekDateTimeNoMillis() { + return Constants.wdtx; + } + + //----------------------------------------------------------------------- + /** + * Returns a basic formatter for a full date as four digit year, two digit + * month of year, and two digit day of month (yyyyMMdd). + *

+ * The returned formatter prints and parses only this format. + * + * @return a formatter for yyyyMMdd + */ + public static DateTimeFormatter basicDate() { + return Constants.bd; + } + + /** + * Returns a basic formatter for a two digit hour of day, two digit minute + * of hour, two digit second of minute, three digit millis, and time zone + * offset (HHmmss.SSSZ). + *

+ * The time zone offset is 'Z' for zero, and of the form '\u00b1HHmm' for non-zero. + * The parser is strict by default, thus time string {@code 24:00} cannot be parsed. + *

+ * The returned formatter prints and parses only this format, which includes milliseconds. + * + * @return a formatter for HHmmss.SSSZ + */ + public static DateTimeFormatter basicTime() { + return Constants.bt; + } + + /** + * Returns a basic formatter for a two digit hour of day, two digit minute + * of hour, two digit second of minute, and time zone offset (HHmmssZ). + *

+ * The time zone offset is 'Z' for zero, and of the form '\u00b1HHmm' for non-zero. + * The parser is strict by default, thus time string {@code 24:00} cannot be parsed. + *

+ * The returned formatter prints and parses only this format, which excludes milliseconds. + * + * @return a formatter for HHmmssZ + */ + public static DateTimeFormatter basicTimeNoMillis() { + return Constants.btx; + } + + /** + * Returns a basic formatter for a two digit hour of day, two digit minute + * of hour, two digit second of minute, three digit millis, and time zone + * offset prefixed by 'T' ('T'HHmmss.SSSZ). + *

+ * The time zone offset is 'Z' for zero, and of the form '\u00b1HHmm' for non-zero. + * The parser is strict by default, thus time string {@code 24:00} cannot be parsed. + *

+ * The returned formatter prints and parses only this format, which includes milliseconds. + * + * @return a formatter for 'T'HHmmss.SSSZ + */ + public static DateTimeFormatter basicTTime() { + return Constants.btt; + } + + /** + * Returns a basic formatter for a two digit hour of day, two digit minute + * of hour, two digit second of minute, and time zone offset prefixed by 'T' + * ('T'HHmmssZ). + *

+ * The time zone offset is 'Z' for zero, and of the form '\u00b1HHmm' for non-zero. + * The parser is strict by default, thus time string {@code 24:00} cannot be parsed. + *

+ * The returned formatter prints and parses only this format, which excludes milliseconds. + * + * @return a formatter for 'T'HHmmssZ + */ + public static DateTimeFormatter basicTTimeNoMillis() { + return Constants.bttx; + } + + /** + * Returns a basic formatter that combines a basic date and time, separated + * by a 'T' (yyyyMMdd'T'HHmmss.SSSZ). + *

+ * The time zone offset is 'Z' for zero, and of the form '\u00b1HHmm' for non-zero. + * The parser is strict by default, thus time string {@code 24:00} cannot be parsed. + *

+ * The returned formatter prints and parses only this format, which includes milliseconds. + * + * @return a formatter for yyyyMMdd'T'HHmmss.SSSZ + */ + public static DateTimeFormatter basicDateTime() { + return Constants.bdt; + } + + /** + * Returns a basic formatter that combines a basic date and time without millis, + * separated by a 'T' (yyyyMMdd'T'HHmmssZ). + *

+ * The time zone offset is 'Z' for zero, and of the form '\u00b1HHmm' for non-zero. + * The parser is strict by default, thus time string {@code 24:00} cannot be parsed. + *

+ * The returned formatter prints and parses only this format, which excludes milliseconds. + * + * @return a formatter for yyyyMMdd'T'HHmmssZ + */ + public static DateTimeFormatter basicDateTimeNoMillis() { + return Constants.bdtx; + } + + /** + * Returns a formatter for a full ordinal date, using a four + * digit year and three digit dayOfYear (yyyyDDD). + *

+ * The returned formatter prints and parses only this format. + * + * @return a formatter for yyyyDDD + * @since 1.1 + */ + public static DateTimeFormatter basicOrdinalDate() { + return Constants.bod; + } + + /** + * Returns a formatter for a full ordinal date and time, using a four + * digit year and three digit dayOfYear (yyyyDDD'T'HHmmss.SSSZ). + *

+ * The time zone offset is 'Z' for zero, and of the form '\u00b1HHmm' for non-zero. + * The parser is strict by default, thus time string {@code 24:00} cannot be parsed. + *

+ * The returned formatter prints and parses only this format, which includes milliseconds. + * + * @return a formatter for yyyyDDD'T'HHmmss.SSSZ + * @since 1.1 + */ + public static DateTimeFormatter basicOrdinalDateTime() { + return Constants.bodt; + } + + /** + * Returns a formatter for a full ordinal date and time without millis, + * using a four digit year and three digit dayOfYear (yyyyDDD'T'HHmmssZ). + *

+ * The time zone offset is 'Z' for zero, and of the form '\u00b1HHmm' for non-zero. + * The parser is strict by default, thus time string {@code 24:00} cannot be parsed. + *

+ * The returned formatter prints and parses only this format, which excludes milliseconds. + * + * @return a formatter for yyyyDDD'T'HHmmssZ + * @since 1.1 + */ + public static DateTimeFormatter basicOrdinalDateTimeNoMillis() { + return Constants.bodtx; + } + + /** + * Returns a basic formatter for a full date as four digit weekyear, two + * digit week of weekyear, and one digit day of week (xxxx'W'wwe). + *

+ * The returned formatter prints and parses only this format. + * + * @return a formatter for xxxx'W'wwe + */ + public static DateTimeFormatter basicWeekDate() { + return Constants.bwd; + } + + /** + * Returns a basic formatter that combines a basic weekyear date and time, + * separated by a 'T' (xxxx'W'wwe'T'HHmmss.SSSZ). + *

+ * The time zone offset is 'Z' for zero, and of the form '\u00b1HHmm' for non-zero. + * The parser is strict by default, thus time string {@code 24:00} cannot be parsed. + *

+ * The returned formatter prints and parses only this format, which includes milliseconds. + * + * @return a formatter for xxxx'W'wwe'T'HHmmss.SSSZ + */ + public static DateTimeFormatter basicWeekDateTime() { + return Constants.bwdt; + } + + /** + * Returns a basic formatter that combines a basic weekyear date and time + * without millis, separated by a 'T' (xxxx'W'wwe'T'HHmmssZ). + *

+ * The time zone offset is 'Z' for zero, and of the form '\u00b1HHmm' for non-zero. + * The parser is strict by default, thus time string {@code 24:00} cannot be parsed. + *

+ * The returned formatter prints and parses only this format, which excludes milliseconds. + * + * @return a formatter for xxxx'W'wwe'T'HHmmssZ + */ + public static DateTimeFormatter basicWeekDateTimeNoMillis() { + return Constants.bwdtx; + } + + //----------------------------------------------------------------------- + /** + * Returns a formatter for a four digit year. (yyyy) + * + * @return a formatter for yyyy + */ + public static DateTimeFormatter year() { + return Constants.ye; + } + + /** + * Returns a formatter for a four digit year and two digit month of + * year. (yyyy-MM) + * + * @return a formatter for yyyy-MM + */ + public static DateTimeFormatter yearMonth() { + return Constants.ym; + } + + /** + * Returns a formatter for a four digit year, two digit month of year, and + * two digit day of month. (yyyy-MM-dd) + * + * @return a formatter for yyyy-MM-dd + */ + public static DateTimeFormatter yearMonthDay() { + return Constants.ymd; + } + + /** + * Returns a formatter for a four digit weekyear. (xxxx) + * + * @return a formatter for xxxx + */ + public static DateTimeFormatter weekyear() { + return Constants.we; + } + + /** + * Returns a formatter for a four digit weekyear and two digit week of + * weekyear. (xxxx-'W'ww) + * + * @return a formatter for xxxx-'W'ww + */ + public static DateTimeFormatter weekyearWeek() { + return Constants.ww; + } + + /** + * Returns a formatter for a four digit weekyear, two digit week of + * weekyear, and one digit day of week. (xxxx-'W'ww-e) + * + * @return a formatter for xxxx-'W'ww-e + */ + public static DateTimeFormatter weekyearWeekDay() { + return Constants.wwd; + } + + /** + * Returns a formatter for a two digit hour of day. (HH) + * + * @return a formatter for HH + */ + public static DateTimeFormatter hour() { + return Constants.hde; + } + + /** + * Returns a formatter for a two digit hour of day and two digit minute of + * hour. (HH:mm) + * + * @return a formatter for HH:mm + */ + public static DateTimeFormatter hourMinute() { + return Constants.hm; + } + + /** + * Returns a formatter for a two digit hour of day, two digit minute of + * hour, and two digit second of minute. (HH:mm:ss) + * + * @return a formatter for HH:mm:ss + */ + public static DateTimeFormatter hourMinuteSecond() { + return Constants.hms; + } + + /** + * Returns a formatter for a two digit hour of day, two digit minute of + * hour, two digit second of minute, and three digit fraction of + * second (HH:mm:ss.SSS). Parsing will parse up to 3 fractional second + * digits. + * + * @return a formatter for HH:mm:ss.SSS + */ + public static DateTimeFormatter hourMinuteSecondMillis() { + return Constants.hmsl; + } + + /** + * Returns a formatter for a two digit hour of day, two digit minute of + * hour, two digit second of minute, and three digit fraction of + * second (HH:mm:ss.SSS). Parsing will parse up to 9 fractional second + * digits, throwing away all except the first three. + * + * @return a formatter for HH:mm:ss.SSS + */ + public static DateTimeFormatter hourMinuteSecondFraction() { + return Constants.hmsf; + } + + /** + * Returns a formatter that combines a full date and two digit hour of + * day. (yyyy-MM-dd'T'HH) + * + * @return a formatter for yyyy-MM-dd'T'HH + */ + public static DateTimeFormatter dateHour() { + return Constants.dh; + } + + /** + * Returns a formatter that combines a full date, two digit hour of day, + * and two digit minute of hour. (yyyy-MM-dd'T'HH:mm) + * + * @return a formatter for yyyy-MM-dd'T'HH:mm + */ + public static DateTimeFormatter dateHourMinute() { + return Constants.dhm; + } + + /** + * Returns a formatter that combines a full date, two digit hour of day, + * two digit minute of hour, and two digit second of + * minute. (yyyy-MM-dd'T'HH:mm:ss) + * + * @return a formatter for yyyy-MM-dd'T'HH:mm:ss + */ + public static DateTimeFormatter dateHourMinuteSecond() { + return Constants.dhms; + } + + /** + * Returns a formatter that combines a full date, two digit hour of day, + * two digit minute of hour, two digit second of minute, and three digit + * fraction of second (yyyy-MM-dd'T'HH:mm:ss.SSS). Parsing will parse up + * to 3 fractional second digits. + * + * @return a formatter for yyyy-MM-dd'T'HH:mm:ss.SSS + */ + public static DateTimeFormatter dateHourMinuteSecondMillis() { + return Constants.dhmsl; + } + + /** + * Returns a formatter that combines a full date, two digit hour of day, + * two digit minute of hour, two digit second of minute, and three digit + * fraction of second (yyyy-MM-dd'T'HH:mm:ss.SSS). Parsing will parse up + * to 9 fractional second digits, throwing away all except the first three. + * + * @return a formatter for yyyy-MM-dd'T'HH:mm:ss.SSS + */ + public static DateTimeFormatter dateHourMinuteSecondFraction() { + return Constants.dhmsf; + } + + //----------------------------------------------------------------------- + static final class Constants { + private static final DateTimeFormatter + ye = yearElement(), // year element (yyyy) + mye = monthElement(), // monthOfYear element (-MM) + dme = dayOfMonthElement(), // dayOfMonth element (-dd) + we = weekyearElement(), // weekyear element (xxxx) + wwe = weekElement(), // weekOfWeekyear element (-ww) + dwe = dayOfWeekElement(), // dayOfWeek element (-ee) + dye = dayOfYearElement(), // dayOfYear element (-DDD) + hde = hourElement(), // hourOfDay element (HH) + mhe = minuteElement(), // minuteOfHour element (:mm) + sme = secondElement(), // secondOfMinute element (:ss) + fse = fractionElement(), // fractionOfSecond element (.SSSSSSSSS) + ze = offsetElement(), // zone offset element + lte = literalTElement(), // literal 'T' element + + //y, // year (same as year element) + ym = yearMonth(), // year month + ymd = yearMonthDay(), // year month day + + //w, // weekyear (same as weekyear element) + ww = weekyearWeek(), // weekyear week + wwd = weekyearWeekDay(), // weekyear week day + + //h, // hour (same as hour element) + hm = hourMinute(), // hour minute + hms = hourMinuteSecond(), // hour minute second + hmsl = hourMinuteSecondMillis(), // hour minute second millis + hmsf = hourMinuteSecondFraction(), // hour minute second fraction + + dh = dateHour(), // date hour + dhm = dateHourMinute(), // date hour minute + dhms = dateHourMinuteSecond(), // date hour minute second + dhmsl = dateHourMinuteSecondMillis(), // date hour minute second millis + dhmsf = dateHourMinuteSecondFraction(), // date hour minute second fraction + + //d, // date (same as ymd) + t = time(), // time + tx = timeNoMillis(), // time no millis + tt = tTime(), // Ttime + ttx = tTimeNoMillis(), // Ttime no millis + dt = dateTime(), // date time + dtx = dateTimeNoMillis(), // date time no millis + + //wd, // week date (same as wwd) + wdt = weekDateTime(), // week date time + wdtx = weekDateTimeNoMillis(), // week date time no millis + + od = ordinalDate(), // ordinal date (same as yd) + odt = ordinalDateTime(), // ordinal date time + odtx = ordinalDateTimeNoMillis(), // ordinal date time no millis + + bd = basicDate(), // basic date + bt = basicTime(), // basic time + btx = basicTimeNoMillis(), // basic time no millis + btt = basicTTime(), // basic Ttime + bttx = basicTTimeNoMillis(), // basic Ttime no millis + bdt = basicDateTime(), // basic date time + bdtx = basicDateTimeNoMillis(), // basic date time no millis + + bod = basicOrdinalDate(), // basic ordinal date + bodt = basicOrdinalDateTime(), // basic ordinal date time + bodtx = basicOrdinalDateTimeNoMillis(), // basic ordinal date time no millis + + bwd = basicWeekDate(), // basic week date + bwdt = basicWeekDateTime(), // basic week date time + bwdtx = basicWeekDateTimeNoMillis(), // basic week date time no millis + + dpe = dateElementParser(), // date parser element + tpe = timeElementParser(), // time parser element + dp = dateParser(), // date parser + ldp = localDateParser(), // local date parser + tp = timeParser(), // time parser + ltp = localTimeParser(), // local time parser + dtp = dateTimeParser(), // date time parser + dotp = dateOptionalTimeParser(), // date optional time parser + ldotp = localDateOptionalTimeParser(); // local date optional time parser + + //----------------------------------------------------------------------- + private static DateTimeFormatter dateParser() { + if (dp == null) { + DateTimeParser tOffset = new DateTimeFormatterBuilder() + .appendLiteral('T') + .append(offsetElement()).toParser(); + return new DateTimeFormatterBuilder() + .append(dateElementParser()) + .appendOptional(tOffset) + .toFormatter(); + } + return dp; + } + + private static DateTimeFormatter localDateParser() { + if (ldp == null) { + return dateElementParser().withZoneUTC(); + } + return ldp; + } + + private static DateTimeFormatter dateElementParser() { + if (dpe == null) { + return new DateTimeFormatterBuilder() + .append(null, new DateTimeParser[] { + new DateTimeFormatterBuilder() + .append(yearElement()) + .appendOptional + (new DateTimeFormatterBuilder() + .append(monthElement()) + .appendOptional(dayOfMonthElement().getParser()) + .toParser()) + .toParser(), + new DateTimeFormatterBuilder() + .append(weekyearElement()) + .append(weekElement()) + .appendOptional(dayOfWeekElement().getParser()) + .toParser(), + new DateTimeFormatterBuilder() + .append(yearElement()) + .append(dayOfYearElement()) + .toParser() + }) + .toFormatter(); + } + return dpe; + } + + private static DateTimeFormatter timeParser() { + if (tp == null) { + return new DateTimeFormatterBuilder() + .appendOptional(literalTElement().getParser()) + .append(timeElementParser()) + .appendOptional(offsetElement().getParser()) + .toFormatter(); + } + return tp; + } + + private static DateTimeFormatter localTimeParser() { + if (ltp == null) { + return new DateTimeFormatterBuilder() + .appendOptional(literalTElement().getParser()) + .append(timeElementParser()) + .toFormatter().withZoneUTC(); + } + return ltp; + } + + private static DateTimeFormatter timeElementParser() { + if (tpe == null) { + // Decimal point can be either '.' or ',' + DateTimeParser decimalPoint = new DateTimeFormatterBuilder() + .append(null, new DateTimeParser[] { + new DateTimeFormatterBuilder() + .appendLiteral('.') + .toParser(), + new DateTimeFormatterBuilder() + .appendLiteral(',') + .toParser() + }) + .toParser(); + + return new DateTimeFormatterBuilder() + // time-element + .append(hourElement()) + .append + (null, new DateTimeParser[] { + new DateTimeFormatterBuilder() + // minute-element + .append(minuteElement()) + .append + (null, new DateTimeParser[] { + new DateTimeFormatterBuilder() + // second-element + .append(secondElement()) + // second fraction + .appendOptional(new DateTimeFormatterBuilder() + .append(decimalPoint) + .appendFractionOfSecond(1, 9) + .toParser()) + .toParser(), + // minute fraction + new DateTimeFormatterBuilder() + .append(decimalPoint) + .appendFractionOfMinute(1, 9) + .toParser(), + null + }) + .toParser(), + // hour fraction + new DateTimeFormatterBuilder() + .append(decimalPoint) + .appendFractionOfHour(1, 9) + .toParser(), + null + }) + .toFormatter(); + } + return tpe; + } + + private static DateTimeFormatter dateTimeParser() { + if (dtp == null) { + // This is different from the general time parser in that the 'T' + // is required. + DateTimeParser time = new DateTimeFormatterBuilder() + .appendLiteral('T') + .append(timeElementParser()) + .appendOptional(offsetElement().getParser()) + .toParser(); + return new DateTimeFormatterBuilder() + .append(null, new DateTimeParser[] {time, dateOptionalTimeParser().getParser()}) + .toFormatter(); + } + return dtp; + } + + private static DateTimeFormatter dateOptionalTimeParser() { + if (dotp == null) { + DateTimeParser timeOrOffset = new DateTimeFormatterBuilder() + .appendLiteral('T') + .appendOptional(timeElementParser().getParser()) + .appendOptional(offsetElement().getParser()) + .toParser(); + return new DateTimeFormatterBuilder() + .append(dateElementParser()) + .appendOptional(timeOrOffset) + .toFormatter(); + } + return dotp; + } + + private static DateTimeFormatter localDateOptionalTimeParser() { + if (ldotp == null) { + DateTimeParser time = new DateTimeFormatterBuilder() + .appendLiteral('T') + .append(timeElementParser()) + .toParser(); + return new DateTimeFormatterBuilder() + .append(dateElementParser()) + .appendOptional(time) + .toFormatter().withZoneUTC(); + } + return ldotp; + } + + //----------------------------------------------------------------------- + private static DateTimeFormatter time() { + if (t == null) { + return new DateTimeFormatterBuilder() + .append(hourMinuteSecondFraction()) + .append(offsetElement()) + .toFormatter(); + } + return t; + } + + private static DateTimeFormatter timeNoMillis() { + if (tx == null) { + return new DateTimeFormatterBuilder() + .append(hourMinuteSecond()) + .append(offsetElement()) + .toFormatter(); + } + return tx; + } + + private static DateTimeFormatter tTime() { + if (tt == null) { + return new DateTimeFormatterBuilder() + .append(literalTElement()) + .append(time()) + .toFormatter(); + } + return tt; + } + + private static DateTimeFormatter tTimeNoMillis() { + if (ttx == null) { + return new DateTimeFormatterBuilder() + .append(literalTElement()) + .append(timeNoMillis()) + .toFormatter(); + } + return ttx; + } + + private static DateTimeFormatter dateTime() { + if (dt == null) { + return new DateTimeFormatterBuilder() + .append(date()) + .append(tTime()) + .toFormatter(); + } + return dt; + } + + private static DateTimeFormatter dateTimeNoMillis() { + if (dtx == null) { + return new DateTimeFormatterBuilder() + .append(date()) + .append(tTimeNoMillis()) + .toFormatter(); + } + return dtx; + } + + private static DateTimeFormatter ordinalDate() { + if (od == null) { + return new DateTimeFormatterBuilder() + .append(yearElement()) + .append(dayOfYearElement()) + .toFormatter(); + } + return od; + } + + private static DateTimeFormatter ordinalDateTime() { + if (odt == null) { + return new DateTimeFormatterBuilder() + .append(ordinalDate()) + .append(tTime()) + .toFormatter(); + } + return odt; + } + + private static DateTimeFormatter ordinalDateTimeNoMillis() { + if (odtx == null) { + return new DateTimeFormatterBuilder() + .append(ordinalDate()) + .append(tTimeNoMillis()) + .toFormatter(); + } + return odtx; + } + + private static DateTimeFormatter weekDateTime() { + if (wdt == null) { + return new DateTimeFormatterBuilder() + .append(weekDate()) + .append(tTime()) + .toFormatter(); + } + return wdt; + } + + private static DateTimeFormatter weekDateTimeNoMillis() { + if (wdtx == null) { + return new DateTimeFormatterBuilder() + .append(weekDate()) + .append(tTimeNoMillis()) + .toFormatter(); + } + return wdtx; + } + + //----------------------------------------------------------------------- + private static DateTimeFormatter basicDate() { + if (bd == null) { + return new DateTimeFormatterBuilder() + .appendYear(4, 4) + .appendFixedDecimal(DateTimeFieldType.monthOfYear(), 2) + .appendFixedDecimal(DateTimeFieldType.dayOfMonth(), 2) + .toFormatter(); + } + return bd; + } + + private static DateTimeFormatter basicTime() { + if (bt == null) { + return new DateTimeFormatterBuilder() + .appendFixedDecimal(DateTimeFieldType.hourOfDay(), 2) + .appendFixedDecimal(DateTimeFieldType.minuteOfHour(), 2) + .appendFixedDecimal(DateTimeFieldType.secondOfMinute(), 2) + .appendLiteral('.') + .appendFractionOfSecond(3, 9) + .appendTimeZoneOffset("Z", false, 2, 2) + .toFormatter(); + } + return bt; + } + + private static DateTimeFormatter basicTimeNoMillis() { + if (btx == null) { + return new DateTimeFormatterBuilder() + .appendFixedDecimal(DateTimeFieldType.hourOfDay(), 2) + .appendFixedDecimal(DateTimeFieldType.minuteOfHour(), 2) + .appendFixedDecimal(DateTimeFieldType.secondOfMinute(), 2) + .appendTimeZoneOffset("Z", false, 2, 2) + .toFormatter(); + } + return btx; + } + + private static DateTimeFormatter basicTTime() { + if (btt == null) { + return new DateTimeFormatterBuilder() + .append(literalTElement()) + .append(basicTime()) + .toFormatter(); + } + return btt; + } + + private static DateTimeFormatter basicTTimeNoMillis() { + if (bttx == null) { + return new DateTimeFormatterBuilder() + .append(literalTElement()) + .append(basicTimeNoMillis()) + .toFormatter(); + } + return bttx; + } + + private static DateTimeFormatter basicDateTime() { + if (bdt == null) { + return new DateTimeFormatterBuilder() + .append(basicDate()) + .append(basicTTime()) + .toFormatter(); + } + return bdt; + } + + private static DateTimeFormatter basicDateTimeNoMillis() { + if (bdtx == null) { + return new DateTimeFormatterBuilder() + .append(basicDate()) + .append(basicTTimeNoMillis()) + .toFormatter(); + } + return bdtx; + } + + private static DateTimeFormatter basicOrdinalDate() { + if (bod == null) { + return new DateTimeFormatterBuilder() + .appendYear(4, 4) + .appendFixedDecimal(DateTimeFieldType.dayOfYear(), 3) + .toFormatter(); + } + return bod; + } + + private static DateTimeFormatter basicOrdinalDateTime() { + if (bodt == null) { + return new DateTimeFormatterBuilder() + .append(basicOrdinalDate()) + .append(basicTTime()) + .toFormatter(); + } + return bodt; + } + + private static DateTimeFormatter basicOrdinalDateTimeNoMillis() { + if (bodtx == null) { + return new DateTimeFormatterBuilder() + .append(basicOrdinalDate()) + .append(basicTTimeNoMillis()) + .toFormatter(); + } + return bodtx; + } + + private static DateTimeFormatter basicWeekDate() { + if (bwd == null) { + return new DateTimeFormatterBuilder() + .appendFixedSignedDecimal(DateTimeFieldType.weekyear(), 4) // ES change, was .appendWeekyear(4, 4) + .appendLiteral('W') + .appendFixedDecimal(DateTimeFieldType.weekOfWeekyear(), 2) + .appendFixedDecimal(DateTimeFieldType.dayOfWeek(), 1) + .toFormatter(); + } + return bwd; + } + + private static DateTimeFormatter basicWeekDateTime() { + if (bwdt == null) { + return new DateTimeFormatterBuilder() + .append(basicWeekDate()) + .append(basicTTime()) + .toFormatter(); + } + return bwdt; + } + + private static DateTimeFormatter basicWeekDateTimeNoMillis() { + if (bwdtx == null) { + return new DateTimeFormatterBuilder() + .append(basicWeekDate()) + .append(basicTTimeNoMillis()) + .toFormatter(); + } + return bwdtx; + } + + //----------------------------------------------------------------------- + private static DateTimeFormatter yearMonth() { + if (ym == null) { + return new DateTimeFormatterBuilder() + .append(yearElement()) + .append(monthElement()) + .toFormatter(); + } + return ym; + } + + private static DateTimeFormatter yearMonthDay() { + if (ymd == null) { + return new DateTimeFormatterBuilder() + .append(yearElement()) + .append(monthElement()) + .append(dayOfMonthElement()) + .toFormatter(); + } + return ymd; + } + + private static DateTimeFormatter weekyearWeek() { + if (ww == null) { + return new DateTimeFormatterBuilder() + .append(weekyearElement()) + .append(weekElement()) + .toFormatter(); + } + return ww; + } + + private static DateTimeFormatter weekyearWeekDay() { + if (wwd == null) { + return new DateTimeFormatterBuilder() + .append(weekyearElement()) + .append(weekElement()) + .append(dayOfWeekElement()) + .toFormatter(); + } + return wwd; + } + + private static DateTimeFormatter hourMinute() { + if (hm == null) { + return new DateTimeFormatterBuilder() + .append(hourElement()) + .append(minuteElement()) + .toFormatter(); + } + return hm; + } + + private static DateTimeFormatter hourMinuteSecond() { + if (hms == null) { + return new DateTimeFormatterBuilder() + .append(hourElement()) + .append(minuteElement()) + .append(secondElement()) + .toFormatter(); + } + return hms; + } + + private static DateTimeFormatter hourMinuteSecondMillis() { + if (hmsl == null) { + return new DateTimeFormatterBuilder() + .append(hourElement()) + .append(minuteElement()) + .append(secondElement()) + .appendLiteral('.') + .appendFractionOfSecond(3, 3) + .toFormatter(); + } + return hmsl; + } + + private static DateTimeFormatter hourMinuteSecondFraction() { + if (hmsf == null) { + return new DateTimeFormatterBuilder() + .append(hourElement()) + .append(minuteElement()) + .append(secondElement()) + .append(fractionElement()) + .toFormatter(); + } + return hmsf; + } + + private static DateTimeFormatter dateHour() { + if (dh == null) { + return new DateTimeFormatterBuilder() + .append(date()) + .append(literalTElement()) + .append(hour()) + .toFormatter(); + } + return dh; + } + + private static DateTimeFormatter dateHourMinute() { + if (dhm == null) { + return new DateTimeFormatterBuilder() + .append(date()) + .append(literalTElement()) + .append(hourMinute()) + .toFormatter(); + } + return dhm; + } + + private static DateTimeFormatter dateHourMinuteSecond() { + if (dhms == null) { + return new DateTimeFormatterBuilder() + .append(date()) + .append(literalTElement()) + .append(hourMinuteSecond()) + .toFormatter(); + } + return dhms; + } + + private static DateTimeFormatter dateHourMinuteSecondMillis() { + if (dhmsl == null) { + return new DateTimeFormatterBuilder() + .append(date()) + .append(literalTElement()) + .append(hourMinuteSecondMillis()) + .toFormatter(); + } + return dhmsl; + } + + private static DateTimeFormatter dateHourMinuteSecondFraction() { + if (dhmsf == null) { + return new DateTimeFormatterBuilder() + .append(date()) + .append(literalTElement()) + .append(hourMinuteSecondFraction()) + .toFormatter(); + } + return dhmsf; + } + + //----------------------------------------------------------------------- + private static DateTimeFormatter yearElement() { + if (ye == null) { + return new DateTimeFormatterBuilder() + .appendFixedSignedDecimal(DateTimeFieldType.year(), 4) // ES change, was .appendYear(4, 9) + .toFormatter(); + } + return ye; + } + + private static DateTimeFormatter monthElement() { + if (mye == null) { + return new DateTimeFormatterBuilder() + .appendLiteral('-') + .appendFixedSignedDecimal(DateTimeFieldType.monthOfYear(), 2) // ES change, was .appendMonthOfYear(2) + .toFormatter(); + } + return mye; + } + + private static DateTimeFormatter dayOfMonthElement() { + if (dme == null) { + return new DateTimeFormatterBuilder() + .appendLiteral('-') + .appendFixedSignedDecimal(DateTimeFieldType.dayOfMonth(), 2) // ES change, was .appendDayOfMonth(2) + .toFormatter(); + } + return dme; + } + + private static DateTimeFormatter weekyearElement() { + if (we == null) { + return new DateTimeFormatterBuilder() + .appendFixedSignedDecimal(DateTimeFieldType.weekyear(), 4) // ES change, was .appendWeekyear(4, 9) + .toFormatter(); + } + return we; + } + + private static DateTimeFormatter weekElement() { + if (wwe == null) { + return new DateTimeFormatterBuilder() + .appendLiteral("-W") + .appendFixedSignedDecimal(DateTimeFieldType.weekOfWeekyear(), 2) // ES change, was .appendWeekOfWeekyear(2) + .toFormatter(); + } + return wwe; + } + + private static DateTimeFormatter dayOfWeekElement() { + if (dwe == null) { + return new DateTimeFormatterBuilder() + .appendLiteral('-') + .appendDayOfWeek(1) + .toFormatter(); + } + return dwe; + } + + private static DateTimeFormatter dayOfYearElement() { + if (dye == null) { + return new DateTimeFormatterBuilder() + .appendLiteral('-') + .appendFixedSignedDecimal(DateTimeFieldType.dayOfYear(), 3) // ES change, was .appendDayOfYear(3) + .toFormatter(); + } + return dye; + } + + private static DateTimeFormatter literalTElement() { + if (lte == null) { + return new DateTimeFormatterBuilder() + .appendLiteral('T') + .toFormatter(); + } + return lte; + } + + private static DateTimeFormatter hourElement() { + if (hde == null) { + return new DateTimeFormatterBuilder() + .appendFixedSignedDecimal(DateTimeFieldType.hourOfDay(), 2) // ES change, was .appendHourOfDay(2) + .toFormatter(); + } + return hde; + } + + private static DateTimeFormatter minuteElement() { + if (mhe == null) { + return new DateTimeFormatterBuilder() + .appendLiteral(':') + .appendFixedSignedDecimal(DateTimeFieldType.minuteOfHour(), 2) // ES change, was .appendMinuteOfHour(2) + .toFormatter(); + } + return mhe; + } + + private static DateTimeFormatter secondElement() { + if (sme == null) { + return new DateTimeFormatterBuilder() + .appendLiteral(':') + .appendFixedSignedDecimal(DateTimeFieldType.secondOfMinute(), 2) // ES change, was .appendSecondOfMinute(2) + .toFormatter(); + } + return sme; + } + + private static DateTimeFormatter fractionElement() { + if (fse == null) { + return new DateTimeFormatterBuilder() + .appendLiteral('.') + // Support parsing up to nanosecond precision even though + // those extra digits will be dropped. + .appendFractionOfSecond(3, 9) + .toFormatter(); + } + return fse; + } + + private static DateTimeFormatter offsetElement() { + if (ze == null) { + return new DateTimeFormatterBuilder() + .appendTimeZoneOffset("Z", true, 2, 4) + .toFormatter(); + } + return ze; + } + + } + +} diff --git a/core/src/main/resources/org/elasticsearch/bootstrap/security.policy b/core/src/main/resources/org/elasticsearch/bootstrap/security.policy index 0434bf5be31aa..157c3cffeb07f 100644 --- a/core/src/main/resources/org/elasticsearch/bootstrap/security.policy +++ b/core/src/main/resources/org/elasticsearch/bootstrap/security.policy @@ -47,6 +47,11 @@ grant codeBase "${es.security.jar.elasticsearch.securemock}" { permission java.lang.RuntimePermission "reflectionFactoryAccess"; }; +grant codeBase "${es.security.jar.bouncycastle.bcprov}" { + // needed to allow installation of bouncycastle crypto provider + permission java.security.SecurityPermission "putProviderProperty.BC"; +}; + //// Everything else: grant { diff --git a/core/src/test/java/org/elasticsearch/NamingConventionTests.java b/core/src/test/java/org/elasticsearch/NamingConventionTests.java index db39c66f30bab..11a87671433f6 100644 --- a/core/src/test/java/org/elasticsearch/NamingConventionTests.java +++ b/core/src/test/java/org/elasticsearch/NamingConventionTests.java @@ -75,7 +75,10 @@ public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IO if (filename.endsWith(".class")) { Class clazz = loadClass(filename); if (Modifier.isAbstract(clazz.getModifiers()) == false && Modifier.isInterface(clazz.getModifiers()) == false) { - if ((clazz.getName().endsWith("Tests") || clazz.getName().endsWith("Test"))) { // don't worry about the ones that match the pattern + if (clazz.getName().endsWith("Tests") || + clazz.getName().endsWith("IT") || + clazz.getName().endsWith("Test")) { // don't worry about the ones that match the pattern + if (isTestCase(clazz) == false) { notImplementing.add(clazz); } diff --git a/core/src/test/java/org/elasticsearch/action/termvectors/GetTermVectorsTests.java b/core/src/test/java/org/elasticsearch/action/termvectors/GetTermVectorsTests.java index 12467ecef5925..a78c8b6e7a162 100644 --- a/core/src/test/java/org/elasticsearch/action/termvectors/GetTermVectorsTests.java +++ b/core/src/test/java/org/elasticsearch/action/termvectors/GetTermVectorsTests.java @@ -37,7 +37,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.index.engine.VersionConflictEngineException; -import org.elasticsearch.index.mapper.core.AbstractFieldMapper; +import org.elasticsearch.index.mapper.FieldMapper; import org.hamcrest.Matcher; import org.junit.Test; @@ -273,7 +273,7 @@ public void testRandomSingleTermVectors() throws IOException { ft.setStoreTermVectorPayloads(storePayloads); ft.setStoreTermVectorPositions(storePositions); - String optionString = AbstractFieldMapper.termVectorOptionsToString(ft); + String optionString = FieldMapper.termVectorOptionsToString(ft); XContentBuilder mapping = jsonBuilder().startObject().startObject("type1") .startObject("properties") .startObject("field") diff --git a/core/src/test/java/org/elasticsearch/action/termvectors/TermVectorsUnitTests.java b/core/src/test/java/org/elasticsearch/action/termvectors/TermVectorsUnitTests.java index a981aaff1e073..f9c4d2f39f0e3 100644 --- a/core/src/test/java/org/elasticsearch/action/termvectors/TermVectorsUnitTests.java +++ b/core/src/test/java/org/elasticsearch/action/termvectors/TermVectorsUnitTests.java @@ -37,8 +37,8 @@ import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.MapperParsingException; -import org.elasticsearch.index.mapper.core.AbstractFieldMapper; import org.elasticsearch.index.mapper.core.TypeParsers; import org.elasticsearch.index.mapper.internal.AllFieldMapper; import org.elasticsearch.rest.action.termvectors.RestTermVectorsAction; @@ -266,7 +266,7 @@ public void testFieldTypeToTermVectorString() throws Exception { ft.setStoreTermVectorPayloads(true); ft.setStoreTermVectors(true); ft.setStoreTermVectorPositions(true); - String ftOpts = AbstractFieldMapper.termVectorOptionsToString(ft); + String ftOpts = FieldMapper.termVectorOptionsToString(ft); assertThat("with_positions_payloads", equalTo(ftOpts)); AllFieldMapper.Builder builder = new AllFieldMapper.Builder(null); boolean exceptiontrown = false; @@ -285,7 +285,7 @@ public void testTermVectorStringGenerationWithoutPositions() throws Exception { ft.setStoreTermVectorPayloads(true); ft.setStoreTermVectors(true); ft.setStoreTermVectorPositions(false); - String ftOpts = AbstractFieldMapper.termVectorOptionsToString(ft); + String ftOpts = FieldMapper.termVectorOptionsToString(ft); assertThat(ftOpts, equalTo("with_offsets")); } diff --git a/core/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java b/core/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java index b5b8d63a2a90e..e127ec81511a8 100644 --- a/core/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java +++ b/core/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java @@ -19,8 +19,13 @@ package org.elasticsearch.action.update; +import org.elasticsearch.Version; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.common.io.stream.Streamable; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.index.get.GetResult; import org.elasticsearch.script.Script; import org.elasticsearch.script.ScriptService.ScriptType; import org.elasticsearch.test.ElasticsearchTestCase; @@ -28,9 +33,10 @@ import java.util.Map; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.notNullValue; -import static org.hamcrest.Matchers.nullValue; +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.hamcrest.Matchers.*; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; public class UpdateRequestTests extends ElasticsearchTestCase { @@ -119,4 +125,45 @@ public void testUpdateRequest() throws Exception { assertThat(doc.get("field1").toString(), equalTo("value1")); assertThat(((Map) doc.get("compound")).get("field2").toString(), equalTo("value2")); } + + @Test // Related to issue 3256 + public void testUpdateRequestWithTTL() throws Exception { + long providedTTLValue = randomIntBetween(500, 1000); + Settings settings = settings(Version.CURRENT).build(); + + UpdateHelper updateHelper = new UpdateHelper(settings, null); + + // We just upsert one document with ttl + IndexRequest indexRequest = new IndexRequest("test", "type1", "1") + .source(jsonBuilder().startObject().field("foo", "bar").endObject()) + .ttl(providedTTLValue); + UpdateRequest updateRequest = new UpdateRequest("test", "type1", "1") + .doc(jsonBuilder().startObject().field("fooz", "baz").endObject()) + .upsert(indexRequest); + + // We simulate that the document is not existing yet + GetResult getResult = new GetResult("test", "type1", "1", 0, false, null, null); + UpdateHelper.Result result = updateHelper.prepare(updateRequest, getResult); + Streamable action = result.action(); + assertThat(action, instanceOf(IndexRequest.class)); + IndexRequest indexAction = (IndexRequest) action; + assertThat(indexAction.ttl(), is(providedTTLValue)); + + // We just upsert one document with ttl using a script + indexRequest = new IndexRequest("test", "type1", "2") + .source(jsonBuilder().startObject().field("foo", "bar").endObject()) + .ttl(providedTTLValue); + updateRequest = new UpdateRequest("test", "type1", "2") + .upsert(indexRequest) + .script(new Script(";")) + .scriptedUpsert(true); + + // We simulate that the document is not existing yet + getResult = new GetResult("test", "type1", "2", 0, false, null, null); + result = updateHelper.prepare(updateRequest, getResult); + action = result.action(); + assertThat(action, instanceOf(IndexRequest.class)); + indexAction = (IndexRequest) action; + assertThat(indexAction.ttl(), is(providedTTLValue)); + } } diff --git a/core/src/test/java/org/elasticsearch/bootstrap/JarHellTests.java b/core/src/test/java/org/elasticsearch/bootstrap/JarHellTests.java new file mode 100644 index 0000000000000..d87a7e295f185 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/bootstrap/JarHellTests.java @@ -0,0 +1,206 @@ +/* + * 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.bootstrap; + +import org.elasticsearch.test.ElasticsearchTestCase; + +import java.io.ByteArrayOutputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; +import java.util.jar.Attributes; +import java.util.jar.JarOutputStream; +import java.util.jar.Manifest; +import java.util.zip.ZipEntry; +import java.util.zip.ZipOutputStream; + +public class JarHellTests extends ElasticsearchTestCase { + + URL makeJar(Path dir, String name, Manifest manifest, String... files) throws IOException { + Path jarpath = dir.resolve(name); + ZipOutputStream out; + if (manifest == null) { + out = new JarOutputStream(Files.newOutputStream(jarpath, StandardOpenOption.CREATE)); + } else { + out = new JarOutputStream(Files.newOutputStream(jarpath, StandardOpenOption.CREATE), manifest); + } + for (String file : files) { + out.putNextEntry(new ZipEntry(file)); + } + out.close(); + return jarpath.toUri().toURL(); + } + + URL makeFile(Path dir, String name) throws IOException { + Path filepath = dir.resolve(name); + Files.newOutputStream(filepath, StandardOpenOption.CREATE).close(); + return filepath.toUri().toURL(); + } + + public void testDifferentJars() throws Exception { + Path dir = createTempDir(); + URL[] jars = {makeJar(dir, "foo.jar", null, "DuplicateClass.class"), makeJar(dir, "bar.jar", null, "DuplicateClass.class")}; + try { + JarHell.checkJarHell(jars); + } catch (IllegalStateException e) { + assertTrue(e.getMessage().contains("jar hell!")); + assertTrue(e.getMessage().contains("DuplicateClass")); + assertTrue(e.getMessage().contains("foo.jar")); + assertTrue(e.getMessage().contains("bar.jar")); + } + } + + public void testBootclasspathLeniency() throws Exception { + Path dir = createTempDir(); + String previousJavaHome = System.getProperty("java.home"); + System.setProperty("java.home", dir.toString()); + URL[] jars = {makeJar(dir, "foo.jar", null, "DuplicateClass.class"), makeJar(dir, "bar.jar", null, "DuplicateClass.class")}; + try { + JarHell.checkJarHell(jars); + } finally { + System.setProperty("java.home", previousJavaHome); + } + } + + public void testDuplicateClasspathLeniency() throws Exception { + Path dir = createTempDir(); + URL jar = makeJar(dir, "foo.jar", null, "Foo.class"); + URL[] jars = {jar, jar}; + JarHell.checkJarHell(jars); + } + + public void testDirsOnClasspath() throws Exception { + Path dir1 = createTempDir(); + Path dir2 = createTempDir(); + URL[] dirs = {makeFile(dir1, "DuplicateClass.class"), makeFile(dir2, "DuplicateClass.class")}; + try { + JarHell.checkJarHell(dirs); + } catch (IllegalStateException e) { + assertTrue(e.getMessage().contains("jar hell!")); + assertTrue(e.getMessage().contains("DuplicateClass")); + assertTrue(e.getMessage().contains(dir1.toString())); + assertTrue(e.getMessage().contains(dir2.toString())); + } + } + + public void testDirAndJar() throws Exception { + Path dir1 = createTempDir(); + Path dir2 = createTempDir(); + URL[] dirs = {makeJar(dir1, "foo.jar", null, "DuplicateClass.class"), makeFile(dir2, "DuplicateClass.class")}; + try { + JarHell.checkJarHell(dirs); + } catch (IllegalStateException e) { + assertTrue(e.getMessage().contains("jar hell!")); + assertTrue(e.getMessage().contains("DuplicateClass")); + assertTrue(e.getMessage().contains("foo.jar")); + assertTrue(e.getMessage().contains(dir2.toString())); + } + } + + public void testLog4jLeniency() throws Exception { + Path dir = createTempDir(); + URL[] jars = {makeJar(dir, "foo.jar", null, "org/apache/log4j/DuplicateClass.class"), makeJar(dir, "bar.jar", null, "org/apache/log4j/DuplicateClass.class")}; + JarHell.checkJarHell(jars); + } + + public void testBaseDateTimeLeniency() throws Exception { + Path dir = createTempDir(); + URL[] jars = {makeJar(dir, "foo.jar", null, "org/joda/time/base/BaseDateTime.class"), makeJar(dir, "bar.jar", null, "org/joda/time/base/BaseDateTime.class")}; + JarHell.checkJarHell(jars); + } + + public void testWithinSingleJar() throws Exception { + // the java api for zip file does not allow creating duplicate entries (good!) so + // this bogus jar had to be constructed with ant + URL[] jars = {JarHellTests.class.getResource("duplicate-classes.jar")}; + try { + JarHell.checkJarHell(jars); + } catch (IllegalStateException e) { + assertTrue(e.getMessage().contains("jar hell!")); + assertTrue(e.getMessage().contains("DuplicateClass")); + assertTrue(e.getMessage().contains("duplicate-classes.jar")); + assertTrue(e.getMessage().contains("exists multiple times in jar")); + } + } + + public void testXmlBeansLeniency() throws Exception { + URL[] jars = {JarHellTests.class.getResource("duplicate-xmlbeans-classes.jar")}; + JarHell.checkJarHell(jars); + } + + public void testRequiredJDKVersionTooOld() throws Exception { + Path dir = createTempDir(); + String previousJavaVersion = System.getProperty("java.specification.version"); + System.setProperty("java.specification.version", "1.7"); + + Manifest manifest = new Manifest(); + manifest.getMainAttributes().put(new Attributes.Name("X-Compile-Target-JDK"), "1.8"); + URL[] jars = {makeJar(dir, "foo.jar", manifest, "Foo.class")}; + try { + JarHell.checkJarHell(jars); + } catch (IllegalStateException e) { + assertTrue(e.getMessage().contains("requires java 1.8")); + assertTrue(e.getMessage().contains("your system: 1.7")); + } finally { + System.setProperty("java.specification.version", previousJavaVersion); + } + } + + public void testRequiredJDKVersionIsOK() throws Exception { + Path dir = createTempDir(); + String previousJavaVersion = System.getProperty("java.specification.version"); + System.setProperty("java.specification.version", "1.7"); + + Manifest manifest = new Manifest(); + manifest.getMainAttributes().put(new Attributes.Name("X-Compile-Target-JDK"), "1.7"); + URL[] jars = {makeJar(dir, "foo.jar", manifest, "Foo.class")}; + try { + JarHell.checkJarHell(jars); + } finally { + System.setProperty("java.specification.version", previousJavaVersion); + } + } + + public void testBadJDKVersionProperty() throws Exception { + Path dir = createTempDir(); + String previousJavaVersion = System.getProperty("java.specification.version"); + System.setProperty("java.specification.version", "bogus"); + + Manifest manifest = new Manifest(); + manifest.getMainAttributes().put(new Attributes.Name("X-Compile-Target-JDK"), "1.7"); + URL[] jars = {makeJar(dir, "foo.jar", manifest, "Foo.class")}; + try { + JarHell.checkJarHell(jars); + } finally { + System.setProperty("java.specification.version", previousJavaVersion); + } + } + + public void testBadJDKVersionInJar() throws Exception { + Path dir = createTempDir(); + Manifest manifest = new Manifest(); + manifest.getMainAttributes().put(new Attributes.Name("X-Compile-Target-JDK"), "bogus"); + URL[] jars = {makeJar(dir, "foo.jar", manifest, "Foo.class")}; + JarHell.checkJarHell(jars); + } +} diff --git a/core/src/test/java/org/elasticsearch/bootstrap/SecurityTests.java b/core/src/test/java/org/elasticsearch/bootstrap/SecurityTests.java index 980e9dca3419b..51d08d05bf73a 100644 --- a/core/src/test/java/org/elasticsearch/bootstrap/SecurityTests.java +++ b/core/src/test/java/org/elasticsearch/bootstrap/SecurityTests.java @@ -204,4 +204,28 @@ public void testProcessExecution() throws Exception { fail("didn't get expected exception"); } catch (SecurityException expected) {} } + + /** When a configured dir is a symlink, test that permissions work on link target */ + public void testSymlinkPermissions() throws IOException { + Path dir = createTempDir(); + + Path target = dir.resolve("target"); + Files.createDirectory(target); + + // symlink + Path link = dir.resolve("link"); + try { + Files.createSymbolicLink(link, target); + } catch (UnsupportedOperationException | IOException e) { + assumeNoException("test requires filesystem that supports symbolic links", e); + } catch (SecurityException e) { + assumeNoException("test cannot create symbolic links with security manager enabled", e); + } + Permissions permissions = new Permissions(); + Security.addPath(permissions, link, "read"); + assertTrue(permissions.implies(new FilePermission(link.toString(), "read"))); + assertTrue(permissions.implies(new FilePermission(link.resolve("foo").toString(), "read"))); + assertTrue(permissions.implies(new FilePermission(target.toString(), "read"))); + assertTrue(permissions.implies(new FilePermission(target.resolve("foo").toString(), "read"))); + } } diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/MockDiskUsagesTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/MockDiskUsagesTests.java index db4bb5765522e..1db310c477087 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/MockDiskUsagesTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/MockDiskUsagesTests.java @@ -32,7 +32,7 @@ import org.elasticsearch.cluster.routing.RoutingNode; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.monitor.fs.FsStats; +import org.elasticsearch.monitor.fs.FsInfo; import org.elasticsearch.node.settings.NodeSettingsService; import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.elasticsearch.threadpool.ThreadPool; @@ -141,15 +141,15 @@ public void run() { /** Create a fake NodeStats for the given node and usage */ public static NodeStats makeStats(String nodeName, DiskUsage usage) { - FsStats.Info[] infos = new FsStats.Info[1]; - FsStats.Info info = new FsStats.Info("/path.data", null, null, - usage.getTotalBytes(), usage.getFreeBytes(), usage.getFreeBytes(), -1, -1, -1, -1, -1, -1); - infos[0] = info; - FsStats fsStats = new FsStats(System.currentTimeMillis(), infos); + FsInfo.Path[] paths = new FsInfo.Path[1]; + FsInfo.Path path = new FsInfo.Path("/path.data", null, + usage.getTotalBytes(), usage.getFreeBytes(), usage.getFreeBytes()); + paths[0] = path; + FsInfo fsInfo = new FsInfo(System.currentTimeMillis(), paths); return new NodeStats(new DiscoveryNode(nodeName, null, Version.V_2_0_0), System.currentTimeMillis(), - null, null, null, null, null, null, - fsStats, + null, null, null, null, null, + fsInfo, null, null, null); } diff --git a/core/src/test/java/org/elasticsearch/deps/joda/SimpleJodaTests.java b/core/src/test/java/org/elasticsearch/deps/joda/SimpleJodaTests.java index 748573db00775..21c16d81b4aa2 100644 --- a/core/src/test/java/org/elasticsearch/deps/joda/SimpleJodaTests.java +++ b/core/src/test/java/org/elasticsearch/deps/joda/SimpleJodaTests.java @@ -22,8 +22,11 @@ import org.elasticsearch.common.joda.FormatDateTimeFormatter; import org.elasticsearch.common.joda.Joda; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.index.mapper.core.DateFieldMapper; +import org.elasticsearch.index.mapper.object.RootObjectMapper; import org.elasticsearch.test.ElasticsearchTestCase; import org.joda.time.DateTime; +import org.joda.time.DateTimeFieldType; import org.joda.time.DateTimeZone; import org.joda.time.LocalDateTime; import org.joda.time.MutableDateTime; @@ -361,6 +364,368 @@ public void testThatEpochParserIsIdempotent() { assertThat(secondsDateTime.getMillis(), is(1234567890000l)); } + public void testThatDefaultFormatterChecksForCorrectYearLength() throws Exception { + // if no strict version is tested, this means the date format is already strict by itself + // yyyyMMdd + assertValidDateFormatParsing("basicDate", "20140303"); + assertDateFormatParsingThrowingException("basicDate", "2010303"); + + // yyyyMMdd’T'HHmmss.SSSZ + assertValidDateFormatParsing("basicDateTime", "20140303T124343.123Z"); + assertValidDateFormatParsing("basicDateTime", "00050303T124343.123Z"); + assertDateFormatParsingThrowingException("basicDateTime", "50303T124343.123Z"); + + // yyyyMMdd’T'HHmmssZ + assertValidDateFormatParsing("basicDateTimeNoMillis", "20140303T124343Z"); + assertValidDateFormatParsing("basicDateTimeNoMillis", "00050303T124343Z"); + assertDateFormatParsingThrowingException("basicDateTimeNoMillis", "50303T124343Z"); + + // yyyyDDD + assertValidDateFormatParsing("basicOrdinalDate", "0005165"); + assertDateFormatParsingThrowingException("basicOrdinalDate", "5165"); + + // yyyyDDD’T'HHmmss.SSSZ + assertValidDateFormatParsing("basicOrdinalDateTime", "0005165T124343.123Z"); + assertValidDateFormatParsing("basicOrdinalDateTime", "0005165T124343.123Z"); + assertDateFormatParsingThrowingException("basicOrdinalDateTime", "5165T124343.123Z"); + + // yyyyDDD’T'HHmmssZ + assertValidDateFormatParsing("basicOrdinalDateTimeNoMillis", "0005165T124343Z"); + assertValidDateFormatParsing("basicOrdinalDateTimeNoMillis", "0005165T124343Z"); + assertDateFormatParsingThrowingException("basicOrdinalDateTimeNoMillis", "5165T124343Z"); + + // HHmmss.SSSZ + assertValidDateFormatParsing("basicTime", "090909.123Z"); + assertDateFormatParsingThrowingException("basicTime", "90909.123Z"); + + // HHmmssZ + assertValidDateFormatParsing("basicTimeNoMillis", "090909Z"); + assertDateFormatParsingThrowingException("basicTimeNoMillis", "90909Z"); + + // 'T’HHmmss.SSSZ + assertValidDateFormatParsing("basicTTime", "T090909.123Z"); + assertDateFormatParsingThrowingException("basicTTime", "T90909.123Z"); + + // T’HHmmssZ + assertValidDateFormatParsing("basicTTimeNoMillis", "T090909Z"); + assertDateFormatParsingThrowingException("basicTTimeNoMillis", "T90909Z"); + + // xxxx’W'wwe + assertValidDateFormatParsing("basicWeekDate", "0005W414"); + assertValidDateFormatParsing("basicWeekDate", "5W414", "0005W414"); + assertDateFormatParsingThrowingException("basicWeekDate", "5W14"); + + assertValidDateFormatParsing("strictBasicWeekDate", "0005W414"); + assertDateFormatParsingThrowingException("strictBasicWeekDate", "0005W47"); + assertDateFormatParsingThrowingException("strictBasicWeekDate", "5W414"); + assertDateFormatParsingThrowingException("strictBasicWeekDate", "5W14"); + + // xxxx’W'wwe’T'HHmmss.SSSZ + assertValidDateFormatParsing("basicWeekDateTime", "0005W414T124343.123Z"); + assertValidDateFormatParsing("basicWeekDateTime", "5W414T124343.123Z", "0005W414T124343.123Z"); + assertDateFormatParsingThrowingException("basicWeekDateTime", "5W14T124343.123Z"); + + assertValidDateFormatParsing("strictBasicWeekDateTime", "0005W414T124343.123Z"); + assertDateFormatParsingThrowingException("strictBasicWeekDateTime", "0005W47T124343.123Z"); + assertDateFormatParsingThrowingException("strictBasicWeekDateTime", "5W414T124343.123Z"); + assertDateFormatParsingThrowingException("strictBasicWeekDateTime", "5W14T124343.123Z"); + + // xxxx’W'wwe’T'HHmmssZ + assertValidDateFormatParsing("basicWeekDateTimeNoMillis", "0005W414T124343Z"); + assertValidDateFormatParsing("basicWeekDateTimeNoMillis", "5W414T124343Z", "0005W414T124343Z"); + assertDateFormatParsingThrowingException("basicWeekDateTimeNoMillis", "5W14T124343Z"); + + assertValidDateFormatParsing("strictBasicWeekDateTimeNoMillis", "0005W414T124343Z"); + assertDateFormatParsingThrowingException("strictBasicWeekDateTimeNoMillis", "0005W47T124343Z"); + assertDateFormatParsingThrowingException("strictBasicWeekDateTimeNoMillis", "5W414T124343Z"); + assertDateFormatParsingThrowingException("strictBasicWeekDateTimeNoMillis", "5W14T124343Z"); + + // yyyy-MM-dd + assertValidDateFormatParsing("date", "0005-06-03"); + assertValidDateFormatParsing("date", "5-6-3", "0005-06-03"); + + assertValidDateFormatParsing("strictDate", "0005-06-03"); + assertDateFormatParsingThrowingException("strictDate", "5-6-3"); + assertDateFormatParsingThrowingException("strictDate", "0005-06-3"); + assertDateFormatParsingThrowingException("strictDate", "0005-6-03"); + assertDateFormatParsingThrowingException("strictDate", "5-06-03"); + + // yyyy-MM-dd'T'HH + assertValidDateFormatParsing("dateHour", "0005-06-03T12"); + assertValidDateFormatParsing("dateHour", "5-6-3T1", "0005-06-03T01"); + + assertValidDateFormatParsing("strictDateHour", "0005-06-03T12"); + assertDateFormatParsingThrowingException("strictDateHour", "5-6-3T1"); + + // yyyy-MM-dd'T'HH:mm + assertValidDateFormatParsing("dateHourMinute", "0005-06-03T12:12"); + assertValidDateFormatParsing("dateHourMinute", "5-6-3T12:1", "0005-06-03T12:01"); + + assertValidDateFormatParsing("strictDateHourMinute", "0005-06-03T12:12"); + assertDateFormatParsingThrowingException("strictDateHourMinute", "5-6-3T12:1"); + + // yyyy-MM-dd'T'HH:mm:ss + assertValidDateFormatParsing("dateHourMinuteSecond", "0005-06-03T12:12:12"); + assertValidDateFormatParsing("dateHourMinuteSecond", "5-6-3T12:12:1", "0005-06-03T12:12:01"); + + assertValidDateFormatParsing("strictDateHourMinuteSecond", "0005-06-03T12:12:12"); + assertDateFormatParsingThrowingException("strictDateHourMinuteSecond", "5-6-3T12:12:1"); + + // yyyy-MM-dd’T'HH:mm:ss.SSS + assertValidDateFormatParsing("dateHourMinuteSecondFraction", "0005-06-03T12:12:12.123"); + assertValidDateFormatParsing("dateHourMinuteSecondFraction", "5-6-3T12:12:1.123", "0005-06-03T12:12:01.123"); + assertValidDateFormatParsing("dateHourMinuteSecondFraction", "5-6-3T12:12:1.1", "0005-06-03T12:12:01.100"); + + assertValidDateFormatParsing("strictDateHourMinuteSecondFraction", "0005-06-03T12:12:12.123"); + assertDateFormatParsingThrowingException("strictDateHourMinuteSecondFraction", "5-6-3T12:12:12.1"); + assertDateFormatParsingThrowingException("strictDateHourMinuteSecondFraction", "5-6-3T12:12:12.12"); + + assertValidDateFormatParsing("dateHourMinuteSecondMillis", "0005-06-03T12:12:12.123"); + assertValidDateFormatParsing("dateHourMinuteSecondMillis", "5-6-3T12:12:1.123", "0005-06-03T12:12:01.123"); + assertValidDateFormatParsing("dateHourMinuteSecondMillis", "5-6-3T12:12:1.1", "0005-06-03T12:12:01.100"); + + assertValidDateFormatParsing("strictDateHourMinuteSecondMillis", "0005-06-03T12:12:12.123"); + assertDateFormatParsingThrowingException("strictDateHourMinuteSecondMillis", "5-6-3T12:12:12.1"); + assertDateFormatParsingThrowingException("strictDateHourMinuteSecondMillis", "5-6-3T12:12:12.12"); + + // yyyy-MM-dd'T'HH:mm:ss.SSSZ + assertValidDateFormatParsing("dateOptionalTime", "2014-03-03", "2014-03-03T00:00:00.000Z"); + assertValidDateFormatParsing("dateOptionalTime", "1257-3-03", "1257-03-03T00:00:00.000Z"); + assertValidDateFormatParsing("dateOptionalTime", "0005-03-3", "0005-03-03T00:00:00.000Z"); + assertValidDateFormatParsing("dateOptionalTime", "5-03-03", "0005-03-03T00:00:00.000Z"); + assertValidDateFormatParsing("dateOptionalTime", "5-03-03T1:1:1.1", "0005-03-03T01:01:01.100Z"); + assertValidDateFormatParsing("strictDateOptionalTime", "2014-03-03", "2014-03-03T00:00:00.000Z"); + assertDateFormatParsingThrowingException("strictDateOptionalTime", "5-03-03"); + assertDateFormatParsingThrowingException("strictDateOptionalTime", "0005-3-03"); + assertDateFormatParsingThrowingException("strictDateOptionalTime", "0005-03-3"); + assertDateFormatParsingThrowingException("strictDateOptionalTime", "5-03-03T1:1:1.1"); + assertDateFormatParsingThrowingException("strictDateOptionalTime", "5-03-03T01:01:01.1"); + assertDateFormatParsingThrowingException("strictDateOptionalTime", "5-03-03T01:01:1.100"); + assertDateFormatParsingThrowingException("strictDateOptionalTime", "5-03-03T01:1:01.100"); + assertDateFormatParsingThrowingException("strictDateOptionalTime", "5-03-03T1:01:01.100"); + + // yyyy-MM-dd’T'HH:mm:ss.SSSZZ + assertValidDateFormatParsing("dateTime", "5-03-03T1:1:1.1Z", "0005-03-03T01:01:01.100Z"); + assertValidDateFormatParsing("strictDateTime", "2014-03-03T11:11:11.100Z", "2014-03-03T11:11:11.100Z"); + assertDateFormatParsingThrowingException("strictDateTime", "0005-03-03T1:1:1.1Z"); + assertDateFormatParsingThrowingException("strictDateTime", "0005-03-03T01:01:1.100Z"); + assertDateFormatParsingThrowingException("strictDateTime", "0005-03-03T01:1:01.100Z"); + assertDateFormatParsingThrowingException("strictDateTime", "0005-03-03T1:01:01.100Z"); + + // yyyy-MM-dd’T'HH:mm:ssZZ + assertValidDateFormatParsing("dateTimeNoMillis", "5-03-03T1:1:1Z", "0005-03-03T01:01:01Z"); + assertValidDateFormatParsing("strictDateTimeNoMillis", "2014-03-03T11:11:11Z", "2014-03-03T11:11:11Z"); + assertDateFormatParsingThrowingException("strictDateTimeNoMillis", "0005-03-03T1:1:1Z"); + assertDateFormatParsingThrowingException("strictDateTimeNoMillis", "0005-03-03T01:01:1Z"); + assertDateFormatParsingThrowingException("strictDateTimeNoMillis", "0005-03-03T01:1:01Z"); + assertDateFormatParsingThrowingException("strictDateTimeNoMillis", "0005-03-03T1:01:01Z"); + + // HH + assertValidDateFormatParsing("hour", "12"); + assertValidDateFormatParsing("hour", "1", "01"); + assertValidDateFormatParsing("strictHour", "12"); + assertValidDateFormatParsing("strictHour", "01"); + assertDateFormatParsingThrowingException("strictHour", "1"); + + // HH:mm + assertValidDateFormatParsing("hourMinute", "12:12"); + assertValidDateFormatParsing("hourMinute", "12:1", "12:01"); + assertValidDateFormatParsing("strictHourMinute", "12:12"); + assertValidDateFormatParsing("strictHourMinute", "12:01"); + assertDateFormatParsingThrowingException("strictHourMinute", "12:1"); + + // HH:mm:ss + assertValidDateFormatParsing("hourMinuteSecond", "12:12:12"); + assertValidDateFormatParsing("hourMinuteSecond", "12:12:1", "12:12:01"); + assertValidDateFormatParsing("strictHourMinuteSecond", "12:12:12"); + assertValidDateFormatParsing("strictHourMinuteSecond", "12:12:01"); + assertDateFormatParsingThrowingException("strictHourMinuteSecond", "12:12:1"); + + // HH:mm:ss.SSS + assertValidDateFormatParsing("hourMinuteSecondFraction", "12:12:12.123"); + assertValidDateFormatParsing("hourMinuteSecondFraction", "12:12:12.1", "12:12:12.100"); + assertValidDateFormatParsing("strictHourMinuteSecondFraction", "12:12:12.123"); + assertValidDateFormatParsing("strictHourMinuteSecondFraction", "12:12:12.1", "12:12:12.100"); + + assertValidDateFormatParsing("hourMinuteSecondMillis", "12:12:12.123"); + assertValidDateFormatParsing("hourMinuteSecondMillis", "12:12:12.1", "12:12:12.100"); + assertValidDateFormatParsing("strictHourMinuteSecondMillis", "12:12:12.123"); + assertValidDateFormatParsing("strictHourMinuteSecondMillis", "12:12:12.1", "12:12:12.100"); + + // yyyy-DDD + assertValidDateFormatParsing("ordinalDate", "5-3", "0005-003"); + assertValidDateFormatParsing("strictOrdinalDate", "0005-003"); + assertDateFormatParsingThrowingException("strictOrdinalDate", "5-3"); + assertDateFormatParsingThrowingException("strictOrdinalDate", "0005-3"); + assertDateFormatParsingThrowingException("strictOrdinalDate", "5-003"); + + // yyyy-DDD’T'HH:mm:ss.SSSZZ + assertValidDateFormatParsing("ordinalDateTime", "5-3T12:12:12.100Z", "0005-003T12:12:12.100Z"); + assertValidDateFormatParsing("strictOrdinalDateTime", "0005-003T12:12:12.100Z"); + assertDateFormatParsingThrowingException("strictOrdinalDateTime", "5-3T1:12:12.123Z"); + assertDateFormatParsingThrowingException("strictOrdinalDateTime", "5-3T12:1:12.123Z"); + assertDateFormatParsingThrowingException("strictOrdinalDateTime", "5-3T12:12:1.123Z"); + + // yyyy-DDD’T'HH:mm:ssZZ + assertValidDateFormatParsing("ordinalDateTimeNoMillis", "5-3T12:12:12Z", "0005-003T12:12:12Z"); + assertValidDateFormatParsing("strictOrdinalDateTimeNoMillis", "0005-003T12:12:12Z"); + assertDateFormatParsingThrowingException("strictOrdinalDateTimeNoMillis", "5-3T1:12:12Z"); + assertDateFormatParsingThrowingException("strictOrdinalDateTimeNoMillis", "5-3T12:1:12Z"); + assertDateFormatParsingThrowingException("strictOrdinalDateTimeNoMillis", "5-3T12:12:1Z"); + + + // HH:mm:ss.SSSZZ + assertValidDateFormatParsing("time", "12:12:12.100Z"); + assertValidDateFormatParsing("time", "01:01:01.1Z", "01:01:01.100Z"); + assertValidDateFormatParsing("time", "1:1:1.1Z", "01:01:01.100Z"); + assertValidDateFormatParsing("strictTime", "12:12:12.100Z"); + assertDateFormatParsingThrowingException("strictTime", "12:12:1.100Z"); + assertDateFormatParsingThrowingException("strictTime", "12:1:12.100Z"); + assertDateFormatParsingThrowingException("strictTime", "1:12:12.100Z"); + + // HH:mm:ssZZ + assertValidDateFormatParsing("timeNoMillis", "12:12:12Z"); + assertValidDateFormatParsing("timeNoMillis", "01:01:01Z", "01:01:01Z"); + assertValidDateFormatParsing("timeNoMillis", "1:1:1Z", "01:01:01Z"); + assertValidDateFormatParsing("strictTimeNoMillis", "12:12:12Z"); + assertDateFormatParsingThrowingException("strictTimeNoMillis", "12:12:1Z"); + assertDateFormatParsingThrowingException("strictTimeNoMillis", "12:1:12Z"); + assertDateFormatParsingThrowingException("strictTimeNoMillis", "1:12:12Z"); + + // 'T’HH:mm:ss.SSSZZ + assertValidDateFormatParsing("tTime", "T12:12:12.100Z"); + assertValidDateFormatParsing("tTime", "T01:01:01.1Z", "T01:01:01.100Z"); + assertValidDateFormatParsing("tTime", "T1:1:1.1Z", "T01:01:01.100Z"); + assertValidDateFormatParsing("strictTTime", "T12:12:12.100Z"); + assertDateFormatParsingThrowingException("strictTTime", "T12:12:1.100Z"); + assertDateFormatParsingThrowingException("strictTTime", "T12:1:12.100Z"); + assertDateFormatParsingThrowingException("strictTTime", "T1:12:12.100Z"); + + // 'T’HH:mm:ssZZ + assertValidDateFormatParsing("tTimeNoMillis", "T12:12:12Z"); + assertValidDateFormatParsing("tTimeNoMillis", "T01:01:01Z", "T01:01:01Z"); + assertValidDateFormatParsing("tTimeNoMillis", "T1:1:1Z", "T01:01:01Z"); + assertValidDateFormatParsing("strictTTimeNoMillis", "T12:12:12Z"); + assertDateFormatParsingThrowingException("strictTTimeNoMillis", "T12:12:1Z"); + assertDateFormatParsingThrowingException("strictTTimeNoMillis", "T12:1:12Z"); + assertDateFormatParsingThrowingException("strictTTimeNoMillis", "T1:12:12Z"); + + // xxxx-'W’ww-e + assertValidDateFormatParsing("weekDate", "0005-W4-1", "0005-W04-1"); + assertValidDateFormatParsing("strictWeekDate", "0005-W04-1"); + assertDateFormatParsingThrowingException("strictWeekDate", "0005-W4-1"); + + // xxxx-'W’ww-e’T'HH:mm:ss.SSSZZ + assertValidDateFormatParsing("weekDateTime", "0005-W41-4T12:43:43.123Z"); + assertValidDateFormatParsing("weekDateTime", "5-W41-4T12:43:43.123Z", "0005-W41-4T12:43:43.123Z"); + assertValidDateFormatParsing("strictWeekDateTime", "0005-W41-4T12:43:43.123Z"); + assertValidDateFormatParsing("strictWeekDateTime", "0005-W06-4T12:43:43.123Z"); + assertDateFormatParsingThrowingException("strictWeekDateTime", "0005-W4-7T12:43:43.123Z"); + assertDateFormatParsingThrowingException("strictWeekDateTime", "5-W41-4T12:43:43.123Z"); + assertDateFormatParsingThrowingException("strictWeekDateTime", "5-W1-4T12:43:43.123Z"); + + // xxxx-'W’ww-e’T'HH:mm:ssZZ + assertValidDateFormatParsing("weekDateTimeNoMillis", "0005-W41-4T12:43:43Z"); + assertValidDateFormatParsing("weekDateTimeNoMillis", "5-W41-4T12:43:43Z", "0005-W41-4T12:43:43Z"); + assertValidDateFormatParsing("strictWeekDateTimeNoMillis", "0005-W41-4T12:43:43Z"); + assertValidDateFormatParsing("strictWeekDateTimeNoMillis", "0005-W06-4T12:43:43Z"); + assertDateFormatParsingThrowingException("strictWeekDateTimeNoMillis", "0005-W4-7T12:43:43Z"); + assertDateFormatParsingThrowingException("strictWeekDateTimeNoMillis", "5-W41-4T12:43:43Z"); + assertDateFormatParsingThrowingException("strictWeekDateTimeNoMillis", "5-W1-4T12:43:43Z"); + + // yyyy + assertValidDateFormatParsing("weekyear", "2014"); + assertValidDateFormatParsing("weekyear", "5", "0005"); + assertValidDateFormatParsing("weekyear", "0005"); + assertValidDateFormatParsing("strictWeekyear", "2014"); + assertValidDateFormatParsing("strictWeekyear", "0005"); + assertDateFormatParsingThrowingException("strictWeekyear", "5"); + + // yyyy-'W'ee + assertValidDateFormatParsing("weekyearWeek", "2014-W41"); + assertValidDateFormatParsing("weekyearWeek", "2014-W1", "2014-W01"); + assertValidDateFormatParsing("strictWeekyearWeek", "2014-W41"); + assertDateFormatParsingThrowingException("strictWeekyearWeek", "2014-W1"); + + // weekyearWeekDay + assertValidDateFormatParsing("weekyearWeekDay", "2014-W41-1"); + assertValidDateFormatParsing("weekyearWeekDay", "2014-W1-1", "2014-W01-1"); + assertValidDateFormatParsing("strictWeekyearWeekDay", "2014-W41-1"); + assertDateFormatParsingThrowingException("strictWeekyearWeekDay", "2014-W1-1"); + + // yyyy + assertValidDateFormatParsing("year", "2014"); + assertValidDateFormatParsing("year", "5", "0005"); + assertValidDateFormatParsing("strictYear", "2014"); + assertDateFormatParsingThrowingException("strictYear", "5"); + + // yyyy-mm + assertValidDateFormatParsing("yearMonth", "2014-12"); + assertValidDateFormatParsing("yearMonth", "2014-5", "2014-05"); + assertValidDateFormatParsing("strictYearMonth", "2014-12"); + assertDateFormatParsingThrowingException("strictYearMonth", "2014-5"); + + // yyyy-mm-dd + assertValidDateFormatParsing("yearMonthDay", "2014-12-12"); + assertValidDateFormatParsing("yearMonthDay", "2014-05-5", "2014-05-05"); + assertValidDateFormatParsing("strictYearMonthDay", "2014-12-12"); + assertDateFormatParsingThrowingException("strictYearMonthDay", "2014-05-5"); + } + + @Test + public void testThatRootObjectParsingIsStrict() throws Exception { + String[] datesThatWork = new String[] { "2014/10/10", "2014/10/10 12:12:12", "2014-05-05", "2014-05-05T12:12:12.123Z" }; + String[] datesThatShouldNotWork = new String[]{ "5-05-05", "2014-5-05", "2014-05-5", + "2014-05-05T1:12:12.123Z", "2014-05-05T12:1:12.123Z", "2014-05-05T12:12:1.123Z", + "4/10/10", "2014/1/10", "2014/10/1", + "2014/10/10 1:12:12", "2014/10/10 12:1:12", "2014/10/10 12:12:1" + }; + + // good case + for (String date : datesThatWork) { + boolean dateParsingSuccessful = false; + for (FormatDateTimeFormatter dateTimeFormatter : RootObjectMapper.Defaults.DYNAMIC_DATE_TIME_FORMATTERS) { + try { + dateTimeFormatter.parser().parseMillis(date); + dateParsingSuccessful = true; + break; + } catch (Exception e) {} + } + if (!dateParsingSuccessful) { + fail("Parsing for date " + date + " in root object mapper failed, but shouldnt"); + } + } + + // bad case + for (String date : datesThatShouldNotWork) { + for (FormatDateTimeFormatter dateTimeFormatter : RootObjectMapper.Defaults.DYNAMIC_DATE_TIME_FORMATTERS) { + try { + dateTimeFormatter.parser().parseMillis(date); + fail(String.format(Locale.ROOT, "Expected exception when parsing date %s in root mapper", date)); + } catch (Exception e) {} + } + } + } + + private void assertValidDateFormatParsing(String pattern, String dateToParse) { + assertValidDateFormatParsing(pattern, dateToParse, dateToParse); + } + + private void assertValidDateFormatParsing(String pattern, String dateToParse, String expectedDate) { + FormatDateTimeFormatter formatter = Joda.forPattern(pattern); + assertThat(formatter.printer().print(formatter.parser().parseMillis(dateToParse)), is(expectedDate)); + } + + private void assertDateFormatParsingThrowingException(String pattern, String invalidDate) { + try { + FormatDateTimeFormatter formatter = Joda.forPattern(pattern); + DateTimeFormatter parser = formatter.parser(); + parser.parseMillis(invalidDate); + fail(String.format(Locale.ROOT, "Expected parsing exception for pattern [%s] with date [%s], but did not happen", pattern, invalidDate)); + } catch (IllegalArgumentException e) { + } + } + private long utcTimeInMillis(String time) { return ISODateTimeFormat.dateOptionalTimeParser().withZone(DateTimeZone.UTC).parseMillis(time); } diff --git a/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsTests.java b/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsTests.java index 6792298bab2b0..c12f57dfb0f86 100644 --- a/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsTests.java @@ -371,7 +371,9 @@ public void testIsolateMasterAndVerifyClusterStateConsensus() throws Exception { // restore isolation networkPartition.stopDisrupting(); - ensureStableCluster(3, new TimeValue(DISRUPTION_HEALING_OVERHEAD.millis() + networkPartition.expectedTimeToHeal().millis())); + for (String node : nodes) { + ensureStableCluster(3, new TimeValue(DISRUPTION_HEALING_OVERHEAD.millis() + networkPartition.expectedTimeToHeal().millis()), true, node); + } logger.info("issue a reroute"); // trigger a reroute now, instead of waiting for the background reroute of RerouteService @@ -578,7 +580,7 @@ public void testMasterNodeGCs() throws Exception { // restore GC masterNodeDisruption.stopDisrupting(); - ensureStableCluster(3, new TimeValue(DISRUPTION_HEALING_OVERHEAD.millis() + masterNodeDisruption.expectedTimeToHeal().millis()), + ensureStableCluster(3, new TimeValue(DISRUPTION_HEALING_OVERHEAD.millis() + masterNodeDisruption.expectedTimeToHeal().millis()), false, oldNonMasterNodes.get(0)); // make sure all nodes agree on master @@ -987,18 +989,18 @@ private ServiceDisruptionScheme addRandomDisruptionScheme() { } private void ensureStableCluster(int nodeCount) { - ensureStableCluster(nodeCount, TimeValue.timeValueSeconds(30), null); + ensureStableCluster(nodeCount, TimeValue.timeValueSeconds(30)); } private void ensureStableCluster(int nodeCount, TimeValue timeValue) { - ensureStableCluster(nodeCount, timeValue, null); + ensureStableCluster(nodeCount, timeValue, false, null); } private void ensureStableCluster(int nodeCount, @Nullable String viaNode) { - ensureStableCluster(nodeCount, TimeValue.timeValueSeconds(30), viaNode); + ensureStableCluster(nodeCount, TimeValue.timeValueSeconds(30), false, viaNode); } - private void ensureStableCluster(int nodeCount, TimeValue timeValue, @Nullable String viaNode) { + private void ensureStableCluster(int nodeCount, TimeValue timeValue, boolean local, @Nullable String viaNode) { if (viaNode == null) { viaNode = randomFrom(internalCluster().getNodeNames()); } @@ -1007,6 +1009,7 @@ private void ensureStableCluster(int nodeCount, TimeValue timeValue, @Nullable S .setWaitForEvents(Priority.LANGUID) .setWaitForNodes(Integer.toString(nodeCount)) .setTimeout(timeValue) + .setLocal(local) .setWaitForRelocatingShards(0) .get(); if (clusterHealthResponse.isTimedOut()) { diff --git a/core/src/test/java/org/elasticsearch/document/BulkTests.java b/core/src/test/java/org/elasticsearch/document/BulkTests.java index 8be32de6feb83..e4d3af7cdcc16 100644 --- a/core/src/test/java/org/elasticsearch/document/BulkTests.java +++ b/core/src/test/java/org/elasticsearch/document/BulkTests.java @@ -725,5 +725,39 @@ public void testThatMissingIndexDoesNotAbortFullBulkRequest() throws Exception{ assertThat(bulkResponse.hasFailures(), is(true)); assertThat(bulkResponse.getItems().length, is(5)); } + + @Test // issue 9821 + public void testFailedRequestsOnClosedIndex() throws Exception { + createIndex("bulkindex1"); + ensureYellow(); + + client().prepareIndex("bulkindex1", "index1_type", "1").setSource("text", "test").get(); + assertAcked(client().admin().indices().prepareClose("bulkindex1")); + + BulkRequest bulkRequest = new BulkRequest(); + bulkRequest.add(new IndexRequest("bulkindex1", "index1_type", "1").source("text", "hallo1")) + .add(new UpdateRequest("bulkindex1", "index1_type", "1").doc("foo", "bar")) + .add(new DeleteRequest("bulkindex1", "index1_type", "1")).refresh(true); + + BulkResponse bulkResponse = client().bulk(bulkRequest).get(); + assertThat(bulkResponse.hasFailures(), is(true)); + BulkItemResponse[] responseItems = bulkResponse.getItems(); + assertThat(responseItems.length, is(3)); + assertThat(responseItems[0].getOpType(), is("index")); + assertThat(responseItems[1].getOpType(), is("update")); + assertThat(responseItems[2].getOpType(), is("delete")); + } + + @Test // issue 9821 + public void testInvalidIndexNamesCorrectOpType() { + BulkResponse bulkResponse = client().prepareBulk() + .add(client().prepareIndex().setIndex("INVALID.NAME").setType("type1").setId("1").setSource("field", 1)) + .add(client().prepareUpdate().setIndex("INVALID.NAME").setType("type1").setId("1").setDoc("field", randomInt())) + .add(client().prepareDelete().setIndex("INVALID.NAME").setType("type1").setId("1")).get(); + assertThat(bulkResponse.getItems().length, is(3)); + assertThat(bulkResponse.getItems()[0].getOpType(), is("index")); + assertThat(bulkResponse.getItems()[1].getOpType(), is("update")); + assertThat(bulkResponse.getItems()[2].getOpType(), is("delete")); + } } diff --git a/core/src/test/java/org/elasticsearch/gateway/PriorityComparatorTests.java b/core/src/test/java/org/elasticsearch/gateway/PriorityComparatorTests.java new file mode 100644 index 0000000000000..a39a6ad8a1f26 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/gateway/PriorityComparatorTests.java @@ -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.gateway; + +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.routing.RoutingNodes; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.ShardRoutingState; +import org.elasticsearch.cluster.routing.UnassignedInfo; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.test.ElasticsearchTestCase; + +import java.util.HashMap; +import java.util.Locale; +import java.util.Map; + +public class PriorityComparatorTests extends ElasticsearchTestCase { + + public void testPriorityComparatorSort() { + RoutingNodes.UnassignedShards shards = new RoutingNodes.UnassignedShards(); + int numIndices = randomIntBetween(3, 99); + IndexMeta[] indices = new IndexMeta[numIndices]; + final Map map = new HashMap<>(); + + for (int i = 0; i < indices.length; i++) { + if (frequently()) { + indices[i] = new IndexMeta("idx_2015_04_" + String.format(Locale.ROOT, "%02d", i), randomIntBetween(1, 1000), randomIntBetween(1, 10000)); + } else { // sometimes just use defaults + indices[i] = new IndexMeta("idx_2015_04_" + String.format(Locale.ROOT, "%02d", i)); + } + map.put(indices[i].name, indices[i]); + } + int numShards = randomIntBetween(10, 100); + for (int i = 0; i < numShards; i++) { + IndexMeta indexMeta = randomFrom(indices); + shards.add(new ShardRouting(indexMeta.name, randomIntBetween(1, 5), null, null, null, + randomBoolean(), ShardRoutingState.UNASSIGNED, randomIntBetween(0, 100), new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "foobar"))); + } + shards.sort(new PriorityComparator() { + @Override + protected Settings getIndexSettings(String index) { + IndexMeta indexMeta = map.get(index); + return indexMeta.settings; + } + }); + ShardRouting previous = null; + for (ShardRouting routing : shards) { + if (previous != null) { + IndexMeta prevMeta = map.get(previous.getIndex()); + IndexMeta currentMeta = map.get(routing.getIndex()); + if (prevMeta.priority == currentMeta.priority) { + if (prevMeta.creationDate == currentMeta.creationDate) { + if (prevMeta.name.equals(currentMeta.name) == false) { + assertTrue("indexName mismatch, expected:" + currentMeta.name + " after " + prevMeta.name + " " + prevMeta.name.compareTo(currentMeta.name), prevMeta.name.compareTo(currentMeta.name) > 0); + } + } else { + assertTrue("creationDate mismatch, expected:" + currentMeta.creationDate + " after " + prevMeta.creationDate, prevMeta.creationDate > currentMeta.creationDate); + } + } else { + assertTrue("priority mismatch, expected:" + currentMeta.priority + " after " + prevMeta.priority, prevMeta.priority > currentMeta.priority); + } + } + previous = routing; + } + } + + private static class IndexMeta { + final String name; + final int priority; + final long creationDate; + final Settings settings; + + private IndexMeta(String name) { // default + this.name = name; + this.priority = 1; + this.creationDate = -1; + this.settings = Settings.EMPTY; + } + + private IndexMeta(String name, int priority, long creationDate) { + this.name = name; + this.priority = priority; + this.creationDate = creationDate; + this.settings = Settings.builder().put(IndexMetaData.SETTING_CREATION_DATE, creationDate) + .put(IndexMetaData.SETTING_PRIORITY, priority).build(); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java b/core/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java index 810118b5be377..73ea27ae84abf 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java +++ b/core/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java @@ -647,7 +647,6 @@ public void run() { } @Test - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/11788") public void testIndexOnSharedFSRecoversToAnyNode() throws Exception { Settings nodeSettings = nodeSettings(); Settings fooSettings = Settings.builder().put(nodeSettings).put("node.affinity", "foo").build(); diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index bfd13a27c981e..70a801abc7049 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -991,6 +991,7 @@ public void run() { } indexed.await(); IOUtils.close(engine); + thread.join(); } } diff --git a/core/src/test/java/org/elasticsearch/index/mapper/FieldTypeLookupTests.java b/core/src/test/java/org/elasticsearch/index/mapper/FieldTypeLookupTests.java index dd02b208a14fa..a3b3ea12af295 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/FieldTypeLookupTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/FieldTypeLookupTests.java @@ -24,8 +24,6 @@ import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.index.fielddata.FieldDataType; -import org.elasticsearch.index.mapper.core.AbstractFieldMapper; import org.elasticsearch.test.ElasticsearchTestCase; import java.io.IOException; @@ -176,10 +174,10 @@ static List newList(FieldMapper... mapper) { } // this sucks how much must be overridden just do get a dummy field mapper... - static class FakeFieldMapper extends AbstractFieldMapper { + static class FakeFieldMapper extends FieldMapper { static Settings dummySettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT.id).build(); public FakeFieldMapper(String fullName, String indexName) { - super(fullName, makeFieldType(fullName, indexName), null, null, dummySettings, null, null); + super(fullName, makeFieldType(fullName, indexName), makeFieldType(fullName, indexName), dummySettings, null, null); } static MappedFieldType makeFieldType(String fullName, String indexName) { FakeFieldType fieldType = new FakeFieldType(); @@ -201,10 +199,6 @@ public String typeName() { } } @Override - public MappedFieldType defaultFieldType() { return null; } - @Override - public FieldDataType defaultFieldDataType() { return null; } - @Override protected String contentType() { return null; } @Override protected void parseCreateField(ParseContext context, List list) throws IOException {} diff --git a/core/src/test/java/org/elasticsearch/index/mapper/all/SimpleAllMapperTests.java b/core/src/test/java/org/elasticsearch/index/mapper/all/SimpleAllMapperTests.java index b46c4e349d714..dabac4e204066 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/all/SimpleAllMapperTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/all/SimpleAllMapperTests.java @@ -226,7 +226,6 @@ public void testRandom() throws Exception { boolean tv_offsets = false; boolean tv_positions = false; String similarity = null; - boolean fieldData = false; XContentBuilder mappingBuilder = jsonBuilder(); mappingBuilder.startObject().startObject("test"); List> booleanOptionList = new ArrayList<>(); @@ -263,12 +262,6 @@ public void testRandom() throws Exception { if (randomBoolean()) { mappingBuilder.field("similarity", similarity = randomBoolean() ? "BM25" : "TF/IDF"); } - if (randomBoolean()) { - fieldData = true; - mappingBuilder.startObject("fielddata"); - mappingBuilder.field("foo", "bar"); - mappingBuilder.endObject(); - } mappingBuilder.endObject(); } @@ -310,7 +303,6 @@ public void testRandom() throws Exception { } else { assertThat(similarity, equalTo(builtDocMapper.allFieldMapper().fieldType().similarity().name())); } - assertThat(builtMapping.contains("fielddata"), is(fieldData)); if (allDefault) { BytesStreamOutput bytesStreamOutput = new BytesStreamOutput(0); XContentBuilder b = new XContentBuilder(XContentType.JSON.xContent(), bytesStreamOutput); diff --git a/core/src/test/java/org/elasticsearch/index/mapper/date/SimpleDateMappingTests.java b/core/src/test/java/org/elasticsearch/index/mapper/date/SimpleDateMappingTests.java index 829d3458f2954..3d9398e426a3b 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/date/SimpleDateMappingTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/date/SimpleDateMappingTests.java @@ -26,6 +26,8 @@ import org.apache.lucene.search.NumericRangeQuery; import org.apache.lucene.util.Constants; import org.elasticsearch.Version; +import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse; +import org.elasticsearch.action.admin.indices.mapping.put.PutMappingResponse; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.settings.Settings; @@ -45,6 +47,7 @@ import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.test.ElasticsearchSingleNodeTest; import org.elasticsearch.test.TestSearchContext; +import org.elasticsearch.test.VersionUtils; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.junit.Before; @@ -55,7 +58,6 @@ import static com.carrotsearch.randomizedtesting.RandomizedTest.systemPropertyAsBoolean; import static org.elasticsearch.common.settings.Settings.settingsBuilder; import static org.elasticsearch.index.mapper.string.SimpleStringMappingTests.docValuesType; -import static org.elasticsearch.test.VersionUtils.randomVersionBetween; import static org.hamcrest.Matchers.*; public class SimpleDateMappingTests extends ElasticsearchSingleNodeTest { @@ -482,4 +484,94 @@ public void testThatEpochCanBeIgnoredWithCustomFormat() throws Exception { indexResponse = client().prepareIndex("test", "test").setSource(document).get(); assertThat(indexResponse.isCreated(), is(true)); } + + public void testThatOlderIndicesAllowNonStrictDates() throws Exception { + String mapping = XContentFactory.jsonBuilder().startObject().startObject("type") + .startObject("properties").startObject("date_field").field("type", "date").endObject().endObject() + .endObject().endObject().string(); + + Version randomVersion = VersionUtils.randomVersionBetween(getRandom(), Version.V_0_90_0, Version.V_1_6_1); + IndexService index = createIndex("test", settingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, randomVersion).build()); + client().admin().indices().preparePutMapping("test").setType("type").setSource(mapping).get(); + assertDateFormat("epoch_millis||dateOptionalTime"); + DocumentMapper defaultMapper = index.mapperService().documentMapper("type"); + + defaultMapper.parse("type", "1", XContentFactory.jsonBuilder() + .startObject() + .field("date_field", "1-1-1T00:00:44.000Z") + .endObject() + .bytes()); + + // also test normal date + defaultMapper.parse("type", "1", XContentFactory.jsonBuilder() + .startObject() + .field("date_field", "2015-06-06T00:00:44.000Z") + .endObject() + .bytes()); + } + + public void testThatNewIndicesOnlyAllowStrictDates() throws Exception { + String mapping = XContentFactory.jsonBuilder().startObject().startObject("type") + .startObject("properties").startObject("date_field").field("type", "date").endObject().endObject() + .endObject().endObject().string(); + + IndexService index = createIndex("test"); + client().admin().indices().preparePutMapping("test").setType("type").setSource(mapping).get(); + assertDateFormat(DateFieldMapper.Defaults.DATE_TIME_FORMATTER.format()); + DocumentMapper defaultMapper = index.mapperService().documentMapper("type"); + + // also test normal date + defaultMapper.parse("type", "1", XContentFactory.jsonBuilder() + .startObject() + .field("date_field", "2015-06-06T00:00:44.000Z") + .endObject() + .bytes()); + + try { + defaultMapper.parse("type", "1", XContentFactory.jsonBuilder() + .startObject() + .field("date_field", "1-1-1T00:00:44.000Z") + .endObject() + .bytes()); + fail("non strict date indexing should have been failed"); + } catch (MapperParsingException e) { + assertThat(e.getCause(), instanceOf(IllegalArgumentException.class)); + } + } + + public void testThatUpgradingAnOlderIndexToStrictDateWorks() throws Exception { + String mapping = XContentFactory.jsonBuilder().startObject().startObject("type") + .startObject("properties").startObject("date_field").field("type", "date").field("format", "dateOptionalTime").endObject().endObject() + .endObject().endObject().string(); + + Version randomVersion = VersionUtils.randomVersionBetween(getRandom(), Version.V_0_90_0, Version.V_1_6_1); + createIndex("test", settingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, randomVersion).build()); + client().admin().indices().preparePutMapping("test").setType("type").setSource(mapping).get(); + assertDateFormat("epoch_millis||dateOptionalTime"); + + // index doc + client().prepareIndex("test", "type", "1").setSource(XContentFactory.jsonBuilder() + .startObject() + .field("date_field", "2015-06-06T00:00:44.000Z") + .endObject()).get(); + + // update mapping + String newMapping = XContentFactory.jsonBuilder().startObject().startObject("type") + .startObject("properties").startObject("date_field") + .field("type", "date") + .field("format", "strictDateOptionalTime||epoch_millis") + .endObject().endObject().endObject().endObject().string(); + PutMappingResponse putMappingResponse = client().admin().indices().preparePutMapping("test").setType("type").setSource(newMapping).get(); + assertThat(putMappingResponse.isAcknowledged(), is(true)); + + assertDateFormat("strictDateOptionalTime||epoch_millis"); + } + + private void assertDateFormat(String expectedFormat) throws IOException { + GetMappingsResponse response = client().admin().indices().prepareGetMappings("test").setTypes("type").get(); + Map mappingMap = response.getMappings().get("test").get("type").getSourceAsMap(); + Map properties = (Map) mappingMap.get("properties"); + Map dateField = (Map) properties.get("date_field"); + assertThat((String) dateField.get("format"), is(expectedFormat)); + } } diff --git a/core/src/test/java/org/elasticsearch/index/mapper/externalvalues/ExternalMapper.java b/core/src/test/java/org/elasticsearch/index/mapper/externalvalues/ExternalMapper.java index 313d4ff575b7c..0024a63a4b5a2 100755 --- a/core/src/test/java/org/elasticsearch/index/mapper/externalvalues/ExternalMapper.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/externalvalues/ExternalMapper.java @@ -28,7 +28,6 @@ import org.elasticsearch.common.geo.builders.ShapeBuilder; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.index.fielddata.FieldDataType; import org.elasticsearch.index.mapper.ContentPath; import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; @@ -37,7 +36,6 @@ import org.elasticsearch.index.mapper.MergeMappingException; import org.elasticsearch.index.mapper.MergeResult; import org.elasticsearch.index.mapper.ParseContext; -import org.elasticsearch.index.mapper.core.AbstractFieldMapper; import org.elasticsearch.index.mapper.core.BinaryFieldMapper; import org.elasticsearch.index.mapper.core.BooleanFieldMapper; import org.elasticsearch.index.mapper.geo.GeoPointFieldMapper; @@ -60,7 +58,7 @@ * .point GeoPoint type * .shape GeoShape type */ -public class ExternalMapper extends AbstractFieldMapper { +public class ExternalMapper extends FieldMapper { public static class Names { public static final String FIELD_BIN = "bin"; @@ -69,7 +67,7 @@ public static class Names { public static final String FIELD_SHAPE = "shape"; } - public static class Builder extends AbstractFieldMapper.Builder { + public static class Builder extends FieldMapper.Builder { private BinaryFieldMapper.Builder binBuilder = new BinaryFieldMapper.Builder(Names.FIELD_BIN); private BooleanFieldMapper.Builder boolBuilder = new BooleanFieldMapper.Builder(Names.FIELD_BOOL); @@ -174,8 +172,7 @@ public ExternalMapper(String simpleName, MappedFieldType fieldType, String generatedValue, String mapperName, BinaryFieldMapper binMapper, BooleanFieldMapper boolMapper, GeoPointFieldMapper pointMapper, GeoShapeFieldMapper shapeMapper, FieldMapper stringMapper, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { - super(simpleName, fieldType, false, null, indexSettings, - multiFields, copyTo); + super(simpleName, fieldType, new ExternalFieldType(), indexSettings, multiFields, copyTo); this.generatedValue = generatedValue; this.mapperName = mapperName; this.binMapper = binMapper; @@ -185,16 +182,6 @@ public ExternalMapper(String simpleName, MappedFieldType fieldType, this.stringMapper = stringMapper; } - @Override - public MappedFieldType defaultFieldType() { - return new ExternalFieldType(); - } - - @Override - public FieldDataType defaultFieldDataType() { - return null; - } - @Override public Mapper parse(ParseContext context) throws IOException { byte[] bytes = "Hello world".getBytes(Charset.defaultCharset()); diff --git a/core/src/test/java/org/elasticsearch/index/mapper/externalvalues/ExternalMetadataMapper.java b/core/src/test/java/org/elasticsearch/index/mapper/externalvalues/ExternalMetadataMapper.java index 031d3aab7afbd..1cda8eee23956 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/externalvalues/ExternalMetadataMapper.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/externalvalues/ExternalMetadataMapper.java @@ -53,7 +53,7 @@ public class ExternalMetadataMapper extends MetadataFieldMapper { } protected ExternalMetadataMapper(Settings indexSettings) { - super(FIELD_NAME, FIELD_TYPE, true, null, indexSettings); + super(FIELD_NAME, FIELD_TYPE, FIELD_TYPE, indexSettings); } @Override @@ -61,16 +61,6 @@ public String name() { return CONTENT_TYPE; } - @Override - public MappedFieldType defaultFieldType() { - return FIELD_TYPE; - } - - @Override - public FieldDataType defaultFieldDataType() { - return new FieldDataType("string"); - } - @Override protected void parseCreateField(ParseContext context, List fields) throws IOException { // handled in post parse diff --git a/core/src/test/java/org/elasticsearch/index/mapper/simple/SimpleMapperTests.java b/core/src/test/java/org/elasticsearch/index/mapper/simple/SimpleMapperTests.java index 50815f55f3815..cf50a1d819e67 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/simple/SimpleMapperTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/simple/SimpleMapperTests.java @@ -23,6 +23,7 @@ import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.index.mapper.*; import org.elasticsearch.index.mapper.ParseContext.Document; import org.elasticsearch.index.IndexService; @@ -135,4 +136,18 @@ public void testNoDocumentSent() throws Exception { assertThat(e.getMessage(), equalTo("failed to parse, document is empty")); } } + + public void testHazardousFieldNames() throws Exception { + IndexService indexService = createIndex("test"); + DocumentMapperParser mapperParser = indexService.mapperService().documentMapperParser(); + String mapping = XContentFactory.jsonBuilder().startObject().startObject("type").startObject("properties") + .startObject("foo.bar").field("type", "string").endObject() + .endObject().endObject().string(); + try { + mapperParser.parse(mapping); + fail("Mapping parse should have failed"); + } catch (MapperParsingException e) { + assertTrue(e.getMessage(), e.getMessage().contains("cannot contain '.'")); + } + } } diff --git a/core/src/test/java/org/elasticsearch/index/mapper/timestamp/TimestampMappingTests.java b/core/src/test/java/org/elasticsearch/index/mapper/timestamp/TimestampMappingTests.java index 784811b26f7a3..e1341cecf026f 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/timestamp/TimestampMappingTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/timestamp/TimestampMappingTests.java @@ -37,7 +37,6 @@ import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.json.JsonXContent; -import org.elasticsearch.index.Index; import org.elasticsearch.index.mapper.*; import org.elasticsearch.index.mapper.internal.TimestampFieldMapper; import org.elasticsearch.test.ElasticsearchSingleNodeTest; @@ -56,14 +55,7 @@ import static org.elasticsearch.test.VersionUtils.randomVersion; import static org.elasticsearch.test.VersionUtils.randomVersionBetween; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.hasKey; -import static org.hamcrest.Matchers.instanceOf; -import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.isIn; -import static org.hamcrest.Matchers.lessThanOrEqualTo; -import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.*; /** */ @@ -113,8 +105,10 @@ public void testDefaultValues() throws Exception { assertThat(docMapper.timestampFieldMapper().fieldType().stored(), equalTo(version.onOrAfter(Version.V_2_0_0))); assertThat(docMapper.timestampFieldMapper().fieldType().indexOptions(), equalTo(TimestampFieldMapper.Defaults.FIELD_TYPE.indexOptions())); assertThat(docMapper.timestampFieldMapper().path(), equalTo(TimestampFieldMapper.Defaults.PATH)); - assertThat(docMapper.timestampFieldMapper().fieldType().dateTimeFormatter().format(), equalTo(TimestampFieldMapper.DEFAULT_DATE_TIME_FORMAT)); assertThat(docMapper.timestampFieldMapper().fieldType().hasDocValues(), equalTo(version.onOrAfter(Version.V_2_0_0))); + String expectedFormat = version.onOrAfter(Version.V_2_0_0) ? TimestampFieldMapper.DEFAULT_DATE_TIME_FORMAT : + TimestampFieldMapper.Defaults.DATE_TIME_FORMATTER_BEFORE_2_0.format(); + assertThat(docMapper.timestampFieldMapper().fieldType().dateTimeFormatter().format(), equalTo(expectedFormat)); assertAcked(client().admin().indices().prepareDelete("test").execute().get()); } } @@ -755,7 +749,7 @@ public void testBackcompatPath() throws Exception { IndexRequest request = new IndexRequest("test", "type", "1").source(doc); request.process(metaData, mappingMetaData, true, "test"); - assertEquals(request.timestamp(), "1"); + assertThat(request.timestamp(), is("1")); } public void testIncludeInObjectBackcompat() throws Exception { diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 12237cea89521..a0da52b62014e 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.index.shard; +import org.apache.lucene.index.CorruptIndexException; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; import org.elasticsearch.action.admin.indices.stats.IndexStats; @@ -32,6 +33,7 @@ import org.elasticsearch.index.IndexService; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.query.QueryParsingException; +import org.elasticsearch.index.settings.IndexSettingsService; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.TranslogConfig; @@ -214,7 +216,7 @@ public void testFailShard() throws Exception { IndexService test = indicesService.indexService("test"); IndexShard shard = test.shard(0); // fail shard - shard.failShard("test shard fail", new IOException("corrupted")); + shard.failShard("test shard fail", new CorruptIndexException("", "")); // check state file still exists ShardStateMetaData shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId)); assertEquals(shardStateMetaData, getShardStateMetadata(shard)); @@ -394,4 +396,13 @@ public void testMinimumCompatVersion() { test.engine().flush(); assertEquals(Version.CURRENT.luceneVersion, test.minimumCompatibleVersion()); } + + public void testUpdatePriority() { + assertAcked(client().admin().indices().prepareCreate("test") + .setSettings(IndexMetaData.SETTING_PRIORITY, 200)); + IndexSettingsService indexSettingsService = getInstanceFromNode(IndicesService.class).indexService("test").settingsService(); + assertEquals(200, indexSettingsService.getSettings().getAsInt(IndexMetaData.SETTING_PRIORITY, 0).intValue()); + client().admin().indices().prepareUpdateSettings("test").setSettings(Settings.builder().put(IndexMetaData.SETTING_PRIORITY, 400).build()).get(); + assertEquals(400, indexSettingsService.getSettings().getAsInt(IndexMetaData.SETTING_PRIORITY, 0).intValue()); + } } diff --git a/core/src/test/java/org/elasticsearch/index/store/CorruptedFileTest.java b/core/src/test/java/org/elasticsearch/index/store/CorruptedFileTest.java index 0ee11bdce533c..baa1f087cb79e 100644 --- a/core/src/test/java/org/elasticsearch/index/store/CorruptedFileTest.java +++ b/core/src/test/java/org/elasticsearch/index/store/CorruptedFileTest.java @@ -60,7 +60,7 @@ import org.elasticsearch.indices.recovery.RecoveryFileChunkRequest; import org.elasticsearch.indices.recovery.RecoverySettings; import org.elasticsearch.indices.recovery.RecoveryTarget; -import org.elasticsearch.monitor.fs.FsStats; +import org.elasticsearch.monitor.fs.FsInfo; import org.elasticsearch.snapshots.SnapshotState; import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.elasticsearch.test.InternalTestCluster; @@ -570,7 +570,7 @@ private Map> findFilesToCorruptForReplica() throws IOExceptio NodeStats nodeStats = nodeStatses.getNodes()[0]; List files = new ArrayList<>(); filesToNodes.put(nodeStats.getNode().getName(), files); - for (FsStats.Info info : nodeStats.getFs()) { + for (FsInfo.Path info : nodeStats.getFs()) { String path = info.getPath(); final String relativeDataLocationPath = "indices/test/" + Integer.toString(shardRouting.getId()) + "/index"; Path file = PathUtils.get(path).resolve(relativeDataLocationPath); @@ -604,7 +604,7 @@ private ShardRouting corruptRandomPrimaryFile(final boolean includePerCommitFile String nodeId = shardRouting.currentNodeId(); NodesStatsResponse nodeStatses = client().admin().cluster().prepareNodesStats(nodeId).setFs(true).get(); Set files = new TreeSet<>(); // treeset makes sure iteration order is deterministic - for (FsStats.Info info : nodeStatses.getNodes()[0].getFs()) { + for (FsInfo.Path info : nodeStatses.getNodes()[0].getFs()) { String path = info.getPath(); final String relativeDataLocationPath = "indices/test/" + Integer.toString(shardRouting.getId()) + "/index"; Path file = PathUtils.get(path).resolve(relativeDataLocationPath); @@ -716,7 +716,7 @@ public List listShardFiles(ShardRouting routing) throws IOException { assertThat(routing.toString(), nodeStatses.getNodes().length, equalTo(1)); List files = new ArrayList<>(); - for (FsStats.Info info : nodeStatses.getNodes()[0].getFs()) { + for (FsInfo.Path info : nodeStatses.getNodes()[0].getFs()) { String path = info.getPath(); Path file = PathUtils.get(path).resolve("indices/test/" + Integer.toString(routing.getId()) + "/index"); if (Files.exists(file)) { // multi data path might only have one path in use diff --git a/core/src/test/java/org/elasticsearch/index/store/CorruptedTranslogTests.java b/core/src/test/java/org/elasticsearch/index/store/CorruptedTranslogTests.java index ae37466981b49..9968e742cb97c 100644 --- a/core/src/test/java/org/elasticsearch/index/store/CorruptedTranslogTests.java +++ b/core/src/test/java/org/elasticsearch/index/store/CorruptedTranslogTests.java @@ -33,7 +33,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.translog.TranslogConfig; -import org.elasticsearch.monitor.fs.FsStats; +import org.elasticsearch.monitor.fs.FsInfo; import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.elasticsearch.test.engine.MockEngineSupport; import org.elasticsearch.test.junit.annotations.TestLogging; @@ -124,8 +124,8 @@ private void corruptRandomTranslogFiles() throws IOException { String nodeId = shardRouting.currentNodeId(); NodesStatsResponse nodeStatses = client().admin().cluster().prepareNodesStats(nodeId).setFs(true).get(); Set files = new TreeSet<>(); // treeset makes sure iteration order is deterministic - for (FsStats.Info info : nodeStatses.getNodes()[0].getFs()) { - String path = info.getPath(); + for (FsInfo.Path fsPath : nodeStatses.getNodes()[0].getFs()) { + String path = fsPath.getPath(); final String relativeDataLocationPath = "indices/test/" + Integer.toString(shardRouting.getId()) + "/translog"; Path file = PathUtils.get(path).resolve(relativeDataLocationPath); if (Files.exists(file)) { diff --git a/core/src/test/java/org/elasticsearch/monitor/fs/FsProbeTests.java b/core/src/test/java/org/elasticsearch/monitor/fs/FsProbeTests.java new file mode 100644 index 0000000000000..82c9097d693e7 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/monitor/fs/FsProbeTests.java @@ -0,0 +1,59 @@ +/* + * 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.monitor.fs; + +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.env.NodeEnvironment; +import org.elasticsearch.test.ElasticsearchTestCase; +import org.junit.Test; + +import java.io.IOException; + +import static org.hamcrest.Matchers.*; + +public class FsProbeTests extends ElasticsearchTestCase { + + @Test + public void testFsInfo() throws IOException { + try (NodeEnvironment env = newNodeEnvironment()) { + FsProbe probe = new FsProbe(Settings.EMPTY, env); + + FsInfo stats = probe.stats(); + assertNotNull(stats); + assertThat(stats.getTimestamp(), greaterThan(0L)); + + FsInfo.Path total = stats.getTotal(); + assertNotNull(total); + assertThat(total.total, greaterThan(0L)); + assertThat(total.free, greaterThan(0L)); + assertThat(total.available, greaterThan(0L)); + + for (FsInfo.Path path : stats) { + assertNotNull(path); + assertThat(path.getPath(), not(isEmptyOrNullString())); + assertThat(path.getMount(), not(isEmptyOrNullString())); + assertThat(path.getType(), not(isEmptyOrNullString())); + assertThat(path.total, greaterThan(0L)); + assertThat(path.free, greaterThan(0L)); + assertThat(path.available, greaterThan(0L)); + } + } + } +} diff --git a/core/src/test/java/org/elasticsearch/plugins/PluginManagerTests.java b/core/src/test/java/org/elasticsearch/plugins/PluginManagerTests.java index e7dde021d63b7..62b768bb35bfb 100644 --- a/core/src/test/java/org/elasticsearch/plugins/PluginManagerTests.java +++ b/core/src/test/java/org/elasticsearch/plugins/PluginManagerTests.java @@ -54,9 +54,7 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertFileExists; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.CoreMatchers.not; -import static org.hamcrest.Matchers.arrayWithSize; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.*; @ClusterScope(scope = Scope.TEST, numDataNodes = 0, transportClientRatio = 0.0) @LuceneTestCase.SuppressFileSystems("*") // TODO: clean up this test to allow extra files @@ -513,6 +511,27 @@ private void runTestWithForbiddenName(String name) throws IOException { } } + @Test + public void testOfficialPluginName_ThrowsException() throws IOException { + PluginManager.checkForOfficialPlugins("elasticsearch-analysis-icu"); + PluginManager.checkForOfficialPlugins("elasticsearch-analysis-kuromoji"); + PluginManager.checkForOfficialPlugins("elasticsearch-analysis-phonetic"); + PluginManager.checkForOfficialPlugins("elasticsearch-analysis-smartcn"); + PluginManager.checkForOfficialPlugins("elasticsearch-analysis-stempel"); + PluginManager.checkForOfficialPlugins("elasticsearch-cloud-aws"); + PluginManager.checkForOfficialPlugins("elasticsearch-cloud-azure"); + PluginManager.checkForOfficialPlugins("elasticsearch-cloud-gce"); + PluginManager.checkForOfficialPlugins("elasticsearch-delete-by-query"); + PluginManager.checkForOfficialPlugins("elasticsearch-lang-javascript"); + PluginManager.checkForOfficialPlugins("elasticsearch-lang-python"); + + try { + PluginManager.checkForOfficialPlugins("elasticsearch-mapper-attachment"); + fail("elasticsearch-mapper-attachment should not be allowed"); + } catch (IllegalArgumentException e) { + // We expect that error + } + } /** * Retrieve a URL string that represents the resource with the given {@code resourceName}. diff --git a/core/src/test/java/org/elasticsearch/plugins/PluginManagerUnitTests.java b/core/src/test/java/org/elasticsearch/plugins/PluginManagerUnitTests.java index 3dfb7a04c5b25..4517f82c2ea08 100644 --- a/core/src/test/java/org/elasticsearch/plugins/PluginManagerUnitTests.java +++ b/core/src/test/java/org/elasticsearch/plugins/PluginManagerUnitTests.java @@ -20,16 +20,18 @@ package org.elasticsearch.plugins; import com.google.common.io.Files; - +import org.elasticsearch.Version; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; import org.elasticsearch.test.ElasticsearchTestCase; import org.junit.Test; import java.io.IOException; +import java.net.URL; import java.nio.file.Path; import static org.elasticsearch.common.settings.Settings.settingsBuilder; +import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.is; /** @@ -55,4 +57,14 @@ public void testThatConfigDirectoryCanBeOutsideOfElasticsearchHomeDirectory() th assertThat(configDirPath, is(expectedDirPath)); } + + @Test + public void testSimplifiedNaming() throws IOException { + String pluginName = randomAsciiOfLength(10); + PluginManager.PluginHandle handle = PluginManager.PluginHandle.parse(pluginName); + assertThat(handle.urls(), hasSize(1)); + URL expected = new URL("http", "download.elastic.co", "/org.elasticsearch.plugins/" + pluginName + "/" + + pluginName + "-" + Version.CURRENT.number() + ".zip"); + assertThat(handle.urls().get(0), is(expected)); + } } diff --git a/core/src/test/java/org/elasticsearch/script/ScriptServiceTests.java b/core/src/test/java/org/elasticsearch/script/ScriptServiceTests.java index b2f7a18676733..19ada40353a82 100644 --- a/core/src/test/java/org/elasticsearch/script/ScriptServiceTests.java +++ b/core/src/test/java/org/elasticsearch/script/ScriptServiceTests.java @@ -164,6 +164,16 @@ public void testScriptsSameNameDifferentLanguage() throws IOException { assertThat(expressionScript.lang(), equalTo(ExpressionScriptEngineService.NAME)); } + @Test + public void testInlineScriptCompiledOnceCache() throws IOException { + buildScriptService(Settings.EMPTY); + CompiledScript compiledScript1 = scriptService.compile(new Script("1+1", ScriptType.INLINE, "test", null), + randomFrom(scriptContexts)); + CompiledScript compiledScript2 = scriptService.compile(new Script("1+1", ScriptType.INLINE, "test", null), + randomFrom(scriptContexts)); + assertThat(compiledScript1, sameInstance(compiledScript2)); + } + @Test public void testInlineScriptCompiledOnceMultipleLangAcronyms() throws IOException { buildScriptService(Settings.EMPTY); diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramTests.java index eedddbd94a02e..496b5eb640ac8 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramTests.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramTests.java @@ -1281,9 +1281,9 @@ public void testDSTBoundaryIssue9491() throws InterruptedException, ExecutionExc public void testIssue8209() throws InterruptedException, ExecutionException { assertAcked(client().admin().indices().prepareCreate("test8209").addMapping("type", "d", "type=date").get()); indexRandom(true, - client().prepareIndex("test8209", "type").setSource("d", "2014-01-01T0:00:00Z"), - client().prepareIndex("test8209", "type").setSource("d", "2014-04-01T0:00:00Z"), - client().prepareIndex("test8209", "type").setSource("d", "2014-04-30T0:00:00Z")); + client().prepareIndex("test8209", "type").setSource("d", "2014-01-01T00:00:00Z"), + client().prepareIndex("test8209", "type").setSource("d", "2014-04-01T00:00:00Z"), + client().prepareIndex("test8209", "type").setSource("d", "2014-04-30T00:00:00Z")); ensureSearchable("test8209"); SearchResponse response = client().prepareSearch("test8209") .addAggregation(dateHistogram("histo").field("d").interval(DateHistogramInterval.MONTH).timeZone("CET") diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/BucketSelectorTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/BucketSelectorTests.java new file mode 100644 index 0000000000000..bd8db68959fa0 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/BucketSelectorTests.java @@ -0,0 +1,468 @@ +/* + * 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.aggregations.pipeline; + +import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.script.Script; +import org.elasticsearch.script.ScriptService.ScriptType; +import org.elasticsearch.script.groovy.GroovyScriptEngineService; +import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; +import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram; +import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram.Bucket; +import org.elasticsearch.search.aggregations.metrics.sum.Sum; +import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy; +import org.elasticsearch.test.ElasticsearchIntegrationTest; +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram; +import static org.elasticsearch.search.aggregations.AggregationBuilders.sum; +import static org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilders.having; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.lessThan; +import static org.hamcrest.Matchers.notNullValue; + +@ElasticsearchIntegrationTest.SuiteScopeTest +public class BucketSelectorTests extends ElasticsearchIntegrationTest { + + private static final String FIELD_1_NAME = "field1"; + private static final String FIELD_2_NAME = "field2"; + private static final String FIELD_3_NAME = "field3"; + private static final String FIELD_4_NAME = "field4"; + + private static int interval; + private static int numDocs; + private static int minNumber; + private static int maxNumber; + + @Override + public void setupSuiteScopeCluster() throws Exception { + createIndex("idx"); + createIndex("idx_unmapped"); + + interval = randomIntBetween(1, 50); + numDocs = randomIntBetween(10, 500); + minNumber = -200; + maxNumber = 200; + + List builders = new ArrayList<>(); + for (int docs = 0; docs < numDocs; docs++) { + builders.add(client().prepareIndex("idx", "type").setSource(newDocBuilder())); + } + + client().preparePutIndexedScript().setId("my_script").setScriptLang(GroovyScriptEngineService.NAME) + .setSource("{ \"script\": \"Double.isNaN(_value0) ? false : (_value0 + _value1 > 100)\" }").get(); + + indexRandom(true, builders); + ensureSearchable(); + } + + private XContentBuilder newDocBuilder() throws IOException { + XContentBuilder jsonBuilder = jsonBuilder(); + jsonBuilder.startObject(); + jsonBuilder.field(FIELD_1_NAME, randomIntBetween(minNumber, maxNumber)); + jsonBuilder.field(FIELD_2_NAME, randomIntBetween(minNumber, maxNumber)); + jsonBuilder.field(FIELD_3_NAME, randomIntBetween(minNumber, maxNumber)); + jsonBuilder.field(FIELD_4_NAME, randomIntBetween(minNumber, maxNumber)); + jsonBuilder.endObject(); + return jsonBuilder; + } + + @Test + public void inlineScript() { + + SearchResponse response = client() + .prepareSearch("idx") + .addAggregation( + histogram("histo") + .field(FIELD_1_NAME) + .interval(interval) + .subAggregation(sum("field2Sum").field(FIELD_2_NAME)) + .subAggregation(sum("field3Sum").field(FIELD_3_NAME)) + .subAggregation( + having("having").setBucketsPaths("field2Sum", "field3Sum").script( + new Script("Double.isNaN(_value0) ? false : (_value0 + _value1 > 100)", ScriptType.INLINE, + null, null)))).execute() + .actionGet(); + + assertSearchResponse(response); + + InternalHistogram histo = response.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + + for (int i = 0; i < buckets.size(); ++i) { + Histogram.Bucket bucket = buckets.get(i); + Sum field2Sum = bucket.getAggregations().get("field2Sum"); + assertThat(field2Sum, notNullValue()); + double field2SumValue = field2Sum.getValue(); + Sum field3Sum = bucket.getAggregations().get("field3Sum"); + assertThat(field3Sum, notNullValue()); + double field3SumValue = field3Sum.getValue(); + assertThat(field2SumValue + field3SumValue, greaterThan(100.0)); + } + } + + @Test + public void inlineScriptNoBucketsPruned() { + + SearchResponse response = client() + .prepareSearch("idx") + .addAggregation( + histogram("histo") + .field(FIELD_1_NAME) + .interval(interval) + .subAggregation(sum("field2Sum").field(FIELD_2_NAME)) + .subAggregation(sum("field3Sum").field(FIELD_3_NAME)) + .subAggregation( + having("having").setBucketsPaths("field2Sum", "field3Sum").script( + new Script("Double.isNaN(_value0) ? true : (_value0 < 10000)", ScriptType.INLINE, null, + null)))).execute() + .actionGet(); + + assertSearchResponse(response); + + InternalHistogram histo = response.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + + for (int i = 0; i < buckets.size(); ++i) { + Histogram.Bucket bucket = buckets.get(i); + Sum field2Sum = bucket.getAggregations().get("field2Sum"); + assertThat(field2Sum, notNullValue()); + double field2SumValue = field2Sum.getValue(); + Sum field3Sum = bucket.getAggregations().get("field3Sum"); + assertThat(field3Sum, notNullValue()); + double field3SumValue = field3Sum.getValue(); + assertThat(field2SumValue + field3SumValue, lessThan(10000.0)); + } + } + + @Test + public void inlineScriptNoBucketsLeft() { + + SearchResponse response = client() + .prepareSearch("idx") + .addAggregation( + histogram("histo") + .field(FIELD_1_NAME) + .interval(interval) + .subAggregation(sum("field2Sum").field(FIELD_2_NAME)) + .subAggregation(sum("field3Sum").field(FIELD_3_NAME)) + .subAggregation( + having("having").setBucketsPaths("field2Sum", "field3Sum").script( + new Script("Double.isNaN(_value0) ? false : (_value0 > 10000)", ScriptType.INLINE, null, + null)))).execute().actionGet(); + + assertSearchResponse(response); + + InternalHistogram histo = response.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + assertThat(buckets.size(), equalTo(0)); + } + + @Test + public void inlineScript2() { + + SearchResponse response = client() + .prepareSearch("idx") + .addAggregation( + histogram("histo") + .field(FIELD_1_NAME) + .interval(interval) + .subAggregation(sum("field2Sum").field(FIELD_2_NAME)) + .subAggregation(sum("field3Sum").field(FIELD_3_NAME)) + .subAggregation( + having("having").setBucketsPaths("field2Sum", "field3Sum").script( + new Script("Double.isNaN(_value0) ? false : (_value0 < _value1)", ScriptType.INLINE, null, + null)))).execute().actionGet(); + + assertSearchResponse(response); + + InternalHistogram histo = response.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + + for (int i = 0; i < buckets.size(); ++i) { + Histogram.Bucket bucket = buckets.get(i); + Sum field2Sum = bucket.getAggregations().get("field2Sum"); + assertThat(field2Sum, notNullValue()); + double field2SumValue = field2Sum.getValue(); + Sum field3Sum = bucket.getAggregations().get("field3Sum"); + assertThat(field3Sum, notNullValue()); + double field3SumValue = field3Sum.getValue(); + assertThat(field3SumValue - field2SumValue, greaterThan(0.0)); + } + } + + @Test + public void inlineScriptSingleVariable() { + + SearchResponse response = client() + .prepareSearch("idx") + .addAggregation( + histogram("histo") + .field(FIELD_1_NAME) + .interval(interval) + .subAggregation(sum("field2Sum").field(FIELD_2_NAME)) + .subAggregation( + having("having").setBucketsPaths("field2Sum") + .script(new Script("Double.isNaN(_value0) ? false : (_value0 > 100)", ScriptType.INLINE, + null, null)))).execute().actionGet(); + + assertSearchResponse(response); + + InternalHistogram histo = response.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + + for (int i = 0; i < buckets.size(); ++i) { + Histogram.Bucket bucket = buckets.get(i); + Sum field2Sum = bucket.getAggregations().get("field2Sum"); + assertThat(field2Sum, notNullValue()); + double field2SumValue = field2Sum.getValue(); + assertThat(field2SumValue, greaterThan(100.0)); + } + } + + @Test + public void inlineScriptNamedVars() { + + Map bucketPathsMap = new HashMap<>(); + bucketPathsMap.put("my_value1", "field2Sum"); + bucketPathsMap.put("my_value2", "field3Sum"); + + SearchResponse response = client() + .prepareSearch("idx") + .addAggregation( + histogram("histo") + .field(FIELD_1_NAME) + .interval(interval) + .subAggregation(sum("field2Sum").field(FIELD_2_NAME)) + .subAggregation(sum("field3Sum").field(FIELD_3_NAME)) + .subAggregation( + having("having").setBucketsPathsMap(bucketPathsMap).script( + new Script("Double.isNaN(my_value1) ? false : (my_value1 + my_value2 > 100)", + ScriptType.INLINE, null, null)))).execute() + .actionGet(); + + assertSearchResponse(response); + + InternalHistogram histo = response.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + + for (int i = 0; i < buckets.size(); ++i) { + Histogram.Bucket bucket = buckets.get(i); + Sum field2Sum = bucket.getAggregations().get("field2Sum"); + assertThat(field2Sum, notNullValue()); + double field2SumValue = field2Sum.getValue(); + Sum field3Sum = bucket.getAggregations().get("field3Sum"); + assertThat(field3Sum, notNullValue()); + double field3SumValue = field3Sum.getValue(); + assertThat(field2SumValue + field3SumValue, greaterThan(100.0)); + } + } + + @Test + public void inlineScriptWithParams() { + + Map params = new HashMap<>(); + params.put("threshold", 100); + SearchResponse response = client() + .prepareSearch("idx") + .addAggregation( + histogram("histo") + .field(FIELD_1_NAME) + .interval(interval) + .subAggregation(sum("field2Sum").field(FIELD_2_NAME)) + .subAggregation(sum("field3Sum").field(FIELD_3_NAME)) + .subAggregation( + having("having").setBucketsPaths("field2Sum", "field3Sum").script( + new Script("Double.isNaN(_value0) ? false : (_value0 + _value1 > threshold)", + ScriptType.INLINE, null, params)))).execute() + .actionGet(); + + assertSearchResponse(response); + + InternalHistogram histo = response.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + + for (int i = 0; i < buckets.size(); ++i) { + Histogram.Bucket bucket = buckets.get(i); + Sum field2Sum = bucket.getAggregations().get("field2Sum"); + assertThat(field2Sum, notNullValue()); + double field2SumValue = field2Sum.getValue(); + Sum field3Sum = bucket.getAggregations().get("field3Sum"); + assertThat(field3Sum, notNullValue()); + double field3SumValue = field3Sum.getValue(); + assertThat(field2SumValue + field3SumValue, greaterThan(100.0)); + } + } + + @Test + public void inlineScriptInsertZeros() { + + SearchResponse response = client() + .prepareSearch("idx") + .addAggregation( + histogram("histo") + .field(FIELD_1_NAME) + .interval(interval) + .subAggregation(sum("field2Sum").field(FIELD_2_NAME)) + .subAggregation(sum("field3Sum").field(FIELD_3_NAME)) + .subAggregation( + having("having").setBucketsPaths("field2Sum", "field3Sum").gapPolicy(GapPolicy.INSERT_ZEROS) + .script(new Script("_value0 + _value1 > 100", ScriptType.INLINE, null, null)))) + .execute().actionGet(); + + assertSearchResponse(response); + + InternalHistogram histo = response.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + + for (int i = 0; i < buckets.size(); ++i) { + Histogram.Bucket bucket = buckets.get(i); + Sum field2Sum = bucket.getAggregations().get("field2Sum"); + assertThat(field2Sum, notNullValue()); + double field2SumValue = field2Sum.getValue(); + Sum field3Sum = bucket.getAggregations().get("field3Sum"); + assertThat(field3Sum, notNullValue()); + double field3SumValue = field3Sum.getValue(); + assertThat(field2SumValue + field3SumValue, greaterThan(100.0)); + } + } + + @Test + public void indexedScript() { + + SearchResponse response = client() + .prepareSearch("idx") + .addAggregation( + histogram("histo") + .field(FIELD_1_NAME) + .interval(interval) + .subAggregation(sum("field2Sum").field(FIELD_2_NAME)) + .subAggregation(sum("field3Sum").field(FIELD_3_NAME)) + .subAggregation( + having("having").setBucketsPaths("field2Sum", "field3Sum").script( + new Script("my_script", ScriptType.INDEXED, null, null)))).execute().actionGet(); + + assertSearchResponse(response); + + InternalHistogram histo = response.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + + for (int i = 0; i < buckets.size(); ++i) { + Histogram.Bucket bucket = buckets.get(i); + Sum field2Sum = bucket.getAggregations().get("field2Sum"); + assertThat(field2Sum, notNullValue()); + double field2SumValue = field2Sum.getValue(); + Sum field3Sum = bucket.getAggregations().get("field3Sum"); + assertThat(field3Sum, notNullValue()); + double field3SumValue = field3Sum.getValue(); + assertThat(field2SumValue + field3SumValue, greaterThan(100.0)); + } + } + + @Test + public void unmapped() throws Exception { + + SearchResponse response = client() + .prepareSearch("idx_unmapped") + .addAggregation( + histogram("histo") + .field(FIELD_1_NAME) + .interval(interval) + .subAggregation(sum("field2Sum").field(FIELD_2_NAME)) + .subAggregation(sum("field3Sum").field(FIELD_3_NAME)) + .subAggregation( + having("having").setBucketsPaths("field2Sum", "field3Sum").script( + new Script("Double.isNaN(_value0) ? false : (_value0 + _value1 > 100)", ScriptType.INLINE, + null, null)))).execute() + .actionGet(); + + assertSearchResponse(response); + + InternalHistogram deriv = response.getAggregations().get("histo"); + assertThat(deriv, notNullValue()); + assertThat(deriv.getName(), equalTo("histo")); + assertThat(deriv.getBuckets().size(), equalTo(0)); + } + + @Test + public void partiallyUnmapped() throws Exception { + + SearchResponse response = client() + .prepareSearch("idx", "idx_unmapped") + .addAggregation( + histogram("histo") + .field(FIELD_1_NAME) + .interval(interval) + .subAggregation(sum("field2Sum").field(FIELD_2_NAME)) + .subAggregation(sum("field3Sum").field(FIELD_3_NAME)) + .subAggregation( + having("having").setBucketsPaths("field2Sum", "field3Sum").script( + new Script("Double.isNaN(_value0) ? false : (_value0 + _value1 > 100)", ScriptType.INLINE, + null, null)))).execute() + .actionGet(); + + assertSearchResponse(response); + + InternalHistogram histo = response.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + + for (int i = 0; i < buckets.size(); ++i) { + Histogram.Bucket bucket = buckets.get(i); + Sum field2Sum = bucket.getAggregations().get("field2Sum"); + assertThat(field2Sum, notNullValue()); + double field2SumValue = field2Sum.getValue(); + Sum field3Sum = bucket.getAggregations().get("field3Sum"); + assertThat(field3Sum, notNullValue()); + double field3SumValue = field3Sum.getValue(); + assertThat(field2SumValue + field3SumValue, greaterThan(100.0)); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/search/functionscore/DecayFunctionScoreTests.java b/core/src/test/java/org/elasticsearch/search/functionscore/DecayFunctionScoreTests.java index 51770fb925f22..4d4c9c71f0c8c 100644 --- a/core/src/test/java/org/elasticsearch/search/functionscore/DecayFunctionScoreTests.java +++ b/core/src/test/java/org/elasticsearch/search/functionscore/DecayFunctionScoreTests.java @@ -42,6 +42,7 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.ExecutionException; +import java.util.Locale; import static org.elasticsearch.client.Requests.indexRequest; import static org.elasticsearch.client.Requests.searchRequest; @@ -530,17 +531,17 @@ public void testDateWithoutOrigin() throws Exception { ensureYellow(); DateTime docDate = dt.minusDays(1); - String docDateString = docDate.getYear() + "-" + docDate.getMonthOfYear() + "-" + docDate.getDayOfMonth(); + String docDateString = docDate.getYear() + "-" + String.format(Locale.ROOT, "%02d", docDate.getMonthOfYear()) + "-" + String.format(Locale.ROOT, "%02d", docDate.getDayOfMonth()); client().index( indexRequest("test").type("type1").id("1") .source(jsonBuilder().startObject().field("test", "value").field("num1", docDateString).endObject())).actionGet(); docDate = dt.minusDays(2); - docDateString = docDate.getYear() + "-" + docDate.getMonthOfYear() + "-" + docDate.getDayOfMonth(); + docDateString = docDate.getYear() + "-" + String.format(Locale.ROOT, "%02d", docDate.getMonthOfYear()) + "-" + String.format(Locale.ROOT, "%02d", docDate.getDayOfMonth()); client().index( indexRequest("test").type("type1").id("2") .source(jsonBuilder().startObject().field("test", "value").field("num1", docDateString).endObject())).actionGet(); docDate = dt.minusDays(3); - docDateString = docDate.getYear() + "-" + docDate.getMonthOfYear() + "-" + docDate.getDayOfMonth(); + docDateString = docDate.getYear() + "-" + String.format(Locale.ROOT, "%02d", docDate.getMonthOfYear()) + "-" + String.format(Locale.ROOT, "%02d", docDate.getDayOfMonth()); client().index( indexRequest("test").type("type1").id("3") .source(jsonBuilder().startObject().field("test", "value").field("num1", docDateString).endObject())).actionGet(); diff --git a/core/src/test/java/org/elasticsearch/search/stats/SearchStatsTests.java b/core/src/test/java/org/elasticsearch/search/stats/SearchStatsTests.java index a4d17090ed7a1..d4446fef52a79 100644 --- a/core/src/test/java/org/elasticsearch/search/stats/SearchStatsTests.java +++ b/core/src/test/java/org/elasticsearch/search/stats/SearchStatsTests.java @@ -33,11 +33,13 @@ import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.search.stats.SearchStats.Stats; import org.elasticsearch.script.Script; +import org.elasticsearch.search.SearchHit; import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.junit.Test; import java.util.HashSet; import java.util.Set; +import java.util.concurrent.TimeUnit; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS; @@ -178,7 +180,27 @@ public void testOpenContexts() { assertSearchResponse(searchResponse); indicesStats = client().admin().indices().prepareStats().execute().actionGet(); - assertThat(indicesStats.getTotal().getSearch().getOpenContexts(), equalTo((long)numAssignedShards("test1"))); + assertThat(indicesStats.getTotal().getSearch().getOpenContexts(), equalTo((long) numAssignedShards("test1"))); + assertThat(indicesStats.getTotal().getSearch().getTotal().getScrollCurrent(), equalTo((long) numAssignedShards("test1"))); + + // force the scan to complete measuring the time taken + // the total time the scroll is open should be greater than this + // the number of queries should equal the number of pages in the scan times the number of shards + int count = 0; + while (true) { + count++; + searchResponse = client().prepareSearchScroll(searchResponse.getScrollId()) + .setScroll(TimeValue.timeValueMinutes(2)) + .execute().actionGet(); + if (searchResponse.getHits().getHits().length == 0) { + break; + } + } + indicesStats = client().admin().indices().prepareStats().execute().actionGet(); + Stats stats = indicesStats.getTotal().getSearch().getTotal(); + assertThat(stats.getQueryCount(), equalTo(count * (long)numAssignedShards("test1"))); + assertThat(stats.getScrollCount(), equalTo((long)numAssignedShards("test1"))); + assertThat(stats.getScrollTimeInMillis(), greaterThan(0l)); // scroll, but with no timeout (so no context) searchResponse = client().prepareSearchScroll(searchResponse.getScrollId()).execute().actionGet(); diff --git a/core/src/test/java/org/elasticsearch/search/stats/SearchStatsUnitTests.java b/core/src/test/java/org/elasticsearch/search/stats/SearchStatsUnitTests.java index a9f134b1be62d..7128991b24fad 100644 --- a/core/src/test/java/org/elasticsearch/search/stats/SearchStatsUnitTests.java +++ b/core/src/test/java/org/elasticsearch/search/stats/SearchStatsUnitTests.java @@ -35,9 +35,9 @@ public void testShardLevelSearchGroupStats() throws Exception { // let's create two dummy search stats with groups Map groupStats1 = new HashMap<>(); Map groupStats2 = new HashMap<>(); - groupStats2.put("group1", new Stats(1, 1, 1, 1, 1, 1)); - SearchStats searchStats1 = new SearchStats(new Stats(1, 1, 1, 1, 1, 1), 0, groupStats1); - SearchStats searchStats2 = new SearchStats(new Stats(1, 1, 1, 1, 1, 1), 0, groupStats2); + groupStats2.put("group1", new Stats(1, 1, 1, 1, 1, 1, 1, 1, 1)); + SearchStats searchStats1 = new SearchStats(new Stats(1, 1, 1, 1, 1, 1, 1, 1, 1), 0, groupStats1); + SearchStats searchStats2 = new SearchStats(new Stats(1, 1, 1, 1, 1, 1, 1, 1, 1), 0, groupStats2); // adding these two search stats and checking group stats are correct searchStats1.add(searchStats2); @@ -62,5 +62,8 @@ private void assertStats(Stats stats, long equalTo) { assertEquals(equalTo, stats.getFetchCount()); assertEquals(equalTo, stats.getFetchTimeInMillis()); assertEquals(equalTo, stats.getFetchCurrent()); + assertEquals(equalTo, stats.getScrollCount()); + assertEquals(equalTo, stats.getScrollTimeInMillis()); + assertEquals(equalTo, stats.getScrollCurrent()); } } \ No newline at end of file diff --git a/core/src/test/java/org/elasticsearch/search/suggest/completion/CompletionPostingsFormatTest.java b/core/src/test/java/org/elasticsearch/search/suggest/completion/CompletionPostingsFormatTest.java index bdf3e2f541afa..1cd9745c49dd6 100644 --- a/core/src/test/java/org/elasticsearch/search/suggest/completion/CompletionPostingsFormatTest.java +++ b/core/src/test/java/org/elasticsearch/search/suggest/completion/CompletionPostingsFormatTest.java @@ -24,7 +24,6 @@ import org.apache.lucene.analysis.standard.StandardAnalyzer; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.FieldsConsumer; -import org.apache.lucene.codecs.FilterCodec; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.lucene50.Lucene50Codec; import org.apache.lucene.document.Document; @@ -52,13 +51,10 @@ import org.apache.lucene.util.LineFileDocs; import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.analysis.NamedAnalyzer; -import org.elasticsearch.index.codec.postingsformat.Elasticsearch090PostingsFormat; -import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.MappedFieldType.Names; -import org.elasticsearch.index.mapper.core.AbstractFieldMapper; import org.elasticsearch.index.mapper.core.CompletionFieldMapper; import org.elasticsearch.search.suggest.SuggestUtils; import org.elasticsearch.search.suggest.completion.Completion090PostingsFormat.LookupFactory; @@ -249,7 +245,7 @@ public boolean hasContexts() { AnalyzingCompletionLookupProvider currentProvider = new AnalyzingCompletionLookupProvider(preserveSeparators, false, preservePositionIncrements, usePayloads); CompletionFieldMapper.CompletionFieldType fieldType = FIELD_TYPE.clone(); fieldType.setProvider(currentProvider); - final CompletionFieldMapper mapper = new CompletionFieldMapper("foo", fieldType, Integer.MAX_VALUE, indexSettings, AbstractFieldMapper.MultiFields.empty(), null); + final CompletionFieldMapper mapper = new CompletionFieldMapper("foo", fieldType, Integer.MAX_VALUE, indexSettings, FieldMapper.MultiFields.empty(), null); Lookup buildAnalyzingLookup = buildAnalyzingLookup(mapper, titles, titles, weights); Field field = buildAnalyzingLookup.getClass().getDeclaredField("maxAnalyzedPathsForOneInput"); field.setAccessible(true); diff --git a/core/src/test/java/org/elasticsearch/stresstest/leaks/GenericStatsLeak.java b/core/src/test/java/org/elasticsearch/stresstest/leaks/GenericStatsLeak.java index 3ea972dce4e39..03d0ca76d0444 100644 --- a/core/src/test/java/org/elasticsearch/stresstest/leaks/GenericStatsLeak.java +++ b/core/src/test/java/org/elasticsearch/stresstest/leaks/GenericStatsLeak.java @@ -21,7 +21,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.monitor.jvm.JvmService; -import org.elasticsearch.monitor.network.NetworkService; import org.elasticsearch.monitor.os.OsService; import org.elasticsearch.monitor.process.ProcessService; import org.elasticsearch.node.Node; @@ -39,13 +38,11 @@ public static void main(String[] args) { JvmService jvmService = node.injector().getInstance(JvmService.class); OsService osService = node.injector().getInstance(OsService.class); ProcessService processService = node.injector().getInstance(ProcessService.class); - NetworkService networkService = node.injector().getInstance(NetworkService.class); while (true) { jvmService.stats(); osService.stats(); processService.stats(); - networkService.stats(); } } } \ No newline at end of file diff --git a/core/src/test/java/org/elasticsearch/test/ElasticsearchTestCase.java b/core/src/test/java/org/elasticsearch/test/ElasticsearchTestCase.java index b3252ea995d16..4bfafb266fcb7 100644 --- a/core/src/test/java/org/elasticsearch/test/ElasticsearchTestCase.java +++ b/core/src/test/java/org/elasticsearch/test/ElasticsearchTestCase.java @@ -435,18 +435,20 @@ public static boolean awaitBusy(Predicate breakPredicate) throws InterruptedE return awaitBusy(breakPredicate, 10, TimeUnit.SECONDS); } + // After 1s, we stop growing the sleep interval exponentially and just sleep 1s until maxWaitTime + private static final long AWAIT_BUSY_THRESHOLD = 1000L; + public static boolean awaitBusy(Predicate breakPredicate, long maxWaitTime, TimeUnit unit) throws InterruptedException { long maxTimeInMillis = TimeUnit.MILLISECONDS.convert(maxWaitTime, unit); - long iterations = Math.max(Math.round(Math.log10(maxTimeInMillis) / Math.log10(2)), 1); long timeInMillis = 1; long sum = 0; - for (int i = 0; i < iterations; i++) { + while (sum + timeInMillis < maxTimeInMillis) { if (breakPredicate.apply(null)) { return true; } - sum += timeInMillis; Thread.sleep(timeInMillis); - timeInMillis *= 2; + sum += timeInMillis; + timeInMillis = Math.min(AWAIT_BUSY_THRESHOLD, timeInMillis * 2); } timeInMillis = maxTimeInMillis - sum; Thread.sleep(Math.max(timeInMillis, 0)); diff --git a/core/src/test/java/org/elasticsearch/test/InternalTestCluster.java b/core/src/test/java/org/elasticsearch/test/InternalTestCluster.java index 18e5c9ce08327..c1bbd9e796f3f 100644 --- a/core/src/test/java/org/elasticsearch/test/InternalTestCluster.java +++ b/core/src/test/java/org/elasticsearch/test/InternalTestCluster.java @@ -220,7 +220,7 @@ public InternalTestCluster(long clusterSeed, Path baseDir, int minNumDataNodes, public InternalTestCluster(long clusterSeed, Path baseDir, int minNumDataNodes, int maxNumDataNodes, String clusterName, SettingsSource settingsSource, int numClientNodes, - boolean enableHttpPipelining, String nodePrefix) { + boolean enableHttpPipelining, String nodePrefix) { super(clusterSeed); this.baseDir = baseDir; this.clusterName = clusterName; @@ -281,8 +281,8 @@ public InternalTestCluster(long clusterSeed, Path baseDir, } builder.put("path.home", baseDir); builder.put("path.repo", baseDir.resolve("repos")); - builder.put("transport.tcp.port", BASE_PORT + "-" + (BASE_PORT+100)); - builder.put("http.port", BASE_PORT+101 + "-" + (BASE_PORT+200)); + builder.put("transport.tcp.port", BASE_PORT + "-" + (BASE_PORT + 100)); + builder.put("http.port", BASE_PORT + 101 + "-" + (BASE_PORT + 200)); builder.put(InternalSettingsPreparer.IGNORE_SYSTEM_PROPERTIES_SETTING, true); builder.put("node.mode", NODE_MODE); builder.put("http.pipelining", enableHttpPipelining); @@ -1566,20 +1566,7 @@ public void clearDisruptionScheme() { if (activeDisruptionScheme != null) { TimeValue expectedHealingTime = activeDisruptionScheme.expectedTimeToHeal(); logger.info("Clearing active scheme {}, expected healing time {}", activeDisruptionScheme, expectedHealingTime); - activeDisruptionScheme.removeFromCluster(this); - // We don't what scheme is picked, certain schemes don't partition the cluster, but process slow, so we need - // to to sleep, cluster health alone doesn't verify if these schemes have been cleared. - if (expectedHealingTime != null && expectedHealingTime.millis() > 0) { - try { - Thread.sleep(expectedHealingTime.millis()); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } - } - assertFalse("cluster failed to form after disruption was healed", client().admin().cluster().prepareHealth() - .setWaitForNodes("" + nodes.size()) - .setWaitForRelocatingShards(0) - .get().isTimedOut()); + activeDisruptionScheme.removeAndEnsureHealthy(this); } activeDisruptionScheme = null; } diff --git a/core/src/test/java/org/elasticsearch/test/disruption/BlockClusterStateProcessing.java b/core/src/test/java/org/elasticsearch/test/disruption/BlockClusterStateProcessing.java index d1972688106a7..c8a0820eac24f 100644 --- a/core/src/test/java/org/elasticsearch/test/disruption/BlockClusterStateProcessing.java +++ b/core/src/test/java/org/elasticsearch/test/disruption/BlockClusterStateProcessing.java @@ -23,6 +23,7 @@ import org.elasticsearch.cluster.ClusterStateNonMasterUpdateTask; import org.elasticsearch.common.Priority; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.test.InternalTestCluster; import java.util.Random; import java.util.concurrent.CountDownLatch; @@ -89,6 +90,11 @@ public void stopDisrupting() { } + @Override + public void removeAndEnsureHealthy(InternalTestCluster cluster) { + removeFromCluster(cluster); + } + @Override public TimeValue expectedTimeToHeal() { return TimeValue.timeValueMinutes(0); diff --git a/core/src/test/java/org/elasticsearch/test/disruption/LongGCDisruption.java b/core/src/test/java/org/elasticsearch/test/disruption/LongGCDisruption.java index de4532269e4cf..ad84c0a98cc04 100644 --- a/core/src/test/java/org/elasticsearch/test/disruption/LongGCDisruption.java +++ b/core/src/test/java/org/elasticsearch/test/disruption/LongGCDisruption.java @@ -21,12 +21,15 @@ import org.elasticsearch.common.SuppressForbidden; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.test.InternalTestCluster; import java.util.HashSet; import java.util.Random; import java.util.Set; import java.util.regex.Pattern; +import static org.junit.Assert.assertFalse; + /** * Suspends all threads on the specified node in order to simulate a long gc. */ @@ -63,6 +66,12 @@ public synchronized void stopDisrupting() { } } + @Override + public void removeAndEnsureHealthy(InternalTestCluster cluster) { + removeFromCluster(cluster); + ensureNodeCount(cluster); + } + @Override public TimeValue expectedTimeToHeal() { return TimeValue.timeValueMillis(0); diff --git a/core/src/test/java/org/elasticsearch/test/disruption/NetworkPartition.java b/core/src/test/java/org/elasticsearch/test/disruption/NetworkPartition.java index 8c379b0a0084d..174e83e15a439 100644 --- a/core/src/test/java/org/elasticsearch/test/disruption/NetworkPartition.java +++ b/core/src/test/java/org/elasticsearch/test/disruption/NetworkPartition.java @@ -32,6 +32,8 @@ import java.util.Random; import java.util.Set; +import static org.junit.Assert.assertFalse; + public abstract class NetworkPartition implements ServiceDisruptionScheme { protected final ESLogger logger = Loggers.getLogger(getClass()); @@ -105,6 +107,19 @@ public void removeFromCluster(InternalTestCluster cluster) { stopDisrupting(); } + @Override + public void removeAndEnsureHealthy(InternalTestCluster cluster) { + removeFromCluster(cluster); + ensureNodeCount(cluster); + } + + protected void ensureNodeCount(InternalTestCluster cluster) { + assertFalse("cluster failed to form after disruption was healed", cluster.client().admin().cluster().prepareHealth() + .setWaitForNodes("" + cluster.size()) + .setWaitForRelocatingShards(0) + .get().isTimedOut()); + } + @Override public synchronized void applyToNode(String node, InternalTestCluster cluster) { if (!autoExpand || nodesSideOne.contains(node) || nodesSideTwo.contains(node)) { diff --git a/core/src/test/java/org/elasticsearch/test/disruption/NoOpDisruptionScheme.java b/core/src/test/java/org/elasticsearch/test/disruption/NoOpDisruptionScheme.java index 7b348b1afea51..06bef2105edb2 100644 --- a/core/src/test/java/org/elasticsearch/test/disruption/NoOpDisruptionScheme.java +++ b/core/src/test/java/org/elasticsearch/test/disruption/NoOpDisruptionScheme.java @@ -59,6 +59,11 @@ public void testClusterClosed() { } + @Override + public void removeAndEnsureHealthy(InternalTestCluster cluster) { + + } + @Override public TimeValue expectedTimeToHeal() { return TimeValue.timeValueSeconds(0); diff --git a/core/src/test/java/org/elasticsearch/test/disruption/ServiceDisruptionScheme.java b/core/src/test/java/org/elasticsearch/test/disruption/ServiceDisruptionScheme.java index 70774a823568a..b5f3bcacbd647 100644 --- a/core/src/test/java/org/elasticsearch/test/disruption/ServiceDisruptionScheme.java +++ b/core/src/test/java/org/elasticsearch/test/disruption/ServiceDisruptionScheme.java @@ -23,20 +23,22 @@ public interface ServiceDisruptionScheme { - public void applyToCluster(InternalTestCluster cluster); + void applyToCluster(InternalTestCluster cluster); - public void removeFromCluster(InternalTestCluster cluster); + void removeFromCluster(InternalTestCluster cluster); - public void applyToNode(String node, InternalTestCluster cluster); + void removeAndEnsureHealthy(InternalTestCluster cluster); - public void removeFromNode(String node, InternalTestCluster cluster); + void applyToNode(String node, InternalTestCluster cluster); - public void startDisrupting(); + void removeFromNode(String node, InternalTestCluster cluster); - public void stopDisrupting(); + void startDisrupting(); - public void testClusterClosed(); + void stopDisrupting(); - public TimeValue expectedTimeToHeal(); + void testClusterClosed(); + + TimeValue expectedTimeToHeal(); } diff --git a/core/src/test/java/org/elasticsearch/test/disruption/SingleNodeDisruption.java b/core/src/test/java/org/elasticsearch/test/disruption/SingleNodeDisruption.java index 3148254011ef2..f74280c014a97 100644 --- a/core/src/test/java/org/elasticsearch/test/disruption/SingleNodeDisruption.java +++ b/core/src/test/java/org/elasticsearch/test/disruption/SingleNodeDisruption.java @@ -24,6 +24,8 @@ import java.util.Random; +import static org.junit.Assert.assertFalse; + public abstract class SingleNodeDisruption implements ServiceDisruptionScheme { protected final ESLogger logger = Loggers.getLogger(getClass()); @@ -80,4 +82,10 @@ public synchronized void testClusterClosed() { disruptedNode = null; } + protected void ensureNodeCount(InternalTestCluster cluster) { + assertFalse("cluster failed to form after disruption was healed", cluster.client().admin().cluster().prepareHealth() + .setWaitForNodes("" + cluster.size()) + .setWaitForRelocatingShards(0) + .get().isTimedOut()); + } } diff --git a/core/src/test/java/org/elasticsearch/test/disruption/SlowClusterStateProcessing.java b/core/src/test/java/org/elasticsearch/test/disruption/SlowClusterStateProcessing.java index 746d7f942baa8..430332acaa223 100644 --- a/core/src/test/java/org/elasticsearch/test/disruption/SlowClusterStateProcessing.java +++ b/core/src/test/java/org/elasticsearch/test/disruption/SlowClusterStateProcessing.java @@ -23,6 +23,7 @@ import org.elasticsearch.cluster.ClusterStateNonMasterUpdateTask; import org.elasticsearch.common.Priority; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.test.InternalTestCluster; import java.util.Random; import java.util.concurrent.CountDownLatch; @@ -132,6 +133,12 @@ public void onFailure(String source, Throwable t) { return true; } + @Override + public void removeAndEnsureHealthy(InternalTestCluster cluster) { + removeFromCluster(cluster); + ensureNodeCount(cluster); + } + @Override public TimeValue expectedTimeToHeal() { return TimeValue.timeValueMillis(0); diff --git a/core/src/main/java/org/elasticsearch/monitor/network/NetworkProbe.java b/core/src/test/java/org/elasticsearch/test/rest/RestIT.java similarity index 55% rename from core/src/main/java/org/elasticsearch/monitor/network/NetworkProbe.java rename to core/src/test/java/org/elasticsearch/test/rest/RestIT.java index 7ff0293f5e3fd..167a5f04f9230 100644 --- a/core/src/main/java/org/elasticsearch/monitor/network/NetworkProbe.java +++ b/core/src/test/java/org/elasticsearch/test/rest/RestIT.java @@ -17,16 +17,22 @@ * under the License. */ -package org.elasticsearch.monitor.network; +package org.elasticsearch.test.rest; -/** - * - */ -public interface NetworkProbe { +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; - NetworkInfo networkInfo(); +import org.elasticsearch.test.rest.parser.RestTestParseException; - NetworkStats networkStats(); +import java.io.IOException; - String ifconfig(); +/** Rest integration test. runs against external cluster in 'mvn verify' */ +public class RestIT extends ElasticsearchRestTestCase { + public RestIT(RestTestCandidate testCandidate) { + super(testCandidate); + } + // we run them all sequentially: start simple! + @ParametersFactory + public static Iterable parameters() throws IOException, RestTestParseException { + return createParameters(0, 1); + } } diff --git a/core/src/test/java/org/elasticsearch/ttl/SimpleTTLTests.java b/core/src/test/java/org/elasticsearch/ttl/SimpleTTLTests.java index 573c0090fc381..e945769275aed 100644 --- a/core/src/test/java/org/elasticsearch/ttl/SimpleTTLTests.java +++ b/core/src/test/java/org/elasticsearch/ttl/SimpleTTLTests.java @@ -39,7 +39,7 @@ import static org.elasticsearch.common.settings.Settings.settingsBuilder; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; -import static org.elasticsearch.test.ElasticsearchIntegrationTest.*; +import static org.elasticsearch.test.ElasticsearchIntegrationTest.Scope; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.*; diff --git a/core/src/test/resources/org/elasticsearch/bootstrap/duplicate-classes.jar b/core/src/test/resources/org/elasticsearch/bootstrap/duplicate-classes.jar new file mode 100644 index 0000000000000..a7d9c55bff308 Binary files /dev/null and b/core/src/test/resources/org/elasticsearch/bootstrap/duplicate-classes.jar differ diff --git a/core/src/test/resources/org/elasticsearch/bootstrap/duplicate-xmlbeans-classes.jar b/core/src/test/resources/org/elasticsearch/bootstrap/duplicate-xmlbeans-classes.jar new file mode 100644 index 0000000000000..a5d3abca13e2b Binary files /dev/null and b/core/src/test/resources/org/elasticsearch/bootstrap/duplicate-xmlbeans-classes.jar differ diff --git a/dev-tools/build_release.py b/dev-tools/build_release.py index 32f7b703aa2e2..526fe335c49c6 100644 --- a/dev-tools/build_release.py +++ b/dev-tools/build_release.py @@ -466,7 +466,7 @@ def smoke_test_release(release, files, expected_hash, plugins): if version['build_hash'].strip() != expected_hash: raise RuntimeError('HEAD hash does not match expected [%s] but got [%s]' % (expected_hash, version['build_hash'])) print(' Running REST Spec tests against package [%s]' % release_file) - run_mvn('test -Dtests.cluster=%s -Dtests.class=*.*RestTests' % ("127.0.0.1:9300")) + run_mvn('test -Dtests.cluster=%s -Dtests.jvms=1 -Dtests.class=*.*RestTests' % ("127.0.0.1:9300")) print(' Verify if plugins are listed in _nodes') conn.request('GET', '/_nodes?plugin=true&pretty=true') res = conn.getresponse() diff --git a/dev-tools/pom.xml b/dev-tools/pom.xml index bebf7d45cfc51..0daa4cbad51ab 100644 --- a/dev-tools/pom.xml +++ b/dev-tools/pom.xml @@ -1,7 +1,7 @@ 4.0.0 org.elasticsearch - dev-tools + elasticsearch-dev-tools 2.0.0-SNAPSHOT Elasticsearch Build Resources diff --git a/dev-tools/src/main/resources/ant/integration-tests.xml b/dev-tools/src/main/resources/ant/integration-tests.xml new file mode 100644 index 0000000000000..82f3749742515 --- /dev/null +++ b/dev-tools/src/main/resources/ant/integration-tests.xml @@ -0,0 +1,139 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Starting up external cluster... + + + + + + + + External cluster started PID ${integ.pid} + + + + + + + + + + + + + + + + + + Installing plugin ${project.artifactId}... + + + + Starting up external cluster... + + + + + + + + External cluster started PID ${integ.pid} + + + + + + + + Shutting down external cluster PID ${integ.pid} + + + + + + + + + + + + + diff --git a/dev-tools/src/main/resources/license-check/check_license_and_sha.pl b/dev-tools/src/main/resources/license-check/check_license_and_sha.pl index 40e78d1debd74..9a865afde532d 100755 --- a/dev-tools/src/main/resources/license-check/check_license_and_sha.pl +++ b/dev-tools/src/main/resources/license-check/check_license_and_sha.pl @@ -41,7 +41,7 @@ sub check_shas_and_licenses { } unless ( $old_sha eq $new{$jar} ) { - say STDERR "$jar: SHA has changed"; + say STDERR "$jar: SHA has changed, expected $old_sha but found $new{$jar}"; $error++; $sha_error++; next; diff --git a/docs/reference/aggregations/pipeline.asciidoc b/docs/reference/aggregations/pipeline.asciidoc index b445cbb0de123..c6bc98926ae41 100644 --- a/docs/reference/aggregations/pipeline.asciidoc +++ b/docs/reference/aggregations/pipeline.asciidoc @@ -134,22 +134,26 @@ count of each bucket, instead of a specific metric: [float] === Dealing with gaps in the data -There are a couple of reasons why the data output by the enclosing histogram may have gaps: +Data in the real world is often noisy and sometimes contains *gaps* -- places where data simply doesn't exist. This can +occur for a variety of reasons, the most common being: -* There are no documents matching the query for some buckets -* The data for a metric is missing in all of the documents falling into a bucket (this is most likely with either a small interval -on the enclosing histogram or with a query matching only a small number of documents) +* Documents falling into a bucket do not contain a required field +* There are no documents matching the query for one or more buckets +* The metric being calculated is unable to generate a value, likely because another dependent bucket is missing a value. +Some pipeline aggregations have specific requirements that must be met (e.g. a derivative cannot calculate a metric for the +first value because there is no previous value, HoltWinters moving average need "warmup" data to begin calculating, etc) -Where there is no data available in a bucket for a given metric it presents a problem for calculating the derivative value for both -the current bucket and the next bucket. In the derivative pipeline aggregation has a `gap policy` parameter to define what the behavior -should be when a gap in the data is found. There are currently two options for controlling the gap policy: +Gap policies are a mechanism to inform the pipeline aggregation about the desired behavior when "gappy" or missing +data is encountered. All pipeline aggregations accept the `gap_policy` parameter. There are currently two gap policies +to choose from: _skip_:: - This option will not produce a derivative value for any buckets where the value in the current or previous bucket is - missing + This option treats missing data as if the bucket does not exist. It will skip the bucket and continue + calculating using the next available value. _insert_zeros_:: - This option will assume the missing value is `0` and calculate the derivative with the value `0`. + This option will replace missing values with a zero (`0`) and pipeline aggregation computation will + proceed as normal. @@ -162,3 +166,4 @@ include::pipeline/sum-bucket-aggregation.asciidoc[] include::pipeline/movavg-aggregation.asciidoc[] include::pipeline/cumulative-sum-aggregation.asciidoc[] include::pipeline/bucket-script-aggregation.asciidoc[] +include::pipeline/bucket-selector-aggregation.asciidoc[] diff --git a/docs/reference/aggregations/pipeline/bucket-script-aggregation.asciidoc b/docs/reference/aggregations/pipeline/bucket-script-aggregation.asciidoc index 6e5cb6d0c2a95..3618138d6efd7 100644 --- a/docs/reference/aggregations/pipeline/bucket-script-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/bucket-script-aggregation.asciidoc @@ -72,7 +72,7 @@ The following snippet calculates the ratio percentage of t-shirt sales compared } }, "t-shirt-percentage": { - "series_arithmetic": { + "bucket_script": { "buckets_paths": { "tShirtSales": "t-shirts>sales", "totalSales": "total_sales" diff --git a/docs/reference/aggregations/pipeline/bucket-selector-aggregation.asciidoc b/docs/reference/aggregations/pipeline/bucket-selector-aggregation.asciidoc new file mode 100644 index 0000000000000..ac29f21bdd973 --- /dev/null +++ b/docs/reference/aggregations/pipeline/bucket-selector-aggregation.asciidoc @@ -0,0 +1,107 @@ +[[search-aggregations-pipeline-bucket-selector-aggregation]] +=== Bucket Selector Aggregation + +coming[2.0.0] + +experimental[] + +A parent pipeline aggregation which executes a script which determines whether the current bucket will be retained +in the parent multi-bucket aggregation. The specified metric must be numeric and the script must return a boolean value. +If the script language is `expression` then a numeric return value is permitted. In this case 0.0 will be evaluated as `false` +and all other values will evaluate to true. + +Note: The bucket_selector aggregation, like all pipeline aggregations, executions after all other sibling aggregations. This means that +using the bucket_selector aggregation to filter the returned buckets in the response does not save on execution time running the aggregations. + +==== Syntax + +A `bucket_selector` aggregation looks like this in isolation: + +[source,js] +-------------------------------------------------- +{ + "bucket_selector": { + "buckets_path": { + "my_var1": "the_sum", <1> + "my_var2": "the_value_count" + }, + script: "my_var1 > my_var2" + } +} +-------------------------------------------------- +<1> Here, `my_var1` is the name of the variable for this buckets path to use in the script, `the_sum` is the path to +the metrics to use for that variable. + + +.`bucket_selector` Parameters +|=== +|Parameter Name |Description |Required |Default Value +|`script` |The script to run for this aggregation. The script can be inline, file or indexed. (see <> +for more details) |Required | +|`buckets_path` |A map of script variables and their associated path to the buckets we wish to use for the variable +(see <> for more details) |Required | + |`gap_policy` |The policy to apply when gaps are found in the data (see <> for more + details)|Optional, defaults to `skip` | +|=== + +The following snippet only retains buckets where the total sales for the month is less than or equal to 50: + +[source,js] +-------------------------------------------------- +{ + "aggs" : { + "sales_per_month" : { + "date_histogram" : { + "field" : "date", + "interval" : "month" + }, + "aggs": { + "total_sales": { + "sum": { + "field": "price" + } + } + "sales_bucket_filter": { + "bucket_selector": { + "buckets_paths": { + "totalSales": "total_sales" + }, + "script": "totalSales <= 50" + } + } + } + } + } +} +-------------------------------------------------- + +And the following may be the response: + +[source,js] +-------------------------------------------------- +{ + "aggregations": { + "sales_per_month": { + "buckets": [ + { + "key_as_string": "2015/01/01 00:00:00", + "key": 1420070400000, + "doc_count": 3, + "total_sales": { + "value": 50 + } + },<1> + { + "key_as_string": "2015/03/01 00:00:00", + "key": 1425168000000, + "doc_count": 2, + "total_sales": { + "value": 40 + }, + } + ] + } + } +} +-------------------------------------------------- +<1> Bucket for `2015/02/01 00:00:00` has been removed as its total sales exceeded 50 diff --git a/docs/reference/cluster/nodes-info.asciidoc b/docs/reference/cluster/nodes-info.asciidoc index 61d8c1a5a6a1f..07a2360acbee2 100644 --- a/docs/reference/cluster/nodes-info.asciidoc +++ b/docs/reference/cluster/nodes-info.asciidoc @@ -17,7 +17,7 @@ The second command selectively retrieves nodes information of only By default, it just returns all attributes and core settings for a node. It also allows to get only information on `settings`, `os`, `process`, `jvm`, -`thread_pool`, `network`, `transport`, `http` and `plugins`: +`thread_pool`, `transport`, `http` and `plugins`: [source,js] -------------------------------------------------- diff --git a/docs/reference/cluster/nodes-stats.asciidoc b/docs/reference/cluster/nodes-stats.asciidoc index 81d027f8779d9..ce65c15153e82 100644 --- a/docs/reference/cluster/nodes-stats.asciidoc +++ b/docs/reference/cluster/nodes-stats.asciidoc @@ -19,7 +19,7 @@ second command selectively retrieves nodes stats of only `nodeId1` and <>. By default, all stats are returned. You can limit this by combining any -of `indices`, `os`, `process`, `jvm`, `network`, `transport`, `http`, +of `indices`, `os`, `process`, `jvm`, `transport`, `http`, `fs`, `breaker` and `thread_pool`. For example: [horizontal] @@ -29,7 +29,7 @@ of `indices`, `os`, `process`, `jvm`, `network`, `transport`, `http`, `fs`:: File system information, data path, free disk space, read/write - stats + stats (see <>) `http`:: HTTP connection information @@ -38,9 +38,6 @@ of `indices`, `os`, `process`, `jvm`, `network`, `transport`, `http`, JVM stats, memory pool information, garbage collection, buffer pools -`network`:: - TCP information - `os`:: Operating system stats, load average, cpu, mem, swap @@ -72,6 +69,52 @@ curl -XGET 'http://localhost:9200/_nodes/10.0.0.1/stats/process' The `all` flag can be set to return all the stats. +[float] +[[fs-info]] +==== FS information + +The `fs` flag can be set to retrieve +information that concern the file system: + +`fs.timestamp`:: + Last time the file stores statistics have been refreshed + +`os.total.total_in_bytes`:: + Total size (in bytes) of all file stores + +`os.total.free_in_bytes`:: + Total number of unallocated bytes in all file stores + +`os.total.available_in_bytes`:: + Totalnumber of bytes available to this Java virtual machine on all file stores + +`os.data`:: + List of all file stores + +`os.data.path`:: + Path to the file store + +`os.data.mount`:: + Mount point of the file store (ex: /dev/sda2) + +`os.data.type`:: + Type of the file store (ex: ext4) + +`os.data.total_in_bytes`:: + Total size (in bytes) of thefile store + +`os.data.free_in_bytes`:: + Total number of unallocated bytes in the file store + +`os.data.available_in_bytes`:: + Totalnumber of bytes available to this Java virtual machine on this file store + +`os.data.spins` (Linux only):: + Indicates if the file store is backed by spinning storage. + `null` means we could not determine it, `true` means the device possibly spins + and `false` means it does not (ex: solid-state disks). + + [float] [[field-data]] === Field data statistics diff --git a/docs/reference/index-modules/translog.asciidoc b/docs/reference/index-modules/translog.asciidoc index a3c04bcf0a8cc..31cc4f7bf9240 100644 --- a/docs/reference/index-modules/translog.asciidoc +++ b/docs/reference/index-modules/translog.asciidoc @@ -44,51 +44,66 @@ How long to wait before triggering a flush regardless of translog size. Defaults How often to check if a flush is needed, randomized between the interval value and 2x the interval value. Defaults to `5s`. + [float] === Translog settings -The translog itself is only persisted to disk when it is ++fsync++ed. Until -then, data recently written to the translog may only exist in the file system -cache and could potentially be lost in the event of hardware failure. +The data in the transaction log is only persisted to disk when the translog is +++fsync++ed and committed. In the event of hardware failure, any data written +since the previous translog commit will be lost. -The following <> settings +By default, Elasticsearch ++fsync++s and commits the translog every 5 seconds +and at the end of every <>, <>, +<>, or <> request. In fact, Elasticsearch +will only report success of an index, delete, update, or bulk request to the +client after the transaction log has been successfully ++fsync++ed and committed +on the primary and on every allocated replica. + +The following <> per-index settings control the behaviour of the transaction log: `index.translog.sync_interval`:: -How often the translog is ++fsync++ed to disk. Defaults to `5s`. Can be set to -`0` to sync after each operation. +How often the translog is ++fsync++ed to disk and committed, regardless of +write operations. Defaults to `5s`. -`index.translog.fs.type`:: +`index.translog.durability`:: ++ +-- + +Whether or not to `fsync` and commit the translog after every index, delete, +update, or bulk request. This setting accepts the following parameters: -Either a `buffered` translog (default) which buffers 64kB in memory before -writing to disk, or a `simple` translog which writes every entry to disk -immediately. Whichever is used, these writes are only ++fsync++ed according -to the `sync_interval`. +`request`:: -The `buffered` translog is written to disk when it reaches 64kB in size, or -whenever a `sync` is triggered by the `sync_interval`. + (default) `fsync` and commit after every request. In the event + of hardware failure, all acknowledged writes will already have been + commited to disk. + +`async`:: + + `fsync` and commit in the background every `sync_interval`. In + the event of hardware failure, all acknowledged writes since the last + automatic commit will be discarded. +-- + +`index.translog.fs.type`:: ++ +-- -.Why don't we `fsync` the translog after every write? -****************************************************** +Whether to buffer writes to the transaction log in memory or not. This +setting accepts the following parameters: -The disk is the slowest part of any server. An `fsync` ensures that data in -the file system buffer has been physically written to disk, but this -persistence comes with a performance cost. +`buffered`:: -However, the translog is not the only persistence mechanism in Elasticsearch. -Any index or update request is first written to the primary shard, then -forwarded in parallel to any replica shards. The primary waits for the action -to be completed on the replicas before returning success to the client. + (default) Translog writes first go to a 64kB buffer in memory, + and are only written to the disk when the buffer is full, or when an + `fsync` is triggered by a write request or the `sync_interval`. -If the node holding the primary shard dies for some reason, its transaction -log could be missing the last 5 seconds of data. However, that data should -already be available on a replica shard on a different node. Of course, if -the whole data centre loses power at the same time, then it is possible that -you could lose the last 5 seconds (or `sync_interval`) of data. +`simple`:: -We are constantly monitoring the perfromance implications of better default -translog sync semantics, so the default might change as time passes and HW, -virtualization, and other aspects improve. + Translog writes are written to the file system immediately, without + buffering. However, these writes will only be persisted to disk when an + `fsync` and commit is triggered by a write request or the `sync_interval`. -****************************************************** \ No newline at end of file +-- diff --git a/docs/reference/mapping/date-format.asciidoc b/docs/reference/mapping/date-format.asciidoc index 0972340c555db..7f12616cc24c1 100644 --- a/docs/reference/mapping/date-format.asciidoc +++ b/docs/reference/mapping/date-format.asciidoc @@ -49,6 +49,13 @@ first millisecond of the rounding scope. The semantics work as follows: [[built-in]] === Built In Formats +Most of the below dates have a `strict` companion dates, which means, that +year, month and day parts of the week must have prepending zeros in order +to be valid. This means, that a date like `5/11/1` would not be valid, but +you would need to specify the full date, which would be `2005/11/01` in this +example. So instead of `date_optional_time` you would need to specify +`strict_date_optional_time`. + The following tables lists all the defaults ISO formats supported: [cols="<,<",options="header",] @@ -92,112 +99,125 @@ offset prefixed by 'T' ('T'HHmmssZ). |`basic_week_date`|A basic formatter for a full date as four digit weekyear, two digit week of weekyear, and one digit day of week -(xxxx'W'wwe). +(xxxx'W'wwe). `strict_basic_week_date` is supported. |`basic_week_date_time`|A basic formatter that combines a basic weekyear date and time, separated by a 'T' (xxxx'W'wwe'T'HHmmss.SSSZ). +`strict_basic_week_date_time` is supported. |`basic_week_date_time_no_millis`|A basic formatter that combines a basic weekyear date and time without millis, separated by a 'T' -(xxxx'W'wwe'T'HHmmssZ). +(xxxx'W'wwe'T'HHmmssZ). `strict_week_date_time` is supported. |`date`|A formatter for a full date as four digit year, two digit month -of year, and two digit day of month (yyyy-MM-dd). - +of year, and two digit day of month (yyyy-MM-dd). `strict_date` is supported. +_ |`date_hour`|A formatter that combines a full date and two digit hour of -day. +day. strict_date_hour` is supported. + |`date_hour_minute`|A formatter that combines a full date, two digit hour -of day, and two digit minute of hour. +of day, and two digit minute of hour. strict_date_hour_minute` is supported. |`date_hour_minute_second`|A formatter that combines a full date, two digit hour of day, two digit minute of hour, and two digit second of -minute. +minute. `strict_date_hour_minute_second` is supported. |`date_hour_minute_second_fraction`|A formatter that combines a full date, two digit hour of day, two digit minute of hour, two digit second of minute, and three digit fraction of second -(yyyy-MM-dd'T'HH:mm:ss.SSS). +(yyyy-MM-dd'T'HH:mm:ss.SSS). `strict_date_hour_minute_second_fraction` is supported. |`date_hour_minute_second_millis`|A formatter that combines a full date, two digit hour of day, two digit minute of hour, two digit second of minute, and three digit fraction of second (yyyy-MM-dd'T'HH:mm:ss.SSS). +`strict_date_hour_minute_second_millis` is supported. |`date_optional_time`|a generic ISO datetime parser where the date is -mandatory and the time is optional. +mandatory and the time is optional. `strict_date_optional_time` is supported. |`date_time`|A formatter that combines a full date and time, separated by -a 'T' (yyyy-MM-dd'T'HH:mm:ss.SSSZZ). +a 'T' (yyyy-MM-dd'T'HH:mm:ss.SSSZZ). `strict_date_time` is supported. |`date_time_no_millis`|A formatter that combines a full date and time without millis, separated by a 'T' (yyyy-MM-dd'T'HH:mm:ssZZ). +`strict_date_time_no_millis` is supported. -|`hour`|A formatter for a two digit hour of day. +|`hour`|A formatter for a two digit hour of day. `strict_hour` is supported. |`hour_minute`|A formatter for a two digit hour of day and two digit -minute of hour. +minute of hour. `strict_hour_minute` is supported. |`hour_minute_second`|A formatter for a two digit hour of day, two digit minute of hour, and two digit second of minute. +`strict_hour_minute_second` is supported. |`hour_minute_second_fraction`|A formatter for a two digit hour of day, two digit minute of hour, two digit second of minute, and three digit fraction of second (HH:mm:ss.SSS). +`strict_hour_minute_second_fraction` is supported. |`hour_minute_second_millis`|A formatter for a two digit hour of day, two digit minute of hour, two digit second of minute, and three digit fraction of second (HH:mm:ss.SSS). +`strict_hour_minute_second_millis` is supported. |`ordinal_date`|A formatter for a full ordinal date, using a four digit -year and three digit dayOfYear (yyyy-DDD). +year and three digit dayOfYear (yyyy-DDD). `strict_ordinal_date` is supported. |`ordinal_date_time`|A formatter for a full ordinal date and time, using a four digit year and three digit dayOfYear (yyyy-DDD'T'HH:mm:ss.SSSZZ). +`strict_ordinal_date_time` is supported. |`ordinal_date_time_no_millis`|A formatter for a full ordinal date and time without millis, using a four digit year and three digit dayOfYear (yyyy-DDD'T'HH:mm:ssZZ). +`strict_ordinal_date_time_no_millis` is supported. |`time`|A formatter for a two digit hour of day, two digit minute of hour, two digit second of minute, three digit fraction of second, and -time zone offset (HH:mm:ss.SSSZZ). +time zone offset (HH:mm:ss.SSSZZ). `strict_time` is supported. |`time_no_millis`|A formatter for a two digit hour of day, two digit minute of hour, two digit second of minute, and time zone offset -(HH:mm:ssZZ). +(HH:mm:ssZZ). `strict_time_no_millis` is supported. |`t_time`|A formatter for a two digit hour of day, two digit minute of hour, two digit second of minute, three digit fraction of second, and time zone offset prefixed by 'T' ('T'HH:mm:ss.SSSZZ). +`strict_t_time` is supported. |`t_time_no_millis`|A formatter for a two digit hour of day, two digit minute of hour, two digit second of minute, and time zone offset -prefixed by 'T' ('T'HH:mm:ssZZ). +prefixed by 'T' ('T'HH:mm:ssZZ). `strict_t_time_no_millis` is supported. |`week_date`|A formatter for a full date as four digit weekyear, two digit week of weekyear, and one digit day of week (xxxx-'W'ww-e). +`strict_week_date` is supported. |`week_date_time`|A formatter that combines a full weekyear date and time, separated by a 'T' (xxxx-'W'ww-e'T'HH:mm:ss.SSSZZ). +`strict_week_date_time` is supported. -|`weekDateTimeNoMillis`|A formatter that combines a full weekyear date +|`week_date_time_no_millis`|A formatter that combines a full weekyear date and time without millis, separated by a 'T' (xxxx-'W'ww-e'T'HH:mm:ssZZ). +`strict_week_date_time` is supported. -|`week_year`|A formatter for a four digit weekyear. +|`weekyear`|A formatter for a four digit weekyear. `strict_week_year` is supported. -|`weekyearWeek`|A formatter for a four digit weekyear and two digit week -of weekyear. +|`weekyear_week`|A formatter for a four digit weekyear and two digit week +of weekyear. `strict_weekyear_week` is supported. -|`weekyearWeekDay`|A formatter for a four digit weekyear, two digit week -of weekyear, and one digit day of week. +|`weekyear_week_day`|A formatter for a four digit weekyear, two digit week +of weekyear, and one digit day of week. `strict_weekyear_week_day` is supported. -|`year`|A formatter for a four digit year. +|`year`|A formatter for a four digit year. `strict_year` is supported. |`year_month`|A formatter for a four digit year and two digit month of -year. +year. `strict_year_month` is supported. |`year_month_day`|A formatter for a four digit year, two digit month of -year, and two digit day of month. +year, and two digit day of month. `strict_year_month_day` is supported. |`epoch_second`|A formatter for the number of seconds since the epoch. Note, that this timestamp allows a max length of 10 chars, so dates diff --git a/docs/reference/mapping/fields/timestamp-field.asciidoc b/docs/reference/mapping/fields/timestamp-field.asciidoc index b510cef9c2ec6..27988b45b50dd 100644 --- a/docs/reference/mapping/fields/timestamp-field.asciidoc +++ b/docs/reference/mapping/fields/timestamp-field.asciidoc @@ -40,7 +40,7 @@ format>> used to parse the provided timestamp value. For example: } -------------------------------------------------- -Note, the default format is `epoch_millis||dateOptionalTime`. The timestamp value will +Note, the default format is `epoch_millis||strictDateOptionalTime`. The timestamp value will first be parsed as a number and if it fails the format will be tried. [float] diff --git a/docs/reference/mapping/types/core-types.asciidoc b/docs/reference/mapping/types/core-types.asciidoc index 1ca05b793c255..945a5c4e7080c 100644 --- a/docs/reference/mapping/types/core-types.asciidoc +++ b/docs/reference/mapping/types/core-types.asciidoc @@ -349,7 +349,7 @@ date type: Defaults to the property/field name. |`format` |The <>. Defaults to `epoch_millis||dateOptionalTime`. +format>>. Defaults to `epoch_millis||strictDateOptionalTime`. |`store` |Set to `true` to store actual field in the index, `false` to not store it. Defaults to `false` (note, the JSON document itself is stored, diff --git a/docs/reference/mapping/types/root-object-type.asciidoc b/docs/reference/mapping/types/root-object-type.asciidoc index a97f7f576358c..090f88bc8464d 100644 --- a/docs/reference/mapping/types/root-object-type.asciidoc +++ b/docs/reference/mapping/types/root-object-type.asciidoc @@ -42,7 +42,7 @@ and will use the matching format as its format attribute. The date format itself is explained <>. -The default formats are: `dateOptionalTime` (ISO), +The default formats are: `strictDateOptionalTime` (ISO) and `yyyy/MM/dd HH:mm:ss Z||yyyy/MM/dd Z` and `epoch_millis`. *Note:* `dynamic_date_formats` are used *only* for dynamically added diff --git a/docs/reference/migration/migrate_2_0.asciidoc b/docs/reference/migration/migrate_2_0.asciidoc index c6cb6be1178c4..1aae0f4d89e66 100644 --- a/docs/reference/migration/migrate_2_0.asciidoc +++ b/docs/reference/migration/migrate_2_0.asciidoc @@ -302,6 +302,13 @@ Meta fields can no longer be specified within a document. They should be specifi via the API. For example, instead of adding a field `_parent` within a document, use the `parent` url parameter when indexing that document. +==== Default date format now is `strictDateOptionalDate` + +Instead of `dateOptionalTime` the new default date format now is `strictDateOptionalTime`, +which is more strict in parsing dates. This means, that dates now need to have a four digit year, +a two-digit month, day, hour, minute and second. This means, you may need to preprend a part of the date +with a zero to make it conform or switch back to the old `dateOptionalTime` format. + ==== Date format does not support unix timestamps by default In earlier versions of elasticsearch, every timestamp was always tried to be parsed as @@ -723,4 +730,43 @@ to prevent clashes with the watcher plugin === Percolator stats -Changed the `percolate.getTime` stat (total time spent on percolating) to `percolate.time` state. \ No newline at end of file +Changed the `percolate.getTime` stat (total time spent on percolating) to `percolate.time` state. + +=== Plugin Manager for official plugins + +Some of the elasticsearch official plugins have been moved to elasticsearch repository and will be released at the +same time as elasticsearch itself, using the same version number. + +In that case, the plugin manager can now use a simpler form to identify an official plugin. Instead of: + +[source,sh] +--------------- +bin/plugin install elasticsearch/plugin_name/version +--------------- + +You can use: + +[source,sh] +--------------- +bin/plugin install plugin_name +--------------- + +The plugin manager will recognize this form and will be able to download the right version for your elasticsearch +version. + +For older versions of elasticsearch, you still have to use the older form. + +For the record, official plugins which can use this new simplified form are: + +* elasticsearch-analysis-icu +* elasticsearch-analysis-kuromoji +* elasticsearch-analysis-phonetic +* elasticsearch-analysis-smartcn +* elasticsearch-analysis-stempel +* elasticsearch-cloud-aws +* elasticsearch-cloud-azure +* elasticsearch-cloud-gce +* elasticsearch-delete-by-query +* elasticsearch-lang-javascript +* elasticsearch-lang-python + diff --git a/docs/reference/modules/plugins.asciidoc b/docs/reference/modules/plugins.asciidoc index 4df3af4496f2f..52ed8fe7abe6d 100644 --- a/docs/reference/modules/plugins.asciidoc +++ b/docs/reference/modules/plugins.asciidoc @@ -14,19 +14,26 @@ and more. ==== Installing plugins Installing plugins can either be done manually by placing them under the -`plugins` directory, or using the `plugin` script. Several plugins can -be found under the https://github.com/elasticsearch[elasticsearch] -organization in GitHub, starting with `elasticsearch-`. +`plugins` directory, or using the `plugin` script. Installing plugins typically take the following form: +[source,shell] +----------------------------------- +bin/plugin --install plugin_name +----------------------------------- + +The plugin will be automatically downloaded in this case from `download.elastic.co` download service using the +same version as your elasticsearch version. + +For older version of elasticsearch (prior to 2.0.0) or community plugins, you would use the following form: + [source,shell] ----------------------------------- bin/plugin --install // ----------------------------------- -The plugins will be -automatically downloaded in this case from `download.elastic.co`, +The plugins will be automatically downloaded in this case from `download.elastic.co` (for older plugins), and in case they don't exist there, from maven (central and sonatype). Note that when the plugin is located in maven central or sonatype diff --git a/docs/reference/query-dsl/filtered-query.asciidoc b/docs/reference/query-dsl/filtered-query.asciidoc index d12d32a974d21..41825b8dbba29 100644 --- a/docs/reference/query-dsl/filtered-query.asciidoc +++ b/docs/reference/query-dsl/filtered-query.asciidoc @@ -94,61 +94,3 @@ Multiple filters can be applied by wrapping them in a } } -------------------------------------------------- - -===== Filter strategy - -You can control how the filter and query are executed with the `strategy` -parameter: - -[source,js] --------------------------------------------------- -{ - "filtered" : { - "query" : { ... }, - "filter" : { ... }, - "strategy": "leap_frog" - } -} --------------------------------------------------- - -IMPORTANT: This is an _expert-level_ setting. Most users can simply ignore it. - -The `strategy` parameter accepts the following options: - -[horizontal] -`leap_frog_query_first`:: - - Look for the first document matching the query, and then alternatively - advance the query and the filter to find common matches. - -`leap_frog_filter_first`:: - - Look for the first document matching the filter, and then alternatively - advance the query and the filter to find common matches. - -`leap_frog`:: - - Same as `leap_frog_query_first`. - -`query_first`:: - - If the filter supports random access, then search for documents using the - query, and then consult the filter to check whether there is a match. - Otherwise fall back to `leap_frog_query_first`. - -`random_access_${threshold}`:: - - If the filter supports random access and if the number of documents in the - index divided by the cardinality of the filter is greater than ${threshold}, - then apply the filter first. - Otherwise fall back to `leap_frog_query_first`. `${threshold}` must be - greater than or equal to `1`. - -`random_access_always`:: - - Apply the filter first if it supports random access. Otherwise fall back - to `leap_frog_query_first`. - -The default strategy is to use `query_first` on filters that are not -advanceable such as geo filters and script filters, and `random_access_100` on -other filters. diff --git a/plugins/analysis-icu/pom.xml b/plugins/analysis-icu/pom.xml index 701ffbfc4050c..34ee6b25304a2 100644 --- a/plugins/analysis-icu/pom.xml +++ b/plugins/analysis-icu/pom.xml @@ -3,22 +3,22 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 - org.elasticsearch.plugin - elasticsearch-analysis-icu - - jar - Elasticsearch ICU Analysis plugin - The ICU Analysis plugin integrates Lucene ICU module into elasticsearch, adding ICU relates analysis components. - org.elasticsearch + org.elasticsearch.plugin elasticsearch-plugin 2.0.0-SNAPSHOT + elasticsearch-analysis-icu + Elasticsearch ICU Analysis plugin + The ICU Analysis plugin integrates Lucene ICU module into elasticsearch, adding ICU relates analysis components. + 1 INFO + analysis_icu + false diff --git a/plugins/analysis-icu/rest-api-spec/test/analysis_icu/10_basic.yaml b/plugins/analysis-icu/rest-api-spec/test/analysis_icu/10_basic.yaml new file mode 100644 index 0000000000000..4497cb7f2cb48 --- /dev/null +++ b/plugins/analysis-icu/rest-api-spec/test/analysis_icu/10_basic.yaml @@ -0,0 +1,37 @@ +# Integration tests for ICU analysis components +# +"Tokenizer": + - do: + indices.analyze: + text: Foo Bar + tokenizer: icu_tokenizer + - length: { tokens: 2 } + - match: { tokens.0.token: Foo } + - match: { tokens.1.token: Bar } +--- +"Normalization filter": + - do: + indices.analyze: + filters: icu_normalizer + text: Foo Bar Ruß + tokenizer: keyword + - length: { tokens: 1 } + - match: { tokens.0.token: foo bar russ } +--- +"Normalization charfilter": + - do: + indices.analyze: + char_filters: icu_normalizer + text: Foo Bar Ruß + tokenizer: keyword + - length: { tokens: 1 } + - match: { tokens.0.token: foo bar russ } +--- +"Folding filter": + - do: + indices.analyze: + filters: icu_folding + text: Foo Bar résumé + tokenizer: keyword + - length: { tokens: 1 } + - match: { tokens.0.token: foo bar resume } diff --git a/plugins/analysis-icu/src/main/java/org/elasticsearch/indices/analysis/IcuIndicesAnalysis.java b/plugins/analysis-icu/src/main/java/org/elasticsearch/indices/analysis/IcuIndicesAnalysis.java index 7ca0a089bbf1f..cc7d56994da55 100644 --- a/plugins/analysis-icu/src/main/java/org/elasticsearch/indices/analysis/IcuIndicesAnalysis.java +++ b/plugins/analysis-icu/src/main/java/org/elasticsearch/indices/analysis/IcuIndicesAnalysis.java @@ -22,20 +22,26 @@ import com.ibm.icu.text.Collator; import com.ibm.icu.text.Normalizer2; import com.ibm.icu.text.Transliterator; + import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.analysis.Tokenizer; import org.apache.lucene.analysis.icu.ICUFoldingFilter; +import org.apache.lucene.analysis.icu.ICUNormalizer2CharFilter; import org.apache.lucene.analysis.icu.ICUTransformFilter; import org.apache.lucene.analysis.icu.segmentation.ICUTokenizer; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.analysis.ICUCollationKeyFilter; +import org.elasticsearch.index.analysis.PreBuiltCharFilterFactoryFactory; import org.elasticsearch.index.analysis.PreBuiltTokenFilterFactoryFactory; import org.elasticsearch.index.analysis.PreBuiltTokenizerFactoryFactory; +import org.elasticsearch.index.analysis.CharFilterFactory; import org.elasticsearch.index.analysis.TokenFilterFactory; import org.elasticsearch.index.analysis.TokenizerFactory; +import java.io.Reader; + /** * Registers indices level analysis components so, if not explicitly configured, will be shared * among all indices. @@ -106,5 +112,17 @@ public TokenStream create(TokenStream tokenStream) { return new ICUTransformFilter(tokenStream, Transliterator.getInstance("Null", Transliterator.FORWARD)); } })); + + indicesAnalysisService.charFilterFactories().put("icu_normalizer", new PreBuiltCharFilterFactoryFactory(new CharFilterFactory() { + @Override + public String name() { + return "icu_normalizer"; + } + + @Override + public Reader create(Reader reader) { + return new ICUNormalizer2CharFilter(reader); + } + })); } } diff --git a/core/src/main/java/org/elasticsearch/monitor/network/JmxNetworkProbe.java b/plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/AnalysisICURestIT.java similarity index 51% rename from core/src/main/java/org/elasticsearch/monitor/network/JmxNetworkProbe.java rename to plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/AnalysisICURestIT.java index 839bb90e2524e..481d6c3f3a0cc 100644 --- a/core/src/main/java/org/elasticsearch/monitor/network/JmxNetworkProbe.java +++ b/plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/AnalysisICURestIT.java @@ -17,37 +17,25 @@ * under the License. */ -package org.elasticsearch.monitor.network; +package org.elasticsearch.index.analysis; -import org.elasticsearch.common.component.AbstractComponent; -import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.settings.Settings; +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; +import org.elasticsearch.test.rest.ElasticsearchRestTestCase; +import org.elasticsearch.test.rest.RestTestCandidate; +import org.elasticsearch.test.rest.parser.RestTestParseException; -/** - * - */ -public class JmxNetworkProbe extends AbstractComponent implements NetworkProbe { - - @Inject - public JmxNetworkProbe(Settings settings) { - super(settings); - } +import java.io.IOException; - @Override - public NetworkInfo networkInfo() { - NetworkInfo info = new NetworkInfo(); - return info; - } +public class AnalysisICURestIT extends ElasticsearchRestTestCase { - @Override - public NetworkStats networkStats() { - NetworkStats stats = new NetworkStats(); - stats.timestamp = System.currentTimeMillis(); - return stats; + public AnalysisICURestIT(@Name("yaml") RestTestCandidate testCandidate) { + super(testCandidate); } - @Override - public String ifconfig() { - return "NA"; + @ParametersFactory + public static Iterable parameters() throws IOException, RestTestParseException { + return ElasticsearchRestTestCase.createParameters(0, 1); } } + diff --git a/plugins/analysis-kuromoji/pom.xml b/plugins/analysis-kuromoji/pom.xml index a4746fd12be5d..66d4cbd5baf04 100644 --- a/plugins/analysis-kuromoji/pom.xml +++ b/plugins/analysis-kuromoji/pom.xml @@ -4,21 +4,20 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 - org.elasticsearch.plugin - elasticsearch-analysis-kuromoji - - jar - Elasticsearch Japanese (kuromoji) Analysis plugin - The Japanese (kuromoji) Analysis plugin integrates Lucene kuromoji analysis module into elasticsearch. - - org.elasticsearch + org.elasticsearch.plugin elasticsearch-plugin 2.0.0-SNAPSHOT + elasticsearch-analysis-kuromoji + jar + Elasticsearch Japanese (kuromoji) Analysis plugin + The Japanese (kuromoji) Analysis plugin integrates Lucene kuromoji analysis module into elasticsearch. + - + analysis_kuromoji + false diff --git a/plugins/analysis-kuromoji/rest-api-spec/test/analysis_kuromoji/10_basic.yaml b/plugins/analysis-kuromoji/rest-api-spec/test/analysis_kuromoji/10_basic.yaml new file mode 100644 index 0000000000000..dfeac8b18ff64 --- /dev/null +++ b/plugins/analysis-kuromoji/rest-api-spec/test/analysis_kuromoji/10_basic.yaml @@ -0,0 +1,54 @@ +# Integration tests for Kuromoji analysis components +# +--- +"Analyzer": + - do: + indices.analyze: + text: JR新宿駅の近くにビールを飲みに行こうか + analyzer: kuromoji + - length: { tokens: 7 } + - match: { tokens.0.token: jr } + - match: { tokens.1.token: 新宿 } + - match: { tokens.2.token: 駅 } + - match: { tokens.3.token: 近く } + - match: { tokens.4.token: ビール } + - match: { tokens.5.token: 飲む } + - match: { tokens.6.token: 行く } +--- +"Tokenizer": + - do: + indices.analyze: + text: 関西国際空港 + tokenizer: kuromoji_tokenizer + - length: { tokens: 4 } + - match: { tokens.0.token: 関西 } + - match: { tokens.1.token: 関西国際空港 } + - match: { tokens.2.token: 国際 } + - match: { tokens.3.token: 空港 } +--- +"Baseform filter": + - do: + indices.analyze: + text: 飲み + tokenizer: kuromoji_tokenizer + filters: kuromoji_baseform + - length: { tokens: 1 } + - match: { tokens.0.token: 飲む } +--- +"Reading filter": + - do: + indices.analyze: + text: 寿司 + tokenizer: kuromoji_tokenizer + filters: kuromoji_readingform + - length: { tokens: 1 } + - match: { tokens.0.token: sushi } +--- +"Stemming filter": + - do: + indices.analyze: + text: サーバー + tokenizer: kuromoji_tokenizer + filters: kuromoji_stemmer + - length: { tokens: 1 } + - match: { tokens.0.token: サーバ } diff --git a/plugins/analysis-kuromoji/rest-api-spec/test/analysis_kuromoji/20_search.yaml b/plugins/analysis-kuromoji/rest-api-spec/test/analysis_kuromoji/20_search.yaml new file mode 100644 index 0000000000000..a4bc502060466 --- /dev/null +++ b/plugins/analysis-kuromoji/rest-api-spec/test/analysis_kuromoji/20_search.yaml @@ -0,0 +1,35 @@ +# Integration tests for Kuromoji analysis components +# +--- +"Index kuromoji content": + - do: + indices.create: + index: test + body: + mappings: + type: + properties: + text: + type: string + analyzer: kuromoji + - do: + cluster.health: + wait_for_status: yellow + + - do: + index: + index: test + type: type + id: 1 + body: { "text": "JR新宿駅の近くにビールを飲みに行こうか" } + - do: + indices.refresh: {} + + - do: + search: + index: test + body: + query: + match: + text: jr + - match: { hits.total: 1 } diff --git a/plugins/analysis-kuromoji/src/test/java/org/elasticsearch/index/analysis/AnalysisKuromojiRestIT.java b/plugins/analysis-kuromoji/src/test/java/org/elasticsearch/index/analysis/AnalysisKuromojiRestIT.java new file mode 100644 index 0000000000000..857e66fef39e3 --- /dev/null +++ b/plugins/analysis-kuromoji/src/test/java/org/elasticsearch/index/analysis/AnalysisKuromojiRestIT.java @@ -0,0 +1,41 @@ +/* + * 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.analysis; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; +import org.elasticsearch.test.rest.ElasticsearchRestTestCase; +import org.elasticsearch.test.rest.RestTestCandidate; +import org.elasticsearch.test.rest.parser.RestTestParseException; + +import java.io.IOException; + +public class AnalysisKuromojiRestIT extends ElasticsearchRestTestCase { + + public AnalysisKuromojiRestIT(@Name("yaml") RestTestCandidate testCandidate) { + super(testCandidate); + } + + @ParametersFactory + public static Iterable parameters() throws IOException, RestTestParseException { + return ElasticsearchRestTestCase.createParameters(0, 1); + } +} + diff --git a/plugins/analysis-kuromoji/src/test/java/org/elasticsearch/index/analysis/KuromojiIntegrationTests.java b/plugins/analysis-kuromoji/src/test/java/org/elasticsearch/index/analysis/KuromojiIntegrationTests.java deleted file mode 100644 index a1c4e8c189ede..0000000000000 --- a/plugins/analysis-kuromoji/src/test/java/org/elasticsearch/index/analysis/KuromojiIntegrationTests.java +++ /dev/null @@ -1,90 +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.analysis; - -import org.elasticsearch.action.admin.indices.analyze.AnalyzeResponse; -import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.index.query.QueryBuilders; -import org.elasticsearch.plugins.PluginsService; -import org.elasticsearch.test.ElasticsearchIntegrationTest; -import org.junit.Test; - -import java.io.IOException; -import java.util.concurrent.ExecutionException; - -import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.CoreMatchers.notNullValue; - -@ElasticsearchIntegrationTest.ClusterScope(scope = ElasticsearchIntegrationTest.Scope.SUITE) -public class KuromojiIntegrationTests extends ElasticsearchIntegrationTest { - - @Override - protected Settings nodeSettings(int nodeOrdinal) { - return Settings.builder() - .put(super.nodeSettings(nodeOrdinal)) - .put("plugins." + PluginsService.LOAD_PLUGIN_FROM_CLASSPATH, true) - .build(); - } - - @Test - public void testKuromojiAnalyzer() throws ExecutionException, InterruptedException { - AnalyzeResponse response = client().admin().indices() - .prepareAnalyze("JR新宿駅の近くにビールを飲みに行こうか").setAnalyzer("kuromoji") - .execute().get(); - - String[] expectedTokens = {"jr", "新宿", "駅", "近く", "ビール", "飲む", "行く"}; - - assertThat(response, notNullValue()); - assertThat(response.getTokens().size(), is(7)); - - for (int i = 0; i < expectedTokens.length; i++) { - assertThat(response.getTokens().get(i).getTerm(), is(expectedTokens[i])); - } - } - - @Test - public void testKuromojiAnalyzerInMapping() throws ExecutionException, InterruptedException, IOException { - createIndex("test"); - ensureGreen("test"); - final XContentBuilder mapping = jsonBuilder().startObject() - .startObject("type") - .startObject("properties") - .startObject("foo") - .field("type", "string") - .field("analyzer", "kuromoji") - .endObject() - .endObject() - .endObject() - .endObject(); - - client().admin().indices().preparePutMapping("test").setType("type").setSource(mapping).get(); - - index("test", "type", "1", "foo", "JR新宿駅の近くにビールを飲みに行こうか"); - refresh(); - - SearchResponse response = client().prepareSearch("test").setQuery( - QueryBuilders.matchQuery("foo", "jr") - ).execute().actionGet(); - - assertThat(response.getHits().getTotalHits(), is(1L)); - } -} diff --git a/plugins/analysis-kuromoji/src/test/java/org/elasticsearch/index/analysis/empty_user_dict.txt b/plugins/analysis-kuromoji/src/test/resources/org/elasticsearch/index/analysis/empty_user_dict.txt similarity index 100% rename from plugins/analysis-kuromoji/src/test/java/org/elasticsearch/index/analysis/empty_user_dict.txt rename to plugins/analysis-kuromoji/src/test/resources/org/elasticsearch/index/analysis/empty_user_dict.txt diff --git a/plugins/analysis-kuromoji/src/test/java/org/elasticsearch/index/analysis/kuromoji_analysis.json b/plugins/analysis-kuromoji/src/test/resources/org/elasticsearch/index/analysis/kuromoji_analysis.json similarity index 100% rename from plugins/analysis-kuromoji/src/test/java/org/elasticsearch/index/analysis/kuromoji_analysis.json rename to plugins/analysis-kuromoji/src/test/resources/org/elasticsearch/index/analysis/kuromoji_analysis.json diff --git a/plugins/analysis-kuromoji/src/test/java/org/elasticsearch/index/analysis/user_dict.txt b/plugins/analysis-kuromoji/src/test/resources/org/elasticsearch/index/analysis/user_dict.txt similarity index 100% rename from plugins/analysis-kuromoji/src/test/java/org/elasticsearch/index/analysis/user_dict.txt rename to plugins/analysis-kuromoji/src/test/resources/org/elasticsearch/index/analysis/user_dict.txt diff --git a/plugins/analysis-phonetic/pom.xml b/plugins/analysis-phonetic/pom.xml index d4cc3e7b871d9..c783a91a1749c 100644 --- a/plugins/analysis-phonetic/pom.xml +++ b/plugins/analysis-phonetic/pom.xml @@ -4,22 +4,20 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 - org.elasticsearch.plugin - elasticsearch-analysis-phonetic - - jar - Elasticsearch Phonetic Analysis plugin - The Phonetic Analysis plugin integrates phonetic token filter analysis with elasticsearch. - - org.elasticsearch + org.elasticsearch.plugin elasticsearch-plugin 2.0.0-SNAPSHOT + elasticsearch-analysis-phonetic + Elasticsearch Phonetic Analysis plugin + The Phonetic Analysis plugin integrates phonetic token filter analysis with elasticsearch. + - - + analysis_phonetic + false + diff --git a/plugins/analysis-phonetic/rest-api-spec/test/analysis_phonetic/10_metaphone.yaml b/plugins/analysis-phonetic/rest-api-spec/test/analysis_phonetic/10_metaphone.yaml new file mode 100644 index 0000000000000..5e45c0a6241ef --- /dev/null +++ b/plugins/analysis-phonetic/rest-api-spec/test/analysis_phonetic/10_metaphone.yaml @@ -0,0 +1,36 @@ +# Integration tests for Phonetic analysis components +# + + +"Metaphone": + - do: + indices.create: + index: phonetic_sample + body: + settings: + index: + analysis: + analyzer: + my_analyzer: + tokenizer: standard + filter: ["standard", "lowercase", "my_metaphone"] + filter: + my_metaphone: + type: phonetic + encoder: metaphone + replace: false + - do: + cluster.health: + wait_for_status: yellow + - do: + indices.analyze: + index: phonetic_sample + analyzer: my_analyzer + text: Joe Bloggs + + - length: { tokens: 4 } + - match: { tokens.0.token: J } + - match: { tokens.1.token: joe } + - match: { tokens.2.token: BLKS } + - match: { tokens.3.token: bloggs } + diff --git a/plugins/analysis-phonetic/rest-api-spec/test/analysis_phonetic/20_double_metaphone.yaml b/plugins/analysis-phonetic/rest-api-spec/test/analysis_phonetic/20_double_metaphone.yaml new file mode 100644 index 0000000000000..6481d37bd8794 --- /dev/null +++ b/plugins/analysis-phonetic/rest-api-spec/test/analysis_phonetic/20_double_metaphone.yaml @@ -0,0 +1,33 @@ +# Integration tests for Phonetic analysis components +# + + +"Double Metaphone": + - do: + indices.create: + index: phonetic_sample + body: + settings: + index: + analysis: + analyzer: + my_analyzer: + tokenizer: standard + filter: ["standard", "lowercase", "my_metaphone"] + filter: + my_metaphone: + type: phonetic + encoder: double_metaphone + max_code_len: 6 + - do: + cluster.health: + wait_for_status: yellow + - do: + indices.analyze: + index: phonetic_sample + analyzer: my_analyzer + text: supercalifragilisticexpialidocious + + - length: { tokens: 1 } + - match: { tokens.0.token: SPRKLF } + diff --git a/plugins/analysis-phonetic/rest-api-spec/test/analysis_phonetic/30_beider_morse.yaml b/plugins/analysis-phonetic/rest-api-spec/test/analysis_phonetic/30_beider_morse.yaml new file mode 100644 index 0000000000000..68b3f764983fc --- /dev/null +++ b/plugins/analysis-phonetic/rest-api-spec/test/analysis_phonetic/30_beider_morse.yaml @@ -0,0 +1,35 @@ +# Integration tests for Phonetic analysis components +# + + +"Double Metaphone": + - do: + indices.create: + index: phonetic_sample + body: + settings: + index: + analysis: + analyzer: + my_analyzer: + tokenizer: standard + filter: ["standard", "lowercase", "beider_morse"] + filter: + beider_morse: + type: phonetic + encoder: beider_morse + rule_type: exact + name_type: ashkenazi + languageset: polish + - do: + cluster.health: + wait_for_status: yellow + - do: + indices.analyze: + index: phonetic_sample + analyzer: my_analyzer + text: Szwarc + + - length: { tokens: 1 } + - match: { tokens.0.token: Svarts } + diff --git a/plugins/analysis-phonetic/rest-api-spec/test/analysis_phonetic/40_search.yaml b/plugins/analysis-phonetic/rest-api-spec/test/analysis_phonetic/40_search.yaml new file mode 100644 index 0000000000000..a32eed25df2d5 --- /dev/null +++ b/plugins/analysis-phonetic/rest-api-spec/test/analysis_phonetic/40_search.yaml @@ -0,0 +1,47 @@ +# Integration tests for Phonetic analysis components +# +--- +"Index phonetic content": + - do: + indices.create: + index: phonetic_sample + body: + settings: + index: + analysis: + analyzer: + my_analyzer: + tokenizer: standard + filter: ["standard", "lowercase", "my_metaphone"] + filter: + my_metaphone: + type: phonetic + encoder: metaphone + replace: false + mappings: + type: + properties: + text: + type: string + analyzer: my_analyzer + - do: + cluster.health: + wait_for_status: yellow + + - do: + index: + index: phonetic_sample + type: type + id: 1 + body: { "text": "hello world" } + - do: + indices.refresh: {} + + - do: + search: + index: phonetic_sample + body: + query: + match: + text: helllo + - match: { hits.total: 1 } diff --git a/plugins/analysis-phonetic/src/test/java/org/elasticsearch/index/analysis/AnalysisPhoneticRestIT.java b/plugins/analysis-phonetic/src/test/java/org/elasticsearch/index/analysis/AnalysisPhoneticRestIT.java new file mode 100644 index 0000000000000..f211e23c59750 --- /dev/null +++ b/plugins/analysis-phonetic/src/test/java/org/elasticsearch/index/analysis/AnalysisPhoneticRestIT.java @@ -0,0 +1,41 @@ +/* + * 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.analysis; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; +import org.elasticsearch.test.rest.ElasticsearchRestTestCase; +import org.elasticsearch.test.rest.RestTestCandidate; +import org.elasticsearch.test.rest.parser.RestTestParseException; + +import java.io.IOException; + +public class AnalysisPhoneticRestIT extends ElasticsearchRestTestCase { + + public AnalysisPhoneticRestIT(@Name("yaml") RestTestCandidate testCandidate) { + super(testCandidate); + } + + @ParametersFactory + public static Iterable parameters() throws IOException, RestTestParseException { + return ElasticsearchRestTestCase.createParameters(0, 1); + } +} + diff --git a/plugins/analysis-phonetic/src/test/java/org/elasticsearch/index/analysis/SimplePhoneticIntegrationTests.java b/plugins/analysis-phonetic/src/test/java/org/elasticsearch/index/analysis/SimplePhoneticIntegrationTests.java deleted file mode 100644 index 7f74879e3ced2..0000000000000 --- a/plugins/analysis-phonetic/src/test/java/org/elasticsearch/index/analysis/SimplePhoneticIntegrationTests.java +++ /dev/null @@ -1,108 +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.analysis; - -import org.elasticsearch.action.admin.indices.analyze.AnalyzeResponse; -import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.index.query.QueryBuilders; -import org.elasticsearch.plugins.PluginsService; -import org.elasticsearch.test.ElasticsearchIntegrationTest; -import org.junit.Test; - -import java.io.IOException; -import java.util.concurrent.ExecutionException; - -import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.CoreMatchers.notNullValue; - -@ElasticsearchIntegrationTest.ClusterScope(numDataNodes = 1, scope = ElasticsearchIntegrationTest.Scope.SUITE) -public class SimplePhoneticIntegrationTests extends ElasticsearchIntegrationTest { - - @Override - protected Settings nodeSettings(int nodeOrdinal) { - return Settings.builder() - .put(super.nodeSettings(nodeOrdinal)) - .put("plugins." + PluginsService.LOAD_PLUGIN_FROM_CLASSPATH, true) - .build(); - } - - @Override - public Settings indexSettings() { - Settings settings = Settings.builder() - .put(super.indexSettings()) - .put("index.analysis.analyzer.my_analyzer.tokenizer", "standard") - .putArray("index.analysis.analyzer.my_analyzer.filter", "standard", "lowercase", "my_metaphone") - .put("index.analysis.filter.my_metaphone.type", "phonetic") - .put("index.analysis.filter.my_metaphone.encoder", "metaphone") - .put("index.analysis.filter.my_metaphone.replace", false) - .build(); - - return settings; - } - - @Test - public void testPhoneticAnalyzer() throws ExecutionException, InterruptedException { - createIndex("test"); - ensureGreen("test"); - AnalyzeResponse response = client().admin().indices() - .prepareAnalyze("hello world") - .setIndex("test") - .setAnalyzer("my_analyzer") - .execute().get(); - - assertThat(response, notNullValue()); - assertThat(response.getTokens().size(), is(4)); - assertThat(response.getTokens().get(0).getTerm(), is("HL")); - assertThat(response.getTokens().get(1).getTerm(), is("hello")); - assertThat(response.getTokens().get(2).getTerm(), is("WRLT")); - assertThat(response.getTokens().get(3).getTerm(), is("world")); - } - - @Test - public void testPhoneticAnalyzerInMapping() throws ExecutionException, InterruptedException, IOException { - createIndex("test"); - ensureGreen("test"); - final XContentBuilder mapping = jsonBuilder().startObject() - .startObject("type") - .startObject("properties") - .startObject("foo") - .field("type", "string") - .field("analyzer", "my_analyzer") - .endObject() - .endObject() - .endObject() - .endObject(); - - client().admin().indices().preparePutMapping("test").setType("type").setSource(mapping).get(); - - index("test", "type", "1", "foo", "hello world"); - refresh(); - - SearchResponse response = client().prepareSearch("test").setQuery( - QueryBuilders.matchQuery("foo", "helllo") - ).execute().actionGet(); - - assertThat(response.getHits().getTotalHits(), is(1L)); - } - -} diff --git a/plugins/analysis-smartcn/pom.xml b/plugins/analysis-smartcn/pom.xml index a2ab996b87ada..0c279084f8ee7 100644 --- a/plugins/analysis-smartcn/pom.xml +++ b/plugins/analysis-smartcn/pom.xml @@ -4,21 +4,19 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 - org.elasticsearch.plugin - elasticsearch-analysis-smartcn - - jar - Elasticsearch Smart Chinese Analysis plugin - Smart Chinese Analysis plugin integrates Lucene Smart Chinese analysis module into elasticsearch. - - org.elasticsearch + org.elasticsearch.plugin elasticsearch-plugin 2.0.0-SNAPSHOT + elasticsearch-analysis-smartcn + Elasticsearch Smart Chinese Analysis plugin + Smart Chinese Analysis plugin integrates Lucene Smart Chinese analysis module into elasticsearch. + - + analysis_smartcn + false diff --git a/plugins/analysis-smartcn/rest-api-spec/test/analysis_smartcn/10_basic.yaml b/plugins/analysis-smartcn/rest-api-spec/test/analysis_smartcn/10_basic.yaml new file mode 100644 index 0000000000000..2549f774f81c0 --- /dev/null +++ b/plugins/analysis-smartcn/rest-api-spec/test/analysis_smartcn/10_basic.yaml @@ -0,0 +1,28 @@ +# Integration tests for Smart Chinese analysis components +# +"Tokenizer": + - do: + indices.analyze: + text: 我购买了道具和服装。 + tokenizer: smartcn_tokenizer + - length: { tokens: 7 } + - match: { tokens.0.token: 我 } + - match: { tokens.1.token: 购买 } + - match: { tokens.2.token: 了 } + - match: { tokens.3.token: 道具 } + - match: { tokens.4.token: 和 } + - match: { tokens.5.token: 服装 } + - match: { tokens.6.token: "," } +--- +"Analyzer": + - do: + indices.analyze: + text: 我购买了道具和服装。 + analyzer: smartcn + - length: { tokens: 6 } + - match: { tokens.0.token: 我 } + - match: { tokens.1.token: 购买 } + - match: { tokens.2.token: 了 } + - match: { tokens.3.token: 道具 } + - match: { tokens.4.token: 和 } + - match: { tokens.5.token: 服装 } diff --git a/plugins/analysis-smartcn/src/test/java/org/elasticsearch/index/analysis/AnalysisSmartChineseRestIT.java b/plugins/analysis-smartcn/src/test/java/org/elasticsearch/index/analysis/AnalysisSmartChineseRestIT.java new file mode 100644 index 0000000000000..bd46d53659ba6 --- /dev/null +++ b/plugins/analysis-smartcn/src/test/java/org/elasticsearch/index/analysis/AnalysisSmartChineseRestIT.java @@ -0,0 +1,41 @@ +/* + * 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.analysis; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; +import org.elasticsearch.test.rest.ElasticsearchRestTestCase; +import org.elasticsearch.test.rest.RestTestCandidate; +import org.elasticsearch.test.rest.parser.RestTestParseException; + +import java.io.IOException; + +public class AnalysisSmartChineseRestIT extends ElasticsearchRestTestCase { + + public AnalysisSmartChineseRestIT(@Name("yaml") RestTestCandidate testCandidate) { + super(testCandidate); + } + + @ParametersFactory + public static Iterable parameters() throws IOException, RestTestParseException { + return ElasticsearchRestTestCase.createParameters(0, 1); + } +} + diff --git a/plugins/analysis-stempel/pom.xml b/plugins/analysis-stempel/pom.xml index a60524176c34b..29bdcafc38b4e 100644 --- a/plugins/analysis-stempel/pom.xml +++ b/plugins/analysis-stempel/pom.xml @@ -4,21 +4,19 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 - org.elasticsearch.plugin - elasticsearch-analysis-stempel - - jar - Elasticsearch Stempel (Polish) Analysis plugin - The Stempel (Polish) Analysis plugin integrates Lucene stempel (polish) analysis module into elasticsearch. - - org.elasticsearch + org.elasticsearch.plugin elasticsearch-plugin 2.0.0-SNAPSHOT + elasticsearch-analysis-stempel + Elasticsearch Stempel (Polish) Analysis plugin + The Stempel (Polish) Analysis plugin integrates Lucene stempel (polish) analysis module into elasticsearch. + - + analysis_stempel + false diff --git a/plugins/analysis-stempel/rest-api-spec/test/analysis_stempel/10_basic.yaml b/plugins/analysis-stempel/rest-api-spec/test/analysis_stempel/10_basic.yaml new file mode 100644 index 0000000000000..3dc825290f401 --- /dev/null +++ b/plugins/analysis-stempel/rest-api-spec/test/analysis_stempel/10_basic.yaml @@ -0,0 +1,18 @@ +# Integration tests for Polish analysis components +# +"Stemmer": + - do: + indices.analyze: + text: studenci + tokenizer: keyword + filters: polish_stem + - length: { tokens: 1 } + - match: { tokens.0.token: student } +--- +"Analyzer": + - do: + indices.analyze: + text: studenta był + analyzer: polish + - length: { tokens: 1 } + - match: { tokens.0.token: student } diff --git a/plugins/analysis-stempel/src/test/java/org/elasticsearch/index/analysis/AnalysisPolishRestIT.java b/plugins/analysis-stempel/src/test/java/org/elasticsearch/index/analysis/AnalysisPolishRestIT.java new file mode 100644 index 0000000000000..c9bee3127e798 --- /dev/null +++ b/plugins/analysis-stempel/src/test/java/org/elasticsearch/index/analysis/AnalysisPolishRestIT.java @@ -0,0 +1,41 @@ +/* + * 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.analysis; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; +import org.elasticsearch.test.rest.ElasticsearchRestTestCase; +import org.elasticsearch.test.rest.RestTestCandidate; +import org.elasticsearch.test.rest.parser.RestTestParseException; + +import java.io.IOException; + +public class AnalysisPolishRestIT extends ElasticsearchRestTestCase { + + public AnalysisPolishRestIT(@Name("yaml") RestTestCandidate testCandidate) { + super(testCandidate); + } + + @ParametersFactory + public static Iterable parameters() throws IOException, RestTestParseException { + return ElasticsearchRestTestCase.createParameters(0, 1); + } +} + diff --git a/plugins/cloud-aws/pom.xml b/plugins/cloud-aws/pom.xml index 6e9d4737cb4c2..e7c8169f37018 100644 --- a/plugins/cloud-aws/pom.xml +++ b/plugins/cloud-aws/pom.xml @@ -4,22 +4,21 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 - org.elasticsearch.plugin - elasticsearch-cloud-aws - - jar - Elasticsearch AWS cloud plugin - The Amazon Web Service (AWS) Cloud plugin allows to use AWS API for the unicast discovery mechanism and add S3 repositories. - - org.elasticsearch + org.elasticsearch.plugin elasticsearch-plugin 2.0.0-SNAPSHOT + elasticsearch-cloud-aws + Elasticsearch AWS cloud plugin + The Amazon Web Service (AWS) Cloud plugin allows to use AWS API for the unicast discovery mechanism and add S3 repositories. + 1.10.0 1 + cloud_aws + false diff --git a/plugins/cloud-aws/rest-api-spec/test/cloud_aws/10_basic.yaml b/plugins/cloud-aws/rest-api-spec/test/cloud_aws/10_basic.yaml new file mode 100644 index 0000000000000..a2c8f882adc01 --- /dev/null +++ b/plugins/cloud-aws/rest-api-spec/test/cloud_aws/10_basic.yaml @@ -0,0 +1,14 @@ +# Integration tests for Cloud AWS components +# +"Cloud AWS loaded": + - do: + cluster.state: {} + + # Get master node id + - set: { master_node: master } + + - do: + nodes.info: {} + + - match: { nodes.$master.plugins.0.name: cloud-aws } + - match: { nodes.$master.plugins.0.jvm: true } diff --git a/plugins/cloud-aws/src/test/java/org/elasticsearch/cloud/aws/CloudAWSRestIT.java b/plugins/cloud-aws/src/test/java/org/elasticsearch/cloud/aws/CloudAWSRestIT.java new file mode 100644 index 0000000000000..d4d99218b013e --- /dev/null +++ b/plugins/cloud-aws/src/test/java/org/elasticsearch/cloud/aws/CloudAWSRestIT.java @@ -0,0 +1,41 @@ +/* + * 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.cloud.aws; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; +import org.elasticsearch.test.rest.ElasticsearchRestTestCase; +import org.elasticsearch.test.rest.RestTestCandidate; +import org.elasticsearch.test.rest.parser.RestTestParseException; + +import java.io.IOException; + +public class CloudAWSRestIT extends ElasticsearchRestTestCase { + + public CloudAWSRestIT(@Name("yaml") RestTestCandidate testCandidate) { + super(testCandidate); + } + + @ParametersFactory + public static Iterable parameters() throws IOException, RestTestParseException { + return ElasticsearchRestTestCase.createParameters(0, 1); + } +} + diff --git a/plugins/cloud-azure/pom.xml b/plugins/cloud-azure/pom.xml index ec48e39af9184..307a726e8d4ae 100644 --- a/plugins/cloud-azure/pom.xml +++ b/plugins/cloud-azure/pom.xml @@ -15,22 +15,21 @@ governing permissions and limitations under the License. --> xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 - org.elasticsearch.plugin - elasticsearch-cloud-azure - - jar - Elasticsearch Azure cloud plugin - The Azure Cloud plugin allows to use Azure API for the unicast discovery mechanism and add Azure storage repositories. - - org.elasticsearch + org.elasticsearch.plugin elasticsearch-plugin 2.0.0-SNAPSHOT + elasticsearch-cloud-azure + Elasticsearch Azure cloud plugin + The Azure Cloud plugin allows to use Azure API for the unicast discovery mechanism and add Azure storage repositories. + 1 + cloud_azure + false diff --git a/plugins/cloud-azure/rest-api-spec/test/cloud_azure/10_basic.yaml b/plugins/cloud-azure/rest-api-spec/test/cloud_azure/10_basic.yaml new file mode 100644 index 0000000000000..1abbf2a916293 --- /dev/null +++ b/plugins/cloud-azure/rest-api-spec/test/cloud_azure/10_basic.yaml @@ -0,0 +1,14 @@ +# Integration tests for Cloud Azure components +# +"Cloud Azure loaded": + - do: + cluster.state: {} + + # Get master node id + - set: { master_node: master } + + - do: + nodes.info: {} + + - match: { nodes.$master.plugins.0.name: cloud-azure } + - match: { nodes.$master.plugins.0.jvm: true } diff --git a/plugins/cloud-azure/src/test/java/org/elasticsearch/cloud/azure/CloudAzureRestIT.java b/plugins/cloud-azure/src/test/java/org/elasticsearch/cloud/azure/CloudAzureRestIT.java new file mode 100644 index 0000000000000..3a5da94f06da6 --- /dev/null +++ b/plugins/cloud-azure/src/test/java/org/elasticsearch/cloud/azure/CloudAzureRestIT.java @@ -0,0 +1,41 @@ +/* + * 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.cloud.azure; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; +import org.elasticsearch.test.rest.ElasticsearchRestTestCase; +import org.elasticsearch.test.rest.RestTestCandidate; +import org.elasticsearch.test.rest.parser.RestTestParseException; + +import java.io.IOException; + +public class CloudAzureRestIT extends ElasticsearchRestTestCase { + + public CloudAzureRestIT(@Name("yaml") RestTestCandidate testCandidate) { + super(testCandidate); + } + + @ParametersFactory + public static Iterable parameters() throws IOException, RestTestParseException { + return ElasticsearchRestTestCase.createParameters(0, 1); + } +} + diff --git a/plugins/cloud-gce/pom.xml b/plugins/cloud-gce/pom.xml index 1640df984652d..fc09a2b22cff4 100644 --- a/plugins/cloud-gce/pom.xml +++ b/plugins/cloud-gce/pom.xml @@ -15,19 +15,15 @@ governing permissions and limitations under the License. --> xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 - org.elasticsearch.plugin - elasticsearch-cloud-gce - - - Elasticsearch Google Compute Engine cloud plugin - The Google Compute Engine (GCE) Cloud plugin allows to use GCE API for the unicast discovery mechanism. - - org.elasticsearch + org.elasticsearch.plugin elasticsearch-plugin 2.0.0-SNAPSHOT + elasticsearch-cloud-gce + Elasticsearch Google Compute Engine cloud plugin + The Google Compute Engine (GCE) Cloud plugin allows to use GCE API for the unicast discovery mechanism. v1-rev59-1.20.0 @@ -35,6 +31,8 @@ governing permissions and limitations under the License. --> warn 1 + cloud_gce + false diff --git a/plugins/cloud-gce/rest-api-spec/test/cloud_gce/10_basic.yaml b/plugins/cloud-gce/rest-api-spec/test/cloud_gce/10_basic.yaml new file mode 100644 index 0000000000000..9cff52e6ca308 --- /dev/null +++ b/plugins/cloud-gce/rest-api-spec/test/cloud_gce/10_basic.yaml @@ -0,0 +1,14 @@ +# Integration tests for Cloud GCE components +# +"Cloud GCE loaded": + - do: + cluster.state: {} + + # Get master node id + - set: { master_node: master } + + - do: + nodes.info: {} + + - match: { nodes.$master.plugins.0.name: cloud-gce } + - match: { nodes.$master.plugins.0.jvm: true } diff --git a/plugins/cloud-gce/src/test/java/org/elasticsearch/cloud/gce/CloudGCERestIT.java b/plugins/cloud-gce/src/test/java/org/elasticsearch/cloud/gce/CloudGCERestIT.java new file mode 100644 index 0000000000000..0439e129c18a5 --- /dev/null +++ b/plugins/cloud-gce/src/test/java/org/elasticsearch/cloud/gce/CloudGCERestIT.java @@ -0,0 +1,41 @@ +/* + * 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.cloud.gce; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; +import org.elasticsearch.test.rest.ElasticsearchRestTestCase; +import org.elasticsearch.test.rest.RestTestCandidate; +import org.elasticsearch.test.rest.parser.RestTestParseException; + +import java.io.IOException; + +public class CloudGCERestIT extends ElasticsearchRestTestCase { + + public CloudGCERestIT(@Name("yaml") RestTestCandidate testCandidate) { + super(testCandidate); + } + + @ParametersFactory + public static Iterable parameters() throws IOException, RestTestParseException { + return ElasticsearchRestTestCase.createParameters(0, 1); + } +} + diff --git a/plugins/delete-by-query/pom.xml b/plugins/delete-by-query/pom.xml index 8be711460cf55..c7dfe7227a370 100644 --- a/plugins/delete-by-query/pom.xml +++ b/plugins/delete-by-query/pom.xml @@ -15,19 +15,16 @@ governing permissions and limitations under the License. --> xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 - org.elasticsearch.plugin - elasticsearch-delete-by-query - - jar - Elasticsearch Delete By Query plugin - The Delete By Query plugin allows to delete documents in Elasticsearch with a single query. - - org.elasticsearch + org.elasticsearch.plugin elasticsearch-plugin 2.0.0-SNAPSHOT + elasticsearch-delete-by-query + Elasticsearch Delete By Query plugin + The Delete By Query plugin allows to delete documents in Elasticsearch with a single query. + warn delete_by_query diff --git a/plugins/delete-by-query/src/test/java/org/elasticsearch/plugin/deletebyquery/test/rest/DeleteByQueryRestIT.java b/plugins/delete-by-query/src/test/java/org/elasticsearch/plugin/deletebyquery/test/rest/DeleteByQueryRestIT.java new file mode 100644 index 0000000000000..ec3931fd8753a --- /dev/null +++ b/plugins/delete-by-query/src/test/java/org/elasticsearch/plugin/deletebyquery/test/rest/DeleteByQueryRestIT.java @@ -0,0 +1,41 @@ +/* + * 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.plugin.deletebyquery.test.rest; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; +import org.elasticsearch.test.rest.ElasticsearchRestTestCase; +import org.elasticsearch.test.rest.RestTestCandidate; +import org.elasticsearch.test.rest.parser.RestTestParseException; + +import java.io.IOException; + +public class DeleteByQueryRestIT extends ElasticsearchRestTestCase { + + public DeleteByQueryRestIT(@Name("yaml") RestTestCandidate testCandidate) { + super(testCandidate); + } + + @ParametersFactory + public static Iterable parameters() throws IOException, RestTestParseException { + return ElasticsearchRestTestCase.createParameters(0, 1); + } +} + diff --git a/plugins/lang-javascript/pom.xml b/plugins/lang-javascript/pom.xml index f9573271de146..d72585b604c3b 100644 --- a/plugins/lang-javascript/pom.xml +++ b/plugins/lang-javascript/pom.xml @@ -4,21 +4,20 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 - org.elasticsearch.plugin - elasticsearch-lang-javascript - - jar - Elasticsearch JavaScript language plugin - The JavaScript language plugin allows to have javascript as the language of scripts to execute. - - org.elasticsearch + org.elasticsearch.plugin elasticsearch-plugin 2.0.0-SNAPSHOT + elasticsearch-lang-javascript + Elasticsearch JavaScript language plugin + The JavaScript language plugin allows to have javascript as the language of scripts to execute. + + lang_javascript + false diff --git a/plugins/lang-javascript/rest-api-spec/test/lang_javascript/10_basic.yaml b/plugins/lang-javascript/rest-api-spec/test/lang_javascript/10_basic.yaml new file mode 100644 index 0000000000000..ee77a848c44e9 --- /dev/null +++ b/plugins/lang-javascript/rest-api-spec/test/lang_javascript/10_basic.yaml @@ -0,0 +1,26 @@ +# Integration tests for Lang JavaScript components +# +setup: + - do: + index: + index: test + type: test + id: 1 + body: { "foo": "aaa" } + - do: + indices.refresh: {} + +--- + +"Lang JavaScript": + - do: + search: + body: + script_fields: + bar: + lang: javascript + script: "doc['foo'].value + x" + params: + x: "bbb" + + - match: { hits.hits.0.fields.bar.0: "aaabbb"} diff --git a/plugins/lang-javascript/src/test/java/org/elasticsearch/script/javascript/LangJavaScriptRestIT.java b/plugins/lang-javascript/src/test/java/org/elasticsearch/script/javascript/LangJavaScriptRestIT.java new file mode 100644 index 0000000000000..3bc18af4c77f6 --- /dev/null +++ b/plugins/lang-javascript/src/test/java/org/elasticsearch/script/javascript/LangJavaScriptRestIT.java @@ -0,0 +1,41 @@ +/* + * 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.script.javascript; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; +import org.elasticsearch.test.rest.ElasticsearchRestTestCase; +import org.elasticsearch.test.rest.RestTestCandidate; +import org.elasticsearch.test.rest.parser.RestTestParseException; + +import java.io.IOException; + +public class LangJavaScriptRestIT extends ElasticsearchRestTestCase { + + public LangJavaScriptRestIT(@Name("yaml") RestTestCandidate testCandidate) { + super(testCandidate); + } + + @ParametersFactory + public static Iterable parameters() throws IOException, RestTestParseException { + return ElasticsearchRestTestCase.createParameters(0, 1); + } +} + diff --git a/plugins/lang-python/pom.xml b/plugins/lang-python/pom.xml index 1d52498628c1c..86458279318e0 100644 --- a/plugins/lang-python/pom.xml +++ b/plugins/lang-python/pom.xml @@ -4,21 +4,20 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 - org.elasticsearch.plugin - elasticsearch-lang-python - - jar - Elasticsearch Python language plugin - The Python language plugin allows to have python as the language of scripts to execute. - - org.elasticsearch + org.elasticsearch.plugin elasticsearch-plugin 2.0.0-SNAPSHOT + elasticsearch-lang-python + Elasticsearch Python language plugin + The Python language plugin allows to have python as the language of scripts to execute. + + lang_python + false diff --git a/plugins/lang-python/rest-api-spec/test/lang_python/10_basic.yaml b/plugins/lang-python/rest-api-spec/test/lang_python/10_basic.yaml new file mode 100644 index 0000000000000..ba7b733e806f5 --- /dev/null +++ b/plugins/lang-python/rest-api-spec/test/lang_python/10_basic.yaml @@ -0,0 +1,26 @@ +# Integration tests for Lang Python components +# +setup: + - do: + index: + index: test + type: test + id: 1 + body: { "foo": "aaa" } + - do: + indices.refresh: {} + +--- + +"Lang Python": + - do: + search: + body: + script_fields: + bar: + lang: python + script: "doc['foo'].value + x" + params: + x: "bbb" + + - match: { hits.hits.0.fields.bar.0: "aaabbb"} diff --git a/plugins/lang-python/src/test/java/org/elasticsearch/script/python/LangPythonScriptRestIT.java b/plugins/lang-python/src/test/java/org/elasticsearch/script/python/LangPythonScriptRestIT.java new file mode 100644 index 0000000000000..d9a0bdd8abba3 --- /dev/null +++ b/plugins/lang-python/src/test/java/org/elasticsearch/script/python/LangPythonScriptRestIT.java @@ -0,0 +1,41 @@ +/* + * 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.script.python; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; +import org.elasticsearch.test.rest.ElasticsearchRestTestCase; +import org.elasticsearch.test.rest.RestTestCandidate; +import org.elasticsearch.test.rest.parser.RestTestParseException; + +import java.io.IOException; + +public class LangPythonScriptRestIT extends ElasticsearchRestTestCase { + + public LangPythonScriptRestIT(@Name("yaml") RestTestCandidate testCandidate) { + super(testCandidate); + } + + @ParametersFactory + public static Iterable parameters() throws IOException, RestTestParseException { + return ElasticsearchRestTestCase.createParameters(0, 1); + } +} + diff --git a/plugins/pom.xml b/plugins/pom.xml index acc9af82ec22b..262c035e6486c 100644 --- a/plugins/pom.xml +++ b/plugins/pom.xml @@ -4,6 +4,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 + org.elasticsearch.plugin elasticsearch-plugin 2.0.0-SNAPSHOT pom @@ -39,6 +40,12 @@ test-jar test + + org.elasticsearch + elasticsearch + zip + test + @@ -268,10 +275,16 @@ api/info.json api/cluster.health.json + api/cluster.state.json api/index.json + api/search.json + api/indices.analyze.json + api/indices.create.json api/indices.refresh.json + api/nodes.info.json api/count.json + api/search.json @@ -281,6 +294,53 @@ + + + + org.apache.maven.plugins + maven-antrun-plugin + + + + integ-setup + pre-integration-test + + run + + + + + + + + + + integ-teardown + post-integration-test + + run + + + + + + + + + + + org.apache.maven.plugins + maven-failsafe-plugin + + + 127.0.0.1:9300 + + + + + diff --git a/pom.xml b/pom.xml index 43678cb4c4250..97a55f618fdc9 100644 --- a/pom.xml +++ b/pom.xml @@ -33,6 +33,7 @@ + UTF-8 2.0.0-SNAPSHOT ${java.home}${file.separator}bin${file.separator}java @@ -93,14 +94,18 @@ + - true fail + ${skipTests} + ${skipTests} + ${project.build.directory}/integ-tests + ${integ.scratch}/temp @@ -125,7 +130,7 @@ org.elasticsearch - dev-tools + elasticsearch-dev-tools ${elasticsearch.version} @@ -185,6 +190,13 @@ test-jar + + org.elasticsearch + elasticsearch + ${elasticsearch.version} + zip + + org.apache.httpcomponents httpclient @@ -552,6 +564,7 @@ ${jvm.executable} ${tests.jvm.argline} + ${skip.unit.tests} 10 warn true @@ -682,6 +695,36 @@ true + + org.apache.maven.plugins + maven-failsafe-plugin + 2.18.1 + + ${skip.integ.tests} + + ${es.logger.level} + ${tests.rest.suite} + ${tests.rest.blacklist} + ${tests.rest.spec} + ${tests.rest.load_packaged} + ${integ.temp} + + + + + integration-test + + integration-test + + + + verify + + verify + + + + org.apache.maven.plugins maven-source-plugin @@ -701,8 +744,8 @@ 1.5 - org.elasticsearch:dev-tools:${elasticsearch.version} - org.elasticsearch:rest-api-spec:${elasticsearch.version} + org.elasticsearch:elasticsearch-dev-tools:${elasticsearch.version} + org.elasticsearch:elasticsearch-rest-api-spec:${elasticsearch.version} ${elasticsearch.tools.directory} @@ -1064,7 +1107,7 @@ org.eclipse.jdt.ui.text.custom_code_templates= org.apache.maven.plugins maven-pmd-plugin - 3.3 + 3.5 ${elasticsearch.tools.directory}/pmd/custom.xml @@ -1133,7 +1176,8 @@ org.eclipse.jdt.ui.text.custom_code_templates= Running license check - + + @@ -1220,20 +1264,9 @@ org.eclipse.jdt.ui.text.custom_code_templates=tests-top-hints test - ${skipTests} - - false + ${skip.unit.tests} - - - - - - - - - - + @@ -1244,12 +1277,19 @@ org.eclipse.jdt.ui.text.custom_code_templates= + + + com.carrotsearch.randomizedtesting + junit4-ant + ${testframework.version} + + org.apache.maven.plugins maven-invoker-plugin - 1.10 + 2.0.0 @@ -1306,6 +1346,10 @@ org.eclipse.jdt.ui.text.custom_code_templates=${basedir}/src + + true + true + diff --git a/rest-api-spec/pom.xml b/rest-api-spec/pom.xml index 80cd730b4c2f8..46f2e6e721317 100644 --- a/rest-api-spec/pom.xml +++ b/rest-api-spec/pom.xml @@ -1,7 +1,7 @@ 4.0.0 org.elasticsearch - rest-api-spec + elasticsearch-rest-api-spec 2.0.0-SNAPSHOT Elasticsearch Rest API Spec diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.info.json b/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.info.json index faebeec174e90..d8044c8a5f6c3 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.info.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.info.json @@ -12,7 +12,7 @@ }, "metric": { "type": "list", - "options": ["settings", "os", "process", "jvm", "thread_pool", "network", "transport", "http", "plugins"], + "options": ["settings", "os", "process", "jvm", "thread_pool", "transport", "http", "plugins"], "description": "A comma-separated list of metrics you wish returned. Leave empty to return all." } }, diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.stats.json b/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.stats.json index 59b69c85e4c1a..5eef2c18d32ff 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.stats.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.stats.json @@ -15,7 +15,7 @@ "parts": { "metric" : { "type" : "list", - "options" : ["_all", "breaker", "fs", "http", "indices", "jvm", "network", "os", "process", "thread_pool", "transport"], + "options" : ["_all", "breaker", "fs", "http", "indices", "jvm", "os", "process", "thread_pool", "transport"], "description" : "Limit the information returned to the specified metrics" }, "index_metric" : { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/snapshot.create/10_basic.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/snapshot.create/10_basic.yaml index b341564e9b5cf..91080f99930f6 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/snapshot.create/10_basic.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/snapshot.create/10_basic.yaml @@ -3,11 +3,11 @@ setup: - do: snapshot.create_repository: - repository: test_repo1 + repository: test_repo_create_1 body: type: fs settings: - location: "test_repo1" + location: "test_repo_create_1_loc" - do: indices.create: @@ -26,7 +26,7 @@ setup: - do: snapshot.create: - repository: test_repo1 + repository: test_repo_create_1 snapshot: test_snapshot wait_for_completion: true diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/snapshot.get_repository/10_basic.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/snapshot.get_repository/10_basic.yaml index d20acc9b63085..5788453a73950 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/snapshot.get_repository/10_basic.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/snapshot.get_repository/10_basic.yaml @@ -3,7 +3,7 @@ setup: - do: snapshot.create_repository: - repository: test_repo1 + repository: test_repo_get_1 body: type: url settings: @@ -11,7 +11,7 @@ setup: - do: snapshot.create_repository: - repository: test_repo2 + repository: test_repo_get_2 body: type: url settings: @@ -22,23 +22,23 @@ setup: - do: snapshot.get_repository: {} - - is_true: test_repo1 - - is_true: test_repo2 + - is_true: test_repo_get_1 + - is_true: test_repo_get_2 --- "Get repository by name": - do: snapshot.get_repository: - repository: test_repo1 + repository: test_repo_get_1 - - is_true: test_repo1 - - is_false: test_repo2 + - is_true: test_repo_get_1 + - is_false: test_repo_get_2 --- "Get missing repository by name": - do: snapshot.get_repository: - repository: test_repo2 + repository: test_repo_get_2 --- "Get all repositories with local flag": @@ -46,13 +46,13 @@ setup: snapshot.get_repository: local: true - - is_true: test_repo1 - - is_true: test_repo2 + - is_true: test_repo_get_1 + - is_true: test_repo_get_2 --- "Verify created repository": - do: snapshot.verify_repository: - repository: test_repo2 + repository: test_repo_get_2 - is_true: nodes diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/snapshot.restore/10_basic.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/snapshot.restore/10_basic.yaml index 8cc84ec3610eb..5fa734d36075b 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/snapshot.restore/10_basic.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/snapshot.restore/10_basic.yaml @@ -3,11 +3,11 @@ setup: - do: snapshot.create_repository: - repository: test_repo1 + repository: test_repo_restore_1 body: type: fs settings: - location: "test_repo1" + location: "test_repo_restore_1_loc" - do: indices.create: @@ -26,7 +26,7 @@ setup: - do: snapshot.create: - repository: test_repo1 + repository: test_repo_restore_1 snapshot: test_snapshot wait_for_completion: true @@ -41,7 +41,7 @@ setup: - do: snapshot.restore: - repository: test_repo1 + repository: test_repo_restore_1 snapshot: test_snapshot wait_for_completion: true diff --git a/securemock/pom.xml b/securemock/pom.xml index 0b7491cff2573..8c2faa704bc89 100644 --- a/securemock/pom.xml +++ b/securemock/pom.xml @@ -2,10 +2,10 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd"> 4.0.0 org.elasticsearch - securemock + elasticsearch-securemock 1.0-SNAPSHOT jar - securemock + Elasticsearch Securemock Allows creating mocks in tests without having to grant dangerous permissions to all of your code. 2015