From 44e9b75296a1be86470f61e12783a332095d07ad Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Wed, 13 May 2015 11:01:38 +0200 Subject: [PATCH 01/56] Mappings: Make FieldNameAnalyzer less lenient. In case FieldNameAnalyzer does not find an explicit analyzer for a given field name, it returns the default analyzer. This behaviour can hide bugs where the analyzer fails to be propagated to FieldNameAnalyzer or an analyzer is requested for a field which is not mapped. --- .../index/analysis/FieldNameAnalyzer.java | 10 +++++++--- .../org/elasticsearch/index/mapper/MapperAnalyzer.java | 2 -- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/analysis/FieldNameAnalyzer.java b/src/main/java/org/elasticsearch/index/analysis/FieldNameAnalyzer.java index 84525e6f53a..66002e4ba3e 100644 --- a/src/main/java/org/elasticsearch/index/analysis/FieldNameAnalyzer.java +++ b/src/main/java/org/elasticsearch/index/analysis/FieldNameAnalyzer.java @@ -63,7 +63,9 @@ public final class FieldNameAnalyzer extends DelegatingAnalyzerWrapper { if (analyzer != null) { return analyzer; } - return defaultAnalyzer; + // Don't be lenient here and return the default analyzer + // Fields need to be explicitly added + throw new IllegalArgumentException("Field [" + name + "] has no associated analyzer"); } /** @@ -72,9 +74,11 @@ public final class FieldNameAnalyzer extends DelegatingAnalyzerWrapper { public FieldNameAnalyzer copyAndAddAll(Collection> mappers) { CopyOnWriteHashMap analyzers = this.analyzers; for (Map.Entry entry : mappers) { - if (entry.getValue() != null) { - analyzers = analyzers.copyAndPut(entry.getKey(), entry.getValue()); + Analyzer analyzer = entry.getValue(); + if (analyzer == null) { + analyzer = defaultAnalyzer; } + analyzers = analyzers.copyAndPut(entry.getKey(), analyzer); } return new FieldNameAnalyzer(analyzers, defaultAnalyzer); } diff --git a/src/main/java/org/elasticsearch/index/mapper/MapperAnalyzer.java b/src/main/java/org/elasticsearch/index/mapper/MapperAnalyzer.java index d76fdc6f1de..336d9a3f602 100644 --- a/src/main/java/org/elasticsearch/index/mapper/MapperAnalyzer.java +++ b/src/main/java/org/elasticsearch/index/mapper/MapperAnalyzer.java @@ -19,8 +19,6 @@ package org.elasticsearch.index.mapper; -import java.util.Objects; - import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.DelegatingAnalyzerWrapper; import org.elasticsearch.index.analysis.FieldNameAnalyzer; From 43ff54411798819eb5e24d50ba865764eb88978e Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 13 May 2015 12:40:35 +0200 Subject: [PATCH 02/56] Add translog checkpoints to prevent translog corruption Today we are almost intentionally corrupt the translog if we loose a node due to powerloss or similary disasters. In the translog reading code we simply read until we hit an EOF exception ignoring the rest of the translog file once hit. There is no information stored how many records we are expecting or what the last written offset was. This commit restructures the translog to add checkpoints that are written with every sync operation recording the number of synced operations as well as the last synced offset. These checkpoints are also used to identify the actual transaction log file to open instead of relying on directory traversal. This change adds a significant amount of additional checks and pickyness to the translog code. For instance is the translog now associated with a specific engine via a UUID that is written to each translog file as part of it's header. If an engine opens a translog file it was not associated with the operation will fail. Closes to #10933 Relates to #11011 --- .../action/termvectors/TermVectorsFields.java | 14 +- .../elasticsearch/cluster/ClusterState.java | 3 +- .../common/bytes/BytesArray.java | 3 +- .../io/stream/ByteBufferStreamInput.java | 2 +- .../common/io/stream/BytesStreamInput.java | 154 ---- .../common/io/stream/StreamInput.java | 16 + .../loader/PropertiesSettingsLoader.java | 4 +- .../common/xcontent/XContentHelper.java | 6 +- .../discovery/local/LocalDiscovery.java | 7 +- .../zen/ping/multicast/MulticastZenPing.java | 2 +- .../index/engine/EngineConfig.java | 40 +- .../index/engine/InternalEngine.java | 116 +-- .../settings/IndexDynamicSettingsModule.java | 5 +- .../elasticsearch/index/shard/IndexShard.java | 46 +- .../translog/BufferedChecksumStreamInput.java | 51 ++ ...File.java => BufferingTranslogWriter.java} | 60 +- .../index/translog/ChannelReader.java | 130 --- .../index/translog/ChannelReference.java | 52 +- .../index/translog/ChannelSnapshot.java | 74 -- .../index/translog/Checkpoint.java | 93 ++ .../translog/ChecksummedTranslogStream.java | 138 --- ...ader.java => ImmutableTranslogReader.java} | 33 +- ...gStream.java => LegacyTranslogReader.java} | 46 +- .../translog/LegacyTranslogReaderBase.java | 60 ++ .../index/translog/Translog.java | 827 +++++++++++------- .../index/translog/TranslogConfig.java | 196 +++++ .../index/translog/TranslogFile.java | 240 ----- .../index/translog/TranslogReader.java | 311 +++++++ .../index/translog/TranslogSnapshot.java | 34 +- .../index/translog/TranslogStream.java | 61 -- .../index/translog/TranslogStreams.java | 149 ---- .../index/translog/TranslogWriter.java | 301 +++++++ .../translog/TruncatedTranslogException.java | 7 +- .../memory/IndexingMemoryController.java | 6 +- .../recovery/RecoverySourceHandler.java | 2 +- .../RecoveryTranslogOperationsRequest.java | 9 +- .../SharedFSRecoverySourceHandler.java | 2 +- .../transport/local/LocalTransport.java | 3 +- .../ElasticsearchExceptionTests.java | 4 +- .../action/OriginalIndicesTests.java | 4 +- .../state/ClusterStateRequestTest.java | 8 +- .../indices/stats/IndicesStatsTests.java | 3 +- .../action/get/MultiGetShardRequestTests.java | 4 +- .../get/GetIndexedScriptRequestTests.java | 4 +- .../action/support/IndicesOptionsTests.java | 8 +- .../cluster/ClusterHealthResponsesTests.java | 4 +- .../cluster/ClusterStateDiffTests.java | 4 +- .../cluster/block/ClusterBlockTests.java | 4 +- .../cluster/node/DiscoveryNodeTests.java | 4 +- .../allocation/AllocationCommandsTests.java | 4 +- .../ClusterSerializationTests.java | 4 +- .../cluster/serialization/DiffableTests.java | 4 +- .../elasticsearch/common/io/StreamsTests.java | 4 +- .../common/io/streams/BytesStreamsTests.java | 4 +- .../common/unit/TimeValueTests.java | 4 +- .../common/xcontent/XContentFactoryTests.java | 4 +- .../index/engine/InternalEngineTests.java | 76 +- .../index/engine/ShadowEngineTests.java | 12 +- .../timestamp/TimestampMappingTests.java | 8 +- .../index/shard/IndexShardTests.java | 3 +- .../index/store/CorruptedTranslogTests.java | 6 +- .../index/translog/BufferedTranslogTests.java | 18 +- .../index/translog/FsSimpleTranslogTests.java | 38 - .../index/translog/TranslogTests.java | 481 +++++++++- .../index/translog/TranslogVersionTests.java | 136 ++- .../indices/recovery/RecoveryStateTest.java | 3 +- .../indices/stats/IndexStatsTests.java | 6 +- .../recovery/RecoveryWhileUnderLoadTests.java | 2 - .../test/ElasticsearchIntegrationTest.java | 7 +- .../test/InternalTestCluster.java | 9 +- .../hamcrest/ElasticsearchAssertions.java | 3 +- .../test/transport/MockTransportService.java | 4 +- .../ThreadPoolSerializationTests.java | 5 +- .../transport/TransportMessageTests.java | 4 +- .../index/translog/legacy_translogs.zip | Bin 0 -> 6807 bytes 75 files changed, 2469 insertions(+), 1734 deletions(-) delete mode 100644 src/main/java/org/elasticsearch/common/io/stream/BytesStreamInput.java rename src/main/java/org/elasticsearch/index/translog/{BufferingTranslogFile.java => BufferingTranslogWriter.java} (75%) delete mode 100644 src/main/java/org/elasticsearch/index/translog/ChannelReader.java delete mode 100644 src/main/java/org/elasticsearch/index/translog/ChannelSnapshot.java create mode 100644 src/main/java/org/elasticsearch/index/translog/Checkpoint.java delete mode 100644 src/main/java/org/elasticsearch/index/translog/ChecksummedTranslogStream.java rename src/main/java/org/elasticsearch/index/translog/{ChannelImmutableReader.java => ImmutableTranslogReader.java} (68%) rename src/main/java/org/elasticsearch/index/translog/{LegacyTranslogStream.java => LegacyTranslogReader.java} (55%) create mode 100644 src/main/java/org/elasticsearch/index/translog/LegacyTranslogReaderBase.java create mode 100644 src/main/java/org/elasticsearch/index/translog/TranslogConfig.java delete mode 100644 src/main/java/org/elasticsearch/index/translog/TranslogFile.java create mode 100644 src/main/java/org/elasticsearch/index/translog/TranslogReader.java delete mode 100644 src/main/java/org/elasticsearch/index/translog/TranslogStream.java delete mode 100644 src/main/java/org/elasticsearch/index/translog/TranslogStreams.java create mode 100644 src/main/java/org/elasticsearch/index/translog/TranslogWriter.java delete mode 100644 src/test/java/org/elasticsearch/index/translog/FsSimpleTranslogTests.java create mode 100644 src/test/resources/org/elasticsearch/index/translog/legacy_translogs.zip diff --git a/src/main/java/org/elasticsearch/action/termvectors/TermVectorsFields.java b/src/main/java/org/elasticsearch/action/termvectors/TermVectorsFields.java index 74abe26422f..fb2ce641643 100644 --- a/src/main/java/org/elasticsearch/action/termvectors/TermVectorsFields.java +++ b/src/main/java/org/elasticsearch/action/termvectors/TermVectorsFields.java @@ -28,7 +28,7 @@ import org.apache.lucene.index.TermsEnum; import org.apache.lucene.search.BoostAttribute; import org.apache.lucene.util.*; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.io.stream.BytesStreamInput; +import org.elasticsearch.common.io.stream.StreamInput; import java.io.IOException; import java.util.Arrays; @@ -125,7 +125,7 @@ public final class TermVectorsFields extends Fields { * @param termVectors Stores the actual term vectors as a {@link BytesRef}. */ public TermVectorsFields(BytesReference headerRef, BytesReference termVectors) throws IOException { - BytesStreamInput header = new BytesStreamInput(headerRef); + StreamInput header = StreamInput.wrap(headerRef.toBytesArray()); fieldMap = new ObjectLongOpenHashMap<>(); // here we read the header to fill the field offset map @@ -184,7 +184,7 @@ public final class TermVectorsFields extends Fields { private final class TermVector extends Terms { - private final BytesStreamInput perFieldTermVectorInput; + private final StreamInput perFieldTermVectorInput; private final long readOffset; private long numTerms; @@ -196,7 +196,7 @@ public final class TermVectorsFields extends Fields { private int docCount; public TermVector(BytesReference termVectors, long readOffset) throws IOException { - this.perFieldTermVectorInput = new BytesStreamInput(termVectors); + this.perFieldTermVectorInput = StreamInput.wrap(termVectors.toBytesArray()); this.readOffset = readOffset; reset(); } @@ -269,7 +269,7 @@ public final class TermVectorsFields extends Fields { } } - private void writeInfos(final BytesStreamInput input) throws IOException { + private void writeInfos(final StreamInput input) throws IOException { for (int i = 0; i < freq; i++) { if (hasPositions) { positions[i] = input.readVInt(); @@ -483,7 +483,7 @@ public final class TermVectorsFields extends Fields { // the writer writes a 0 for -1 or value +1 and accordingly we have to // substract 1 again // adds one to mock not existing term freq - int readPotentiallyNegativeVInt(BytesStreamInput stream) throws IOException { + int readPotentiallyNegativeVInt(StreamInput stream) throws IOException { return stream.readVInt() - 1; } @@ -491,7 +491,7 @@ public final class TermVectorsFields extends Fields { // case, the writer writes a 0 for -1 or value +1 and accordingly we have to // substract 1 again // adds one to mock not existing term freq - long readPotentiallyNegativeVLong(BytesStreamInput stream) throws IOException { + long readPotentiallyNegativeVLong(StreamInput stream) throws IOException { return stream.readVLong() - 1; } } \ No newline at end of file diff --git a/src/main/java/org/elasticsearch/cluster/ClusterState.java b/src/main/java/org/elasticsearch/cluster/ClusterState.java index 355b3f19875..85a2313daeb 100644 --- a/src/main/java/org/elasticsearch/cluster/ClusterState.java +++ b/src/main/java/org/elasticsearch/cluster/ClusterState.java @@ -38,7 +38,6 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.compress.CompressedString; -import org.elasticsearch.common.io.stream.BytesStreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -675,7 +674,7 @@ public class ClusterState implements ToXContent, Diffable { * @param localNode used to set the local node in the cluster state. */ public static ClusterState fromBytes(byte[] data, DiscoveryNode localNode) throws IOException { - return readFrom(new BytesStreamInput(data), localNode); + return readFrom(StreamInput.wrap(data), localNode); } /** diff --git a/src/main/java/org/elasticsearch/common/bytes/BytesArray.java b/src/main/java/org/elasticsearch/common/bytes/BytesArray.java index 9fc4098cad3..fd2a9e59e17 100644 --- a/src/main/java/org/elasticsearch/common/bytes/BytesArray.java +++ b/src/main/java/org/elasticsearch/common/bytes/BytesArray.java @@ -22,7 +22,6 @@ package org.elasticsearch.common.bytes; import com.google.common.base.Charsets; import org.apache.lucene.util.BytesRef; import org.elasticsearch.common.io.Channels; -import org.elasticsearch.common.io.stream.BytesStreamInput; import org.elasticsearch.common.io.stream.StreamInput; import org.jboss.netty.buffer.ChannelBuffer; import org.jboss.netty.buffer.ChannelBuffers; @@ -96,7 +95,7 @@ public class BytesArray implements BytesReference { @Override public StreamInput streamInput() { - return new BytesStreamInput(bytes, offset, length); + return StreamInput.wrap(bytes, offset, length); } @Override diff --git a/src/main/java/org/elasticsearch/common/io/stream/ByteBufferStreamInput.java b/src/main/java/org/elasticsearch/common/io/stream/ByteBufferStreamInput.java index 18bc6a0a4a0..d13f539a670 100644 --- a/src/main/java/org/elasticsearch/common/io/stream/ByteBufferStreamInput.java +++ b/src/main/java/org/elasticsearch/common/io/stream/ByteBufferStreamInput.java @@ -72,7 +72,7 @@ public class ByteBufferStreamInput extends StreamInput { @Override public void readBytes(byte[] b, int offset, int len) throws IOException { - if (buffer.remaining() > len) { + if (buffer.remaining() < len) { throw new EOFException(); } buffer.get(b, offset, len); diff --git a/src/main/java/org/elasticsearch/common/io/stream/BytesStreamInput.java b/src/main/java/org/elasticsearch/common/io/stream/BytesStreamInput.java deleted file mode 100644 index db2a676446a..00000000000 --- a/src/main/java/org/elasticsearch/common/io/stream/BytesStreamInput.java +++ /dev/null @@ -1,154 +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.common.io.stream; - -import org.apache.lucene.util.BytesRef; -import org.elasticsearch.common.bytes.BytesArray; -import org.elasticsearch.common.bytes.BytesReference; - -import java.io.EOFException; -import java.io.IOException; - -/** - * - */ -public class BytesStreamInput extends StreamInput { - - protected byte buf[]; - - protected int pos; - - protected int end; - - public BytesStreamInput(BytesReference bytes) { - if (!bytes.hasArray()) { - bytes = bytes.toBytesArray(); - } - this.buf = bytes.array(); - this.pos = bytes.arrayOffset(); - this.end = pos + bytes.length(); - } - - public BytesStreamInput(byte buf[]) { - this(buf, 0, buf.length); - } - - public BytesStreamInput(byte buf[], int offset, int length) { - this.buf = buf; - this.pos = offset; - this.end = offset + length; - } - - @Override - public BytesReference readBytesReference(int length) throws IOException { - BytesArray bytes = new BytesArray(buf, pos, length); - pos += length; - return bytes; - } - - @Override - public BytesRef readBytesRef(int length) throws IOException { - BytesRef bytes = new BytesRef(buf, pos, length); - pos += length; - return bytes; - } - - @Override - public long skip(long n) throws IOException { - if (pos + n > end) { - n = end - pos; - } - if (n < 0) { - return 0; - } - pos += n; - return n; - } - - public int position() { - return this.pos; - } - - @Override - public int read() throws IOException { - return (pos < end) ? (buf[pos++] & 0xff) : -1; - } - - @Override - public int read(byte[] b, int off, int len) throws IOException { - if (b == null) { - throw new NullPointerException(); - } else if (off < 0 || len < 0 || len > b.length - off) { - throw new IndexOutOfBoundsException(); - } - if (pos >= end) { - return -1; - } - if (pos + len > end) { - len = end - pos; - } - if (len <= 0) { - return 0; - } - System.arraycopy(buf, pos, b, off, len); - pos += len; - return len; - } - - public byte[] underlyingBuffer() { - return buf; - } - - @Override - public byte readByte() throws IOException { - if (pos >= end) { - throw new EOFException(); - } - return buf[pos++]; - } - - @Override - public void readBytes(byte[] b, int offset, int len) throws IOException { - if (len == 0) { - return; - } - if (pos >= end) { - throw new EOFException(); - } - if (pos + len > end) { - len = end - pos; - } - if (len <= 0) { - throw new EOFException(); - } - System.arraycopy(buf, pos, b, offset, len); - pos += len; - } - - @Override - public void reset() throws IOException { - pos = 0; - } - - @Override - public void close() throws IOException { - // nothing to do here... - } -} diff --git a/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java b/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java index 5a19b12be0a..82322b518c1 100644 --- a/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java +++ b/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java @@ -30,6 +30,7 @@ import org.elasticsearch.common.text.StringAndBytesText; import org.elasticsearch.common.text.Text; import org.joda.time.DateTime; +import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.InputStream; import java.io.ObjectInputStream; @@ -490,4 +491,19 @@ public abstract class StreamInput extends InputStream { throw new IOException("failed to deserialize exception", e); } } + + public static StreamInput wrap(BytesReference reference) { + if (reference.hasArray() == false) { + reference = reference.toBytesArray(); + } + return wrap(reference.array(), reference.arrayOffset(), reference.length()); + } + + public static StreamInput wrap(byte[] bytes) { + return wrap(bytes, 0, bytes.length); + } + + public static StreamInput wrap(byte[] bytes, int offset, int length) { + return new InputStreamStreamInput(new ByteArrayInputStream(bytes, offset, length)); + } } diff --git a/src/main/java/org/elasticsearch/common/settings/loader/PropertiesSettingsLoader.java b/src/main/java/org/elasticsearch/common/settings/loader/PropertiesSettingsLoader.java index c148913f845..0bc97376bb0 100644 --- a/src/main/java/org/elasticsearch/common/settings/loader/PropertiesSettingsLoader.java +++ b/src/main/java/org/elasticsearch/common/settings/loader/PropertiesSettingsLoader.java @@ -21,7 +21,7 @@ package org.elasticsearch.common.settings.loader; import org.apache.lucene.util.IOUtils; import org.elasticsearch.common.io.FastStringReader; -import org.elasticsearch.common.io.stream.BytesStreamInput; +import org.elasticsearch.common.io.stream.StreamInput; import java.io.IOException; import java.util.Map; @@ -53,7 +53,7 @@ public class PropertiesSettingsLoader implements SettingsLoader { @Override public Map load(byte[] source) throws IOException { Properties props = new Properties(); - BytesStreamInput stream = new BytesStreamInput(source); + StreamInput stream = StreamInput.wrap(source); try { props.load(stream); Map result = newHashMap(); diff --git a/src/main/java/org/elasticsearch/common/xcontent/XContentHelper.java b/src/main/java/org/elasticsearch/common/xcontent/XContentHelper.java index 7c3221a4d62..e92582ed555 100644 --- a/src/main/java/org/elasticsearch/common/xcontent/XContentHelper.java +++ b/src/main/java/org/elasticsearch/common/xcontent/XContentHelper.java @@ -30,7 +30,7 @@ import org.elasticsearch.common.compress.CompressedStreamInput; import org.elasticsearch.common.compress.Compressor; import org.elasticsearch.common.compress.CompressorFactory; import org.elasticsearch.common.io.Streams; -import org.elasticsearch.common.io.stream.BytesStreamInput; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.xcontent.ToXContent.Params; import java.io.IOException; @@ -65,7 +65,7 @@ public class XContentHelper { public static XContentParser createParser(byte[] data, int offset, int length) throws IOException { Compressor compressor = CompressorFactory.compressor(data, offset, length); if (compressor != null) { - CompressedStreamInput compressedInput = compressor.streamInput(new BytesStreamInput(data, offset, length)); + CompressedStreamInput compressedInput = compressor.streamInput(StreamInput.wrap(data, offset, length)); XContentType contentType = XContentFactory.xContentType(compressedInput); compressedInput.resetToBufferStart(); return XContentFactory.xContent(contentType).createParser(compressedInput); @@ -111,7 +111,7 @@ public class XContentHelper { XContentType contentType; Compressor compressor = CompressorFactory.compressor(data, offset, length); if (compressor != null) { - CompressedStreamInput compressedStreamInput = compressor.streamInput(new BytesStreamInput(data, offset, length)); + CompressedStreamInput compressedStreamInput = compressor.streamInput(StreamInput.wrap(data, offset, length)); contentType = XContentFactory.xContentType(compressedStreamInput); compressedStreamInput.resetToBufferStart(); parser = XContentFactory.xContent(contentType).createParser(compressedStreamInput); diff --git a/src/main/java/org/elasticsearch/discovery/local/LocalDiscovery.java b/src/main/java/org/elasticsearch/discovery/local/LocalDiscovery.java index b9ffc1abae9..3df4ffafc84 100644 --- a/src/main/java/org/elasticsearch/discovery/local/LocalDiscovery.java +++ b/src/main/java/org/elasticsearch/discovery/local/LocalDiscovery.java @@ -20,7 +20,6 @@ package org.elasticsearch.discovery.local; import com.google.common.base.Objects; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; import org.elasticsearch.cluster.*; import org.elasticsearch.cluster.block.ClusterBlocks; @@ -32,8 +31,8 @@ import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.internal.Nullable; -import org.elasticsearch.common.io.stream.BytesStreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; @@ -47,8 +46,6 @@ import java.util.Set; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; import static com.google.common.collect.Sets.newHashSet; import static org.elasticsearch.cluster.ClusterState.Builder; @@ -330,7 +327,7 @@ public class LocalDiscovery extends AbstractLifecycleComponent implem clusterStateDiffBytes = os.bytes().toBytes(); } try { - newNodeSpecificClusterState = discovery.lastProcessedClusterState.readDiffFrom(new BytesStreamInput(clusterStateDiffBytes)).apply(discovery.lastProcessedClusterState); + newNodeSpecificClusterState = discovery.lastProcessedClusterState.readDiffFrom(StreamInput.wrap(clusterStateDiffBytes)).apply(discovery.lastProcessedClusterState); logger.debug("sending diff cluster state version with size {} to [{}]", clusterStateDiffBytes.length, discovery.localNode.getName()); } catch (IncompatibleClusterStateVersionException ex) { logger.warn("incompatible cluster state version - resending complete cluster state", ex); diff --git a/src/main/java/org/elasticsearch/discovery/zen/ping/multicast/MulticastZenPing.java b/src/main/java/org/elasticsearch/discovery/zen/ping/multicast/MulticastZenPing.java index 0d5ca7260b4..bc75092dc82 100644 --- a/src/main/java/org/elasticsearch/discovery/zen/ping/multicast/MulticastZenPing.java +++ b/src/main/java/org/elasticsearch/discovery/zen/ping/multicast/MulticastZenPing.java @@ -391,7 +391,7 @@ public class MulticastZenPing extends AbstractLifecycleComponent implem } } if (internal) { - StreamInput input = new BytesStreamInput(new BytesArray(data.toBytes(), INTERNAL_HEADER.length, data.length() - INTERNAL_HEADER.length)); + StreamInput input = StreamInput.wrap(new BytesArray(data.toBytes(), INTERNAL_HEADER.length, data.length() - INTERNAL_HEADER.length)); Version version = Version.readVersion(input); input.setVersion(version); id = input.readInt(); diff --git a/src/main/java/org/elasticsearch/index/engine/EngineConfig.java b/src/main/java/org/elasticsearch/index/engine/EngineConfig.java index bf192c10612..19d5a145c10 100644 --- a/src/main/java/org/elasticsearch/index/engine/EngineConfig.java +++ b/src/main/java/org/elasticsearch/index/engine/EngineConfig.java @@ -29,7 +29,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy; @@ -40,10 +39,10 @@ import org.elasticsearch.index.settings.IndexSettingsService; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.TranslogRecoveryPerformer; import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.translog.TranslogConfig; import org.elasticsearch.indices.IndicesWarmer; import org.elasticsearch.threadpool.ThreadPool; -import java.nio.file.Path; import java.util.concurrent.TimeUnit; /* @@ -76,11 +75,9 @@ public final class EngineConfig { private final Similarity similarity; private final CodecService codecService; private final Engine.FailedEngineListener failedEngineListener; - private final boolean ignoreUnknownTranslog; + private final boolean forceNewTranslog; private final QueryCache filterCache; private final QueryCachingPolicy filterCachingPolicy; - private final BigArrays bigArrays; - private final Path translogPath; /** * Index setting for index concurrency / number of threadstates in the indexwriter. @@ -126,7 +123,7 @@ public final class EngineConfig { /** if set to true the engine will start even if the translog id in the commit point can not be found */ - public static final String INDEX_IGNORE_UNKNOWN_TRANSLOG = "index.engine.ignore_unknown_translog"; + public static final String INDEX_FORCE_NEW_TRANSLOG = "index.engine.force_new_translog"; public static final TimeValue DEFAULT_REFRESH_INTERVAL = new TimeValue(1, TimeUnit.SECONDS); @@ -137,6 +134,7 @@ public final class EngineConfig { public static final String DEFAULT_VERSION_MAP_SIZE = "25%"; private static final String DEFAULT_CODEC_NAME = "default"; + private TranslogConfig translogConfig; /** @@ -146,7 +144,7 @@ public final class EngineConfig { IndexSettingsService indexSettingsService, IndicesWarmer warmer, Store store, SnapshotDeletionPolicy deletionPolicy, MergePolicyProvider mergePolicyProvider, MergeSchedulerProvider mergeScheduler, Analyzer analyzer, Similarity similarity, CodecService codecService, Engine.FailedEngineListener failedEngineListener, - TranslogRecoveryPerformer translogRecoveryPerformer, QueryCache filterCache, QueryCachingPolicy filterCachingPolicy, BigArrays bigArrays, Path translogPath) { + TranslogRecoveryPerformer translogRecoveryPerformer, QueryCache filterCache, QueryCachingPolicy filterCachingPolicy, TranslogConfig translogConfig) { this.shardId = shardId; this.threadPool = threadPool; this.indexingService = indexingService; @@ -160,8 +158,6 @@ public final class EngineConfig { this.similarity = similarity; this.codecService = codecService; this.failedEngineListener = failedEngineListener; - this.bigArrays = bigArrays; - this.translogPath = translogPath; Settings indexSettings = indexSettingsService.getSettings(); this.optimizeAutoGenerateId = indexSettings.getAsBoolean(EngineConfig.INDEX_OPTIMIZE_AUTOGENERATED_ID_SETTING, false); this.compoundOnFlush = indexSettings.getAsBoolean(EngineConfig.INDEX_COMPOUND_ON_FLUSH, compoundOnFlush); @@ -172,9 +168,10 @@ public final class EngineConfig { versionMapSizeSetting = indexSettings.get(INDEX_VERSION_MAP_SIZE, DEFAULT_VERSION_MAP_SIZE); updateVersionMapSize(); this.translogRecoveryPerformer = translogRecoveryPerformer; - this.ignoreUnknownTranslog = indexSettings.getAsBoolean(INDEX_IGNORE_UNKNOWN_TRANSLOG, false); + this.forceNewTranslog = indexSettings.getAsBoolean(INDEX_FORCE_NEW_TRANSLOG, false); this.filterCache = filterCache; this.filterCachingPolicy = filterCachingPolicy; + this.translogConfig = translogConfig; } /** updates {@link #versionMapSize} based on current setting and {@link #indexingBufferSize} */ @@ -203,8 +200,8 @@ public final class EngineConfig { } /** if true the engine will start even if the translog id in the commit point can not be found */ - public boolean getIgnoreUnknownTranslog() { - return ignoreUnknownTranslog; + public boolean forceNewTranlog() { + return forceNewTranslog; } /** @@ -429,23 +426,14 @@ public final class EngineConfig { } /** - * Returns a BigArrays instance for this engine + * Returns the translog config for this engine */ - public BigArrays getBigArrays() { - return bigArrays; + public TranslogConfig getTranslogConfig() { + return translogConfig; } - /** - * Returns the translog path for this engine - */ - public Path getTranslogPath() { - return translogPath; - } - - /** - * Returns the {@link org.elasticsearch.index.settings.IndexSettingsService} for this engine. - */ - public IndexSettingsService getIndesSettingService() { + IndexSettingsService getIndexSettingsService() { // for testing return indexSettingsService; } + } diff --git a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 95e3a202819..499a9495333 100644 --- a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -21,19 +21,8 @@ package org.elasticsearch.index.engine; import com.google.common.collect.Lists; -import org.apache.lucene.index.DirectoryReader; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.*; import org.apache.lucene.index.IndexWriter.IndexReaderWarmer; -import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.LeafReader; -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.index.LiveIndexWriterConfig; -import org.apache.lucene.index.MergePolicy; -import org.apache.lucene.index.MultiReader; -import org.apache.lucene.index.SegmentCommitInfo; -import org.apache.lucene.index.SegmentInfos; -import org.apache.lucene.index.Term; import org.apache.lucene.search.BooleanClause.Occur; import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.IndexSearcher; @@ -46,6 +35,7 @@ import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.InfoStream; import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.Version; import org.elasticsearch.cluster.routing.DjbHashFunction; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.lease.Releasable; @@ -66,17 +56,14 @@ import org.elasticsearch.index.merge.scheduler.MergeSchedulerProvider; import org.elasticsearch.index.search.nested.IncludeNestedDocsQuery; import org.elasticsearch.index.shard.TranslogRecoveryPerformer; import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.index.translog.TranslogConfig; +import org.elasticsearch.index.translog.TranslogCorruptedException; import org.elasticsearch.indices.IndicesWarmer; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.threadpool.ThreadPool; -import java.io.FileNotFoundException; import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Set; +import java.util.*; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.Lock; @@ -86,7 +73,6 @@ import java.util.concurrent.locks.ReentrantLock; * */ public class InternalEngine extends Engine { - private final FailEngineOnMergeFailure mergeSchedulerFailureListener; private final MergeSchedulerListener mergeSchedulerListener; @@ -144,13 +130,14 @@ public class InternalEngine extends Engine { throttle = new IndexThrottle(); this.searcherFactory = new SearchFactory(engineConfig); - final Long committedTranslogId; + final Translog.TranslogGeneration translogGeneration; try { writer = createWriter(); indexWriter = writer; - translog = new Translog(engineConfig.getShardId(), engineConfig.getIndesSettingService(), engineConfig.getBigArrays(), engineConfig.getTranslogPath(), engineConfig.getThreadPool()); - committedTranslogId = loadCommittedTranslogId(writer, translog); - } catch (IOException e) { + translog = openTranslog(engineConfig, writer, skipInitialTranslogRecovery || engineConfig.forceNewTranlog()); + translogGeneration = translog.getGeneration(); + assert translogGeneration != null; + } catch (IOException | TranslogCorruptedException e) { throw new EngineCreationFailureException(shardId, "failed to create engine", e); } this.translog = translog; @@ -164,9 +151,9 @@ public class InternalEngine extends Engine { try { if (skipInitialTranslogRecovery) { // make sure we point at the latest translog from now on.. - commitIndexWriter(writer, translog.currentId()); + commitIndexWriter(writer, translog.getGeneration()); } else { - recoverFromTranslog(engineConfig, committedTranslogId); + recoverFromTranslog(engineConfig, translogGeneration); } } catch (IOException | EngineException ex) { throw new EngineCreationFailureException(shardId, "failed to recover from translog", ex); @@ -185,26 +172,35 @@ public class InternalEngine extends Engine { logger.trace("created new InternalEngine"); } + private Translog openTranslog(EngineConfig engineConfig, IndexWriter writer, boolean createNew) throws IOException { + final Translog.TranslogGeneration generation = loadTranslogIdFromCommit(writer); + Translog translog; + TranslogConfig translogConfig = engineConfig.getTranslogConfig(); + if (createNew) { + translog = new Translog(translogConfig); + } else { + translogConfig.setTranslogGeneration(generation); + if (generation != null && generation.translogUUID == null) { + // only upgrade on pre-2.0 indices... + Translog.upgradeLegacyTranslog(logger, translogConfig); + } + translog = new Translog(translogConfig); + } + + if (generation == null) { + logger.debug("no translog ID present in the current generation - creating one"); + commitIndexWriter(writer, translog.getGeneration()); + } + return translog; + } + @Override public Translog getTranslog() { ensureOpen(); return translog; } - protected void recoverFromTranslog(EngineConfig engineConfig, Long committedTranslogId) throws IOException { - if (committedTranslogId != null) { - try { - // trim unneeded files - translog.markCommitted(committedTranslogId); - } catch (FileNotFoundException ex) { - if (engineConfig.getIgnoreUnknownTranslog()) { - logger.warn("ignoring committed translog id [{}] ([{}] set to true)", committedTranslogId, - EngineConfig.INDEX_IGNORE_UNKNOWN_TRANSLOG); - } else { - throw ex; - } - } - } + protected void recoverFromTranslog(EngineConfig engineConfig, Translog.TranslogGeneration translogGeneration) throws IOException { int opsRecovered = 0; final TranslogRecoveryPerformer handler = engineConfig.getTranslogRecoveryPerformer(); try (Translog.Snapshot snapshot = translog.newSnapshot()) { @@ -225,12 +221,12 @@ public class InternalEngine extends Engine { } catch (Throwable e) { throw new EngineException(shardId, "failed to recover from translog", e); } + // flush if we recovered something or if we have references to older translogs // note: if opsRecovered == 0 and we have older translogs it means they are corrupted or 0 length. - if (opsRecovered > 0 || - (committedTranslogId != null && translog.currentId() != committedTranslogId)) { + if (opsRecovered > 0 || translog.isCurrent(translogGeneration) == false) { logger.trace("flushing post recovery from translog. ops recovered [{}]. committed translog id [{}]. current id [{}]", - opsRecovered, committedTranslogId, translog.currentId()); + opsRecovered, translogGeneration == null ? null : translogGeneration.translogFileGeneration, translog.currentFileGeneration()); flush(true, true); } } @@ -240,15 +236,19 @@ public class InternalEngine extends Engine { * translog id into lucene and returns null. */ @Nullable - private Long loadCommittedTranslogId(IndexWriter writer, Translog translog) throws IOException { + private Translog.TranslogGeneration loadTranslogIdFromCommit(IndexWriter writer) throws IOException { // commit on a just opened writer will commit even if there are no changes done to it // we rely on that for the commit data translog id key final Map commitUserData = writer.getCommitData(); - if (commitUserData.containsKey(Translog.TRANSLOG_ID_KEY)) { - return Long.parseLong(commitUserData.get(Translog.TRANSLOG_ID_KEY)); + if (commitUserData.containsKey("translog_id")) { + assert commitUserData.containsKey(Translog.TRANSLOG_UUID_KEY) == false : "legacy commit contains translog UUID"; + return new Translog.TranslogGeneration(null, Long.parseLong(commitUserData.get("translog_id"))); + } else if (commitUserData.containsKey(Translog.TRANSLOG_GENERATION_KEY)) { + assert commitUserData.containsKey(Translog.TRANSLOG_UUID_KEY) : "commit doesn't contain translog UUID"; + final String translogUUID = commitUserData.get(Translog.TRANSLOG_UUID_KEY); + final long translogGen = Long.parseLong(commitUserData.get(Translog.TRANSLOG_GENERATION_KEY)); + return new Translog.TranslogGeneration(translogUUID, translogGen); } - logger.debug("no translog ID present in the current commit - creating one"); - commitIndexWriter(writer, translog.currentId()); return null; } @@ -702,15 +702,16 @@ public class InternalEngine extends Engine { if (commitTranslog) { if (flushNeeded || force) { flushNeeded = false; - final long translogId; + final Translog.TranslogGeneration translogGeneration; try { - translogId = translog.newTranslog(); + translog.prepareCommit(); + translogGeneration = translog.getGeneration(); logger.trace("starting commit for flush; commitTranslog=true"); - commitIndexWriter(indexWriter, translogId); + commitIndexWriter(indexWriter, translogGeneration); logger.trace("finished commit for flush"); + translog.commit(); // we need to refresh in order to clear older version values refresh("version_table_flush"); - translog.markCommitted(translogId); } catch (Throwable e) { throw new FlushFailedEngineException(shardId, e); @@ -721,11 +722,11 @@ public class InternalEngine extends Engine { // translog on an index that was opened on a committed point in time that is "in the future" // of that translog // we allow to *just* commit if there is an ongoing recovery happening... - // its ok to use this, only a flush will cause a new translogId, and we are locked here from + // its ok to use this, only a flush will cause a new translogFileGeneration, and we are locked here from // other flushes use flushLock try { logger.trace("starting commit for flush; commitTranslog=false"); - commitIndexWriter(indexWriter, translog.currentId()); + commitIndexWriter(indexWriter, translog.getGeneration()); logger.trace("finished commit for flush"); } catch (Throwable e) { throw new FlushFailedEngineException(shardId, e); @@ -1175,10 +1176,13 @@ public class InternalEngine extends Engine { } - private void commitIndexWriter(IndexWriter writer, long translogId) throws IOException { + private void commitIndexWriter(IndexWriter writer, Translog.TranslogGeneration commit) throws IOException { try { - logger.trace("committing writer with translog id [{}] ", translogId); - indexWriter.setCommitData(Collections.singletonMap(Translog.TRANSLOG_ID_KEY, Long.toString(translogId))); + logger.trace("committing writer with translog id [{}] ", commit.translogFileGeneration); + Map commitData = new HashMap<>(2); + commitData.put(Translog.TRANSLOG_GENERATION_KEY, Long.toString(commit.translogFileGeneration)); + commitData.put(Translog.TRANSLOG_UUID_KEY, commit.translogUUID); + indexWriter.setCommitData(commitData); writer.commit(); } catch (Throwable ex) { failEngine("lucene commit failed", ex); diff --git a/src/main/java/org/elasticsearch/index/settings/IndexDynamicSettingsModule.java b/src/main/java/org/elasticsearch/index/settings/IndexDynamicSettingsModule.java index 0bd73d92143..ad11d3a3caa 100644 --- a/src/main/java/org/elasticsearch/index/settings/IndexDynamicSettingsModule.java +++ b/src/main/java/org/elasticsearch/index/settings/IndexDynamicSettingsModule.java @@ -37,6 +37,7 @@ import org.elasticsearch.index.merge.scheduler.ConcurrentMergeSchedulerProvider; import org.elasticsearch.index.search.slowlog.ShardSlowLogSearchService; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.store.IndexStore; +import org.elasticsearch.index.translog.TranslogConfig; import org.elasticsearch.index.translog.TranslogService; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.IndicesWarmer; @@ -64,7 +65,7 @@ public class IndexDynamicSettingsModule extends AbstractModule { indexDynamicSettings.addDynamicSetting(DisableAllocationDecider.INDEX_ROUTING_ALLOCATION_DISABLE_ALLOCATION); indexDynamicSettings.addDynamicSetting(DisableAllocationDecider.INDEX_ROUTING_ALLOCATION_DISABLE_NEW_ALLOCATION); indexDynamicSettings.addDynamicSetting(DisableAllocationDecider.INDEX_ROUTING_ALLOCATION_DISABLE_REPLICA_ALLOCATION); - indexDynamicSettings.addDynamicSetting(Translog.INDEX_TRANSLOG_FS_TYPE); + indexDynamicSettings.addDynamicSetting(TranslogConfig.INDEX_TRANSLOG_FS_TYPE); indexDynamicSettings.addDynamicSetting(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, Validator.NON_NEGATIVE_INTEGER); indexDynamicSettings.addDynamicSetting(IndexMetaData.SETTING_AUTO_EXPAND_REPLICAS); indexDynamicSettings.addDynamicSetting(IndexMetaData.SETTING_READ_ONLY); @@ -118,7 +119,7 @@ public class IndexDynamicSettingsModule extends AbstractModule { indexDynamicSettings.addDynamicSetting(TranslogService.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE, Validator.BYTES_SIZE); indexDynamicSettings.addDynamicSetting(TranslogService.INDEX_TRANSLOG_FLUSH_THRESHOLD_PERIOD, Validator.TIME); indexDynamicSettings.addDynamicSetting(TranslogService.INDEX_TRANSLOG_DISABLE_FLUSH); - indexDynamicSettings.addDynamicSetting(Translog.INDEX_TRANSLOG_DURABILITY); + indexDynamicSettings.addDynamicSetting(TranslogConfig.INDEX_TRANSLOG_DURABILITY); indexDynamicSettings.addDynamicSetting(IndicesWarmer.INDEX_WARMER_ENABLED); indexDynamicSettings.addDynamicSetting(IndicesQueryCache.INDEX_CACHE_QUERY_ENABLED, Validator.BOOLEAN); } diff --git a/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 4839e4debbb..2d241a361ac 100644 --- a/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -101,7 +101,9 @@ import org.elasticsearch.index.suggest.stats.ShardSuggestService; import org.elasticsearch.index.suggest.stats.SuggestStats; import org.elasticsearch.index.termvectors.ShardTermVectorsService; import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.index.translog.TranslogConfig; import org.elasticsearch.index.translog.TranslogStats; +import org.elasticsearch.index.translog.TranslogWriter; import org.elasticsearch.index.warmer.ShardIndexWarmerService; import org.elasticsearch.index.warmer.WarmerStats; import org.elasticsearch.indices.IndicesLifecycle; @@ -163,6 +165,7 @@ public class IndexShard extends AbstractIndexShardComponent { private final MergePolicyProvider mergePolicyProvider; private final BigArrays bigArrays; private final EngineConfig engineConfig; + private final TranslogConfig translogConfig; private TimeValue refreshInterval; @@ -252,7 +255,10 @@ public class IndexShard extends AbstractIndexShardComponent { logger.debug("state: [CREATED]"); this.checkIndexOnStartup = indexSettings.get("index.shard.check_on_startup", "false"); - this.engineConfig = newEngineConfig(); + this.translogConfig = new TranslogConfig(shardId, shardPath().resolveTranslog(), indexSettings, getFromSettings(logger, indexSettings, Translog.Durabilty.REQUEST), + bigArrays, threadPool); + this.engineConfig = newEngineConfig(translogConfig); + this.indexShardOperationCounter = new IndexShardOperationCounter(logger, shardId); } @@ -265,10 +271,6 @@ public class IndexShard extends AbstractIndexShardComponent { return true; } - public Translog.View newTranslogView() { - return engine().getTranslog().newView(); - } - public ShardIndexingService indexingService() { return this.indexingService; } @@ -844,10 +846,6 @@ public class IndexShard extends AbstractIndexShardComponent { Map recoveredTypes = internalPerformTranslogRecovery(true); assert recoveredTypes.isEmpty(); assert recoveryState.getTranslog().recoveredOperations() == 0; - if (wipeTranslogs) { - final Translog translog = engine().getTranslog(); - translog.markCommitted(translog.currentId()); - } } /** @@ -1006,7 +1004,7 @@ public class IndexShard extends AbstractIndexShardComponent { } public void markAsInactive() { - updateBufferSize(EngineConfig.INACTIVE_SHARD_INDEXING_BUFFER, Translog.INACTIVE_SHARD_TRANSLOG_BUFFER); + updateBufferSize(EngineConfig.INACTIVE_SHARD_INDEXING_BUFFER, TranslogConfig.INACTIVE_SHARD_TRANSLOG_BUFFER); } public final boolean isFlushOnClose() { @@ -1044,6 +1042,18 @@ public class IndexShard extends AbstractIndexShardComponent { IndexShard.this.flushOnClose = flushOnClose; } + TranslogWriter.Type type = TranslogWriter.Type.fromString(settings.get(TranslogConfig.INDEX_TRANSLOG_FS_TYPE, translogConfig.getType().name())); + if (type != translogConfig.getType()) { + logger.info("updating type from [{}] to [{}]", translogConfig.getType(), type); + translogConfig.setType(type); + } + + final Translog.Durabilty durabilty = getFromSettings(logger, settings, translogConfig.getDurabilty()); + if (durabilty != translogConfig.getDurabilty()) { + logger.info("updating durability from [{}] to [{}]", translogConfig.getDurabilty(), durabilty); + translogConfig.setDurabilty(durabilty); + } + TimeValue refreshInterval = settings.getAsTime(INDEX_REFRESH_INTERVAL, IndexShard.this.refreshInterval); if (!refreshInterval.equals(IndexShard.this.refreshInterval)) { logger.info("updating refresh_interval from [{}] to [{}]", IndexShard.this.refreshInterval, refreshInterval); @@ -1308,7 +1318,7 @@ public class IndexShard extends AbstractIndexShardComponent { return mapperService.documentMapperWithAutoCreate(type); } - private final EngineConfig newEngineConfig() { + private final EngineConfig newEngineConfig(TranslogConfig translogConfig) { final TranslogRecoveryPerformer translogRecoveryPerformer = new TranslogRecoveryPerformer(mapperService, mapperAnalyzer, queryParserService, indexAliasesService, indexCache) { @Override protected void operationProcessed() { @@ -1318,7 +1328,7 @@ public class IndexShard extends AbstractIndexShardComponent { }; return new EngineConfig(shardId, threadPool, indexingService, indexSettingsService, warmer, store, deletionPolicy, mergePolicyProvider, mergeScheduler, - mapperAnalyzer, similarityService.similarity(), codecService, failedEngineListener, translogRecoveryPerformer, indexCache.filter(), indexCache.filterPolicy(), bigArrays, shardPath().resolveTranslog()); + mapperAnalyzer, similarityService.similarity(), codecService, failedEngineListener, translogRecoveryPerformer, indexCache.filter(), indexCache.filterPolicy(), translogConfig); } private static class IndexShardOperationCounter extends AbstractRefCounted { @@ -1371,6 +1381,16 @@ public class IndexShard extends AbstractIndexShardComponent { * Returns the current translog durability mode */ public Translog.Durabilty getTranslogDurability() { - return engine().getTranslog().getDurabilty(); + return translogConfig.getDurabilty(); + } + + private static Translog.Durabilty getFromSettings(ESLogger logger, Settings settings, Translog.Durabilty defaultValue) { + final String value = settings.get(TranslogConfig.INDEX_TRANSLOG_DURABILITY, defaultValue.name()); + try { + return Translog.Durabilty.valueOf(value.toUpperCase(Locale.ROOT)); + } catch (IllegalArgumentException ex) { + logger.warn("Can't apply {} illegal value: {} using {} instead, use one of: {}", TranslogConfig.INDEX_TRANSLOG_DURABILITY, value, defaultValue, Arrays.toString(Translog.Durabilty.values())); + return defaultValue; + } } } diff --git a/src/main/java/org/elasticsearch/index/translog/BufferedChecksumStreamInput.java b/src/main/java/org/elasticsearch/index/translog/BufferedChecksumStreamInput.java index 6c83ed34b6b..58aa60a23c8 100644 --- a/src/main/java/org/elasticsearch/index/translog/BufferedChecksumStreamInput.java +++ b/src/main/java/org/elasticsearch/index/translog/BufferedChecksumStreamInput.java @@ -31,6 +31,8 @@ import java.util.zip.Checksum; * {@link StreamInput} so anything read will update the checksum */ public final class BufferedChecksumStreamInput extends StreamInput { + private static final int SKIP_BUFFER_SIZE = 1024; + private byte[] skipBuffer; private final StreamInput in; private final Checksum digest; @@ -39,6 +41,17 @@ public final class BufferedChecksumStreamInput extends StreamInput { this.digest = new BufferedChecksum(new CRC32()); } + public BufferedChecksumStreamInput(StreamInput in, BufferedChecksumStreamInput reuse) { + this.in = in; + if (reuse == null ) { + this.digest = new BufferedChecksum(new CRC32()); + } else { + this.digest = reuse.digest; + digest.reset(); + this.skipBuffer = reuse.skipBuffer; + } + } + public long getChecksum() { return this.digest.getValue(); } @@ -71,4 +84,42 @@ public final class BufferedChecksumStreamInput extends StreamInput { public void close() throws IOException { in.close(); } + + @Override + public boolean markSupported() { + return in.markSupported(); + } + + + @Override + public long skip(long numBytes) throws IOException { + if (numBytes < 0) { + throw new IllegalArgumentException("numBytes must be >= 0, got " + numBytes); + } + if (skipBuffer == null) { + skipBuffer = new byte[SKIP_BUFFER_SIZE]; + } + assert skipBuffer.length == SKIP_BUFFER_SIZE; + long skipped = 0; + for (; skipped < numBytes; ) { + final int step = (int) Math.min(SKIP_BUFFER_SIZE, numBytes - skipped); + readBytes(skipBuffer, 0, step); + skipped += step; + } + return skipped; + } + + @Override + public int available() throws IOException { + return in.available(); + } + + @Override + public synchronized void mark(int readlimit) { + in.mark(readlimit); + } + + public void resetDigest() { + digest.reset(); + } } diff --git a/src/main/java/org/elasticsearch/index/translog/BufferingTranslogFile.java b/src/main/java/org/elasticsearch/index/translog/BufferingTranslogWriter.java similarity index 75% rename from src/main/java/org/elasticsearch/index/translog/BufferingTranslogFile.java rename to src/main/java/org/elasticsearch/index/translog/BufferingTranslogWriter.java index b7eb4dc9f2a..0e84c73f47a 100644 --- a/src/main/java/org/elasticsearch/index/translog/BufferingTranslogFile.java +++ b/src/main/java/org/elasticsearch/index/translog/BufferingTranslogWriter.java @@ -30,8 +30,7 @@ import java.nio.ByteBuffer; /** */ -public final class BufferingTranslogFile extends TranslogFile { - +public final class BufferingTranslogWriter extends TranslogWriter { private byte[] buffer; private int bufferCount; private WrapperOutputStream bufferOs = new WrapperOutputStream(); @@ -39,8 +38,8 @@ public final class BufferingTranslogFile extends TranslogFile { /* the total offset of this file including the bytes written to the file as well as into the buffer */ private volatile long totalOffset; - public BufferingTranslogFile(ShardId shardId, long id, ChannelReference channelReference, int bufferSize) throws IOException { - super(shardId, id, channelReference); + public BufferingTranslogWriter(ShardId shardId, long generation, ChannelReference channelReference, int bufferSize) throws IOException { + super(shardId, generation, channelReference); this.buffer = new byte[bufferSize]; this.totalOffset = writtenOffset; } @@ -54,17 +53,17 @@ public final class BufferingTranslogFile extends TranslogFile { flush(); // we use the channel to write, since on windows, writing to the RAF might not be reflected // when reading through the channel - data.writeTo(channelReference.channel()); + data.writeTo(channel); writtenOffset += data.length(); totalOffset += data.length(); - return new Translog.Location(id, offset, data.length()); + return new Translog.Location(generation, offset, data.length()); } if (data.length() > buffer.length - bufferCount) { flush(); } data.writeTo(bufferOs); totalOffset += data.length(); - return new Translog.Location(id, offset, data.length()); + return new Translog.Location(generation, offset, data.length()); } } @@ -73,7 +72,7 @@ public final class BufferingTranslogFile extends TranslogFile { if (bufferCount > 0) { // we use the channel to write, since on windows, writing to the RAF might not be reflected // when reading through the channel - Channels.writeToChannel(buffer, 0, bufferCount, channelReference.channel()); + Channels.writeToChannel(buffer, 0, bufferCount, channel); writtenOffset += bufferCount; bufferCount = 0; } @@ -83,14 +82,17 @@ public final class BufferingTranslogFile extends TranslogFile { protected void readBytes(ByteBuffer targetBuffer, long position) throws IOException { try (ReleasableLock lock = readLock.acquire()) { if (position >= writtenOffset) { - System.arraycopy(buffer, (int) (position - writtenOffset), + assert targetBuffer.hasArray() : "buffer must have array"; + final int sourcePosition = (int) (position - writtenOffset); + System.arraycopy(buffer, sourcePosition, targetBuffer.array(), targetBuffer.position(), targetBuffer.limit()); + targetBuffer.position(targetBuffer.limit()); return; } } // we don't have to have a read lock here because we only write ahead to the file, so all writes has been complete // for the requested location. - Channels.readFromFileChannelWithEofException(channelReference.channel(), position, targetBuffer); + Channels.readFromFileChannelWithEofException(channel, position, targetBuffer); } @Override @@ -103,35 +105,24 @@ public final class BufferingTranslogFile extends TranslogFile { if (!syncNeeded()) { return; } - try (ReleasableLock lock = writeLock.acquire()) { - flush(); - lastSyncedOffset = totalOffset; + synchronized (this) { + try (ReleasableLock lock = writeLock.acquire()) { + flush(); + lastSyncedOffset = totalOffset; + } + // we can do this outside of the write lock but we have to protect from + // concurrent syncs + checkpoint(lastSyncedOffset, operationCounter, channelReference); } - channelReference.channel().force(false); } - @Override - public void reuse(TranslogFile other) { - if (!(other instanceof BufferingTranslogFile)) { - return; - } - try (ReleasableLock lock = writeLock.acquire()) { - try { - flush(); - this.buffer = ((BufferingTranslogFile) other).buffer; - } catch (IOException e) { - throw new TranslogException(shardId, "failed to flush", e); - } - } - } public void updateBufferSize(int bufferSize) { try (ReleasableLock lock = writeLock.acquire()) { - if (this.buffer.length == bufferSize) { - return; + if (this.buffer.length != bufferSize) { + flush(); + this.buffer = new byte[bufferSize]; } - flush(); - this.buffer = new byte[bufferSize]; } catch (IOException e) { throw new TranslogException(shardId, "failed to flush", e); } @@ -151,4 +142,9 @@ public final class BufferingTranslogFile extends TranslogFile { bufferCount += len; } } + + @Override + public long sizeInBytes() { + return totalOffset; + } } diff --git a/src/main/java/org/elasticsearch/index/translog/ChannelReader.java b/src/main/java/org/elasticsearch/index/translog/ChannelReader.java deleted file mode 100644 index c4769d8698e..00000000000 --- a/src/main/java/org/elasticsearch/index/translog/ChannelReader.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.index.translog; - -import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.common.bytes.BytesArray; - -import java.io.Closeable; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.channels.FileChannel; -import java.util.concurrent.atomic.AtomicBoolean; - -/** - * A base class for all classes that allows reading ops from translog files - */ -public abstract class ChannelReader implements Closeable, Comparable { - - public static final int UNKNOWN_OP_COUNT = -1; - - protected final long id; - protected final ChannelReference channelReference; - protected final FileChannel channel; - protected final AtomicBoolean closed = new AtomicBoolean(false); - - public ChannelReader(long id, ChannelReference channelReference) { - this.id = id; - this.channelReference = channelReference; - this.channel = channelReference.channel(); - } - - public long translogId() { - return this.id; - } - - abstract public long sizeInBytes(); - - /** the position the first operation is written at */ - public long firstPosition() { - return channelReference.stream().headerLength(); - } - - abstract public int totalOperations(); - - public Translog.Operation read(Translog.Location location) throws IOException { - assert location.translogId == id : "read location's translog id [" + location.translogId + "] is not [" + id + "]"; - ByteBuffer buffer = ByteBuffer.allocate(location.size); - return read(buffer, location.translogLocation, location.size); - } - - /** read the size of the op (i.e., number of bytes, including the op size) written at the given position */ - public int readSize(ByteBuffer reusableBuffer, long position) { - // read op size from disk - assert reusableBuffer.capacity() >= 4 : "reusable buffer must have capacity >=4 when reading opSize. got [" + reusableBuffer.capacity() + "]"; - try { - reusableBuffer.clear(); - reusableBuffer.limit(4); - readBytes(reusableBuffer, position); - reusableBuffer.flip(); - // Add an extra 4 to account for the operation size integer itself - return reusableBuffer.getInt() + 4; - } catch (IOException e) { - throw new ElasticsearchException("unexpected exception reading from translog snapshot of " + this.channelReference.file(), e); - } - } - - /** - * reads an operation at the given position and returns it. The buffer length is equal to the number - * of bytes reads. - */ - public Translog.Operation read(ByteBuffer reusableBuffer, long position, int opSize) throws IOException { - final ByteBuffer buffer; - if (reusableBuffer.capacity() >= opSize) { - buffer = reusableBuffer; - } else { - buffer = ByteBuffer.allocate(opSize); - } - buffer.clear(); - buffer.limit(opSize); - readBytes(buffer, position); - BytesArray bytesArray = new BytesArray(buffer.array(), 0, buffer.limit()); - return channelReference.stream().read(bytesArray.streamInput()); - } - - /** - * reads bytes at position into the given buffer, filling it. - */ - abstract protected void readBytes(ByteBuffer buffer, long position) throws IOException; - - /** create snapshot for this channel */ - abstract public ChannelSnapshot newSnapshot(); - - @Override - public void close() throws IOException { - if (closed.compareAndSet(false, true)) { - doClose(); - } - } - - protected void doClose() throws IOException { - channelReference.decRef(); - } - - @Override - public String toString() { - return "translog [" + id + "][" + channelReference.file() + "]"; - } - - @Override - public int compareTo(ChannelReader o) { - return Long.compare(translogId(), o.translogId()); - } -} diff --git a/src/main/java/org/elasticsearch/index/translog/ChannelReference.java b/src/main/java/org/elasticsearch/index/translog/ChannelReference.java index 580e4d391c5..30424222c60 100644 --- a/src/main/java/org/elasticsearch/index/translog/ChannelReference.java +++ b/src/main/java/org/elasticsearch/index/translog/ChannelReference.java @@ -19,54 +19,40 @@ package org.elasticsearch.index.translog; -import com.google.common.collect.Iterables; import org.apache.lucene.util.IOUtils; -import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.common.util.Callback; import org.elasticsearch.common.util.concurrent.AbstractRefCounted; -import org.elasticsearch.common.util.concurrent.ConcurrentCollections; -import org.elasticsearch.index.translog.TranslogStream; -import org.elasticsearch.index.translog.TranslogStreams; import java.io.IOException; import java.nio.channels.FileChannel; -import java.nio.file.OpenOption; -import java.nio.file.Path; -import java.util.Collections; -import java.util.IdentityHashMap; -import java.util.Map; - -class ChannelReference extends AbstractRefCounted { +import java.nio.file.*; +final class ChannelReference extends AbstractRefCounted { private final Path file; - private final FileChannel channel; + protected final long generation; + private final Callback onClose; - private final TranslogStream stream; - - public ChannelReference(Path file, OpenOption... openOptions) throws IOException { + ChannelReference(Path file, long generation, FileChannel channel, Callback onClose) throws IOException { super(file.toString()); + this.generation = generation; this.file = file; - this.channel = FileChannel.open(file, openOptions); - try { - this.stream = TranslogStreams.translogStreamFor(file); - } catch (Throwable t) { - IOUtils.closeWhileHandlingException(channel); - throw t; - } + this.channel = channel; + this.onClose = onClose; } - public Path file() { + public long getGeneration() { + return generation; + } + + public Path getPath() { return this.file; } - public FileChannel channel() { + public FileChannel getChannel() { return this.channel; } - public TranslogStream stream() { - return this.stream; - } - @Override public String toString() { return "channel: file [" + file + "], ref count [" + refCount() + "]"; @@ -74,6 +60,12 @@ class ChannelReference extends AbstractRefCounted { @Override protected void closeInternal() { - IOUtils.closeWhileHandlingException(channel); + try { + IOUtils.closeWhileHandlingException(channel); + } finally { + if (onClose != null) { + onClose.handle(this); + } + } } } diff --git a/src/main/java/org/elasticsearch/index/translog/ChannelSnapshot.java b/src/main/java/org/elasticsearch/index/translog/ChannelSnapshot.java deleted file mode 100644 index 6febe7217f5..00000000000 --- a/src/main/java/org/elasticsearch/index/translog/ChannelSnapshot.java +++ /dev/null @@ -1,74 +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.translog; - -import org.apache.lucene.util.IOUtils; -import org.elasticsearch.ElasticsearchException; - -import java.io.Closeable; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.concurrent.atomic.AtomicBoolean; - -/** - * an implementation of {@link org.elasticsearch.index.translog.Translog.Snapshot}, wrapping - * a {@link ChannelReader}. This class is NOT thread-safe. - */ -public class ChannelSnapshot implements Closeable { - - protected final ChannelReader reader; - protected final AtomicBoolean closed = new AtomicBoolean(false); - - // we use an atomic long to allow passing it by reference :( - protected long position; - - public ChannelSnapshot(ChannelReader reader) { - this.reader = reader; - this.position = reader.firstPosition(); - } - - public long translogId() { - return reader.translogId(); - } - - public int estimatedTotalOperations() { - return reader.totalOperations(); - } - - public Translog.Operation next(ByteBuffer reusableBuffer) throws IOException { - if (position >= reader.sizeInBytes()) { - return null; - } - final int opSize = reader.readSize(reusableBuffer, position); - Translog.Operation op = reader.read(reusableBuffer, position, opSize); - position += opSize; - return op; - } - - @Override - public void close() { - if (closed.compareAndSet(false, true)) { - try { - IOUtils.close(reader); - } catch (IOException e) { - throw new ElasticsearchException("failed to close translogs", e); - } - } - } -} diff --git a/src/main/java/org/elasticsearch/index/translog/Checkpoint.java b/src/main/java/org/elasticsearch/index/translog/Checkpoint.java new file mode 100644 index 00000000000..586b1f7d8f7 --- /dev/null +++ b/src/main/java/org/elasticsearch/index/translog/Checkpoint.java @@ -0,0 +1,93 @@ +/* + * 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.translog; + +import org.apache.lucene.store.ByteArrayDataOutput; +import org.apache.lucene.store.DataInput; +import org.apache.lucene.store.DataOutput; +import org.apache.lucene.store.InputStreamDataInput; +import org.apache.lucene.util.RamUsageEstimator; +import org.elasticsearch.common.io.Channels; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.channels.FileChannel; +import java.nio.file.Files; +import java.nio.file.OpenOption; +import java.nio.file.Path; + +/** + */ +class Checkpoint { + + static final int BUFFER_SIZE = RamUsageEstimator.NUM_BYTES_INT // ops + + RamUsageEstimator.NUM_BYTES_LONG // offset + + RamUsageEstimator.NUM_BYTES_LONG;// generation + final long offset; + final int numOps; + final long generation; + + Checkpoint(long offset, int numOps, long generation) { + this.offset = offset; + this.numOps = numOps; + this.generation = generation; + } + + Checkpoint(DataInput in) throws IOException { + offset = in.readLong(); + numOps = in.readInt(); + generation = in.readLong(); + } + + void write(FileChannel channel) throws IOException { + byte[] buffer = new byte[BUFFER_SIZE]; + final ByteArrayDataOutput out = new ByteArrayDataOutput(buffer); + write(out); + Channels.writeToChannel(buffer, channel); + } + + public void write(DataOutput out) throws IOException { + out.writeLong(offset); + out.writeInt(numOps); + out.writeLong(generation); + } + + @Override + public String toString() { + return "TranslogInfo{" + + "offset=" + offset + + ", numOps=" + numOps + + ", translogFileGeneration= " + generation + + '}'; + } + + public static Checkpoint read(Path path) throws IOException { + try (InputStream in = Files.newInputStream(path)) { + return new Checkpoint(new InputStreamDataInput(in)); + } + } + + public static void write(Path checkpointFile, Checkpoint checkpoint, OpenOption... options) throws IOException { + try (FileChannel channel = FileChannel.open(checkpointFile, options)) { + checkpoint.write(channel); + channel.force(false); + } + } + +} diff --git a/src/main/java/org/elasticsearch/index/translog/ChecksummedTranslogStream.java b/src/main/java/org/elasticsearch/index/translog/ChecksummedTranslogStream.java deleted file mode 100644 index e62436522c2..00000000000 --- a/src/main/java/org/elasticsearch/index/translog/ChecksummedTranslogStream.java +++ /dev/null @@ -1,138 +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.translog; - -import org.apache.lucene.codecs.CodecUtil; -import org.apache.lucene.store.InputStreamDataInput; -import org.apache.lucene.store.OutputStreamDataOutput; -import org.apache.lucene.util.IOUtils; -import org.elasticsearch.common.io.stream.InputStreamStreamInput; -import org.elasticsearch.common.io.stream.NoopStreamOutput; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; - -import java.io.EOFException; -import java.io.IOException; -import java.io.InputStream; -import java.nio.channels.Channels; -import java.nio.channels.FileChannel; -import java.nio.file.Files; -import java.nio.file.Path; - -/** - * Version 1 of the translog file format. Writes a header to identify the - * format, also writes checksums for each operation - */ -public class ChecksummedTranslogStream implements TranslogStream { - - public static final int VERSION = 1; - - ChecksummedTranslogStream() { - } - - private void verifyChecksum(BufferedChecksumStreamInput in) throws IOException { - // This absolutely must come first, or else reading the checksum becomes part of the checksum - long expectedChecksum = in.getChecksum(); - long readChecksum = in.readInt() & 0xFFFF_FFFFL; - if (readChecksum != expectedChecksum) { - throw new TranslogCorruptedException("translog stream is corrupted, expected: 0x" + - Long.toHexString(expectedChecksum) + ", got: 0x" + Long.toHexString(readChecksum)); - } - } - - @Override - public Translog.Operation read(StreamInput inStream) throws IOException { - // TODO: validate size to prevent OOME - int opSize = inStream.readInt(); - // This BufferedChecksumStreamInput remains unclosed on purpose, - // because closing it closes the underlying stream, which we don't - // want to do here. - BufferedChecksumStreamInput in = new BufferedChecksumStreamInput(inStream); - Translog.Operation operation; - try { - Translog.Operation.Type type = Translog.Operation.Type.fromId(in.readByte()); - operation = TranslogStreams.newOperationFromType(type); - operation.readFrom(in); - verifyChecksum(in); - } catch (EOFException e) { - throw new TruncatedTranslogException("reached premature end of file, translog is truncated", e); - } catch (AssertionError|Exception e) { - throw new TranslogCorruptedException("translog corruption while reading from stream", e); - } - return operation; - } - - @Override - public void write(StreamOutput outStream, Translog.Operation op) throws IOException { - // We first write to a NoopStreamOutput to get the size of the - // operation. We could write to a byte array and then send that as an - // alternative, but here we choose to use CPU over allocating new - // byte arrays. - NoopStreamOutput noopOut = new NoopStreamOutput(); - noopOut.writeByte(op.opType().id()); - op.writeTo(noopOut); - noopOut.writeInt(0); // checksum holder - int size = noopOut.getCount(); - - // This BufferedChecksumStreamOutput remains unclosed on purpose, - // because closing it closes the underlying stream, which we don't - // want to do here. - BufferedChecksumStreamOutput out = new BufferedChecksumStreamOutput(outStream); - outStream.writeInt(size); // opSize is not checksummed - out.writeByte(op.opType().id()); - op.writeTo(out); - long checksum = out.getChecksum(); - out.writeInt((int)checksum); - } - - @Override - public int writeHeader(FileChannel channel) throws IOException { - // This OutputStreamDataOutput is intentionally not closed because - // closing it will close the FileChannel - OutputStreamDataOutput out = new OutputStreamDataOutput(Channels.newOutputStream(channel)); - CodecUtil.writeHeader(out, TranslogStreams.TRANSLOG_CODEC, VERSION); - return headerLength(); - } - - @Override - public int headerLength() { - return CodecUtil.headerLength(TranslogStreams.TRANSLOG_CODEC); - } - - @Override - public StreamInput openInput(Path translogFile) throws IOException { - final InputStream fileInputStream = Files.newInputStream(translogFile); - boolean success = false; - try { - final InputStreamStreamInput in = new InputStreamStreamInput(fileInputStream); - CodecUtil.checkHeader(new InputStreamDataInput(in), TranslogStreams.TRANSLOG_CODEC, VERSION, VERSION); - success = true; - return in; - } catch (EOFException e) { - throw new TruncatedTranslogException("translog header truncated", e); - } catch (IOException e) { - throw new TranslogCorruptedException("translog header corrupted", e); - } finally { - if (success == false) { - IOUtils.closeWhileHandlingException(fileInputStream); - } - } - } -} diff --git a/src/main/java/org/elasticsearch/index/translog/ChannelImmutableReader.java b/src/main/java/org/elasticsearch/index/translog/ImmutableTranslogReader.java similarity index 68% rename from src/main/java/org/elasticsearch/index/translog/ChannelImmutableReader.java rename to src/main/java/org/elasticsearch/index/translog/ImmutableTranslogReader.java index 61ee844a1bc..6d141108b5c 100644 --- a/src/main/java/org/elasticsearch/index/translog/ChannelImmutableReader.java +++ b/src/main/java/org/elasticsearch/index/translog/ImmutableTranslogReader.java @@ -26,38 +26,43 @@ import java.io.IOException; import java.nio.ByteBuffer; /** - * a channel reader which is fixed in length + * a translog reader which is fixed in length */ -public final class ChannelImmutableReader extends ChannelReader { +public class ImmutableTranslogReader extends TranslogReader { private final int totalOperations; - private final long length; + protected final long length; /** * Create a snapshot of translog file channel. The length parameter should be consistent with totalOperations and point * at the end of the last operation in this snapshot. */ - public ChannelImmutableReader(long id, ChannelReference channelReference, long length, int totalOperations) { - super(id, channelReference); + public ImmutableTranslogReader(long generation, ChannelReference channelReference, long offset, long length, int totalOperations) { + super(generation, channelReference, offset); this.length = length; this.totalOperations = totalOperations; } - - public ChannelImmutableReader clone() { + @Override + public final TranslogReader clone() { if (channelReference.tryIncRef()) { try { - ChannelImmutableReader reader = new ChannelImmutableReader(id, channelReference, length, totalOperations); + ImmutableTranslogReader reader = newReader(generation, channelReference, firstOperationOffset, length, totalOperations); channelReference.incRef(); // for the new object return reader; } finally { channelReference.decRef(); } } else { - throw new IllegalStateException("can't increment translog [" + id + "] channel ref count"); + throw new IllegalStateException("can't increment translog [" + generation + "] channel ref count"); } } + + protected ImmutableTranslogReader newReader(long generation, ChannelReference channelReference, long offset, long length, int totalOperations) { + return new ImmutableTranslogReader(generation, channelReference, offset, length, totalOperations); + } + public long sizeInBytes() { return length; } @@ -73,14 +78,14 @@ public final class ChannelImmutableReader extends ChannelReader { if (position >= length) { throw new EOFException("read requested past EOF. pos [" + position + "] end: [" + length + "]"); } - if (position < firstPosition()) { - throw new IOException("read requested before position of first ops. pos [" + position + "] first op on: [" + firstPosition() + "]"); + if (position < firstOperationOffset) { + throw new IOException("read requested before position of first ops. pos [" + position + "] first op on: [" + firstOperationOffset + "]"); } Channels.readFromFileChannelWithEofException(channel, position, buffer); } - @Override - public ChannelSnapshot newSnapshot() { - return new ChannelSnapshot(clone()); + public Checkpoint getInfo() { + return new Checkpoint(length, totalOperations, getGeneration()); } + } diff --git a/src/main/java/org/elasticsearch/index/translog/LegacyTranslogStream.java b/src/main/java/org/elasticsearch/index/translog/LegacyTranslogReader.java similarity index 55% rename from src/main/java/org/elasticsearch/index/translog/LegacyTranslogStream.java rename to src/main/java/org/elasticsearch/index/translog/LegacyTranslogReader.java index 4611524861e..7a131f9d0d8 100644 --- a/src/main/java/org/elasticsearch/index/translog/LegacyTranslogStream.java +++ b/src/main/java/org/elasticsearch/index/translog/LegacyTranslogReader.java @@ -19,55 +19,45 @@ package org.elasticsearch.index.translog; -import org.elasticsearch.common.io.stream.InputStreamStreamInput; import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; import java.io.IOException; -import java.nio.channels.FileChannel; -import java.nio.file.Files; -import java.nio.file.Path; /** * Version 0 of the translog format, there is no header in this file */ -public class LegacyTranslogStream implements TranslogStream { +@Deprecated +public final class LegacyTranslogReader extends LegacyTranslogReaderBase { - LegacyTranslogStream() { + /** + * Create a snapshot of translog file channel. The length parameter should be consistent with totalOperations and point + * at the end of the last operation in this snapshot. + * + * @param generation + * @param channelReference + */ + LegacyTranslogReader(long generation, ChannelReference channelReference, long fileLength) { + super(generation, channelReference, 0, fileLength); } @Override - public Translog.Operation read(StreamInput in) throws IOException { + protected Translog.Operation read(BufferedChecksumStreamInput in) throws IOException { // read the opsize before an operation. // Note that this was written & read out side of the stream when this class was used, but it makes things more consistent // to read this here in.readInt(); Translog.Operation.Type type = Translog.Operation.Type.fromId(in.readByte()); - Translog.Operation operation = TranslogStreams.newOperationFromType(type); + Translog.Operation operation = Translog.newOperationFromType(type); operation.readFrom(in); return operation; } - @Override - public void write(StreamOutput out, Translog.Operation op) throws IOException { - throw new UnsupportedOperationException("LegacyTranslogStream is depracated. Use TranslogStreams.LATEST"); - } + @Override - public int writeHeader(FileChannel channel) { - // nothing, there is no header for version 0 translog files - return 0; + protected ImmutableTranslogReader newReader(long generation, ChannelReference channelReference, long firstOperationOffset, long length, int totalOperations) { + assert totalOperations == -1 : "expected unknown but was: " + totalOperations; + assert firstOperationOffset == 0; + return new LegacyTranslogReader(generation, channelReference, length); } - - @Override - public int headerLength() { - return 0; - } - - @Override - public StreamInput openInput(Path translogFile) throws IOException { - // nothing to do, legacy translogs have no header - return new InputStreamStreamInput(Files.newInputStream(translogFile)); - } - } diff --git a/src/main/java/org/elasticsearch/index/translog/LegacyTranslogReaderBase.java b/src/main/java/org/elasticsearch/index/translog/LegacyTranslogReaderBase.java new file mode 100644 index 00000000000..eff0d0675db --- /dev/null +++ b/src/main/java/org/elasticsearch/index/translog/LegacyTranslogReaderBase.java @@ -0,0 +1,60 @@ +/* + * 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.translog; + +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * Version 1 of the translog format, there is checkpoint and therefore no notion of op count + */ +@Deprecated +class LegacyTranslogReaderBase extends ImmutableTranslogReader { + + /** + * Create a snapshot of translog file channel. The length parameter should be consistent with totalOperations and point + * at the end of the last operation in this snapshot. + * + */ + LegacyTranslogReaderBase(long generation, ChannelReference channelReference, long firstOperationOffset, long fileLength) { + super(generation, channelReference, firstOperationOffset, fileLength, TranslogReader.UNKNOWN_OP_COUNT); + } + + + @Override + protected Translog.Snapshot newReaderSnapshot(final int totalOperations, ByteBuffer reusableBuffer) { + assert totalOperations == -1 : "legacy we had no idea how many ops: " + totalOperations; + return new ReaderSnapshot(totalOperations, reusableBuffer) { + @Override + public Translog.Operation next() throws IOException { + if (position >= sizeInBytes()) { // this is the legacy case.... + return null; + } + return readOperation(); + } + }; + } + + @Override + protected ImmutableTranslogReader newReader(long generation, ChannelReference channelReference, long firstOperationOffset, long length, int totalOperations) { + assert totalOperations == -1 : "expected unknown but was: " + totalOperations; + return new LegacyTranslogReaderBase(generation, channelReference, firstOperationOffset, length); + } +} diff --git a/src/main/java/org/elasticsearch/index/translog/Translog.java b/src/main/java/org/elasticsearch/index/translog/Translog.java index bd8fc819383..6116fe9f8a6 100644 --- a/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -19,8 +19,9 @@ package org.elasticsearch.index.translog; -import com.google.common.collect.Iterables; import org.apache.lucene.index.Term; +import org.apache.lucene.index.TwoPhaseCommit; +import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.util.Accountable; import org.apache.lucene.util.CollectionUtil; import org.apache.lucene.util.IOUtils; @@ -31,33 +32,29 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.bytes.ReleasablePagedBytesReference; -import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput; -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.collect.Tuple; +import org.elasticsearch.common.io.stream.*; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.lucene.uid.Versions; -import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.Callback; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.util.concurrent.FutureUtils; import org.elasticsearch.common.util.concurrent.ReleasableLock; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.engine.Engine; -import org.elasticsearch.index.settings.IndexSettings; -import org.elasticsearch.index.settings.IndexSettingsService; import org.elasticsearch.index.shard.AbstractIndexShardComponent; import org.elasticsearch.index.shard.IndexShardComponent; -import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.threadpool.ThreadPool; import java.io.Closeable; +import java.io.EOFException; import java.io.FileNotFoundException; import java.io.IOException; +import java.nio.channels.FileChannel; import java.nio.file.*; import java.util.*; import java.util.concurrent.ScheduledFuture; @@ -68,147 +65,251 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; /** - * + * A Translog is a per index shard component that records all non-committed index operations in a durable manner. + * In Elasticsearch there is one Translog instance per {@link org.elasticsearch.index.engine.InternalEngine}. The engine + * records the current translog generation {@link Translog#getGeneration()} in it's commit metadata using {@link #TRANSLOG_GENERATION_KEY} + * to reference the generation that contains all operations that have not yet successfully been committed to the engines lucene index. + * Additionally, since Elasticsearch 2.0 the engine also records a {@link #TRANSLOG_UUID_KEY} with each commit to ensure a strong association + * between the lucene index an the transaction log file. This UUID is used to prevent accidential recovery from a transaction log that belongs to a + * different engine. + *

+ * Each Translog has only one translog file open at any time referenced by a translog generation ID. This ID is written to a translog.ckp file that is designed + * to fit in a single disk block such that a write of the file is atomic. The checkpoint file is written on each fsync operation of the translog and records the number of operations + * written, the current tranlogs file generation and it's fsynced offset in bytes. + *

+ *

+ * When a translog is opened the checkpoint is use to retrieve the latest translog file generation and subsequently to open the last written file to recovery operations. + * The {@link org.elasticsearch.index.translog.Translog.TranslogGeneration} on {@link TranslogConfig#getTranslogGeneration()} given when the translog is opened is compared against + * the latest generation and all consecutive translog files singe the given generation and the last generation in the checkpoint will be recovered and preserved until the next + * generation is committed using {@link Translog#commit()}. In the common case the translog file generation in the checkpoint and the generation passed to the translog on creation are + * the same. The only situation when they can be different is when an actual translog commit fails in between {@link Translog#prepareCommit()} and {@link Translog#commit()}. In such a case + * the currently being committed translog file will not be deleted since it's commit was not successful. Yet, a new/current translog file is already opened at that point such that there is more than + * one translog file present. Such an uncommitted translog file always has a translog-${gen}.ckp associated with it which is an fsynced copy of the it's last translog.ckp such that in + * disaster recovery last fsynced offsets, number of operation etc. are still preserved. + *

*/ -public class Translog extends AbstractIndexShardComponent implements IndexShardComponent, Closeable { +public class Translog extends AbstractIndexShardComponent implements IndexShardComponent, Closeable, TwoPhaseCommit { - public static ByteSizeValue INACTIVE_SHARD_TRANSLOG_BUFFER = ByteSizeValue.parseBytesSizeValue("1kb"); - public static final String TRANSLOG_ID_KEY = "translog_id"; - public static final String INDEX_TRANSLOG_DURABILITY = "index.translog.durability"; - public static final String INDEX_TRANSLOG_FS_TYPE = "index.translog.fs.type"; - public static final String INDEX_TRANSLOG_BUFFER_SIZE = "index.translog.fs.buffer_size"; - public static final String INDEX_TRANSLOG_SYNC_INTERVAL = "index.translog.sync_interval"; + /* + * TODO + * - we might need something like a deletion policy to hold on to more than one translog eventually (I think sequence IDs needs this) but we can refactor as we go + * - use a simple BufferedOuputStream to write stuff and fold BufferedTranslogWriter into it's super class... the tricky bit is we need to be able to do random access reads even from the buffer + * - we need random exception on the FileSystem API tests for all this. + * - we need to page align the last write before we sync, we can take advantage of ensureSynced for this since we might have already fsynced far enough + */ + public static final String TRANSLOG_GENERATION_KEY = "translog_generation"; + public static final String TRANSLOG_UUID_KEY = "translog_uuid"; public static final String TRANSLOG_FILE_PREFIX = "translog-"; - static final Pattern PARSE_ID_PATTERN = Pattern.compile(TRANSLOG_FILE_PREFIX + "(\\d+)(\\.recovering)?$"); - private final TimeValue syncInterval; + public static final String TRANSLOG_FILE_SUFFIX = ".tlog"; + public static final String CHECKPOINT_SUFFIX = ".ckp"; + public static final String CHECKPOINT_FILE_NAME = "translog" + CHECKPOINT_SUFFIX; + + static final Pattern PARSE_ID_PATTERN = Pattern.compile("^" + TRANSLOG_FILE_PREFIX + "(\\d+)((\\.recovering)|(\\.tlog))?$"); + + private final List recoveredTranslogs; private volatile ScheduledFuture syncScheduler; - private volatile Durabilty durabilty = Durabilty.REQUEST; - - // this is a concurrent set and is not protected by any of the locks. The main reason // is that is being accessed by two separate classes (additions & reading are done by FsTranslog, remove by FsView when closed) private final Set outstandingViews = ConcurrentCollections.newConcurrentSet(); - - - class ApplySettings implements IndexSettingsService.Listener { - @Override - public void onRefreshSettings(Settings settings) { - TranslogFile.Type type = TranslogFile.Type.fromString(settings.get(INDEX_TRANSLOG_FS_TYPE, Translog.this.type.name())); - if (type != Translog.this.type) { - logger.info("updating type from [{}] to [{}]", Translog.this.type, type); - Translog.this.type = type; - } - - final Durabilty durabilty = Durabilty.getFromSettings(logger, settings, Translog.this.durabilty); - if (durabilty != Translog.this.durabilty) { - logger.info("updating durability from [{}] to [{}]", Translog.this.durabilty, durabilty); - Translog.this.durabilty = durabilty; - } - } - } - - private final IndexSettingsService indexSettingsService; - private final BigArrays bigArrays; - private final ThreadPool threadPool; - + private BigArrays bigArrays; protected final ReleasableLock readLock; protected final ReleasableLock writeLock; - private final Path location; - - // protected by the write lock - private long idGenerator = 1; - private TranslogFile current; - // ordered by age - private final List uncommittedTranslogs = new ArrayList<>(); - private long lastCommittedTranslogId = -1; // -1 is safe as it will not cause an translog deletion. - - private TranslogFile.Type type; - - private boolean syncOnEachOperation = false; - - private volatile int bufferSize; - - private final ApplySettings applySettings = new ApplySettings(); - + private TranslogWriter current; + private volatile ImmutableTranslogReader currentCommittingTranslog; + private long lastCommittedTranslogFileGeneration = -1; // -1 is safe as it will not cause an translog deletion. private final AtomicBoolean closed = new AtomicBoolean(); + private final TranslogConfig config; + private final String translogUUID; - public Translog(ShardId shardId, IndexSettingsService indexSettingsService, - BigArrays bigArrays, Path location, ThreadPool threadPool) throws IOException { - this(shardId, indexSettingsService.getSettings(), indexSettingsService, bigArrays, location, threadPool); - } - public Translog(ShardId shardId, @IndexSettings Settings indexSettings, - BigArrays bigArrays, Path location) throws IOException { - this(shardId, indexSettings, null, bigArrays, location, null); - } + /** + * Creates a new Translog instance. This method will create a new transaction log unless the given {@link TranslogConfig} has + * a non-null {@link org.elasticsearch.index.translog.Translog.TranslogGeneration}. If the generation is null this method + * us destructive and will delete all files in the translog path given. + * @see TranslogConfig#getTranslogPath() + */ + public Translog(TranslogConfig config) throws IOException { + super(config.getShardId(), config.getIndexSettings()); + this.config = config; + TranslogGeneration translogGeneration = config.getTranslogGeneration(); - private Translog(ShardId shardId, @IndexSettings Settings indexSettings, @Nullable IndexSettingsService indexSettingsService, - BigArrays bigArrays, Path location, @Nullable ThreadPool threadPool) throws IOException { - super(shardId, indexSettings); + if (translogGeneration == null || translogGeneration.translogUUID == null) { // legacy case + translogUUID = Strings.randomBase64UUID(); + } else { + translogUUID = translogGeneration.translogUUID; + } + bigArrays = config.getBigArrays(); ReadWriteLock rwl = new ReentrantReadWriteLock(); readLock = new ReleasableLock(rwl.readLock()); writeLock = new ReleasableLock(rwl.writeLock()); - this.durabilty = Durabilty.getFromSettings(logger, indexSettings, durabilty); - this.indexSettingsService = indexSettingsService; - this.bigArrays = bigArrays; - this.location = location; + this.location = config.getTranslogPath(); Files.createDirectories(this.location); - this.threadPool = threadPool; - - this.type = TranslogFile.Type.fromString(indexSettings.get(INDEX_TRANSLOG_FS_TYPE, TranslogFile.Type.BUFFERED.name())); - this.bufferSize = (int) indexSettings.getAsBytesSize(INDEX_TRANSLOG_BUFFER_SIZE, ByteSizeValue.parseBytesSizeValue("64k")).bytes(); // Not really interesting, updated by IndexingMemoryController... - - syncInterval = indexSettings.getAsTime(INDEX_TRANSLOG_SYNC_INTERVAL, TimeValue.timeValueSeconds(5)); - if (syncInterval.millis() > 0 && threadPool != null) { - this.syncOnEachOperation = false; - syncScheduler = threadPool.schedule(syncInterval, ThreadPool.Names.SAME, new Sync()); - } else if (syncInterval.millis() == 0) { - this.syncOnEachOperation = true; + if (config.getSyncInterval().millis() > 0 && config.getThreadPool() != null) { + syncScheduler = config.getThreadPool().schedule(config.getSyncInterval(), ThreadPool.Names.SAME, new Sync()); } - if (indexSettingsService != null) { - indexSettingsService.addListener(applySettings); - } try { - recoverFromFiles(); + if (translogGeneration != null) { + final Checkpoint checkpoint = Checkpoint.read(location.resolve(CHECKPOINT_FILE_NAME)); + this.recoveredTranslogs = recoverFromFiles(translogGeneration, checkpoint); + if (recoveredTranslogs.isEmpty()) { + throw new IllegalStateException("at least one reader must be recovered"); + } + current = createWriter(checkpoint.generation + 1); + this.lastCommittedTranslogFileGeneration = translogGeneration.translogFileGeneration; + } else { + this.recoveredTranslogs = Collections.EMPTY_LIST; + IOUtils.rm(location); + logger.debug("wipe translog location - creating new translog"); + Files.createDirectories(location); + final long generation = 1; + Checkpoint checkpoint = new Checkpoint(0, 0, generation); + Checkpoint.write(location.resolve(CHECKPOINT_FILE_NAME), checkpoint, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW); + current = createWriter(generation); + this.lastCommittedTranslogFileGeneration = -1; // playing safe + + } // now that we know which files are there, create a new current one. - current = createTranslogFile(null); } catch (Throwable t) { // close the opened translog files if we fail to create a new translog... - IOUtils.closeWhileHandlingException(uncommittedTranslogs); + IOUtils.closeWhileHandlingException(currentCommittingTranslog, current); throw t; } } - /** recover all translog files found on disk */ - private void recoverFromFiles() throws IOException { - boolean success = false; - ArrayList foundTranslogs = new ArrayList<>(); - try (ReleasableLock lock = writeLock.acquire()) { - try (DirectoryStream stream = Files.newDirectoryStream(location, TRANSLOG_FILE_PREFIX + "[0-9]*")) { - for (Path file : stream) { - final long id = parseIdFromFileName(file); - if (id < 0) { - throw new TranslogException(shardId, "failed to parse id from file name matching pattern " + file); + /** + * This method is used to upgarde a pre 2.0 translog structure to the new checkpoint based structure. + * The {@link org.elasticsearch.index.translog.Translog.TranslogGeneration} in the given config is + * used to determine the smallest file generation to upgrade. The procedure will travers the translog + * directory to find all files that have a generation greater or equal to the translog generation and + * renames the files to the new .tlog file format. + *

+ * For each of the files a ${filename}.ckp + * file is written containing the size of the translog in bytes, it's ID and the number of operations. Since + * these files are all relying on the pre 2.0 truncation feature where we read operations until hitting an {@link EOFException} + * the number of operations are recoreded as -1. Later once these files are opened for reading legacy readers will + * allow for unknown number of operations and mimic the old behavior. + *

+ */ + public static void upgradeLegacyTranslog(ESLogger logger, TranslogConfig config) throws IOException { + Path translogPath = config.getTranslogPath(); + TranslogGeneration translogGeneration = config.getTranslogGeneration(); + if (translogGeneration == null) { + throw new IllegalArgumentException("TranslogGeneration must be set in order to upgrade"); + } + if (translogGeneration.translogUUID != null) { + throw new IllegalArgumentException("TranslogGeneration has a non-null UUID - index must have already been upgraded"); + } + assert translogGeneration.translogUUID == null : "Already upgrade"; + try { + assert Checkpoint.read(translogPath.resolve(CHECKPOINT_FILE_NAME)) == null; + } catch (NoSuchFileException | FileNotFoundException ex) { + logger.debug("upgrading translog - no checkpoint found"); + } + + try (DirectoryStream stream = Files.newDirectoryStream(translogPath, new DirectoryStream.Filter() { + @Override + public boolean accept(Path entry) throws IOException { + Matcher matcher = PARSE_ID_PATTERN.matcher(entry.getFileName().toString()); + return matcher.matches(); + } + })) { + long latestGeneration = -1; + List> filesToUpgrade = new ArrayList<>(); + for (Path path : stream) { + Matcher matcher = PARSE_ID_PATTERN.matcher(path.getFileName().toString()); + if (matcher.matches()) { + long generation = Long.parseLong(matcher.group(1)); + if (generation >= translogGeneration.translogFileGeneration) { + latestGeneration = Math.max(translogGeneration.translogFileGeneration, generation); } - idGenerator = Math.max(idGenerator, id + 1); - final ChannelReference raf = new InternalChannelReference(id, location.resolve(getFilename(id)), StandardOpenOption.READ); - foundTranslogs.add(new ChannelImmutableReader(id, raf, raf.channel().size(), ChannelReader.UNKNOWN_OP_COUNT)); - logger.debug("found local translog with id [{}]", id); + filesToUpgrade.add(new Tuple<>(path, generation)); } } - CollectionUtil.timSort(foundTranslogs); - uncommittedTranslogs.addAll(foundTranslogs); + if (latestGeneration < translogGeneration.translogFileGeneration) { + throw new IllegalStateException("latest found translog has a lower generation that the excepcted uncommitted " + translogGeneration.translogFileGeneration + " > " + latestGeneration); + } + CollectionUtil.timSort(filesToUpgrade, new Comparator>() { + @Override + public int compare(Tuple o1, Tuple o2) { + long gen1 = o1.v2(); + long gen2 = o2.v2(); + return Long.compare(gen1, gen2); + } + }); + for (Tuple pathAndGeneration : filesToUpgrade) { + final Path path = pathAndGeneration.v1(); + final long generation = pathAndGeneration.v2(); + final Path target = path.resolveSibling(getFilename(generation)); + logger.debug("upgrading translog copy file from {} to {}", path, target); + Files.move(path, target, StandardCopyOption.ATOMIC_MOVE); + logger.debug("write commit point for {}", target); + if (generation == latestGeneration) { + // for the last one we only write a checkpoint not a real commit + Checkpoint checkpoint = new Checkpoint(Files.size(translogPath.resolve(getFilename(latestGeneration))), -1, latestGeneration); + Checkpoint.write(translogPath.resolve(CHECKPOINT_FILE_NAME), checkpoint, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW); + } else { + Checkpoint checkpoint = new Checkpoint(Files.size(target), -1, generation); + Checkpoint.write(translogPath.resolve(getCommitFileName(generation)), checkpoint, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW); + } + } + + + + IOUtils.fsync(translogPath, true); + + } + } + + /** recover all translog files found on disk */ + private ArrayList recoverFromFiles(TranslogGeneration translogGeneration, Checkpoint checkpoint) throws IOException { + boolean success = false; + ArrayList foundTranslogs = new ArrayList<>(); + try (ReleasableLock lock = writeLock.acquire()) { + + logger.debug("open uncommitted translog checkpoint {}", checkpoint); + final String checkpointTranslogFile = getFilename(checkpoint.generation); + for (long i = translogGeneration.translogFileGeneration; i < checkpoint.generation; i++) { + Path committedTranslogFile = location.resolve(getFilename(i)); + if (Files.exists(committedTranslogFile) == false) { + throw new IllegalStateException("translog file doesn't exist with generation: " + i + " lastCommitted: " + lastCommittedTranslogFileGeneration + " checkpoint: " + checkpoint.generation + " - translog ids must be consecutive"); + } + final ImmutableTranslogReader reader = openReader(committedTranslogFile, Checkpoint.read(location.resolve(getCommitFileName(i)))); + foundTranslogs.add(reader); + logger.debug("recovered local translog from checkpoint {}", checkpoint); + } + foundTranslogs.add(openReader(location.resolve(checkpointTranslogFile), checkpoint)); + Path commitCheckpoint = location.resolve(getCommitFileName(checkpoint.generation)); + Files.copy(location.resolve(CHECKPOINT_FILE_NAME), commitCheckpoint); + IOUtils.fsync(commitCheckpoint, false); + IOUtils.fsync(commitCheckpoint.getParent(), true); success = true; } finally { if (success == false) { IOUtils.closeWhileHandlingException(foundTranslogs); } } + return foundTranslogs; } - /* extracts the translog id from a file name. returns -1 upon failure */ + ImmutableTranslogReader openReader(Path path, Checkpoint checkpoint) throws IOException { + final long generation = parseIdFromFileName(path); + if (generation < 0) { + throw new TranslogException(shardId, "failed to parse generation from file name matching pattern " + path); + } + FileChannel channel = FileChannel.open(path, StandardOpenOption.READ); + try { + final ChannelReference raf = new ChannelReference(path, generation, channel, new OnCloseRunnable()); + ImmutableTranslogReader reader = ImmutableTranslogReader.open(raf, checkpoint, translogUUID); + channel = null; + return reader; + } finally { + IOUtils.close(channel); + } + } + + /* extracts the translog generation from a file name. returns -1 upon failure */ public static long parseIdFromFileName(Path translogFile) { final String fileName = translogFile.getFileName().toString(); final Matcher matcher = PARSE_ID_PATTERN.matcher(fileName); @@ -223,9 +324,9 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC } public void updateBuffer(ByteSizeValue bufferSize) { - this.bufferSize = bufferSize.bytesAsInt(); + config.setBufferSize(bufferSize.bytesAsInt()); try (ReleasableLock lock = writeLock.acquire()) { - current.updateBufferSize(this.bufferSize); + current.updateBufferSize(config.getBufferSize()); } } @@ -236,15 +337,12 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC @Override public void close() throws IOException { if (closed.compareAndSet(false, true)) { - if (indexSettingsService != null) { - indexSettingsService.removeListener(applySettings); - } - try (ReleasableLock lock = writeLock.acquire()) { try { - IOUtils.close(this.current); + IOUtils.close(current, currentCommittingTranslog); } finally { - IOUtils.close(uncommittedTranslogs); + IOUtils.close(recoveredTranslogs); + recoveredTranslogs.clear(); } } finally { FutureUtils.cancel(syncScheduler); @@ -263,11 +361,11 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC } /** - * Returns the id of the current transaction log. + * Returns the generation of the current transaction log. */ - public long currentId() { + public long currentFileGeneration() { try (ReleasableLock lock = readLock.acquire()) { - return current.translogId(); + return current.getGeneration(); } } @@ -279,11 +377,9 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC int ops = 0; try (ReleasableLock lock = readLock.acquire()) { ops += current.totalOperations(); - for (ChannelReader translog : uncommittedTranslogs) { - int tops = translog.totalOperations(); - if (tops == ChannelReader.UNKNOWN_OP_COUNT) { - return ChannelReader.UNKNOWN_OP_COUNT; - } + if (currentCommittingTranslog != null) { + int tops = currentCommittingTranslog.totalOperations(); + assert tops != TranslogReader.UNKNOWN_OP_COUNT; ops += tops; } } @@ -297,121 +393,40 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC long size = 0; try (ReleasableLock lock = readLock.acquire()) { size += current.sizeInBytes(); - for (ChannelReader translog : uncommittedTranslogs) { - size += translog.sizeInBytes(); + if (currentCommittingTranslog != null) { + size += currentCommittingTranslog.sizeInBytes(); } } return size; } - /** - * notifies the translog that translogId was committed as part of the commit data in lucene, together - * with all operations from previous translogs. This allows releasing all previous translogs. - * - * @throws FileNotFoundException if the given translog id can not be found. - */ - public void markCommitted(final long translogId) throws FileNotFoundException { - try (ReleasableLock lock = writeLock.acquire()) { - logger.trace("updating translogs on commit of [{}]", translogId); - if (translogId < lastCommittedTranslogId) { - throw new IllegalArgumentException("committed translog id can only go up (current [" - + lastCommittedTranslogId + "], got [" + translogId + "]"); - } - boolean found = false; - if (current.translogId() == translogId) { - found = true; - } else { - if (translogId > current.translogId()) { - throw new IllegalArgumentException("committed translog id must be lower or equal to current id (current [" - + current.translogId() + "], got [" + translogId + "]"); - } - } - if (found == false) { - // try to find it in uncommittedTranslogs - for (ChannelImmutableReader translog : uncommittedTranslogs) { - if (translog.translogId() == translogId) { - found = true; - break; - } - } - } - if (found == false) { - ArrayList currentIds = new ArrayList<>(); - for (ChannelReader translog : Iterables.concat(uncommittedTranslogs, Collections.singletonList(current))) { - currentIds.add(translog.translogId()); - } - throw new FileNotFoundException("committed translog id can not be found (current [" - + Strings.collectionToCommaDelimitedString(currentIds) + "], got [" + translogId + "]"); - } - lastCommittedTranslogId = translogId; - while (uncommittedTranslogs.isEmpty() == false && uncommittedTranslogs.get(0).translogId() < translogId) { - ChannelReader old = uncommittedTranslogs.remove(0); - logger.trace("removed [{}] from uncommitted translog list", old.translogId()); - try { - old.close(); - } catch (IOException e) { - logger.error("failed to closed old translog [{}] (committed id [{}])", e, old, translogId); - } - } - } - } - /** - * Creates a new transaction log file internally. That new file will be visible to all outstanding views. - * The id of the new translog file is returned. - */ - public long newTranslog() throws TranslogException, IOException { - try (ReleasableLock lock = writeLock.acquire()) { - final TranslogFile old = current; - final TranslogFile newFile = createTranslogFile(old); - current = newFile; - ChannelImmutableReader reader = old.immutableReader(); - uncommittedTranslogs.add(reader); - // notify all outstanding views of the new translog (no views are created now as - // we hold a write lock). - for (FsView view : outstandingViews) { - view.onNewTranslog(old.immutableReader(), current.reader()); - } - IOUtils.close(old); - logger.trace("current translog set to [{}]", current.translogId()); - return current.translogId(); - } - } - protected TranslogFile createTranslogFile(@Nullable TranslogFile reuse) throws IOException { - TranslogFile newFile; - long size = Long.MAX_VALUE; + TranslogWriter createWriter(long fileGeneration) throws IOException { + TranslogWriter newFile; try { - long id = idGenerator++; - newFile = type.create(shardId, id, new InternalChannelReference(id, location.resolve(getFilename(id)), StandardOpenOption.READ, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW), bufferSize); + newFile = TranslogWriter.create(config.getType(), shardId, translogUUID, fileGeneration, location.resolve(getFilename(fileGeneration)), new OnCloseRunnable(), config.getBufferSize()); } catch (IOException e) { throw new TranslogException(shardId, "failed to create new translog file", e); } - if (reuse != null) { - newFile.reuse(reuse); - } return newFile; } /** - * Read the Operation object from the given location, returns null if the - * Operation could not be read. + * Read the Operation object from the given location. */ public Translog.Operation read(Location location) { try (ReleasableLock lock = readLock.acquire()) { - ChannelReader reader = null; - if (current.translogId() == location.translogId) { + final TranslogReader reader; + if (current.getGeneration() == location.generation) { reader = current; + } else if (currentCommittingTranslog != null && currentCommittingTranslog.getGeneration() == location.generation) { + reader = currentCommittingTranslog; } else { - for (ChannelReader translog : uncommittedTranslogs) { - if (translog.translogId() == location.translogId) { - reader = translog; - break; - } - } + throw new IllegalStateException("Can't read from translog location" + location); } - return reader == null ? null : reader.read(location); + return reader.read(location); } catch (IOException e) { throw new ElasticsearchException("failed to read source from translog location " + location, e); } @@ -423,14 +438,13 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC public Location add(Operation operation) throws TranslogException { ReleasableBytesStreamOutput out = new ReleasableBytesStreamOutput(bigArrays); try { - TranslogStreams.writeTranslogOperation(out, operation); + writeOperation(out, operation); ReleasablePagedBytesReference bytes = out.bytes(); try (ReleasableLock lock = readLock.acquire()) { Location location = current.add(bytes); - if (syncOnEachOperation) { + if (config.isSyncOnEachOperation()) { current.sync(); } - assert current.assertBytesAtLocation(location, bytes); return location; } @@ -447,26 +461,29 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC */ public Snapshot newSnapshot() { try (ReleasableLock lock = readLock.acquire()) { - // leave one place for current. - final ChannelReader[] readers = uncommittedTranslogs.toArray(new ChannelReader[uncommittedTranslogs.size() + 1]); - readers[readers.length - 1] = current; - return createdSnapshot(readers); + ArrayList toOpen = new ArrayList<>(); + toOpen.addAll(recoveredTranslogs); + if (currentCommittingTranslog != null) { + toOpen.add(currentCommittingTranslog); + } + toOpen.add(current); + return createdSnapshot(toOpen.toArray(new TranslogReader[toOpen.size()])); } } - private Snapshot createdSnapshot(ChannelReader... translogs) { - ArrayList channelSnapshots = new ArrayList<>(); + private Snapshot createdSnapshot(TranslogReader... translogs) { + ArrayList channelSnapshots = new ArrayList<>(); boolean success = false; try { - for (ChannelReader translog : translogs) { + for (TranslogReader translog : translogs) { channelSnapshots.add(translog.newSnapshot()); } - Snapshot snapshot = new TranslogSnapshot(channelSnapshots, logger); + Snapshot snapshot = new TranslogSnapshot(channelSnapshots); success = true; return snapshot; } finally { if (success == false) { - IOUtils.closeWhileHandlingException(channelSnapshots); + Releasables.close(channelSnapshots); } } } @@ -479,12 +496,12 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC // we need to acquire the read lock to make sure new translog is created // and will be missed by the view we're making try (ReleasableLock lock = readLock.acquire()) { - ArrayList translogs = new ArrayList<>(); + ArrayList translogs = new ArrayList<>(); try { - for (ChannelImmutableReader translog : uncommittedTranslogs) { - translogs.add(translog.clone()); + if (currentCommittingTranslog != null) { + translogs.add(currentCommittingTranslog.clone()); } - translogs.add(current.reader()); + translogs.add(current.newReaderFromWriter()); FsView view = new FsView(translogs); // this is safe as we know that no new translog is being made at the moment // (we hold a read lock) and the view will be notified of any future one @@ -516,8 +533,12 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC } /** package private for testing */ - String getFilename(long translogId) { - return TRANSLOG_FILE_PREFIX + translogId; + public static String getFilename(long generation) { + return TRANSLOG_FILE_PREFIX + generation + TRANSLOG_FILE_SUFFIX; + } + + static String getCommitFileName(long generation) { + return TRANSLOG_FILE_PREFIX + generation + CHECKPOINT_SUFFIX; } @@ -527,7 +548,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC */ public boolean ensureSynced(Location location) throws IOException { try (ReleasableLock lock = readLock.acquire()) { - if (location.translogId == current.id) { // if we have a new one it's already synced + if (location.generation == current.generation) { // if we have a new one it's already synced return current.syncUpTo(location.translogLocation + location.size); } } @@ -544,32 +565,47 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC } } - private boolean isReferencedTranslogId(long translogId) { - return translogId >= lastCommittedTranslogId; + private boolean isReferencedGeneration(long generation) { // pkg private for testing + return generation >= lastCommittedTranslogFileGeneration; } - private final class InternalChannelReference extends ChannelReference { - final long translogId; + public TranslogConfig getConfig() { + return config; + } - public InternalChannelReference(long translogId, Path file, OpenOption... openOptions) throws IOException { - super(file, openOptions); - this.translogId = translogId; - } + private final class OnCloseRunnable implements Callback { @Override - protected void closeInternal() { - super.closeInternal(); + public void handle(ChannelReference channelReference) { try (ReleasableLock lock = writeLock.acquire()) { - if (isReferencedTranslogId(translogId) == false) { - // if the given path is not the current we can safely delete the file since all references are released - logger.trace("delete translog file - not referenced and not current anymore {}", file()); - IOUtils.deleteFilesIgnoringExceptions(file()); + if (isReferencedGeneration(channelReference.getGeneration()) == false) { + Path translogPath = channelReference.getPath(); + assert channelReference.getPath().getParent().equals(location) : "translog files must be in the location folder: " + location + " but was: " + translogPath; + // if the given translogPath is not the current we can safely delete the file since all references are released + logger.trace("delete translog file - not referenced and not current anymore {}", translogPath); + IOUtils.deleteFilesIgnoringExceptions(translogPath); + IOUtils.deleteFilesIgnoringExceptions(translogPath.resolveSibling(getCommitFileName(channelReference.getGeneration()))); + + } + try (DirectoryStream stream = Files.newDirectoryStream(location)) { + for (Path path : stream) { + Matcher matcher = PARSE_ID_PATTERN.matcher(path.getFileName().toString()); + if (matcher.matches()) { + long generation = Long.parseLong(matcher.group(1)); + if (isReferencedGeneration(generation) == false) { + logger.trace("delete translog file - not referenced and not current anymore {}", path); + IOUtils.deleteFilesIgnoringExceptions(path); + IOUtils.deleteFilesIgnoringExceptions(path.resolveSibling(getCommitFileName(channelReference.getGeneration()))); + } + } + } + } catch (IOException e) { + logger.warn("failed to delete unreferenced translog files", e); } } } } - /** * a view into the translog, capturing all translog file at the moment of creation * and updated with any future translog. @@ -578,9 +614,9 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC boolean closed; // last in this list is always FsTranslog.current - final List orderedTranslogs; + final List orderedTranslogs; - FsView(List orderedTranslogs) { + FsView(List orderedTranslogs) { assert orderedTranslogs.isEmpty() == false; // clone so we can safely mutate.. this.orderedTranslogs = new ArrayList<>(orderedTranslogs); @@ -592,7 +628,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC * @param oldCurrent a new read only reader for the old current (should replace the previous reference) * @param newCurrent a reader into the new current. */ - synchronized void onNewTranslog(ChannelReader oldCurrent, ChannelReader newCurrent) throws IOException { + synchronized void onNewTranslog(TranslogReader oldCurrent, TranslogReader newCurrent) throws IOException { // even though the close method removes this view from outstandingViews, there is no synchronisation in place // between that operation and an ongoing addition of a new translog, already having an iterator. // As such, this method can be called despite of the fact that we are closed. We need to check and ignore. @@ -607,17 +643,17 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC } @Override - public synchronized long minTranslogId() { + public synchronized long minTranslogGeneration() { ensureOpen(); - return orderedTranslogs.get(0).translogId(); + return orderedTranslogs.get(0).getGeneration(); } @Override public synchronized int totalOperations() { int ops = 0; - for (ChannelReader translog : orderedTranslogs) { + for (TranslogReader translog : orderedTranslogs) { int tops = translog.totalOperations(); - if (tops == ChannelReader.UNKNOWN_OP_COUNT) { + if (tops == TranslogReader.UNKNOWN_OP_COUNT) { return -1; } ops += tops; @@ -628,7 +664,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC @Override public synchronized long sizeInBytes() { long size = 0; - for (ChannelReader translog : orderedTranslogs) { + for (TranslogReader translog : orderedTranslogs) { size += translog.sizeInBytes(); } return size; @@ -636,7 +672,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC public synchronized Snapshot snapshot() { ensureOpen(); - return createdSnapshot(orderedTranslogs.toArray(new ChannelReader[orderedTranslogs.size()])); + return createdSnapshot(orderedTranslogs.toArray(new TranslogReader[orderedTranslogs.size()])); } @@ -648,11 +684,11 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC @Override public void close() { - List toClose = new ArrayList<>(); + List toClose = new ArrayList<>(); try { synchronized (this) { if (closed == false) { - logger.trace("closing view starting at translog [{}]", minTranslogId()); + logger.trace("closing view starting at translog [{}]", minTranslogGeneration()); closed = true; outstandingViews.remove(this); toClose.addAll(orderedTranslogs); @@ -679,6 +715,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC if (closed.get()) { return; } + final ThreadPool threadPool = config.getThreadPool(); if (syncNeeded()) { threadPool.executor(ThreadPool.Names.FLUSH).execute(new Runnable() { @Override @@ -689,24 +726,24 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC logger.warn("failed to sync translog", e); } if (closed.get() == false) { - syncScheduler = threadPool.schedule(syncInterval, ThreadPool.Names.SAME, Sync.this); + syncScheduler = threadPool.schedule(config.getSyncInterval(), ThreadPool.Names.SAME, Sync.this); } } }); } else { - syncScheduler = threadPool.schedule(syncInterval, ThreadPool.Names.SAME, Sync.this); + syncScheduler = threadPool.schedule(config.getSyncInterval(), ThreadPool.Names.SAME, Sync.this); } } } public static class Location implements Accountable, Comparable { - public final long translogId; + public final long generation; public final long translogLocation; public final int size; - public Location(long translogId, long translogLocation, int size) { - this.translogId = translogId; + public Location(long generation, long translogLocation, int size) { + this.generation = generation; this.translogLocation = translogLocation; this.size = size; } @@ -723,15 +760,15 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC @Override public String toString() { - return "[id: " + translogId + ", location: " + translogLocation + ", size: " + size + "]"; + return "[generation: " + generation + ", location: " + translogLocation + ", size: " + size + "]"; } @Override public int compareTo(Location o) { - if (translogId == o.translogId) { + if (generation == o.generation) { return Long.compare(translogLocation, o.translogLocation); } - return Long.compare(translogId, o.translogId); + return Long.compare(generation, o.generation); } @Override @@ -741,7 +778,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC Location location = (Location) o; - if (translogId != location.translogId) return false; + if (generation != location.generation) return false; if (translogLocation != location.translogLocation) return false; return size == location.size; @@ -749,7 +786,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC @Override public int hashCode() { - int result = (int) (translogId ^ (translogId >>> 32)); + int result = (int) (generation ^ (generation >>> 32)); result = 31 * result + (int) (translogLocation ^ (translogLocation >>> 32)); result = 31 * result + size; return result; @@ -789,8 +826,8 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC /** create a snapshot from this view */ Snapshot snapshot(); - /** this smallest translog id in this view */ - long minTranslogId(); + /** this smallest translog generation in this view */ + long minTranslogGeneration(); } @@ -1483,13 +1520,6 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC } } - /** - * Returns the current durability mode of this translog. - */ - public Durabilty getDurabilty() { - return durabilty; - } - public enum Durabilty { /** * Async durability - translogs are synced based on a time interval. @@ -1500,14 +1530,225 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC */ REQUEST; - public static Durabilty getFromSettings(ESLogger logger, Settings settings, Durabilty defaultValue) { - final String value = settings.get(INDEX_TRANSLOG_DURABILITY, defaultValue.name()); - try { - return valueOf(value.toUpperCase(Locale.ROOT)); - } catch (IllegalArgumentException ex) { - logger.warn("Can't apply {} illegal value: {} using {} instead, use one of: {}", INDEX_TRANSLOG_DURABILITY, value, defaultValue, Arrays.toString(values())); - return defaultValue; - } + } + + private static void verifyChecksum(BufferedChecksumStreamInput in) throws IOException { + // This absolutely must come first, or else reading the checksum becomes part of the checksum + long expectedChecksum = in.getChecksum(); + long readChecksum = in.readInt() & 0xFFFF_FFFFL; + if (readChecksum != expectedChecksum) { + throw new TranslogCorruptedException("translog stream is corrupted, expected: 0x" + + Long.toHexString(expectedChecksum) + ", got: 0x" + Long.toHexString(readChecksum)); } } + + public static Snapshot snapshotFromStream(StreamInput input, final int numOps) { + final BufferedChecksumStreamInput checksumStreamInput = new BufferedChecksumStreamInput(input); + return new Snapshot() { + int read = 0; + @Override + public int estimatedTotalOperations() { + return numOps; + } + + @Override + public Operation next() throws IOException { + if (read < numOps) { + read++; + return readOperation(checksumStreamInput); + } + return null; + } + + @Override + public void close() { + // doNothing + } + }; + } + + static Translog.Operation readOperation(BufferedChecksumStreamInput in) throws IOException { + int opSize = in.readInt(); + Translog.Operation operation; + try { + in.resetDigest(); // size is not part of the checksum? + if (in.markSupported()) { // if we can we validate the checksum first + in.mark(opSize); + if (opSize < 4) { // 4byte for the checksum + throw new TranslogCorruptedException("operation size must be at least 4 but was: " + opSize); + } + in.skip(opSize-4); + verifyChecksum(in); + in.reset(); + } + Translog.Operation.Type type = Translog.Operation.Type.fromId(in.readByte()); + operation = newOperationFromType(type); + operation.readFrom(in); + verifyChecksum(in); + } catch (EOFException e) { + throw new TruncatedTranslogException("reached premature end of file, translog is truncated", e); + } catch (AssertionError|Exception e) { + throw new TranslogCorruptedException("translog corruption while reading from stream", e); + } + return operation; + } + + public static void writeOperation(StreamOutput outStream, Translog.Operation op) throws IOException { + // We first write to a NoopStreamOutput to get the size of the + // operation. We could write to a byte array and then send that as an + // alternative, but here we choose to use CPU over allocating new + // byte arrays. + NoopStreamOutput noopOut = new NoopStreamOutput(); + noopOut.writeByte(op.opType().id()); + op.writeTo(noopOut); + noopOut.writeInt(0); // checksum holder + int size = noopOut.getCount(); + + // This BufferedChecksumStreamOutput remains unclosed on purpose, + // because closing it closes the underlying stream, which we don't + // want to do here. + BufferedChecksumStreamOutput out = new BufferedChecksumStreamOutput(outStream); + outStream.writeInt(size); // opSize is not checksummed + out.writeByte(op.opType().id()); + op.writeTo(out); + long checksum = out.getChecksum(); + out.writeInt((int)checksum); + } + + /** + * Returns a new empty translog operation for the given {@link Translog.Operation.Type} + */ + static Translog.Operation newOperationFromType(Translog.Operation.Type type) throws IOException { + switch (type) { + case CREATE: + return new Translog.Create(); + case DELETE: + return new Translog.Delete(); + case DELETE_BY_QUERY: + return new Translog.DeleteByQuery(); + case SAVE: + return new Translog.Index(); + default: + throw new IOException("No type for [" + type + "]"); + } + } + + + @Override + public void prepareCommit() throws IOException { + ensureOpen(); + TranslogWriter writer = null; + try (ReleasableLock lock = writeLock.acquire()) { + if (currentCommittingTranslog != null) { + throw new IllegalStateException("already committing a translog with generation: " + currentCommittingTranslog.getGeneration()); + } + writer = current; + writer.sync(); + currentCommittingTranslog = current.immutableReader(); + Path checkpoint = location.resolve(CHECKPOINT_FILE_NAME); + assert Checkpoint.read(checkpoint).generation == currentCommittingTranslog.getGeneration(); + Path commitCheckpoint = location.resolve(getCommitFileName(currentCommittingTranslog.getGeneration())); + Files.copy(checkpoint, commitCheckpoint); + IOUtils.fsync(commitCheckpoint, false); + IOUtils.fsync(commitCheckpoint.getParent(), true); + // create a new translog file - this will sync it and update the checkpoint data; + final TranslogWriter newFile = createWriter(current.getGeneration() + 1); + current = newFile; + // notify all outstanding views of the new translog (no views are created now as + // we hold a write lock). + for (FsView view : outstandingViews) { + view.onNewTranslog(currentCommittingTranslog.clone(), newFile.newReaderFromWriter()); + } + IOUtils.close(writer); + logger.trace("current translog set to [{}]", current.getGeneration()); + assert writer.syncNeeded() == false : "old translog writer must not need a sync"; + + } catch (Throwable t) { + close(); // tragic event + throw t; + } + } + + @Override + public void commit() throws IOException { + ensureOpen(); + ImmutableTranslogReader toClose = null; + try (ReleasableLock lock = writeLock.acquire()) { + if (currentCommittingTranslog == null) { + prepareCommit(); + } + current.sync(); + lastCommittedTranslogFileGeneration = current.getGeneration(); // this is important - otherwise old files will not be cleaned up + if (recoveredTranslogs.isEmpty() == false) { + IOUtils.close(recoveredTranslogs); + recoveredTranslogs.clear(); + } + toClose = this.currentCommittingTranslog; + this.currentCommittingTranslog = null; + } finally { + IOUtils.close(toClose); + } + } + + @Override + public void rollback() throws IOException { + ensureOpen(); + close(); + } + + /** + * References a transaction log generation + */ + public final static class TranslogGeneration { + public final String translogUUID; + public final long translogFileGeneration; + + public TranslogGeneration(String translogUUID, long translogFileGeneration) { + this.translogUUID = translogUUID; + this.translogFileGeneration = translogFileGeneration; + } + + } + + /** + * Returns the current generation of this translog. This corresponds to the latest uncommitted translog generation + */ + public TranslogGeneration getGeneration() { + try (ReleasableLock lock = writeLock.acquire()) { + return new TranslogGeneration(translogUUID, currentFileGeneration()); + } + } + + /** + * Returns true iff the given generation is the current gbeneration of this translog + * @param generation + * @return + */ + public boolean isCurrent(TranslogGeneration generation) { + try (ReleasableLock lock = writeLock.acquire()) { + if (generation != null) { + if (generation.translogUUID.equals(translogUUID) == false) { + throw new IllegalArgumentException("commit belongs to a different translog: " + generation.translogUUID + " vs. " + translogUUID); + } + return generation.translogFileGeneration == currentFileGeneration(); + } + } + return false; + } + + long getFirstOperationPosition() { // for testing + return current.getFirstOperationOffset(); + } + + List getRecoveredReaders() { // for testing + return this.recoveredTranslogs; + } + + private void ensureOpen() { + if (closed.get()) { + throw new AlreadyClosedException("translog is already closed"); + } + } + + } diff --git a/src/main/java/org/elasticsearch/index/translog/TranslogConfig.java b/src/main/java/org/elasticsearch/index/translog/TranslogConfig.java new file mode 100644 index 00000000000..e9f4089b0b7 --- /dev/null +++ b/src/main/java/org/elasticsearch/index/translog/TranslogConfig.java @@ -0,0 +1,196 @@ +/* + * 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.translog; + +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.index.settings.IndexSettings; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.translog.Translog.TranslogGeneration; +import org.elasticsearch.threadpool.ThreadPool; + +import java.nio.file.Path; + +/* + * Holds all the configuration that is used to create a {@link Translog}. + * Once {@link Translog} has been created with this object, changes to this + * object will affect the {@link Translog} instance. + */ +public final class TranslogConfig { + + public static final String INDEX_TRANSLOG_DURABILITY = "index.translog.durability"; + public static final String INDEX_TRANSLOG_FS_TYPE = "index.translog.fs.type"; + public static final String INDEX_TRANSLOG_BUFFER_SIZE = "index.translog.fs.buffer_size"; + public static final String INDEX_TRANSLOG_SYNC_INTERVAL = "index.translog.sync_interval"; + public static final ByteSizeValue INACTIVE_SHARD_TRANSLOG_BUFFER = ByteSizeValue.parseBytesSizeValue("1kb"); + + private final TimeValue syncInterval; + private final BigArrays bigArrays; + private final ThreadPool threadPool; + private final boolean syncOnEachOperation; + private volatile int bufferSize; + private volatile TranslogGeneration translogGeneration; + private volatile Translog.Durabilty durabilty = Translog.Durabilty.REQUEST; + private volatile TranslogWriter.Type type; + private final Settings indexSettings; + private final ShardId shardId; + private final Path translogPath; + + /** + * Creates a new TranslogConfig instance + * @param shardId the shard ID this translog belongs to + * @param translogPath the path to use for the transaction log files + * @param indexSettings the index settings used to set internal variables + * @param durabilty the default durability setting for the translog + * @param bigArrays a bigArrays instance used for temporarily allocating write operations + * @param threadPool a {@link ThreadPool} to schedule async sync durability + */ + public TranslogConfig(ShardId shardId, Path translogPath, @IndexSettings Settings indexSettings, Translog.Durabilty durabilty, BigArrays bigArrays, @Nullable ThreadPool threadPool) { + this.indexSettings = indexSettings; + this.shardId = shardId; + this.translogPath = translogPath; + this.durabilty = durabilty; + this.threadPool = threadPool; + this.bigArrays = bigArrays; + this.type = TranslogWriter.Type.fromString(indexSettings.get(INDEX_TRANSLOG_FS_TYPE, TranslogWriter.Type.BUFFERED.name())); + this.bufferSize = (int) indexSettings.getAsBytesSize(INDEX_TRANSLOG_BUFFER_SIZE, ByteSizeValue.parseBytesSizeValue("64k")).bytes(); // Not really interesting, updated by IndexingMemoryController... + + syncInterval = indexSettings.getAsTime(INDEX_TRANSLOG_SYNC_INTERVAL, TimeValue.timeValueSeconds(5)); + if (syncInterval.millis() > 0 && threadPool != null) { + syncOnEachOperation = false; + } else if (syncInterval.millis() == 0) { + syncOnEachOperation = true; + } else { + syncOnEachOperation = false; + } + } + + /** + * Returns a {@link ThreadPool} to schedule async durability operations + */ + public ThreadPool getThreadPool() { + return threadPool; + } + + /** + * Returns the current durability mode of this translog. + */ + public Translog.Durabilty getDurabilty() { + return durabilty; + } + + /** + * Sets the current durability mode for the translog. + */ + public void setDurabilty(Translog.Durabilty durabilty) { + this.durabilty = durabilty; + } + + /** + * Returns the translog type + */ + public TranslogWriter.Type getType() { + return type; + } + + /** + * Sets the TranslogType for this Translog. The change will affect all subsequent translog files. + */ + public void setType(TranslogWriter.Type type) { + this.type = type; + } + + /** + * Returns true iff each low level operation shoudl be fsynced + */ + public boolean isSyncOnEachOperation() { + return syncOnEachOperation; + } + + /** + * Retruns the current translog buffer size. + */ + public int getBufferSize() { + return bufferSize; + } + + /** + * Sets the current buffer size - for setting a live setting use {@link Translog#updateBuffer(ByteSizeValue)} + */ + public void setBufferSize(int bufferSize) { + this.bufferSize = bufferSize; + } + + /** + * Returns the current async fsync interval + */ + public TimeValue getSyncInterval() { + return syncInterval; + } + + /** + * Returns the current index settings + */ + public Settings getIndexSettings() { + return indexSettings; + } + + /** + * Returns the shard ID this config is created for + */ + public ShardId getShardId() { + return shardId; + } + + /** + * Returns a BigArrays instance for this engine + */ + public BigArrays getBigArrays() { + return bigArrays; + } + + /** + * Returns the translog path for this engine + */ + public Path getTranslogPath() { + return translogPath; + } + + /** + * Returns the translog generation to open. If this is null a new translog is created. If non-null + * the translog tries to open the given translog generation. The generation is treated as the last generation referenced + * form already committed data. This means all operations that have not yet been committed should be in the translog + * file referenced by this generation. The translog creation will fail if this generation can't be opened. + */ + public TranslogGeneration getTranslogGeneration() { + return translogGeneration; + } + + /** + * Set the generation to be opened. Use null to start with a fresh translog. + * @see #getTranslogGeneration() + */ + public void setTranslogGeneration(TranslogGeneration translogGeneration) { + this.translogGeneration = translogGeneration; + } +} diff --git a/src/main/java/org/elasticsearch/index/translog/TranslogFile.java b/src/main/java/org/elasticsearch/index/translog/TranslogFile.java deleted file mode 100644 index 3f767752c53..00000000000 --- a/src/main/java/org/elasticsearch/index/translog/TranslogFile.java +++ /dev/null @@ -1,240 +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.translog; - -import org.elasticsearch.common.bytes.BytesArray; -import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.io.Channels; -import org.elasticsearch.common.util.concurrent.ReleasableLock; -import org.elasticsearch.index.shard.ShardId; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.concurrent.locks.ReadWriteLock; -import java.util.concurrent.locks.ReentrantReadWriteLock; - -public class TranslogFile extends ChannelReader { - - protected final ShardId shardId; - protected final ReleasableLock readLock; - protected final ReleasableLock writeLock; - /* the offset in bytes that was written when the file was last synced*/ - protected volatile long lastSyncedOffset; - /* the number of translog operations written to this file */ - protected volatile int operationCounter; - /* the offset in bytes written to the file */ - protected volatile long writtenOffset; - - public TranslogFile(ShardId shardId, long id, ChannelReference channelReference) throws IOException { - super(id, channelReference); - this.shardId = shardId; - ReadWriteLock rwl = new ReentrantReadWriteLock(); - readLock = new ReleasableLock(rwl.readLock()); - writeLock = new ReleasableLock(rwl.writeLock()); - final TranslogStream stream = this.channelReference.stream(); - int headerSize = stream.writeHeader(channelReference.channel()); - this.writtenOffset += headerSize; - this.lastSyncedOffset += headerSize; - } - - - public enum Type { - - SIMPLE() { - @Override - public TranslogFile create(ShardId shardId, long id, ChannelReference channelReference, int bufferSize) throws IOException { - return new TranslogFile(shardId, id, channelReference); - } - }, - BUFFERED() { - @Override - public TranslogFile create(ShardId shardId, long id, ChannelReference channelReference, int bufferSize) throws IOException { - return new BufferingTranslogFile(shardId, id, channelReference, bufferSize); - } - }; - - public abstract TranslogFile create(ShardId shardId, long id, ChannelReference raf, int bufferSize) throws IOException; - - public static Type fromString(String type) { - if (SIMPLE.name().equalsIgnoreCase(type)) { - return SIMPLE; - } else if (BUFFERED.name().equalsIgnoreCase(type)) { - return BUFFERED; - } - throw new IllegalArgumentException("No translog fs type [" + type + "]"); - } - } - - - /** add the given bytes to the translog and return the location they were written at */ - public Translog.Location add(BytesReference data) throws IOException { - try (ReleasableLock lock = writeLock.acquire()) { - long position = writtenOffset; - data.writeTo(channelReference.channel()); - writtenOffset = writtenOffset + data.length(); - operationCounter = operationCounter + 1; - return new Translog.Location(id, position, data.length()); - } - } - - /** reuse resources from another translog file, which is guaranteed not to be used anymore */ - public void reuse(TranslogFile other) throws TranslogException {} - - /** change the size of the internal buffer if relevant */ - public void updateBufferSize(int bufferSize) throws TranslogException {} - - /** write all buffered ops to disk and fsync file */ - public void sync() throws IOException { - // check if we really need to sync here... - if (syncNeeded()) { - try (ReleasableLock lock = writeLock.acquire()) { - lastSyncedOffset = writtenOffset; - channelReference.channel().force(false); - } - } - } - - /** returns true if there are buffered ops */ - public boolean syncNeeded() { - return writtenOffset != lastSyncedOffset; // by default nothing is buffered - } - - @Override - public int totalOperations() { - return operationCounter; - } - - @Override - public long sizeInBytes() { - return writtenOffset; - } - - @Override - public ChannelSnapshot newSnapshot() { - return new ChannelSnapshot(immutableReader()); - } - - /** - * Flushes the buffer if the translog is buffered. - */ - protected void flush() throws IOException {} - - /** - * returns a new reader that follows the current writes (most importantly allows making - * repeated snapshots that includes new content) - */ - public ChannelReader reader() { - channelReference.incRef(); - boolean success = false; - try { - ChannelReader reader = new InnerReader(this.id, channelReference); - success = true; - return reader; - } finally { - if (!success) { - channelReference.decRef(); - } - } - } - - - /** returns a new immutable reader which only exposes the current written operation * */ - public ChannelImmutableReader immutableReader() throws TranslogException { - if (channelReference.tryIncRef()) { - try (ReleasableLock lock = writeLock.acquire()) { - flush(); - ChannelImmutableReader reader = new ChannelImmutableReader(this.id, channelReference, writtenOffset, operationCounter); - channelReference.incRef(); // for new reader - return reader; - } catch (Exception e) { - throw new TranslogException(shardId, "exception while creating an immutable reader", e); - } finally { - channelReference.decRef(); - } - } else { - throw new TranslogException(shardId, "can't increment channel [" + channelReference + "] ref count"); - } - } - - boolean assertBytesAtLocation(Translog.Location location, BytesReference expectedBytes) throws IOException { - ByteBuffer buffer = ByteBuffer.allocate(location.size); - readBytes(buffer, location.translogLocation); - return new BytesArray(buffer.array()).equals(expectedBytes); - } - - /** - * this class is used when one wants a reference to this file which exposes all recently written operation. - * as such it needs access to the internals of the current reader - */ - final class InnerReader extends ChannelReader { - - public InnerReader(long id, ChannelReference channelReference) { - super(id, channelReference); - } - - @Override - public long sizeInBytes() { - return TranslogFile.this.sizeInBytes(); - } - - @Override - public int totalOperations() { - return TranslogFile.this.totalOperations(); - } - - @Override - protected void readBytes(ByteBuffer buffer, long position) throws IOException { - TranslogFile.this.readBytes(buffer, position); - } - - @Override - public ChannelSnapshot newSnapshot() { - return TranslogFile.this.newSnapshot(); - } - } - - /** - * Syncs the translog up to at least the given offset unless already synced - * @return true if this call caused an actual sync operation - */ - public boolean syncUpTo(long offset) throws IOException { - if (lastSyncedOffset < offset) { - sync(); - return true; - } - return false; - } - - @Override - protected final void doClose() throws IOException { - try { - sync(); - } finally { - super.doClose(); - } - } - - @Override - protected void readBytes(ByteBuffer buffer, long position) throws IOException { - try (ReleasableLock lock = readLock.acquire()) { - Channels.readFromFileChannelWithEofException(channelReference.channel(), position, buffer); - } - } -} diff --git a/src/main/java/org/elasticsearch/index/translog/TranslogReader.java b/src/main/java/org/elasticsearch/index/translog/TranslogReader.java new file mode 100644 index 00000000000..769905ae75e --- /dev/null +++ b/src/main/java/org/elasticsearch/index/translog/TranslogReader.java @@ -0,0 +1,311 @@ +/* + * 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.translog; + +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.IndexFormatTooNewException; +import org.apache.lucene.index.IndexFormatTooOldException; +import org.apache.lucene.store.AlreadyClosedException; +import org.apache.lucene.store.InputStreamDataInput; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.RamUsageEstimator; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.common.io.stream.ByteBufferStreamInput; +import org.elasticsearch.common.io.stream.InputStreamStreamInput; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.Channels; +import java.nio.channels.FileChannel; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * A base class for all classes that allows reading ops from translog files + */ +public abstract class TranslogReader implements Closeable, Comparable { + public static final int UNKNOWN_OP_COUNT = -1; + private static final byte LUCENE_CODEC_HEADER_BYTE = 0x3f; + private static final byte UNVERSIONED_TRANSLOG_HEADER_BYTE = 0x00; + + protected final long generation; + protected final ChannelReference channelReference; + protected final FileChannel channel; + protected final AtomicBoolean closed = new AtomicBoolean(false); + protected final long firstOperationOffset; + + public TranslogReader(long generation, ChannelReference channelReference, long firstOperationOffset) { + this.generation = generation; + this.channelReference = channelReference; + this.channel = channelReference.getChannel(); + this.firstOperationOffset = firstOperationOffset; + } + + public long getGeneration() { + return this.generation; + } + + public abstract long sizeInBytes(); + + abstract public int totalOperations(); + + public final long getFirstOperationOffset() { + return firstOperationOffset; + } + + public Translog.Operation read(Translog.Location location) throws IOException { + assert location.generation == generation : "read location's translog generation [" + location.generation + "] is not [" + generation + "]"; + ByteBuffer buffer = ByteBuffer.allocate(location.size); + try (BufferedChecksumStreamInput checksumStreamInput = checksummedStream(buffer, location.translogLocation, location.size, null)) { + return read(checksumStreamInput); + } + } + + /** read the size of the op (i.e., number of bytes, including the op size) written at the given position */ + private final int readSize(ByteBuffer reusableBuffer, long position) { + // read op size from disk + assert reusableBuffer.capacity() >= 4 : "reusable buffer must have capacity >=4 when reading opSize. got [" + reusableBuffer.capacity() + "]"; + try { + reusableBuffer.clear(); + reusableBuffer.limit(4); + readBytes(reusableBuffer, position); + reusableBuffer.flip(); + // Add an extra 4 to account for the operation size integer itself + final int size = reusableBuffer.getInt() + 4; + final long maxSize = sizeInBytes() - position; + if (size < 0 || size > maxSize) { + throw new TranslogCorruptedException("operation size is corrupted must be [0.." + maxSize + "] but was: " + size); + } + + return size; + } catch (IOException e) { + throw new ElasticsearchException("unexpected exception reading from translog snapshot of " + this.channelReference.getPath(), e); + } + } + + public Translog.Snapshot newSnapshot() { + final ByteBuffer reusableBuffer = ByteBuffer.allocate(1024); + final int totalOperations = totalOperations(); + channelReference.incRef(); + return newReaderSnapshot(totalOperations, reusableBuffer); + } + + /** + * reads an operation at the given position and returns it. The buffer length is equal to the number + * of bytes reads. + */ + private final BufferedChecksumStreamInput checksummedStream(ByteBuffer reusableBuffer, long position, int opSize, BufferedChecksumStreamInput reuse) throws IOException { + final ByteBuffer buffer; + if (reusableBuffer.capacity() >= opSize) { + buffer = reusableBuffer; + } else { + buffer = ByteBuffer.allocate(opSize); + } + buffer.clear(); + buffer.limit(opSize); + readBytes(buffer, position); + buffer.flip(); + return new BufferedChecksumStreamInput(new ByteBufferStreamInput(buffer), reuse); + } + + protected Translog.Operation read(BufferedChecksumStreamInput inStream) throws IOException { + return Translog.readOperation(inStream); + } + + /** + * reads bytes at position into the given buffer, filling it. + */ + abstract protected void readBytes(ByteBuffer buffer, long position) throws IOException; + + @Override + public void close() throws IOException { + if (closed.compareAndSet(false, true)) { + doClose(); + } + } + + protected void doClose() throws IOException { + channelReference.decRef(); + } + + protected void ensureOpen() { + if (closed.get()) { + throw new AlreadyClosedException("translog [" + getGeneration() + "] is already closed"); + } + } + + @Override + public String toString() { + return "translog [" + generation + "][" + channelReference.getPath() + "]"; + } + + @Override + public int compareTo(TranslogReader o) { + return Long.compare(getGeneration(), o.getGeneration()); + } + + + /** + * Given a file, return a VersionedTranslogStream based on an + * optionally-existing header in the file. If the file does not exist, or + * has zero length, returns the latest version. If the header does not + * exist, assumes Version 0 of the translog file format. + *

+ * + * @throws IOException + */ + public static ImmutableTranslogReader open(ChannelReference channelReference, Checkpoint checkpoint, String translogUUID) throws IOException { + final FileChannel channel = channelReference.getChannel(); + final Path path = channelReference.getPath(); + assert channelReference.getGeneration() == checkpoint.generation : "expected generation: " + channelReference.getGeneration() + " but got: " + checkpoint.generation; + + try { + if (checkpoint.offset == 0 && checkpoint.numOps == TranslogReader.UNKNOWN_OP_COUNT) { // only old files can be empty + return new LegacyTranslogReader(channelReference.getGeneration(), channelReference, 0); + } + + InputStreamStreamInput headerStream = new InputStreamStreamInput(Channels.newInputStream(channel)); // don't close + // Lucene's CodecUtil writes a magic number of 0x3FD76C17 with the + // header, in binary this looks like: + // + // binary: 0011 1111 1101 0111 0110 1100 0001 0111 + // hex : 3 f d 7 6 c 1 7 + // + // With version 0 of the translog, the first byte is the + // Operation.Type, which will always be between 0-4, so we know if + // we grab the first byte, it can be: + // 0x3f => Lucene's magic number, so we can assume it's version 1 or later + // 0x00 => version 0 of the translog + // + // otherwise the first byte of the translog is corrupted and we + // should bail + byte b1 = headerStream.readByte(); + if (b1 == LUCENE_CODEC_HEADER_BYTE) { + // Read 3 more bytes, meaning a whole integer has been read + byte b2 = headerStream.readByte(); + byte b3 = headerStream.readByte(); + byte b4 = headerStream.readByte(); + // Convert the 4 bytes that were read into an integer + int header = ((b1 & 0xFF) << 24) + ((b2 & 0xFF) << 16) + ((b3 & 0xFF) << 8) + ((b4 & 0xFF) << 0); + // We confirm CodecUtil's CODEC_MAGIC number (0x3FD76C17) + // ourselves here, because it allows us to read the first + // byte separately + if (header != CodecUtil.CODEC_MAGIC) { + throw new TranslogCorruptedException("translog looks like version 1 or later, but has corrupted header"); + } + // Confirm the rest of the header using CodecUtil, extracting + // the translog version + int version = CodecUtil.checkHeaderNoMagic(new InputStreamDataInput(headerStream), TranslogWriter.TRANSLOG_CODEC, 1, Integer.MAX_VALUE); + switch (version) { + case TranslogWriter.VERSION_CHECKSUMS: + assert checkpoint.numOps == TranslogReader.UNKNOWN_OP_COUNT : "expected unknown op count but got: " + checkpoint.numOps; + assert checkpoint.offset == Files.size(path) : "offset(" + checkpoint.offset + ") != file_size(" + Files.size(path) + ") for: " + path; + // legacy - we still have to support it somehow + return new LegacyTranslogReaderBase(channelReference.getGeneration(), channelReference, CodecUtil.headerLength(TranslogWriter.TRANSLOG_CODEC), checkpoint.offset); + case TranslogWriter.VERSION_CHECKPOINTS: + assert path.getFileName().toString().endsWith(Translog.TRANSLOG_FILE_SUFFIX) : "new file ends with old suffix: " + path; + assert checkpoint.numOps > TranslogReader.UNKNOWN_OP_COUNT: "expected at least 0 operatin but got: " + checkpoint.numOps; + assert checkpoint.offset <= channel.size() : "checkpoint is inconsistent with channel length: " + channel.size() + " " + checkpoint; + int len = headerStream.readInt(); + if (len > channel.size()) { + throw new TranslogCorruptedException("uuid length can't be larger than the translog"); + } + BytesRef ref = new BytesRef(len); + ref.length = len; + headerStream.read(ref.bytes, ref.offset, ref.length); + if (ref.utf8ToString().equals(translogUUID) == false) { + throw new TranslogCorruptedException("expected shard UUID [" + translogUUID + "] but got: [" + ref.utf8ToString() + "] this translog file belongs to a different translog"); + } + return new ImmutableTranslogReader(channelReference.getGeneration(), channelReference, ref.length + CodecUtil.headerLength(TranslogWriter.TRANSLOG_CODEC) + RamUsageEstimator.NUM_BYTES_INT, checkpoint.offset, checkpoint.numOps); + default: + throw new TranslogCorruptedException("No known translog stream version: " + version + " path:" + path); + } + } else if (b1 == UNVERSIONED_TRANSLOG_HEADER_BYTE) { + assert checkpoint.numOps == TranslogReader.UNKNOWN_OP_COUNT : "expected unknown op count but got: " + checkpoint.numOps; + assert checkpoint.offset == Files.size(path) : "offset(" + checkpoint.offset + ") != file_size(" + Files.size(path) + ") for: " + path; + return new LegacyTranslogReader(channelReference.getGeneration(), channelReference, checkpoint.offset); + } else { + throw new TranslogCorruptedException("Invalid first byte in translog file, got: " + Long.toHexString(b1) + ", expected 0x00 or 0x3f"); + } + } catch (CorruptIndexException | IndexFormatTooOldException | IndexFormatTooNewException e) { + throw new TranslogCorruptedException("Translog header corrupted", e); + } + } + + public Path path() { + return channelReference.getPath(); + } + + protected Translog.Snapshot newReaderSnapshot(int totalOperations, ByteBuffer reusableBuffer) { + return new ReaderSnapshot(totalOperations, reusableBuffer); + } + + class ReaderSnapshot implements Translog.Snapshot { + private final AtomicBoolean closed; + private final int totalOperations; + private final ByteBuffer reusableBuffer; + long position; + int readOperations; + private BufferedChecksumStreamInput reuse; + + public ReaderSnapshot(int totalOperations, ByteBuffer reusableBuffer) { + this.totalOperations = totalOperations; + this.reusableBuffer = reusableBuffer; + closed = new AtomicBoolean(false); + position = firstOperationOffset; + readOperations = 0; + reuse = null; + } + + @Override + public final int estimatedTotalOperations() { + return totalOperations; + } + + @Override + public Translog.Operation next() throws IOException { + if (readOperations < totalOperations) { + assert readOperations < totalOperations : "readOpeartions must be less than totalOperations"; + return readOperation(); + } else { + return null; + } + } + + protected final Translog.Operation readOperation() throws IOException { + final int opSize = readSize(reusableBuffer, position); + reuse = checksummedStream(reusableBuffer, position, opSize, reuse); + Translog.Operation op = read(reuse); + position += opSize; + readOperations++; + return op; + } + + @Override + public void close() { + if (closed.compareAndSet(false, true)) { + channelReference.decRef(); + } + } + } +} diff --git a/src/main/java/org/elasticsearch/index/translog/TranslogSnapshot.java b/src/main/java/org/elasticsearch/index/translog/TranslogSnapshot.java index 9c0f5561143..dfd69eddf4b 100644 --- a/src/main/java/org/elasticsearch/index/translog/TranslogSnapshot.java +++ b/src/main/java/org/elasticsearch/index/translog/TranslogSnapshot.java @@ -22,6 +22,8 @@ package org.elasticsearch.index.translog; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.util.IOUtils; import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.logging.ESLogger; import java.io.IOException; @@ -31,9 +33,7 @@ import java.util.concurrent.atomic.AtomicBoolean; public class TranslogSnapshot implements Translog.Snapshot { - private final List orderedTranslogs; - private final ESLogger logger; - private final ByteBuffer cacheBuffer; + private final List orderedTranslogs; private AtomicBoolean closed = new AtomicBoolean(false); private final int estimatedTotalOperations; private int currentTranslog; @@ -42,21 +42,19 @@ public class TranslogSnapshot implements Translog.Snapshot { * Create a snapshot of translog file channel. The length parameter should be consistent with totalOperations and point * at the end of the last operation in this snapshot. */ - public TranslogSnapshot(List orderedTranslogs, ESLogger logger) { + public TranslogSnapshot(List orderedTranslogs) { this.orderedTranslogs = orderedTranslogs; - this.logger = logger; int ops = 0; - for (ChannelSnapshot translog : orderedTranslogs) { + for (Translog.Snapshot translog : orderedTranslogs) { final int tops = translog.estimatedTotalOperations(); if (tops < 0) { - ops = ChannelReader.UNKNOWN_OP_COUNT; + ops = TranslogReader.UNKNOWN_OP_COUNT; break; } ops += tops; } estimatedTotalOperations = ops; - cacheBuffer = ByteBuffer.allocate(1024); currentTranslog = 0; } @@ -70,13 +68,17 @@ public class TranslogSnapshot implements Translog.Snapshot { public Translog.Operation next() throws IOException { ensureOpen(); for (; currentTranslog < orderedTranslogs.size(); currentTranslog++) { - final ChannelSnapshot current = orderedTranslogs.get(currentTranslog); + final Translog.Snapshot current = orderedTranslogs.get(currentTranslog); Translog.Operation op = null; try { - op = current.next(cacheBuffer); + op = current.next(); } catch (TruncatedTranslogException e) { - // file is empty or header has been half-written and should be ignored - logger.trace("ignoring truncation exception, the translog [{}] is either empty or half-written", e, current.translogId()); + if (estimatedTotalOperations == TranslogReader.UNKNOWN_OP_COUNT) { + // legacy translog file - can have UNKNOWN_OP_COUNT + // file is empty or header has been half-written and should be ignored + } else { + throw e; + } } if (op != null) { return op; @@ -94,13 +96,7 @@ public class TranslogSnapshot implements Translog.Snapshot { @Override public void close() throws ElasticsearchException { if (closed.compareAndSet(false, true)) { - try { - IOUtils.close(orderedTranslogs); - } catch (IOException e) { - throw new ElasticsearchException("failed to close channel snapshots", e); - } finally { - orderedTranslogs.clear(); - } + Releasables.close(orderedTranslogs); } } } diff --git a/src/main/java/org/elasticsearch/index/translog/TranslogStream.java b/src/main/java/org/elasticsearch/index/translog/TranslogStream.java deleted file mode 100644 index 061a12ff445..00000000000 --- a/src/main/java/org/elasticsearch/index/translog/TranslogStream.java +++ /dev/null @@ -1,61 +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.translog; - -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; - -import java.io.IOException; -import java.nio.channels.FileChannel; -import java.nio.file.Path; - -/** - * A translog stream that will read and write operations in the - * version-specific format - */ -public interface TranslogStream { - - /** - * Read the next translog operation from the input stream - */ - public Translog.Operation read(StreamInput in) throws IOException; - - /** - * Write the given translog operation to the output stream - */ - public void write(StreamOutput out, Translog.Operation op) throws IOException; - - /** - * Optionally write a header identifying the translog version to the - * file channel - */ - public int writeHeader(FileChannel channel) throws IOException; - - /** - * returns the site of the header in bytes - */ - public int headerLength(); - - /** - * Seek past the header, if any header is present - */ - public StreamInput openInput(Path translogFile) throws IOException; - -} diff --git a/src/main/java/org/elasticsearch/index/translog/TranslogStreams.java b/src/main/java/org/elasticsearch/index/translog/TranslogStreams.java deleted file mode 100644 index 345c027e161..00000000000 --- a/src/main/java/org/elasticsearch/index/translog/TranslogStreams.java +++ /dev/null @@ -1,149 +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.translog; - -import org.apache.lucene.codecs.CodecUtil; -import org.apache.lucene.index.CorruptIndexException; -import org.apache.lucene.index.IndexFormatTooNewException; -import org.apache.lucene.index.IndexFormatTooOldException; -import org.apache.lucene.store.InputStreamDataInput; -import org.elasticsearch.common.io.stream.InputStreamStreamInput; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; - -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; - -/** - * Encapsulating class used for operating on translog streams. Static methods - * on this class use the latest version of the stream. - */ -public class TranslogStreams { - - /** V0, no header, no checksums */ - public static TranslogStream LEGACY_TRANSLOG_STREAM = new LegacyTranslogStream(); - /** V1, header, with per-op checksums */ - public static TranslogStream CHECKSUMMED_TRANSLOG_STREAM = new ChecksummedTranslogStream(); - - public static TranslogStream LATEST = CHECKSUMMED_TRANSLOG_STREAM; - - public static final String TRANSLOG_CODEC = "translog"; - private static final byte LUCENE_CODEC_HEADER_BYTE = 0x3f; - private static final byte UNVERSIONED_TRANSLOG_HEADER_BYTE = 0x00; - - /** - * Returns a new empty translog operation for the given {@link Translog.Operation.Type} - */ - static Translog.Operation newOperationFromType(Translog.Operation.Type type) throws IOException { - switch (type) { - case CREATE: - return new Translog.Create(); - case DELETE: - return new Translog.Delete(); - case DELETE_BY_QUERY: - return new Translog.DeleteByQuery(); - case SAVE: - return new Translog.Index(); - default: - throw new IOException("No type for [" + type + "]"); - } - } - - /** - * Read the next {@link Translog.Operation} from the stream using the - * latest translog version - */ - public static Translog.Operation readTranslogOperation(StreamInput in) throws IOException { - return LATEST.read(in); - } - - /** - * Write the {@link Translog.Operation} to the output stream using the - * latest translog version - */ - public static void writeTranslogOperation(StreamOutput out, Translog.Operation op) throws IOException { - LATEST.write(out, op); - } - - /** - * Given a file, return a VersionedTranslogStream based on an - * optionally-existing header in the file. If the file does not exist, or - * has zero length, returns the latest version. If the header does not - * exist, assumes Version 0 of the translog file format. - *

- * - * @throws IOException - */ - public static TranslogStream translogStreamFor(Path translogFile) throws IOException { - - try (InputStreamStreamInput headerStream = new InputStreamStreamInput(Files.newInputStream(translogFile))) { - if (Files.exists(translogFile) == false || Files.size(translogFile) == 0) { - // if it doesn't exist or has no data, use the latest version, - // there aren't any backwards compatibility issues - return CHECKSUMMED_TRANSLOG_STREAM; - } - // Lucene's CodecUtil writes a magic number of 0x3FD76C17 with the - // header, in binary this looks like: - // - // binary: 0011 1111 1101 0111 0110 1100 0001 0111 - // hex : 3 f d 7 6 c 1 7 - // - // With version 0 of the translog, the first byte is the - // Operation.Type, which will always be between 0-4, so we know if - // we grab the first byte, it can be: - // 0x3f => Lucene's magic number, so we can assume it's version 1 or later - // 0x00 => version 0 of the translog - // - // otherwise the first byte of the translog is corrupted and we - // should bail - byte b1 = headerStream.readByte(); - if (b1 == LUCENE_CODEC_HEADER_BYTE) { - // Read 3 more bytes, meaning a whole integer has been read - byte b2 = headerStream.readByte(); - byte b3 = headerStream.readByte(); - byte b4 = headerStream.readByte(); - // Convert the 4 bytes that were read into an integer - int header = ((b1 & 0xFF) << 24) + ((b2 & 0xFF) << 16) + ((b3 & 0xFF) << 8) + ((b4 & 0xFF) << 0); - // We confirm CodecUtil's CODEC_MAGIC number (0x3FD76C17) - // ourselves here, because it allows us to read the first - // byte separately - if (header != CodecUtil.CODEC_MAGIC) { - throw new TranslogCorruptedException("translog looks like version 1 or later, but has corrupted header"); - } - // Confirm the rest of the header using CodecUtil, extracting - // the translog version - int version = CodecUtil.checkHeaderNoMagic(new InputStreamDataInput(headerStream), TRANSLOG_CODEC, 1, Integer.MAX_VALUE); - switch (version) { - case ChecksummedTranslogStream.VERSION: - return CHECKSUMMED_TRANSLOG_STREAM; - default: - throw new TranslogCorruptedException("No known translog stream version: " + version); - } - } else if (b1 == UNVERSIONED_TRANSLOG_HEADER_BYTE) { - return LEGACY_TRANSLOG_STREAM; - } else { - throw new TranslogCorruptedException("Invalid first byte in translog file, got: " + Long.toHexString(b1) + ", expected 0x00 or 0x3f"); - } - } catch (CorruptIndexException | IndexFormatTooOldException | IndexFormatTooNewException e) { - throw new TranslogCorruptedException("Translog header corrupted", e); - } - } -} diff --git a/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java b/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java new file mode 100644 index 00000000000..df1beef0870 --- /dev/null +++ b/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java @@ -0,0 +1,301 @@ +/* + * 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.translog; + +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.store.OutputStreamDataOutput; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.RamUsageEstimator; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.io.Channels; +import org.elasticsearch.common.util.Callback; +import org.elasticsearch.common.util.concurrent.ReleasableLock; +import org.elasticsearch.index.shard.ShardId; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.file.*; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +public class TranslogWriter extends TranslogReader { + + public static final String TRANSLOG_CODEC = "translog"; + public static final int VERSION_CHECKSUMS = 1; + public static final int VERSION_CHECKPOINTS = 2; // since 2.0 we have checkpoints? + public static final int VERSION = VERSION_CHECKPOINTS; + + protected final ShardId shardId; + protected final ReleasableLock readLock; + protected final ReleasableLock writeLock; + /* the offset in bytes that was written when the file was last synced*/ + protected volatile long lastSyncedOffset; + /* the number of translog operations written to this file */ + protected volatile int operationCounter; + /* the offset in bytes written to the file */ + protected volatile long writtenOffset; + + public TranslogWriter(ShardId shardId, long generation, ChannelReference channelReference) throws IOException { + super(generation, channelReference, channelReference.getChannel().position()); + this.shardId = shardId; + ReadWriteLock rwl = new ReentrantReadWriteLock(); + readLock = new ReleasableLock(rwl.readLock()); + writeLock = new ReleasableLock(rwl.writeLock()); + this.writtenOffset = channelReference.getChannel().position(); + this.lastSyncedOffset = channelReference.getChannel().position();; + } + + public static TranslogWriter create(Type type, ShardId shardId, String translogUUID, long fileGeneration, Path file, Callback onClose, int bufferSize) throws IOException { + Path pendingFile = file.resolveSibling("pending_" + file.getFileName()); + final BytesRef ref = new BytesRef(translogUUID); + final int headerLength = CodecUtil.headerLength(TRANSLOG_CODEC) + ref.length + RamUsageEstimator.NUM_BYTES_INT; + /** + * We first create pending_translog, write the header, fsync it and write a checkpoint. Then we rename the pending file into + * the actual file such that there is never a file without valid header. If the header is missing it's corrupted + */ + try (FileChannel channel = FileChannel.open(pendingFile, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW)) { + // This OutputStreamDataOutput is intentionally not closed because + // closing it will close the FileChannel + OutputStreamDataOutput out = new OutputStreamDataOutput(java.nio.channels.Channels.newOutputStream(channel)); + CodecUtil.writeHeader(out, TRANSLOG_CODEC, VERSION); + out.writeInt(ref.length); + out.writeBytes(ref.bytes, ref.offset, ref.length); + channel.force(false); + writeCheckpoint(headerLength, 0, file.getParent(), fileGeneration, StandardOpenOption.WRITE); + } + Files.move(pendingFile, file, StandardCopyOption.ATOMIC_MOVE); + FileChannel channel = FileChannel.open(file, StandardOpenOption.READ, StandardOpenOption.WRITE); + try { + channel.position(headerLength); + final TranslogWriter writer = type.create(shardId, fileGeneration, new ChannelReference(file, fileGeneration, channel, onClose), bufferSize); + return writer; + } catch (Throwable throwable){ + IOUtils.closeWhileHandlingException(channel); + try { + Files.delete(file); // remove the file as well + } catch (IOException ex) { + throwable.addSuppressed(ex); + } + throw throwable; + } + } + + public enum Type { + + SIMPLE() { + @Override + public TranslogWriter create(ShardId shardId, long generation, ChannelReference channelReference, int bufferSize) throws IOException { + return new TranslogWriter(shardId, generation, channelReference); + } + }, + BUFFERED() { + @Override + public TranslogWriter create(ShardId shardId, long generation, ChannelReference channelReference, int bufferSize) throws IOException { + return new BufferingTranslogWriter(shardId, generation, channelReference, bufferSize); + } + }; + + public abstract TranslogWriter create(ShardId shardId, long generation, ChannelReference raf, int bufferSize) throws IOException; + + public static Type fromString(String type) { + if (SIMPLE.name().equalsIgnoreCase(type)) { + return SIMPLE; + } else if (BUFFERED.name().equalsIgnoreCase(type)) { + return BUFFERED; + } + throw new IllegalArgumentException("No translog fs type [" + type + "]"); + } + } + + + /** + * add the given bytes to the translog and return the location they were written at + */ + public Translog.Location add(BytesReference data) throws IOException { + ensureOpen(); + final long position; + try (ReleasableLock lock = writeLock.acquire()) { + position = writtenOffset; + data.writeTo(channel); + writtenOffset = writtenOffset + data.length(); + operationCounter = operationCounter + 1; + } + return new Translog.Location(generation, position, data.length()); + } + + /** + * change the size of the internal buffer if relevant + */ + public void updateBufferSize(int bufferSize) throws TranslogException { + } + + /** + * write all buffered ops to disk and fsync file + */ + public void sync() throws IOException { + // check if we really need to sync here... + if (syncNeeded()) { + try (ReleasableLock lock = writeLock.acquire()) { + lastSyncedOffset = writtenOffset; + checkpoint(lastSyncedOffset, operationCounter, channelReference); + } + } + } + + /** + * returns true if there are buffered ops + */ + public boolean syncNeeded() { + return writtenOffset != lastSyncedOffset; // by default nothing is buffered + } + + @Override + public int totalOperations() { + return operationCounter; + } + + @Override + public long sizeInBytes() { + return writtenOffset; + } + + + /** + * Flushes the buffer if the translog is buffered. + */ + protected void flush() throws IOException { + } + + /** + * returns a new reader that follows the current writes (most importantly allows making + * repeated snapshots that includes new content) + */ + public TranslogReader newReaderFromWriter() { + ensureOpen(); + channelReference.incRef(); + boolean success = false; + try { + TranslogReader reader = new InnerReader(this.generation, firstOperationOffset, channelReference); + success = true; + return reader; + } finally { + if (!success) { + channelReference.decRef(); + } + } + } + + + /** + * returns a new immutable reader which only exposes the current written operation * + */ + public ImmutableTranslogReader immutableReader() throws TranslogException { + ensureOpen(); + if (channelReference.tryIncRef()) { + try (ReleasableLock lock = writeLock.acquire()) { + flush(); + ImmutableTranslogReader reader = new ImmutableTranslogReader(this.generation, channelReference, firstOperationOffset, writtenOffset, operationCounter); + channelReference.incRef(); // for new reader + return reader; + } catch (Exception e) { + throw new TranslogException(shardId, "exception while creating an immutable reader", e); + } finally { + channelReference.decRef(); + } + } else { + throw new TranslogException(shardId, "can't increment channel [" + channelReference + "] ref count"); + } + } + + boolean assertBytesAtLocation(Translog.Location location, BytesReference expectedBytes) throws IOException { + ByteBuffer buffer = ByteBuffer.allocate(location.size); + readBytes(buffer, location.translogLocation); + return new BytesArray(buffer.array()).equals(expectedBytes); + } + + /** + * this class is used when one wants a reference to this file which exposes all recently written operation. + * as such it needs access to the internals of the current reader + */ + final class InnerReader extends TranslogReader { + + public InnerReader(long generation, long fistOperationOffset, ChannelReference channelReference) { + super(generation, channelReference, fistOperationOffset); + } + + @Override + public long sizeInBytes() { + return TranslogWriter.this.sizeInBytes(); + } + + @Override + public int totalOperations() { + return TranslogWriter.this.totalOperations(); + } + + @Override + protected void readBytes(ByteBuffer buffer, long position) throws IOException { + TranslogWriter.this.readBytes(buffer, position); + } + } + + /** + * Syncs the translog up to at least the given offset unless already synced + * + * @return true if this call caused an actual sync operation + */ + public boolean syncUpTo(long offset) throws IOException { + if (lastSyncedOffset < offset) { + sync(); + return true; + } + return false; + } + + @Override + protected final void doClose() throws IOException { + try (ReleasableLock lock = writeLock.acquire()) { + sync(); + } finally { + super.doClose(); + } + } + + @Override + protected void readBytes(ByteBuffer buffer, long position) throws IOException { + try (ReleasableLock lock = readLock.acquire()) { + Channels.readFromFileChannelWithEofException(channel, position, buffer); + } + } + + protected synchronized void checkpoint(long lastSyncPosition, int operationCounter, ChannelReference channelReference) throws IOException { + channelReference.getChannel().force(false); + writeCheckpoint(lastSyncPosition, operationCounter, channelReference.getPath().getParent(), channelReference.getGeneration(), StandardOpenOption.WRITE); + } + + private static void writeCheckpoint(long syncPosition, int numOperations, Path translogFile, long generation, OpenOption... options) throws IOException { + final Path checkpointFile = translogFile.resolve(Translog.CHECKPOINT_FILE_NAME); + Checkpoint checkpoint = new Checkpoint(syncPosition, numOperations, generation); + Checkpoint.write(checkpointFile, checkpoint, options); + } +} diff --git a/src/main/java/org/elasticsearch/index/translog/TruncatedTranslogException.java b/src/main/java/org/elasticsearch/index/translog/TruncatedTranslogException.java index 0464bcf9c0d..7338b4e356a 100644 --- a/src/main/java/org/elasticsearch/index/translog/TruncatedTranslogException.java +++ b/src/main/java/org/elasticsearch/index/translog/TruncatedTranslogException.java @@ -19,12 +19,7 @@ package org.elasticsearch.index.translog; -import java.io.IOException; - -public class TruncatedTranslogException extends IOException { - public TruncatedTranslogException(String msg) { - super(msg); - } +public class TruncatedTranslogException extends TranslogCorruptedException { public TruncatedTranslogException(String msg, Throwable cause) { super(msg, cause); diff --git a/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java b/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java index 426f4cf89d8..b70a21c9df9 100644 --- a/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java +++ b/src/main/java/org/elasticsearch/indices/memory/IndexingMemoryController.java @@ -210,8 +210,8 @@ public class IndexingMemoryController extends AbstractLifecycleComponent implem Transports.assertTransportThread(); try { transportServiceAdapter.received(data.length); - StreamInput stream = new BytesStreamInput(data); + StreamInput stream = StreamInput.wrap(data); stream.setVersion(version); long requestId = stream.readLong(); diff --git a/src/test/java/org/elasticsearch/ElasticsearchExceptionTests.java b/src/test/java/org/elasticsearch/ElasticsearchExceptionTests.java index 16914ab6eef..b00c76ded6f 100644 --- a/src/test/java/org/elasticsearch/ElasticsearchExceptionTests.java +++ b/src/test/java/org/elasticsearch/ElasticsearchExceptionTests.java @@ -21,8 +21,8 @@ package org.elasticsearch; import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.ShardSearchFailure; -import org.elasticsearch.common.io.stream.BytesStreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; @@ -225,7 +225,7 @@ public class ElasticsearchExceptionTests extends ElasticsearchTestCase { QueryParsingException ex = new TestQueryParsingException(new Index("foo"), 1, 2, "foobar", null); out.writeThrowable(ex); - BytesStreamInput in = new BytesStreamInput(out.bytes()); + StreamInput in = StreamInput.wrap(out.bytes()); QueryParsingException e = in.readThrowable(); assertEquals(ex.index(), e.index()); assertEquals(ex.getMessage(), e.getMessage()); diff --git a/src/test/java/org/elasticsearch/action/OriginalIndicesTests.java b/src/test/java/org/elasticsearch/action/OriginalIndicesTests.java index 6ff8bb5a118..83411fcd40e 100644 --- a/src/test/java/org/elasticsearch/action/OriginalIndicesTests.java +++ b/src/test/java/org/elasticsearch/action/OriginalIndicesTests.java @@ -20,8 +20,8 @@ package org.elasticsearch.action; import org.elasticsearch.action.support.IndicesOptions; -import org.elasticsearch.common.io.stream.BytesStreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.test.ElasticsearchTestCase; import org.junit.Test; @@ -46,7 +46,7 @@ public class OriginalIndicesTests extends ElasticsearchTestCase { out.setVersion(randomVersion(random())); OriginalIndices.writeOriginalIndices(originalIndices, out); - BytesStreamInput in = new BytesStreamInput(out.bytes()); + StreamInput in = StreamInput.wrap(out.bytes()); in.setVersion(out.getVersion()); OriginalIndices originalIndices2 = OriginalIndices.readOriginalIndices(in); diff --git a/src/test/java/org/elasticsearch/action/admin/cluster/state/ClusterStateRequestTest.java b/src/test/java/org/elasticsearch/action/admin/cluster/state/ClusterStateRequestTest.java index fca09aa861d..dd66049adfc 100644 --- a/src/test/java/org/elasticsearch/action/admin/cluster/state/ClusterStateRequestTest.java +++ b/src/test/java/org/elasticsearch/action/admin/cluster/state/ClusterStateRequestTest.java @@ -21,7 +21,7 @@ package org.elasticsearch.action.admin.cluster.state; import org.elasticsearch.Version; import org.elasticsearch.action.support.IndicesOptions; -import org.elasticsearch.common.io.stream.BytesStreamInput; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.test.ElasticsearchTestCase; import org.elasticsearch.test.VersionUtils; @@ -48,10 +48,10 @@ public class ClusterStateRequestTest extends ElasticsearchTestCase { output.setVersion(testVersion); clusterStateRequest.writeTo(output); - BytesStreamInput bytesStreamInput = new BytesStreamInput(output.bytes()); - bytesStreamInput.setVersion(testVersion); + StreamInput streamInput = StreamInput.wrap(output.bytes()); + streamInput.setVersion(testVersion); ClusterStateRequest deserializedCSRequest = new ClusterStateRequest(); - deserializedCSRequest.readFrom(bytesStreamInput); + deserializedCSRequest.readFrom(streamInput); assertThat(deserializedCSRequest.routingTable(), equalTo(clusterStateRequest.routingTable())); assertThat(deserializedCSRequest.metaData(), equalTo(clusterStateRequest.metaData())); diff --git a/src/test/java/org/elasticsearch/action/admin/indices/stats/IndicesStatsTests.java b/src/test/java/org/elasticsearch/action/admin/indices/stats/IndicesStatsTests.java index c8bcb27d038..a8c40a8ce4e 100644 --- a/src/test/java/org/elasticsearch/action/admin/indices/stats/IndicesStatsTests.java +++ b/src/test/java/org/elasticsearch/action/admin/indices/stats/IndicesStatsTests.java @@ -93,7 +93,8 @@ public class IndicesStatsTests extends ElasticsearchSingleNodeTest { assertNotNull(commitStats); assertThat(commitStats.getGeneration(), greaterThan(0l)); assertThat(commitStats.getId(), notNullValue()); - assertThat(commitStats.getUserData(), hasKey(Translog.TRANSLOG_ID_KEY)); + assertThat(commitStats.getUserData(), hasKey(Translog.TRANSLOG_GENERATION_KEY)); + assertThat(commitStats.getUserData(), hasKey(Translog.TRANSLOG_UUID_KEY)); } } diff --git a/src/test/java/org/elasticsearch/action/get/MultiGetShardRequestTests.java b/src/test/java/org/elasticsearch/action/get/MultiGetShardRequestTests.java index 24d7ac410ab..073d2596636 100644 --- a/src/test/java/org/elasticsearch/action/get/MultiGetShardRequestTests.java +++ b/src/test/java/org/elasticsearch/action/get/MultiGetShardRequestTests.java @@ -19,8 +19,8 @@ package org.elasticsearch.action.get; -import org.elasticsearch.common.io.stream.BytesStreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.index.VersionType; import org.elasticsearch.search.fetch.source.FetchSourceContext; import org.elasticsearch.test.ElasticsearchTestCase; @@ -73,7 +73,7 @@ public class MultiGetShardRequestTests extends ElasticsearchTestCase { out.setVersion(randomVersion(random())); multiGetShardRequest.writeTo(out); - BytesStreamInput in = new BytesStreamInput(out.bytes()); + StreamInput in = StreamInput.wrap(out.bytes()); in.setVersion(out.getVersion()); MultiGetShardRequest multiGetShardRequest2 = new MultiGetShardRequest(); multiGetShardRequest2.readFrom(in); diff --git a/src/test/java/org/elasticsearch/action/indexedscripts/get/GetIndexedScriptRequestTests.java b/src/test/java/org/elasticsearch/action/indexedscripts/get/GetIndexedScriptRequestTests.java index f940e055550..3ff5cf93922 100644 --- a/src/test/java/org/elasticsearch/action/indexedscripts/get/GetIndexedScriptRequestTests.java +++ b/src/test/java/org/elasticsearch/action/indexedscripts/get/GetIndexedScriptRequestTests.java @@ -19,8 +19,8 @@ package org.elasticsearch.action.indexedscripts.get; -import org.elasticsearch.common.io.stream.BytesStreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.index.VersionType; import org.elasticsearch.test.ElasticsearchTestCase; import org.junit.Test; @@ -44,7 +44,7 @@ public class GetIndexedScriptRequestTests extends ElasticsearchTestCase { out.setVersion(randomVersion(random())); request.writeTo(out); - BytesStreamInput in = new BytesStreamInput(out.bytes()); + StreamInput in = StreamInput.wrap(out.bytes()); in.setVersion(out.getVersion()); GetIndexedScriptRequest request2 = new GetIndexedScriptRequest(); request2.readFrom(in); diff --git a/src/test/java/org/elasticsearch/action/support/IndicesOptionsTests.java b/src/test/java/org/elasticsearch/action/support/IndicesOptionsTests.java index e757f9ca4ac..13657fb06da 100644 --- a/src/test/java/org/elasticsearch/action/support/IndicesOptionsTests.java +++ b/src/test/java/org/elasticsearch/action/support/IndicesOptionsTests.java @@ -20,8 +20,8 @@ package org.elasticsearch.action.support; import org.elasticsearch.Version; -import org.elasticsearch.common.io.stream.BytesStreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.test.ElasticsearchTestCase; import org.junit.Test; @@ -41,9 +41,9 @@ public class IndicesOptionsTests extends ElasticsearchTestCase { output.setVersion(outputVersion); indicesOptions.writeIndicesOptions(output); - BytesStreamInput bytesStreamInput = new BytesStreamInput(output.bytes()); - bytesStreamInput.setVersion(randomVersion(random())); - IndicesOptions indicesOptions2 = IndicesOptions.readIndicesOptions(bytesStreamInput); + StreamInput streamInput = StreamInput.wrap(output.bytes()); + streamInput.setVersion(randomVersion(random())); + IndicesOptions indicesOptions2 = IndicesOptions.readIndicesOptions(streamInput); assertThat(indicesOptions2.ignoreUnavailable(), equalTo(indicesOptions.ignoreUnavailable())); assertThat(indicesOptions2.allowNoIndices(), equalTo(indicesOptions.allowNoIndices())); diff --git a/src/test/java/org/elasticsearch/cluster/ClusterHealthResponsesTests.java b/src/test/java/org/elasticsearch/cluster/ClusterHealthResponsesTests.java index 41794aa0c0b..50ad43e2bb7 100644 --- a/src/test/java/org/elasticsearch/cluster/ClusterHealthResponsesTests.java +++ b/src/test/java/org/elasticsearch/cluster/ClusterHealthResponsesTests.java @@ -29,8 +29,8 @@ import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.routing.*; -import org.elasticsearch.common.io.stream.BytesStreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ElasticsearchTestCase; import org.hamcrest.Matchers; @@ -206,7 +206,7 @@ public class ClusterHealthResponsesTests extends ElasticsearchTestCase { if (randomBoolean()) { BytesStreamOutput out = new BytesStreamOutput(); clusterHealth.writeTo(out); - BytesStreamInput in = new BytesStreamInput(out.bytes()); + StreamInput in = StreamInput.wrap(out.bytes()); clusterHealth = ClusterHealthResponse.readResponseFrom(in); } return clusterHealth; diff --git a/src/test/java/org/elasticsearch/cluster/ClusterStateDiffTests.java b/src/test/java/org/elasticsearch/cluster/ClusterStateDiffTests.java index 4832e380abb..86a2a06c719 100644 --- a/src/test/java/org/elasticsearch/cluster/ClusterStateDiffTests.java +++ b/src/test/java/org/elasticsearch/cluster/ClusterStateDiffTests.java @@ -32,8 +32,8 @@ import org.elasticsearch.cluster.routing.*; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.collect.ImmutableOpenMap; -import org.elasticsearch.common.io.stream.BytesStreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.LocalTransportAddress; @@ -105,7 +105,7 @@ public class ClusterStateDiffTests extends ElasticsearchIntegrationTest { diffBeforeSerialization.writeTo(os); byte[] diffBytes = os.bytes().toBytes(); Diff diff; - try (BytesStreamInput input = new BytesStreamInput(diffBytes)) { + try (StreamInput input = StreamInput.wrap(diffBytes)) { diff = previousClusterStateFromDiffs.readDiffFrom(input); clusterStateFromDiffs = diff.apply(previousClusterStateFromDiffs); } diff --git a/src/test/java/org/elasticsearch/cluster/block/ClusterBlockTests.java b/src/test/java/org/elasticsearch/cluster/block/ClusterBlockTests.java index efccb3f778c..659f552e0a2 100644 --- a/src/test/java/org/elasticsearch/cluster/block/ClusterBlockTests.java +++ b/src/test/java/org/elasticsearch/cluster/block/ClusterBlockTests.java @@ -20,8 +20,8 @@ package org.elasticsearch.cluster.block; import org.elasticsearch.Version; -import org.elasticsearch.common.io.stream.BytesStreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.ElasticsearchTestCase; import org.junit.Test; @@ -54,7 +54,7 @@ public class ClusterBlockTests extends ElasticsearchTestCase { out.setVersion(version); clusterBlock.writeTo(out); - BytesStreamInput in = new BytesStreamInput(out.bytes()); + StreamInput in = StreamInput.wrap(out.bytes()); in.setVersion(version); ClusterBlock result = ClusterBlock.readClusterBlock(in); diff --git a/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeTests.java b/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeTests.java index 1db6a4b894b..ff20115b20e 100644 --- a/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeTests.java +++ b/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeTests.java @@ -21,8 +21,8 @@ package org.elasticsearch.cluster.node; import org.elasticsearch.Version; import org.elasticsearch.common.io.ThrowableObjectInputStream; import org.elasticsearch.common.io.ThrowableObjectOutputStream; -import org.elasticsearch.common.io.stream.BytesStreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.test.ElasticsearchTestCase; @@ -57,7 +57,7 @@ public class DiscoveryNodeTests extends ElasticsearchTestCase { ThrowableObjectOutputStream too = new ThrowableObjectOutputStream(bytesOutput); too.writeObject(discoveryNode); too.close(); - ThrowableObjectInputStream from = new ThrowableObjectInputStream(new BytesStreamInput(bytesOutput.bytes())); + ThrowableObjectInputStream from = new ThrowableObjectInputStream(StreamInput.wrap(bytesOutput.bytes())); DiscoveryNode readDiscoveryNode = (DiscoveryNode) from.readObject(); from.close(); assertThat(readDiscoveryNode, Matchers.equalTo(discoveryNode)); diff --git a/src/test/java/org/elasticsearch/cluster/routing/allocation/AllocationCommandsTests.java b/src/test/java/org/elasticsearch/cluster/routing/allocation/AllocationCommandsTests.java index 7a1ffaecff6..62583a274cc 100644 --- a/src/test/java/org/elasticsearch/cluster/routing/allocation/AllocationCommandsTests.java +++ b/src/test/java/org/elasticsearch/cluster/routing/allocation/AllocationCommandsTests.java @@ -32,8 +32,8 @@ import org.elasticsearch.cluster.routing.allocation.command.AllocationCommands; import org.elasticsearch.cluster.routing.allocation.command.CancelAllocationCommand; import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand; import org.elasticsearch.cluster.routing.allocation.decider.DisableAllocationDecider; -import org.elasticsearch.common.io.stream.BytesStreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.xcontent.XContentFactory; @@ -345,7 +345,7 @@ public class AllocationCommandsTests extends ElasticsearchAllocationTestCase { ); BytesStreamOutput bytes = new BytesStreamOutput(); AllocationCommands.writeTo(commands, bytes); - AllocationCommands sCommands = AllocationCommands.readFrom(new BytesStreamInput(bytes.bytes())); + AllocationCommands sCommands = AllocationCommands.readFrom(StreamInput.wrap(bytes.bytes())); assertThat(sCommands.commands().size(), equalTo(3)); assertThat(((AllocateAllocationCommand) (sCommands.commands().get(0))).shardId(), equalTo(new ShardId("test", 1))); diff --git a/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java b/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java index 83a27850591..1accb6216c4 100644 --- a/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java +++ b/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java @@ -27,8 +27,8 @@ import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.allocation.AllocationService; -import org.elasticsearch.common.io.stream.BytesStreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.test.ElasticsearchAllocationTestCase; import org.junit.Test; @@ -82,7 +82,7 @@ public class ClusterSerializationTests extends ElasticsearchAllocationTestCase { BytesStreamOutput outStream = new BytesStreamOutput(); source.writeTo(outStream); - BytesStreamInput inStream = new BytesStreamInput(outStream.bytes().toBytes()); + StreamInput inStream = StreamInput.wrap(outStream.bytes().toBytes()); RoutingTable target = RoutingTable.Builder.readFrom(inStream); assertThat(target.prettyPrint(), equalTo(source.prettyPrint())); diff --git a/src/test/java/org/elasticsearch/cluster/serialization/DiffableTests.java b/src/test/java/org/elasticsearch/cluster/serialization/DiffableTests.java index d87d900a0e8..0cc1c8c5ba8 100644 --- a/src/test/java/org/elasticsearch/cluster/serialization/DiffableTests.java +++ b/src/test/java/org/elasticsearch/cluster/serialization/DiffableTests.java @@ -53,7 +53,7 @@ public class DiffableTests extends ElasticsearchTestCase { Diff diff = DiffableUtils.diff(before, after); BytesStreamOutput out = new BytesStreamOutput(); diff.writeTo(out); - BytesStreamInput in = new BytesStreamInput(out.bytes()); + StreamInput in = StreamInput.wrap(out.bytes()); ImmutableMap serialized = DiffableUtils.readImmutableMapDiff(in, TestDiffable.PROTO).apply(before); assertThat(serialized.size(), equalTo(3)); assertThat(serialized.get("foo").value(), equalTo("1")); @@ -76,7 +76,7 @@ public class DiffableTests extends ElasticsearchTestCase { Diff diff = DiffableUtils.diff(before, after); BytesStreamOutput out = new BytesStreamOutput(); diff.writeTo(out); - BytesStreamInput in = new BytesStreamInput(out.bytes()); + StreamInput in = StreamInput.wrap(out.bytes()); ImmutableOpenMap serialized = DiffableUtils.readImmutableOpenMapDiff(in, new KeyedReader() { @Override public TestDiffable readFrom(StreamInput in, String key) throws IOException { diff --git a/src/test/java/org/elasticsearch/common/io/StreamsTests.java b/src/test/java/org/elasticsearch/common/io/StreamsTests.java index 8ce4167889d..2528b25a92e 100644 --- a/src/test/java/org/elasticsearch/common/io/StreamsTests.java +++ b/src/test/java/org/elasticsearch/common/io/StreamsTests.java @@ -22,7 +22,7 @@ package org.elasticsearch.common.io; import com.google.common.base.Charsets; import org.apache.lucene.util.BytesRef; import org.elasticsearch.common.bytes.BytesArray; -import org.elasticsearch.common.io.stream.BytesStreamInput; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.test.ElasticsearchTestCase; import org.junit.Test; @@ -87,7 +87,7 @@ public class StreamsTests extends ElasticsearchTestCase { byte stuff[] = new byte[] { 0, 1, 2, 3 }; BytesRef stuffRef = new BytesRef(stuff, 2, 2); BytesArray stuffArray = new BytesArray(stuffRef); - BytesStreamInput input = new BytesStreamInput(stuffArray); + StreamInput input = StreamInput.wrap(stuffArray); assertEquals(2, input.read()); assertEquals(3, input.read()); assertEquals(-1, input.read()); diff --git a/src/test/java/org/elasticsearch/common/io/streams/BytesStreamsTests.java b/src/test/java/org/elasticsearch/common/io/streams/BytesStreamsTests.java index 1c5e0034b08..d21c62b01c3 100644 --- a/src/test/java/org/elasticsearch/common/io/streams/BytesStreamsTests.java +++ b/src/test/java/org/elasticsearch/common/io/streams/BytesStreamsTests.java @@ -20,8 +20,8 @@ package org.elasticsearch.common.io.streams; import org.apache.lucene.util.Constants; -import org.elasticsearch.common.io.stream.BytesStreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.lucene.BytesRefs; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.test.ElasticsearchTestCase; @@ -286,7 +286,7 @@ public class BytesStreamsTests extends ElasticsearchTestCase { out.writeString("goodbye"); out.writeStringList(Arrays.asList(new String[]{"Hello", "Again"})); out.writeGenericValue(BytesRefs.toBytesRef("bytesref")); - BytesStreamInput in = new BytesStreamInput(out.bytes().toBytes()); + StreamInput in = StreamInput.wrap(out.bytes().toBytes()); assertThat(in.readBoolean(), equalTo(false)); assertThat(in.readByte(), equalTo((byte)1)); assertThat(in.readShort(), equalTo((short)-1)); diff --git a/src/test/java/org/elasticsearch/common/unit/TimeValueTests.java b/src/test/java/org/elasticsearch/common/unit/TimeValueTests.java index 371c5b8e231..90fa0b747ed 100644 --- a/src/test/java/org/elasticsearch/common/unit/TimeValueTests.java +++ b/src/test/java/org/elasticsearch/common/unit/TimeValueTests.java @@ -19,8 +19,8 @@ package org.elasticsearch.common.unit; -import org.elasticsearch.common.io.stream.BytesStreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.test.ElasticsearchTestCase; import org.joda.time.PeriodType; import org.junit.Test; @@ -73,7 +73,7 @@ public class TimeValueTests extends ElasticsearchTestCase { BytesStreamOutput out = new BytesStreamOutput(); value.writeTo(out); - BytesStreamInput in = new BytesStreamInput(out.bytes()); + StreamInput in = StreamInput.wrap(out.bytes()); TimeValue inValue = TimeValue.readTimeValue(in); assertThat(inValue, equalTo(value)); diff --git a/src/test/java/org/elasticsearch/common/xcontent/XContentFactoryTests.java b/src/test/java/org/elasticsearch/common/xcontent/XContentFactoryTests.java index 1bd403ab82d..e904116221f 100644 --- a/src/test/java/org/elasticsearch/common/xcontent/XContentFactoryTests.java +++ b/src/test/java/org/elasticsearch/common/xcontent/XContentFactoryTests.java @@ -22,7 +22,7 @@ package org.elasticsearch.common.xcontent; import com.fasterxml.jackson.dataformat.cbor.CBORConstants; import com.fasterxml.jackson.dataformat.smile.SmileConstants; import org.elasticsearch.common.bytes.BytesArray; -import org.elasticsearch.common.io.stream.BytesStreamInput; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.test.ElasticsearchTestCase; import org.junit.Test; @@ -65,7 +65,7 @@ public class XContentFactoryTests extends ElasticsearchTestCase { assertThat(XContentFactory.xContentType(builder.bytes()), equalTo(type)); BytesArray bytesArray = builder.bytes().toBytesArray(); - assertThat(XContentFactory.xContentType(new BytesStreamInput(bytesArray.array(), bytesArray.arrayOffset(), bytesArray.length())), equalTo(type)); + assertThat(XContentFactory.xContentType(StreamInput.wrap(bytesArray.array(), bytesArray.arrayOffset(), bytesArray.length())), equalTo(type)); // CBOR is binary, cannot use String if (type != XContentType.CBOR) { diff --git a/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 5835710fea9..46a2129bc48 100644 --- a/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -79,6 +79,7 @@ import org.elasticsearch.index.store.DirectoryService; import org.elasticsearch.index.store.DirectoryUtils; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.index.translog.TranslogConfig; import org.elasticsearch.test.DummyShardLock; import org.elasticsearch.test.ElasticsearchTestCase; import org.elasticsearch.test.junit.annotations.TestLogging; @@ -89,6 +90,7 @@ import org.junit.Before; import org.junit.Test; import java.io.IOException; +import java.nio.charset.Charset; import java.nio.file.Path; import java.util.Arrays; import java.util.List; @@ -220,7 +222,8 @@ public class InternalEngineTests extends ElasticsearchTestCase { } protected Translog createTranslog(Path translogPath) throws IOException { - return new Translog(shardId, EMPTY_SETTINGS, BigArrays.NON_RECYCLING_INSTANCE, translogPath); + TranslogConfig translogConfig = new TranslogConfig(shardId, translogPath, EMPTY_SETTINGS, Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, threadPool); + return new Translog(translogConfig); } protected Translog createTranslogReplica() throws IOException { @@ -254,6 +257,8 @@ public class InternalEngineTests extends ElasticsearchTestCase { public EngineConfig config(IndexSettingsService indexSettingsService, Store store, Path translogPath, MergeSchedulerProvider mergeSchedulerProvider) { IndexWriterConfig iwc = newIndexWriterConfig(); + TranslogConfig translogConfig = new TranslogConfig(shardId, translogPath, indexSettingsService.getSettings(), Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, threadPool); + EngineConfig config = new EngineConfig(shardId, threadPool, new ShardIndexingService(shardId, EMPTY_SETTINGS, new ShardSlowLogIndexingService(shardId, EMPTY_SETTINGS, indexSettingsService)), indexSettingsService , null, store, createSnapshotDeletionPolicy(), createMergePolicy(), mergeSchedulerProvider, iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(shardId.index()), new Engine.FailedEngineListener() { @@ -261,7 +266,7 @@ public class InternalEngineTests extends ElasticsearchTestCase { public void onFailedEngine(ShardId shardId, String reason, @Nullable Throwable t) { // we don't need to notify anybody in this test } - }, new TranslogHandler(shardId.index().getName()), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), BigArrays.NON_RECYCLING_INSTANCE, translogPath); + }, new TranslogHandler(shardId.index().getName()), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig); return config; } @@ -482,15 +487,18 @@ public class InternalEngineTests extends ElasticsearchTestCase { CommitStats stats1 = engine.commitStats(); assertThat(stats1.getGeneration(), greaterThan(0l)); assertThat(stats1.getId(), notNullValue()); - assertThat(stats1.getUserData(), hasKey(Translog.TRANSLOG_ID_KEY)); + assertThat(stats1.getUserData(), hasKey(Translog.TRANSLOG_GENERATION_KEY)); engine.flush(true, true); CommitStats stats2 = engine.commitStats(); assertThat(stats2.getGeneration(), greaterThan(stats1.getGeneration())); assertThat(stats2.getId(), notNullValue()); assertThat(stats2.getId(), not(equalTo(stats1.getId()))); - assertThat(stats2.getUserData(), hasKey(Translog.TRANSLOG_ID_KEY)); - assertThat(stats2.getUserData().get(Translog.TRANSLOG_ID_KEY), not(equalTo(stats1.getUserData().get(Translog.TRANSLOG_ID_KEY)))); + assertThat(stats2.getUserData(), hasKey(Translog.TRANSLOG_GENERATION_KEY)); + assertThat(stats2.getUserData(), hasKey(Translog.TRANSLOG_UUID_KEY)); + assertThat(stats2.getUserData().get(Translog.TRANSLOG_GENERATION_KEY), not(equalTo(stats1.getUserData().get(Translog.TRANSLOG_GENERATION_KEY)))); + assertThat(stats2.getUserData().get(Translog.TRANSLOG_UUID_KEY), equalTo(stats1.getUserData().get(Translog.TRANSLOG_UUID_KEY))) + ; } @Test @@ -1546,7 +1554,8 @@ public class InternalEngineTests extends ElasticsearchTestCase { engine.close(); // fake a new translog, causing the engine to point to a missing one. Translog translog = createTranslog(); - translog.markCommitted(translog.currentId()); + long id = translog.currentFileGeneration(); + IOUtils.rm(translog.location().resolve(Translog.getFilename(id))); // we have to re-open the translog because o.w. it will complain about commit information going backwards, which is OK as we did a fake markComitted translog.close(); try { @@ -1557,7 +1566,7 @@ public class InternalEngineTests extends ElasticsearchTestCase { } // now it should be OK. IndexSettingsService indexSettingsService = new IndexSettingsService(shardId.index(), ImmutableSettings.builder().put(defaultSettings) - .put(EngineConfig.INDEX_IGNORE_UNKNOWN_TRANSLOG, true).build()); + .put(EngineConfig.INDEX_FORCE_NEW_TRANSLOG, true).build()); engine = createEngine(indexSettingsService, store, primaryTranslogDir, createMergeScheduler(indexSettingsService)); } @@ -1778,4 +1787,57 @@ public class InternalEngineTests extends ElasticsearchTestCase { } } + public void testRecoverFromForeignTranslog() throws IOException { + boolean canHaveDuplicates = true; + boolean autoGeneratedId = true; + final int numDocs = randomIntBetween(1, 10); + for (int i = 0; i < numDocs; i++) { + ParsedDocument doc = testParsedDocument(Integer.toString(i), Integer.toString(i), "test", null, -1, -1, testDocument(), new BytesArray("{}"), null); + Engine.Create firstIndexRequest = new Engine.Create(null, newUid(Integer.toString(i)), doc, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), canHaveDuplicates, autoGeneratedId); + engine.create(firstIndexRequest); + assertThat(firstIndexRequest.version(), equalTo(1l)); + } + engine.refresh("test"); + try (Engine.Searcher searcher = engine.acquireSearcher("test")) { + TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10)); + assertThat(topDocs.totalHits, equalTo(numDocs)); + } + final MockDirectoryWrapper directory = DirectoryUtils.getLeaf(store.directory(), MockDirectoryWrapper.class); + if (directory != null) { + // since we rollback the IW we are writing the same segment files again after starting IW but MDW prevents + // this so we have to disable the check explicitly + directory.setPreventDoubleWrite(false); + } + Translog.TranslogGeneration generation = engine.getTranslog().getGeneration(); + engine.close(); + + Translog translog = new Translog(new TranslogConfig(shardId, createTempDir(), ImmutableSettings.EMPTY, Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, threadPool)); + translog.add(new Translog.Create("test", "SomeBogusId", "{}".getBytes(Charset.forName("UTF-8")))); + assertEquals(generation.translogFileGeneration, translog.currentFileGeneration()); + translog.close(); + + EngineConfig config = engine.config(); + Path translogPath = config.getTranslogConfig().getTranslogPath(); + + /* create a TranslogConfig that has been created with a different UUID */ + TranslogConfig translogConfig = new TranslogConfig(shardId, translog.location(), config.getIndexSettings(), Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, threadPool); + + EngineConfig brokenConfig = new EngineConfig(shardId, threadPool, config.getIndexingService(), config.getIndexSettingsService() + , null, store, createSnapshotDeletionPolicy(), createMergePolicy(), config.getMergeScheduler(), + config.getAnalyzer(), config.getSimilarity(), new CodecService(shardId.index()), config.getFailedEngineListener() + , config.getTranslogRecoveryPerformer(), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig); + + try { + new InternalEngine(brokenConfig, false); + fail("translog belongs to a different engine"); + } catch (EngineCreationFailureException ex) { + } + + engine = createEngine(store, primaryTranslogDir); // and recover again! + try (Engine.Searcher searcher = engine.acquireSearcher("test")) { + TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10)); + assertThat(topDocs.totalHits, equalTo(numDocs)); + } + } + } diff --git a/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java b/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java index 558b095d29a..19c60bfaf38 100644 --- a/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java +++ b/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java @@ -63,6 +63,7 @@ import org.elasticsearch.index.store.DirectoryService; import org.elasticsearch.index.store.DirectoryUtils; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.index.translog.TranslogConfig; import org.elasticsearch.test.DummyShardLock; import org.elasticsearch.test.ElasticsearchTestCase; import org.elasticsearch.threadpool.ThreadPool; @@ -230,13 +231,14 @@ public class ShadowEngineTests extends ElasticsearchTestCase { public EngineConfig config(IndexSettingsService indexSettingsService, Store store, Path translogPath, MergeSchedulerProvider mergeSchedulerProvider) { IndexWriterConfig iwc = newIndexWriterConfig(); + TranslogConfig translogConfig = new TranslogConfig(shardId, translogPath, indexSettingsService.getSettings(), Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, threadPool); EngineConfig config = new EngineConfig(shardId, threadPool, new ShardIndexingService(shardId, EMPTY_SETTINGS, new ShardSlowLogIndexingService(shardId, EMPTY_SETTINGS, indexSettingsService)), indexSettingsService , null, store, createSnapshotDeletionPolicy(), createMergePolicy(), mergeSchedulerProvider, iwc.getAnalyzer(), iwc.getSimilarity() , new CodecService(shardId.index()), new Engine.FailedEngineListener() { @Override public void onFailedEngine(ShardId shardId, String reason, @Nullable Throwable t) { // we don't need to notify anybody in this test - }}, null, IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), BigArrays.NON_RECYCLING_INSTANCE, translogPath); + }}, null, IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig); return config; } @@ -256,7 +258,7 @@ public class ShadowEngineTests extends ElasticsearchTestCase { CommitStats stats1 = replicaEngine.commitStats(); assertThat(stats1.getGeneration(), greaterThan(0l)); assertThat(stats1.getId(), notNullValue()); - assertThat(stats1.getUserData(), hasKey(Translog.TRANSLOG_ID_KEY)); + assertThat(stats1.getUserData(), hasKey(Translog.TRANSLOG_GENERATION_KEY)); // flush the primary engine primaryEngine.flush(); @@ -267,8 +269,10 @@ public class ShadowEngineTests extends ElasticsearchTestCase { assertThat(stats2.getGeneration(), greaterThan(stats1.getGeneration())); assertThat(stats2.getId(), notNullValue()); assertThat(stats2.getId(), not(equalTo(stats1.getId()))); - assertThat(stats2.getUserData(), hasKey(Translog.TRANSLOG_ID_KEY)); - assertThat(stats2.getUserData().get(Translog.TRANSLOG_ID_KEY), not(equalTo(stats1.getUserData().get(Translog.TRANSLOG_ID_KEY)))); + assertThat(stats2.getUserData(), hasKey(Translog.TRANSLOG_GENERATION_KEY)); + assertThat(stats2.getUserData(), hasKey(Translog.TRANSLOG_UUID_KEY)); + assertThat(stats2.getUserData().get(Translog.TRANSLOG_GENERATION_KEY), not(equalTo(stats1.getUserData().get(Translog.TRANSLOG_GENERATION_KEY)))); + assertThat(stats2.getUserData().get(Translog.TRANSLOG_UUID_KEY), equalTo(stats1.getUserData().get(Translog.TRANSLOG_UUID_KEY))); } diff --git a/src/test/java/org/elasticsearch/index/mapper/timestamp/TimestampMappingTests.java b/src/test/java/org/elasticsearch/index/mapper/timestamp/TimestampMappingTests.java index 876bdf3ffed..f4d449ed3d0 100644 --- a/src/test/java/org/elasticsearch/index/mapper/timestamp/TimestampMappingTests.java +++ b/src/test/java/org/elasticsearch/index/mapper/timestamp/TimestampMappingTests.java @@ -28,8 +28,8 @@ import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.compress.CompressedString; -import org.elasticsearch.common.io.stream.BytesStreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.joda.Joda; import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.Settings; @@ -459,7 +459,7 @@ public class TimestampMappingTests extends ElasticsearchSingleNodeTest { out.close(); BytesReference bytes = out.bytes(); - MappingMetaData metaData = MappingMetaData.PROTO.readFrom(new BytesStreamInput(bytes)); + MappingMetaData metaData = MappingMetaData.PROTO.readFrom(StreamInput.wrap(bytes)); assertThat(metaData, is(expected)); } @@ -476,7 +476,7 @@ public class TimestampMappingTests extends ElasticsearchSingleNodeTest { out.close(); BytesReference bytes = out.bytes(); - MappingMetaData metaData = MappingMetaData.PROTO.readFrom(new BytesStreamInput(bytes)); + MappingMetaData metaData = MappingMetaData.PROTO.readFrom(StreamInput.wrap(bytes)); assertThat(metaData, is(expected)); } @@ -493,7 +493,7 @@ public class TimestampMappingTests extends ElasticsearchSingleNodeTest { out.close(); BytesReference bytes = out.bytes(); - MappingMetaData metaData = MappingMetaData.PROTO.readFrom(new BytesStreamInput(bytes)); + MappingMetaData metaData = MappingMetaData.PROTO.readFrom(StreamInput.wrap(bytes)); assertThat(metaData, is(expected)); } diff --git a/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index eb50b5e331c..03efbad3e5d 100644 --- a/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -29,6 +29,7 @@ import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.env.ShardLock; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.index.translog.TranslogConfig; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.test.DummyShardLock; import org.elasticsearch.test.ElasticsearchSingleNodeTest; @@ -303,7 +304,7 @@ public class IndexShardTests extends ElasticsearchSingleNodeTest { } private void setDurability(IndexShard shard, Translog.Durabilty durabilty) { - client().admin().indices().prepareUpdateSettings(shard.shardId.getIndex()).setSettings(settingsBuilder().put(Translog.INDEX_TRANSLOG_DURABILITY, durabilty.name()).build()).get(); + client().admin().indices().prepareUpdateSettings(shard.shardId.getIndex()).setSettings(settingsBuilder().put(TranslogConfig.INDEX_TRANSLOG_DURABILITY, durabilty.name()).build()).get(); assertEquals(durabilty, shard.getTranslogDurability()); } } diff --git a/src/test/java/org/elasticsearch/index/store/CorruptedTranslogTests.java b/src/test/java/org/elasticsearch/index/store/CorruptedTranslogTests.java index 9b55cf2cb2e..5890233fe27 100644 --- a/src/test/java/org/elasticsearch/index/store/CorruptedTranslogTests.java +++ b/src/test/java/org/elasticsearch/index/store/CorruptedTranslogTests.java @@ -35,7 +35,7 @@ import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.translog.Translog; -import org.elasticsearch.index.translog.TranslogStreams; +import org.elasticsearch.index.translog.TranslogConfig; import org.elasticsearch.monitor.fs.FsStats; import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.elasticsearch.test.engine.MockEngineSupport; @@ -83,7 +83,7 @@ public class CorruptedTranslogTests extends ElasticsearchIntegrationTest { .put("index.refresh_interval", "-1") .put(MockEngineSupport.FLUSH_ON_CLOSE_RATIO, 0.0d) // never flush - always recover from translog .put(IndexShard.INDEX_FLUSH_ON_CLOSE, false) // never flush - always recover from translog - .put(Translog.INDEX_TRANSLOG_SYNC_INTERVAL, "1s") // fsync the translog every second + .put(TranslogConfig.INDEX_TRANSLOG_SYNC_INTERVAL, "1s") // fsync the translog every second )); ensureYellow(); @@ -150,7 +150,7 @@ public class CorruptedTranslogTests extends ElasticsearchIntegrationTest { fileToCorrupt = RandomPicks.randomFrom(getRandom(), files); try (FileChannel raf = FileChannel.open(fileToCorrupt, StandardOpenOption.READ, StandardOpenOption.WRITE)) { // read - raf.position(randomIntBetween(0, (int) Math.min(TranslogStreams.LATEST.headerLength(), raf.size() - 1))); // only corrupt the header to ensure we actually fail + raf.position(randomIntBetween(0, (int) Math.min(Integer.MAX_VALUE, raf.size() - 1))); long filePointer = raf.position(); ByteBuffer bb = ByteBuffer.wrap(new byte[1]); raf.read(bb); diff --git a/src/test/java/org/elasticsearch/index/translog/BufferedTranslogTests.java b/src/test/java/org/elasticsearch/index/translog/BufferedTranslogTests.java index 1f317bad7a3..857f0b7728d 100644 --- a/src/test/java/org/elasticsearch/index/translog/BufferedTranslogTests.java +++ b/src/test/java/org/elasticsearch/index/translog/BufferedTranslogTests.java @@ -20,10 +20,11 @@ package org.elasticsearch.index.translog; import org.elasticsearch.common.settings.ImmutableSettings; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.BigArrays; -import org.elasticsearch.test.junit.annotations.TestLogging; import java.io.IOException; +import java.nio.file.Path; /** * @@ -31,13 +32,12 @@ import java.io.IOException; public class BufferedTranslogTests extends TranslogTests { @Override - protected Translog create() throws IOException { - return new Translog(shardId, - ImmutableSettings.settingsBuilder() - .put("index.translog.fs.type", TranslogFile.Type.BUFFERED.name()) - .put("index.translog.fs.buffer_size", 10 + randomInt(128 * 1024)) - .build(), - BigArrays.NON_RECYCLING_INSTANCE, translogDir - ); + protected Translog create(Path path) throws IOException { + Settings build = ImmutableSettings.settingsBuilder() + .put("index.translog.fs.type", TranslogWriter.Type.BUFFERED.name()) + .put("index.translog.fs.buffer_size", 10 + randomInt(128 * 1024)) + .build(); + TranslogConfig translogConfig = new TranslogConfig(shardId, path, build, Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, null); + return new Translog(translogConfig); } } diff --git a/src/test/java/org/elasticsearch/index/translog/FsSimpleTranslogTests.java b/src/test/java/org/elasticsearch/index/translog/FsSimpleTranslogTests.java deleted file mode 100644 index cdbeeaaab60..00000000000 --- a/src/test/java/org/elasticsearch/index/translog/FsSimpleTranslogTests.java +++ /dev/null @@ -1,38 +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.translog; - -import org.elasticsearch.common.settings.ImmutableSettings; -import org.elasticsearch.common.util.BigArrays; - -import java.io.IOException; - -/** - * - */ -public class FsSimpleTranslogTests extends TranslogTests { - - @Override - protected Translog create() throws IOException { - return new Translog(shardId, - ImmutableSettings.settingsBuilder().put("index.translog.fs.type", TranslogFile.Type.SIMPLE.name()).build(), - BigArrays.NON_RECYCLING_INSTANCE, translogDir); - } -} \ No newline at end of file diff --git a/src/test/java/org/elasticsearch/index/translog/TranslogTests.java b/src/test/java/org/elasticsearch/index/translog/TranslogTests.java index bf8603dbd8c..8d0e1d80808 100644 --- a/src/test/java/org/elasticsearch/index/translog/TranslogTests.java +++ b/src/test/java/org/elasticsearch/index/translog/TranslogTests.java @@ -19,15 +19,23 @@ package org.elasticsearch.index.translog; +import com.carrotsearch.randomizedtesting.generators.RandomPicks; +import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.index.Term; +import org.apache.lucene.store.AlreadyClosedException; +import org.apache.lucene.store.ByteArrayDataOutput; import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util.TestUtil; import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.Version; +import org.elasticsearch.bwcompat.OldIndexBackwardsCompatibilityTests; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.io.FileSystemUtils; -import org.elasticsearch.common.io.stream.BytesStreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.ImmutableSettings; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; @@ -42,9 +50,11 @@ import org.junit.Test; import java.io.EOFException; import java.io.IOException; +import java.io.InputStream; import java.nio.ByteBuffer; import java.nio.channels.FileChannel; import java.nio.charset.Charset; +import java.nio.file.DirectoryStream; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.StandardOpenOption; @@ -54,6 +64,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import java.util.regex.Matcher; import static com.google.common.collect.Lists.newArrayList; import static org.hamcrest.Matchers.*; @@ -74,13 +85,13 @@ public class TranslogTests extends ElasticsearchTestCase { super.afterIfSuccessful(); if (translog.isOpen()) { - if (translog.currentId() > 1) { - translog.markCommitted(translog.currentId()); - assertFileDeleted(translog, translog.currentId() - 1); + if (translog.currentFileGeneration() > 1) { + translog.commit(); + assertFileDeleted(translog, translog.currentFileGeneration() - 1); } translog.close(); } - assertFileIsPresent(translog, translog.currentId()); + assertFileIsPresent(translog, translog.currentFileGeneration()); IOUtils.rm(translog.location()); // delete all the locations } @@ -91,7 +102,7 @@ public class TranslogTests extends ElasticsearchTestCase { super.setUp(); // if a previous test failed we clean up things here translogDir = createTempDir(); - translog = create(); + translog = create(translogDir); } @Override @@ -104,10 +115,12 @@ public class TranslogTests extends ElasticsearchTestCase { } } - protected Translog create() throws IOException { - return new Translog(shardId, - ImmutableSettings.settingsBuilder().put("index.translog.fs.type", TranslogFile.Type.SIMPLE.name()).build(), - BigArrays.NON_RECYCLING_INSTANCE, translogDir); + protected Translog create(Path path) throws IOException { + Settings build = ImmutableSettings.settingsBuilder() + .put("index.translog.fs.type", TranslogWriter.Type.SIMPLE.name()) + .build(); + TranslogConfig translogConfig = new TranslogConfig(shardId, path, build, Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, null); + return new Translog(translogConfig); } protected void addToTranslogAndList(Translog translog, ArrayList list, Translog.Operation op) { @@ -152,6 +165,8 @@ public class TranslogTests extends ElasticsearchTestCase { assertThat(translog.read(loc3).getSource().source.toBytesArray(), equalTo(new BytesArray(new byte[]{3}))); translog.sync(); assertThat(translog.read(loc3).getSource().source.toBytesArray(), equalTo(new BytesArray(new byte[]{3}))); + + } @Test @@ -197,16 +212,16 @@ public class TranslogTests extends ElasticsearchTestCase { snapshot.close(); - long firstId = translog.currentId(); - translog.newTranslog(); - assertThat(translog.currentId(), Matchers.not(equalTo(firstId))); + long firstId = translog.currentFileGeneration(); + translog.prepareCommit(); + assertThat(translog.currentFileGeneration(), Matchers.not(equalTo(firstId))); snapshot = translog.newSnapshot(); assertThat(snapshot, SnapshotMatchers.equalsTo(ops)); assertThat(snapshot.estimatedTotalOperations(), equalTo(ops.size())); snapshot.close(); - translog.markCommitted(translog.currentId()); + translog.commit(); snapshot = translog.newSnapshot(); assertThat(snapshot, SnapshotMatchers.size(0)); assertThat(snapshot.estimatedTotalOperations(), equalTo(0)); @@ -220,7 +235,7 @@ public class TranslogTests extends ElasticsearchTestCase { if (randomBoolean()) { BytesStreamOutput out = new BytesStreamOutput(); stats.writeTo(out); - BytesStreamInput in = new BytesStreamInput(out.bytes()); + StreamInput in = StreamInput.wrap(out.bytes()); stats = new TranslogStats(); stats.readFrom(in); } @@ -229,10 +244,12 @@ public class TranslogTests extends ElasticsearchTestCase { @Test public void testStats() throws IOException { + final long firstOperationPosition = translog.getFirstOperationPosition(); TranslogStats stats = stats(); assertThat(stats.estimatedNumberOfOperations(), equalTo(0l)); long lastSize = stats.translogSizeInBytes().bytes(); - assertThat(lastSize, equalTo(17l)); + assertThat((int) firstOperationPosition, greaterThan(CodecUtil.headerLength(TranslogWriter.TRANSLOG_CODEC))); + assertThat(lastSize, equalTo(firstOperationPosition)); translog.add(new Translog.Create("test", "1", new byte[]{1})); stats = stats(); @@ -253,15 +270,15 @@ public class TranslogTests extends ElasticsearchTestCase { lastSize = stats.translogSizeInBytes().bytes(); translog.add(new Translog.Delete(newUid("4"))); - translog.newTranslog(); + translog.prepareCommit(); stats = stats(); assertThat(stats.estimatedNumberOfOperations(), equalTo(4l)); assertThat(stats.translogSizeInBytes().bytes(), greaterThan(lastSize)); - translog.markCommitted(2); + translog.commit(); stats = stats(); assertThat(stats.estimatedNumberOfOperations(), equalTo(0l)); - assertThat(stats.translogSizeInBytes().bytes(), equalTo(17l)); + assertThat(stats.translogSizeInBytes().bytes(), equalTo(firstOperationPosition)); } @Test @@ -303,11 +320,11 @@ public class TranslogTests extends ElasticsearchTestCase { addToTranslogAndList(translog, ops, new Translog.Index("test", "2", new byte[]{2})); - translog.newTranslog(); - + translog.prepareCommit(); addToTranslogAndList(translog, ops, new Translog.Index("test", "3", new byte[]{3})); Translog.Snapshot snapshot2 = translog.newSnapshot(); + translog.commit(); assertThat(snapshot2, SnapshotMatchers.equalsTo(ops)); assertThat(snapshot2.estimatedTotalOperations(), equalTo(ops.size())); @@ -318,14 +335,14 @@ public class TranslogTests extends ElasticsearchTestCase { } public void testSnapshotOnClosedTranslog() throws IOException { - assertTrue(Files.exists(translogDir.resolve("translog-1"))); + assertTrue(Files.exists(translogDir.resolve(translog.getFilename(1)))); translog.add(new Translog.Create("test", "1", new byte[]{1})); translog.close(); try { Translog.Snapshot snapshot = translog.newSnapshot(); fail("translog is closed"); - } catch (TranslogException ex) { - assertThat(ex.getMessage(), containsString("can't increment channel")); + } catch (AlreadyClosedException ex) { + assertThat(ex.getMessage(), containsString("translog-1.tlog is already closed can't increment")); } } @@ -336,8 +353,7 @@ public class TranslogTests extends ElasticsearchTestCase { Translog.Snapshot firstSnapshot = translog.newSnapshot(); assertThat(firstSnapshot.estimatedTotalOperations(), equalTo(1)); - translog.newTranslog(); - translog.markCommitted(translog.currentId()); + translog.commit(); assertFileIsPresent(translog, 1); @@ -357,8 +373,7 @@ public class TranslogTests extends ElasticsearchTestCase { assertFileIsPresent(translog, 2); secondSnapshot.close(); assertFileIsPresent(translog, 2); // it's the current nothing should be deleted - translog.newTranslog(); - translog.markCommitted(translog.currentId()); + translog.commit(); assertFileIsPresent(translog, 3); // it's the current nothing should be deleted assertFileDeleted(translog, 2); @@ -490,7 +505,6 @@ public class TranslogTests extends ElasticsearchTestCase { } @Test - @LuceneTestCase.BadApple(bugUrl = "corrupting size can cause OOME") public void testTranslogChecksums() throws Exception { List locations = newArrayList(); @@ -691,7 +705,7 @@ public class TranslogTests extends ElasticsearchTestCase { view = translog.newView(); // captures the currently written ops so we know what to expect from the view writtenOpsAtView = new HashSet<>(writtenOps.keySet()); - logger.debug("--> [{}] opened view from [{}]", threadId, view.minTranslogId()); + logger.debug("--> [{}] opened view from [{}]", threadId, view.minTranslogGeneration()); } @Override @@ -718,7 +732,7 @@ public class TranslogTests extends ElasticsearchTestCase { boolean failed = false; for (Translog.Operation op : expectedOps) { final Translog.Location loc = writtenOps.get(op); - if (loc.translogId < view.minTranslogId()) { + if (loc.generation < view.minTranslogGeneration()) { // writtenOps is only updated after the op was written to the translog. This mean // that ops written to the translog before the view was taken (and will be missing from the view) // may yet be available in writtenOpsAtView, meaning we will erroneously expect them @@ -743,7 +757,6 @@ public class TranslogTests extends ElasticsearchTestCase { barrier.await(); try { - long previousId = translog.currentId(); for (int iterations = scaledRandomIntBetween(10, 200); iterations > 0 && errors.isEmpty(); iterations--) { writtenOpsLatch.set(new CountDownLatch(flushEveryOps)); while (writtenOpsLatch.get().await(200, TimeUnit.MILLISECONDS) == false) { @@ -751,9 +764,7 @@ public class TranslogTests extends ElasticsearchTestCase { break; } } - long newId = translog.newTranslog(); - translog.markCommitted(previousId); - previousId = newId; + translog.commit(); } } finally { run.set(false); @@ -791,7 +802,7 @@ public class TranslogTests extends ElasticsearchTestCase { assertTrue("we only synced a previous operation yet", translog.syncNeeded()); } if (rarely()) { - translog.newTranslog(); + translog.commit(); assertFalse("location is from a previous translog - already synced", translog.ensureSynced(location)); // not syncing now assertFalse("no sync needed since no operations in current translog", translog.syncNeeded()); } @@ -810,7 +821,7 @@ public class TranslogTests extends ElasticsearchTestCase { for (int op = 0; op < translogOperations; op++) { locations.add(translog.add(new Translog.Create("test", "" + op, Integer.toString(++count).getBytes(Charset.forName("UTF-8"))))); if (rarely() && translogOperations > op+1) { - translog.newTranslog(); + translog.commit(); } } Collections.shuffle(locations, random()); @@ -819,7 +830,7 @@ public class TranslogTests extends ElasticsearchTestCase { max = max(max, location); } - assertEquals(max.translogId, translog.currentId()); + assertEquals(max.generation, translog.currentFileGeneration()); final Translog.Operation read = translog.read(max); assertEquals(read.getSource().source.toUtf8(), Integer.toString(count)); } @@ -830,4 +841,398 @@ public class TranslogTests extends ElasticsearchTestCase { } return b; } + + + public void testBasicCheckpoint() throws IOException { + List locations = newArrayList(); + int translogOperations = randomIntBetween(10, 100); + int lastSynced = -1; + for (int op = 0; op < translogOperations; op++) { + locations.add(translog.add(new Translog.Create("test", "" + op, Integer.toString(op).getBytes(Charset.forName("UTF-8"))))); + if (frequently()) { + translog.sync(); + lastSynced = op; + } + } + assertEquals(translogOperations, translog.totalOperations()); + final Translog.Location lastLocation = translog.add(new Translog.Create("test", "" + translogOperations, Integer.toString(translogOperations).getBytes(Charset.forName("UTF-8")))); + + final Checkpoint checkpoint = Checkpoint.read(translog.location().resolve(Translog.CHECKPOINT_FILE_NAME)); + try (final ImmutableTranslogReader reader = translog.openReader(translog.location().resolve(translog.getFilename(translog.currentFileGeneration())), checkpoint)) { + assertEquals(lastSynced + 1, reader.totalOperations()); + for (int op = 0; op < translogOperations; op++) { + Translog.Location location = locations.get(op); + if (op <= lastSynced) { + final Translog.Operation read = reader.read(location); + assertEquals(Integer.toString(op), read.getSource().source.toUtf8()); + } else { + try { + reader.read(location); + fail("read past checkpoint"); + } catch (EOFException ex) { + + } + } + } + try { + reader.read(lastLocation); + fail("read past checkpoint"); + } catch (EOFException ex) { + } + } + assertEquals(translogOperations + 1, translog.totalOperations()); + translog.close(); + } + + public void testTranslogWriter() throws IOException { + final TranslogWriter writer = translog.createWriter(0); + final int numOps = randomIntBetween(10, 100); + byte[] bytes = new byte[4]; + ByteArrayDataOutput out = new ByteArrayDataOutput(bytes); + for (int i = 0; i < numOps; i++) { + out.reset(bytes); + out.writeInt(i); + writer.add(new BytesArray(bytes)); + } + writer.sync(); + + final TranslogReader reader = randomBoolean() ? writer : translog.openReader(writer.path(), Checkpoint.read(translog.location().resolve(Translog.CHECKPOINT_FILE_NAME))); + for (int i = 0; i < numOps; i++) { + ByteBuffer buffer = ByteBuffer.allocate(4); + reader.readBytes(buffer, reader.getFirstOperationOffset() + 4*i); + buffer.flip(); + final int value = buffer.getInt(); + assertEquals(i, value); + } + + out.reset(bytes); + out.writeInt(2048); + writer.add(new BytesArray(bytes)); + + if (reader instanceof ImmutableTranslogReader) { + ByteBuffer buffer = ByteBuffer.allocate(4); + try { + reader.readBytes(buffer, reader.getFirstOperationOffset() + 4 * numOps); + fail("read past EOF?"); + } catch (EOFException ex) { + // expected + } + } else { + // live reader! + ByteBuffer buffer = ByteBuffer.allocate(4); + final long pos = reader.getFirstOperationOffset() + 4 * numOps; + reader.readBytes(buffer, pos); + buffer.flip(); + final int value = buffer.getInt(); + assertEquals(2048, value); + } + IOUtils.close(writer, reader); + } + + public void testBasicRecovery() throws IOException { + List locations = newArrayList(); + int translogOperations = randomIntBetween(10, 100); + Translog.TranslogGeneration translogGeneration = null; + int minUncommittedOp = -1; + final boolean commitOften = randomBoolean(); + for (int op = 0; op < translogOperations; op++) { + locations.add(translog.add(new Translog.Create("test", "" + op, Integer.toString(op).getBytes(Charset.forName("UTF-8"))))); + final boolean commit = commitOften ? frequently() : rarely(); + if (commit && op < translogOperations-1) { + translog.commit(); + minUncommittedOp = op+1; + translogGeneration = translog.getGeneration(); + } + } + translog.sync(); + TranslogConfig config = translog.getConfig(); + + translog.close(); + config.setTranslogGeneration(translogGeneration); + translog = new Translog(config); + if (translogGeneration == null) { + assertEquals(0, translog.stats().estimatedNumberOfOperations()); + assertEquals(1, translog.currentFileGeneration()); + assertFalse(translog.syncNeeded()); + try (Translog.Snapshot snapshot = translog.newSnapshot()) { + assertNull(snapshot.next()); + } + } else { + assertEquals("lastCommitted must be 1 less than current", translogGeneration.translogFileGeneration + 1, translog.currentFileGeneration()); + assertFalse(translog.syncNeeded()); + try (Translog.Snapshot snapshot = translog.newSnapshot()) { + for (int i = minUncommittedOp; i < translogOperations; i++) { + assertEquals("expected operation" + i + " to be in the previous translog but wasn't", translog.currentFileGeneration() - 1, locations.get(i).generation); + Translog.Operation next = snapshot.next(); + assertNotNull("operation " + i + " must be non-null", next); + assertEquals(i, Integer.parseInt(next.getSource().source.toUtf8())); + } + } + } + } + + public void testRecoveryUncommitted() throws IOException { + List locations = newArrayList(); + int translogOperations = randomIntBetween(10, 100); + final int prepareOp = randomIntBetween(0, translogOperations-1); + Translog.TranslogGeneration translogGeneration = null; + final boolean sync = randomBoolean(); + for (int op = 0; op < translogOperations; op++) { + locations.add(translog.add(new Translog.Create("test", "" + op, Integer.toString(op).getBytes(Charset.forName("UTF-8"))))); + if (op == prepareOp) { + translogGeneration = translog.getGeneration(); + translog.prepareCommit(); + assertEquals("expected this to be the first commit", 1l, translogGeneration.translogFileGeneration); + assertNotNull(translogGeneration.translogUUID); + } + } + if (sync) { + translog.sync(); + } + // we intentionally don't close the tlog that is in the prepareCommit stage since we try to recovery the uncommitted + // translog here as well. + TranslogConfig config = translog.getConfig(); + config.setTranslogGeneration(translogGeneration); + try (Translog translog = new Translog(config)) { + assertNotNull(translogGeneration); + assertEquals("lastCommitted must be 2 less than current - we never finished the commit", translogGeneration.translogFileGeneration + 2, translog.currentFileGeneration()); + assertFalse(translog.syncNeeded()); + try (Translog.Snapshot snapshot = translog.newSnapshot()) { + int upTo = sync ? translogOperations : prepareOp; + for (int i = 0; i < upTo; i++) { + Translog.Operation next = snapshot.next(); + assertNotNull("operation " + i + " must be non-null synced: " + sync, next); + assertEquals("payload missmatch, synced: " + sync, i, Integer.parseInt(next.getSource().source.toUtf8())); + } + } + } + if (randomBoolean()) { // recover twice + try (Translog translog = new Translog(config)) { + assertNotNull(translogGeneration); + assertEquals("lastCommitted must be 3 less than current - we never finished the commit and run recovery twice", translogGeneration.translogFileGeneration + 3, translog.currentFileGeneration()); + assertFalse(translog.syncNeeded()); + try (Translog.Snapshot snapshot = translog.newSnapshot()) { + int upTo = sync ? translogOperations : prepareOp; + for (int i = 0; i < upTo; i++) { + Translog.Operation next = snapshot.next(); + assertNotNull("operation " + i + " must be non-null synced: " + sync, next); + assertEquals("payload missmatch, synced: " + sync, i, Integer.parseInt(next.getSource().source.toUtf8())); + } + } + } + } + + } + + + public void testSnapshotFromStreamInput() throws IOException { + BytesStreamOutput out = new BytesStreamOutput(); + List ops = newArrayList(); + int translogOperations = randomIntBetween(10, 100); + for (int op = 0; op < translogOperations; op++) { + Translog.Create test = new Translog.Create("test", "" + op, Integer.toString(op).getBytes(Charset.forName("UTF-8"))); + Translog.writeOperation(out, test); + ops.add(test); + } + Translog.Snapshot snapshot = Translog.snapshotFromStream(StreamInput.wrap(out.bytes()), ops.size()); + assertEquals(ops.size(), snapshot.estimatedTotalOperations()); + for (Translog.Operation op : ops) { + assertEquals(op, snapshot.next()); + } + assertNull(snapshot.next()); + // no need to close + } + + public void testLocationHashCodeEquals() throws IOException { + List locations = newArrayList(); + List locations2 = newArrayList(); + int translogOperations = randomIntBetween(10, 100); + try(Translog translog2 = create(createTempDir())) { + for (int op = 0; op < translogOperations; op++) { + locations.add(translog.add(new Translog.Create("test", "" + op, Integer.toString(op).getBytes(Charset.forName("UTF-8"))))); + locations2.add(translog2.add(new Translog.Create("test", "" + op, Integer.toString(op).getBytes(Charset.forName("UTF-8"))))); + } + int iters = randomIntBetween(10, 100); + for (int i = 0; i < iters; i++) { + Translog.Location location = RandomPicks.randomFrom(random(), locations); + for (Translog.Location loc : locations) { + if (loc == location) { + assertTrue(loc.equals(location)); + assertEquals(loc.hashCode(), location.hashCode()); + } else { + assertFalse(loc.equals(location)); + } + } + for (int j = 0; j < translogOperations; j++) { + assertTrue(locations.get(j).equals(locations2.get(j))); + assertEquals(locations.get(j).hashCode(), locations2.get(j).hashCode()); + } + } + } + } + + public void testOpenForeignTranslog() throws IOException { + List locations = newArrayList(); + int translogOperations = randomIntBetween(1, 10); + int firstUncommitted = 0; + for (int op = 0; op < translogOperations; op++) { + locations.add(translog.add(new Translog.Create("test", "" + op, Integer.toString(op).getBytes(Charset.forName("UTF-8"))))); + if (randomBoolean()) { + translog.commit(); + firstUncommitted = op + 1; + } + } + TranslogConfig config = translog.getConfig(); + Translog.TranslogGeneration translogGeneration = translog.getGeneration(); + translog.close(); + + config.setTranslogGeneration(new Translog.TranslogGeneration(randomRealisticUnicodeOfCodepointLengthBetween(1, translogGeneration.translogUUID.length()),translogGeneration.translogFileGeneration)); + try { + new Translog(config); + fail("translog doesn't belong to this UUID"); + } catch (TranslogCorruptedException ex) { + + } + config.setTranslogGeneration(translogGeneration); + this.translog = new Translog(config); + try (Translog.Snapshot snapshot = this.translog.newSnapshot()) { + for (int i = firstUncommitted; i < translogOperations; i++) { + Translog.Operation next = snapshot.next(); + assertNotNull("" + i, next); + assertEquals(Integer.parseInt(next.getSource().source.toUtf8()), i); + } + assertNull(snapshot.next()); + } + } + + public void testUpgradeOldTranslogFiles() throws IOException { + List indexes = new ArrayList<>(); + Path dir = getDataPath("/" + OldIndexBackwardsCompatibilityTests.class.getPackage().getName().replace('.', '/')); // the files are in the same pkg as the OldIndexBackwardsCompatibilityTests test + try (DirectoryStream stream = Files.newDirectoryStream(dir, "index-*.zip")) { + for (Path path : stream) { + indexes.add(path); + } + } + TranslogConfig config = this.translog.getConfig(); + Translog.TranslogGeneration gen = translog.getGeneration(); + this.translog.close(); + try { + Translog.upgradeLegacyTranslog(logger, translog.getConfig()); + fail("no generation set"); + } catch (IllegalArgumentException ex) { + + } + translog.getConfig().setTranslogGeneration(gen); + try { + Translog.upgradeLegacyTranslog(logger, translog.getConfig()); + fail("already upgraded generation set"); + } catch (IllegalArgumentException ex) { + + } + + for (Path indexFile : indexes) { + final String indexName = indexFile.getFileName().toString().replace(".zip", "").toLowerCase(Locale.ROOT); + Version version = Version.fromString(indexName.replace("index-", "")); + if (version.onOrAfter(Version.V_2_0_0)) { + continue; + } + Path unzipDir = createTempDir(); + Path unzipDataDir = unzipDir.resolve("data"); + // decompress the index + try (InputStream stream = Files.newInputStream(indexFile)) { + TestUtil.unzip(stream, unzipDir); + } + // check it is unique + assertTrue(Files.exists(unzipDataDir)); + Path[] list = FileSystemUtils.files(unzipDataDir); + if (list.length != 1) { + throw new IllegalStateException("Backwards index must contain exactly one cluster but was " + list.length); + } + // the bwc scripts packs the indices under this path + Path src = list[0].resolve("nodes/0/indices/" + indexName); + Path translog = list[0].resolve("nodes/0/indices/" + indexName).resolve("0").resolve("translog"); + + assertTrue("[" + indexFile + "] missing index dir: " + src.toString(), Files.exists(src)); + assertTrue("[" + indexFile + "] missing translog dir: " + translog.toString(), Files.exists(translog)); + Path[] tlogFiles = FileSystemUtils.files(translog); + assertEquals(tlogFiles.length, 1); + final long size = Files.size(tlogFiles[0]); + + final long generation = Translog.parseIdFromFileName(tlogFiles[0]); + assertTrue(generation >= 1); + logger.debug("upgrading index {} file: {} size: {}", indexName, tlogFiles[0].getFileName(), size); + TranslogConfig upgradeConfig = new TranslogConfig(config.getShardId(), translog, config.getIndexSettings(), config.getDurabilty(), config.getBigArrays(), config.getThreadPool()); + upgradeConfig.setTranslogGeneration(new Translog.TranslogGeneration(null, generation)); + Translog.upgradeLegacyTranslog(logger, upgradeConfig); + try (Translog upgraded = new Translog(upgradeConfig)) { + assertEquals(generation + 1, upgraded.getGeneration().translogFileGeneration); + assertEquals(upgraded.getRecoveredReaders().size(), 1); + final long headerSize; + if (version.before(Version.V_1_4_0)) { + assertTrue(upgraded.getRecoveredReaders().get(0).getClass().toString(), upgraded.getRecoveredReaders().get(0).getClass() == LegacyTranslogReader.class); + headerSize = 0; + } else { + assertTrue(upgraded.getRecoveredReaders().get(0).getClass().toString(), upgraded.getRecoveredReaders().get(0).getClass() == LegacyTranslogReaderBase.class); + headerSize = CodecUtil.headerLength(TranslogWriter.TRANSLOG_CODEC); + } + List operations = new ArrayList<>(); + try (Translog.Snapshot snapshot = upgraded.newSnapshot()) { + Translog.Operation op = null; + while ((op = snapshot.next()) != null) { + operations.add(op); + } + } + if (size > headerSize) { + assertFalse(operations.toString(), operations.isEmpty()); + } else { + assertTrue(operations.toString(), operations.isEmpty()); + } + } + } + } + + /** + * this tests a set of files that has some of the operations flushed with a buffered translog such that tlogs are truncated. + * 3 of the 6 files are created with ES 1.3 and the rest is created wiht ES 1.4 such that both the checksummed as well as the + * super old version of the translog without a header is tested. + */ + public void testOpenAndReadTruncatedLegacyTranslogs() throws IOException { + Path zip = getDataPath("/org/elasticsearch/index/translog/legacy_translogs.zip"); + Path unzipDir = createTempDir(); + try (InputStream stream = Files.newInputStream(zip)) { + TestUtil.unzip(stream, unzipDir); + } + TranslogConfig config = this.translog.getConfig(); + int count = 0; + try (DirectoryStream stream = Files.newDirectoryStream(unzipDir)) { + + for (Path legacyTranslog : stream) { + logger.debug("upgrading {} ", legacyTranslog.getFileName()); + Path directory = legacyTranslog.resolveSibling("translog_" + count++); + Files.createDirectories(directory); + Files.copy(legacyTranslog, directory.resolve(legacyTranslog.getFileName())); + TranslogConfig upgradeConfig = new TranslogConfig(config.getShardId(), directory, config.getIndexSettings(), config.getDurabilty(), config.getBigArrays(), config.getThreadPool()); + try { + Translog.upgradeLegacyTranslog(logger, upgradeConfig); + fail("no generation set"); + } catch (IllegalArgumentException ex) { + // expected + } + long generation = Translog.parseIdFromFileName(legacyTranslog); + upgradeConfig.setTranslogGeneration(new Translog.TranslogGeneration(null, generation)); + Translog.upgradeLegacyTranslog(logger, upgradeConfig); + try (Translog tlog = new Translog(upgradeConfig)) { + List operations = new ArrayList<>(); + try (Translog.Snapshot snapshot = tlog.newSnapshot()) { + Translog.Operation op = null; + while ((op = snapshot.next()) != null) { + operations.add(op); + } + } + logger.debug("num ops recovered: {} for file {} ", operations.size(), legacyTranslog.getFileName()); + assertFalse(operations.isEmpty()); + } + } + } + } } diff --git a/src/test/java/org/elasticsearch/index/translog/TranslogVersionTests.java b/src/test/java/org/elasticsearch/index/translog/TranslogVersionTests.java index 80e620c9ccb..b5603a2fa46 100644 --- a/src/test/java/org/elasticsearch/index/translog/TranslogVersionTests.java +++ b/src/test/java/org/elasticsearch/index/translog/TranslogVersionTests.java @@ -19,14 +19,16 @@ package org.elasticsearch.index.translog; -import org.elasticsearch.common.io.stream.StreamInput; +import org.apache.lucene.util.IOUtils; import org.elasticsearch.index.VersionType; import org.elasticsearch.test.ElasticsearchTestCase; import org.junit.Test; -import java.io.EOFException; +import java.io.IOException; +import java.nio.channels.FileChannel; import java.nio.file.Files; import java.nio.file.Path; +import java.nio.file.StandardOpenOption; import static org.hamcrest.Matchers.equalTo; @@ -39,68 +41,58 @@ public class TranslogVersionTests extends ElasticsearchTestCase { public void testV0LegacyTranslogVersion() throws Exception { Path translogFile = getDataPath("/org/elasticsearch/index/translog/translog-v0.binary"); assertThat("test file should exist", Files.exists(translogFile), equalTo(true)); - TranslogStream stream = TranslogStreams.translogStreamFor(translogFile); - assertThat("a version0 stream is returned", stream instanceof LegacyTranslogStream, equalTo(true)); + try (ImmutableTranslogReader reader = openReader(translogFile, 0)) { + assertThat("a version0 stream is returned", reader instanceof LegacyTranslogReader, equalTo(true)); + try (final Translog.Snapshot snapshot = reader.newSnapshot()) { + final Translog.Operation operation = snapshot.next(); + assertThat("operation is the correct type correctly", operation.opType() == Translog.Operation.Type.SAVE, equalTo(true)); + Translog.Index op = (Translog.Index) operation; + assertThat(op.id(), equalTo("1")); + assertThat(op.type(), equalTo("doc")); + assertThat(op.source().toUtf8(), equalTo("{\"body\": \"worda wordb wordc wordd \\\"worde\\\" wordf\"}")); + assertThat(op.routing(), equalTo(null)); + assertThat(op.parent(), equalTo(null)); + assertThat(op.version(), equalTo(1L)); + assertThat(op.timestamp(), equalTo(1407312091791L)); + assertThat(op.ttl(), equalTo(-1L)); + assertThat(op.versionType(), equalTo(VersionType.INTERNAL)); - StreamInput in = stream.openInput(translogFile); - - Translog.Operation operation = stream.read(in); - - assertThat("operation is the correct type correctly", operation.opType() == Translog.Operation.Type.SAVE, equalTo(true)); - Translog.Index op = (Translog.Index) operation; - assertThat(op.id(), equalTo("1")); - assertThat(op.type(), equalTo("doc")); - assertThat(op.source().toUtf8(), equalTo("{\"body\": \"worda wordb wordc wordd \\\"worde\\\" wordf\"}")); - assertThat(op.routing(), equalTo(null)); - assertThat(op.parent(), equalTo(null)); - assertThat(op.version(), equalTo(1L)); - assertThat(op.timestamp(), equalTo(1407312091791L)); - assertThat(op.ttl(), equalTo(-1L)); - assertThat(op.versionType(), equalTo(VersionType.INTERNAL)); - - try { - stream.read(in); - fail("should have been the end of the file"); - } catch (EOFException e) { - // success + assertNull(snapshot.next()); + } } - in.close(); } @Test public void testV1ChecksummedTranslogVersion() throws Exception { Path translogFile = getDataPath("/org/elasticsearch/index/translog/translog-v1.binary"); assertThat("test file should exist", Files.exists(translogFile), equalTo(true)); - TranslogStream stream = TranslogStreams.translogStreamFor(translogFile); - assertThat("a version1 stream is returned", stream instanceof ChecksummedTranslogStream, equalTo(true)); + try (ImmutableTranslogReader reader = openReader(translogFile, 0)) { + try (final Translog.Snapshot snapshot = reader.newSnapshot()) { - StreamInput in = stream.openInput(translogFile); - Translog.Operation operation = stream.read(in); + assertThat("a version1 stream is returned", reader instanceof ImmutableTranslogReader, equalTo(true)); - assertThat("operation is the correct type correctly", operation.opType() == Translog.Operation.Type.CREATE, equalTo(true)); - Translog.Create op = (Translog.Create) operation; - assertThat(op.id(), equalTo("Bwiq98KFSb6YjJQGeSpeiw")); - assertThat(op.type(), equalTo("doc")); - assertThat(op.source().toUtf8(), equalTo("{\"body\": \"foo\"}")); - assertThat(op.routing(), equalTo(null)); - assertThat(op.parent(), equalTo(null)); - assertThat(op.version(), equalTo(1L)); - assertThat(op.timestamp(), equalTo(1408627184844L)); - assertThat(op.ttl(), equalTo(-1L)); - assertThat(op.versionType(), equalTo(VersionType.INTERNAL)); + Translog.Operation operation = snapshot.next(); - // There are more operations - int opNum = 1; - while (true) { - try { - stream.read(in); - opNum++; - } catch (EOFException e) { - break; + assertThat("operation is the correct type correctly", operation.opType() == Translog.Operation.Type.CREATE, equalTo(true)); + Translog.Create op = (Translog.Create) operation; + assertThat(op.id(), equalTo("Bwiq98KFSb6YjJQGeSpeiw")); + assertThat(op.type(), equalTo("doc")); + assertThat(op.source().toUtf8(), equalTo("{\"body\": \"foo\"}")); + assertThat(op.routing(), equalTo(null)); + assertThat(op.parent(), equalTo(null)); + assertThat(op.version(), equalTo(1L)); + assertThat(op.timestamp(), equalTo(1408627184844L)); + assertThat(op.ttl(), equalTo(-1L)); + assertThat(op.versionType(), equalTo(VersionType.INTERNAL)); + + // There are more operations + int opNum = 1; + while (snapshot.next() != null) { + opNum++; + } + assertThat("there should be 5 translog operations", opNum, equalTo(5)); } } - assertThat("there should be 5 translog operations", opNum, equalTo(5)); - in.close(); } @Test @@ -108,7 +100,7 @@ public class TranslogVersionTests extends ElasticsearchTestCase { try { Path translogFile = getDataPath("/org/elasticsearch/index/translog/translog-v1-corrupted-magic.binary"); assertThat("test file should exist", Files.exists(translogFile), equalTo(true)); - TranslogStream stream = TranslogStreams.translogStreamFor(translogFile); + openReader(translogFile, 0); fail("should have thrown an exception about the header being corrupt"); } catch (TranslogCorruptedException e) { assertThat("translog corruption from header: " + e.getMessage(), @@ -118,7 +110,7 @@ public class TranslogVersionTests extends ElasticsearchTestCase { try { Path translogFile = getDataPath("/org/elasticsearch/index/translog/translog-invalid-first-byte.binary"); assertThat("test file should exist", Files.exists(translogFile), equalTo(true)); - TranslogStream stream = TranslogStreams.translogStreamFor(translogFile); + openReader(translogFile, 0); fail("should have thrown an exception about the header being corrupt"); } catch (TranslogCorruptedException e) { assertThat("translog corruption from header: " + e.getMessage(), @@ -128,13 +120,10 @@ public class TranslogVersionTests extends ElasticsearchTestCase { try { Path translogFile = getDataPath("/org/elasticsearch/index/translog/translog-v1-corrupted-body.binary"); assertThat("test file should exist", Files.exists(translogFile), equalTo(true)); - TranslogStream stream = TranslogStreams.translogStreamFor(translogFile); - try (StreamInput in = stream.openInput(translogFile)) { - while (true) { - try { - stream.read(in); - } catch (EOFException e) { - break; + try (ImmutableTranslogReader reader = openReader(translogFile, 0)) { + try (final Translog.Snapshot snapshot = reader.newSnapshot()) { + while(snapshot.next() != null) { + } } } @@ -151,20 +140,29 @@ public class TranslogVersionTests extends ElasticsearchTestCase { try { Path translogFile = getDataPath("/org/elasticsearch/index/translog/translog-v1-truncated.binary"); assertThat("test file should exist", Files.exists(translogFile), equalTo(true)); - TranslogStream stream = TranslogStreams.translogStreamFor(translogFile); - try (StreamInput in = stream.openInput(translogFile)) { - while (true) { - try { - stream.read(in); - } catch (EOFException e) { - break; + try (ImmutableTranslogReader reader = openReader(translogFile, 0)) { + try (final Translog.Snapshot snapshot = reader.newSnapshot()) { + while(snapshot.next() != null) { + } } } fail("should have thrown an exception about the body being truncated"); - } catch (TruncatedTranslogException e) { + } catch (TranslogCorruptedException e) { assertThat("translog truncated: " + e.getMessage(), - e.getMessage().contains("reached premature end of file, translog is truncated"), equalTo(true)); + e.getMessage().contains("operation size is corrupted must be"), equalTo(true)); + } + } + + public ImmutableTranslogReader openReader(Path path, long id) throws IOException { + FileChannel channel = FileChannel.open(path, StandardOpenOption.READ); + try { + final ChannelReference raf = new ChannelReference(path, id, channel, null); + ImmutableTranslogReader reader = ImmutableTranslogReader.open(raf, new Checkpoint(Files.size(path), TranslogReader.UNKNOWN_OP_COUNT, id), null); + channel = null; + return reader; + } finally { + IOUtils.close(channel); } } } diff --git a/src/test/java/org/elasticsearch/indices/recovery/RecoveryStateTest.java b/src/test/java/org/elasticsearch/indices/recovery/RecoveryStateTest.java index cc8e7c5fae8..6d3c5fe6a5c 100644 --- a/src/test/java/org/elasticsearch/indices/recovery/RecoveryStateTest.java +++ b/src/test/java/org/elasticsearch/indices/recovery/RecoveryStateTest.java @@ -21,7 +21,6 @@ package org.elasticsearch.indices.recovery; import org.elasticsearch.Version; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.io.stream.BytesStreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.Streamable; @@ -73,7 +72,7 @@ public class RecoveryStateTest extends ElasticsearchTestCase { BytesStreamOutput out = new BytesStreamOutput(); source.writeTo(out); out.close(); - StreamInput in = new BytesStreamInput(out.bytes()); + StreamInput in = StreamInput.wrap(out.bytes()); T obj = deserialize(in); lastRead = obj; return obj; diff --git a/src/test/java/org/elasticsearch/indices/stats/IndexStatsTests.java b/src/test/java/org/elasticsearch/indices/stats/IndexStatsTests.java index 96d875b2e18..0c5344cd71e 100644 --- a/src/test/java/org/elasticsearch/indices/stats/IndexStatsTests.java +++ b/src/test/java/org/elasticsearch/indices/stats/IndexStatsTests.java @@ -35,7 +35,7 @@ import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchType; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.io.stream.BytesStreamInput; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.Settings; @@ -615,7 +615,7 @@ public class IndexStatsTests extends ElasticsearchIntegrationTest { flags.writeTo(out); out.close(); BytesReference bytes = out.bytes(); - CommonStatsFlags readStats = CommonStatsFlags.readCommonStatsFlags(new BytesStreamInput(bytes)); + CommonStatsFlags readStats = CommonStatsFlags.readCommonStatsFlags(StreamInput.wrap(bytes)); for (Flag flag : values) { assertThat(flags.isSet(flag), equalTo(readStats.isSet(flag))); } @@ -629,7 +629,7 @@ public class IndexStatsTests extends ElasticsearchIntegrationTest { flags.writeTo(out); out.close(); BytesReference bytes = out.bytes(); - CommonStatsFlags readStats = CommonStatsFlags.readCommonStatsFlags(new BytesStreamInput(bytes)); + CommonStatsFlags readStats = CommonStatsFlags.readCommonStatsFlags(StreamInput.wrap(bytes)); for (Flag flag : values) { assertThat(flags.isSet(flag), equalTo(readStats.isSet(flag))); } diff --git a/src/test/java/org/elasticsearch/recovery/RecoveryWhileUnderLoadTests.java b/src/test/java/org/elasticsearch/recovery/RecoveryWhileUnderLoadTests.java index 000325a3cb4..76f3d8f8c52 100644 --- a/src/test/java/org/elasticsearch/recovery/RecoveryWhileUnderLoadTests.java +++ b/src/test/java/org/elasticsearch/recovery/RecoveryWhileUnderLoadTests.java @@ -159,7 +159,6 @@ public class RecoveryWhileUnderLoadTests extends ElasticsearchIntegrationTest { } @Test - @TestLogging("action.search.type:TRACE,action.admin.indices.refresh:TRACE") @Slow public void recoverWhileUnderLoadWithReducedAllowedNodes() throws Exception { logger.info("--> creating test index ..."); @@ -231,7 +230,6 @@ public class RecoveryWhileUnderLoadTests extends ElasticsearchIntegrationTest { } @Test - @TestLogging("action.search.type:TRACE,action.admin.indices.refresh:TRACE,action.index:TRACE,action.support.replication:TRACE,cluster.service:DEBUG") @Slow public void recoverWhileRelocating() throws Exception { final int numShards = between(2, 10); diff --git a/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java b/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java index 67991a896ea..53099aaa42e 100644 --- a/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java +++ b/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java @@ -108,9 +108,10 @@ import org.elasticsearch.index.merge.policy.MergePolicyProvider; import org.elasticsearch.index.merge.policy.TieredMergePolicyProvider; import org.elasticsearch.index.merge.scheduler.ConcurrentMergeSchedulerProvider; import org.elasticsearch.index.merge.scheduler.MergeSchedulerModule; +import org.elasticsearch.index.translog.TranslogConfig; import org.elasticsearch.index.translog.TranslogService; import org.elasticsearch.index.translog.Translog; -import org.elasticsearch.index.translog.TranslogFile; +import org.elasticsearch.index.translog.TranslogWriter; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.cache.query.IndicesQueryCache; import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache; @@ -498,7 +499,7 @@ public abstract class ElasticsearchIntegrationTest extends ElasticsearchTestCase } if (random.nextBoolean()) { - builder.put(Translog.INDEX_TRANSLOG_FS_TYPE, RandomPicks.randomFrom(random, TranslogFile.Type.values()).name()); + builder.put(TranslogConfig.INDEX_TRANSLOG_FS_TYPE, RandomPicks.randomFrom(random, TranslogWriter.Type.values()).name()); } if (random.nextBoolean()) { @@ -586,7 +587,7 @@ public abstract class ElasticsearchIntegrationTest extends ElasticsearchTestCase builder.put(TranslogService.INDEX_TRANSLOG_DISABLE_FLUSH, random.nextBoolean()); } if (random.nextBoolean()) { - builder.put(Translog.INDEX_TRANSLOG_DURABILITY, RandomPicks.randomFrom(random, Translog.Durabilty.values())); + builder.put(TranslogConfig.INDEX_TRANSLOG_DURABILITY, RandomPicks.randomFrom(random, Translog.Durabilty.values())); } return builder; } diff --git a/src/test/java/org/elasticsearch/test/InternalTestCluster.java b/src/test/java/org/elasticsearch/test/InternalTestCluster.java index 505c9f49d48..5d8209b753e 100644 --- a/src/test/java/org/elasticsearch/test/InternalTestCluster.java +++ b/src/test/java/org/elasticsearch/test/InternalTestCluster.java @@ -80,7 +80,8 @@ import org.elasticsearch.index.shard.IndexShardState; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.IndexStoreModule; import org.elasticsearch.index.translog.Translog; -import org.elasticsearch.index.translog.TranslogFile; +import org.elasticsearch.index.translog.TranslogConfig; +import org.elasticsearch.index.translog.TranslogWriter; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService; @@ -440,11 +441,11 @@ public final class InternalTestCluster extends TestCluster { } if (random.nextBoolean()) { - builder.put(Translog.INDEX_TRANSLOG_FS_TYPE, RandomPicks.randomFrom(random, TranslogFile.Type.values())); + builder.put(TranslogConfig.INDEX_TRANSLOG_FS_TYPE, RandomPicks.randomFrom(random, TranslogWriter.Type.values())); if (rarely(random)) { - builder.put(Translog.INDEX_TRANSLOG_SYNC_INTERVAL, 0); // 0 has special meaning to sync each op + builder.put(TranslogConfig.INDEX_TRANSLOG_SYNC_INTERVAL, 0); // 0 has special meaning to sync each op } else { - builder.put(Translog.INDEX_TRANSLOG_SYNC_INTERVAL, RandomInts.randomIntBetween(random, 100, 5000)); + builder.put(TranslogConfig.INDEX_TRANSLOG_SYNC_INTERVAL, RandomInts.randomIntBetween(random, 100, 5000)); } } diff --git a/src/test/java/org/elasticsearch/test/hamcrest/ElasticsearchAssertions.java b/src/test/java/org/elasticsearch/test/hamcrest/ElasticsearchAssertions.java index ab59da837e4..41d56121422 100644 --- a/src/test/java/org/elasticsearch/test/hamcrest/ElasticsearchAssertions.java +++ b/src/test/java/org/elasticsearch/test/hamcrest/ElasticsearchAssertions.java @@ -58,7 +58,6 @@ import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.metadata.IndexTemplateMetaData; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.io.stream.BytesStreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.Streamable; @@ -635,7 +634,7 @@ public class ElasticsearchAssertions { ((ActionRequest) streamable).validate(); } BytesReference orig = serialize(version, streamable); - StreamInput input = new BytesStreamInput(orig); + StreamInput input = StreamInput.wrap(orig); input.setVersion(version); newInstance.readFrom(input); assertThat("Stream should be fully read with version [" + version + "] for streamable [" + streamable + "]", input.available(), equalTo(0)); diff --git a/src/test/java/org/elasticsearch/test/transport/MockTransportService.java b/src/test/java/org/elasticsearch/test/transport/MockTransportService.java index 5e8b135a61c..c5bc4db69d4 100644 --- a/src/test/java/org/elasticsearch/test/transport/MockTransportService.java +++ b/src/test/java/org/elasticsearch/test/transport/MockTransportService.java @@ -24,8 +24,8 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.component.Lifecycle; import org.elasticsearch.common.component.LifecycleListener; import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.io.stream.BytesStreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.BoundTransportAddress; @@ -232,7 +232,7 @@ public class MockTransportService extends TransportService { BytesStreamOutput bStream = new BytesStreamOutput(); request.writeTo(bStream); final TransportRequest clonedRequest = reg.newRequest(); - clonedRequest.readFrom(new BytesStreamInput(bStream.bytes())); + clonedRequest.readFrom(StreamInput.wrap(bStream.bytes())); threadPool.schedule(delay, ThreadPool.Names.GENERIC, new AbstractRunnable() { @Override diff --git a/src/test/java/org/elasticsearch/threadpool/ThreadPoolSerializationTests.java b/src/test/java/org/elasticsearch/threadpool/ThreadPoolSerializationTests.java index 226937b5804..d89fa06371f 100644 --- a/src/test/java/org/elasticsearch/threadpool/ThreadPoolSerializationTests.java +++ b/src/test/java/org/elasticsearch/threadpool/ThreadPoolSerializationTests.java @@ -20,7 +20,6 @@ package org.elasticsearch.threadpool; import org.elasticsearch.Version; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.io.stream.BytesStreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; @@ -55,7 +54,7 @@ public class ThreadPoolSerializationTests extends ElasticsearchTestCase { output.setVersion(Version.CURRENT); info.writeTo(output); - StreamInput input = new BytesStreamInput(output.bytes()); + StreamInput input = StreamInput.wrap(output.bytes()); ThreadPool.Info newInfo = new ThreadPool.Info(); newInfo.readFrom(input); @@ -68,7 +67,7 @@ public class ThreadPoolSerializationTests extends ElasticsearchTestCase { output.setVersion(Version.CURRENT); info.writeTo(output); - StreamInput input = new BytesStreamInput(output.bytes()); + StreamInput input = StreamInput.wrap(output.bytes()); ThreadPool.Info newInfo = new ThreadPool.Info(); newInfo.readFrom(input); diff --git a/src/test/java/org/elasticsearch/transport/TransportMessageTests.java b/src/test/java/org/elasticsearch/transport/TransportMessageTests.java index f2f7aa70bb6..5d06ee6f69f 100644 --- a/src/test/java/org/elasticsearch/transport/TransportMessageTests.java +++ b/src/test/java/org/elasticsearch/transport/TransportMessageTests.java @@ -20,8 +20,8 @@ package org.elasticsearch.transport; import org.elasticsearch.Version; -import org.elasticsearch.common.io.stream.BytesStreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.test.ElasticsearchTestCase; import org.junit.Test; @@ -43,7 +43,7 @@ public class TransportMessageTests extends ElasticsearchTestCase { BytesStreamOutput out = new BytesStreamOutput(); out.setVersion(Version.CURRENT); message.writeTo(out); - BytesStreamInput in = new BytesStreamInput(out.bytes()); + StreamInput in = StreamInput.wrap(out.bytes()); in.setVersion(Version.CURRENT); message = new Message(); message.readFrom(in); diff --git a/src/test/resources/org/elasticsearch/index/translog/legacy_translogs.zip b/src/test/resources/org/elasticsearch/index/translog/legacy_translogs.zip new file mode 100644 index 0000000000000000000000000000000000000000..c77e678801dc92699b4943de74046e834a578173 GIT binary patch literal 6807 zcma)B2|UzW`yXR4WErv*YDO6(jCE{@v6L8D6I#?OOqOgzE?LGP(klCw(v;+;3@J^P zlqgHtB9%}gl!&V?{?9M(`}yDA_ul)t^*Nt8<~+~$Jm);$^L@_mH+I(C7;zK|g+=up zEVEQ35L%9*Q7BMTG_*2ng{z;;pKot)-!(p{b!y)*z9co%m7Q-xM63kw!nd z3dM!_%7sGxz`5MlY};usyKUORs<4jFXZL~a<;!iVeyJM-P8~-JaMfp6@Y%(xT;0K{OzU9jd`P8DinD2D-2LpZwa|F` z***OJZXX%CW%&uZW%ju}ZKk!i;KEOD zV~6ql=+_9q-z)ClU{07X+NZN$2V6p5v%}>2*}5oyHyM~i2m-VOi%S5Uy8^5VmXmf; zHooC2o;5HO3GF*H4H(m7)BMqZ08vKb#xK$)2@pQo-|doanKP`-aRp@qSZaeMtY=QI zO96>>k{|w-{1IOy#0U1Ru(Seb@4+u-tm^2H$$8D2TNKHa96b3BXCs-@{gznmGQz6n z2~OK17myQ44Y(L-7I4iSBh@m-Ip^wJU5U3#Dn z(eF&|xNb9*?hb#ShPD%0J2Z7@{=trWPYyWEOYCVCpjVd!%#)Xe?S3M2n37xW6!214 z>d(onsiCQ_NBTQ5r$6u8(G?_}G#zy24qAP%J&%8v*6@T6zt|mjwRA7rJIrL6Q$q%- zSM35kXqjD?wKLBO7dL*=pI-Q~j`LYR^5D(g$^||MKrjR(HN08b#*GV!`)CxL1E zD9{oNe39I95_QX{D@Oy)K_z*GHgFUsRl(7qOaBMGfme}y&_w**a^rw70AdVIl0eP% zFnvcvXTLm{^Y^o>c>UenlU6tgOs+&|tP4YYFeEe1p!thH2VhDea$}?c6dcR_fV6*Q zA@O!nRNo|UMYv0b77k4V0)kb9!72=475U($M%k-bwiv5hw#fW(K-IwA1>E+Be_Q)A zdp@4cNKpYPEMEH;?9n7?>*@a;doDk>vRxf0oj7ef#;<4cfd2!^e_9}zS8`m;mA%p9 zA(t*=h^Z8>d53h$ez8&q1Cj=RktrP;U;Kgg(CB&Z2QU0Zj%>D*2B_Um!FC-W0^&*sPJrrYN}ZKLG2 z@m`?%#CNdRAsxg@0_DlQz$%N?qEObxDmObw}RriKbtmrS*< zTju+jdYab1^RReOs0qTKQ%{>jB5D0C_450C?an*PHU)Vckid!aOfzl`V$iHTVSNwZ zzW3+wGRSAn3mzdbo?H~SNxj_?B>s>gACiTauy)X5iFpYIKBuVGKNC$uvmc!o{A%&x zYTwhD<^0==zj)1cPepkZj4l*tjZRoq=L{Q`<6|s--J*EpWPE)5@XXGfjSmyxboQetADVqdt^BYb$D=(rShm7nb&tjHB7} z=U&_CFB+HpY*oY3^Kxf-l8$hK$IOWZD-5ye0M6KGB*F#5+NLnkE4ObGdUY|GIfV7C z#oC-T#U^LF?l%K6=94%OoS#ZMfMsl6Dz9b{}Mw=;yWfg-ZleH&t zakS^CqQS(E>qN7N++81xyA4cMUxY5fb38sNrB1tq`A+9@;c@U3ZfM@JlnfKxN4;fF z*Q|yfuW8nP*}y%3rcfL@)p4RudyQbx^wQ%10BbVKn*SoVM#|nMZAaGIZ$9}D5CLcW zq;l4Wah9ejPWk%voB-H8=nC=ilT|@?7K@#8DDyV=$jL($sZA8sfW11UJfrgQ+DDHe zjO<5#t`DTMiHelp?vXcBF|1}QoLKnl44qU!lchM@Uw9g9$!f-8t2Am`YkW+g0b_~a-|p-u7G9^qb4oJ@1zzhY*v2l=WbLX*~rM1F}7wHW=4 z0=@Ll)RFy~ojF;=)@w=eh9MRnIcyWX_eQ` z%o;IEP5Nwh;E5xQt;S>a9DT7F-RUXM9PIE{K^Ay%in4W96`7#JtrMpJ*>l1YsQcW( zF^~7{L{S)-O46d*CJnxJQHb29)j1-Fy}c>pE2M*#KvL>!R{{?k?5klz8N`F9o;;j$ zj|ygf*u|TdWMBrThT$r;lK!XK`g{zRq4+1T3vKa(!}rxqU>7!G%`~T?uJKDk+8Vn) zNpFDIG?}ITAuR}+h^b4qYK_;bPEP^m>noMFLT;0&EJ~ok**#!aEEf8@b^+LM2h~uR z(O`@$A@GyQs10~lkFX0K!&~qW!L^`R} zyq9J*Z^qL3GJdZK*%|o!qkKcQk_*G9^l=ssqr^dU(tf^>;c;sZkNoSczT#NclkRKD zud4oq&m#54ZYR0F;i*4=N3!vB`u-k<$8)E~goffMoK5bHIBBwH?*^PrULQW^OZl@SWDjuWQ zB^jpp2kfaT)S~Mo zfJ+1o+Bar7n}eFaA~{oRElzc11zoi5un9U`1rhq2el%uLv2U961d6D|(SyW8BA zcrZn|q{vR#p8Kg#B~EB}#jfjO;^H>F_wd+@gFM@3ejVX&q*&N@jw<0JOf3jYNADLF z#>|cNwe4^3^ZlMbv$PZ)Imhg7nh&@7<`+#l`1?7%_!=WQ z`#z)a`g2*0Mf(W*qAzXUS1K4xkLlGr^rhtGe9i0=QeAY!S>fLn!w1_eQ41EOA#@LM zE`P9`HH@LmfF9a`q9!Md{VK>IAqMiy>YzW04 zM}6VamTkFT%$>2!^MWlnhR;F?pmsY1udO-4%*mrho2P_VQGTN>&wxa#E(sFGe*Zou zPtpj8Kl7>!j+o=HI!-dPcDE8bZCF?Vv-!InhSijhdoFM7Cr>inMEDw5eF9{^1|Jo5 zeVB>0Iafote$pm?Bb-eFlR19fcCYk9ke4Q99z{5%1Kjn-Q3c3}kjECyl}N!Oveq)_8$f~Uwl{Ol~Dx55EDOdreD)) zlO}~9Xf@d6eLI`>oaNPdv}gxX!SvYQ9{2fs<^0Y~2}vZ@aI$H)BpDUck2V9yNBlst zT(GGW8i30mO$E8bB2M6pT@U0CVOO^>>75lVuy6M;goVDSzIR}@%sk_rPcZbycaH4G zo%0)(gAMbGJ@d*n97>m!f?pOVI1yscBy8-@7zJUE;Gs<2e;$1o*QgyKK{@Yx9;wj^ zPy4%x7CahMU5e6TaDkH?3<@>WxEQg1ROSOOiQ5oYxrXWewKOQ@xA+?#$LSCu;B?Z^ zcE1vsfQ-+ZUFQjRfG^;Ki5e^d;aORLCG!6QT{ zZ)V$(uly1}d!{?;qybjd!|UcfYb)N0~EQs>la;ry8k)>UZwf!x|u_T4h8xv#!GGuc6O z%}2|2OubVdmwZ)8g(xQNvb^MMXehcqbV1hqm*74c&L&IUq_MZyn796XAOBG4hwDN3 zENzLKRU@4s)VX`{lHF4iLX2cbQH>G_cT*hwuqCc$o7$upBjWNU{i|E)ha*+77j0YZ zFYEX3{w#^Xhw-vU6*Q(knPO;pRuG;|Ze#h3lM0Sd!?fuVXs@2YsD#=KOn zO$ULGWPETuzYD5Ir%+?(y322%{)F;93x6<&Lm)nIqS`8BxPI1k*Zj)`l^7Iioy=d{ z<~8&+NV*+6!**Jr)2UU+tG}q{gYP(o5Td@q4%p_~EbisFFD%=m;4u$lf)%Qv5 z*>Q-J$qr>_?5K*8KCUd9f+d<)oue7;8yag+m=!J;XPT!Tzp{Br?@jx{?e-&vG5Hm} zbB49^pTetFcc$NMe2`Jqms4 zC%<%m6uR#kpy^dekEtIBEd03tP_>TQro&FC#brLpXyL(2ZMhEx%2QDHQRPhmG4@;( z{y|c#i15kljwo&}S*}3s4w_>u39W>>U^r3a7#oPz6TUFtkQpz4GeyBoelg^YUF>$W z9+yY;LAC`~D(169?01=-&4rw4wq@zF)yir{{6!`;2XA0)PF%rfeUZOt;c;*?!_J8g zFPK=5ZV>!t`#RLoX_q!_bldWag~ug!@q;XAxl}_yi23an`hGmk7kYOsRme z-z6z0uN;CKd-Ez>fsY6lnNX}DeTMrB(JY%-if4A6x-Jgq(h&PA;{qCjC?%Lr*t}AP zE80Bn?s_#jKn%@1=ptC$4}6pR#?Dy&f7n1 z3W8Ysc-}URDIr%;TdX?AVH7r@c$3lk^P|XwWM&$x_!Xjl0`*aW$$6x=HfdXM0!^}I z2t1{E`pE~_sZC6J+x$)y3Z`>J2m5_4HyBbkkud0g&sh~P)-(A(*(?m1lc;lMyJv%1 z&cIZ?1RKjE<6_`T#y&ZilnMI2V92aZtMEcZIw2@B$qJSvgOe<5EkMdEky�#8?Wh zSz8}fudbxH4@OO~R93Y$#35|zb5#GU25d_b!RY>T9pt%TJ?^luU<++MIG~HF-fQ)_ zp0!B}#~+qB`**qR3Lr zxq^E;0J(`^PLtgQ9Yy^0@*Jcd*iw9^9uFXMiDQeNcXXV2)?FUF@ySnCzlqpcbD`I9|J6?m((seR%D>_`LZapG(>wv7J``3HHAW++Shy@FEQ0h5t&B JE!Q7#{|CPog75$U literal 0 HcmV?d00001 From 5c9c4776cd3dfb634ec9292eb73682f4b987413e Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Wed, 13 May 2015 14:13:10 +0200 Subject: [PATCH 03/56] Internal: remove dependency on hppc:esoteric. The esoteric classifier contains in particular maps that take bytes or doubles as keys. In the byte case, we can just use integer, and in the double case we can use their long bits instead. --- pom.xml | 7 ------- .../index/mapper/core/NumberFieldMapper.java | 2 -- .../percolator/PercolatorService.java | 6 +++--- .../index/fielddata/LongFieldDataTests.java | 16 ++++++++-------- 4 files changed, 11 insertions(+), 20 deletions(-) diff --git a/pom.xml b/pom.xml index a396fa4f8c9..83e45062a88 100644 --- a/pom.xml +++ b/pom.xml @@ -229,13 +229,6 @@ 0.7.1 - - com.carrotsearch - hppc - 0.7.1 - esoteric - - joda-time joda-time diff --git a/src/main/java/org/elasticsearch/index/mapper/core/NumberFieldMapper.java b/src/main/java/org/elasticsearch/index/mapper/core/NumberFieldMapper.java index 088be23c111..05f4044d142 100644 --- a/src/main/java/org/elasticsearch/index/mapper/core/NumberFieldMapper.java +++ b/src/main/java/org/elasticsearch/index/mapper/core/NumberFieldMapper.java @@ -19,9 +19,7 @@ package org.elasticsearch.index.mapper.core; -import com.carrotsearch.hppc.DoubleHashSet; import com.carrotsearch.hppc.LongArrayList; -import com.carrotsearch.hppc.LongHashSet; import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.NumericTokenStream; diff --git a/src/main/java/org/elasticsearch/percolator/PercolatorService.java b/src/main/java/org/elasticsearch/percolator/PercolatorService.java index 3edab192d33..9fd56f11778 100644 --- a/src/main/java/org/elasticsearch/percolator/PercolatorService.java +++ b/src/main/java/org/elasticsearch/percolator/PercolatorService.java @@ -18,7 +18,7 @@ */ package org.elasticsearch.percolator; -import com.carrotsearch.hppc.ByteObjectHashMap; +import com.carrotsearch.hppc.IntObjectHashMap; import com.google.common.collect.Lists; import org.apache.lucene.index.LeafReaderContext; @@ -109,7 +109,7 @@ public class PercolatorService extends AbstractComponent { public final static String TYPE_NAME = ".percolator"; private final IndicesService indicesService; - private final ByteObjectHashMap percolatorTypes; + private final IntObjectHashMap percolatorTypes; private final PageCacheRecycler pageCacheRecycler; private final BigArrays bigArrays; private final ClusterService clusterService; @@ -153,7 +153,7 @@ public class PercolatorService extends AbstractComponent { single = new SingleDocumentPercolatorIndex(cache); multi = new MultiDocumentPercolatorIndex(cache); - percolatorTypes = new ByteObjectHashMap<>(6); + percolatorTypes = new IntObjectHashMap<>(6); percolatorTypes.put(countPercolator.id(), countPercolator); percolatorTypes.put(queryCountPercolator.id(), queryCountPercolator); percolatorTypes.put(matchPercolator.id(), matchPercolator); diff --git a/src/test/java/org/elasticsearch/index/fielddata/LongFieldDataTests.java b/src/test/java/org/elasticsearch/index/fielddata/LongFieldDataTests.java index 2b293bd088f..09a24a4283c 100644 --- a/src/test/java/org/elasticsearch/index/fielddata/LongFieldDataTests.java +++ b/src/test/java/org/elasticsearch/index/fielddata/LongFieldDataTests.java @@ -19,9 +19,7 @@ package org.elasticsearch.index.fielddata; -import com.carrotsearch.hppc.DoubleHashSet; import com.carrotsearch.hppc.LongHashSet; -import com.carrotsearch.hppc.cursors.DoubleCursor; import com.carrotsearch.hppc.cursors.LongCursor; import org.apache.lucene.document.Document; @@ -37,7 +35,9 @@ import java.util.ArrayList; import java.util.List; import java.util.Random; -import static org.hamcrest.Matchers.*; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThan; /** * Tests for all integer types (byte, short, int, long). @@ -348,7 +348,7 @@ public class LongFieldDataTests extends AbstractNumericFieldDataTests { final SortedNumericDocValues data = atomicFieldData.getLongValues(); final SortedNumericDoubleValues doubleData = atomicFieldData.getDoubleValues(); final LongHashSet set = new LongHashSet(); - final DoubleHashSet doubleSet = new DoubleHashSet(); + final LongHashSet doubleSet = new LongHashSet(); for (int i = 0; i < values.size(); ++i) { final LongHashSet v = values.get(i); @@ -365,17 +365,17 @@ public class LongFieldDataTests extends AbstractNumericFieldDataTests { } assertThat(set, equalTo(v)); - final DoubleHashSet doubleV = new DoubleHashSet(); + final LongHashSet doubleV = new LongHashSet(); for (LongCursor c : v) { - doubleV.add(c.value); + doubleV.add(Double.doubleToLongBits(c.value)); } doubleSet.clear(); doubleData.setDocument(i); numValues = doubleData.count(); double prev = 0; for (int j = 0; j < numValues; j++) { - double current; - doubleSet.add(current = doubleData.valueAt(j)); + double current = doubleData.valueAt(j); + doubleSet.add(Double.doubleToLongBits(current)); if (j > 0) { assertThat(prev, lessThan(current)); } From 96df7ba7eb72380be4e0a0b2cdaafd8f78173811 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 13 May 2015 20:32:25 +0200 Subject: [PATCH 04/56] Applied review comments from @mikemccand - Renamed TranslogSnapshot to MultiSnapshot - moved legacy logic for trucation into LegacyTranslogReaderBase - made several methods private and pkg private where applicable - renamed arguments for consistency --- .../index/translog/Checkpoint.java | 6 +-- .../translog/ImmutableTranslogReader.java | 4 +- .../translog/LegacyTranslogReaderBase.java | 6 ++- ...anslogSnapshot.java => MultiSnapshot.java} | 48 +++++++------------ .../index/translog/Translog.java | 44 ++++++++++------- 5 files changed, 55 insertions(+), 53 deletions(-) rename src/main/java/org/elasticsearch/index/translog/{TranslogSnapshot.java => MultiSnapshot.java} (57%) diff --git a/src/main/java/org/elasticsearch/index/translog/Checkpoint.java b/src/main/java/org/elasticsearch/index/translog/Checkpoint.java index 586b1f7d8f7..9b73d0346a5 100644 --- a/src/main/java/org/elasticsearch/index/translog/Checkpoint.java +++ b/src/main/java/org/elasticsearch/index/translog/Checkpoint.java @@ -55,14 +55,14 @@ class Checkpoint { generation = in.readLong(); } - void write(FileChannel channel) throws IOException { + private void write(FileChannel channel) throws IOException { byte[] buffer = new byte[BUFFER_SIZE]; final ByteArrayDataOutput out = new ByteArrayDataOutput(buffer); write(out); Channels.writeToChannel(buffer, channel); } - public void write(DataOutput out) throws IOException { + private void write(DataOutput out) throws IOException { out.writeLong(offset); out.writeInt(numOps); out.writeLong(generation); @@ -70,7 +70,7 @@ class Checkpoint { @Override public String toString() { - return "TranslogInfo{" + + return "Checkpoint{" + "offset=" + offset + ", numOps=" + numOps + ", translogFileGeneration= " + generation + diff --git a/src/main/java/org/elasticsearch/index/translog/ImmutableTranslogReader.java b/src/main/java/org/elasticsearch/index/translog/ImmutableTranslogReader.java index 6d141108b5c..1d6d3b45a63 100644 --- a/src/main/java/org/elasticsearch/index/translog/ImmutableTranslogReader.java +++ b/src/main/java/org/elasticsearch/index/translog/ImmutableTranslogReader.java @@ -37,8 +37,8 @@ public class ImmutableTranslogReader extends TranslogReader { * Create a snapshot of translog file channel. The length parameter should be consistent with totalOperations and point * at the end of the last operation in this snapshot. */ - public ImmutableTranslogReader(long generation, ChannelReference channelReference, long offset, long length, int totalOperations) { - super(generation, channelReference, offset); + public ImmutableTranslogReader(long generation, ChannelReference channelReference, long firstOperationOffset, long length, int totalOperations) { + super(generation, channelReference, firstOperationOffset); this.length = length; this.totalOperations = totalOperations; } diff --git a/src/main/java/org/elasticsearch/index/translog/LegacyTranslogReaderBase.java b/src/main/java/org/elasticsearch/index/translog/LegacyTranslogReaderBase.java index eff0d0675db..d9e9e17f792 100644 --- a/src/main/java/org/elasticsearch/index/translog/LegacyTranslogReaderBase.java +++ b/src/main/java/org/elasticsearch/index/translog/LegacyTranslogReaderBase.java @@ -47,7 +47,11 @@ class LegacyTranslogReaderBase extends ImmutableTranslogReader { if (position >= sizeInBytes()) { // this is the legacy case.... return null; } - return readOperation(); + try { + return readOperation(); + } catch (TruncatedTranslogException ex) { + return null; // legacy case + } } }; } diff --git a/src/main/java/org/elasticsearch/index/translog/TranslogSnapshot.java b/src/main/java/org/elasticsearch/index/translog/MultiSnapshot.java similarity index 57% rename from src/main/java/org/elasticsearch/index/translog/TranslogSnapshot.java rename to src/main/java/org/elasticsearch/index/translog/MultiSnapshot.java index dfd69eddf4b..b76214dc2e7 100644 --- a/src/main/java/org/elasticsearch/index/translog/TranslogSnapshot.java +++ b/src/main/java/org/elasticsearch/index/translog/MultiSnapshot.java @@ -20,42 +20,40 @@ package org.elasticsearch.index.translog; import org.apache.lucene.store.AlreadyClosedException; -import org.apache.lucene.util.IOUtils; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasables; -import org.elasticsearch.common.logging.ESLogger; import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; -public class TranslogSnapshot implements Translog.Snapshot { +/** + * A snapshot composed out of multiple snapshots + */ +final class MultiSnapshot implements Translog.Snapshot { - private final List orderedTranslogs; + private final Translog.Snapshot[] translogs; private AtomicBoolean closed = new AtomicBoolean(false); private final int estimatedTotalOperations; - private int currentTranslog; + private int index; /** - * Create a snapshot of translog file channel. The length parameter should be consistent with totalOperations and point - * at the end of the last operation in this snapshot. + * Creates a new point in time snapshot of the given snapshots. Those snapshots are always iterated in-order. */ - public TranslogSnapshot(List orderedTranslogs) { - this.orderedTranslogs = orderedTranslogs; + MultiSnapshot(Translog.Snapshot[] translogs) { + this.translogs = translogs; int ops = 0; - for (Translog.Snapshot translog : orderedTranslogs) { + for (Translog.Snapshot translog : translogs) { final int tops = translog.estimatedTotalOperations(); - if (tops < 0) { + if (tops == TranslogReader.UNKNOWN_OP_COUNT) { ops = TranslogReader.UNKNOWN_OP_COUNT; break; } + assert tops >= 0 : "tops must be positive but was: " + tops; ops += tops; } estimatedTotalOperations = ops; - currentTranslog = 0; + index = 0; } @@ -67,20 +65,10 @@ public class TranslogSnapshot implements Translog.Snapshot { @Override public Translog.Operation next() throws IOException { ensureOpen(); - for (; currentTranslog < orderedTranslogs.size(); currentTranslog++) { - final Translog.Snapshot current = orderedTranslogs.get(currentTranslog); - Translog.Operation op = null; - try { - op = current.next(); - } catch (TruncatedTranslogException e) { - if (estimatedTotalOperations == TranslogReader.UNKNOWN_OP_COUNT) { - // legacy translog file - can have UNKNOWN_OP_COUNT - // file is empty or header has been half-written and should be ignored - } else { - throw e; - } - } - if (op != null) { + for (; index < translogs.length; index++) { + final Translog.Snapshot current = translogs[index]; + Translog.Operation op = current.next(); + if (op != null) { // if we are null we move to the next snapshot return op; } } @@ -96,7 +84,7 @@ public class TranslogSnapshot implements Translog.Snapshot { @Override public void close() throws ElasticsearchException { if (closed.compareAndSet(false, true)) { - Releasables.close(orderedTranslogs); + Releasables.close(translogs); } } } diff --git a/src/main/java/org/elasticsearch/index/translog/Translog.java b/src/main/java/org/elasticsearch/index/translog/Translog.java index 6116fe9f8a6..7aeff4a624c 100644 --- a/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -433,7 +433,12 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC } /** - * Adds a create operation to the transaction log. + * Adds a created / delete / index operations to the transaction log. + * + * @see org.elasticsearch.index.translog.Translog.Operation + * @see org.elasticsearch.index.translog.Translog.Create + * @see org.elasticsearch.index.translog.Translog.Index + * @see org.elasticsearch.index.translog.Translog.Delete */ public Location add(Operation operation) throws TranslogException { ReleasableBytesStreamOutput out = new ReleasableBytesStreamOutput(bigArrays); @@ -467,23 +472,24 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC toOpen.add(currentCommittingTranslog); } toOpen.add(current); - return createdSnapshot(toOpen.toArray(new TranslogReader[toOpen.size()])); + return createSnapshot(toOpen.toArray(new TranslogReader[toOpen.size()])); } } - private Snapshot createdSnapshot(TranslogReader... translogs) { - ArrayList channelSnapshots = new ArrayList<>(); + private Snapshot createSnapshot(TranslogReader... translogs) { + Snapshot[] snapshots = new Snapshot[translogs.length]; boolean success = false; try { - for (TranslogReader translog : translogs) { - channelSnapshots.add(translog.newSnapshot()); + for (int i = 0; i < translogs.length; i++) { + snapshots[i] = translogs[i].newSnapshot(); } - Snapshot snapshot = new TranslogSnapshot(channelSnapshots); + + Snapshot snapshot = new MultiSnapshot(snapshots); success = true; return snapshot; } finally { if (success == false) { - Releasables.close(channelSnapshots); + Releasables.close(snapshots); } } } @@ -672,7 +678,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC public synchronized Snapshot snapshot() { ensureOpen(); - return createdSnapshot(orderedTranslogs.toArray(new TranslogReader[orderedTranslogs.size()])); + return createSnapshot(orderedTranslogs.toArray(new TranslogReader[orderedTranslogs.size()])); } @@ -1568,15 +1574,18 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC } static Translog.Operation readOperation(BufferedChecksumStreamInput in) throws IOException { - int opSize = in.readInt(); + final int opSize = in.readInt(); + if (opSize < 4) { // 4byte for the checksum + throw new TranslogCorruptedException("operation size must be at least 4 but was: " + opSize); + } Translog.Operation operation; try { - in.resetDigest(); // size is not part of the checksum? + in.resetDigest(); // size is not part of the checksum! if (in.markSupported()) { // if we can we validate the checksum first + // we are sometimes called when mark is not supported this is the case when + // we are sending translogs across the network - currently there is no way to prevent this unfortunately. in.mark(opSize); - if (opSize < 4) { // 4byte for the checksum - throw new TranslogCorruptedException("operation size must be at least 4 but was: " + opSize); - } + in.skip(opSize-4); verifyChecksum(in); in.reset(); @@ -1594,6 +1603,8 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC } public static void writeOperation(StreamOutput outStream, Translog.Operation op) throws IOException { + //TODO lets get rid of this crazy double writing here. + // We first write to a NoopStreamOutput to get the size of the // operation. We could write to a byte array and then send that as an // alternative, but here we choose to use CPU over allocating new @@ -1637,12 +1648,11 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC @Override public void prepareCommit() throws IOException { ensureOpen(); - TranslogWriter writer = null; try (ReleasableLock lock = writeLock.acquire()) { if (currentCommittingTranslog != null) { throw new IllegalStateException("already committing a translog with generation: " + currentCommittingTranslog.getGeneration()); } - writer = current; + final TranslogWriter writer = current; writer.sync(); currentCommittingTranslog = current.immutableReader(); Path checkpoint = location.resolve(CHECKPOINT_FILE_NAME); @@ -1664,7 +1674,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC assert writer.syncNeeded() == false : "old translog writer must not need a sync"; } catch (Throwable t) { - close(); // tragic event + IOUtils.closeWhileHandlingException(this); // tragic event throw t; } } From 19155d8fd82b53533d8d6b8883eb3b8fe5a1dbb0 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 13 May 2015 20:52:59 +0200 Subject: [PATCH 05/56] Simplify new translog creation - pending files are not needed anymore --- .../index/translog/TranslogWriter.java | 16 +++------------- 1 file changed, 3 insertions(+), 13 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java b/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java index df1beef0870..2290dd69d87 100644 --- a/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java +++ b/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java @@ -66,27 +66,18 @@ public class TranslogWriter extends TranslogReader { } public static TranslogWriter create(Type type, ShardId shardId, String translogUUID, long fileGeneration, Path file, Callback onClose, int bufferSize) throws IOException { - Path pendingFile = file.resolveSibling("pending_" + file.getFileName()); final BytesRef ref = new BytesRef(translogUUID); final int headerLength = CodecUtil.headerLength(TRANSLOG_CODEC) + ref.length + RamUsageEstimator.NUM_BYTES_INT; - /** - * We first create pending_translog, write the header, fsync it and write a checkpoint. Then we rename the pending file into - * the actual file such that there is never a file without valid header. If the header is missing it's corrupted - */ - try (FileChannel channel = FileChannel.open(pendingFile, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW)) { + final FileChannel channel = FileChannel.open(file, StandardOpenOption.WRITE, StandardOpenOption.READ, StandardOpenOption.CREATE_NEW); + try { // This OutputStreamDataOutput is intentionally not closed because // closing it will close the FileChannel - OutputStreamDataOutput out = new OutputStreamDataOutput(java.nio.channels.Channels.newOutputStream(channel)); + final OutputStreamDataOutput out = new OutputStreamDataOutput(java.nio.channels.Channels.newOutputStream(channel)); CodecUtil.writeHeader(out, TRANSLOG_CODEC, VERSION); out.writeInt(ref.length); out.writeBytes(ref.bytes, ref.offset, ref.length); channel.force(false); writeCheckpoint(headerLength, 0, file.getParent(), fileGeneration, StandardOpenOption.WRITE); - } - Files.move(pendingFile, file, StandardCopyOption.ATOMIC_MOVE); - FileChannel channel = FileChannel.open(file, StandardOpenOption.READ, StandardOpenOption.WRITE); - try { - channel.position(headerLength); final TranslogWriter writer = type.create(shardId, fileGeneration, new ChannelReference(file, fileGeneration, channel, onClose), bufferSize); return writer; } catch (Throwable throwable){ @@ -205,7 +196,6 @@ public class TranslogWriter extends TranslogReader { } } - /** * returns a new immutable reader which only exposes the current written operation * */ From 91cfba34850712dd12d44052d905fb35cc7e7ce8 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 13 May 2015 21:12:54 +0200 Subject: [PATCH 06/56] move reading opSize into the catch block to transform EOF into TruncatedTranslogException --- .../org/elasticsearch/index/translog/Translog.java | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/translog/Translog.java b/src/main/java/org/elasticsearch/index/translog/Translog.java index 7aeff4a624c..9d2d81f97ec 100644 --- a/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -1574,16 +1574,17 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC } static Translog.Operation readOperation(BufferedChecksumStreamInput in) throws IOException { - final int opSize = in.readInt(); - if (opSize < 4) { // 4byte for the checksum - throw new TranslogCorruptedException("operation size must be at least 4 but was: " + opSize); - } Translog.Operation operation; try { + final int opSize = in.readInt(); + if (opSize < 4) { // 4byte for the checksum + throw new AssertionError("operation size must be at least 4 but was: " + opSize); + } in.resetDigest(); // size is not part of the checksum! if (in.markSupported()) { // if we can we validate the checksum first // we are sometimes called when mark is not supported this is the case when - // we are sending translogs across the network - currently there is no way to prevent this unfortunately. + // we are sending translogs across the network with LZ4 compression enabled - currently there is no way s + // to prevent this unfortunately. in.mark(opSize); in.skip(opSize-4); @@ -1596,7 +1597,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC verifyChecksum(in); } catch (EOFException e) { throw new TruncatedTranslogException("reached premature end of file, translog is truncated", e); - } catch (AssertionError|Exception e) { + } catch (AssertionError | Exception e) { throw new TranslogCorruptedException("translog corruption while reading from stream", e); } return operation; From 08d7638ed19d6ea236caee967dbe7d06c234ce82 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Thu, 14 May 2015 22:57:53 +0200 Subject: [PATCH 07/56] apply review comments --- .../index/engine/EngineConfig.java | 2 +- .../index/engine/InternalEngine.java | 39 +++++++++---------- .../index/translog/Translog.java | 22 +++++------ 3 files changed, 31 insertions(+), 32 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/engine/EngineConfig.java b/src/main/java/org/elasticsearch/index/engine/EngineConfig.java index 19d5a145c10..5c3fe0b616a 100644 --- a/src/main/java/org/elasticsearch/index/engine/EngineConfig.java +++ b/src/main/java/org/elasticsearch/index/engine/EngineConfig.java @@ -200,7 +200,7 @@ public final class EngineConfig { } /** if true the engine will start even if the translog id in the commit point can not be found */ - public boolean forceNewTranlog() { + public boolean forceNewTranslog() { return forceNewTranslog; } diff --git a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 499a9495333..c68c3a114d1 100644 --- a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -35,7 +35,6 @@ import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.InfoStream; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.Version; import org.elasticsearch.cluster.routing.DjbHashFunction; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.lease.Releasable; @@ -134,7 +133,7 @@ public class InternalEngine extends Engine { try { writer = createWriter(); indexWriter = writer; - translog = openTranslog(engineConfig, writer, skipInitialTranslogRecovery || engineConfig.forceNewTranlog()); + translog = openTranslog(engineConfig, writer, skipInitialTranslogRecovery || engineConfig.forceNewTranslog()); translogGeneration = translog.getGeneration(); assert translogGeneration != null; } catch (IOException | TranslogCorruptedException e) { @@ -151,7 +150,7 @@ public class InternalEngine extends Engine { try { if (skipInitialTranslogRecovery) { // make sure we point at the latest translog from now on.. - commitIndexWriter(writer, translog.getGeneration()); + commitIndexWriter(writer, translog); } else { recoverFromTranslog(engineConfig, translogGeneration); } @@ -174,22 +173,21 @@ public class InternalEngine extends Engine { private Translog openTranslog(EngineConfig engineConfig, IndexWriter writer, boolean createNew) throws IOException { final Translog.TranslogGeneration generation = loadTranslogIdFromCommit(writer); - Translog translog; - TranslogConfig translogConfig = engineConfig.getTranslogConfig(); - if (createNew) { - translog = new Translog(translogConfig); - } else { + final TranslogConfig translogConfig = engineConfig.getTranslogConfig(); + if (createNew == false) { translogConfig.setTranslogGeneration(generation); if (generation != null && generation.translogUUID == null) { // only upgrade on pre-2.0 indices... Translog.upgradeLegacyTranslog(logger, translogConfig); } - translog = new Translog(translogConfig); } - + final Translog translog = new Translog(translogConfig); if (generation == null) { + if (createNew) { + throw new IllegalStateException("no tranlog generation present in commit data but tranlog is expected to exists"); + } logger.debug("no translog ID present in the current generation - creating one"); - commitIndexWriter(writer, translog.getGeneration()); + commitIndexWriter(writer, translog); } return translog; } @@ -244,7 +242,9 @@ public class InternalEngine extends Engine { assert commitUserData.containsKey(Translog.TRANSLOG_UUID_KEY) == false : "legacy commit contains translog UUID"; return new Translog.TranslogGeneration(null, Long.parseLong(commitUserData.get("translog_id"))); } else if (commitUserData.containsKey(Translog.TRANSLOG_GENERATION_KEY)) { - assert commitUserData.containsKey(Translog.TRANSLOG_UUID_KEY) : "commit doesn't contain translog UUID"; + if (commitUserData.containsKey(Translog.TRANSLOG_UUID_KEY) == false) { + throw new IllegalStateException("commit doesn't contain translog UUID"); + } final String translogUUID = commitUserData.get(Translog.TRANSLOG_UUID_KEY); final long translogGen = Long.parseLong(commitUserData.get(Translog.TRANSLOG_GENERATION_KEY)); return new Translog.TranslogGeneration(translogUUID, translogGen); @@ -702,12 +702,10 @@ public class InternalEngine extends Engine { if (commitTranslog) { if (flushNeeded || force) { flushNeeded = false; - final Translog.TranslogGeneration translogGeneration; try { translog.prepareCommit(); - translogGeneration = translog.getGeneration(); logger.trace("starting commit for flush; commitTranslog=true"); - commitIndexWriter(indexWriter, translogGeneration); + commitIndexWriter(indexWriter, translog); logger.trace("finished commit for flush"); translog.commit(); // we need to refresh in order to clear older version values @@ -726,7 +724,7 @@ public class InternalEngine extends Engine { // other flushes use flushLock try { logger.trace("starting commit for flush; commitTranslog=false"); - commitIndexWriter(indexWriter, translog.getGeneration()); + commitIndexWriter(indexWriter, translog); logger.trace("finished commit for flush"); } catch (Throwable e) { throw new FlushFailedEngineException(shardId, e); @@ -1176,12 +1174,13 @@ public class InternalEngine extends Engine { } - private void commitIndexWriter(IndexWriter writer, Translog.TranslogGeneration commit) throws IOException { + private void commitIndexWriter(IndexWriter writer, Translog translog) throws IOException { try { - logger.trace("committing writer with translog id [{}] ", commit.translogFileGeneration); + Translog.TranslogGeneration translogGeneration = translog.getGeneration(); + logger.trace("committing writer with translog id [{}] ", translogGeneration.translogFileGeneration); Map commitData = new HashMap<>(2); - commitData.put(Translog.TRANSLOG_GENERATION_KEY, Long.toString(commit.translogFileGeneration)); - commitData.put(Translog.TRANSLOG_UUID_KEY, commit.translogUUID); + commitData.put(Translog.TRANSLOG_GENERATION_KEY, Long.toString(translogGeneration.translogFileGeneration)); + commitData.put(Translog.TRANSLOG_UUID_KEY, translogGeneration.translogUUID); indexWriter.setCommitData(commitData); writer.commit(); } catch (Throwable ex) { diff --git a/src/main/java/org/elasticsearch/index/translog/Translog.java b/src/main/java/org/elasticsearch/index/translog/Translog.java index 9d2d81f97ec..b247189fb5f 100644 --- a/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -380,6 +380,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC if (currentCommittingTranslog != null) { int tops = currentCommittingTranslog.totalOperations(); assert tops != TranslogReader.UNKNOWN_OP_COUNT; + assert tops >= 0; ops += tops; } } @@ -499,7 +500,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC * while receiving future ones as well */ public Translog.View newView() { - // we need to acquire the read lock to make sure new translog is created + // we need to acquire the read lock to make sure no new translog is created // and will be missed by the view we're making try (ReleasableLock lock = readLock.acquire()) { ArrayList translogs = new ArrayList<>(); @@ -571,7 +572,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC } } - private boolean isReferencedGeneration(long generation) { // pkg private for testing + private boolean isReferencedGeneration(long generation) { // used to make decisions if a file can be deleted return generation >= lastCommittedTranslogFileGeneration; } @@ -662,6 +663,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC if (tops == TranslogReader.UNKNOWN_OP_COUNT) { return -1; } + assert tops >= 0; ops += tops; } return ops; @@ -812,7 +814,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC /** * Returns the next operation in the snapshot or null if we reached the end. */ - public Translog.Operation next() throws IOException; + Translog.Operation next() throws IOException; } @@ -1653,8 +1655,8 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC if (currentCommittingTranslog != null) { throw new IllegalStateException("already committing a translog with generation: " + currentCommittingTranslog.getGeneration()); } - final TranslogWriter writer = current; - writer.sync(); + final TranslogWriter oldCurrent = current; + oldCurrent.sync(); currentCommittingTranslog = current.immutableReader(); Path checkpoint = location.resolve(CHECKPOINT_FILE_NAME); assert Checkpoint.read(checkpoint).generation == currentCommittingTranslog.getGeneration(); @@ -1663,16 +1665,15 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC IOUtils.fsync(commitCheckpoint, false); IOUtils.fsync(commitCheckpoint.getParent(), true); // create a new translog file - this will sync it and update the checkpoint data; - final TranslogWriter newFile = createWriter(current.getGeneration() + 1); - current = newFile; + current = createWriter(current.getGeneration() + 1); // notify all outstanding views of the new translog (no views are created now as // we hold a write lock). for (FsView view : outstandingViews) { - view.onNewTranslog(currentCommittingTranslog.clone(), newFile.newReaderFromWriter()); + view.onNewTranslog(currentCommittingTranslog.clone(), current.newReaderFromWriter()); } - IOUtils.close(writer); + IOUtils.close(oldCurrent); logger.trace("current translog set to [{}]", current.getGeneration()); - assert writer.syncNeeded() == false : "old translog writer must not need a sync"; + assert oldCurrent.syncNeeded() == false : "old translog oldCurrent must not need a sync"; } catch (Throwable t) { IOUtils.closeWhileHandlingException(this); // tragic event @@ -1688,7 +1689,6 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC if (currentCommittingTranslog == null) { prepareCommit(); } - current.sync(); lastCommittedTranslogFileGeneration = current.getGeneration(); // this is important - otherwise old files will not be cleaned up if (recoveredTranslogs.isEmpty() == false) { IOUtils.close(recoveredTranslogs); From 50c771be290aa6ca07eff37bf4be79fa5aaeedbb Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Fri, 15 May 2015 08:31:54 +0200 Subject: [PATCH 08/56] [ENGINE] Signal when a new index is created also in Translog creation code Clarify when a new tranlog should be created by passing the same create flag to the IndexWriter as well as to the Translog creation --- .../elasticsearch/index/engine/InternalEngine.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index c68c3a114d1..50c8f7104a3 100644 --- a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -131,9 +131,10 @@ public class InternalEngine extends Engine { this.searcherFactory = new SearchFactory(engineConfig); final Translog.TranslogGeneration translogGeneration; try { - writer = createWriter(); + boolean create = !Lucene.indexExists(store.directory()); + writer = createWriter(create); indexWriter = writer; - translog = openTranslog(engineConfig, writer, skipInitialTranslogRecovery || engineConfig.forceNewTranslog()); + translog = openTranslog(engineConfig, writer, create || skipInitialTranslogRecovery || engineConfig.forceNewTranslog()); translogGeneration = translog.getGeneration(); assert translogGeneration != null; } catch (IOException | TranslogCorruptedException e) { @@ -183,8 +184,8 @@ public class InternalEngine extends Engine { } final Translog translog = new Translog(translogConfig); if (generation == null) { - if (createNew) { - throw new IllegalStateException("no tranlog generation present in commit data but tranlog is expected to exists"); + if (createNew == false) { + throw new IllegalStateException("no tranlog generation present in commit data but translog is expected to exist"); } logger.debug("no translog ID present in the current generation - creating one"); commitIndexWriter(writer, translog); @@ -986,9 +987,8 @@ public class InternalEngine extends Engine { } } - private IndexWriter createWriter() throws IOException { + private IndexWriter createWriter(boolean create) throws IOException { try { - boolean create = !Lucene.indexExists(store.directory()); final IndexWriterConfig iwc = new IndexWriterConfig(engineConfig.getAnalyzer()); iwc.setCommitOnClose(false); // we by default don't commit on close iwc.setOpenMode(create ? IndexWriterConfig.OpenMode.CREATE : IndexWriterConfig.OpenMode.APPEND); From 66921ffa50ec1f0c7ea09580e60002fa017f9147 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Fri, 15 May 2015 12:33:19 +0200 Subject: [PATCH 09/56] Remove unused docidset-related code. --- .../lucene/docset/BitsDocIdSetIterator.java | 67 ---------------- .../common/lucene/docset/ContextDocIdSet.java | 37 --------- .../common/lucene/docset/DocIdSets.java | 80 ------------------- .../lucene/docset/MatchDocIdSetIterator.java | 68 ---------------- 4 files changed, 252 deletions(-) delete mode 100644 src/main/java/org/elasticsearch/common/lucene/docset/BitsDocIdSetIterator.java delete mode 100644 src/main/java/org/elasticsearch/common/lucene/docset/ContextDocIdSet.java delete mode 100644 src/main/java/org/elasticsearch/common/lucene/docset/MatchDocIdSetIterator.java diff --git a/src/main/java/org/elasticsearch/common/lucene/docset/BitsDocIdSetIterator.java b/src/main/java/org/elasticsearch/common/lucene/docset/BitsDocIdSetIterator.java deleted file mode 100644 index 8ad2b6c5901..00000000000 --- a/src/main/java/org/elasticsearch/common/lucene/docset/BitsDocIdSetIterator.java +++ /dev/null @@ -1,67 +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.common.lucene.docset; - -import org.apache.lucene.search.DocIdSetIterator; -import org.apache.lucene.search.FilteredDocIdSetIterator; -import org.apache.lucene.util.Bits; - -/** - * A {@link Bits} based iterator. - */ -public class BitsDocIdSetIterator extends MatchDocIdSetIterator { - - private final Bits bits; - - public BitsDocIdSetIterator(Bits bits) { - super(bits.length()); - this.bits = bits; - } - - public BitsDocIdSetIterator(int maxDoc, Bits bits) { - super(maxDoc); - this.bits = bits; - } - - @Override - protected boolean matchDoc(int doc) { - return bits.get(doc); - } - - public static class FilteredIterator extends FilteredDocIdSetIterator { - - private final Bits bits; - - FilteredIterator(DocIdSetIterator innerIter, Bits bits) { - super(innerIter); - this.bits = bits; - } - - @Override - protected boolean match(int doc) { - return bits.get(doc); - } - } - - @Override - public long cost() { - return this.bits.length(); - } -} diff --git a/src/main/java/org/elasticsearch/common/lucene/docset/ContextDocIdSet.java b/src/main/java/org/elasticsearch/common/lucene/docset/ContextDocIdSet.java deleted file mode 100644 index 76a1c1595aa..00000000000 --- a/src/main/java/org/elasticsearch/common/lucene/docset/ContextDocIdSet.java +++ /dev/null @@ -1,37 +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.common.lucene.docset; - -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.search.DocIdSet; - -/** - * A holder for a {@link DocIdSet} and the {@link LeafReaderContext} it is associated with. - */ -public class ContextDocIdSet { - - public final LeafReaderContext context; - public final DocIdSet docSet; - - public ContextDocIdSet(LeafReaderContext context, DocIdSet docSet) { - this.context = context; - this.docSet = docSet; - } -} diff --git a/src/main/java/org/elasticsearch/common/lucene/docset/DocIdSets.java b/src/main/java/org/elasticsearch/common/lucene/docset/DocIdSets.java index 71cc5d7f9c2..ae24da88f0c 100644 --- a/src/main/java/org/elasticsearch/common/lucene/docset/DocIdSets.java +++ b/src/main/java/org/elasticsearch/common/lucene/docset/DocIdSets.java @@ -19,17 +19,11 @@ package org.elasticsearch.common.lucene.docset; -import org.apache.lucene.index.LeafReader; import org.apache.lucene.search.DocIdSet; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.Scorer; import org.apache.lucene.search.TwoPhaseIterator; -import org.apache.lucene.util.BitDocIdSet; -import org.apache.lucene.util.BitSet; import org.apache.lucene.util.Bits; -import org.apache.lucene.util.RamUsageEstimator; -import org.apache.lucene.util.RoaringDocIdSet; -import org.apache.lucene.util.SparseFixedBitSet; import org.elasticsearch.common.Nullable; import java.io.IOException; @@ -38,13 +32,6 @@ import java.io.IOException; */ public class DocIdSets { - /** - * Return the size of the doc id set, plus a reference to it. - */ - public static long sizeInBytes(DocIdSet docIdSet) { - return RamUsageEstimator.NUM_BYTES_OBJECT_REF + docIdSet.ramBytesUsed(); - } - /** * Is it an empty {@link DocIdSet}? */ @@ -52,59 +39,6 @@ public class DocIdSets { return set == null || set == DocIdSet.EMPTY; } - /** - * Converts to a cacheable {@link DocIdSet} - *

- * This never returns null. - */ - public static DocIdSet toCacheable(LeafReader reader, @Nullable DocIdSet set) throws IOException { - if (set == null || set == DocIdSet.EMPTY) { - return DocIdSet.EMPTY; - } - final DocIdSetIterator it = set.iterator(); - if (it == null) { - return DocIdSet.EMPTY; - } - final int firstDoc = it.nextDoc(); - if (firstDoc == DocIdSetIterator.NO_MORE_DOCS) { - return DocIdSet.EMPTY; - } - if (set instanceof BitDocIdSet) { - return set; - } - - final RoaringDocIdSet.Builder builder = new RoaringDocIdSet.Builder(reader.maxDoc()); - builder.add(firstDoc); - for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) { - builder.add(doc); - } - - return builder.build(); - } - - /** - * Get a build a {@link Bits} instance that will match all documents - * contained in {@code set}. Note that this is a potentially heavy - * operation as this might require to consume an iterator of this set - * entirely and to load it into a {@link BitSet}. Prefer using - * {@link #asSequentialAccessBits} if you only need to consume the - * {@link Bits} once and in order. - */ - public static Bits toSafeBits(int maxDoc, @Nullable DocIdSet set) throws IOException { - if (set == null) { - return new Bits.MatchNoBits(maxDoc); - } - Bits bits = set.bits(); - if (bits != null) { - return bits; - } - DocIdSetIterator iterator = set.iterator(); - if (iterator == null) { - return new Bits.MatchNoBits(maxDoc); - } - return toBitSet(iterator, maxDoc); - } - /** * Given a {@link Scorer}, return a {@link Bits} instance that will match * all documents contained in the set. Note that the returned {@link Bits} @@ -168,18 +102,4 @@ public class DocIdSets { }; } - /** - * Creates a {@link BitSet} from an iterator. - */ - public static BitSet toBitSet(DocIdSetIterator iterator, int numBits) throws IOException { - BitDocIdSet.Builder builder = new BitDocIdSet.Builder(numBits); - builder.or(iterator); - BitDocIdSet result = builder.build(); - if (result != null) { - return result.bits(); - } else { - return new SparseFixedBitSet(numBits); - } - } - } diff --git a/src/main/java/org/elasticsearch/common/lucene/docset/MatchDocIdSetIterator.java b/src/main/java/org/elasticsearch/common/lucene/docset/MatchDocIdSetIterator.java deleted file mode 100644 index 653e17ce5c1..00000000000 --- a/src/main/java/org/elasticsearch/common/lucene/docset/MatchDocIdSetIterator.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.common.lucene.docset; - -import org.apache.lucene.search.DocIdSetIterator; - -import java.io.IOException; - -/** - */ -public abstract class MatchDocIdSetIterator extends DocIdSetIterator { - private final int maxDoc; - private int doc = -1; - - public MatchDocIdSetIterator(int maxDoc) { - this.maxDoc = maxDoc; - } - - protected abstract boolean matchDoc(int doc); - - @Override - public int docID() { - return doc; - } - - @Override - public int nextDoc() throws IOException { - do { - doc++; - if (doc >= maxDoc) { - return doc = NO_MORE_DOCS; - } - } while (!matchDoc(doc)); - return doc; - } - - @Override - public int advance(int target) throws IOException { - if (target >= maxDoc) { - return doc = NO_MORE_DOCS; - } - doc = target; - while (!matchDoc(doc)) { - doc++; - if (doc >= maxDoc) { - return doc = NO_MORE_DOCS; - } - } - return doc; - } -} From 3a1a4d3e89a31320db5ef48d091fc5dca3cae455 Mon Sep 17 00:00:00 2001 From: Jun Ohtani Date: Mon, 27 Apr 2015 10:55:21 +0900 Subject: [PATCH 10/56] Analysis: Add multi-valued text support Add support array text as a multi-valued for AnalyzeRequestBuilder Add support array text as a multi-valued for Analyze REST API Add docs Closes #3023 --- docs/reference/indices/analyze.asciidoc | 13 ++++ rest-api-spec/api/indices.analyze.json | 2 +- .../test/indices.analyze/10_analyze.yaml | 8 +++ .../admin/indices/analyze/AnalyzeRequest.java | 12 ++-- .../analyze/AnalyzeRequestBuilder.java | 10 +++- .../analyze/TransportAnalyzeAction.java | 60 ++++++++++--------- .../client/IndicesAdminClient.java | 6 ++ .../client/support/AbstractClient.java | 5 ++ .../indices/analyze/RestAnalyzeAction.java | 26 +++++--- .../indices/analyze/AnalyzeActionTests.java | 48 ++++++++++----- 10 files changed, 133 insertions(+), 57 deletions(-) diff --git a/docs/reference/indices/analyze.asciidoc b/docs/reference/indices/analyze.asciidoc index 393aa4a9ad7..1026a113f1e 100644 --- a/docs/reference/indices/analyze.asciidoc +++ b/docs/reference/indices/analyze.asciidoc @@ -18,6 +18,19 @@ curl -XGET 'localhost:9200/_analyze' -d ' coming[2.0.0, body based parameters were added in 2.0.0] +If text parameter is provided as array of strings, it is analyzed as a multi-valued field. + +[source,js] +-------------------------------------------------- +curl -XGET 'localhost:9200/_analyze' -d ' +{ + "analyzer" : "standard", + "text" : ["this is a test", "the second text"] +}' +-------------------------------------------------- + +coming[2.0.0, body based parameters were added in 2.0.0] + Or by building a custom transient analyzer out of tokenizers, token filters and char filters. Token filters can use the shorter 'filters' parameter name: diff --git a/rest-api-spec/api/indices.analyze.json b/rest-api-spec/api/indices.analyze.json index 372693b794a..00b0ec13a5c 100644 --- a/rest-api-spec/api/indices.analyze.json +++ b/rest-api-spec/api/indices.analyze.json @@ -37,7 +37,7 @@ "description" : "With `true`, specify that a local shard should be used if available, with `false`, use a random shard (default: true)" }, "text": { - "type" : "string", + "type" : "list", "description" : "The text on which the analysis should be performed (when request body is not used)" }, "tokenizer": { diff --git a/rest-api-spec/test/indices.analyze/10_analyze.yaml b/rest-api-spec/test/indices.analyze/10_analyze.yaml index 16ab85a72e5..49420672861 100644 --- a/rest-api-spec/test/indices.analyze/10_analyze.yaml +++ b/rest-api-spec/test/indices.analyze/10_analyze.yaml @@ -63,3 +63,11 @@ setup: body: { "text": "Bar Foo", "filters": ["lowercase"], "tokenizer": keyword } - length: {tokens: 1 } - match: { tokens.0.token: bar foo } +--- +"Array text": + - do: + indices.analyze: + body: { "text": ["Foo Bar", "Baz"], "filters": ["lowercase"], "tokenizer": keyword } + - length: {tokens: 2 } + - match: { tokens.0.token: foo bar } + - match: { tokens.1.token: baz } diff --git a/src/main/java/org/elasticsearch/action/admin/indices/analyze/AnalyzeRequest.java b/src/main/java/org/elasticsearch/action/admin/indices/analyze/AnalyzeRequest.java index cabc75c559b..d631f8b8d0a 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/analyze/AnalyzeRequest.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/analyze/AnalyzeRequest.java @@ -36,7 +36,7 @@ import static org.elasticsearch.action.ValidateActions.addValidationError; */ public class AnalyzeRequest extends SingleCustomOperationRequest { - private String text; + private String[] text; private String analyzer; @@ -61,11 +61,11 @@ public class AnalyzeRequest extends SingleCustomOperationRequest this.index(index); } - public String text() { + public String[] text() { return this.text; } - public AnalyzeRequest text(String text) { + public AnalyzeRequest text(String... text) { this.text = text; return this; } @@ -118,7 +118,7 @@ public class AnalyzeRequest extends SingleCustomOperationRequest @Override public ActionRequestValidationException validate() { ActionRequestValidationException validationException = super.validate(); - if (text == null) { + if (text == null || text.length == 0) { validationException = addValidationError("text is missing", validationException); } if (tokenFilters == null) { @@ -133,7 +133,7 @@ public class AnalyzeRequest extends SingleCustomOperationRequest @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); - text = in.readString(); + text = in.readStringArray(); analyzer = in.readOptionalString(); tokenizer = in.readOptionalString(); tokenFilters = in.readStringArray(); @@ -144,7 +144,7 @@ public class AnalyzeRequest extends SingleCustomOperationRequest @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeString(text); + out.writeStringArray(text); out.writeOptionalString(analyzer); out.writeOptionalString(tokenizer); out.writeStringArray(tokenFilters); diff --git a/src/main/java/org/elasticsearch/action/admin/indices/analyze/AnalyzeRequestBuilder.java b/src/main/java/org/elasticsearch/action/admin/indices/analyze/AnalyzeRequestBuilder.java index b9d057f0cfe..2707419e304 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/analyze/AnalyzeRequestBuilder.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/analyze/AnalyzeRequestBuilder.java @@ -30,7 +30,7 @@ public class AnalyzeRequestBuilder extends SingleCustomOperationRequestBuilder tokens = Lists.newArrayList(); TokenStream stream = null; - try { - stream = analyzer.tokenStream(field, request.text()); - stream.reset(); - CharTermAttribute term = stream.addAttribute(CharTermAttribute.class); - PositionIncrementAttribute posIncr = stream.addAttribute(PositionIncrementAttribute.class); - OffsetAttribute offset = stream.addAttribute(OffsetAttribute.class); - TypeAttribute type = stream.addAttribute(TypeAttribute.class); + int lastPosition = -1; + int lastOffset = 0; + for (String text : request.text()) { + try { + stream = analyzer.tokenStream(field, text); + stream.reset(); + CharTermAttribute term = stream.addAttribute(CharTermAttribute.class); + PositionIncrementAttribute posIncr = stream.addAttribute(PositionIncrementAttribute.class); + OffsetAttribute offset = stream.addAttribute(OffsetAttribute.class); + TypeAttribute type = stream.addAttribute(TypeAttribute.class); + + while (stream.incrementToken()) { + int increment = posIncr.getPositionIncrement(); + if (increment > 0) { + lastPosition = lastPosition + increment; + } + tokens.add(new AnalyzeResponse.AnalyzeToken(term.toString(), lastPosition, lastOffset + offset.startOffset(), lastOffset + offset.endOffset(), type.type())); - int position = -1; - while (stream.incrementToken()) { - int increment = posIncr.getPositionIncrement(); - if (increment > 0) { - position = position + increment; } - tokens.add(new AnalyzeResponse.AnalyzeToken(term.toString(), position, offset.startOffset(), offset.endOffset(), type.type())); - } - stream.end(); - } catch (IOException e) { - throw new ElasticsearchException("failed to analyze", e); - } finally { - if (stream != null) { - try { - stream.close(); - } catch (IOException e) { - // ignore - } - } - if (closeAnalyzer) { - analyzer.close(); + stream.end(); + lastOffset += offset.endOffset(); + lastPosition += posIncr.getPositionIncrement(); + + lastPosition += analyzer.getPositionIncrementGap(field); + lastOffset += analyzer.getOffsetGap(field); + + } catch (IOException e) { + throw new ElasticsearchException("failed to analyze", e); + } finally { + IOUtils.closeWhileHandlingException(stream); } } + if (closeAnalyzer) { + analyzer.close(); + } + return new AnalyzeResponse(tokens); } } diff --git a/src/main/java/org/elasticsearch/client/IndicesAdminClient.java b/src/main/java/org/elasticsearch/client/IndicesAdminClient.java index bd1af4586a7..239f48e3b60 100644 --- a/src/main/java/org/elasticsearch/client/IndicesAdminClient.java +++ b/src/main/java/org/elasticsearch/client/IndicesAdminClient.java @@ -587,6 +587,12 @@ public interface IndicesAdminClient extends ElasticsearchClient { */ AnalyzeRequestBuilder prepareAnalyze(String text); + /** + * Analyze text/texts. + * + */ + AnalyzeRequestBuilder prepareAnalyze(); + /** * Puts an index template. */ diff --git a/src/main/java/org/elasticsearch/client/support/AbstractClient.java b/src/main/java/org/elasticsearch/client/support/AbstractClient.java index edec9af848a..703b03f0dc5 100644 --- a/src/main/java/org/elasticsearch/client/support/AbstractClient.java +++ b/src/main/java/org/elasticsearch/client/support/AbstractClient.java @@ -1478,6 +1478,11 @@ public abstract class AbstractClient extends AbstractComponent implements Client return new AnalyzeRequestBuilder(this, AnalyzeAction.INSTANCE, null, text); } + @Override + public AnalyzeRequestBuilder prepareAnalyze() { + return new AnalyzeRequestBuilder(this, AnalyzeAction.INSTANCE); + } + @Override public ActionFuture putTemplate(final PutIndexTemplateRequest request) { return execute(PutIndexTemplateAction.INSTANCE, request); diff --git a/src/main/java/org/elasticsearch/rest/action/admin/indices/analyze/RestAnalyzeAction.java b/src/main/java/org/elasticsearch/rest/action/admin/indices/analyze/RestAnalyzeAction.java index 5ce5eaef4ac..2c7d2eed465 100644 --- a/src/main/java/org/elasticsearch/rest/action/admin/indices/analyze/RestAnalyzeAction.java +++ b/src/main/java/org/elasticsearch/rest/action/admin/indices/analyze/RestAnalyzeAction.java @@ -22,6 +22,7 @@ import com.google.common.collect.Lists; import org.elasticsearch.action.admin.indices.analyze.AnalyzeRequest; import org.elasticsearch.action.admin.indices.analyze.AnalyzeResponse; import org.elasticsearch.client.Client; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; @@ -55,10 +56,10 @@ public class RestAnalyzeAction extends BaseRestHandler { @Override public void handleRequest(final RestRequest request, final RestChannel channel, final Client client) { - String text = request.param("text"); + String[] texts = request.paramAsStringArrayOrEmptyIfAll("text"); AnalyzeRequest analyzeRequest = new AnalyzeRequest(request.param("index")); - analyzeRequest.text(text); + analyzeRequest.text(texts); analyzeRequest.preferLocal(request.paramAsBoolean("prefer_local", analyzeRequest.preferLocalShard())); analyzeRequest.analyzer(request.param("analyzer")); analyzeRequest.field(request.param("field")); @@ -69,9 +70,9 @@ public class RestAnalyzeAction extends BaseRestHandler { if (RestActions.hasBodyContent(request)) { XContentType type = RestActions.guessBodyContentType(request); if (type == null) { - if (text == null) { - text = RestActions.getRestContent(request).toUtf8(); - analyzeRequest.text(text); + if (texts == null || texts.length == 0) { + texts = new String[]{ RestActions.getRestContent(request).toUtf8() }; + analyzeRequest.text(texts); } } else { // NOTE: if rest request with xcontent body has request parameters, the parameters does not override xcontent values @@ -95,7 +96,16 @@ public class RestAnalyzeAction extends BaseRestHandler { } else if ("prefer_local".equals(currentFieldName) && token == XContentParser.Token.VALUE_BOOLEAN) { analyzeRequest.preferLocal(parser.booleanValue()); } else if ("text".equals(currentFieldName) && token == XContentParser.Token.VALUE_STRING) { - analyzeRequest.text(parser.text()); + analyzeRequest.text(parser.text()); + } else if ("text".equals(currentFieldName) && token == XContentParser.Token.START_ARRAY) { + List texts = Lists.newArrayList(); + while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { + if (token.isValue() == false) { + throw new IllegalArgumentException(currentFieldName + " array element should only contain text"); + } + texts.add(parser.text()); + } + analyzeRequest.text(texts.toArray(Strings.EMPTY_ARRAY)); } else if ("analyzer".equals(currentFieldName) && token == XContentParser.Token.VALUE_STRING) { analyzeRequest.analyzer(parser.text()); } else if ("field".equals(currentFieldName) && token == XContentParser.Token.VALUE_STRING) { @@ -110,7 +120,7 @@ public class RestAnalyzeAction extends BaseRestHandler { } filters.add(parser.text()); } - analyzeRequest.tokenFilters(filters.toArray(new String[0])); + analyzeRequest.tokenFilters(filters.toArray(Strings.EMPTY_ARRAY)); } else if ("char_filters".equals(currentFieldName) && token == XContentParser.Token.START_ARRAY) { List charFilters = Lists.newArrayList(); while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { @@ -119,7 +129,7 @@ public class RestAnalyzeAction extends BaseRestHandler { } charFilters.add(parser.text()); } - analyzeRequest.tokenFilters(charFilters.toArray(new String[0])); + analyzeRequest.tokenFilters(charFilters.toArray(Strings.EMPTY_ARRAY)); } else { throw new IllegalArgumentException("Unknown parameter [" + currentFieldName + "] in request body or parameter is of the wrong type[" + token + "] "); } diff --git a/src/test/java/org/elasticsearch/indices/analyze/AnalyzeActionTests.java b/src/test/java/org/elasticsearch/indices/analyze/AnalyzeActionTests.java index bf169d254e6..436483b6869 100644 --- a/src/test/java/org/elasticsearch/indices/analyze/AnalyzeActionTests.java +++ b/src/test/java/org/elasticsearch/indices/analyze/AnalyzeActionTests.java @@ -158,18 +158,7 @@ public class AnalyzeActionTests extends ElasticsearchIntegrationTest { ensureGreen(); client().admin().indices().preparePutMapping("test") - .setType("document").setSource( - "{\n" + - " \"document\":{\n" + - " \"properties\":{\n" + - " \"simple\":{\n" + - " \"type\":\"string\",\n" + - " \"analyzer\": \"simple\"\n" + - " }\n" + - " }\n" + - " }\n" + - "}" - ).get(); + .setType("document").setSource("simple", "type=string,analyzer=simple").get(); for (int i = 0; i < 10; i++) { final AnalyzeRequestBuilder requestBuilder = client().admin().indices().prepareAnalyze("THIS IS A TEST"); @@ -220,7 +209,8 @@ public class AnalyzeActionTests extends ElasticsearchIntegrationTest { RestAnalyzeAction.buildFromContent(content, analyzeRequest); - assertThat(analyzeRequest.text(), equalTo("THIS IS A TEST")); + assertThat(analyzeRequest.text().length, equalTo(1)); + assertThat(analyzeRequest.text(), equalTo(new String[]{"THIS IS A TEST"})); assertThat(analyzeRequest.tokenizer(), equalTo("keyword")); assertThat(analyzeRequest.tokenFilters(), equalTo(new String[]{"lowercase"})); } @@ -239,7 +229,6 @@ public class AnalyzeActionTests extends ElasticsearchIntegrationTest { } } - @Test public void testParseXContentForAnalyzeRequestWithUnknownParamThrowsException() throws Exception { AnalyzeRequest analyzeRequest = new AnalyzeRequest("for test"); @@ -258,4 +247,35 @@ public class AnalyzeActionTests extends ElasticsearchIntegrationTest { } } + @Test + public void analyzerWithMultiValues() throws Exception { + + assertAcked(prepareCreate("test").addAlias(new Alias("alias"))); + ensureGreen(); + + client().admin().indices().preparePutMapping("test") + .setType("document").setSource("simple", "type=string,analyzer=simple,position_offset_gap=100").get(); + + String[] texts = new String[]{"THIS IS A TEST", "THE SECOND TEXT"}; + + final AnalyzeRequestBuilder requestBuilder = client().admin().indices().prepareAnalyze(); + requestBuilder.setText(texts); + requestBuilder.setIndex(indexOrAlias()); + requestBuilder.setField("simple"); + AnalyzeResponse analyzeResponse = requestBuilder.get(); + assertThat(analyzeResponse.getTokens().size(), equalTo(7)); + AnalyzeResponse.AnalyzeToken token = analyzeResponse.getTokens().get(3); + assertThat(token.getTerm(), equalTo("test")); + assertThat(token.getPosition(), equalTo(3)); + assertThat(token.getStartOffset(), equalTo(10)); + assertThat(token.getEndOffset(), equalTo(14)); + + token = analyzeResponse.getTokens().get(5); + assertThat(token.getTerm(), equalTo("second")); + assertThat(token.getPosition(), equalTo(105)); + assertThat(token.getStartOffset(), equalTo(19)); + assertThat(token.getEndOffset(), equalTo(25)); + + } + } From ece18f162ef06a6c7658ab7641d48133115b82ef Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Fri, 15 May 2015 14:06:18 +0200 Subject: [PATCH 11/56] Removed `id_cache` from stats and cat apis. Also removed the `id_cache` option from the clear cache api. Closes #5269 --- docs/reference/cat/nodes.asciidoc | 2 - docs/reference/cluster/stats.asciidoc | 4 - docs/reference/indices/clearcache.asciidoc | 3 +- docs/reference/migration/migrate_2_0.asciidoc | 26 ++++++ .../query-dsl/has-child-query.asciidoc | 6 +- .../query-dsl/has-parent-query.asciidoc | 6 +- rest-api-spec/api/indices.clear_cache.json | 8 -- rest-api-spec/api/indices.stats.json | 2 +- rest-api-spec/api/nodes.stats.json | 2 +- rest-api-spec/test/cat.shards/10_basic.yaml | 1 - .../test/indices.stats/11_metric.yaml | 5 -- .../cluster/stats/ClusterStatsIndices.java | 11 --- .../stats/TransportClusterStatsAction.java | 4 +- .../cache/clear/ClearIndicesCacheRequest.java | 12 --- .../ClearIndicesCacheRequestBuilder.java | 4 - .../clear/ShardClearIndicesCacheRequest.java | 13 --- .../TransportClearIndicesCacheAction.java | 4 - .../admin/indices/stats/CommonStats.java | 41 +-------- .../admin/indices/stats/CommonStatsFlags.java | 1 - .../indices/stats/IndicesStatsRequest.java | 9 -- .../stats/IndicesStatsRequestBuilder.java | 5 -- .../stats/TransportIndicesStatsAction.java | 3 - .../index/cache/id/IdCacheStats.java | 89 ------------------- .../index/fielddata/ShardFieldData.java | 9 +- .../elasticsearch/index/shard/IndexShard.java | 6 -- .../indices/NodeIndicesStats.java | 6 -- .../clear/RestClearIndicesCacheAction.java | 4 - .../indices/stats/RestIndicesStatsAction.java | 1 - .../rest/action/cat/RestIndicesAction.java | 6 -- .../rest/action/cat/RestNodesAction.java | 6 -- .../rest/action/cat/RestShardsAction.java | 4 - .../clear/ClearIndicesCacheBlocksTests.java | 4 +- .../search/child/ChildSearchBenchmark.java | 4 +- .../ChildSearchShortCircuitBenchmark.java | 4 +- .../document/DocumentActionsTests.java | 2 +- .../indices/stats/IndexStatsTests.java | 7 +- .../search/child/ParentFieldLoadingTest.java | 20 ++--- .../child/SimpleChildQuerySearchTests.java | 30 +++---- 38 files changed, 70 insertions(+), 304 deletions(-) delete mode 100644 src/main/java/org/elasticsearch/index/cache/id/IdCacheStats.java diff --git a/docs/reference/cat/nodes.asciidoc b/docs/reference/cat/nodes.asciidoc index 717b97a3f6b..d5c6036ef35 100644 --- a/docs/reference/cat/nodes.asciidoc +++ b/docs/reference/cat/nodes.asciidoc @@ -131,8 +131,6 @@ get operations |2 gets |0s |`get.missing_total` |`gmto`, `getMissingTotal` |No |Number of failed get operations |1 -|`id_cache.memory_size` |`im`, `idCacheMemory` |No |Used ID cache -memory |216b |`indexing.delete_current` |`idc`, `indexingDeleteCurrent` |No |Number of current deletion operations |0 |`indexing.delete_time` |`idti`, `indexingDeleteTime` |No |Time spent in diff --git a/docs/reference/cluster/stats.asciidoc b/docs/reference/cluster/stats.asciidoc index 512c5a4da37..c0241764d3d 100644 --- a/docs/reference/cluster/stats.asciidoc +++ b/docs/reference/cluster/stats.asciidoc @@ -61,10 +61,6 @@ Will return, for example: "memory_size_in_bytes": 0, "evictions": 0 }, - "id_cache": { - "memory_size": "0b", - "memory_size_in_bytes": 0 - }, "completion": { "size": "0b", "size_in_bytes": 0 diff --git a/docs/reference/indices/clearcache.asciidoc b/docs/reference/indices/clearcache.asciidoc index 73a44e1debb..5015d09fb03 100644 --- a/docs/reference/indices/clearcache.asciidoc +++ b/docs/reference/indices/clearcache.asciidoc @@ -10,8 +10,7 @@ $ curl -XPOST 'http://localhost:9200/twitter/_cache/clear' -------------------------------------------------- The API, by default, will clear all caches. Specific caches can be cleaned -explicitly by setting `filter`, `fielddata`, `query_cache`, -or `id_cache` to `true`. +explicitly by setting `filter`, `fielddata` or `query_cache`. All caches relating to a specific field(s) can also be cleared by specifying `fields` parameter with a comma delimited list of the diff --git a/docs/reference/migration/migrate_2_0.asciidoc b/docs/reference/migration/migrate_2_0.asciidoc index 744570ef5d3..b6d3e29e360 100644 --- a/docs/reference/migration/migrate_2_0.asciidoc +++ b/docs/reference/migration/migrate_2_0.asciidoc @@ -560,3 +560,29 @@ same search request will likely be off if `top_children` was used. === Removed file based index templates Index templates can no longer be configured on disk. Use the `_template` API instead. + +[float] +=== Removed `id_cache` from stats apis + +Removed `id_cache` metric from nodes stats, indices stats and cluster stats apis. This metric has also been removed +from the shards cat, indices cat and nodes cat apis. Parent/child memory is now reported under fielddata, because it +has internally be using fielddata for a while now. + +To just see how much parent/child related field data is taking, the `fielddata_fields` option can be used on the stats +apis. Indices stats example: + +[source,js] +-------------------------------------------------- +curl -XGET "http://localhost:9200/_stats/fielddata?pretty&human&fielddata_fields=_parent" +-------------------------------------------------- + +Parent/child is using field data for the `_parent` field since version `1.1.0`, but the memory stats for the `_parent` +field were still shown under `id_cache` metric in the stats apis for backwards compatible reasons between 1.x versions. + +Before version `1.1.0` the parent/child had its own in-memory data structures for id values in the `_parent` field. + +[float] +=== Removed `id_cache` from clear cache api + +Removed `id_cache` option from the clear cache apis. The `fielddata` option should be used to clear `_parent` field +from fielddata. diff --git a/docs/reference/query-dsl/has-child-query.asciidoc b/docs/reference/query-dsl/has-child-query.asciidoc index fde49306104..e974f58001e 100644 --- a/docs/reference/query-dsl/has-child-query.asciidoc +++ b/docs/reference/query-dsl/has-child-query.asciidoc @@ -82,11 +82,11 @@ Additionally, every child document is mapped to its parent using a long value (approximately). It is advisable to keep the string parent ID short in order to reduce memory usage. -You can check how much memory is being used by the ID cache using the -<> or <> +You can check how much memory is being used by the `_parent` field in the fielddata cache +using the <> or <> APIS, eg: [source,js] -------------------------------------------------- -curl -XGET "http://localhost:9200/_stats/id_cache?pretty&human" +curl -XGET "http://localhost:9200/_stats/fielddata?pretty&human&fielddata_fields=_parent" -------------------------------------------------- diff --git a/docs/reference/query-dsl/has-parent-query.asciidoc b/docs/reference/query-dsl/has-parent-query.asciidoc index 9c31fd179e0..f0ab59a1b1e 100644 --- a/docs/reference/query-dsl/has-parent-query.asciidoc +++ b/docs/reference/query-dsl/has-parent-query.asciidoc @@ -57,13 +57,13 @@ Additionally, every child document is mapped to its parent using a long value (approximately). It is advisable to keep the string parent ID short in order to reduce memory usage. -You can check how much memory is being used by the ID cache using the -<> or <> +You can check how much memory is being used by the `_parent` field in the fielddata cache +using the <> or <> APIS, eg: [source,js] -------------------------------------------------- -curl -XGET "http://localhost:9200/_stats/id_cache?pretty&human" +curl -XGET "http://localhost:9200/_stats/fielddata?pretty&human&fielddata_fields=_parent" -------------------------------------------------- diff --git a/rest-api-spec/api/indices.clear_cache.json b/rest-api-spec/api/indices.clear_cache.json index c8e3e84de88..6873e91b37b 100644 --- a/rest-api-spec/api/indices.clear_cache.json +++ b/rest-api-spec/api/indices.clear_cache.json @@ -32,14 +32,6 @@ "type" : "boolean", "description" : "Clear filter caches" }, - "id": { - "type" : "boolean", - "description" : "Clear ID caches for parent/child" - }, - "id_cache": { - "type" : "boolean", - "description" : "Clear ID caches for parent/child" - }, "ignore_unavailable": { "type" : "boolean", "description" : "Whether specified concrete indices should be ignored when unavailable (missing or closed)" diff --git a/rest-api-spec/api/indices.stats.json b/rest-api-spec/api/indices.stats.json index d0f997a0083..0375721ed43 100644 --- a/rest-api-spec/api/indices.stats.json +++ b/rest-api-spec/api/indices.stats.json @@ -17,7 +17,7 @@ }, "metric" : { "type" : "list", - "options" : ["_all", "completion", "docs", "fielddata", "filter_cache", "flush", "get", "id_cache", "indexing", "merge", "percolate", "query_cache", "refresh", "search", "segments", "store", "warmer", "suggest"], + "options" : ["_all", "completion", "docs", "fielddata", "filter_cache", "flush", "get", "indexing", "merge", "percolate", "query_cache", "refresh", "search", "segments", "store", "warmer", "suggest"], "description" : "Limit the information returned the specific metrics." } }, diff --git a/rest-api-spec/api/nodes.stats.json b/rest-api-spec/api/nodes.stats.json index a9af238c465..0a0870020b3 100644 --- a/rest-api-spec/api/nodes.stats.json +++ b/rest-api-spec/api/nodes.stats.json @@ -20,7 +20,7 @@ }, "index_metric" : { "type" : "list", - "options" : ["_all", "completion", "docs", "fielddata", "filter_cache", "flush", "get", "id_cache", "indexing", "merge", "percolate", "query_cache", "refresh", "search", "segments", "store", "warmer", "suggest"], + "options" : ["_all", "completion", "docs", "fielddata", "filter_cache", "flush", "get", "indexing", "merge", "percolate", "query_cache", "refresh", "search", "segments", "store", "warmer", "suggest"], "description" : "Limit the information returned for `indices` metric to the specific index metrics. Isn't used if `indices` (or `all`) metric isn't specified." }, "node_id": { diff --git a/rest-api-spec/test/cat.shards/10_basic.yaml b/rest-api-spec/test/cat.shards/10_basic.yaml index 8063bf3d040..c525cb8475c 100755 --- a/rest-api-spec/test/cat.shards/10_basic.yaml +++ b/rest-api-spec/test/cat.shards/10_basic.yaml @@ -29,7 +29,6 @@ get.exists_total .+ \n get.missing_time .+ \n get.missing_total .+ \n - id_cache.memory_size .+ \n indexing.delete_current .+ \n indexing.delete_time .+ \n indexing.delete_total .+ \n diff --git a/rest-api-spec/test/indices.stats/11_metric.yaml b/rest-api-spec/test/indices.stats/11_metric.yaml index 0718449a784..2857ff6f008 100644 --- a/rest-api-spec/test/indices.stats/11_metric.yaml +++ b/rest-api-spec/test/indices.stats/11_metric.yaml @@ -30,7 +30,6 @@ setup: - is_true: _all.total.flush - is_true: _all.total.warmer - is_true: _all.total.filter_cache - - is_true: _all.total.id_cache - is_true: _all.total.fielddata - is_true: _all.total.percolate - is_true: _all.total.completion @@ -54,7 +53,6 @@ setup: - is_true: _all.total.flush - is_true: _all.total.warmer - is_true: _all.total.filter_cache - - is_true: _all.total.id_cache - is_true: _all.total.fielddata - is_true: _all.total.percolate - is_true: _all.total.completion @@ -78,7 +76,6 @@ setup: - is_false: _all.total.flush - is_false: _all.total.warmer - is_false: _all.total.filter_cache - - is_false: _all.total.id_cache - is_false: _all.total.fielddata - is_false: _all.total.percolate - is_false: _all.total.completion @@ -102,7 +99,6 @@ setup: - is_false: _all.total.flush - is_false: _all.total.warmer - is_false: _all.total.filter_cache - - is_false: _all.total.id_cache - is_false: _all.total.fielddata - is_false: _all.total.percolate - is_false: _all.total.completion @@ -127,7 +123,6 @@ setup: - is_false: _all.total.flush - is_false: _all.total.warmer - is_false: _all.total.filter_cache - - is_false: _all.total.id_cache - is_false: _all.total.fielddata - is_false: _all.total.percolate - is_false: _all.total.completion diff --git a/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsIndices.java b/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsIndices.java index ccf4969bf5e..f2a1dd6e587 100644 --- a/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsIndices.java +++ b/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsIndices.java @@ -30,7 +30,6 @@ import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilderString; import org.elasticsearch.index.cache.filter.FilterCacheStats; -import org.elasticsearch.index.cache.id.IdCacheStats; import org.elasticsearch.index.engine.SegmentsStats; import org.elasticsearch.index.fielddata.FieldDataStats; import org.elasticsearch.index.percolator.stats.PercolateStats; @@ -48,7 +47,6 @@ public class ClusterStatsIndices implements ToXContent, Streamable { private StoreStats store; private FieldDataStats fieldData; private FilterCacheStats filterCache; - private IdCacheStats idCache; private CompletionStats completion; private SegmentsStats segments; private PercolateStats percolate; @@ -63,7 +61,6 @@ public class ClusterStatsIndices implements ToXContent, Streamable { this.store = new StoreStats(); this.fieldData = new FieldDataStats(); this.filterCache = new FilterCacheStats(); - this.idCache = new IdCacheStats(); this.completion = new CompletionStats(); this.segments = new SegmentsStats(); this.percolate = new PercolateStats(); @@ -87,7 +84,6 @@ public class ClusterStatsIndices implements ToXContent, Streamable { store.add(shardCommonStats.store); fieldData.add(shardCommonStats.fieldData); filterCache.add(shardCommonStats.filterCache); - idCache.add(shardCommonStats.idCache); completion.add(shardCommonStats.completion); segments.add(shardCommonStats.segments); percolate.add(shardCommonStats.percolate); @@ -125,10 +121,6 @@ public class ClusterStatsIndices implements ToXContent, Streamable { return filterCache; } - public IdCacheStats getIdCache() { - return idCache; - } - public CompletionStats getCompletion() { return completion; } @@ -149,7 +141,6 @@ public class ClusterStatsIndices implements ToXContent, Streamable { store = StoreStats.readStoreStats(in); fieldData = FieldDataStats.readFieldDataStats(in); filterCache = FilterCacheStats.readFilterCacheStats(in); - idCache = IdCacheStats.readIdCacheStats(in); completion = CompletionStats.readCompletionStats(in); segments = SegmentsStats.readSegmentsStats(in); percolate = PercolateStats.readPercolateStats(in); @@ -163,7 +154,6 @@ public class ClusterStatsIndices implements ToXContent, Streamable { store.writeTo(out); fieldData.writeTo(out); filterCache.writeTo(out); - idCache.writeTo(out); completion.writeTo(out); segments.writeTo(out); percolate.writeTo(out); @@ -187,7 +177,6 @@ public class ClusterStatsIndices implements ToXContent, Streamable { store.toXContent(builder, params); fieldData.toXContent(builder, params); filterCache.toXContent(builder, params); - idCache.toXContent(builder, params); completion.toXContent(builder, params); segments.toXContent(builder, params); percolate.toXContent(builder, params); diff --git a/src/main/java/org/elasticsearch/action/admin/cluster/stats/TransportClusterStatsAction.java b/src/main/java/org/elasticsearch/action/admin/cluster/stats/TransportClusterStatsAction.java index 7f7d00ad529..240411c6235 100644 --- a/src/main/java/org/elasticsearch/action/admin/cluster/stats/TransportClusterStatsAction.java +++ b/src/main/java/org/elasticsearch/action/admin/cluster/stats/TransportClusterStatsAction.java @@ -56,8 +56,8 @@ public class TransportClusterStatsAction extends TransportNodesOperationAction { private static final CommonStatsFlags SHARD_STATS_FLAGS = new CommonStatsFlags(CommonStatsFlags.Flag.Docs, CommonStatsFlags.Flag.Store, - CommonStatsFlags.Flag.FieldData, CommonStatsFlags.Flag.FilterCache, CommonStatsFlags.Flag.IdCache, - CommonStatsFlags.Flag.Completion, CommonStatsFlags.Flag.Segments, CommonStatsFlags.Flag.Percolate); + CommonStatsFlags.Flag.FieldData, CommonStatsFlags.Flag.FilterCache, CommonStatsFlags.Flag.Completion, CommonStatsFlags.Flag.Segments, + CommonStatsFlags.Flag.Percolate); private final NodeService nodeService; private final IndicesService indicesService; diff --git a/src/main/java/org/elasticsearch/action/admin/indices/cache/clear/ClearIndicesCacheRequest.java b/src/main/java/org/elasticsearch/action/admin/indices/cache/clear/ClearIndicesCacheRequest.java index 3a96c83b3ac..323d8869024 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/cache/clear/ClearIndicesCacheRequest.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/cache/clear/ClearIndicesCacheRequest.java @@ -33,7 +33,6 @@ public class ClearIndicesCacheRequest extends BroadcastOperationRequest 0) { // only clear caches relating to the specified fields diff --git a/src/main/java/org/elasticsearch/action/admin/indices/stats/CommonStats.java b/src/main/java/org/elasticsearch/action/admin/indices/stats/CommonStats.java index d57b24fb113..e4bae7f637b 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/stats/CommonStats.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/stats/CommonStats.java @@ -27,7 +27,6 @@ import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.index.cache.filter.FilterCacheStats; -import org.elasticsearch.index.cache.id.IdCacheStats; import org.elasticsearch.index.cache.query.QueryCacheStats; import org.elasticsearch.index.engine.SegmentsStats; import org.elasticsearch.index.fielddata.FieldDataStats; @@ -92,9 +91,6 @@ public class CommonStats implements Streamable, ToXContent { case FilterCache: filterCache = new FilterCacheStats(); break; - case IdCache: - idCache = new IdCacheStats(); - break; case FieldData: fieldData = new FieldDataStats(); break; @@ -161,9 +157,6 @@ public class CommonStats implements Streamable, ToXContent { case FilterCache: filterCache = indexShard.filterCacheStats(); break; - case IdCache: - idCache = indexShard.idCacheStats(); - break; case FieldData: fieldData = indexShard.fieldDataStats(flags.fieldDataFields()); break; @@ -224,9 +217,6 @@ public class CommonStats implements Streamable, ToXContent { @Nullable public FilterCacheStats filterCache; - @Nullable - public IdCacheStats idCache; - @Nullable public FieldDataStats fieldData; @@ -333,15 +323,6 @@ public class CommonStats implements Streamable, ToXContent { filterCache.add(stats.getFilterCache()); } - if (idCache == null) { - if (stats.getIdCache() != null) { - idCache = new IdCacheStats(); - idCache.add(stats.getIdCache()); - } - } else { - idCache.add(stats.getIdCache()); - } - if (fieldData == null) { if (stats.getFieldData() != null) { fieldData = new FieldDataStats(); @@ -458,11 +439,6 @@ public class CommonStats implements Streamable, ToXContent { return this.filterCache; } - @Nullable - public IdCacheStats getIdCache() { - return this.idCache; - } - @Nullable public FieldDataStats getFieldData() { return this.fieldData; @@ -511,7 +487,7 @@ public class CommonStats implements Streamable, ToXContent { /** * Utility method which computes total memory by adding - * FieldData, IdCache, Percolate, Segments (memory, index writer, version map) + * FieldData, Percolate, Segments (memory, index writer, version map) */ public ByteSizeValue getTotalMemory() { long size = 0; @@ -521,9 +497,6 @@ public class CommonStats implements Streamable, ToXContent { if (this.getFilterCache() != null) { size += this.getFilterCache().getMemorySizeInBytes(); } - if (this.getIdCache() != null) { - size += this.getIdCache().getMemorySizeInBytes(); - } if (this.getPercolate() != null) { size += this.getPercolate().getMemorySizeInBytes(); } @@ -568,9 +541,6 @@ public class CommonStats implements Streamable, ToXContent { if (in.readBoolean()) { filterCache = FilterCacheStats.readFilterCacheStats(in); } - if (in.readBoolean()) { - idCache = IdCacheStats.readIdCacheStats(in); - } if (in.readBoolean()) { fieldData = FieldDataStats.readFieldDataStats(in); } @@ -651,12 +621,6 @@ public class CommonStats implements Streamable, ToXContent { out.writeBoolean(true); filterCache.writeTo(out); } - if (idCache == null) { - out.writeBoolean(false); - } else { - out.writeBoolean(true); - idCache.writeTo(out); - } if (fieldData == null) { out.writeBoolean(false); } else { @@ -720,9 +684,6 @@ public class CommonStats implements Streamable, ToXContent { if (filterCache != null) { filterCache.toXContent(builder, params); } - if (idCache != null) { - idCache.toXContent(builder, params); - } if (fieldData != null) { fieldData.toXContent(builder, params); } diff --git a/src/main/java/org/elasticsearch/action/admin/indices/stats/CommonStatsFlags.java b/src/main/java/org/elasticsearch/action/admin/indices/stats/CommonStatsFlags.java index 42be114b13d..9ac34ddf6d2 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/stats/CommonStatsFlags.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/stats/CommonStatsFlags.java @@ -216,7 +216,6 @@ public class CommonStatsFlags implements Streamable, Cloneable { Flush("flush"), Refresh("refresh"), FilterCache("filter_cache"), - IdCache("id_cache"), FieldData("fielddata"), Docs("docs"), Warmer("warmer"), diff --git a/src/main/java/org/elasticsearch/action/admin/indices/stats/IndicesStatsRequest.java b/src/main/java/org/elasticsearch/action/admin/indices/stats/IndicesStatsRequest.java index 68a6e2dcc2d..b8a94ab4d51 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/stats/IndicesStatsRequest.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/stats/IndicesStatsRequest.java @@ -175,15 +175,6 @@ public class IndicesStatsRequest extends BroadcastOperationRequest= 2.0 - long memorySize = totalMetric.count(); - if (perFieldTotals.containsKey(ParentFieldMapper.NAME)) { - memorySize -= perFieldTotals.get(ParentFieldMapper.NAME).count(); - } - return new FieldDataStats(memorySize, evictionsMetric.count(), fieldTotals); + return new FieldDataStats(totalMetric.count(), evictionsMetric.count(), fieldTotals); } @Override diff --git a/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 4839e4debbb..2239a778f08 100644 --- a/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -65,7 +65,6 @@ import org.elasticsearch.index.cache.IndexCache; import org.elasticsearch.index.cache.bitset.ShardBitsetFilterCache; import org.elasticsearch.index.cache.filter.FilterCacheStats; import org.elasticsearch.index.cache.filter.ShardFilterCache; -import org.elasticsearch.index.cache.id.IdCacheStats; import org.elasticsearch.index.cache.query.ShardQueryCache; import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy; @@ -665,11 +664,6 @@ public class IndexShard extends AbstractIndexShardComponent { return shardPercolateService; } - public IdCacheStats idCacheStats() { - long memorySizeInBytes = shardFieldData.stats(ParentFieldMapper.NAME).getFields().get(ParentFieldMapper.NAME); - return new IdCacheStats(memorySizeInBytes); - } - public TranslogStats translogStats() { return engine().getTranslog().stats(); } diff --git a/src/main/java/org/elasticsearch/indices/NodeIndicesStats.java b/src/main/java/org/elasticsearch/indices/NodeIndicesStats.java index ddc04403e7f..e0f286e325b 100644 --- a/src/main/java/org/elasticsearch/indices/NodeIndicesStats.java +++ b/src/main/java/org/elasticsearch/indices/NodeIndicesStats.java @@ -34,7 +34,6 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilderString; import org.elasticsearch.index.Index; import org.elasticsearch.index.cache.filter.FilterCacheStats; -import org.elasticsearch.index.cache.id.IdCacheStats; import org.elasticsearch.index.cache.query.QueryCacheStats; import org.elasticsearch.index.engine.SegmentsStats; import org.elasticsearch.index.fielddata.FieldDataStats; @@ -142,11 +141,6 @@ public class NodeIndicesStats implements Streamable, Serializable, ToXContent { return stats.getQueryCache(); } - @Nullable - public IdCacheStats getIdCache() { - return stats.getIdCache(); - } - @Nullable public CompletionStats getCompletion() { return stats.getCompletion(); diff --git a/src/main/java/org/elasticsearch/rest/action/admin/indices/cache/clear/RestClearIndicesCacheAction.java b/src/main/java/org/elasticsearch/rest/action/admin/indices/cache/clear/RestClearIndicesCacheAction.java index 8eb83cba0d1..e676409dbec 100644 --- a/src/main/java/org/elasticsearch/rest/action/admin/indices/cache/clear/RestClearIndicesCacheAction.java +++ b/src/main/java/org/elasticsearch/rest/action/admin/indices/cache/clear/RestClearIndicesCacheAction.java @@ -79,9 +79,6 @@ public class RestClearIndicesCacheAction extends BaseRestHandler { if (Fields.FIELD_DATA.match(entry.getKey())) { clearIndicesCacheRequest.fieldDataCache(request.paramAsBoolean(entry.getKey(), clearIndicesCacheRequest.fieldDataCache())); } - if (Fields.ID.match(entry.getKey())) { - clearIndicesCacheRequest.idCache(request.paramAsBoolean(entry.getKey(), clearIndicesCacheRequest.idCache())); - } if (Fields.RECYCLER.match(entry.getKey())) { clearIndicesCacheRequest.recycler(request.paramAsBoolean(entry.getKey(), clearIndicesCacheRequest.recycler())); } @@ -96,7 +93,6 @@ public class RestClearIndicesCacheAction extends BaseRestHandler { public static class Fields { public static final ParseField FILTER = new ParseField("filter", "filter_cache"); public static final ParseField FIELD_DATA = new ParseField("field_data", "fielddata"); - public static final ParseField ID = new ParseField("id", "id_cache"); public static final ParseField RECYCLER = new ParseField("recycler"); public static final ParseField FIELDS = new ParseField("fields"); } diff --git a/src/main/java/org/elasticsearch/rest/action/admin/indices/stats/RestIndicesStatsAction.java b/src/main/java/org/elasticsearch/rest/action/admin/indices/stats/RestIndicesStatsAction.java index 43fcbd57171..a71e5a3f814 100644 --- a/src/main/java/org/elasticsearch/rest/action/admin/indices/stats/RestIndicesStatsAction.java +++ b/src/main/java/org/elasticsearch/rest/action/admin/indices/stats/RestIndicesStatsAction.java @@ -73,7 +73,6 @@ public class RestIndicesStatsAction extends BaseRestHandler { indicesStatsRequest.flush(metrics.contains("flush")); indicesStatsRequest.warmer(metrics.contains("warmer")); indicesStatsRequest.filterCache(metrics.contains("filter_cache")); - indicesStatsRequest.idCache(metrics.contains("id_cache")); indicesStatsRequest.percolate(metrics.contains("percolate")); indicesStatsRequest.segments(metrics.contains("segments")); indicesStatsRequest.fieldData(metrics.contains("fielddata")); diff --git a/src/main/java/org/elasticsearch/rest/action/cat/RestIndicesAction.java b/src/main/java/org/elasticsearch/rest/action/cat/RestIndicesAction.java index 546975867e5..f4ab318ae84 100644 --- a/src/main/java/org/elasticsearch/rest/action/cat/RestIndicesAction.java +++ b/src/main/java/org/elasticsearch/rest/action/cat/RestIndicesAction.java @@ -163,9 +163,6 @@ public class RestIndicesAction extends AbstractCatAction { table.addCell("get.missing_total", "sibling:pri;alias:gmto,getMissingTotal;default:false;text-align:right;desc:number of failed gets"); table.addCell("pri.get.missing_total", "default:false;text-align:right;desc:number of failed gets"); - table.addCell("id_cache.memory_size", "sibling:pri;alias:im,idCacheMemory;default:false;text-align:right;desc:used id cache"); - table.addCell("pri.id_cache.memory_size", "default:false;text-align:right;desc:used id cache"); - table.addCell("indexing.delete_current", "sibling:pri;alias:idc,indexingDeleteCurrent;default:false;text-align:right;desc:number of current deletions"); table.addCell("pri.indexing.delete_current", "default:false;text-align:right;desc:number of current deletions"); @@ -365,9 +362,6 @@ public class RestIndicesAction extends AbstractCatAction { table.addCell(indexStats == null ? null : indexStats.getTotal().getGet().getMissingCount()); table.addCell(indexStats == null ? null : indexStats.getPrimaries().getGet().getMissingCount()); - table.addCell(indexStats == null ? null : indexStats.getTotal().getIdCache().getMemorySize()); - table.addCell(indexStats == null ? null : indexStats.getPrimaries().getIdCache().getMemorySize()); - table.addCell(indexStats == null ? null : indexStats.getTotal().getIndexing().getTotal().getDeleteCurrent()); table.addCell(indexStats == null ? null : indexStats.getPrimaries().getIndexing().getTotal().getDeleteCurrent()); diff --git a/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java b/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java index 960288051f2..081e3b55f6e 100644 --- a/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java +++ b/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java @@ -36,7 +36,6 @@ import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.InetSocketTransportAddress; import org.elasticsearch.index.cache.filter.FilterCacheStats; -import org.elasticsearch.index.cache.id.IdCacheStats; import org.elasticsearch.index.cache.query.QueryCacheStats; import org.elasticsearch.index.engine.SegmentsStats; import org.elasticsearch.index.fielddata.FieldDataStats; @@ -162,8 +161,6 @@ public class RestNodesAction extends AbstractCatAction { table.addCell("get.missing_time", "alias:gmti,getMissingTime;default:false;text-align:right;desc:time spent in failed gets"); table.addCell("get.missing_total", "alias:gmto,getMissingTotal;default:false;text-align:right;desc:number of failed gets"); - table.addCell("id_cache.memory_size", "alias:im,idCacheMemory;default:false;text-align:right;desc:used id cache"); - table.addCell("indexing.delete_current", "alias:idc,indexingDeleteCurrent;default:false;text-align:right;desc:number of current deletions"); table.addCell("indexing.delete_time", "alias:idti,indexingDeleteTime;default:false;text-align:right;desc:time spent in deletions"); table.addCell("indexing.delete_total", "alias:idto,indexingDeleteTotal;default:false;text-align:right;desc:number of delete ops"); @@ -295,9 +292,6 @@ public class RestNodesAction extends AbstractCatAction { table.addCell(getStats == null ? null : getStats.getMissingTime()); table.addCell(getStats == null ? null : getStats.getMissingCount()); - IdCacheStats idCacheStats = indicesStats == null ? null : indicesStats.getIdCache(); - table.addCell(idCacheStats == null ? null : idCacheStats.getMemorySize()); - IndexingStats indexingStats = indicesStats == null ? null : indicesStats.getIndexing(); table.addCell(indexingStats == null ? null : indexingStats.getTotal().getDeleteCurrent()); table.addCell(indexingStats == null ? null : indexingStats.getTotal().getDeleteTime()); diff --git a/src/main/java/org/elasticsearch/rest/action/cat/RestShardsAction.java b/src/main/java/org/elasticsearch/rest/action/cat/RestShardsAction.java index 0a32ac6771d..14703b1694d 100644 --- a/src/main/java/org/elasticsearch/rest/action/cat/RestShardsAction.java +++ b/src/main/java/org/elasticsearch/rest/action/cat/RestShardsAction.java @@ -108,8 +108,6 @@ public class RestShardsAction extends AbstractCatAction { table.addCell("get.missing_time", "alias:gmti,getMissingTime;default:false;text-align:right;desc:time spent in failed gets"); table.addCell("get.missing_total", "alias:gmto,getMissingTotal;default:false;text-align:right;desc:number of failed gets"); - table.addCell("id_cache.memory_size", "alias:im,idCacheMemory;default:false;text-align:right;desc:used id cache"); - table.addCell("indexing.delete_current", "alias:idc,indexingDeleteCurrent;default:false;text-align:right;desc:number of current deletions"); table.addCell("indexing.delete_time", "alias:idti,indexingDeleteTime;default:false;text-align:right;desc:time spent in deletions"); table.addCell("indexing.delete_total", "alias:idto,indexingDeleteTotal;default:false;text-align:right;desc:number of delete ops"); @@ -229,8 +227,6 @@ public class RestShardsAction extends AbstractCatAction { table.addCell(shardStats == null ? null : shardStats.getGet().getMissingTime()); table.addCell(shardStats == null ? null : shardStats.getGet().getMissingCount()); - table.addCell(shardStats == null ? null : shardStats.getIdCache().getMemorySize()); - table.addCell(shardStats == null ? null : shardStats.getIndexing().getTotal().getDeleteCurrent()); table.addCell(shardStats == null ? null : shardStats.getIndexing().getTotal().getDeleteTime()); table.addCell(shardStats == null ? null : shardStats.getIndexing().getTotal().getDeleteCount()); diff --git a/src/test/java/org/elasticsearch/action/admin/indices/cache/clear/ClearIndicesCacheBlocksTests.java b/src/test/java/org/elasticsearch/action/admin/indices/cache/clear/ClearIndicesCacheBlocksTests.java index b9d4b2e685c..b93d7a2f72b 100644 --- a/src/test/java/org/elasticsearch/action/admin/indices/cache/clear/ClearIndicesCacheBlocksTests.java +++ b/src/test/java/org/elasticsearch/action/admin/indices/cache/clear/ClearIndicesCacheBlocksTests.java @@ -44,7 +44,7 @@ public class ClearIndicesCacheBlocksTests extends ElasticsearchIntegrationTest { for (String blockSetting : Arrays.asList(SETTING_BLOCKS_READ, SETTING_BLOCKS_WRITE)) { try { enableIndexBlock("test", blockSetting); - ClearIndicesCacheResponse clearIndicesCacheResponse = client().admin().indices().prepareClearCache("test").setFieldDataCache(true).setFilterCache(true).setIdCache(true).execute().actionGet(); + ClearIndicesCacheResponse clearIndicesCacheResponse = client().admin().indices().prepareClearCache("test").setFieldDataCache(true).setFilterCache(true).setFieldDataCache(true).execute().actionGet(); assertNoFailures(clearIndicesCacheResponse); assertThat(clearIndicesCacheResponse.getSuccessfulShards(), equalTo(numShards.totalNumShards)); } finally { @@ -55,7 +55,7 @@ public class ClearIndicesCacheBlocksTests extends ElasticsearchIntegrationTest { for (String blockSetting : Arrays.asList(SETTING_READ_ONLY, SETTING_BLOCKS_METADATA)) { try { enableIndexBlock("test", blockSetting); - assertBlocked(client().admin().indices().prepareClearCache("test").setFieldDataCache(true).setFilterCache(true).setIdCache(true)); + assertBlocked(client().admin().indices().prepareClearCache("test").setFieldDataCache(true).setFilterCache(true).setFieldDataCache(true)); } finally { disableIndexBlock("test", blockSetting); } diff --git a/src/test/java/org/elasticsearch/benchmark/search/child/ChildSearchBenchmark.java b/src/test/java/org/elasticsearch/benchmark/search/child/ChildSearchBenchmark.java index e930c2cf328..c69618f7e46 100644 --- a/src/test/java/org/elasticsearch/benchmark/search/child/ChildSearchBenchmark.java +++ b/src/test/java/org/elasticsearch/benchmark/search/child/ChildSearchBenchmark.java @@ -260,7 +260,7 @@ public class ChildSearchBenchmark { statsResponse = client.admin().cluster().prepareNodesStats() .setJvm(true).setIndices(true).execute().actionGet(); - System.out.println("--> Id cache size: " + statsResponse.getNodes()[0].getIndices().getIdCache().getMemorySize()); + System.out.println("--> Field data size: " + statsResponse.getNodes()[0].getIndices().getFieldData().getMemorySize()); System.out.println("--> Used heap size: " + statsResponse.getNodes()[0].getJvm().getMem().getHeapUsed()); System.out.println("--> Running has_child query with score type"); @@ -319,7 +319,7 @@ public class ChildSearchBenchmark { statsResponse = client.admin().cluster().prepareNodesStats() .setJvm(true).setIndices(true).execute().actionGet(); - System.out.println("--> Id cache size: " + statsResponse.getNodes()[0].getIndices().getIdCache().getMemorySize()); + System.out.println("--> Field data size: " + statsResponse.getNodes()[0].getIndices().getFieldData().getMemorySize()); System.out.println("--> Used heap size: " + statsResponse.getNodes()[0].getJvm().getMem().getHeapUsed()); client.close(); diff --git a/src/test/java/org/elasticsearch/benchmark/search/child/ChildSearchShortCircuitBenchmark.java b/src/test/java/org/elasticsearch/benchmark/search/child/ChildSearchShortCircuitBenchmark.java index d13f8fb095d..78bec43ea05 100644 --- a/src/test/java/org/elasticsearch/benchmark/search/child/ChildSearchShortCircuitBenchmark.java +++ b/src/test/java/org/elasticsearch/benchmark/search/child/ChildSearchShortCircuitBenchmark.java @@ -171,7 +171,7 @@ public class ChildSearchShortCircuitBenchmark { statsResponse = client.admin().cluster().prepareNodesStats() .setJvm(true).setIndices(true).execute().actionGet(); - System.out.println("--> Id cache size: " + statsResponse.getNodes()[0].getIndices().getIdCache().getMemorySize()); + System.out.println("--> Field data size: " + statsResponse.getNodes()[0].getIndices().getFieldData().getMemorySize()); System.out.println("--> Used heap size: " + statsResponse.getNodes()[0].getJvm().getMem().getHeapUsed()); totalQueryTime = 0; @@ -192,7 +192,7 @@ public class ChildSearchShortCircuitBenchmark { statsResponse = client.admin().cluster().prepareNodesStats() .setJvm(true).setIndices(true).execute().actionGet(); - System.out.println("--> Id cache size: " + statsResponse.getNodes()[0].getIndices().getIdCache().getMemorySize()); + System.out.println("--> Field data size: " + statsResponse.getNodes()[0].getIndices().getFieldData().getMemorySize()); System.out.println("--> Used heap size: " + statsResponse.getNodes()[0].getJvm().getMem().getHeapUsed()); client.close(); diff --git a/src/test/java/org/elasticsearch/document/DocumentActionsTests.java b/src/test/java/org/elasticsearch/document/DocumentActionsTests.java index 4de1e72b87b..8673a93dc3e 100644 --- a/src/test/java/org/elasticsearch/document/DocumentActionsTests.java +++ b/src/test/java/org/elasticsearch/document/DocumentActionsTests.java @@ -77,7 +77,7 @@ public class DocumentActionsTests extends ElasticsearchIntegrationTest { assertThat(indexExists("test1234565"), equalTo(false)); logger.info("Clearing cache"); - ClearIndicesCacheResponse clearIndicesCacheResponse = client().admin().indices().clearCache(clearIndicesCacheRequest("test").recycler(true).fieldDataCache(true).filterCache(true).idCache(true)).actionGet(); + ClearIndicesCacheResponse clearIndicesCacheResponse = client().admin().indices().clearCache(clearIndicesCacheRequest("test").recycler(true).fieldDataCache(true).filterCache(true)).actionGet(); assertNoFailures(clearIndicesCacheResponse); assertThat(clearIndicesCacheResponse.getSuccessfulShards(), equalTo(numShards.totalNumShards)); diff --git a/src/test/java/org/elasticsearch/indices/stats/IndexStatsTests.java b/src/test/java/org/elasticsearch/indices/stats/IndexStatsTests.java index 2f04e0a21c3..9a351d133e3 100644 --- a/src/test/java/org/elasticsearch/indices/stats/IndexStatsTests.java +++ b/src/test/java/org/elasticsearch/indices/stats/IndexStatsTests.java @@ -639,7 +639,7 @@ public class IndexStatsTests extends ElasticsearchIntegrationTest { @Test public void testFlagOrdinalOrder() { Flag[] flags = new Flag[]{Flag.Store, Flag.Indexing, Flag.Get, Flag.Search, Flag.Merge, Flag.Flush, Flag.Refresh, - Flag.FilterCache, Flag.IdCache, Flag.FieldData, Flag.Docs, Flag.Warmer, Flag.Percolate, Flag.Completion, Flag.Segments, + Flag.FilterCache, Flag.FieldData, Flag.Docs, Flag.Warmer, Flag.Percolate, Flag.Completion, Flag.Segments, Flag.Translog, Flag.Suggest, Flag.QueryCache, Flag.Recovery}; assertThat(flags.length, equalTo(Flag.values().length)); @@ -872,9 +872,6 @@ public class IndexStatsTests extends ElasticsearchIntegrationTest { case Get: builder.setGet(set); break; - case IdCache: - builder.setIdCache(set); - break; case Indexing: builder.setIndexing(set); break; @@ -932,8 +929,6 @@ public class IndexStatsTests extends ElasticsearchIntegrationTest { return response.getFlush() != null; case Get: return response.getGet() != null; - case IdCache: - return response.getIdCache() != null; case Indexing: return response.getIndexing() != null; case Merge: diff --git a/src/test/java/org/elasticsearch/search/child/ParentFieldLoadingTest.java b/src/test/java/org/elasticsearch/search/child/ParentFieldLoadingTest.java index 203466e4326..f5fa9693b6c 100644 --- a/src/test/java/org/elasticsearch/search/child/ParentFieldLoadingTest.java +++ b/src/test/java/org/elasticsearch/search/child/ParentFieldLoadingTest.java @@ -71,7 +71,7 @@ public class ParentFieldLoadingTest extends ElasticsearchIntegrationTest { refresh(); ClusterStatsResponse response = client().admin().cluster().prepareClusterStats().get(); - assertThat(response.getIndicesStats().getIdCache().getMemorySizeInBytes(), equalTo(0l)); + assertThat(response.getIndicesStats().getFieldData().getMemorySizeInBytes(), equalTo(0l)); logger.info("testing default loading..."); assertAcked(client().admin().indices().prepareDelete("test").get()); @@ -86,8 +86,8 @@ public class ParentFieldLoadingTest extends ElasticsearchIntegrationTest { refresh(); response = client().admin().cluster().prepareClusterStats().get(); - long idCacheSizeDefault = response.getIndicesStats().getIdCache().getMemorySizeInBytes(); - assertThat(idCacheSizeDefault, greaterThan(0l)); + long fielddataSizeDefault = response.getIndicesStats().getFieldData().getMemorySizeInBytes(); + assertThat(fielddataSizeDefault, greaterThan(0l)); logger.info("testing eager loading..."); assertAcked(client().admin().indices().prepareDelete("test").get()); @@ -102,7 +102,7 @@ public class ParentFieldLoadingTest extends ElasticsearchIntegrationTest { refresh(); response = client().admin().cluster().prepareClusterStats().get(); - assertThat(response.getIndicesStats().getIdCache().getMemorySizeInBytes(), equalTo(idCacheSizeDefault)); + assertThat(response.getIndicesStats().getFieldData().getMemorySizeInBytes(), equalTo(fielddataSizeDefault)); logger.info("testing eager global ordinals loading..."); assertAcked(client().admin().indices().prepareDelete("test").get()); @@ -113,14 +113,14 @@ public class ParentFieldLoadingTest extends ElasticsearchIntegrationTest { ensureGreen(); // Need to do 2 separate refreshes, otherwise we have 1 segment and then we can't measure if global ordinals - // is loaded by the size of the id_cache, because global ordinals on 1 segment shards takes no extra memory. + // is loaded by the size of the field data cache, because global ordinals on 1 segment shards takes no extra memory. client().prepareIndex("test", "parent", "1").setSource("{}").get(); refresh(); client().prepareIndex("test", "child", "1").setParent("1").setSource("{}").get(); refresh(); response = client().admin().cluster().prepareClusterStats().get(); - assertThat(response.getIndicesStats().getIdCache().getMemorySizeInBytes(), greaterThan(idCacheSizeDefault)); + assertThat(response.getIndicesStats().getFieldData().getMemorySizeInBytes(), greaterThan(fielddataSizeDefault)); } @Test @@ -136,8 +136,8 @@ public class ParentFieldLoadingTest extends ElasticsearchIntegrationTest { refresh(); ClusterStatsResponse response = client().admin().cluster().prepareClusterStats().get(); - long idCacheSizeDefault = response.getIndicesStats().getIdCache().getMemorySizeInBytes(); - assertThat(idCacheSizeDefault, greaterThan(0l)); + long fielddataSizeDefault = response.getIndicesStats().getFieldData().getMemorySizeInBytes(); + assertThat(fielddataSizeDefault, greaterThan(0l)); PutMappingResponse putMappingResponse = client().admin().indices().preparePutMapping("test").setType("child") .setSource(childMapping(FieldMapper.Loading.EAGER_GLOBAL_ORDINALS)) @@ -165,11 +165,11 @@ public class ParentFieldLoadingTest extends ElasticsearchIntegrationTest { }); // Need to add a new doc otherwise the refresh doesn't trigger a new searcher - // Because it ends up in its own segment, but isn't of type parent or child, this doc doesn't contribute to the size of the id_cache + // Because it ends up in its own segment, but isn't of type parent or child, this doc doesn't contribute to the size of the fielddata cache client().prepareIndex("test", "dummy", "dummy").setSource("{}").get(); refresh(); response = client().admin().cluster().prepareClusterStats().get(); - assertThat(response.getIndicesStats().getIdCache().getMemorySizeInBytes(), greaterThan(idCacheSizeDefault)); + assertThat(response.getIndicesStats().getFieldData().getMemorySizeInBytes(), greaterThan(fielddataSizeDefault)); } private XContentBuilder childMapping(FieldMapper.Loading loading) throws IOException { diff --git a/src/test/java/org/elasticsearch/search/child/SimpleChildQuerySearchTests.java b/src/test/java/org/elasticsearch/search/child/SimpleChildQuerySearchTests.java index 778c5d6c05b..cc975c11f3f 100644 --- a/src/test/java/org/elasticsearch/search/child/SimpleChildQuerySearchTests.java +++ b/src/test/java/org/elasticsearch/search/child/SimpleChildQuerySearchTests.java @@ -286,8 +286,7 @@ public class SimpleChildQuerySearchTests extends ElasticsearchIntegrationTest { @Test @LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/elasticsearch/elasticsearch/issues/9270") - public void testClearIdCacheBug() throws Exception { - // enforce lazy loading to make sure that p/c stats are not counted as part of field data + public void testParentFieldDataCacheBug() throws Exception { assertAcked(prepareCreate("test") .setSettings(ImmutableSettings.builder().put(indexSettings()) .put("index.refresh_interval", -1)) // Disable automatic refresh, so that the _parent doesn't get warmed @@ -307,10 +306,10 @@ public class SimpleChildQuerySearchTests extends ElasticsearchIntegrationTest { client().prepareIndex("test", "parent", "p1").setSource("p_field", "p_value1").get(); refresh(); - // No _parent field yet, there shouldn't be anything in the parent id cache + // No _parent field yet, there shouldn't be anything in the field data for _parent field IndicesStatsResponse indicesStatsResponse = client().admin().indices() - .prepareStats("test").setIdCache(true).get(); - assertThat(indicesStatsResponse.getTotal().getIdCache().getMemorySizeInBytes(), equalTo(0l)); + .prepareStats("test").setFieldData(true).get(); + assertThat(indicesStatsResponse.getTotal().getFieldData().getMemorySizeInBytes(), equalTo(0l)); // Now add mapping + children client().admin().indices().preparePutMapping("test").setType("child") @@ -338,12 +337,9 @@ public class SimpleChildQuerySearchTests extends ElasticsearchIntegrationTest { refresh(); indicesStatsResponse = client().admin().indices() - .prepareStats("test").setFieldData(true).get(); - // automatic warm-up has populated the cache since it found a parent field mapper - assertThat(indicesStatsResponse.getTotal().getIdCache().getMemorySizeInBytes(), greaterThan(0l)); - // Even though p/c is field data based the stats stay zero, because _parent field data field is kept - // track of under id cache stats memory wise for bwc - assertThat(indicesStatsResponse.getTotal().getFieldData().getMemorySizeInBytes(), equalTo(0l)); + .prepareStats("test").setFieldData(true).setFieldDataFields("_parent").get(); + assertThat(indicesStatsResponse.getTotal().getFieldData().getMemorySizeInBytes(), greaterThan(0l)); + assertThat(indicesStatsResponse.getTotal().getFieldData().getFields().get("_parent"), greaterThan(0l)); SearchResponse searchResponse = client().prepareSearch("test") .setQuery(constantScoreQuery(hasChildQuery("child", termQuery("c_field", "blue")))) @@ -352,17 +348,17 @@ public class SimpleChildQuerySearchTests extends ElasticsearchIntegrationTest { assertThat(searchResponse.getHits().totalHits(), equalTo(1l)); indicesStatsResponse = client().admin().indices() - .prepareStats("test").setFieldData(true).get(); - assertThat(indicesStatsResponse.getTotal().getIdCache().getMemorySizeInBytes(), greaterThan(0l)); - assertThat(indicesStatsResponse.getTotal().getFieldData().getMemorySizeInBytes(), equalTo(0l)); + .prepareStats("test").setFieldData(true).setFieldDataFields("_parent").get(); + assertThat(indicesStatsResponse.getTotal().getFieldData().getMemorySizeInBytes(), greaterThan(0l)); + assertThat(indicesStatsResponse.getTotal().getFieldData().getFields().get("_parent"), greaterThan(0l)); - ClearIndicesCacheResponse clearCacheResponse = client().admin().indices().prepareClearCache("test").setIdCache(true).get(); + ClearIndicesCacheResponse clearCacheResponse = client().admin().indices().prepareClearCache("test").setFieldDataCache(true).get(); assertNoFailures(clearCacheResponse); assertAllSuccessful(clearCacheResponse); indicesStatsResponse = client().admin().indices() - .prepareStats("test").setFieldData(true).get(); - assertThat(indicesStatsResponse.getTotal().getIdCache().getMemorySizeInBytes(), equalTo(0l)); + .prepareStats("test").setFieldData(true).setFieldDataFields("_parent").get(); assertThat(indicesStatsResponse.getTotal().getFieldData().getMemorySizeInBytes(), equalTo(0l)); + assertThat(indicesStatsResponse.getTotal().getFieldData().getFields().get("_parent"), equalTo(0l)); } @Test From 32e23b9100b769cf9f45fb7459648422dd473ccb Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Thu, 7 May 2015 16:46:40 +0200 Subject: [PATCH 12/56] Aggs: Make it possible to configure missing values. Most aggregations (terms, histogram, stats, percentiles, geohash-grid) now support a new `missing` option which defines the value to consider when a field does not have a value. This can be handy if you eg. want a terms aggregation to handle the same way documents that have "N/A" or no value for a `tag` field. This works in a very similar way to the `missing` option on the `sort` element. One known issue is that this option sometimes cannot make the right decision in the unmapped case: it needs to replace all values with the `missing` value but might not know what kind of values source should be produced (numerics, strings, geo points?). For this reason, we might want to add an `unmapped_type` option in the future like we did for sorting. Related to #5324 --- .../bucket/datehistogram-aggregation.asciidoc | 23 ++ .../bucket/histogram-aggregation.asciidoc | 23 ++ .../bucket/terms-aggregation.asciidoc | 22 ++ .../metrics/avg-aggregation.asciidoc | 24 +- .../metrics/cardinality-aggregation.asciidoc | 21 ++ .../extendedstats-aggregation.asciidoc | 24 +- .../metrics/max-aggregation.asciidoc | 21 ++ .../metrics/min-aggregation.asciidoc | 22 ++ .../metrics/percentile-aggregation.asciidoc | 22 ++ .../percentile-rank-aggregation.asciidoc | 22 ++ .../metrics/stats-aggregation.asciidoc | 24 +- .../metrics/sum-aggregation.asciidoc | 22 ++ .../elasticsearch/common/geo/GeoUtils.java | 21 +- .../ValuesSourceAggregationBuilder.java | 12 + .../GlobalOrdinalsStringTermsAggregator.java | 6 +- .../bucket/terms/TermsAggregatorFactory.java | 4 +- ...ValuesSourceMetricsAggregationBuilder.java | 13 + .../support/AggregationContext.java | 82 ++++- .../aggregations/support/MissingValues.java | 337 ++++++++++++++++++ .../aggregations/support/ValuesSource.java | 109 ++++-- .../ValuesSourceAggregatorFactory.java | 4 +- .../support/ValuesSourceConfig.java | 1 + .../support/ValuesSourceParser.java | 8 + .../aggregations/MissingValueTests.java | 195 ++++++++++ .../support/MissingValuesTests.java | 297 +++++++++++++++ 25 files changed, 1311 insertions(+), 48 deletions(-) create mode 100644 src/main/java/org/elasticsearch/search/aggregations/support/MissingValues.java create mode 100644 src/test/java/org/elasticsearch/search/aggregations/MissingValueTests.java create mode 100644 src/test/java/org/elasticsearch/search/aggregations/support/MissingValuesTests.java diff --git a/docs/reference/aggregations/bucket/datehistogram-aggregation.asciidoc b/docs/reference/aggregations/bucket/datehistogram-aggregation.asciidoc index 256ef62d766..dfff6691af1 100644 --- a/docs/reference/aggregations/bucket/datehistogram-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/datehistogram-aggregation.asciidoc @@ -123,3 +123,26 @@ settings and filter the returned buckets based on a `min_doc_count` setting (by bucket that matches documents and the last one are returned). This histogram also supports the `extended_bounds` setting, which enables extending the bounds of the histogram beyond the data itself (to read more on why you'd want to do that please refer to the explanation <>). + +==== Missing value + +The `missing` parameter defines how documents that are missing a value should be treated. +By default they will be ignored but it is also possible to treat them as if they +had a value. + +[source,js] +-------------------------------------------------- +{ + "aggs" : { + "publish_date" : { + "datehistogram" : { + "field" : "publish_date", + "interval": "year", + "missing": "2000-01-01" <1> + } + } + } +} +-------------------------------------------------- + +<1> Documents without a value in the `publish_date` field will fall into the same bucket as documents that have the value `2000-01-01`. diff --git a/docs/reference/aggregations/bucket/histogram-aggregation.asciidoc b/docs/reference/aggregations/bucket/histogram-aggregation.asciidoc index 129732c0410..99d7950e7de 100644 --- a/docs/reference/aggregations/bucket/histogram-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/histogram-aggregation.asciidoc @@ -317,3 +317,26 @@ Response: } } -------------------------------------------------- + +==== Missing value + +The `missing` parameter defines how documents that are missing a value should be treated. +By default they will be ignored but it is also possible to treat them as if they +had a value. + +[source,js] +-------------------------------------------------- +{ + "aggs" : { + "quantity" : { + "histogram" : { + "field" : "quantity", + "interval": 10, + "missing": 0 <1> + } + } + } +} +-------------------------------------------------- + +<1> Documents without a value in the `quantity` field will fall into the same bucket as documents that have the value `0`. diff --git a/docs/reference/aggregations/bucket/terms-aggregation.asciidoc b/docs/reference/aggregations/bucket/terms-aggregation.asciidoc index 58a6ca2449c..cf401126c8a 100644 --- a/docs/reference/aggregations/bucket/terms-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/terms-aggregation.asciidoc @@ -655,3 +655,25 @@ in inner aggregations. <1> experimental[] the possible values are `map`, `global_ordinals`, `global_ordinals_hash` and `global_ordinals_low_cardinality` Please note that Elasticsearch will ignore this execution hint if it is not applicable and that there is no backward compatibility guarantee on these hints. + +==== Missing value + +The `missing` parameter defines how documents that are missing a value should be treated. +By default they will be ignored but it is also possible to treat them as if they +had a value. + +[source,js] +-------------------------------------------------- +{ + "aggs" : { + "tags" : { + "terms" : { + "field" : "tags", + "missing": "N/A" <1> + } + } + } +} +-------------------------------------------------- + +<1> Documents without a value in the `tags` field will fall into the same bucket as documents that have the value `N/A`. diff --git a/docs/reference/aggregations/metrics/avg-aggregation.asciidoc b/docs/reference/aggregations/metrics/avg-aggregation.asciidoc index 3f029984ba8..8e0d2b4b5e7 100644 --- a/docs/reference/aggregations/metrics/avg-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/avg-aggregation.asciidoc @@ -72,4 +72,26 @@ It turned out that the exam was way above the level of the students and a grade } } } --------------------------------------------------- \ No newline at end of file +-------------------------------------------------- + +==== Missing value + +The `missing` parameter defines how documents that are missing a value should be treated. +By default they will be ignored but it is also possible to treat them as if they +had a value. + +[source,js] +-------------------------------------------------- +{ + "aggs" : { + "grade_avg" : { + "avg" : { + "field" : "grade", + "missing": 10 <1> + } + } + } +} +-------------------------------------------------- + +<1> Documents without a value in the `grade` field will fall into the same bucket as documents that have the value `10`. diff --git a/docs/reference/aggregations/metrics/cardinality-aggregation.asciidoc b/docs/reference/aggregations/metrics/cardinality-aggregation.asciidoc index 07943a06c2d..8e34e16f7a8 100644 --- a/docs/reference/aggregations/metrics/cardinality-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/cardinality-aggregation.asciidoc @@ -155,3 +155,24 @@ however since hashes need to be computed on the fly. TIP: The `script` parameter expects an inline script. Use `script_id` for indexed scripts and `script_file` for scripts in the `config/scripts/` directory. +==== Missing value + +The `missing` parameter defines how documents that are missing a value should be treated. +By default they will be ignored but it is also possible to treat them as if they +had a value. + +[source,js] +-------------------------------------------------- +{ + "aggs" : { + "tag_cardinality" : { + "cardinality" : { + "field" : "tag", + "missing": "N/A" <1> + } + } + } +} +-------------------------------------------------- + +<1> Documents without a value in the `tag` field will fall into the same bucket as documents that have the value `N/A`. diff --git a/docs/reference/aggregations/metrics/extendedstats-aggregation.asciidoc b/docs/reference/aggregations/metrics/extendedstats-aggregation.asciidoc index 07d25fac65d..0f65b7670cf 100644 --- a/docs/reference/aggregations/metrics/extendedstats-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/extendedstats-aggregation.asciidoc @@ -116,4 +116,26 @@ It turned out that the exam was way above the level of the students and a grade } } } --------------------------------------------------- \ No newline at end of file +-------------------------------------------------- + +==== Missing value + +The `missing` parameter defines how documents that are missing a value should be treated. +By default they will be ignored but it is also possible to treat them as if they +had a value. + +[source,js] +-------------------------------------------------- +{ + "aggs" : { + "grades_stats" : { + "extended_stats" : { + "field" : "grade", + "missing": 0 <1> + } + } + } +} +-------------------------------------------------- + +<1> Documents without a value in the `grade` field will fall into the same bucket as documents that have the value `0`. diff --git a/docs/reference/aggregations/metrics/max-aggregation.asciidoc b/docs/reference/aggregations/metrics/max-aggregation.asciidoc index facefc1201d..856adc4b03d 100644 --- a/docs/reference/aggregations/metrics/max-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/max-aggregation.asciidoc @@ -67,3 +67,24 @@ Let's say that the prices of the documents in our index are in USD, but we would } -------------------------------------------------- +==== Missing value + +The `missing` parameter defines how documents that are missing a value should be treated. +By default they will be ignored but it is also possible to treat them as if they +had a value. + +[source,js] +-------------------------------------------------- +{ + "aggs" : { + "grade_max" : { + "max" : { + "field" : "grade", + "missing": 10 <1> + } + } + } +} +-------------------------------------------------- + +<1> Documents without a value in the `grade` field will fall into the same bucket as documents that have the value `10`. diff --git a/docs/reference/aggregations/metrics/min-aggregation.asciidoc b/docs/reference/aggregations/metrics/min-aggregation.asciidoc index 1383cc08322..c7424d5570b 100644 --- a/docs/reference/aggregations/metrics/min-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/min-aggregation.asciidoc @@ -66,3 +66,25 @@ Let's say that the prices of the documents in our index are in USD, but we would } } -------------------------------------------------- + +==== Missing value + +The `missing` parameter defines how documents that are missing a value should be treated. +By default they will be ignored but it is also possible to treat them as if they +had a value. + +[source,js] +-------------------------------------------------- +{ + "aggs" : { + "grade_min" : { + "min" : { + "field" : "grade", + "missing": 10 <1> + } + } + } +} +-------------------------------------------------- + +<1> Documents without a value in the `grade` field will fall into the same bucket as documents that have the value `10`. diff --git a/docs/reference/aggregations/metrics/percentile-aggregation.asciidoc b/docs/reference/aggregations/metrics/percentile-aggregation.asciidoc index 6bd10110077..d5262beb6ef 100644 --- a/docs/reference/aggregations/metrics/percentile-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/percentile-aggregation.asciidoc @@ -190,3 +190,25 @@ A "node" uses roughly 32 bytes of memory, so under worst-case scenarios (large a of data which arrives sorted and in-order) the default settings will produce a TDigest roughly 64KB in size. In practice data tends to be more random and the TDigest will use less memory. + +==== Missing value + +The `missing` parameter defines how documents that are missing a value should be treated. +By default they will be ignored but it is also possible to treat them as if they +had a value. + +[source,js] +-------------------------------------------------- +{ + "aggs" : { + "grade_percentiles" : { + "percentiles" : { + "field" : "grade", + "missing": 10 <1> + } + } + } +} +-------------------------------------------------- + +<1> Documents without a value in the `grade` field will fall into the same bucket as documents that have the value `10`. diff --git a/docs/reference/aggregations/metrics/percentile-rank-aggregation.asciidoc b/docs/reference/aggregations/metrics/percentile-rank-aggregation.asciidoc index d327fc66303..a494a0a5d00 100644 --- a/docs/reference/aggregations/metrics/percentile-rank-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/percentile-rank-aggregation.asciidoc @@ -86,3 +86,25 @@ script to generate values which percentile ranks are calculated on <2> Scripting supports parameterized input just like any other script TIP: The `script` parameter expects an inline script. Use `script_id` for indexed scripts and `script_file` for scripts in the `config/scripts/` directory. + +==== Missing value + +The `missing` parameter defines how documents that are missing a value should be treated. +By default they will be ignored but it is also possible to treat them as if they +had a value. + +[source,js] +-------------------------------------------------- +{ + "aggs" : { + "grade_ranks" : { + "percentile_ranks" : { + "field" : "grade", + "missing": 10 <1> + } + } + } +} +-------------------------------------------------- + +<1> Documents without a value in the `grade` field will fall into the same bucket as documents that have the value `10`. diff --git a/docs/reference/aggregations/metrics/stats-aggregation.asciidoc b/docs/reference/aggregations/metrics/stats-aggregation.asciidoc index 7fbdecd6011..429be4b8c4d 100644 --- a/docs/reference/aggregations/metrics/stats-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/stats-aggregation.asciidoc @@ -78,4 +78,26 @@ It turned out that the exam was way above the level of the students and a grade } } } --------------------------------------------------- \ No newline at end of file +-------------------------------------------------- + +==== Missing value + +The `missing` parameter defines how documents that are missing a value should be treated. +By default they will be ignored but it is also possible to treat them as if they +had a value. + +[source,js] +-------------------------------------------------- +{ + "aggs" : { + "grades_stats" : { + "stats" : { + "field" : "grade", + "missing": 0 <1> + } + } + } +} +-------------------------------------------------- + +<1> Documents without a value in the `grade` field will fall into the same bucket as documents that have the value `0`. diff --git a/docs/reference/aggregations/metrics/sum-aggregation.asciidoc b/docs/reference/aggregations/metrics/sum-aggregation.asciidoc index 8857ff306ee..2d16129d15f 100644 --- a/docs/reference/aggregations/metrics/sum-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/sum-aggregation.asciidoc @@ -77,3 +77,25 @@ Computing the sum of squares over all stock tick changes: } } -------------------------------------------------- + +==== Missing value + +The `missing` parameter defines how documents that are missing a value should be treated. +By default they will be ignored but it is also possible to treat them as if they +had a value. + +[source,js] +-------------------------------------------------- +{ + "aggs" : { + "total_time" : { + "sum" : { + "field" : "took", + "missing": 100 <1> + } + } + } +} +-------------------------------------------------- + +<1> Documents without a value in the `took` field will fall into the same bucket as documents that have the value `100`. diff --git a/src/main/java/org/elasticsearch/common/geo/GeoUtils.java b/src/main/java/org/elasticsearch/common/geo/GeoUtils.java index c4bc51d7bfb..77ff66db688 100644 --- a/src/main/java/org/elasticsearch/common/geo/GeoUtils.java +++ b/src/main/java/org/elasticsearch/common/geo/GeoUtils.java @@ -409,19 +409,24 @@ public class GeoUtils { return point.reset(lat, lon); } else if(parser.currentToken() == Token.VALUE_STRING) { String data = parser.text(); - int comma = data.indexOf(','); - if(comma > 0) { - lat = Double.parseDouble(data.substring(0, comma).trim()); - lon = Double.parseDouble(data.substring(comma + 1).trim()); - return point.reset(lat, lon); - } else { - return point.resetFromGeoHash(data); - } + return parseGeoPoint(data, point); } else { throw new ElasticsearchParseException("geo_point expected"); } } + /** parse a {@link GeoPoint} from a String */ + public static GeoPoint parseGeoPoint(String data, GeoPoint point) { + int comma = data.indexOf(','); + if(comma > 0) { + double lat = Double.parseDouble(data.substring(0, comma).trim()); + double lon = Double.parseDouble(data.substring(comma + 1).trim()); + return point.reset(lat, lon); + } else { + return point.resetFromGeoHash(data); + } + } + private GeoUtils() { } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/ValuesSourceAggregationBuilder.java b/src/main/java/org/elasticsearch/search/aggregations/ValuesSourceAggregationBuilder.java index c61164386f8..8d34d68ead3 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/ValuesSourceAggregationBuilder.java +++ b/src/main/java/org/elasticsearch/search/aggregations/ValuesSourceAggregationBuilder.java @@ -34,6 +34,7 @@ public abstract class ValuesSourceAggregationBuilder params; + private Object missing; /** * Constructs a new builder. @@ -117,6 +118,14 @@ public abstract class ValuesSourceAggregationBuilder reducers, Map metaData) throws IOException { super(name, factories, aggregationContext, parent, order, bucketCountThresholds, collectionMode, showTermDocCountError, reducers, @@ -334,7 +334,7 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr private RandomAccessOrds segmentOrds; - public LowCardinality(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, + public LowCardinality(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals valuesSource, Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, List reducers, Map metaData) throws IOException { super(name, factories, valuesSource, order, bucketCountThresholds, null, aggregationContext, parent, collectionMode, showTermDocCountError, reducers, metaData); assert factories == null || factories.count() == 0; diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java index 4056409517e..84196067a86 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java @@ -69,7 +69,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory reducers, Map metaData) throws IOException { final IncludeExclude.OrdinalsFilter filter = includeExclude == null ? null : includeExclude.convertToOrdinalsFilter(); - return new GlobalOrdinalsStringTermsAggregator(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, order, bucketCountThresholds, filter, aggregationContext, parent, subAggCollectMode, showTermDocCountError, reducers, metaData); + return new GlobalOrdinalsStringTermsAggregator(name, factories, (ValuesSource.Bytes.WithOrdinals) valuesSource, order, bucketCountThresholds, filter, aggregationContext, parent, subAggCollectMode, showTermDocCountError, reducers, metaData); } @Override @@ -103,7 +103,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory 0) { return GLOBAL_ORDINALS.create(name, factories, valuesSource, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, reducers, metaData); } - return new GlobalOrdinalsStringTermsAggregator.LowCardinality(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, order, bucketCountThresholds, aggregationContext, parent, subAggCollectMode, showTermDocCountError, reducers, metaData); + return new GlobalOrdinalsStringTermsAggregator.LowCardinality(name, factories, (ValuesSource.Bytes.WithOrdinals) valuesSource, order, bucketCountThresholds, aggregationContext, parent, subAggCollectMode, showTermDocCountError, reducers, metaData); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/ValuesSourceMetricsAggregationBuilder.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/ValuesSourceMetricsAggregationBuilder.java index dd97b3ccfb6..cf2d644b70f 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/ValuesSourceMetricsAggregationBuilder.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/ValuesSourceMetricsAggregationBuilder.java @@ -36,6 +36,7 @@ public abstract class ValuesSourceMetricsAggregationBuilder params; + private Object missing; protected ValuesSourceMetricsAggregationBuilder(String name, String type) { super(name, type); @@ -84,6 +85,14 @@ public abstract class ValuesSourceMetricsAggregationBuilder VS valuesSource(ValuesSourceConfig config) throws IOException { + /** Get a value source given its configuration. A return value of null indicates that + * no value source could be built. */ + @Nullable + public VS valuesSource(ValuesSourceConfig config, SearchContext context) throws IOException { assert config.valid() : "value source config is invalid - must have either a field context or a script or marked as unmapped"; - assert !config.unmapped : "value source should not be created for unmapped fields"; + final VS vs; + if (config.unmapped) { + if (config.missing == null) { + // otherwise we will have values because of the missing value + vs = null; + } else if (ValuesSource.Numeric.class.isAssignableFrom(config.valueSourceType)) { + vs = (VS) ValuesSource.Numeric.EMPTY; + } else if (ValuesSource.GeoPoint.class.isAssignableFrom(config.valueSourceType)) { + vs = (VS) ValuesSource.GeoPoint.EMPTY; + } else if (ValuesSource.class.isAssignableFrom(config.valueSourceType) + || ValuesSource.Bytes.class.isAssignableFrom(config.valueSourceType) + || ValuesSource.Bytes.WithOrdinals.class.isAssignableFrom(config.valueSourceType)) { + vs = (VS) ValuesSource.Bytes.EMPTY; + } else { + throw new SearchParseException(searchContext, "Can't deal with unmapped ValuesSource type " + config.valueSourceType, null); + } + } else { + vs = originalValuesSource(config); + } + + if (config.missing == null) { + return vs; + } + + if (vs instanceof ValuesSource.Bytes) { + final BytesRef missing = new BytesRef(config.missing.toString()); + if (vs instanceof ValuesSource.Bytes.WithOrdinals) { + return (VS) MissingValues.replaceMissing((ValuesSource.Bytes.WithOrdinals) vs, missing); + } else { + return (VS) MissingValues.replaceMissing((ValuesSource.Bytes) vs, missing); + } + } else if (vs instanceof ValuesSource.Numeric) { + Number missing = null; + if (config.missing instanceof Number) { + missing = (Number) config.missing; + } else { + if (config.fieldContext != null && config.fieldContext.mapper() instanceof DateFieldMapper) { + final DateFieldMapper mapper = (DateFieldMapper) config.fieldContext.mapper(); + try { + missing = mapper.dateTimeFormatter().parser().parseDateTime(config.missing.toString()).getMillis(); + } catch (IllegalArgumentException e) { + throw new SearchParseException(context, "Expected a date value in [missing] but got [" + config.missing + "]", null, e); + } + } else { + try { + missing = Double.parseDouble(config.missing.toString()); + } catch (NumberFormatException e) { + throw new SearchParseException(context, "Expected a numeric value in [missing] but got [" + config.missing + "]", null, e); + } + } + } + return (VS) MissingValues.replaceMissing((ValuesSource.Numeric) vs, missing); + } else if (vs instanceof ValuesSource.GeoPoint) { + // TODO: also support the structured formats of geo points + final GeoPoint missing = GeoUtils.parseGeoPoint(config.missing.toString(), new GeoPoint()); + return (VS) MissingValues.replaceMissing((ValuesSource.GeoPoint) vs, missing); + } else { + // Should not happen + throw new SearchParseException(searchContext, "Can't apply missing values on a " + vs.getClass(), null); + } + } + + /** + * Return the original values source, before we apply `missing`. + */ + private VS originalValuesSource(ValuesSourceConfig config) throws IOException { if (config.fieldContext == null) { if (ValuesSource.Numeric.class.isAssignableFrom(config.valueSourceType)) { return (VS) numericScript(config); @@ -111,7 +185,7 @@ public class AggregationContext { } private ValuesSource.GeoPoint geoPointField(ValuesSourceConfig config) throws IOException { - return new ValuesSource.GeoPoint((IndexGeoPointFieldData) config.fieldContext.indexFieldData()); + return new ValuesSource.GeoPoint.Fielddata((IndexGeoPointFieldData) config.fieldContext.indexFieldData()); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/support/MissingValues.java b/src/main/java/org/elasticsearch/search/aggregations/support/MissingValues.java new file mode 100644 index 00000000000..28a4bd2567c --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/support/MissingValues.java @@ -0,0 +1,337 @@ +/* + * 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.support; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.RandomAccessOrds; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.geo.GeoPoint; +import org.elasticsearch.index.fielddata.AbstractRandomAccessOrds; +import org.elasticsearch.index.fielddata.MultiGeoPointValues; +import org.elasticsearch.index.fielddata.SortedBinaryDocValues; +import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; + +import java.io.IOException; + +/** + * Utility class that allows to return views of {@link ValuesSource}s that + * replace the missing value with a configured value. + */ +public enum MissingValues { + ; + + // TODO: we could specialize the single value case + + public static ValuesSource.Bytes replaceMissing(final ValuesSource.Bytes valuesSource, final BytesRef missing) { + return new ValuesSource.Bytes() { + @Override + public SortedBinaryDocValues bytesValues(LeafReaderContext context) throws IOException { + SortedBinaryDocValues values = valuesSource.bytesValues(context); + return replaceMissing(values, missing); + } + }; + } + + static SortedBinaryDocValues replaceMissing(final SortedBinaryDocValues values, final BytesRef missing) { + return new SortedBinaryDocValues() { + + private int count; + + @Override + public BytesRef valueAt(int index) { + if (count > 0) { + return values.valueAt(index); + } else if (index == 0) { + return missing; + } else { + throw new IndexOutOfBoundsException(); + } + } + + @Override + public void setDocument(int docId) { + values.setDocument(docId); + count = values.count(); + } + + @Override + public int count() { + return count == 0 ? 1 : count; + } + }; + } + + public static ValuesSource.Numeric replaceMissing(final ValuesSource.Numeric valuesSource, final Number missing) { + final boolean missingIsFloat = missing.longValue() != (long) missing.doubleValue(); + final boolean isFloatingPoint = valuesSource.isFloatingPoint() || missingIsFloat; + return new ValuesSource.Numeric() { + + @Override + public boolean isFloatingPoint() { + return isFloatingPoint; + } + + @Override + public SortedBinaryDocValues bytesValues(LeafReaderContext context) throws IOException { + return replaceMissing(valuesSource.bytesValues(context), new BytesRef(missing.toString())); + } + + @Override + public SortedNumericDocValues longValues(LeafReaderContext context) throws IOException { + final SortedNumericDocValues values = valuesSource.longValues(context); + return replaceMissing(values, missing.longValue()); + } + + @Override + public SortedNumericDoubleValues doubleValues(LeafReaderContext context) throws IOException { + final SortedNumericDoubleValues values = valuesSource.doubleValues(context); + return replaceMissing(values, missing.doubleValue()); + } + }; + } + + static SortedNumericDocValues replaceMissing(final SortedNumericDocValues values, final long missing) { + return new SortedNumericDocValues() { + + private int count; + + @Override + public void setDocument(int doc) { + values.setDocument(doc); + count = values.count(); + } + + @Override + public long valueAt(int index) { + if (count > 0) { + return values.valueAt(index); + } else if (index == 0) { + return missing; + } else { + throw new IndexOutOfBoundsException(); + } + } + + @Override + public int count() { + return count == 0 ? 1 : count; + } + + }; + } + + static SortedNumericDoubleValues replaceMissing(final SortedNumericDoubleValues values, final double missing) { + return new SortedNumericDoubleValues() { + + private int count; + + @Override + public void setDocument(int doc) { + values.setDocument(doc); + count = values.count(); + } + + @Override + public double valueAt(int index) { + if (count > 0) { + return values.valueAt(index); + } else if (index == 0) { + return missing; + } else { + throw new IndexOutOfBoundsException(); + } + } + + @Override + public int count() { + return count == 0 ? 1 : count; + } + + }; + } + + public static ValuesSource.Bytes replaceMissing(final ValuesSource.Bytes.WithOrdinals valuesSource, final BytesRef missing) { + return new ValuesSource.Bytes.WithOrdinals() { + @Override + public SortedBinaryDocValues bytesValues(LeafReaderContext context) throws IOException { + SortedBinaryDocValues values = valuesSource.bytesValues(context); + return replaceMissing(values, missing); + } + + @Override + public RandomAccessOrds ordinalsValues(LeafReaderContext context) { + RandomAccessOrds values = valuesSource.ordinalsValues(context); + return replaceMissing(values, missing); + } + + @Override + public RandomAccessOrds globalOrdinalsValues(LeafReaderContext context) { + RandomAccessOrds values = valuesSource.globalOrdinalsValues(context); + return replaceMissing(values, missing); + } + }; + } + + static RandomAccessOrds replaceMissing(final RandomAccessOrds values, final BytesRef missing) { + final long missingOrd = values.lookupTerm(missing); + if (missingOrd >= 0) { + // The value already exists + return replaceMissingOrd(values, missingOrd); + } else { + final long insertedOrd = -1 - missingOrd; + return insertOrd(values, insertedOrd, missing); + } + } + + static RandomAccessOrds replaceMissingOrd(final RandomAccessOrds values, final long missingOrd) { + return new AbstractRandomAccessOrds() { + + private int cardinality = 0; + + @Override + public void doSetDocument(int docID) { + values.setDocument(docID); + cardinality = values.cardinality(); + } + + @Override + public BytesRef lookupOrd(long ord) { + return values.lookupOrd(ord); + } + + @Override + public long getValueCount() { + return values.getValueCount(); + } + + @Override + public long ordAt(int index) { + if (cardinality > 0) { + return values.ordAt(index); + } else if (index == 0) { + return missingOrd; + } else { + throw new IndexOutOfBoundsException(); + } + } + + @Override + public int cardinality() { + return cardinality == 0 ? 1 : cardinality; + } + }; + } + + static RandomAccessOrds insertOrd(final RandomAccessOrds values, final long insertedOrd, final BytesRef missingValue) { + return new AbstractRandomAccessOrds() { + + private int cardinality = 0; + + @Override + public void doSetDocument(int docID) { + values.setDocument(docID); + cardinality = values.cardinality(); + } + + @Override + public BytesRef lookupOrd(long ord) { + if (ord < insertedOrd) { + return values.lookupOrd(ord); + } else if (ord > insertedOrd) { + return values.lookupOrd(ord - 1); + } else { + return missingValue; + } + } + + @Override + public long getValueCount() { + return 1 + values.getValueCount(); + } + + @Override + public long ordAt(int index) { + if (cardinality > 0) { + final long ord = values.ordAt(index); + if (ord < insertedOrd) { + return ord; + } else { + return ord + 1; + } + } else if (index == 0) { + return insertedOrd; + } else { + throw new IndexOutOfBoundsException(); + } + } + + @Override + public int cardinality() { + return cardinality == 0 ? 1 : cardinality; + } + }; + } + + public static ValuesSource.GeoPoint replaceMissing(final ValuesSource.GeoPoint valuesSource, final GeoPoint missing) { + return new ValuesSource.GeoPoint() { + + @Override + public SortedBinaryDocValues bytesValues(LeafReaderContext context) throws IOException { + return replaceMissing(valuesSource.bytesValues(context), new BytesRef(missing.toString())); + } + + @Override + public MultiGeoPointValues geoPointValues(LeafReaderContext context) { + final MultiGeoPointValues values = valuesSource.geoPointValues(context); + return replaceMissing(values, missing); + } + }; + } + + static MultiGeoPointValues replaceMissing(final MultiGeoPointValues values, final GeoPoint missing) { + return new MultiGeoPointValues() { + + private int count; + + @Override + public GeoPoint valueAt(int index) { + if (count > 0) { + return values.valueAt(index); + } else if (index == 0) { + return missing; + } else { + throw new IndexOutOfBoundsException(); + } + } + + @Override + public void setDocument(int docId) { + values.setDocument(docId); + count = values.count(); + } + + @Override + public int count() { + return count == 0 ? 1 : count; + } + }; + } +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSource.java b/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSource.java index 90509f8193e..577b3b7a801 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSource.java +++ b/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSource.java @@ -68,6 +68,25 @@ public abstract class ValuesSource { public static abstract class Bytes extends ValuesSource { + public static final WithOrdinals EMPTY = new WithOrdinals() { + + @Override + public RandomAccessOrds ordinalsValues(LeafReaderContext context) { + return DocValues.emptySortedSet(); + } + + @Override + public RandomAccessOrds globalOrdinalsValues(LeafReaderContext context) { + return DocValues.emptySortedSet(); + } + + @Override + public SortedBinaryDocValues bytesValues(LeafReaderContext context) throws IOException { + return org.elasticsearch.index.fielddata.FieldData.emptySortedBinary(context.reader().maxDoc()); + } + + }; + @Override public Bits docsWithValue(LeafReaderContext context) throws IOException { final SortedBinaryDocValues bytes = bytesValues(context); @@ -94,7 +113,16 @@ public abstract class ValuesSource { public abstract RandomAccessOrds globalOrdinalsValues(LeafReaderContext context); - public abstract long globalMaxOrd(IndexSearcher indexSearcher); + public long globalMaxOrd(IndexSearcher indexSearcher) { + IndexReader indexReader = indexSearcher.getIndexReader(); + if (indexReader.leaves().isEmpty()) { + return 0; + } else { + LeafReaderContext atomicReaderContext = indexReader.leaves().get(0); + RandomAccessOrds values = globalOrdinalsValues(atomicReaderContext); + return values.getValueCount(); + } + } public static class FieldData extends WithOrdinals { @@ -122,20 +150,6 @@ public abstract class ValuesSource { final AtomicOrdinalsFieldData atomicFieldData = global.load(context); return atomicFieldData.getOrdinalsValues(); } - - @Override - public long globalMaxOrd(IndexSearcher indexSearcher) { - IndexReader indexReader = indexSearcher.getIndexReader(); - if (indexReader.leaves().isEmpty()) { - return 0; - } else { - LeafReaderContext atomicReaderContext = indexReader.leaves().get(0); - IndexOrdinalsFieldData globalFieldData = indexFieldData.loadGlobal(indexReader); - AtomicOrdinalsFieldData afd = globalFieldData.load(atomicReaderContext); - RandomAccessOrds values = afd.getOrdinalsValues(); - return values.getValueCount(); - } - } } } @@ -212,6 +226,30 @@ public abstract class ValuesSource { public static abstract class Numeric extends ValuesSource { + public static final Numeric EMPTY = new Numeric() { + + @Override + public boolean isFloatingPoint() { + return false; + } + + @Override + public SortedNumericDocValues longValues(LeafReaderContext context) { + return DocValues.emptySortedNumeric(context.reader().maxDoc()); + } + + @Override + public SortedNumericDoubleValues doubleValues(LeafReaderContext context) throws IOException { + return org.elasticsearch.index.fielddata.FieldData.emptySortedNumericDoubles(context.reader().maxDoc()); + } + + @Override + public SortedBinaryDocValues bytesValues(LeafReaderContext context) throws IOException { + return org.elasticsearch.index.fielddata.FieldData.emptySortedBinary(context.reader().maxDoc()); + } + + }; + /** Whether the underlying data is floating-point or not. */ public abstract boolean isFloatingPoint(); @@ -452,13 +490,21 @@ public abstract class ValuesSource { } } - public static class GeoPoint extends ValuesSource { + public static abstract class GeoPoint extends ValuesSource { - protected final IndexGeoPointFieldData indexFieldData; + public static final GeoPoint EMPTY = new GeoPoint() { - public GeoPoint(IndexGeoPointFieldData indexFieldData) { - this.indexFieldData = indexFieldData; - } + @Override + public MultiGeoPointValues geoPointValues(LeafReaderContext context) { + return org.elasticsearch.index.fielddata.FieldData.emptyMultiGeoPoints(context.reader().maxDoc()); + } + + @Override + public SortedBinaryDocValues bytesValues(LeafReaderContext context) throws IOException { + return org.elasticsearch.index.fielddata.FieldData.emptySortedBinary(context.reader().maxDoc()); + } + + }; @Override public Bits docsWithValue(LeafReaderContext context) { @@ -470,13 +516,24 @@ public abstract class ValuesSource { } } - @Override - public SortedBinaryDocValues bytesValues(LeafReaderContext context) { - return indexFieldData.load(context).getBytesValues(); - } + public abstract MultiGeoPointValues geoPointValues(LeafReaderContext context); - public org.elasticsearch.index.fielddata.MultiGeoPointValues geoPointValues(LeafReaderContext context) { - return indexFieldData.load(context).getGeoPointValues(); + public static class Fielddata extends GeoPoint { + + protected final IndexGeoPointFieldData indexFieldData; + + public Fielddata(IndexGeoPointFieldData indexFieldData) { + this.indexFieldData = indexFieldData; + } + + @Override + public SortedBinaryDocValues bytesValues(LeafReaderContext context) { + return indexFieldData.load(context).getBytesValues(); + } + + public org.elasticsearch.index.fielddata.MultiGeoPointValues geoPointValues(LeafReaderContext context) { + return indexFieldData.load(context).getGeoPointValues(); + } } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceAggregatorFactory.java b/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceAggregatorFactory.java index dbefc2e2612..d083ae306cc 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceAggregatorFactory.java +++ b/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceAggregatorFactory.java @@ -57,10 +57,10 @@ public abstract class ValuesSourceAggregatorFactory ext @Override public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, List reducers, Map metaData) throws IOException { - if (config.unmapped()) { + VS vs = context.valuesSource(config, context.searchContext()); + if (vs == null) { return createUnmapped(context, parent, reducers, metaData); } - VS vs = context.valuesSource(config); return doCreateInternal(vs, context, parent, collectsFromSingleBucket, reducers, metaData); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceConfig.java b/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceConfig.java index 3ad7559b431..d855a909c59 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceConfig.java +++ b/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceConfig.java @@ -35,6 +35,7 @@ public class ValuesSourceConfig { boolean unmapped = false; String formatPattern; ValueFormat format; + Object missing; public ValuesSourceConfig(Class valueSourceType) { this.valueSourceType = valueSourceType; diff --git a/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceParser.java b/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceParser.java index 88c3f64b089..f7313885cbb 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceParser.java @@ -68,6 +68,7 @@ public class ValuesSourceParser { Map params = null; ValueType valueType = null; String format = null; + Object missing = null; } private final String aggName; @@ -90,6 +91,10 @@ public class ValuesSourceParser { } public boolean token(String currentFieldName, XContentParser.Token token, XContentParser parser) throws IOException { + if ("missing".equals(currentFieldName) && token.isValue()) { + input.missing = parser.objectText(); + return true; + } if (token == XContentParser.Token.VALUE_STRING) { if ("field".equals(currentFieldName)) { input.field = parser.text(); @@ -146,6 +151,7 @@ public class ValuesSourceParser { valuesSourceType = ValuesSource.Bytes.class; } ValuesSourceConfig config = new ValuesSourceConfig(valuesSourceType); + config.missing = input.missing; config.format = resolveFormat(input.format, valueType); config.script = createScript(); config.scriptValueType = valueType; @@ -156,6 +162,7 @@ public class ValuesSourceParser { if (mapper == null) { Class valuesSourceType = valueType != null ? (Class) valueType.getValuesSourceType() : this.valuesSourceType; ValuesSourceConfig config = new ValuesSourceConfig<>(valuesSourceType); + config.missing = input.missing; config.format = resolveFormat(input.format, valueType); config.unmapped = true; if (valueType != null) { @@ -181,6 +188,7 @@ public class ValuesSourceParser { } config.fieldContext = new FieldContext(input.field, indexFieldData, mapper); + config.missing = input.missing; config.script = createScript(); config.format = resolveFormat(input.format, mapper); return config; diff --git a/src/test/java/org/elasticsearch/search/aggregations/MissingValueTests.java b/src/test/java/org/elasticsearch/search/aggregations/MissingValueTests.java new file mode 100644 index 00000000000..157dc528f80 --- /dev/null +++ b/src/test/java/org/elasticsearch/search/aggregations/MissingValueTests.java @@ -0,0 +1,195 @@ +/* + * 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; + +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.common.geo.GeoPoint; +import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; +import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; +import org.elasticsearch.search.aggregations.bucket.terms.Terms; +import org.elasticsearch.search.aggregations.metrics.cardinality.Cardinality; +import org.elasticsearch.search.aggregations.metrics.geobounds.GeoBounds; +import org.elasticsearch.search.aggregations.metrics.percentiles.Percentiles; +import org.elasticsearch.search.aggregations.metrics.stats.Stats; +import org.elasticsearch.test.ElasticsearchIntegrationTest; + +import static org.elasticsearch.search.aggregations.AggregationBuilders.cardinality; +import static org.elasticsearch.search.aggregations.AggregationBuilders.dateHistogram; +import static org.elasticsearch.search.aggregations.AggregationBuilders.geoBounds; +import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram; +import static org.elasticsearch.search.aggregations.AggregationBuilders.percentiles; +import static org.elasticsearch.search.aggregations.AggregationBuilders.stats; +import static org.elasticsearch.search.aggregations.AggregationBuilders.terms; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse; + +@ElasticsearchIntegrationTest.SuiteScopeTest +public class MissingValueTests extends ElasticsearchIntegrationTest { + + @Override + protected int maximumNumberOfShards() { + return 2; + } + + @Override + protected void setupSuiteScopeCluster() throws Exception { + assertAcked(prepareCreate("idx").addMapping("type", "date", "type=date", "location", "type=geo_point").get()); + indexRandom(true, + client().prepareIndex("idx", "type", "1").setSource(), + client().prepareIndex("idx", "type", "2").setSource("str", "foo", "long", 3L, "double", 5.5, "date", "2015-05-07", "location", "1,2")); + } + + public void testUnmappedTerms() { + SearchResponse response = client().prepareSearch("idx").addAggregation(terms("my_terms").field("non_existing_field").missing("bar")).get(); + assertSearchResponse(response); + Terms terms = response.getAggregations().get("my_terms"); + assertEquals(1, terms.getBuckets().size()); + assertEquals(2, terms.getBucketByKey("bar").getDocCount()); + } + + public void testStringTerms() { + SearchResponse response = client().prepareSearch("idx").addAggregation(terms("my_terms").field("str").missing("bar")).get(); + assertSearchResponse(response); + Terms terms = response.getAggregations().get("my_terms"); + assertEquals(2, terms.getBuckets().size()); + assertEquals(1, terms.getBucketByKey("foo").getDocCount()); + assertEquals(1, terms.getBucketByKey("bar").getDocCount()); + + response = client().prepareSearch("idx").addAggregation(terms("my_terms").field("str").missing("foo")).get(); + assertSearchResponse(response); + terms = response.getAggregations().get("my_terms"); + assertEquals(1, terms.getBuckets().size()); + assertEquals(2, terms.getBucketByKey("foo").getDocCount()); + } + + public void testLongTerms() { + SearchResponse response = client().prepareSearch("idx").addAggregation(terms("my_terms").field("long").missing(4)).get(); + assertSearchResponse(response); + Terms terms = response.getAggregations().get("my_terms"); + assertEquals(2, terms.getBuckets().size()); + assertEquals(1, terms.getBucketByKey("3").getDocCount()); + assertEquals(1, terms.getBucketByKey("4").getDocCount()); + + response = client().prepareSearch("idx").addAggregation(terms("my_terms").field("long").missing(3)).get(); + assertSearchResponse(response); + terms = response.getAggregations().get("my_terms"); + assertEquals(1, terms.getBuckets().size()); + assertEquals(2, terms.getBucketByKey("3").getDocCount()); + } + + public void testDoubleTerms() { + SearchResponse response = client().prepareSearch("idx").addAggregation(terms("my_terms").field("double").missing(4.5)).get(); + assertSearchResponse(response); + Terms terms = response.getAggregations().get("my_terms"); + assertEquals(2, terms.getBuckets().size()); + assertEquals(1, terms.getBucketByKey("4.5").getDocCount()); + assertEquals(1, terms.getBucketByKey("5.5").getDocCount()); + + response = client().prepareSearch("idx").addAggregation(terms("my_terms").field("double").missing(5.5)).get(); + assertSearchResponse(response); + terms = response.getAggregations().get("my_terms"); + assertEquals(1, terms.getBuckets().size()); + assertEquals(2, terms.getBucketByKey("5.5").getDocCount()); + } + + public void testUnmappedHistogram() { + SearchResponse response = client().prepareSearch("idx").addAggregation(histogram("my_histogram").field("non-existing_field").interval(5).missing(12)).get(); + assertSearchResponse(response); + Histogram histogram = response.getAggregations().get("my_histogram"); + assertEquals(1, histogram.getBuckets().size()); + assertEquals(10L, histogram.getBuckets().get(0).getKey()); + assertEquals(2, histogram.getBuckets().get(0).getDocCount()); + } + + public void testHistogram() { + SearchResponse response = client().prepareSearch("idx").addAggregation(histogram("my_histogram").field("long").interval(5).missing(7)).get(); + assertSearchResponse(response); + Histogram histogram = response.getAggregations().get("my_histogram"); + assertEquals(2, histogram.getBuckets().size()); + assertEquals(0L, histogram.getBuckets().get(0).getKey()); + assertEquals(1, histogram.getBuckets().get(0).getDocCount()); + assertEquals(5L, histogram.getBuckets().get(1).getKey()); + assertEquals(1, histogram.getBuckets().get(1).getDocCount()); + + response = client().prepareSearch("idx").addAggregation(histogram("my_histogram").field("long").interval(5).missing(3)).get(); + assertSearchResponse(response); + histogram = response.getAggregations().get("my_histogram"); + assertEquals(1, histogram.getBuckets().size()); + assertEquals(0L, histogram.getBuckets().get(0).getKey()); + assertEquals(2, histogram.getBuckets().get(0).getDocCount()); + } + + public void testDateHistogram() { + SearchResponse response = client().prepareSearch("idx").addAggregation(dateHistogram("my_histogram").field("date").interval(DateHistogramInterval.YEAR).missing("2014-05-07")).get(); + assertSearchResponse(response); + Histogram histogram = response.getAggregations().get("my_histogram"); + assertEquals(2, histogram.getBuckets().size()); + assertEquals("2014-01-01T00:00:00.000Z", histogram.getBuckets().get(0).getKeyAsString()); + assertEquals(1, histogram.getBuckets().get(0).getDocCount()); + assertEquals("2015-01-01T00:00:00.000Z", histogram.getBuckets().get(1).getKeyAsString()); + assertEquals(1, histogram.getBuckets().get(1).getDocCount()); + + response = client().prepareSearch("idx").addAggregation(dateHistogram("my_histogram").field("date").interval(DateHistogramInterval.YEAR).missing("2015-05-07")).get(); + assertSearchResponse(response); + histogram = response.getAggregations().get("my_histogram"); + assertEquals(1, histogram.getBuckets().size()); + assertEquals("2015-01-01T00:00:00.000Z", histogram.getBuckets().get(0).getKeyAsString()); + assertEquals(2, histogram.getBuckets().get(0).getDocCount()); + } + + public void testCardinality() { + SearchResponse response = client().prepareSearch("idx").addAggregation(cardinality("card").field("long").missing(2)).get(); + assertSearchResponse(response); + Cardinality cardinality = response.getAggregations().get("card"); + assertEquals(2, cardinality.getValue()); + } + + public void testPercentiles() { + SearchResponse response = client().prepareSearch("idx").addAggregation(percentiles("percentiles").field("long").missing(1000)).get(); + assertSearchResponse(response); + Percentiles percentiles = response.getAggregations().get("percentiles"); + assertEquals(1000, percentiles.percentile(100), 0); + } + + public void testStats() { + SearchResponse response = client().prepareSearch("idx").addAggregation(stats("stats").field("long").missing(5)).get(); + assertSearchResponse(response); + Stats stats = response.getAggregations().get("stats"); + assertEquals(2, stats.getCount()); + assertEquals(4, stats.getAvg(), 0); + } + + public void testUnmappedGeoBounds() { + SearchResponse response = client().prepareSearch("idx").addAggregation(geoBounds("bounds").field("non_existing_field").missing("2,1")).get(); + assertSearchResponse(response); + GeoBounds bounds = response.getAggregations().get("bounds"); + assertEquals(new GeoPoint(2,1), bounds.bottomRight()); + assertEquals(new GeoPoint(2,1), bounds.topLeft()); + } + + public void testGeoBounds() { + SearchResponse response = client().prepareSearch("idx").addAggregation(geoBounds("bounds").field("location").missing("2,1")).get(); + assertSearchResponse(response); + GeoBounds bounds = response.getAggregations().get("bounds"); + assertEquals(new GeoPoint(1,2), bounds.bottomRight()); + assertEquals(new GeoPoint(2,1), bounds.topLeft()); + } + +} diff --git a/src/test/java/org/elasticsearch/search/aggregations/support/MissingValuesTests.java b/src/test/java/org/elasticsearch/search/aggregations/support/MissingValuesTests.java new file mode 100644 index 00000000000..cd72d7f069b --- /dev/null +++ b/src/test/java/org/elasticsearch/search/aggregations/support/MissingValuesTests.java @@ -0,0 +1,297 @@ +/* + * 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.support; + +import com.carrotsearch.randomizedtesting.generators.RandomPicks; +import com.carrotsearch.randomizedtesting.generators.RandomStrings; + +import org.apache.lucene.index.RandomAccessOrds; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.TestUtil; +import org.elasticsearch.common.geo.GeoPoint; +import org.elasticsearch.index.fielddata.AbstractRandomAccessOrds; +import org.elasticsearch.index.fielddata.MultiGeoPointValues; +import org.elasticsearch.index.fielddata.SortedBinaryDocValues; +import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; +import org.elasticsearch.test.ElasticsearchTestCase; + +import java.util.Arrays; +import java.util.HashSet; +import java.util.Set; + +public class MissingValuesTests extends ElasticsearchTestCase { + + public void testMissingBytes() { + final int numDocs = TestUtil.nextInt(random(), 1, 100); + final BytesRef[][] values = new BytesRef[numDocs][]; + for (int i = 0; i < numDocs; ++i) { + values[i] = new BytesRef[random().nextInt(4)]; + for (int j = 0; j < values[i].length; ++j) { + values[i][j] = new BytesRef(RandomStrings.randomAsciiOfLength(random(), 2)); + } + Arrays.sort(values[i]); + } + SortedBinaryDocValues asBinaryValues = new SortedBinaryDocValues() { + + int i = -1; + + @Override + public BytesRef valueAt(int index) { + return values[i][index]; + } + + @Override + public void setDocument(int docId) { + i = docId; + } + + @Override + public int count() { + return values[i].length; + } + }; + final BytesRef missing = new BytesRef(RandomStrings.randomAsciiOfLength(random(), 2)); + SortedBinaryDocValues withMissingReplaced = MissingValues.replaceMissing(asBinaryValues, missing); + for (int i = 0; i < numDocs; ++i) { + withMissingReplaced.setDocument(i); + if (values[i].length > 0) { + assertEquals(values[i].length, withMissingReplaced.count()); + for (int j = 0; j < values[i].length; ++j) { + assertEquals(values[i][j], withMissingReplaced.valueAt(j)); + } + } else { + assertEquals(1, withMissingReplaced.count()); + assertEquals(missing, withMissingReplaced.valueAt(0)); + } + } + } + + public void testMissingOrds() { + final int numDocs = TestUtil.nextInt(random(), 1, 100); + final int numOrds = TestUtil.nextInt(random(), 1, 10); + + final Set valueSet = new HashSet<>(); + while (valueSet.size() < numOrds) { + valueSet.add(new BytesRef(RandomStrings.randomAsciiOfLength(random(), 5))); + } + final BytesRef[] values = valueSet.toArray(new BytesRef[numOrds]); + Arrays.sort(values); + + final int[][] ords = new int[numDocs][]; + for (int i = 0; i < numDocs; ++i) { + ords[i] = new int[random().nextInt(numOrds)]; + for (int j = 0; j < ords[i].length; ++j) { + ords[i][j] = j; + } + for (int j = ords[i].length - 1; j >= 0; --j) { + final int maxOrd = j == ords[i].length - 1 ? numOrds : ords[i][j+1]; + ords[i][j] = TestUtil.nextInt(random(), ords[i][j], maxOrd - 1); + } + } + RandomAccessOrds asRandomAccessOrds = new AbstractRandomAccessOrds() { + + int i = -1; + + @Override + public void doSetDocument(int docID) { + i = docID; + } + + @Override + public BytesRef lookupOrd(long ord) { + return values[(int) ord]; + } + + @Override + public long getValueCount() { + return values.length; + } + + @Override + public long ordAt(int index) { + return ords[i][index]; + } + + @Override + public int cardinality() { + return ords[i].length; + } + }; + + final BytesRef existingMissing = RandomPicks.randomFrom(random(), values); + final BytesRef missingMissing = new BytesRef(RandomStrings.randomAsciiOfLength(random(), 5)); + + for (BytesRef missing : Arrays.asList(existingMissing, missingMissing)) { + RandomAccessOrds withMissingReplaced = MissingValues.replaceMissing(asRandomAccessOrds, missing); + if (valueSet.contains(missing)) { + assertEquals(values.length, withMissingReplaced.getValueCount()); + } else { + assertEquals(values.length + 1, withMissingReplaced.getValueCount()); + } + for (int i = 0; i < numDocs; ++i) { + withMissingReplaced.setDocument(i); + if (ords[i].length > 0) { + assertEquals(ords[i].length, withMissingReplaced.cardinality()); + for (int j = 0; j < ords[i].length; ++j) { + assertEquals(values[ords[i][j]], withMissingReplaced.lookupOrd(withMissingReplaced.ordAt(j))); + } + } else { + assertEquals(1, withMissingReplaced.cardinality()); + assertEquals(missing, withMissingReplaced.lookupOrd(withMissingReplaced.ordAt(0))); + } + } + } + } + + public void testMissingLongs() { + final int numDocs = TestUtil.nextInt(random(), 1, 100); + final int[][] values = new int[numDocs][]; + for (int i = 0; i < numDocs; ++i) { + values[i] = new int[random().nextInt(4)]; + for (int j = 0; j < values[i].length; ++j) { + values[i][j] = randomInt(); + } + Arrays.sort(values[i]); + } + SortedNumericDocValues asNumericValues = new SortedNumericDocValues() { + + int i = -1; + + @Override + public long valueAt(int index) { + return values[i][index]; + } + + @Override + public void setDocument(int docId) { + i = docId; + } + + @Override + public int count() { + return values[i].length; + } + }; + final long missing = randomInt(); + SortedNumericDocValues withMissingReplaced = MissingValues.replaceMissing(asNumericValues, missing); + for (int i = 0; i < numDocs; ++i) { + withMissingReplaced.setDocument(i); + if (values[i].length > 0) { + assertEquals(values[i].length, withMissingReplaced.count()); + for (int j = 0; j < values[i].length; ++j) { + assertEquals(values[i][j], withMissingReplaced.valueAt(j)); + } + } else { + assertEquals(1, withMissingReplaced.count()); + assertEquals(missing, withMissingReplaced.valueAt(0)); + } + } + } + + public void testMissingDoubles() { + final int numDocs = TestUtil.nextInt(random(), 1, 100); + final double[][] values = new double[numDocs][]; + for (int i = 0; i < numDocs; ++i) { + values[i] = new double[random().nextInt(4)]; + for (int j = 0; j < values[i].length; ++j) { + values[i][j] = randomDouble(); + } + Arrays.sort(values[i]); + } + SortedNumericDoubleValues asNumericValues = new SortedNumericDoubleValues() { + + int i = -1; + + @Override + public double valueAt(int index) { + return values[i][index]; + } + + @Override + public void setDocument(int docId) { + i = docId; + } + + @Override + public int count() { + return values[i].length; + } + }; + final long missing = randomInt(); + SortedNumericDoubleValues withMissingReplaced = MissingValues.replaceMissing(asNumericValues, missing); + for (int i = 0; i < numDocs; ++i) { + withMissingReplaced.setDocument(i); + if (values[i].length > 0) { + assertEquals(values[i].length, withMissingReplaced.count()); + for (int j = 0; j < values[i].length; ++j) { + assertEquals(values[i][j], withMissingReplaced.valueAt(j), 0); + } + } else { + assertEquals(1, withMissingReplaced.count()); + assertEquals(missing, withMissingReplaced.valueAt(0), 0); + } + } + } + + public void testMissingGeoPoints() { + final int numDocs = TestUtil.nextInt(random(), 1, 100); + final GeoPoint[][] values = new GeoPoint[numDocs][]; + for (int i = 0; i < numDocs; ++i) { + values[i] = new GeoPoint[random().nextInt(4)]; + for (int j = 0; j < values[i].length; ++j) { + values[i][j] = new GeoPoint(randomDouble() * 90, randomDouble() * 180); + } + } + MultiGeoPointValues asGeoValues = new MultiGeoPointValues() { + + int i = -1; + + @Override + public GeoPoint valueAt(int index) { + return values[i][index]; + } + + @Override + public void setDocument(int docId) { + i = docId; + } + + @Override + public int count() { + return values[i].length; + } + }; + final GeoPoint missing = new GeoPoint(randomDouble() * 90, randomDouble() * 180); + MultiGeoPointValues withMissingReplaced = MissingValues.replaceMissing(asGeoValues, missing); + for (int i = 0; i < numDocs; ++i) { + withMissingReplaced.setDocument(i); + if (values[i].length > 0) { + assertEquals(values[i].length, withMissingReplaced.count()); + for (int j = 0; j < values[i].length; ++j) { + assertEquals(values[i][j], withMissingReplaced.valueAt(j)); + } + } else { + assertEquals(1, withMissingReplaced.count()); + assertEquals(missing, withMissingReplaced.valueAt(0)); + } + } + } +} From a42d92df21f774aa1f3fd51550ea570f14dfaba5 Mon Sep 17 00:00:00 2001 From: Michael McCandless Date: Fri, 15 May 2015 10:47:17 -0400 Subject: [PATCH 13/56] Small fixes: - rename one method - move "generation == null" check under existing "if (createNew == false)" - fix typo/whitespace - add a TODO --- .../index/engine/InternalEngine.java | 10 +++++++--- .../index/engine/InternalEngineFactory.java | 1 + .../elasticsearch/index/translog/Translog.java | 16 +++++++--------- 3 files changed, 15 insertions(+), 12 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 50c8f7104a3..9952178b542 100644 --- a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -131,6 +131,8 @@ public class InternalEngine extends Engine { this.searcherFactory = new SearchFactory(engineConfig); final Translog.TranslogGeneration translogGeneration; try { + // TODO: would be better if ES could tell us "from above" whether this shard was already here, instead of using Lucene's API + // (which relies on IO ops, directory listing, and has had scary bugs in the past): boolean create = !Lucene.indexExists(store.directory()); writer = createWriter(create); indexWriter = writer; @@ -175,7 +177,12 @@ public class InternalEngine extends Engine { private Translog openTranslog(EngineConfig engineConfig, IndexWriter writer, boolean createNew) throws IOException { final Translog.TranslogGeneration generation = loadTranslogIdFromCommit(writer); final TranslogConfig translogConfig = engineConfig.getTranslogConfig(); + if (createNew == false) { + // We expect that this shard already exists, so it must already have an existing translog else something is badly wrong! + if (generation == null) { + throw new IllegalStateException("no translog generation present in commit data but translog is expected to exist"); + } translogConfig.setTranslogGeneration(generation); if (generation != null && generation.translogUUID == null) { // only upgrade on pre-2.0 indices... @@ -184,9 +191,6 @@ public class InternalEngine extends Engine { } final Translog translog = new Translog(translogConfig); if (generation == null) { - if (createNew == false) { - throw new IllegalStateException("no tranlog generation present in commit data but translog is expected to exist"); - } logger.debug("no translog ID present in the current generation - creating one"); commitIndexWriter(writer, translog); } diff --git a/src/main/java/org/elasticsearch/index/engine/InternalEngineFactory.java b/src/main/java/org/elasticsearch/index/engine/InternalEngineFactory.java index fdf708cfd51..7b458f93904 100644 --- a/src/main/java/org/elasticsearch/index/engine/InternalEngineFactory.java +++ b/src/main/java/org/elasticsearch/index/engine/InternalEngineFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.elasticsearch.index.engine; public class InternalEngineFactory implements EngineFactory { diff --git a/src/main/java/org/elasticsearch/index/translog/Translog.java b/src/main/java/org/elasticsearch/index/translog/Translog.java index b247189fb5f..8bc71be6576 100644 --- a/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -251,12 +251,10 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC Checkpoint.write(translogPath.resolve(CHECKPOINT_FILE_NAME), checkpoint, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW); } else { Checkpoint checkpoint = new Checkpoint(Files.size(target), -1, generation); - Checkpoint.write(translogPath.resolve(getCommitFileName(generation)), checkpoint, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW); + Checkpoint.write(translogPath.resolve(getCommitCheckpointFileName(generation)), checkpoint, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW); } } - - IOUtils.fsync(translogPath, true); } @@ -275,12 +273,12 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC if (Files.exists(committedTranslogFile) == false) { throw new IllegalStateException("translog file doesn't exist with generation: " + i + " lastCommitted: " + lastCommittedTranslogFileGeneration + " checkpoint: " + checkpoint.generation + " - translog ids must be consecutive"); } - final ImmutableTranslogReader reader = openReader(committedTranslogFile, Checkpoint.read(location.resolve(getCommitFileName(i)))); + final ImmutableTranslogReader reader = openReader(committedTranslogFile, Checkpoint.read(location.resolve(getCommitCheckpointFileName(i)))); foundTranslogs.add(reader); logger.debug("recovered local translog from checkpoint {}", checkpoint); } foundTranslogs.add(openReader(location.resolve(checkpointTranslogFile), checkpoint)); - Path commitCheckpoint = location.resolve(getCommitFileName(checkpoint.generation)); + Path commitCheckpoint = location.resolve(getCommitCheckpointFileName(checkpoint.generation)); Files.copy(location.resolve(CHECKPOINT_FILE_NAME), commitCheckpoint); IOUtils.fsync(commitCheckpoint, false); IOUtils.fsync(commitCheckpoint.getParent(), true); @@ -544,7 +542,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC return TRANSLOG_FILE_PREFIX + generation + TRANSLOG_FILE_SUFFIX; } - static String getCommitFileName(long generation) { + static String getCommitCheckpointFileName(long generation) { return TRANSLOG_FILE_PREFIX + generation + CHECKPOINT_SUFFIX; } @@ -591,7 +589,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC // if the given translogPath is not the current we can safely delete the file since all references are released logger.trace("delete translog file - not referenced and not current anymore {}", translogPath); IOUtils.deleteFilesIgnoringExceptions(translogPath); - IOUtils.deleteFilesIgnoringExceptions(translogPath.resolveSibling(getCommitFileName(channelReference.getGeneration()))); + IOUtils.deleteFilesIgnoringExceptions(translogPath.resolveSibling(getCommitCheckpointFileName(channelReference.getGeneration()))); } try (DirectoryStream stream = Files.newDirectoryStream(location)) { @@ -602,7 +600,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC if (isReferencedGeneration(generation) == false) { logger.trace("delete translog file - not referenced and not current anymore {}", path); IOUtils.deleteFilesIgnoringExceptions(path); - IOUtils.deleteFilesIgnoringExceptions(path.resolveSibling(getCommitFileName(channelReference.getGeneration()))); + IOUtils.deleteFilesIgnoringExceptions(path.resolveSibling(getCommitCheckpointFileName(channelReference.getGeneration()))); } } } @@ -1660,7 +1658,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC currentCommittingTranslog = current.immutableReader(); Path checkpoint = location.resolve(CHECKPOINT_FILE_NAME); assert Checkpoint.read(checkpoint).generation == currentCommittingTranslog.getGeneration(); - Path commitCheckpoint = location.resolve(getCommitFileName(currentCommittingTranslog.getGeneration())); + Path commitCheckpoint = location.resolve(getCommitCheckpointFileName(currentCommittingTranslog.getGeneration())); Files.copy(checkpoint, commitCheckpoint); IOUtils.fsync(commitCheckpoint, false); IOUtils.fsync(commitCheckpoint.getParent(), true); From 5d4db0edde9353fce3be08bfb7319c974a0b5df2 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Fri, 15 May 2015 17:03:45 +0200 Subject: [PATCH 14/56] Remove accidential modification --- .../org/elasticsearch/gateway/RecoveryFromGatewayTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayTests.java b/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayTests.java index f48777f83a3..b072d8c2ced 100644 --- a/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayTests.java +++ b/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayTests.java @@ -56,7 +56,7 @@ import static org.hamcrest.Matchers.notNullValue; */ @ClusterScope(numDataNodes = 0, scope = Scope.TEST) @Slow -public class RecoveryFromGatewayTesTermVectorsFields.javats extends ElasticsearchIntegrationTest { +public class RecoveryFromGatewayTests extends ElasticsearchIntegrationTest { @Test @Slow From 597c53a0bb36dafde0ffb17edb5792c1b5422f00 Mon Sep 17 00:00:00 2001 From: Jun Ohtani Date: Sat, 16 May 2015 00:25:53 +0900 Subject: [PATCH 15/56] Add migrationi note for AnalyzeRequest --- docs/reference/migration/migrate_2_0.asciidoc | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/reference/migration/migrate_2_0.asciidoc b/docs/reference/migration/migrate_2_0.asciidoc index b6d3e29e360..382dd02cace 100644 --- a/docs/reference/migration/migrate_2_0.asciidoc +++ b/docs/reference/migration/migrate_2_0.asciidoc @@ -498,7 +498,8 @@ systems and the provided start/stop scripts. === Analyze API -The Analyze API return 0 as first Token's position instead of 1. +* The Analyze API return 0 as first Token's position instead of 1. +* The `text()` method on `AnalyzeRequest` now returns `String[]` instead of `String`. === Multiple data.path striping From 22bba91a160d306b14588e897cf0c34423a58da3 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Fri, 15 May 2015 09:52:13 +0200 Subject: [PATCH 16/56] Search: Make SCAN faster. When scrolling, SCAN previously collected documents until it reached where it had stopped on the previous iteration. This makes pagination slower and slower as you request deep pages. With this change, SCAN now directly jumps to the doc ID where is had previously stopped. --- .../common/lucene/docset/AllDocIdSet.java | 100 --------- .../search/internal/DefaultSearchContext.java | 4 +- .../search/scan/ScanContext.java | 196 ++++++++++-------- .../search/scan/ScanContextTests.java | 116 +++++++++++ 4 files changed, 227 insertions(+), 189 deletions(-) delete mode 100644 src/main/java/org/elasticsearch/common/lucene/docset/AllDocIdSet.java create mode 100644 src/test/java/org/elasticsearch/search/scan/ScanContextTests.java diff --git a/src/main/java/org/elasticsearch/common/lucene/docset/AllDocIdSet.java b/src/main/java/org/elasticsearch/common/lucene/docset/AllDocIdSet.java deleted file mode 100644 index fe7613879df..00000000000 --- a/src/main/java/org/elasticsearch/common/lucene/docset/AllDocIdSet.java +++ /dev/null @@ -1,100 +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.common.lucene.docset; - -import org.apache.lucene.search.DocIdSet; -import org.apache.lucene.search.DocIdSetIterator; -import org.apache.lucene.util.Bits; -import org.apache.lucene.util.RamUsageEstimator; - -import java.io.IOException; - -/** - * A {@link DocIdSet} that matches all docs up to a {@code maxDoc}. - */ -public class AllDocIdSet extends DocIdSet { - - private final int maxDoc; - - public AllDocIdSet(int maxDoc) { - this.maxDoc = maxDoc; - } - - /** - * Does not go to the reader and ask for data, so can be cached. - */ - @Override - public boolean isCacheable() { - return true; - } - - @Override - public long ramBytesUsed() { - return RamUsageEstimator.NUM_BYTES_INT; - } - - @Override - public DocIdSetIterator iterator() throws IOException { - return new Iterator(maxDoc); - } - - @Override - public Bits bits() throws IOException { - return new Bits.MatchAllBits(maxDoc); - } - - public static final class Iterator extends DocIdSetIterator { - - private final int maxDoc; - private int doc = -1; - - public Iterator(int maxDoc) { - this.maxDoc = maxDoc; - } - - @Override - public int docID() { - return doc; - } - - @Override - public int nextDoc() throws IOException { - if (++doc < maxDoc) { - return doc; - } - return doc = NO_MORE_DOCS; - } - - @Override - public int advance(int target) throws IOException { - doc = target; - if (doc < maxDoc) { - return doc; - } - return doc = NO_MORE_DOCS; - } - - @Override - public long cost() { - - return maxDoc; - } - } -} diff --git a/src/main/java/org/elasticsearch/search/internal/DefaultSearchContext.java b/src/main/java/org/elasticsearch/search/internal/DefaultSearchContext.java index 001219d5129..4bbb3ec09af 100644 --- a/src/main/java/org/elasticsearch/search/internal/DefaultSearchContext.java +++ b/src/main/java/org/elasticsearch/search/internal/DefaultSearchContext.java @@ -204,9 +204,7 @@ public class DefaultSearchContext extends SearchContext { @Override public void doClose() { - if (scanContext != null) { - scanContext.clear(); - } + scanContext = null; // clear and scope phase we have Releasables.close(searcher, engineSearcher); } diff --git a/src/main/java/org/elasticsearch/search/scan/ScanContext.java b/src/main/java/org/elasticsearch/search/scan/ScanContext.java index 14bf1feb3ad..7aaead1c42f 100644 --- a/src/main/java/org/elasticsearch/search/scan/ScanContext.java +++ b/src/main/java/org/elasticsearch/search/scan/ScanContext.java @@ -19,59 +19,50 @@ package org.elasticsearch.search.scan; -import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.search.BitsFilteredDocIdSet; -import org.apache.lucene.search.DocIdSet; -import org.apache.lucene.search.Filter; -import org.apache.lucene.search.FilteredQuery; +import org.apache.lucene.search.CollectionTerminatedException; +import org.apache.lucene.search.ConstantScoreScorer; +import org.apache.lucene.search.ConstantScoreWeight; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.Scorer; import org.apache.lucene.search.SimpleCollector; import org.apache.lucene.search.TopDocs; +import org.apache.lucene.search.Weight; import org.apache.lucene.util.Bits; -import org.elasticsearch.common.lucene.docset.AllDocIdSet; -import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; import java.util.ArrayList; -import java.util.concurrent.ConcurrentMap; +import java.util.List; /** * The scan context allows to optimize readers we already processed during scanning. We do that by keeping track - * of the count per reader, and if we are done with it, we no longer process it by using a filter that returns - * null docIdSet for this reader. + * of the last collected doc ID and only collecting doc IDs that are greater. */ public class ScanContext { - private final ConcurrentMap readerStates = ConcurrentCollections.newConcurrentMap(); - - public void clear() { - readerStates.clear(); - } + private volatile int docUpTo; public TopDocs execute(SearchContext context) throws IOException { - ScanCollector collector = new ScanCollector(readerStates, context.from(), context.size(), context.trackScores()); - Query query = new FilteredQuery(context.query(), new ScanFilter(readerStates, collector)); - try { - context.searcher().search(query, collector); - } catch (ScanCollector.StopCollectingException e) { - // all is well - } + return execute(context.searcher(), context.query(), context.size(), context.trackScores()); + } + + TopDocs execute(IndexSearcher searcher, Query query, int size, boolean trackScores) throws IOException { + ScanCollector collector = new ScanCollector(size, trackScores); + Query q = Queries.filtered(query, new MinDocQuery(docUpTo)); + searcher.search(q, collector); return collector.topDocs(); } - static class ScanCollector extends SimpleCollector { + private class ScanCollector extends SimpleCollector { - private final ConcurrentMap readerStates; + private final List docs; - private final int from; - - private final int to; - - private final ArrayList docs; + private final int size; private final boolean trackScores; @@ -79,21 +70,10 @@ public class ScanContext { private int docBase; - private int counter; - - private IndexReader currentReader; - private ReaderState readerState; - - ScanCollector(ConcurrentMap readerStates, int from, int size, boolean trackScores) { - this.readerStates = readerStates; - this.from = from; - this.to = from + size; + ScanCollector(int size, boolean trackScores) { this.trackScores = trackScores; this.docs = new ArrayList<>(size); - } - - void incCounter(int count) { - this.counter += count; + this.size = size; } public TopDocs topDocs() { @@ -112,70 +92,114 @@ public class ScanContext { @Override public void collect(int doc) throws IOException { - if (counter >= from) { - docs.add(new ScoreDoc(docBase + doc, trackScores ? scorer.score() : 0f)); - } - readerState.count++; - counter++; - if (counter >= to) { - throw StopCollectingException; + int topLevelDoc = docBase + doc; + docs.add(new ScoreDoc(topLevelDoc, trackScores ? scorer.score() : 0f)); + // record that we collected up to this document + assert topLevelDoc >= docUpTo; + docUpTo = topLevelDoc + 1; + if (docs.size() >= size) { + throw new CollectionTerminatedException(); } } @Override public void doSetNextReader(LeafReaderContext context) throws IOException { - // if we have a reader state, and we haven't registered one already, register it - // we need to check in readersState since even when the filter return null, setNextReader is still - // called for that reader (before) - if (currentReader != null && !readerStates.containsKey(currentReader)) { - assert readerState != null; - readerState.done = true; - readerStates.put(currentReader, readerState); - } - this.currentReader = context.reader(); - this.docBase = context.docBase; - this.readerState = new ReaderState(); - } - - public static final RuntimeException StopCollectingException = new StopCollectingException(); - - static class StopCollectingException extends RuntimeException { - @Override - public Throwable fillInStackTrace() { - return null; + if (docs.size() >= size || context.docBase + context.reader().maxDoc() <= docUpTo) { + // no need to collect a new segment, we either already collected enough + // or the segment is not competitive + throw new CollectionTerminatedException(); } + docBase = context.docBase; } } - public static class ScanFilter extends Filter { + /** + * A filtering query that matches all doc IDs that are not deleted and + * greater than or equal to the configured doc ID. + */ + // pkg-private for testing + static class MinDocQuery extends Query { - private final ConcurrentMap readerStates; + private final int minDoc; - private final ScanCollector scanCollector; - - public ScanFilter(ConcurrentMap readerStates, ScanCollector scanCollector) { - this.readerStates = readerStates; - this.scanCollector = scanCollector; + MinDocQuery(int minDoc) { + this.minDoc = minDoc; } @Override - public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptedDocs) throws IOException { - ReaderState readerState = readerStates.get(context.reader()); - if (readerState != null && readerState.done) { - scanCollector.incCounter(readerState.count); - return null; + public int hashCode() { + return 31 * super.hashCode() + minDoc; + } + + @Override + public boolean equals(Object obj) { + if (super.equals(obj) == false) { + return false; } - return BitsFilteredDocIdSet.wrap(new AllDocIdSet(context.reader().maxDoc()), acceptedDocs); + MinDocQuery that = (MinDocQuery) obj; + return minDoc == that.minDoc; + } + + @Override + public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException { + return new ConstantScoreWeight(this) { + @Override + public Scorer scorer(LeafReaderContext context, final Bits acceptDocs) throws IOException { + final int maxDoc = context.reader().maxDoc(); + if (context.docBase + maxDoc <= minDoc) { + return null; + } + final int segmentMinDoc = Math.max(0, minDoc - context.docBase); + final DocIdSetIterator disi = new DocIdSetIterator() { + + int doc = -1; + + @Override + public int docID() { + return doc; + } + + @Override + public int nextDoc() throws IOException { + return advance(doc + 1); + } + + @Override + public int advance(int target) throws IOException { + assert target > doc; + if (doc == -1) { + // skip directly to minDoc + doc = Math.max(target, segmentMinDoc); + } else { + doc = target; + } + while (doc < maxDoc) { + if (acceptDocs == null || acceptDocs.get(doc)) { + break; + } + doc += 1; + } + if (doc >= maxDoc) { + doc = NO_MORE_DOCS; + } + return doc; + } + + @Override + public long cost() { + return maxDoc - minDoc; + } + + }; + return new ConstantScoreScorer(this, score(), disi); + } + }; } @Override public String toString(String field) { - return "ScanFilter"; + return "MinDocQuery(minDoc=" + minDoc + ")"; } - } - static class ReaderState { - public int count; - public boolean done; } } diff --git a/src/test/java/org/elasticsearch/search/scan/ScanContextTests.java b/src/test/java/org/elasticsearch/search/scan/ScanContextTests.java new file mode 100644 index 00000000000..4a43b867ba7 --- /dev/null +++ b/src/test/java/org/elasticsearch/search/scan/ScanContextTests.java @@ -0,0 +1,116 @@ +/* + * 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.scan; + +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field.Store; +import org.apache.lucene.document.StringField; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.RandomIndexWriter; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.QueryUtils; +import org.apache.lucene.search.ScoreDoc; +import org.apache.lucene.search.Sort; +import org.apache.lucene.search.TermQuery; +import org.apache.lucene.store.Directory; +import org.elasticsearch.search.scan.ScanContext.MinDocQuery; +import org.elasticsearch.test.ElasticsearchTestCase; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +public class ScanContextTests extends ElasticsearchTestCase { + + public void testMinDocQueryBasics() { + MinDocQuery query1 = new MinDocQuery(42); + MinDocQuery query2 = new MinDocQuery(42); + MinDocQuery query3 = new MinDocQuery(43); + QueryUtils.check(query1); + QueryUtils.checkEqual(query1, query2); + QueryUtils.checkUnequal(query1, query3); + } + + public void testMinDocQueryRandom() throws IOException { + final int numDocs = randomIntBetween(10, 200); + final Document doc = new Document(); + final Directory dir = newDirectory(); + final RandomIndexWriter w = new RandomIndexWriter(getRandom(), dir); + for (int i = 0; i < numDocs; ++i) { + w.addDocument(doc); + } + final IndexReader reader = w.getReader(); + final IndexSearcher searcher = newSearcher(reader); + for (int i = 0; i <= numDocs; ++i) { + assertEquals(numDocs - i, searcher.count(new MinDocQuery(i))); + } + w.close(); + reader.close(); + dir.close(); + } + + public void testRandom() throws Exception { + final int numDocs = randomIntBetween(10, 200); + final Document doc1 = new Document(); + doc1.add(new StringField("foo", "bar", Store.NO)); + final Document doc2 = new Document(); + final Directory dir = newDirectory(); + final RandomIndexWriter w = new RandomIndexWriter(getRandom(), dir); + for (int i = 0; i < numDocs; ++i) { + w.addDocument(randomBoolean() ? doc1 : doc2); + } + final IndexReader reader = w.getReader(); + final IndexSearcher searcher = newSearcher(reader); + + final boolean trackScores = randomBoolean(); + final int pageSize = randomIntBetween(1, numDocs / 2); + Query query = new TermQuery(new Term("foo", "bar")); + if (trackScores == false) { + query.setBoost(0f); + } + final ScoreDoc[] expected = searcher.search(query, numDocs, Sort.INDEXORDER, true, true).scoreDocs; + + final List actual = new ArrayList<>(); + ScanContext context = new ScanContext(); + while (true) { + final ScoreDoc[] page = context.execute(searcher, query, pageSize, trackScores).scoreDocs; + assertTrue(page.length <= pageSize); + if (page.length == 0) { + assertEquals(0, context.execute(searcher, query, pageSize, trackScores).scoreDocs.length); + break; + } + actual.addAll(Arrays.asList(page)); + } + assertEquals(expected.length, actual.size()); + for (int i = 0; i < expected.length; ++i) { + ScoreDoc sd1 = expected[i]; + ScoreDoc sd2 = actual.get(i); + assertEquals(sd1.doc, sd2.doc); + assertEquals(sd1.score, sd2.score, 0.001f); + } + w.close(); + reader.close(); + dir.close(); + } + +} From 3a69b65e88b7d0f6cc4bdb6ee675c18611b583a8 Mon Sep 17 00:00:00 2001 From: Clinton Gormley Date: Fri, 15 May 2015 18:40:16 +0200 Subject: [PATCH 17/56] Docs: Fixed the backslash escaping on the pattern analyzer docs Closes #11099 --- .../analyzers/pattern-analyzer.asciidoc | 143 +++++++++--------- 1 file changed, 71 insertions(+), 72 deletions(-) diff --git a/docs/reference/analysis/analyzers/pattern-analyzer.asciidoc b/docs/reference/analysis/analyzers/pattern-analyzer.asciidoc index 6f43469baa3..873e6e3102f 100644 --- a/docs/reference/analysis/analyzers/pattern-analyzer.asciidoc +++ b/docs/reference/analysis/analyzers/pattern-analyzer.asciidoc @@ -7,16 +7,13 @@ via a regular expression. Accepts the following settings: The following are settings that can be set for a `pattern` analyzer type: -[cols="<,<",options="header",] -|=================================================================== -|Setting |Description -|`lowercase` |Should terms be lowercased or not. Defaults to `true`. -|`pattern` |The regular expression pattern, defaults to `\W+`. -|`flags` |The regular expression flags. -|`stopwords` |A list of stopwords to initialize the stop filter with. -Defaults to an 'empty' stopword list Check -<> for more details. -|=================================================================== +[horizontal] +`lowercase`:: Should terms be lowercased or not. Defaults to `true`. +`pattern`:: The regular expression pattern, defaults to `\W+`. +`flags`:: The regular expression flags. +`stopwords`:: A list of stopwords to initialize the stop filter with. + Defaults to an 'empty' stopword list Check + <> for more details. *IMPORTANT*: The regular expression should match the *token separators*, not the tokens themselves. @@ -29,101 +26,103 @@ Pattern API] for more details about `flags` options. ==== Pattern Analyzer Examples In order to try out these examples, you should delete the `test` index -before running each example: - -[source,js] --------------------------------------------------- - curl -XDELETE localhost:9200/test --------------------------------------------------- +before running each example. [float] ===== Whitespace tokenizer [source,js] -------------------------------------------------- - curl -XPUT 'localhost:9200/test' -d ' - { - "settings":{ - "analysis": { - "analyzer": { - "whitespace":{ - "type": "pattern", - "pattern":"\\\\s+" - } - } - } - } - }' +DELETE test - curl 'localhost:9200/test/_analyze?pretty=1&analyzer=whitespace' -d 'foo,bar baz' - # "foo,bar", "baz" +PUT /test +{ + "settings": { + "analysis": { + "analyzer": { + "whitespace": { + "type": "pattern", + "pattern": "\\s+" + } + } + } + } +} + +GET /test/_analyze?analyzer=whitespace&text=foo,bar baz + +# "foo,bar", "baz" -------------------------------------------------- +// AUTOSENSE [float] ===== Non-word character tokenizer [source,js] -------------------------------------------------- +DELETE test - curl -XPUT 'localhost:9200/test' -d ' - { - "settings":{ - "analysis": { - "analyzer": { - "nonword":{ - "type": "pattern", - "pattern":"[^\\\\w]+" - } - } - } +PUT /test +{ + "settings": { + "analysis": { + "analyzer": { + "nonword": { + "type": "pattern", + "pattern": "[^\\w]+" <1> } - }' + } + } + } +} - curl 'localhost:9200/test/_analyze?pretty=1&analyzer=nonword' -d 'foo,bar baz' - # "foo,bar baz" becomes "foo", "bar", "baz" +GET /test/_analyze?analyzer=nonword&text=foo,bar baz +# "foo,bar baz" becomes "foo", "bar", "baz" - curl 'localhost:9200/test/_analyze?pretty=1&analyzer=nonword' -d 'type_1-type_4' - # "type_1","type_4" +GET /test/_analyze?analyzer=nonword&text=type_1-type_4 +# "type_1","type_4" -------------------------------------------------- +// AUTOSENSE + [float] ===== CamelCase tokenizer [source,js] -------------------------------------------------- +DELETE test - curl -XPUT 'localhost:9200/test?pretty=1' -d ' - { - "settings":{ - "analysis": { - "analyzer": { - "camel":{ - "type": "pattern", - "pattern":"([^\\\\p{L}\\\\d]+)|(?<=\\\\D)(?=\\\\d)|(?<=\\\\d)(?=\\\\D)|(?<=[\\\\p{L}&&[^\\\\p{Lu}]])(?=\\\\p{Lu})|(?<=\\\\p{Lu})(?=\\\\p{Lu}[\\\\p{L}&&[^\\\\p{Lu}]])" - } - } - } +PUT /test?pretty=1 +{ + "settings": { + "analysis": { + "analyzer": { + "camel": { + "type": "pattern", + "pattern": "([^\\p{L}\\d]+)|(?<=\\D)(?=\\d)|(?<=\\d)(?=\\D)|(?<=[\\p{L}&&[^\\p{Lu}]])(?=\\p{Lu})|(?<=\\p{Lu})(?=\\p{Lu}[\\p{L}&&[^\\p{Lu}]])" } - }' + } + } + } +} - curl 'localhost:9200/test/_analyze?pretty=1&analyzer=camel' -d ' - MooseX::FTPClass2_beta - ' - # "moose","x","ftp","class","2","beta" +GET /test/_analyze?analyzer=camel&text=MooseX::FTPClass2_beta +# "moose","x","ftp","class","2","beta" -------------------------------------------------- +// AUTOSENSE The regex above is easier to understand as: [source,js] -------------------------------------------------- - ([^\\p{L}\\d]+) # swallow non letters and numbers, - | (?<=\\D)(?=\\d) # or non-number followed by number, - | (?<=\\d)(?=\\D) # or number followed by non-number, - | (?<=[ \\p{L} && [^\\p{Lu}]]) # or lower case - (?=\\p{Lu}) # followed by upper case, - | (?<=\\p{Lu}) # or upper case - (?=\\p{Lu} # followed by upper case - [\\p{L}&&[^\\p{Lu}]] # then lower case - ) + ([^\p{L}\d]+) # swallow non letters and numbers, +| (?<=\D)(?=\d) # or non-number followed by number, +| (?<=\d)(?=\D) # or number followed by non-number, +| (?<=[ \p{L} && [^\p{Lu}]]) # or lower case + (?=\p{Lu}) # followed by upper case, +| (?<=\p{Lu}) # or upper case + (?=\p{Lu} # followed by upper case + [\p{L}&&[^\p{Lu}]] # then lower case + ) -------------------------------------------------- From caf723570d8e52c52b1f6bd378f197c0af338c13 Mon Sep 17 00:00:00 2001 From: markharwood Date: Fri, 15 May 2015 17:03:16 +0100 Subject: [PATCH 18/56] Aggregations improvement: exclude clauses with a medium/large number of clauses fail. The underlying automaton-backed implementation throws an error if there are too many states. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This fix changes to using an implementation based on Set lookups for lists of excluded terms. If the global-ordinals execution mode is in effect this implementation also addresses the slowness identified in issue 11181 which is caused by traversing the TermsEnum - instead the excluded terms’ global ordinals are looked up individually and unset the bits of acceptable terms. This is significantly faster. Closes #11176 --- .../bucket/terms/support/IncludeExclude.java | 89 +++++++++++++++++-- 1 file changed, 83 insertions(+), 6 deletions(-) diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/support/IncludeExclude.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/support/IncludeExclude.java index fb255e45373..1eff5885b1c 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/support/IncludeExclude.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/support/IncludeExclude.java @@ -37,6 +37,7 @@ import org.apache.lucene.util.automaton.RegExp; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.aggregations.support.ValuesSource.Bytes.WithOrdinals; import java.io.IOException; import java.util.HashSet; @@ -80,33 +81,65 @@ public class IncludeExclude { } // Only used for the 'map' execution mode (ie. scripts) - public static class StringFilter { + public abstract static class StringFilter { + public abstract boolean accept(BytesRef value); + } + + static class AutomatonBackedStringFilter extends StringFilter { private final ByteRunAutomaton runAutomaton; - private StringFilter(Automaton automaton) { + private AutomatonBackedStringFilter(Automaton automaton) { this.runAutomaton = new ByteRunAutomaton(automaton); } /** * Returns whether the given value is accepted based on the {@code include} & {@code exclude} patterns. */ + @Override public boolean accept(BytesRef value) { return runAutomaton.run(value.bytes, value.offset, value.length); } } - public static class OrdinalsFilter { + static class TermListBackedStringFilter extends StringFilter { + + private final Set valids; + private final Set invalids; + + public TermListBackedStringFilter(Set includeValues, Set excludeValues) { + this.valids = includeValues; + this.invalids = excludeValues; + } + + /** + * Returns whether the given value is accepted based on the + * {@code include} & {@code exclude} sets. + */ + @Override + public boolean accept(BytesRef value) { + return ((valids == null) || (valids.contains(value))) && ((invalids == null) || (!invalids.contains(value))); + } + } + + public static abstract class OrdinalsFilter { + public abstract LongBitSet acceptedGlobalOrdinals(RandomAccessOrds globalOrdinals, ValuesSource.Bytes.WithOrdinals valueSource) throws IOException; + + } + + static class AutomatonBackedOrdinalsFilter extends OrdinalsFilter { private final CompiledAutomaton compiled; - private OrdinalsFilter(Automaton automaton) { + private AutomatonBackedOrdinalsFilter(Automaton automaton) { this.compiled = new CompiledAutomaton(automaton); } /** * Computes which global ordinals are accepted by this IncludeExclude instance. + * */ + @Override public LongBitSet acceptedGlobalOrdinals(RandomAccessOrds globalOrdinals, ValuesSource.Bytes.WithOrdinals valueSource) throws IOException { LongBitSet acceptedGlobalOrdinals = new LongBitSet(globalOrdinals.getValueCount()); TermsEnum globalTermsEnum; @@ -120,6 +153,43 @@ public class IncludeExclude { } } + + static class TermListBackedOrdinalsFilter extends OrdinalsFilter { + + private final SortedSet includeValues; + private final SortedSet excludeValues; + + public TermListBackedOrdinalsFilter(SortedSet includeValues, SortedSet excludeValues) { + this.includeValues = includeValues; + this.excludeValues = excludeValues; + } + + @Override + public LongBitSet acceptedGlobalOrdinals(RandomAccessOrds globalOrdinals, WithOrdinals valueSource) throws IOException { + LongBitSet acceptedGlobalOrdinals = new LongBitSet(globalOrdinals.getValueCount()); + if(includeValues!=null){ + for (BytesRef term : includeValues) { + long ord = globalOrdinals.lookupTerm(term); + if (ord >= 0) { + acceptedGlobalOrdinals.set(ord); + } + } + } else { + // default to all terms being acceptable + acceptedGlobalOrdinals.set(0, acceptedGlobalOrdinals.length()); + } + if (excludeValues != null) { + for (BytesRef term : excludeValues) { + long ord = globalOrdinals.lookupTerm(term); + if (ord >= 0) { + acceptedGlobalOrdinals.clear(ord); + } + } + } + return acceptedGlobalOrdinals; + } + + } private final RegExp include, exclude; private final SortedSet includeValues, excludeValues; @@ -325,11 +395,18 @@ public class IncludeExclude { } public StringFilter convertToStringFilter() { - return new StringFilter(toAutomaton()); + if (isRegexBased()) { + return new AutomatonBackedStringFilter(toAutomaton()); + } + return new TermListBackedStringFilter(includeValues, excludeValues); } public OrdinalsFilter convertToOrdinalsFilter() { - return new OrdinalsFilter(toAutomaton()); + + if (isRegexBased()) { + return new AutomatonBackedOrdinalsFilter(toAutomaton()); + } + return new TermListBackedOrdinalsFilter(includeValues, excludeValues); } public LongFilter convertToLongFilter() { From 46c521f7eca7f8fa421abb233528b4cc04882580 Mon Sep 17 00:00:00 2001 From: javanna Date: Sat, 9 May 2015 14:17:41 +0200 Subject: [PATCH 19/56] Highlighting: nuke XPostingsHighlighter Our own fork of the lucene PostingsHighlighter is not easy to maintain and doesn't give us any added value at this point. In particular, it was introduced to support the require_field_match option and discrete per value highlighting, used in case one wants to highlight the whole content of a field, but get back one snippet per value. These two features won't make it into lucene as they slow things down and shouldn't have been supported from day one on our end probably. One other customization we had was support for a wider range of queries via custom rewrite etc. (yet another way to slow things down), which got added to lucene and works much much better than what we used to do (instead of or rewrite, term s are pulled out of the automata for multi term queries). Removing our fork means the following in terms of features: - dropped support for require_field_match: the postings highlighter will only highlight fields that were queried - some custom es queries won't be supported anymore, meaning they won't be highlighted. The only one I found up until now is the phrase_prefix. Postings highlighter rewrites against an empty reader to avoid slow operations (like the ones that we were performing with the fork that we are removing here), thus the prefix will not be expanded to any term. What the postings highlighter does instead is pulling the automata out of multi term queries, but this is not supported at the moment with our MultiPhrasePrefixQuery. Closes #10625 Closes #11077 --- docs/reference/migration/migrate_2_0.asciidoc | 9 + .../search/request/highlighting.asciidoc | 8 +- .../CustomPassageFormatter.java | 2 + .../CustomPostingsHighlighter.java | 176 +- .../CustomSeparatorBreakIterator.java | 153 ++ .../search/postingshighlight/Snippet.java | 2 +- .../XPostingsHighlighter.java | 772 -------- .../search/highlight/HighlightPhase.java | 1 - .../search/highlight/HighlightUtils.java | 4 +- .../search/highlight/PostingsHighlighter.java | 187 +- .../CustomPostingsHighlighterTests.java | 353 +--- .../CustomSeparatorBreakIteratorTests.java | 186 ++ .../XPostingsHighlighterTests.java | 1691 ----------------- .../highlight/HighlighterSearchTests.java | 224 +-- 14 files changed, 517 insertions(+), 3251 deletions(-) create mode 100644 src/main/java/org/apache/lucene/search/postingshighlight/CustomSeparatorBreakIterator.java delete mode 100644 src/main/java/org/apache/lucene/search/postingshighlight/XPostingsHighlighter.java create mode 100644 src/test/java/org/apache/lucene/search/postingshighlight/CustomSeparatorBreakIteratorTests.java delete mode 100644 src/test/java/org/apache/lucene/search/postingshighlight/XPostingsHighlighterTests.java diff --git a/docs/reference/migration/migrate_2_0.asciidoc b/docs/reference/migration/migrate_2_0.asciidoc index 382dd02cace..c91e6e0a74f 100644 --- a/docs/reference/migration/migrate_2_0.asciidoc +++ b/docs/reference/migration/migrate_2_0.asciidoc @@ -587,3 +587,12 @@ Before version `1.1.0` the parent/child had its own in-memory data structures fo Removed `id_cache` option from the clear cache apis. The `fielddata` option should be used to clear `_parent` field from fielddata. + +[float] +=== Highlighting + +The postings highlighter doesn't support the `require_field_match` option +anymore, it will only highlight fields that were queried. + +The `match` query with type set to `match_phrase_prefix` is not supported by the +postings highlighter. No highlighted snippets will be returned. diff --git a/docs/reference/search/request/highlighting.asciidoc b/docs/reference/search/request/highlighting.asciidoc index a8b9f1c7745..7c43fdc37db 100644 --- a/docs/reference/search/request/highlighting.asciidoc +++ b/docs/reference/search/request/highlighting.asciidoc @@ -72,11 +72,9 @@ that the query is composed of, regardless of whether they are actually part of a query match, effectively ignoring their positions. [WARNING] -The postings highlighter does support highlighting of multi term queries, like -prefix queries, wildcard queries and so on. On the other hand, this requires -the queries to be rewritten using a proper -<> that supports multi term -extraction, which is a potentially expensive operation. +The postings highlighter doesn't support highlighting some complex queries, +like a `match` query with `type` set to `match_phrase_prefix`. No highlighted +snippets will be returned in that case. [[fast-vector-highlighter]] ==== Fast vector highlighter diff --git a/src/main/java/org/apache/lucene/search/postingshighlight/CustomPassageFormatter.java b/src/main/java/org/apache/lucene/search/postingshighlight/CustomPassageFormatter.java index 59ef78423c8..75ad81b07d8 100644 --- a/src/main/java/org/apache/lucene/search/postingshighlight/CustomPassageFormatter.java +++ b/src/main/java/org/apache/lucene/search/postingshighlight/CustomPassageFormatter.java @@ -65,6 +65,8 @@ public class CustomPassageFormatter extends PassageFormatter { //we remove the paragraph separator if present at the end of the snippet (we used it as separator between values) if (sb.charAt(sb.length() - 1) == HighlightUtils.PARAGRAPH_SEPARATOR) { sb.deleteCharAt(sb.length() - 1); + } else if (sb.charAt(sb.length() - 1) == HighlightUtils.NULL_SEPARATOR) { + sb.deleteCharAt(sb.length() - 1); } //and we trim the snippets too snippets[j] = new Snippet(sb.toString().trim(), passage.score, passage.numMatches > 0); diff --git a/src/main/java/org/apache/lucene/search/postingshighlight/CustomPostingsHighlighter.java b/src/main/java/org/apache/lucene/search/postingshighlight/CustomPostingsHighlighter.java index 936fe490a5d..67373ef116d 100644 --- a/src/main/java/org/apache/lucene/search/postingshighlight/CustomPostingsHighlighter.java +++ b/src/main/java/org/apache/lucene/search/postingshighlight/CustomPostingsHighlighter.java @@ -18,121 +18,85 @@ package org.apache.lucene.search.postingshighlight; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.IndexReaderContext; -import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.util.BytesRef; -import org.elasticsearch.common.Strings; -import org.elasticsearch.search.highlight.HighlightUtils; +import org.apache.lucene.search.Query; import java.io.IOException; import java.text.BreakIterator; -import java.util.List; import java.util.Map; /** - * Subclass of the {@link XPostingsHighlighter} that works for a single field in a single document. - * It receives the field values as input and it performs discrete highlighting on each single value - * calling the highlightDoc method multiple times. - * It allows to pass in the query terms to avoid calling extract terms multiple times. - * - * The use that we make of the postings highlighter is not optimal. It would be much better to - * highlight multiple docs in a single call, as we actually lose its sequential IO. But that would require: - * 1) to make our fork more complex and harder to maintain to perform discrete highlighting (needed to return - * a different snippet per value when number_of_fragments=0 and the field has multiple values) - * 2) refactoring of the elasticsearch highlight api which currently works per hit + * Subclass of the {@link PostingsHighlighter} that works for a single field in a single document. + * Uses a custom {@link PassageFormatter}. Accepts field content as a constructor argument, given that loading + * is custom and can be done reading from _source field. Supports using different {@link BreakIterator} to break + * the text into fragments. Considers every distinct field value as a discrete passage for highlighting (unless + * the whole content needs to be highlighted). Supports both returning empty snippets and non highlighted snippets + * when no highlighting can be performed. * + * The use that we make of the postings highlighter is not optimal. It would be much better to highlight + * multiple docs in a single call, as we actually lose its sequential IO. That would require to + * refactor the elasticsearch highlight api which currently works per hit. */ -public final class CustomPostingsHighlighter extends XPostingsHighlighter { +public final class CustomPostingsHighlighter extends PostingsHighlighter { private static final Snippet[] EMPTY_SNIPPET = new Snippet[0]; private static final Passage[] EMPTY_PASSAGE = new Passage[0]; + private final Analyzer analyzer; private final CustomPassageFormatter passageFormatter; - private final int noMatchSize; - private final int totalContentLength; - private final String[] fieldValues; - private final int[] fieldValuesOffsets; - private int currentValueIndex = 0; + private final BreakIterator breakIterator; + private final boolean returnNonHighlightedSnippets; + private final String fieldValue; - private BreakIterator breakIterator; - - public CustomPostingsHighlighter(CustomPassageFormatter passageFormatter, List fieldValues, boolean mergeValues, int maxLength, int noMatchSize) { - super(maxLength); - this.passageFormatter = passageFormatter; - this.noMatchSize = noMatchSize; - - if (mergeValues) { - String rawValue = Strings.collectionToDelimitedString(fieldValues, String.valueOf(getMultiValuedSeparator(""))); - String fieldValue = rawValue.substring(0, Math.min(rawValue.length(), maxLength)); - this.fieldValues = new String[]{fieldValue}; - this.fieldValuesOffsets = new int[]{0}; - this.totalContentLength = fieldValue.length(); - } else { - this.fieldValues = new String[fieldValues.size()]; - this.fieldValuesOffsets = new int[fieldValues.size()]; - int contentLength = 0; - int offset = 0; - int previousLength = -1; - for (int i = 0; i < fieldValues.size(); i++) { - String rawValue = fieldValues.get(i).toString(); - String fieldValue = rawValue.substring(0, Math.min(rawValue.length(), maxLength)); - this.fieldValues[i] = fieldValue; - contentLength += fieldValue.length(); - offset += previousLength + 1; - this.fieldValuesOffsets[i] = offset; - previousLength = fieldValue.length(); - } - this.totalContentLength = contentLength; - } + /** + * Creates a new instance of {@link CustomPostingsHighlighter} + * + * @param analyzer the analyzer used for the field at index time, used for multi term queries internally + * @param passageFormatter our own {@link PassageFormatter} which generates snippets in forms of {@link Snippet} objects + * @param fieldValue the original field values as constructor argument, loaded from te _source field or the relevant stored field. + * @param returnNonHighlightedSnippets whether non highlighted snippets should be returned rather than empty snippets when + * no highlighting can be performed + */ + public CustomPostingsHighlighter(Analyzer analyzer, CustomPassageFormatter passageFormatter, String fieldValue, boolean returnNonHighlightedSnippets) { + this(analyzer, passageFormatter, null, fieldValue, returnNonHighlightedSnippets); } - /* - Our own api to highlight a single document field, passing in the query terms, and get back our own Snippet object + /** + * Creates a new instance of {@link CustomPostingsHighlighter} + * + * @param analyzer the analyzer used for the field at index time, used for multi term queries internally + * @param passageFormatter our own {@link PassageFormatter} which generates snippets in forms of {@link Snippet} objects + * @param breakIterator an instance {@link BreakIterator} selected depending on the highlighting options + * @param fieldValue the original field values as constructor argument, loaded from te _source field or the relevant stored field. + * @param returnNonHighlightedSnippets whether non highlighted snippets should be returned rather than empty snippets when + * no highlighting can be performed */ - public Snippet[] highlightDoc(String field, BytesRef[] terms, IndexReader reader, int docId, int maxPassages) throws IOException { - IndexReaderContext readerContext = reader.getContext(); - List leaves = readerContext.leaves(); + public CustomPostingsHighlighter(Analyzer analyzer, CustomPassageFormatter passageFormatter, BreakIterator breakIterator, String fieldValue, boolean returnNonHighlightedSnippets) { + this.analyzer = analyzer; + this.passageFormatter = passageFormatter; + this.breakIterator = breakIterator; + this.returnNonHighlightedSnippets = returnNonHighlightedSnippets; + this.fieldValue = fieldValue; + } - String[] contents = new String[]{loadCurrentFieldValue()}; - Map snippetsMap = highlightField(field, contents, getBreakIterator(field), terms, new int[]{docId}, leaves, maxPassages); - - //increment the current value index so that next time we'll highlight the next value if available - currentValueIndex++; - - Object snippetObject = snippetsMap.get(docId); - if (snippetObject != null && snippetObject instanceof Snippet[]) { - return (Snippet[]) snippetObject; + /** + * Highlights terms extracted from the provided query within the content of the provided field name + */ + public Snippet[] highlightField(String field, Query query, IndexSearcher searcher, int docId, int maxPassages) throws IOException { + Map fieldsAsObjects = super.highlightFieldsAsObjects(new String[]{field}, query, searcher, new int[]{docId}, new int[]{maxPassages}); + Object[] snippetObjects = fieldsAsObjects.get(field); + if (snippetObjects != null) { + //one single document at a time + assert snippetObjects.length == 1; + Object snippetObject = snippetObjects[0]; + if (snippetObject != null && snippetObject instanceof Snippet[]) { + return (Snippet[]) snippetObject; + } } return EMPTY_SNIPPET; } - /* - Method provided through our own fork: allows to do proper scoring when doing per value discrete highlighting. - Used to provide the total length of the field (all values) for proper scoring. - */ - @Override - protected int getContentLength(String field, int docId) { - return totalContentLength; - } - - /* - Method provided through our own fork: allows to perform proper per value discrete highlighting. - Used to provide the offset for the current value. - */ - @Override - protected int getOffsetForCurrentValue(String field, int docId) { - if (currentValueIndex < fieldValuesOffsets.length) { - return fieldValuesOffsets[currentValueIndex]; - } - throw new IllegalArgumentException("No more values offsets to return"); - } - - public void setBreakIterator(BreakIterator breakIterator) { - this.breakIterator = breakIterator; - } - @Override protected PassageFormatter getFormatter(String field) { return passageFormatter; @@ -146,41 +110,27 @@ public final class CustomPostingsHighlighter extends XPostingsHighlighter { return breakIterator; } - @Override - protected char getMultiValuedSeparator(String field) { - //U+2029 PARAGRAPH SEPARATOR (PS): each value holds a discrete passage for highlighting - return HighlightUtils.PARAGRAPH_SEPARATOR; - } - /* By default the postings highlighter returns non highlighted snippet when there are no matches. We want to return no snippets by default, unless no_match_size is greater than 0 */ @Override protected Passage[] getEmptyHighlight(String fieldName, BreakIterator bi, int maxPassages) { - if (noMatchSize > 0) { + if (returnNonHighlightedSnippets) { //we want to return the first sentence of the first snippet only return super.getEmptyHighlight(fieldName, bi, 1); } return EMPTY_PASSAGE; } - /* - Not needed since we call our own loadCurrentFieldValue explicitly, but we override it anyway for consistency. - */ @Override - protected String[][] loadFieldValues(IndexSearcher searcher, String[] fields, int[] docids, int maxLength) throws IOException { - return new String[][]{new String[]{loadCurrentFieldValue()}}; + protected Analyzer getIndexAnalyzer(String field) { + return analyzer; } - /* - Our own method that returns the field values, which relies on the content that was provided when creating the highlighter. - Supports per value discrete highlighting calling the highlightDoc method multiple times, one per value. - */ - protected String loadCurrentFieldValue() { - if (currentValueIndex < fieldValues.length) { - return fieldValues[currentValueIndex]; - } - throw new IllegalArgumentException("No more values to return"); + @Override + protected String[][] loadFieldValues(IndexSearcher searcher, String[] fields, int[] docids, int maxLength) throws IOException { + //we only highlight one field, one document at a time + return new String[][]{new String[]{fieldValue}}; } } diff --git a/src/main/java/org/apache/lucene/search/postingshighlight/CustomSeparatorBreakIterator.java b/src/main/java/org/apache/lucene/search/postingshighlight/CustomSeparatorBreakIterator.java new file mode 100644 index 00000000000..efdddf5260e --- /dev/null +++ b/src/main/java/org/apache/lucene/search/postingshighlight/CustomSeparatorBreakIterator.java @@ -0,0 +1,153 @@ +/* +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.apache.lucene.search.postingshighlight; + +import java.text.BreakIterator; +import java.text.CharacterIterator; + +/** + * A {@link BreakIterator} that breaks the text whenever a certain separator, provided as a constructor argument, is found. + */ +public class CustomSeparatorBreakIterator extends BreakIterator { + + private final char separator; + private CharacterIterator text; + private int current; + + public CustomSeparatorBreakIterator(char separator) { + this.separator = separator; + } + + @Override + public int current() { + return current; + } + + @Override + public int first() { + text.setIndex(text.getBeginIndex()); + return current = text.getIndex(); + } + + @Override + public int last() { + text.setIndex(text.getEndIndex()); + return current = text.getIndex(); + } + + @Override + public int next() { + if (text.getIndex() == text.getEndIndex()) { + return DONE; + } else { + return advanceForward(); + } + } + + private int advanceForward() { + char c; + while( (c = text.next()) != CharacterIterator.DONE) { + if (c == separator) { + return current = text.getIndex() + 1; + } + } + assert text.getIndex() == text.getEndIndex(); + return current = text.getIndex(); + } + + @Override + public int following(int pos) { + if (pos < text.getBeginIndex() || pos > text.getEndIndex()) { + throw new IllegalArgumentException("offset out of bounds"); + } else if (pos == text.getEndIndex()) { + // this conflicts with the javadocs, but matches actual behavior (Oracle has a bug in something) + // http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=9000909 + text.setIndex(text.getEndIndex()); + current = text.getIndex(); + return DONE; + } else { + text.setIndex(pos); + current = text.getIndex(); + return advanceForward(); + } + } + + @Override + public int previous() { + if (text.getIndex() == text.getBeginIndex()) { + return DONE; + } else { + return advanceBackward(); + } + } + + private int advanceBackward() { + char c; + while( (c = text.previous()) != CharacterIterator.DONE) { + if (c == separator) { + return current = text.getIndex() + 1; + } + } + assert text.getIndex() == text.getBeginIndex(); + return current = text.getIndex(); + } + + @Override + public int preceding(int pos) { + if (pos < text.getBeginIndex() || pos > text.getEndIndex()) { + throw new IllegalArgumentException("offset out of bounds"); + } else if (pos == text.getBeginIndex()) { + // this conflicts with the javadocs, but matches actual behavior (Oracle has a bug in something) + // http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=9000909 + text.setIndex(text.getBeginIndex()); + current = text.getIndex(); + return DONE; + } else { + text.setIndex(pos); + current = text.getIndex(); + return advanceBackward(); + } + } + + @Override + public int next(int n) { + if (n < 0) { + for (int i = 0; i < -n; i++) { + previous(); + } + } else { + for (int i = 0; i < n; i++) { + next(); + } + } + return current(); + } + + @Override + public CharacterIterator getText() { + return text; + } + + @Override + public void setText(CharacterIterator newText) { + text = newText; + current = text.getBeginIndex(); + } +} diff --git a/src/main/java/org/apache/lucene/search/postingshighlight/Snippet.java b/src/main/java/org/apache/lucene/search/postingshighlight/Snippet.java index bf6802000fa..a756de65115 100644 --- a/src/main/java/org/apache/lucene/search/postingshighlight/Snippet.java +++ b/src/main/java/org/apache/lucene/search/postingshighlight/Snippet.java @@ -22,7 +22,7 @@ package org.apache.lucene.search.postingshighlight; * Represents a scored highlighted snippet. * It's our own arbitrary object that we get back from the postings highlighter when highlighting a document. * Every snippet contains its formatted text and its score. - * The score is needed since we highlight every single value separately and we might want to return snippets sorted by score. + * The score is needed in case we want to sort snippets by score, they get sorted by position in the text by default. */ public class Snippet { diff --git a/src/main/java/org/apache/lucene/search/postingshighlight/XPostingsHighlighter.java b/src/main/java/org/apache/lucene/search/postingshighlight/XPostingsHighlighter.java deleted file mode 100644 index 6b725c48f4f..00000000000 --- a/src/main/java/org/apache/lucene/search/postingshighlight/XPostingsHighlighter.java +++ /dev/null @@ -1,772 +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.apache.lucene.search.postingshighlight; - -import org.apache.lucene.index.*; -import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.Query; -import org.apache.lucene.search.ScoreDoc; -import org.apache.lucene.search.TopDocs; -import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.InPlaceMergeSorter; -import org.apache.lucene.util.UnicodeUtil; - -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.text.BreakIterator; -import java.util.*; - -/* -FORKED from Lucene 4.5 to be able to: -1) support discrete highlighting for multiple values, so that we can return a different snippet per value when highlighting the whole text -2) call the highlightField method directly from subclasses and provide the terms by ourselves -3) Applied LUCENE-4906 to allow PassageFormatter to return arbitrary objects (LUCENE 4.6) - -All our changes start with //BEGIN EDIT - */ -public class XPostingsHighlighter { - - //BEGIN EDIT added method to override offset for current value (default 0) - //we need this to perform discrete highlighting per field - protected int getOffsetForCurrentValue(String field, int docId) { - return 0; - } - //END EDIT - - //BEGIN EDIT - //we need this to fix scoring when highlighting every single value separately, since the score depends on the total length of the field (all values rather than only the current one) - protected int getContentLength(String field, int docId) { - return -1; - } - //END EDIT - - - // TODO: maybe allow re-analysis for tiny fields? currently we require offsets, - // but if the analyzer is really fast and the field is tiny, this might really be - // unnecessary. - - /** for rewriting: we don't want slow processing from MTQs */ - private static final IndexSearcher EMPTY_INDEXSEARCHER; - static { - try { - IndexReader emptyReader = new MultiReader(); - EMPTY_INDEXSEARCHER = new IndexSearcher(emptyReader); - EMPTY_INDEXSEARCHER.setQueryCache(null); - } catch (IOException bogus) { - throw new RuntimeException(bogus); - } - } - - /** Default maximum content size to process. Typically snippets - * closer to the beginning of the document better summarize its content */ - public static final int DEFAULT_MAX_LENGTH = 10000; - - private final int maxLength; - - /** Set the first time {@link #getFormatter} is called, - * and then reused. */ - private PassageFormatter defaultFormatter; - - /** Set the first time {@link #getScorer} is called, - * and then reused. */ - private PassageScorer defaultScorer; - - /** - * Creates a new highlighter with default parameters. - */ - public XPostingsHighlighter() { - this(DEFAULT_MAX_LENGTH); - } - - /** - * Creates a new highlighter, specifying maximum content length. - * @param maxLength maximum content size to process. - * @throws IllegalArgumentException if maxLength is negative or Integer.MAX_VALUE - */ - public XPostingsHighlighter(int maxLength) { - if (maxLength < 0 || maxLength == Integer.MAX_VALUE) { - // two reasons: no overflow problems in BreakIterator.preceding(offset+1), - // our sentinel in the offsets queue uses this value to terminate. - throw new IllegalArgumentException("maxLength must be < Integer.MAX_VALUE"); - } - this.maxLength = maxLength; - } - - /** Returns the {@link java.text.BreakIterator} to use for - * dividing text into passages. This returns - * {@link java.text.BreakIterator#getSentenceInstance(java.util.Locale)} by default; - * subclasses can override to customize. */ - protected BreakIterator getBreakIterator(String field) { - return BreakIterator.getSentenceInstance(Locale.ROOT); - } - - /** Returns the {@link PassageFormatter} to use for - * formatting passages into highlighted snippets. This - * returns a new {@code PassageFormatter} by default; - * subclasses can override to customize. */ - protected PassageFormatter getFormatter(String field) { - if (defaultFormatter == null) { - defaultFormatter = new DefaultPassageFormatter(); - } - return defaultFormatter; - } - - /** Returns the {@link PassageScorer} to use for - * ranking passages. This - * returns a new {@code PassageScorer} by default; - * subclasses can override to customize. */ - protected PassageScorer getScorer(String field) { - if (defaultScorer == null) { - defaultScorer = new PassageScorer(); - } - return defaultScorer; - } - - /** - * Highlights the top passages from a single field. - * - * @param field field name to highlight. - * Must have a stored string value and also be indexed with offsets. - * @param query query to highlight. - * @param searcher searcher that was previously used to execute the query. - * @param topDocs TopDocs containing the summary result documents to highlight. - * @return Array of formatted snippets corresponding to the documents in topDocs. - * If no highlights were found for a document, the - * first sentence for the field will be returned. - * @throws java.io.IOException if an I/O error occurred during processing - * @throws IllegalArgumentException if field was indexed without - * {@link org.apache.lucene.index.FieldInfo.IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS} - */ - public String[] highlight(String field, Query query, IndexSearcher searcher, TopDocs topDocs) throws IOException { - return highlight(field, query, searcher, topDocs, 1); - } - - /** - * Highlights the top-N passages from a single field. - * - * @param field field name to highlight. - * Must have a stored string value and also be indexed with offsets. - * @param query query to highlight. - * @param searcher searcher that was previously used to execute the query. - * @param topDocs TopDocs containing the summary result documents to highlight. - * @param maxPassages The maximum number of top-N ranked passages used to - * form the highlighted snippets. - * @return Array of formatted snippets corresponding to the documents in topDocs. - * If no highlights were found for a document, the - * first {@code maxPassages} sentences from the - * field will be returned. - * @throws IOException if an I/O error occurred during processing - * @throws IllegalArgumentException if field was indexed without - * {@link org.apache.lucene.index.FieldInfo.IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS} - */ - public String[] highlight(String field, Query query, IndexSearcher searcher, TopDocs topDocs, int maxPassages) throws IOException { - Map res = highlightFields(new String[] { field }, query, searcher, topDocs, new int[] { maxPassages }); - return res.get(field); - } - - /** - * Highlights the top passages from multiple fields. - *

- * Conceptually, this behaves as a more efficient form of: - *

-     * Map m = new HashMap();
-     * for (String field : fields) {
-     *   m.put(field, highlight(field, query, searcher, topDocs));
-     * }
-     * return m;
-     * 
- * - * @param fields field names to highlight. - * Must have a stored string value and also be indexed with offsets. - * @param query query to highlight. - * @param searcher searcher that was previously used to execute the query. - * @param topDocs TopDocs containing the summary result documents to highlight. - * @return Map keyed on field name, containing the array of formatted snippets - * corresponding to the documents in topDocs. - * If no highlights were found for a document, the - * first sentence from the field will be returned. - * @throws IOException if an I/O error occurred during processing - * @throws IllegalArgumentException if field was indexed without - * {@link org.apache.lucene.index.FieldInfo.IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS} - */ - public Map highlightFields(String fields[], Query query, IndexSearcher searcher, TopDocs topDocs) throws IOException { - int maxPassages[] = new int[fields.length]; - Arrays.fill(maxPassages, 1); - return highlightFields(fields, query, searcher, topDocs, maxPassages); - } - - /** - * Highlights the top-N passages from multiple fields. - *

- * Conceptually, this behaves as a more efficient form of: - *

-     * Map m = new HashMap();
-     * for (String field : fields) {
-     *   m.put(field, highlight(field, query, searcher, topDocs, maxPassages));
-     * }
-     * return m;
-     * 
- * - * @param fields field names to highlight. - * Must have a stored string value and also be indexed with offsets. - * @param query query to highlight. - * @param searcher searcher that was previously used to execute the query. - * @param topDocs TopDocs containing the summary result documents to highlight. - * @param maxPassages The maximum number of top-N ranked passages per-field used to - * form the highlighted snippets. - * @return Map keyed on field name, containing the array of formatted snippets - * corresponding to the documents in topDocs. - * If no highlights were found for a document, the - * first {@code maxPassages} sentences from the - * field will be returned. - * @throws IOException if an I/O error occurred during processing - * @throws IllegalArgumentException if field was indexed without - * {@link org.apache.lucene.index.FieldInfo.IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS} - */ - public Map highlightFields(String fields[], Query query, IndexSearcher searcher, TopDocs topDocs, int maxPassages[]) throws IOException { - final ScoreDoc scoreDocs[] = topDocs.scoreDocs; - int docids[] = new int[scoreDocs.length]; - for (int i = 0; i < docids.length; i++) { - docids[i] = scoreDocs[i].doc; - } - - return highlightFields(fields, query, searcher, docids, maxPassages); - } - - /** - * Highlights the top-N passages from multiple fields, - * for the provided int[] docids. - * - * @param fieldsIn field names to highlight. - * Must have a stored string value and also be indexed with offsets. - * @param query query to highlight. - * @param searcher searcher that was previously used to execute the query. - * @param docidsIn containing the document IDs to highlight. - * @param maxPassagesIn The maximum number of top-N ranked passages per-field used to - * form the highlighted snippets. - * @return Map keyed on field name, containing the array of formatted snippets - * corresponding to the documents in topDocs. - * If no highlights were found for a document, the - * first {@code maxPassages} from the field will - * be returned. - * @throws IOException if an I/O error occurred during processing - * @throws IllegalArgumentException if field was indexed without - * {@link org.apache.lucene.index.FieldInfo.IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS} - */ - public Map highlightFields(String fieldsIn[], Query query, IndexSearcher searcher, int[] docidsIn, int maxPassagesIn[]) throws IOException { - Map snippets = new HashMap<>(); - for(Map.Entry ent : highlightFieldsAsObjects(fieldsIn, query, searcher, docidsIn, maxPassagesIn).entrySet()) { - Object[] snippetObjects = ent.getValue(); - String[] snippetStrings = new String[snippetObjects.length]; - snippets.put(ent.getKey(), snippetStrings); - for(int i=0;i highlightFieldsAsObjects(String fieldsIn[], Query query, IndexSearcher searcher, int[] docidsIn, int maxPassagesIn[]) throws IOException { - if (fieldsIn.length < 1) { - throw new IllegalArgumentException("fieldsIn must not be empty"); - } - if (fieldsIn.length != maxPassagesIn.length) { - throw new IllegalArgumentException("invalid number of maxPassagesIn"); - } - SortedSet queryTerms = new TreeSet<>(); - EMPTY_INDEXSEARCHER.createNormalizedWeight(query, false).extractTerms(queryTerms); - - IndexReaderContext readerContext = searcher.getIndexReader().getContext(); - List leaves = readerContext.leaves(); - - // Make our own copies because we sort in-place: - int[] docids = new int[docidsIn.length]; - System.arraycopy(docidsIn, 0, docids, 0, docidsIn.length); - final String fields[] = new String[fieldsIn.length]; - System.arraycopy(fieldsIn, 0, fields, 0, fieldsIn.length); - final int maxPassages[] = new int[maxPassagesIn.length]; - System.arraycopy(maxPassagesIn, 0, maxPassages, 0, maxPassagesIn.length); - - // sort for sequential io - Arrays.sort(docids); - new InPlaceMergeSorter() { - - @Override - protected void swap(int i, int j) { - String tmp = fields[i]; - fields[i] = fields[j]; - fields[j] = tmp; - int tmp2 = maxPassages[i]; - maxPassages[i] = maxPassages[j]; - maxPassages[j] = tmp2; - } - - @Override - protected int compare(int i, int j) { - return fields[i].compareTo(fields[j]); - } - - }.sort(0, fields.length); - - // pull stored data: - String[][] contents = loadFieldValues(searcher, fields, docids, maxLength); - - Map highlights = new HashMap<>(); - for (int i = 0; i < fields.length; i++) { - String field = fields[i]; - int numPassages = maxPassages[i]; - - Term floor = new Term(field, ""); - Term ceiling = new Term(field, UnicodeUtil.BIG_TERM); - SortedSet fieldTerms = queryTerms.subSet(floor, ceiling); - // TODO: should we have some reasonable defaults for term pruning? (e.g. stopwords) - - // Strip off the redundant field: - BytesRef terms[] = new BytesRef[fieldTerms.size()]; - int termUpto = 0; - for(Term term : fieldTerms) { - terms[termUpto++] = term.bytes(); - } - Map fieldHighlights = highlightField(field, contents[i], getBreakIterator(field), terms, docids, leaves, numPassages); - - Object[] result = new Object[docids.length]; - for (int j = 0; j < docidsIn.length; j++) { - result[j] = fieldHighlights.get(docidsIn[j]); - } - highlights.put(field, result); - } - return highlights; - } - - /** Loads the String values for each field X docID to be - * highlighted. By default this loads from stored - * fields, but a subclass can change the source. This - * method should allocate the String[fields.length][docids.length] - * and fill all values. The returned Strings must be - * identical to what was indexed. */ - protected String[][] loadFieldValues(IndexSearcher searcher, String[] fields, int[] docids, int maxLength) throws IOException { - String contents[][] = new String[fields.length][docids.length]; - char valueSeparators[] = new char[fields.length]; - for (int i = 0; i < fields.length; i++) { - valueSeparators[i] = getMultiValuedSeparator(fields[i]); - } - LimitedStoredFieldVisitor visitor = new LimitedStoredFieldVisitor(fields, valueSeparators, maxLength); - for (int i = 0; i < docids.length; i++) { - searcher.doc(docids[i], visitor); - for (int j = 0; j < fields.length; j++) { - contents[j][i] = visitor.getValue(j); - } - visitor.reset(); - } - return contents; - } - - /** - * Returns the logical separator between values for multi-valued fields. - * The default value is a space character, which means passages can span across values, - * but a subclass can override, for example with {@code U+2029 PARAGRAPH SEPARATOR (PS)} - * if each value holds a discrete passage for highlighting. - */ - protected char getMultiValuedSeparator(String field) { - return ' '; - } - - //BEGIN EDIT: made protected so that we can call from our subclass and pass in the terms by ourselves - protected Map highlightField(String field, String contents[], BreakIterator bi, BytesRef terms[], int[] docids, List leaves, int maxPassages) throws IOException { - //private Map highlightField(String field, String contents[], BreakIterator bi, BytesRef terms[], int[] docids, List leaves, int maxPassages) throws IOException { - //END EDIT - - Map highlights = new HashMap<>(); - - // reuse in the real sense... for docs in same segment we just advance our old enum - PostingsEnum postings[] = null; - TermsEnum termsEnum = null; - int lastLeaf = -1; - - PassageFormatter fieldFormatter = getFormatter(field); - if (fieldFormatter == null) { - throw new NullPointerException("PassageFormatter cannot be null"); - } - - for (int i = 0; i < docids.length; i++) { - String content = contents[i]; - if (content.length() == 0) { - continue; // nothing to do - } - bi.setText(content); - int doc = docids[i]; - int leaf = ReaderUtil.subIndex(doc, leaves); - LeafReaderContext subContext = leaves.get(leaf); - LeafReader r = subContext.reader(); - Terms t = r.terms(field); - if (t == null) { - continue; // nothing to do - } - if (!t.hasOffsets()) { - // no offsets available - throw new IllegalArgumentException("field '" + field + "' was indexed without offsets, cannot highlight"); - } - if (leaf != lastLeaf) { - termsEnum = t.iterator(); - postings = new PostingsEnum[terms.length]; - } - Passage passages[] = highlightDoc(field, terms, content.length(), bi, doc - subContext.docBase, termsEnum, postings, maxPassages); - if (passages.length == 0) { - passages = getEmptyHighlight(field, bi, maxPassages); - } - if (passages.length > 0) { - // otherwise a null snippet (eg if field is missing - // entirely from the doc) - highlights.put(doc, fieldFormatter.format(passages, content)); - } - lastLeaf = leaf; - } - - return highlights; - } - - // algorithm: treat sentence snippets as miniature documents - // we can intersect these with the postings lists via BreakIterator.preceding(offset),s - // score each sentence as norm(sentenceStartOffset) * sum(weight * tf(freq)) - private Passage[] highlightDoc(String field, BytesRef terms[], int contentLength, BreakIterator bi, int doc, - TermsEnum termsEnum, PostingsEnum[] postings, int n) throws IOException { - - //BEGIN EDIT added call to method that returns the offset for the current value (discrete highlighting) - int valueOffset = getOffsetForCurrentValue(field, doc); - //END EDIT - - PassageScorer scorer = getScorer(field); - if (scorer == null) { - throw new NullPointerException("PassageScorer cannot be null"); - } - - - //BEGIN EDIT discrete highlighting - // the scoring needs to be based on the length of the whole field (all values rather than only the current one) - int totalContentLength = getContentLength(field, doc); - if (totalContentLength == -1) { - totalContentLength = contentLength; - } - //END EDIT - - - PriorityQueue pq = new PriorityQueue<>(); - float weights[] = new float[terms.length]; - // initialize postings - for (int i = 0; i < terms.length; i++) { - PostingsEnum de = postings[i]; - int pDoc; - if (de == EMPTY) { - continue; - } else if (de == null) { - postings[i] = EMPTY; // initially - if (!termsEnum.seekExact(terms[i])) { - continue; // term not found - } - de = postings[i] = termsEnum.postings(null, null, PostingsEnum.OFFSETS); - assert de != null; - pDoc = de.advance(doc); - } else { - pDoc = de.docID(); - if (pDoc < doc) { - pDoc = de.advance(doc); - } - } - - if (doc == pDoc) { - //BEGIN EDIT we take into account the length of the whole field (all values) to properly score the snippets - weights[i] = scorer.weight(totalContentLength, de.freq()); - //weights[i] = scorer.weight(contentLength, de.freq()); - //END EDIT - de.nextPosition(); - pq.add(new OffsetsEnum(de, i)); - } - } - - pq.add(new OffsetsEnum(EMPTY, Integer.MAX_VALUE)); // a sentinel for termination - - PriorityQueue passageQueue = new PriorityQueue<>(n, new Comparator() { - @Override - public int compare(Passage left, Passage right) { - if (left.score < right.score) { - return -1; - } else if (left.score > right.score) { - return 1; - } else { - return left.startOffset - right.startOffset; - } - } - }); - Passage current = new Passage(); - - OffsetsEnum off; - while ((off = pq.poll()) != null) { - final PostingsEnum dp = off.dp; - - int start = dp.startOffset(); - assert start >= 0; - int end = dp.endOffset(); - // LUCENE-5166: this hit would span the content limit... however more valid - // hits may exist (they are sorted by start). so we pretend like we never - // saw this term, it won't cause a passage to be added to passageQueue or anything. - assert EMPTY.startOffset() == Integer.MAX_VALUE; - if (start < contentLength && end > contentLength) { - continue; - } - - - //BEGIN EDIT support for discrete highlighting (added block code) - //switch to the first match in the current value if there is one - boolean seenEnough = false; - while (start < valueOffset) { - if (off.pos == dp.freq()) { - seenEnough = true; - break; - } else { - off.pos++; - dp.nextPosition(); - start = dp.startOffset(); - end = dp.endOffset(); - } - } - - //continue with next term if we've already seen the current one all the times it appears - //that means that the current value doesn't hold matches for the current term - if (seenEnough) { - continue; - } - - //we now subtract the offset of the current value to both start and end - start -= valueOffset; - end -= valueOffset; - //END EDIT - - - if (start >= current.endOffset) { - if (current.startOffset >= 0) { - // finalize current - //BEGIN EDIT we take into account the value offset when scoring the snippet based on its position - current.score *= scorer.norm(current.startOffset + valueOffset); - //current.score *= scorer.norm(current.startOffset); - //END EDIT - // new sentence: first add 'current' to queue - if (passageQueue.size() == n && current.score < passageQueue.peek().score) { - current.reset(); // can't compete, just reset it - } else { - passageQueue.offer(current); - if (passageQueue.size() > n) { - current = passageQueue.poll(); - current.reset(); - } else { - current = new Passage(); - } - } - } - // if we exceed limit, we are done - if (start >= contentLength) { - Passage passages[] = new Passage[passageQueue.size()]; - passageQueue.toArray(passages); - for (Passage p : passages) { - p.sort(); - } - // sort in ascending order - Arrays.sort(passages, new Comparator() { - @Override - public int compare(Passage left, Passage right) { - return left.startOffset - right.startOffset; - } - }); - return passages; - } - // advance breakiterator - assert BreakIterator.DONE < 0; - current.startOffset = Math.max(bi.preceding(start+1), 0); - current.endOffset = Math.min(bi.next(), contentLength); - } - int tf = 0; - while (true) { - tf++; - current.addMatch(start, end, terms[off.id]); - if (off.pos == dp.freq()) { - break; // removed from pq - } else { - off.pos++; - dp.nextPosition(); - //BEGIN EDIT support for discrete highlighting - start = dp.startOffset() - valueOffset; - end = dp.endOffset() - valueOffset; - //start = dp.startOffset(); - //end = dp.endOffset(); - //END EDIT - } - if (start >= current.endOffset || end > contentLength) { - pq.offer(off); - break; - } - } - current.score += weights[off.id] * scorer.tf(tf, current.endOffset - current.startOffset); - } - - // Dead code but compiler disagrees: - assert false; - return null; - } - - /** Called to summarize a document when no hits were - * found. By default this just returns the first - * {@code maxPassages} sentences; subclasses can override - * to customize. */ - protected Passage[] getEmptyHighlight(String fieldName, BreakIterator bi, int maxPassages) { - // BreakIterator should be un-next'd: - List passages = new ArrayList<>(); - int pos = bi.current(); - assert pos == 0; - while (passages.size() < maxPassages) { - int next = bi.next(); - if (next == BreakIterator.DONE) { - break; - } - Passage passage = new Passage(); - passage.score = Float.NaN; - passage.startOffset = pos; - passage.endOffset = next; - passages.add(passage); - pos = next; - } - - return passages.toArray(new Passage[passages.size()]); - } - - private static class OffsetsEnum implements Comparable { - PostingsEnum dp; - int pos; - int id; - - OffsetsEnum(PostingsEnum dp, int id) throws IOException { - this.dp = dp; - this.id = id; - this.pos = 1; - } - - @Override - public int compareTo(OffsetsEnum other) { - try { - int off = dp.startOffset(); - int otherOff = other.dp.startOffset(); - if (off == otherOff) { - return id - other.id; - } else { - return Long.signum(((long)off) - otherOff); - } - } catch (IOException e) { - throw new RuntimeException(e); - } - } - } - - private static final PostingsEnum EMPTY = new PostingsEnum() { - - @Override - public int nextPosition() throws IOException { return 0; } - - @Override - public int startOffset() throws IOException { return Integer.MAX_VALUE; } - - @Override - public int endOffset() throws IOException { return Integer.MAX_VALUE; } - - @Override - public BytesRef getPayload() throws IOException { return null; } - - @Override - public int freq() throws IOException { return 0; } - - @Override - public int docID() { return NO_MORE_DOCS; } - - @Override - public int nextDoc() throws IOException { return NO_MORE_DOCS; } - - @Override - public int advance(int target) throws IOException { return NO_MORE_DOCS; } - - @Override - public long cost() { return 0; } - }; - - private static class LimitedStoredFieldVisitor extends StoredFieldVisitor { - private final String fields[]; - private final char valueSeparators[]; - private final int maxLength; - private final StringBuilder builders[]; - private int currentField = -1; - - public LimitedStoredFieldVisitor(String fields[], char valueSeparators[], int maxLength) { - assert fields.length == valueSeparators.length; - this.fields = fields; - this.valueSeparators = valueSeparators; - this.maxLength = maxLength; - builders = new StringBuilder[fields.length]; - for (int i = 0; i < builders.length; i++) { - builders[i] = new StringBuilder(); - } - } - - @Override - public void stringField(FieldInfo fieldInfo, byte[] bytes) throws IOException { - String value = new String(bytes, StandardCharsets.UTF_8); - assert currentField >= 0; - StringBuilder builder = builders[currentField]; - if (builder.length() > 0 && builder.length() < maxLength) { - builder.append(valueSeparators[currentField]); - } - if (builder.length() + value.length() > maxLength) { - builder.append(value, 0, maxLength - builder.length()); - } else { - builder.append(value); - } - } - - @Override - public Status needsField(FieldInfo fieldInfo) throws IOException { - currentField = Arrays.binarySearch(fields, fieldInfo.name); - if (currentField < 0) { - return Status.NO; - } else if (builders[currentField].length() > maxLength) { - return fields.length == 1 ? Status.STOP : Status.NO; - } - return Status.YES; - } - - String getValue(int i) { - return builders[i].toString(); - } - - void reset() { - currentField = -1; - for (int i = 0; i < fields.length; i++) { - builders[i].setLength(0); - } - } - } -} diff --git a/src/main/java/org/elasticsearch/search/highlight/HighlightPhase.java b/src/main/java/org/elasticsearch/search/highlight/HighlightPhase.java index cd3c12591f7..d7d303138d8 100644 --- a/src/main/java/org/elasticsearch/search/highlight/HighlightPhase.java +++ b/src/main/java/org/elasticsearch/search/highlight/HighlightPhase.java @@ -22,7 +22,6 @@ package org.elasticsearch.search.highlight; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.lucene.index.IndexOptions; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.regex.Regex; diff --git a/src/main/java/org/elasticsearch/search/highlight/HighlightUtils.java b/src/main/java/org/elasticsearch/search/highlight/HighlightUtils.java index 0012ad20d52..c6620d247bf 100644 --- a/src/main/java/org/elasticsearch/search/highlight/HighlightUtils.java +++ b/src/main/java/org/elasticsearch/search/highlight/HighlightUtils.java @@ -20,8 +20,6 @@ package org.elasticsearch.search.highlight; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; - -import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.highlight.DefaultEncoder; import org.apache.lucene.search.highlight.Encoder; import org.apache.lucene.search.highlight.SimpleHTMLEncoder; @@ -29,7 +27,6 @@ import org.elasticsearch.index.fieldvisitor.CustomFieldsVisitor; import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.search.fetch.FetchSubPhase; import org.elasticsearch.search.internal.SearchContext; -import org.elasticsearch.search.lookup.SearchLookup; import org.elasticsearch.search.lookup.SourceLookup; import java.io.IOException; @@ -39,6 +36,7 @@ public final class HighlightUtils { //U+2029 PARAGRAPH SEPARATOR (PS): each value holds a discrete passage for highlighting (postings highlighter) public static final char PARAGRAPH_SEPARATOR = 8233; + public static final char NULL_SEPARATOR = '\u0000'; private HighlightUtils() { diff --git a/src/main/java/org/elasticsearch/search/highlight/PostingsHighlighter.java b/src/main/java/org/elasticsearch/search/highlight/PostingsHighlighter.java index f30a0545d95..882a4464357 100644 --- a/src/main/java/org/elasticsearch/search/highlight/PostingsHighlighter.java +++ b/src/main/java/org/elasticsearch/search/highlight/PostingsHighlighter.java @@ -18,31 +18,14 @@ */ package org.elasticsearch.search.highlight; -import com.google.common.collect.Lists; import com.google.common.collect.Maps; - +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.index.IndexOptions; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.Term; -import org.apache.lucene.search.BooleanClause; -import org.apache.lucene.search.BooleanQuery; -import org.apache.lucene.search.ConstantScoreQuery; -import org.apache.lucene.search.FilteredQuery; -import org.apache.lucene.search.MultiTermQuery; -import org.apache.lucene.search.Query; -import org.apache.lucene.search.ScoringRewrite; -import org.apache.lucene.search.TopTermsRewrite; -import org.apache.lucene.search.Weight; +import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.highlight.Encoder; -import org.apache.lucene.search.postingshighlight.CustomPassageFormatter; -import org.apache.lucene.search.postingshighlight.CustomPostingsHighlighter; -import org.apache.lucene.search.postingshighlight.Snippet; -import org.apache.lucene.search.postingshighlight.WholeBreakIterator; -import org.apache.lucene.util.BytesRef; +import org.apache.lucene.search.postingshighlight.*; import org.apache.lucene.util.CollectionUtil; -import org.apache.lucene.util.UnicodeUtil; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.text.StringText; import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.search.fetch.FetchPhaseExecutionException; @@ -51,13 +34,7 @@ import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; import java.text.BreakIterator; -import java.util.ArrayList; -import java.util.Comparator; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.SortedSet; -import java.util.TreeSet; +import java.util.*; public class PostingsHighlighter implements Highlighter { @@ -81,15 +58,7 @@ public class PostingsHighlighter implements Highlighter { FetchSubPhase.HitContext hitContext = highlighterContext.hitContext; if (!hitContext.cache().containsKey(CACHE_KEY)) { - //get the non rewritten query and rewrite it - Query query; - try { - query = rewrite(highlighterContext, hitContext.topLevelReader()); - SortedSet queryTerms = extractTerms(context.searcher().createNormalizedWeight(query, false)); - hitContext.cache().put(CACHE_KEY, new HighlighterEntry(queryTerms)); - } catch (IOException e) { - throw new FetchPhaseExecutionException(context, "Failed to highlight field [" + highlighterContext.fieldName + "]", e); - } + hitContext.cache().put(CACHE_KEY, new HighlighterEntry()); } HighlighterEntry highlighterEntry = (HighlighterEntry) hitContext.cache().get(CACHE_KEY); @@ -98,37 +67,34 @@ public class PostingsHighlighter implements Highlighter { if (mapperHighlighterEntry == null) { Encoder encoder = field.fieldOptions().encoder().equals("html") ? HighlightUtils.Encoders.HTML : HighlightUtils.Encoders.DEFAULT; CustomPassageFormatter passageFormatter = new CustomPassageFormatter(field.fieldOptions().preTags()[0], field.fieldOptions().postTags()[0], encoder); - BytesRef[] filteredQueryTerms = filterTerms(highlighterEntry.queryTerms, fieldMapper.names().indexName(), field.fieldOptions().requireFieldMatch()); - mapperHighlighterEntry = new MapperHighlighterEntry(passageFormatter, filteredQueryTerms); + mapperHighlighterEntry = new MapperHighlighterEntry(passageFormatter); } - //we merge back multiple values into a single value using the paragraph separator, unless we have to highlight every single value separately (number_of_fragments=0). - boolean mergeValues = field.fieldOptions().numberOfFragments() != 0; List snippets = new ArrayList<>(); int numberOfFragments; - try { - //we manually load the field values (from source if needed) - List textsToHighlight = HighlightUtils.loadFieldValues(field, fieldMapper, context, hitContext); - CustomPostingsHighlighter highlighter = new CustomPostingsHighlighter(mapperHighlighterEntry.passageFormatter, textsToHighlight, mergeValues, Integer.MAX_VALUE-1, field.fieldOptions().noMatchSize()); - - if (field.fieldOptions().numberOfFragments() == 0) { - highlighter.setBreakIterator(new WholeBreakIterator()); - numberOfFragments = 1; //1 per value since we highlight per value + Analyzer analyzer = context.mapperService().documentMapper(hitContext.hit().type()).mappers().indexAnalyzer(); + List fieldValues = HighlightUtils.loadFieldValues(field, fieldMapper, context, hitContext); + CustomPostingsHighlighter highlighter; + if (field.fieldOptions().numberOfFragments() == 0) { + //we use a control char to separate values, which is the only char that the custom break iterator breaks the text on, + //so we don't lose the distinction between the different values of a field and we get back a snippet per value + String fieldValue = mergeFieldValues(fieldValues, HighlightUtils.NULL_SEPARATOR); + CustomSeparatorBreakIterator breakIterator = new CustomSeparatorBreakIterator(HighlightUtils.NULL_SEPARATOR); + highlighter = new CustomPostingsHighlighter(analyzer, mapperHighlighterEntry.passageFormatter, breakIterator, fieldValue, field.fieldOptions().noMatchSize() > 0); + numberOfFragments = fieldValues.size(); //we are highlighting the whole content, one snippet per value } else { + //using paragraph separator we make sure that each field value holds a discrete passage for highlighting + String fieldValue = mergeFieldValues(fieldValues, HighlightUtils.PARAGRAPH_SEPARATOR); + highlighter = new CustomPostingsHighlighter(analyzer, mapperHighlighterEntry.passageFormatter, fieldValue, field.fieldOptions().noMatchSize() > 0); numberOfFragments = field.fieldOptions().numberOfFragments(); } - //we highlight every value separately calling the highlight method multiple times, only if we need to have back a snippet per value (whole value) - int values = mergeValues ? 1 : textsToHighlight.size(); - for (int i = 0; i < values; i++) { - Snippet[] fieldSnippets = highlighter.highlightDoc(fieldMapper.names().indexName(), mapperHighlighterEntry.filteredQueryTerms, hitContext.reader(), hitContext.docId(), numberOfFragments); - if (fieldSnippets != null) { - for (Snippet fieldSnippet : fieldSnippets) { - if (Strings.hasText(fieldSnippet.getText())) { - snippets.add(fieldSnippet); - } - } + IndexSearcher searcher = new IndexSearcher(hitContext.reader()); + Snippet[] fieldSnippets = highlighter.highlightField(fieldMapper.names().indexName(), highlighterContext.query.originalQuery(), searcher, hitContext.docId(), numberOfFragments); + for (Snippet fieldSnippet : fieldSnippets) { + if (Strings.hasText(fieldSnippet.getText())) { + snippets.add(fieldSnippet); } } @@ -160,97 +126,17 @@ public class PostingsHighlighter implements Highlighter { return null; } - private static Query rewrite(HighlighterContext highlighterContext, IndexReader reader) throws IOException { - - Query original = highlighterContext.query.originalQuery(); - - //we walk the query tree and when we encounter multi term queries we need to make sure the rewrite method - //supports multi term extraction. If not we temporarily override it (and restore it after the rewrite). - List> modifiedMultiTermQueries = Lists.newArrayList(); - overrideMultiTermRewriteMethod(original, modifiedMultiTermQueries); - - //rewrite is expensive: if the query was already rewritten we try not to rewrite it again - if (highlighterContext.query.queryRewritten() && modifiedMultiTermQueries.size() == 0) { - //return the already rewritten query - return highlighterContext.query.query(); - } - - Query query = original; - for (Query rewrittenQuery = query.rewrite(reader); rewrittenQuery != query; - rewrittenQuery = query.rewrite(reader)) { - query = rewrittenQuery; - } - - //set back the original rewrite method after the rewrite is done - for (Tuple modifiedMultiTermQuery : modifiedMultiTermQueries) { - modifiedMultiTermQuery.v1().setRewriteMethod(modifiedMultiTermQuery.v2()); - } - - return query; - } - - private static void overrideMultiTermRewriteMethod(Query query, List> modifiedMultiTermQueries) { - - if (query instanceof MultiTermQuery) { - MultiTermQuery originalMultiTermQuery = (MultiTermQuery) query; - if (!allowsForTermExtraction(originalMultiTermQuery.getRewriteMethod())) { - MultiTermQuery.RewriteMethod originalRewriteMethod = originalMultiTermQuery.getRewriteMethod(); - originalMultiTermQuery.setRewriteMethod(new MultiTermQuery.TopTermsScoringBooleanQueryRewrite(50)); - //we need to rewrite anyway if it is a multi term query which was rewritten with the wrong rewrite method - modifiedMultiTermQueries.add(Tuple.tuple(originalMultiTermQuery, originalRewriteMethod)); - } - } - - if (query instanceof BooleanQuery) { - BooleanQuery booleanQuery = (BooleanQuery) query; - for (BooleanClause booleanClause : booleanQuery) { - overrideMultiTermRewriteMethod(booleanClause.getQuery(), modifiedMultiTermQueries); - } - } - - if (query instanceof FilteredQuery) { - overrideMultiTermRewriteMethod(((FilteredQuery) query).getQuery(), modifiedMultiTermQueries); - } - - if (query instanceof ConstantScoreQuery) { - overrideMultiTermRewriteMethod(((ConstantScoreQuery) query).getQuery(), modifiedMultiTermQueries); - } - } - - private static boolean allowsForTermExtraction(MultiTermQuery.RewriteMethod rewriteMethod) { - return rewriteMethod instanceof TopTermsRewrite || rewriteMethod instanceof ScoringRewrite; - } - - private static SortedSet extractTerms(Weight weight) { - SortedSet queryTerms = new TreeSet<>(); - weight.extractTerms(queryTerms); - return queryTerms; - } - - private static BytesRef[] filterTerms(SortedSet queryTerms, String field, boolean requireFieldMatch) { - SortedSet fieldTerms; - if (requireFieldMatch) { - Term floor = new Term(field, ""); - Term ceiling = new Term(field, UnicodeUtil.BIG_TERM); - fieldTerms = queryTerms.subSet(floor, ceiling); - } else { - fieldTerms = queryTerms; - } - - BytesRef terms[] = new BytesRef[fieldTerms.size()]; - int termUpto = 0; - for(Term term : fieldTerms) { - terms[termUpto++] = term.bytes(); - } - - return terms; + private static String mergeFieldValues(List fieldValues, char valuesSeparator) { + //postings highlighter accepts all values in a single string, as offsets etc. need to match with content + //loaded from stored fields, we merge all values using a proper separator + String rawValue = Strings.collectionToDelimitedString(fieldValues, String.valueOf(valuesSeparator)); + return rawValue.substring(0, Math.min(rawValue.length(), Integer.MAX_VALUE - 1)); } private static List filterSnippets(List snippets, int numberOfFragments) { //We need to filter the snippets as due to no_match_size we could have - //either highlighted snippets together non highlighted ones - //We don't want to mix those up + //either highlighted snippets or non highlighted ones and we don't want to mix those up List filteredSnippets = new ArrayList<>(snippets.size()); for (Snippet snippet : snippets) { if (snippet.isHighlighted()) { @@ -263,8 +149,8 @@ public class PostingsHighlighter implements Highlighter { if (filteredSnippets.size() == 0) { if (snippets.size() > 0) { Snippet snippet = snippets.get(0); - //if we did discrete per value highlighting using whole break iterator (as number_of_fragments was 0) - //we need to obtain the first sentence of the first value + //if we tried highlighting the whole content using whole break iterator (as number_of_fragments was 0) + //we need to return the first sentence of the content rather than the whole content if (numberOfFragments == 0) { BreakIterator bi = BreakIterator.getSentenceInstance(Locale.ROOT); String text = snippet.getText(); @@ -283,21 +169,14 @@ public class PostingsHighlighter implements Highlighter { } private static class HighlighterEntry { - final SortedSet queryTerms; Map, MapperHighlighterEntry> mappers = Maps.newHashMap(); - - private HighlighterEntry(SortedSet queryTerms) { - this.queryTerms = queryTerms; - } } private static class MapperHighlighterEntry { final CustomPassageFormatter passageFormatter; - final BytesRef[] filteredQueryTerms; - private MapperHighlighterEntry(CustomPassageFormatter passageFormatter, BytesRef[] filteredQueryTerms) { + private MapperHighlighterEntry(CustomPassageFormatter passageFormatter) { this.passageFormatter = passageFormatter; - this.filteredQueryTerms = filteredQueryTerms; } } } diff --git a/src/test/java/org/apache/lucene/search/postingshighlight/CustomPostingsHighlighterTests.java b/src/test/java/org/apache/lucene/search/postingshighlight/CustomPostingsHighlighterTests.java index 43165fa4b1c..8d04caae60c 100644 --- a/src/test/java/org/apache/lucene/search/postingshighlight/CustomPostingsHighlighterTests.java +++ b/src/test/java/org/apache/lucene/search/postingshighlight/CustomPostingsHighlighterTests.java @@ -27,106 +27,16 @@ import org.apache.lucene.index.*; import org.apache.lucene.search.*; import org.apache.lucene.search.highlight.DefaultEncoder; import org.apache.lucene.store.Directory; -import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.UnicodeUtil; import org.elasticsearch.search.highlight.HighlightUtils; import org.elasticsearch.test.ElasticsearchTestCase; import org.junit.Test; -import java.io.IOException; -import java.util.*; - import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.CoreMatchers.notNullValue; public class CustomPostingsHighlighterTests extends ElasticsearchTestCase { @Test - public void testDiscreteHighlightingPerValue() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random())); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Field body = new Field("body", "", offsetsType); - final String firstValue = "This is a test. Just a test highlighting from postings highlighter."; - Document doc = new Document(); - doc.add(body); - body.setStringValue(firstValue); - - final String secondValue = "This is the second value to perform highlighting on."; - Field body2 = new Field("body", "", offsetsType); - doc.add(body2); - body2.setStringValue(secondValue); - - final String thirdValue = "This is the third value to test highlighting with postings."; - Field body3 = new Field("body", "", offsetsType); - doc.add(body3); - body3.setStringValue(thirdValue); - - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - List fieldValues = new ArrayList<>(); - fieldValues.add(firstValue); - fieldValues.add(secondValue); - fieldValues.add(thirdValue); - - - IndexSearcher searcher = newSearcher(ir); - - Query query = new TermQuery(new Term("body", "highlighting")); - BytesRef[] queryTerms = filterTerms(extractTerms(searcher, query), "body", true); - - TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER); - assertThat(topDocs.totalHits, equalTo(1)); - int docId = topDocs.scoreDocs[0].doc; - - //highlighting per value, considering whole values (simulating number_of_fragments=0) - CustomPostingsHighlighter highlighter = new CustomPostingsHighlighter(new CustomPassageFormatter("", "", new DefaultEncoder()), fieldValues, false, Integer.MAX_VALUE - 1, 0); - highlighter.setBreakIterator(new WholeBreakIterator()); - - Snippet[] snippets = highlighter.highlightDoc("body", queryTerms, ir, docId, 5); - assertThat(snippets.length, equalTo(1)); - assertThat(snippets[0].getText(), equalTo("This is a test. Just a test highlighting from postings highlighter.")); - - snippets = highlighter.highlightDoc("body", queryTerms, ir, docId, 5); - assertThat(snippets.length, equalTo(1)); - assertThat(snippets[0].getText(), equalTo("This is the second value to perform highlighting on.")); - - snippets = highlighter.highlightDoc("body", queryTerms, ir, docId, 5); - assertThat(snippets.length, equalTo(1)); - assertThat(snippets[0].getText(), equalTo("This is the third value to test highlighting with postings.")); - - - //let's try without whole break iterator as well, to prove that highlighting works the same when working per value (not optimized though) - highlighter = new CustomPostingsHighlighter(new CustomPassageFormatter("", "", new DefaultEncoder()), fieldValues, false, Integer.MAX_VALUE - 1, 0); - - snippets = highlighter.highlightDoc("body", queryTerms, ir, docId, 5); - assertThat(snippets.length, equalTo(1)); - assertThat(snippets[0].getText(), equalTo("Just a test highlighting from postings highlighter.")); - - snippets = highlighter.highlightDoc("body", queryTerms, ir, docId, 5); - assertThat(snippets.length, equalTo(1)); - assertThat(snippets[0].getText(), equalTo("This is the second value to perform highlighting on.")); - - snippets = highlighter.highlightDoc("body", queryTerms, ir, docId, 5); - assertThat(snippets.length, equalTo(1)); - assertThat(snippets[0].getText(), equalTo("This is the third value to test highlighting with postings.")); - - ir.close(); - dir.close(); - } - - /* - Tests that scoring works properly even when using discrete per value highlighting - */ - @Test - public void testDiscreteHighlightingScoring() throws Exception { + public void testCustomPostingsHighlighter() throws Exception { Directory dir = newDirectory(); IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random())); @@ -166,31 +76,23 @@ public class CustomPostingsHighlighterTests extends ElasticsearchTestCase { IndexReader ir = iw.getReader(); iw.close(); - String firstHlValue = "Just a test1 highlighting from postings highlighter."; String secondHlValue = "This is the second highlighting value to perform highlighting on a longer text that gets scored lower."; String thirdHlValue = "This is highlighting the third short highlighting value."; String fourthHlValue = "Just a test4 highlighting from postings highlighter."; - IndexSearcher searcher = newSearcher(ir); Query query = new TermQuery(new Term("body", "highlighting")); - BytesRef[] queryTerms = filterTerms(extractTerms(searcher, query), "body", true); TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER); assertThat(topDocs.totalHits, equalTo(1)); int docId = topDocs.scoreDocs[0].doc; - List fieldValues = new ArrayList<>(); - fieldValues.add(firstValue); - fieldValues.add(secondValue); - fieldValues.add(thirdValue); - fieldValues.add(fourthValue); + String fieldValue = firstValue + HighlightUtils.PARAGRAPH_SEPARATOR + secondValue + HighlightUtils.PARAGRAPH_SEPARATOR + thirdValue + HighlightUtils.PARAGRAPH_SEPARATOR + fourthValue; - boolean mergeValues = true; - CustomPostingsHighlighter highlighter = new CustomPostingsHighlighter(new CustomPassageFormatter("", "", new DefaultEncoder()), fieldValues, mergeValues, Integer.MAX_VALUE-1, 0); - Snippet[] snippets = highlighter.highlightDoc("body", queryTerms, ir, docId, 5); + CustomPostingsHighlighter highlighter = new CustomPostingsHighlighter(null, new CustomPassageFormatter("", "", new DefaultEncoder()), fieldValue, false); + Snippet[] snippets = highlighter.highlightField("body", query, searcher, docId, 5); assertThat(snippets.length, equalTo(4)); @@ -199,211 +101,6 @@ public class CustomPostingsHighlighterTests extends ElasticsearchTestCase { assertThat(snippets[2].getText(), equalTo(thirdHlValue)); assertThat(snippets[3].getText(), equalTo(fourthHlValue)); - - //Let's highlight each separate value and check how the snippets are scored - mergeValues = false; - highlighter = new CustomPostingsHighlighter(new CustomPassageFormatter("", "", new DefaultEncoder()), fieldValues, mergeValues, Integer.MAX_VALUE-1, 0); - List snippets2 = new ArrayList<>(); - for (int i = 0; i < fieldValues.size(); i++) { - snippets2.addAll(Arrays.asList(highlighter.highlightDoc("body", queryTerms, ir, docId, 5))); - } - - assertThat(snippets2.size(), equalTo(4)); - assertThat(snippets2.get(0).getText(), equalTo(firstHlValue)); - assertThat(snippets2.get(1).getText(), equalTo(secondHlValue)); - assertThat(snippets2.get(2).getText(), equalTo(thirdHlValue)); - assertThat(snippets2.get(3).getText(), equalTo(fourthHlValue)); - - Comparator comparator = new Comparator() { - @Override - public int compare(Snippet o1, Snippet o2) { - return (int)Math.signum(o1.getScore() - o2.getScore()); - } - }; - - //sorting both groups of snippets - Arrays.sort(snippets, comparator); - Collections.sort(snippets2, comparator); - - //checking that the snippets are in the same order, regardless of whether we used per value discrete highlighting or not - //we can't compare the scores directly since they are slightly different due to the multiValued separator added when merging values together - //That causes slightly different lengths and start offsets, thus a slightly different score. - //Anyways, that's not an issue. What's important is that the score is computed the same way, so that the produced order is always the same. - for (int i = 0; i < snippets.length; i++) { - assertThat(snippets[i].getText(), equalTo(snippets2.get(i).getText())); - } - - ir.close(); - dir.close(); - } - - /* - Tests that we produce the same snippets and scores when manually merging values in our own custom highlighter rather than using the built-in code - */ - @Test - public void testMergeValuesScoring() throws Exception { - - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random())); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - - //good position but only one match - final String firstValue = "This is a test. Just a test1 highlighting from postings highlighter."; - Field body = new Field("body", "", offsetsType); - Document doc = new Document(); - doc.add(body); - body.setStringValue(firstValue); - - //two matches, not the best snippet due to its length though - final String secondValue = "This is the second highlighting value to perform highlighting on a longer text that gets scored lower."; - Field body2 = new Field("body", "", offsetsType); - doc.add(body2); - body2.setStringValue(secondValue); - - //two matches and short, will be scored highest - final String thirdValue = "This is highlighting the third short highlighting value."; - Field body3 = new Field("body", "", offsetsType); - doc.add(body3); - body3.setStringValue(thirdValue); - - //one match, same as first but at the end, will be scored lower due to its position - final String fourthValue = "Just a test4 highlighting from postings highlighter."; - Field body4 = new Field("body", "", offsetsType); - doc.add(body4); - body4.setStringValue(fourthValue); - - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - - String firstHlValue = "Just a test1 highlighting from postings highlighter."; - String secondHlValue = "This is the second highlighting value to perform highlighting on a longer text that gets scored lower."; - String thirdHlValue = "This is highlighting the third short highlighting value."; - String fourthHlValue = "Just a test4 highlighting from postings highlighter."; - - - IndexSearcher searcher = newSearcher(ir); - Query query = new TermQuery(new Term("body", "highlighting")); - BytesRef[] queryTerms = filterTerms(extractTerms(searcher, query), "body", true); - - TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER); - assertThat(topDocs.totalHits, equalTo(1)); - - int docId = topDocs.scoreDocs[0].doc; - - List fieldValues = new ArrayList<>(); - fieldValues.add(firstValue); - fieldValues.add(secondValue); - fieldValues.add(thirdValue); - fieldValues.add(fourthValue); - - boolean mergeValues = true; - CustomPostingsHighlighter highlighter = new CustomPostingsHighlighter(new CustomPassageFormatter("", "", new DefaultEncoder()), fieldValues, mergeValues, Integer.MAX_VALUE-1, 0); - Snippet[] snippets = highlighter.highlightDoc("body", queryTerms, ir, docId, 5); - - assertThat(snippets.length, equalTo(4)); - - assertThat(snippets[0].getText(), equalTo(firstHlValue)); - assertThat(snippets[1].getText(), equalTo(secondHlValue)); - assertThat(snippets[2].getText(), equalTo(thirdHlValue)); - assertThat(snippets[3].getText(), equalTo(fourthHlValue)); - - - //testing now our fork / normal postings highlighter, which merges multiple values together using the paragraph separator - XPostingsHighlighter highlighter2 = new XPostingsHighlighter(Integer.MAX_VALUE - 1) { - @Override - protected char getMultiValuedSeparator(String field) { - return HighlightUtils.PARAGRAPH_SEPARATOR; - } - - @Override - protected PassageFormatter getFormatter(String field) { - return new CustomPassageFormatter("", "", new DefaultEncoder()); - } - }; - - Map highlightMap = highlighter2.highlightFieldsAsObjects(new String[]{"body"}, query, searcher, new int[]{docId}, new int[]{5}); - Object[] objects = highlightMap.get("body"); - assertThat(objects, notNullValue()); - assertThat(objects.length, equalTo(1)); - Snippet[] normalSnippets = (Snippet[])objects[0]; - - assertThat(normalSnippets.length, equalTo(4)); - - assertThat(normalSnippets[0].getText(), equalTo(firstHlValue)); - assertThat(normalSnippets[1].getText(), equalTo(secondHlValue)); - assertThat(normalSnippets[2].getText(), equalTo(thirdHlValue)); - assertThat(normalSnippets[3].getText(), equalTo(fourthHlValue)); - - - for (int i = 0; i < normalSnippets.length; i++) { - Snippet normalSnippet = snippets[0]; - Snippet customSnippet = normalSnippets[0]; - assertThat(customSnippet.getText(), equalTo(normalSnippet.getText())); - assertThat(customSnippet.getScore(), equalTo(normalSnippet.getScore())); - } - - ir.close(); - dir.close(); - } - - @Test - public void testRequireFieldMatch() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random())); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Field body = new Field("body", "", offsetsType); - Field none = new Field("none", "", offsetsType); - Document doc = new Document(); - doc.add(body); - doc.add(none); - - String firstValue = "This is a test. Just a test highlighting from postings. Feel free to ignore."; - body.setStringValue(firstValue); - none.setStringValue(firstValue); - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - Query query = new TermQuery(new Term("none", "highlighting")); - IndexSearcher searcher = newSearcher(ir); - SortedSet queryTerms = extractTerms(searcher, query); - TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER); - assertThat(topDocs.totalHits, equalTo(1)); - int docId = topDocs.scoreDocs[0].doc; - - List values = new ArrayList<>(); - values.add(firstValue); - - CustomPassageFormatter passageFormatter = new CustomPassageFormatter("", "", new DefaultEncoder()); - CustomPostingsHighlighter highlighter = new CustomPostingsHighlighter(passageFormatter, values, true, Integer.MAX_VALUE - 1, 0); - - //no snippets with simulated require field match (we filter the terms ourselves) - boolean requireFieldMatch = true; - BytesRef[] filteredQueryTerms = filterTerms(queryTerms, "body", requireFieldMatch); - Snippet[] snippets = highlighter.highlightDoc("body", filteredQueryTerms, ir, docId, 5); - assertThat(snippets.length, equalTo(0)); - - - highlighter = new CustomPostingsHighlighter(passageFormatter, values, true, Integer.MAX_VALUE - 1, 0); - //one snippet without require field match, just passing in the query terms with no filtering on our side - requireFieldMatch = false; - filteredQueryTerms = filterTerms(queryTerms, "body", requireFieldMatch); - snippets = highlighter.highlightDoc("body", filteredQueryTerms, ir, docId, 5); - assertThat(snippets.length, equalTo(1)); - assertThat(snippets[0].getText(), equalTo("Just a test highlighting from postings.")); - ir.close(); dir.close(); } @@ -434,56 +131,22 @@ public class CustomPostingsHighlighterTests extends ElasticsearchTestCase { Query query = new TermQuery(new Term("none", "highlighting")); IndexSearcher searcher = newSearcher(ir); - SortedSet queryTerms = extractTerms(searcher, query); TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER); assertThat(topDocs.totalHits, equalTo(1)); int docId = topDocs.scoreDocs[0].doc; - List values = new ArrayList<>(); - values.add(firstValue); - - BytesRef[] filteredQueryTerms = filterTerms(queryTerms, "body", true); CustomPassageFormatter passageFormatter = new CustomPassageFormatter("", "", new DefaultEncoder()); - CustomPostingsHighlighter highlighter = new CustomPostingsHighlighter(passageFormatter, values, true, Integer.MAX_VALUE - 1, 0); - Snippet[] snippets = highlighter.highlightDoc("body", filteredQueryTerms, ir, docId, 5); + CustomPostingsHighlighter highlighter = new CustomPostingsHighlighter(null, passageFormatter, firstValue, false); + Snippet[] snippets = highlighter.highlightField("body", query, searcher, docId, 5); assertThat(snippets.length, equalTo(0)); - highlighter = new CustomPostingsHighlighter(passageFormatter, values, true, Integer.MAX_VALUE - 1, scaledRandomIntBetween(1, 10)); - snippets = highlighter.highlightDoc("body", filteredQueryTerms, ir, docId, 5); + highlighter = new CustomPostingsHighlighter(null, passageFormatter, firstValue, true); + snippets = highlighter.highlightField("body", query, searcher, docId, 5); assertThat(snippets.length, equalTo(1)); assertThat(snippets[0].getText(), equalTo("This is a test.")); ir.close(); dir.close(); } - - private static SortedSet extractTerms(IndexSearcher searcher, Query query) throws IOException { - return extractTerms(searcher.createNormalizedWeight(query, false)); - } - - private static SortedSet extractTerms(Weight weight) { - SortedSet queryTerms = new TreeSet<>(); - weight.extractTerms(queryTerms); - return queryTerms; - } - - private static BytesRef[] filterTerms(SortedSet queryTerms, String field, boolean requireFieldMatch) { - SortedSet fieldTerms; - if (requireFieldMatch) { - Term floor = new Term(field, ""); - Term ceiling = new Term(field, UnicodeUtil.BIG_TERM); - fieldTerms = queryTerms.subSet(floor, ceiling); - } else { - fieldTerms = queryTerms; - } - - BytesRef terms[] = new BytesRef[fieldTerms.size()]; - int termUpto = 0; - for(Term term : fieldTerms) { - terms[termUpto++] = term.bytes(); - } - - return terms; - } } diff --git a/src/test/java/org/apache/lucene/search/postingshighlight/CustomSeparatorBreakIteratorTests.java b/src/test/java/org/apache/lucene/search/postingshighlight/CustomSeparatorBreakIteratorTests.java new file mode 100644 index 00000000000..0df6add5371 --- /dev/null +++ b/src/test/java/org/apache/lucene/search/postingshighlight/CustomSeparatorBreakIteratorTests.java @@ -0,0 +1,186 @@ +/* +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.apache.lucene.search.postingshighlight; + +import org.elasticsearch.search.highlight.HighlightUtils; +import org.elasticsearch.test.ElasticsearchTestCase; +import org.junit.Test; + +import java.text.BreakIterator; +import java.text.CharacterIterator; +import java.text.StringCharacterIterator; +import java.util.Locale; + +import static org.hamcrest.CoreMatchers.equalTo; + +public class CustomSeparatorBreakIteratorTests extends ElasticsearchTestCase { + + @Test + public void testBreakOnCustomSeparator() throws Exception { + Character separator = randomSeparator(); + BreakIterator bi = new CustomSeparatorBreakIterator(separator); + String source = "this" + separator + "is" + separator + "the" + separator + "first" + separator + "sentence"; + bi.setText(source); + assertThat(bi.current(), equalTo(0)); + assertThat(bi.first(), equalTo(0)); + assertThat(source.substring(bi.current(), bi.next()), equalTo("this" + separator)); + assertThat(source.substring(bi.current(), bi.next()), equalTo("is" + separator)); + assertThat(source.substring(bi.current(), bi.next()), equalTo("the" + separator)); + assertThat(source.substring(bi.current(), bi.next()), equalTo("first" + separator)); + assertThat(source.substring(bi.current(), bi.next()), equalTo("sentence")); + assertThat(bi.next(), equalTo(BreakIterator.DONE)); + + assertThat(bi.last(), equalTo(source.length())); + int current = bi.current(); + assertThat(source.substring(bi.previous(), current), equalTo("sentence")); + current = bi.current(); + assertThat(source.substring(bi.previous(), current), equalTo("first" + separator)); + current = bi.current(); + assertThat(source.substring(bi.previous(), current), equalTo("the" + separator)); + current = bi.current(); + assertThat(source.substring(bi.previous(), current), equalTo("is" + separator)); + current = bi.current(); + assertThat(source.substring(bi.previous(), current), equalTo("this" + separator)); + assertThat(bi.previous(), equalTo(BreakIterator.DONE)); + assertThat(bi.current(), equalTo(0)); + + assertThat(source.substring(0, bi.following(9)), equalTo("this" + separator + "is" + separator + "the" + separator)); + + assertThat(source.substring(0, bi.preceding(9)), equalTo("this" + separator + "is" + separator)); + + assertThat(bi.first(), equalTo(0)); + assertThat(source.substring(0, bi.next(3)), equalTo("this" + separator + "is" + separator + "the" + separator)); + } + + @Test + public void testSingleSentences() throws Exception { + BreakIterator expected = BreakIterator.getSentenceInstance(Locale.ROOT); + BreakIterator actual = new CustomSeparatorBreakIterator(randomSeparator()); + assertSameBreaks("a", expected, actual); + assertSameBreaks("ab", expected, actual); + assertSameBreaks("abc", expected, actual); + assertSameBreaks("", expected, actual); + } + + @Test + public void testSliceEnd() throws Exception { + BreakIterator expected = BreakIterator.getSentenceInstance(Locale.ROOT); + BreakIterator actual = new CustomSeparatorBreakIterator(randomSeparator()); + assertSameBreaks("a000", 0, 1, expected, actual); + assertSameBreaks("ab000", 0, 1, expected, actual); + assertSameBreaks("abc000", 0, 1, expected, actual); + assertSameBreaks("000", 0, 0, expected, actual); + } + + @Test + public void testSliceStart() throws Exception { + BreakIterator expected = BreakIterator.getSentenceInstance(Locale.ROOT); + BreakIterator actual = new CustomSeparatorBreakIterator(randomSeparator()); + assertSameBreaks("000a", 3, 1, expected, actual); + assertSameBreaks("000ab", 3, 2, expected, actual); + assertSameBreaks("000abc", 3, 3, expected, actual); + assertSameBreaks("000", 3, 0, expected, actual); + } + + @Test + public void testSliceMiddle() throws Exception { + BreakIterator expected = BreakIterator.getSentenceInstance(Locale.ROOT); + BreakIterator actual = new CustomSeparatorBreakIterator(randomSeparator()); + assertSameBreaks("000a000", 3, 1, expected, actual); + assertSameBreaks("000ab000", 3, 2, expected, actual); + assertSameBreaks("000abc000", 3, 3, expected, actual); + assertSameBreaks("000000", 3, 0, expected, actual); + } + + /** the current position must be ignored, initial position is always first() */ + @Test + public void testFirstPosition() throws Exception { + BreakIterator expected = BreakIterator.getSentenceInstance(Locale.ROOT); + BreakIterator actual = new CustomSeparatorBreakIterator(randomSeparator()); + assertSameBreaks("000ab000", 3, 2, 4, expected, actual); + } + + private static char randomSeparator() { + return randomFrom(' ', HighlightUtils.NULL_SEPARATOR, HighlightUtils.PARAGRAPH_SEPARATOR); + } + + private static void assertSameBreaks(String text, BreakIterator expected, BreakIterator actual) { + assertSameBreaks(new StringCharacterIterator(text), + new StringCharacterIterator(text), + expected, + actual); + } + + private static void assertSameBreaks(String text, int offset, int length, BreakIterator expected, BreakIterator actual) { + assertSameBreaks(text, offset, length, offset, expected, actual); + } + + private static void assertSameBreaks(String text, int offset, int length, int current, BreakIterator expected, BreakIterator actual) { + assertSameBreaks(new StringCharacterIterator(text, offset, offset + length, current), + new StringCharacterIterator(text, offset, offset + length, current), + expected, + actual); + } + + /** Asserts that two breakiterators break the text the same way */ + private static void assertSameBreaks(CharacterIterator one, CharacterIterator two, BreakIterator expected, BreakIterator actual) { + expected.setText(one); + actual.setText(two); + + assertEquals(expected.current(), actual.current()); + + // next() + int v = expected.current(); + while (v != BreakIterator.DONE) { + assertEquals(v = expected.next(), actual.next()); + assertEquals(expected.current(), actual.current()); + } + + // first() + assertEquals(expected.first(), actual.first()); + assertEquals(expected.current(), actual.current()); + // last() + assertEquals(expected.last(), actual.last()); + assertEquals(expected.current(), actual.current()); + + // previous() + v = expected.current(); + while (v != BreakIterator.DONE) { + assertEquals(v = expected.previous(), actual.previous()); + assertEquals(expected.current(), actual.current()); + } + + // following() + for (int i = one.getBeginIndex(); i <= one.getEndIndex(); i++) { + expected.first(); + actual.first(); + assertEquals(expected.following(i), actual.following(i)); + assertEquals(expected.current(), actual.current()); + } + + // preceding() + for (int i = one.getBeginIndex(); i <= one.getEndIndex(); i++) { + expected.last(); + actual.last(); + assertEquals(expected.preceding(i), actual.preceding(i)); + assertEquals(expected.current(), actual.current()); + } + } +} diff --git a/src/test/java/org/apache/lucene/search/postingshighlight/XPostingsHighlighterTests.java b/src/test/java/org/apache/lucene/search/postingshighlight/XPostingsHighlighterTests.java deleted file mode 100644 index 7bd77155962..00000000000 --- a/src/test/java/org/apache/lucene/search/postingshighlight/XPostingsHighlighterTests.java +++ /dev/null @@ -1,1691 +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.apache.lucene.search.postingshighlight; - -import org.apache.lucene.analysis.Analyzer; -import org.apache.lucene.analysis.MockAnalyzer; -import org.apache.lucene.analysis.MockTokenizer; -import org.apache.lucene.document.*; -import org.apache.lucene.index.*; -import org.apache.lucene.search.*; -import org.apache.lucene.search.highlight.DefaultEncoder; -import org.apache.lucene.store.Directory; -import org.elasticsearch.test.ElasticsearchTestCase; -import org.junit.Test; - -import java.io.BufferedReader; -import java.io.IOException; -import java.io.InputStreamReader; -import java.text.BreakIterator; -import java.util.Arrays; -import java.util.Iterator; -import java.util.Map; - -import static org.hamcrest.CoreMatchers.*; - -public class XPostingsHighlighterTests extends ElasticsearchTestCase { - - /* - Tests changes needed to make possible to perform discrete highlighting. - We want to highlight every field value separately in case of multiple values, at least when needing to return the whole field content - This is needed to be able to get back a single snippet per value when number_of_fragments=0 - */ - @Test - public void testDiscreteHighlightingPerValue() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random())); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Field body = new Field("body", "", offsetsType); - final String firstValue = "This is a test. Just a test highlighting from postings highlighter."; - Document doc = new Document(); - doc.add(body); - body.setStringValue(firstValue); - - final String secondValue = "This is the second value to perform highlighting on."; - Field body2 = new Field("body", "", offsetsType); - doc.add(body2); - body2.setStringValue(secondValue); - - final String thirdValue = "This is the third value to test highlighting with postings."; - Field body3 = new Field("body", "", offsetsType); - doc.add(body3); - body3.setStringValue(thirdValue); - - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - XPostingsHighlighter highlighter = new XPostingsHighlighter() { - @Override - protected BreakIterator getBreakIterator(String field) { - return new WholeBreakIterator(); - } - - @Override - protected char getMultiValuedSeparator(String field) { - //U+2029 PARAGRAPH SEPARATOR (PS): each value holds a discrete passage for highlighting - return 8233; - } - }; - Query query = new TermQuery(new Term("body", "highlighting")); - TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER); - assertThat(topDocs.totalHits, equalTo(1)); - String snippets[] = highlighter.highlight("body", query, searcher, topDocs); - assertThat(snippets.length, equalTo(1)); - - String firstHlValue = "This is a test. Just a test highlighting from postings highlighter."; - String secondHlValue = "This is the second value to perform highlighting on."; - String thirdHlValue = "This is the third value to test highlighting with postings."; - - //default behaviour: using the WholeBreakIterator, despite the multi valued paragraph separator we get back a single snippet for multiple values - assertThat(snippets[0], equalTo(firstHlValue + (char)8233 + secondHlValue + (char)8233 + thirdHlValue)); - - - - highlighter = new XPostingsHighlighter() { - Iterator valuesIterator = Arrays.asList(firstValue, secondValue, thirdValue).iterator(); - Iterator offsetsIterator = Arrays.asList(0, firstValue.length() + 1, firstValue.length() + secondValue.length() + 2).iterator(); - - @Override - protected String[][] loadFieldValues(IndexSearcher searcher, String[] fields, int[] docids, int maxLength) throws IOException { - return new String[][]{new String[]{valuesIterator.next()}}; - } - - @Override - protected int getOffsetForCurrentValue(String field, int docId) { - return offsetsIterator.next(); - } - - @Override - protected BreakIterator getBreakIterator(String field) { - return new WholeBreakIterator(); - } - }; - - //first call using the WholeBreakIterator, we get now only the first value properly highlighted as we wish - snippets = highlighter.highlight("body", query, searcher, topDocs); - assertThat(snippets.length, equalTo(1)); - assertThat(snippets[0], equalTo(firstHlValue)); - - //second call using the WholeBreakIterator, we get now only the second value properly highlighted as we wish - snippets = highlighter.highlight("body", query, searcher, topDocs); - assertThat(snippets.length, equalTo(1)); - assertThat(snippets[0], equalTo(secondHlValue)); - - //third call using the WholeBreakIterator, we get now only the third value properly highlighted as we wish - snippets = highlighter.highlight("body", query, searcher, topDocs); - assertThat(snippets.length, equalTo(1)); - assertThat(snippets[0], equalTo(thirdHlValue)); - - ir.close(); - dir.close(); - } - - @Test - public void testDiscreteHighlightingPerValue_secondValueWithoutMatches() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random())); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Field body = new Field("body", "", offsetsType); - final String firstValue = "This is a test. Just a test highlighting from postings highlighter."; - Document doc = new Document(); - doc.add(body); - body.setStringValue(firstValue); - - final String secondValue = "This is the second value without matches."; - Field body2 = new Field("body", "", offsetsType); - doc.add(body2); - body2.setStringValue(secondValue); - - final String thirdValue = "This is the third value to test highlighting with postings."; - Field body3 = new Field("body", "", offsetsType); - doc.add(body3); - body3.setStringValue(thirdValue); - - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - - Query query = new TermQuery(new Term("body", "highlighting")); - TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER); - assertThat(topDocs.totalHits, equalTo(1)); - - XPostingsHighlighter highlighter = new XPostingsHighlighter() { - @Override - protected BreakIterator getBreakIterator(String field) { - return new WholeBreakIterator(); - } - - @Override - protected char getMultiValuedSeparator(String field) { - //U+2029 PARAGRAPH SEPARATOR (PS): each value holds a discrete passage for highlighting - return 8233; - } - - @Override - protected Passage[] getEmptyHighlight(String fieldName, BreakIterator bi, int maxPassages) { - return new Passage[0]; - } - }; - String snippets[] = highlighter.highlight("body", query, searcher, topDocs); - assertThat(snippets.length, equalTo(1)); - String firstHlValue = "This is a test. Just a test highlighting from postings highlighter."; - String thirdHlValue = "This is the third value to test highlighting with postings."; - //default behaviour: using the WholeBreakIterator, despite the multi valued paragraph separator we get back a single snippet for multiple values - //but only the first and the third value are returned since there are no matches in the second one. - assertThat(snippets[0], equalTo(firstHlValue + (char)8233 + secondValue + (char)8233 + thirdHlValue)); - - - highlighter = new XPostingsHighlighter() { - Iterator valuesIterator = Arrays.asList(firstValue, secondValue, thirdValue).iterator(); - Iterator offsetsIterator = Arrays.asList(0, firstValue.length() + 1, firstValue.length() + secondValue.length() + 2).iterator(); - - @Override - protected String[][] loadFieldValues(IndexSearcher searcher, String[] fields, int[] docids, int maxLength) throws IOException { - return new String[][]{new String[]{valuesIterator.next()}}; - } - - @Override - protected int getOffsetForCurrentValue(String field, int docId) { - return offsetsIterator.next(); - } - - @Override - protected BreakIterator getBreakIterator(String field) { - return new WholeBreakIterator(); - } - - @Override - protected Passage[] getEmptyHighlight(String fieldName, BreakIterator bi, int maxPassages) { - return new Passage[0]; - } - }; - - //first call using the WholeBreakIterator, we get now only the first value properly highlighted as we wish - snippets = highlighter.highlight("body", query, searcher, topDocs); - assertThat(snippets.length, equalTo(1)); - assertThat(snippets[0], equalTo(firstHlValue)); - - //second call using the WholeBreakIterator, we get now nothing back because there's nothing to highlight in the second value - snippets = highlighter.highlight("body", query, searcher, topDocs); - assertThat(snippets.length, equalTo(1)); - assertThat(snippets[0], nullValue()); - - //third call using the WholeBreakIterator, we get now only the third value properly highlighted as we wish - snippets = highlighter.highlight("body", query, searcher, topDocs); - assertThat(snippets.length, equalTo(1)); - assertThat(snippets[0], equalTo(thirdHlValue)); - - ir.close(); - dir.close(); - } - - @Test - public void testDiscreteHighlightingPerValue_MultipleMatches() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random())); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Field body = new Field("body", "", offsetsType); - final String firstValue = "This is a highlighting test. Just a test highlighting from postings highlighter."; - Document doc = new Document(); - doc.add(body); - body.setStringValue(firstValue); - - final String secondValue = "This is the second highlighting value to test highlighting with postings."; - Field body2 = new Field("body", "", offsetsType); - doc.add(body2); - body2.setStringValue(secondValue); - - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - - Query query = new TermQuery(new Term("body", "highlighting")); - TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER); - assertThat(topDocs.totalHits, equalTo(1)); - - String firstHlValue = "This is a highlighting test. Just a test highlighting from postings highlighter."; - String secondHlValue = "This is the second highlighting value to test highlighting with postings."; - - XPostingsHighlighter highlighter = new XPostingsHighlighter() { - Iterator valuesIterator = Arrays.asList(firstValue, secondValue).iterator(); - Iterator offsetsIterator = Arrays.asList(0, firstValue.length() + 1).iterator(); - - @Override - protected String[][] loadFieldValues(IndexSearcher searcher, String[] fields, int[] docids, int maxLength) throws IOException { - return new String[][]{new String[]{valuesIterator.next()}}; - } - - @Override - protected int getOffsetForCurrentValue(String field, int docId) { - return offsetsIterator.next(); - } - - @Override - protected BreakIterator getBreakIterator(String field) { - return new WholeBreakIterator(); - } - - @Override - protected Passage[] getEmptyHighlight(String fieldName, BreakIterator bi, int maxPassages) { - return new Passage[0]; - } - }; - - //first call using the WholeBreakIterator, we get now only the first value properly highlighted as we wish - String[] snippets = highlighter.highlight("body", query, searcher, topDocs); - assertThat(snippets.length, equalTo(1)); - assertThat(snippets[0], equalTo(firstHlValue)); - - //second call using the WholeBreakIterator, we get now only the second value properly highlighted as we wish - snippets = highlighter.highlight("body", query, searcher, topDocs); - assertThat(snippets.length, equalTo(1)); - assertThat(snippets[0], equalTo(secondHlValue)); - - ir.close(); - dir.close(); - } - - @Test - public void testDiscreteHighlightingPerValue_MultipleQueryTerms() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random())); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Field body = new Field("body", "", offsetsType); - final String firstValue = "This is the first sentence. This is the second sentence."; - Document doc = new Document(); - doc.add(body); - body.setStringValue(firstValue); - - final String secondValue = "This is the third sentence. This is the fourth sentence."; - Field body2 = new Field("body", "", offsetsType); - doc.add(body2); - body2.setStringValue(secondValue); - - final String thirdValue = "This is the fifth sentence"; - Field body3 = new Field("body", "", offsetsType); - doc.add(body3); - body3.setStringValue(thirdValue); - - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - - BooleanQuery query = new BooleanQuery(); - query.add(new BooleanClause(new TermQuery(new Term("body", "third")), BooleanClause.Occur.SHOULD)); - query.add(new BooleanClause(new TermQuery(new Term("body", "seventh")), BooleanClause.Occur.SHOULD)); - query.add(new BooleanClause(new TermQuery(new Term("body", "fifth")), BooleanClause.Occur.SHOULD)); - query.setMinimumNumberShouldMatch(1); - - TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER); - assertThat(topDocs.totalHits, equalTo(1)); - - String secondHlValue = "This is the third sentence. This is the fourth sentence."; - String thirdHlValue = "This is the fifth sentence"; - - XPostingsHighlighter highlighter = new XPostingsHighlighter() { - Iterator valuesIterator = Arrays.asList(firstValue, secondValue, thirdValue).iterator(); - Iterator offsetsIterator = Arrays.asList(0, firstValue.length() + 1, secondValue.length() + 1).iterator(); - - @Override - protected String[][] loadFieldValues(IndexSearcher searcher, String[] fields, int[] docids, int maxLength) throws IOException { - return new String[][]{new String[]{valuesIterator.next()}}; - } - - @Override - protected int getOffsetForCurrentValue(String field, int docId) { - return offsetsIterator.next(); - } - - @Override - protected BreakIterator getBreakIterator(String field) { - return new WholeBreakIterator(); - } - - @Override - protected Passage[] getEmptyHighlight(String fieldName, BreakIterator bi, int maxPassages) { - return new Passage[0]; - } - }; - - //first call using the WholeBreakIterator, we get now null as the first value doesn't hold any match - String[] snippets = highlighter.highlight("body", query, searcher, topDocs); - assertThat(snippets.length, equalTo(1)); - assertThat(snippets[0], nullValue()); - - //second call using the WholeBreakIterator, we get now only the second value properly highlighted as we wish - snippets = highlighter.highlight("body", query, searcher, topDocs); - assertThat(snippets.length, equalTo(1)); - assertThat(snippets[0], equalTo(secondHlValue)); - - //second call using the WholeBreakIterator, we get now only the third value properly highlighted as we wish - snippets = highlighter.highlight("body", query, searcher, topDocs); - assertThat(snippets.length, equalTo(1)); - assertThat(snippets[0], equalTo(thirdHlValue)); - - ir.close(); - dir.close(); - } - - /* - The following are tests that we added to make sure that certain behaviours are possible using the postings highlighter - They don't require our forked version, but only custom versions of methods that can be overridden and are already exposed to subclasses - */ - - /* - Tests that it's possible to obtain different fragments per document instead of a big string of concatenated fragments. - We use our own PassageFormatter for that and override the getFormatter method. - */ - @Test - public void testCustomPassageFormatterMultipleFragments() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random())); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Field body = new Field("body", "", offsetsType); - Document doc = new Document(); - doc.add(body); - body.setStringValue("This test is another test. Not a good sentence. Test test test test."); - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - XPostingsHighlighter highlighter = new XPostingsHighlighter(); - IndexSearcher searcher = newSearcher(ir); - Query query = new TermQuery(new Term("body", "test")); - TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER); - assertThat(topDocs.totalHits, equalTo(1)); - String snippets[] = highlighter.highlight("body", query, searcher, topDocs, 5); - assertThat(snippets.length, equalTo(1)); - //default behaviour that we want to change - assertThat(snippets[0], equalTo("This test is another test. ... Test test test test.")); - - - final CustomPassageFormatter passageFormatter = new CustomPassageFormatter("", "", new DefaultEncoder()); - highlighter = new XPostingsHighlighter() { - @Override - protected PassageFormatter getFormatter(String field) { - return passageFormatter; - } - }; - - final ScoreDoc scoreDocs[] = topDocs.scoreDocs; - int docids[] = new int[scoreDocs.length]; - int maxPassages[] = new int[scoreDocs.length]; - for (int i = 0; i < docids.length; i++) { - docids[i] = scoreDocs[i].doc; - maxPassages[i] = 5; - } - Map highlights = highlighter.highlightFieldsAsObjects(new String[]{"body"}, query, searcher, docids, maxPassages); - assertThat(highlights, notNullValue()); - assertThat(highlights.size(), equalTo(1)); - Object[] objectSnippets = highlights.get("body"); - assertThat(objectSnippets, notNullValue()); - assertThat(objectSnippets.length, equalTo(1)); - assertThat(objectSnippets[0], instanceOf(Snippet[].class)); - - Snippet[] snippetsSnippet = (Snippet[]) objectSnippets[0]; - assertThat(snippetsSnippet.length, equalTo(2)); - //multiple fragments as we wish - assertThat(snippetsSnippet[0].getText(), equalTo("This test is another test.")); - assertThat(snippetsSnippet[1].getText(), equalTo("Test test test test.")); - - ir.close(); - dir.close(); - } - - /* - Tests that it's possible to return no fragments when there's nothing to highlight - We do that by overriding the getEmptyHighlight method - */ - @Test - public void testHighlightWithNoMatches() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random())); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Field body = new Field("body", "", offsetsType); - Field none = new Field("none", "", offsetsType); - Document doc = new Document(); - doc.add(body); - doc.add(none); - - body.setStringValue("This is a test. Just a test highlighting from postings. Feel free to ignore."); - none.setStringValue(body.stringValue()); - iw.addDocument(doc); - body.setStringValue("Highlighting the first term. Hope it works."); - none.setStringValue(body.stringValue()); - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - XPostingsHighlighter highlighter = new XPostingsHighlighter(); - Query query = new TermQuery(new Term("none", "highlighting")); - TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER); - assertThat(topDocs.totalHits, equalTo(2)); - String snippets[] = highlighter.highlight("body", query, searcher, topDocs, 1); - //Two null snippets if there are no matches (thanks to our own custom passage formatter) - assertThat(snippets.length, equalTo(2)); - //default behaviour: returns the first sentence with num passages = 1 - assertThat(snippets[0], equalTo("This is a test. ")); - assertThat(snippets[1], equalTo("Highlighting the first term. ")); - - highlighter = new XPostingsHighlighter() { - @Override - protected Passage[] getEmptyHighlight(String fieldName, BreakIterator bi, int maxPassages) { - return new Passage[0]; - } - }; - snippets = highlighter.highlight("body", query, searcher, topDocs); - //Two null snippets if there are no matches, as we wish - assertThat(snippets.length, equalTo(2)); - assertThat(snippets[0], nullValue()); - assertThat(snippets[1], nullValue()); - - ir.close(); - dir.close(); - } - - /* - Tests that it's possible to avoid having fragments that span across different values - We do that by overriding the getMultiValuedSeparator and using a proper separator between values - */ - @Test - public void testCustomMultiValuedSeparator() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random())); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Field body = new Field("body", "", offsetsType); - Document doc = new Document(); - doc.add(body); - - body.setStringValue("This is a test. Just a test highlighting from postings"); - - Field body2 = new Field("body", "", offsetsType); - doc.add(body2); - body2.setStringValue("highlighter."); - iw.addDocument(doc); - - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - XPostingsHighlighter highlighter = new XPostingsHighlighter(); - Query query = new TermQuery(new Term("body", "highlighting")); - TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER); - assertThat(topDocs.totalHits, equalTo(1)); - String snippets[] = highlighter.highlight("body", query, searcher, topDocs); - assertThat(snippets.length, equalTo(1)); - //default behaviour: getting a fragment that spans across different values - assertThat(snippets[0], equalTo("Just a test highlighting from postings highlighter.")); - - - highlighter = new XPostingsHighlighter() { - @Override - protected char getMultiValuedSeparator(String field) { - //U+2029 PARAGRAPH SEPARATOR (PS): each value holds a discrete passage for highlighting - return 8233; - } - }; - snippets = highlighter.highlight("body", query, searcher, topDocs); - assertThat(snippets.length, equalTo(1)); - //getting a fragment that doesn't span across different values since we used the paragraph separator between the different values - assertThat(snippets[0], equalTo("Just a test highlighting from postings" + (char)8233)); - - ir.close(); - dir.close(); - } - - - - - /* - The following are all the existing postings highlighter tests, to make sure we don't have regression in our own fork - */ - - @Test - public void testBasics() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random())); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Field body = new Field("body", "", offsetsType); - Document doc = new Document(); - doc.add(body); - - body.setStringValue("This is a test. Just a test highlighting from postings. Feel free to ignore."); - iw.addDocument(doc); - body.setStringValue("Highlighting the first term. Hope it works."); - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - XPostingsHighlighter highlighter = new XPostingsHighlighter(); - Query query = new TermQuery(new Term("body", "highlighting")); - TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER); - assertEquals(2, topDocs.totalHits); - String snippets[] = highlighter.highlight("body", query, searcher, topDocs); - assertEquals(2, snippets.length); - assertEquals("Just a test highlighting from postings. ", snippets[0]); - assertEquals("Highlighting the first term. ", snippets[1]); - - ir.close(); - dir.close(); - } - - public void testFormatWithMatchExceedingContentLength2() throws Exception { - - String bodyText = "123 TEST 01234 TEST"; - - String[] snippets = formatWithMatchExceedingContentLength(bodyText); - - assertEquals(1, snippets.length); - assertEquals("123 TEST 01234 TE", snippets[0]); - } - - public void testFormatWithMatchExceedingContentLength3() throws Exception { - - String bodyText = "123 5678 01234 TEST TEST"; - - String[] snippets = formatWithMatchExceedingContentLength(bodyText); - - assertEquals(1, snippets.length); - assertEquals("123 5678 01234 TE", snippets[0]); - } - - public void testFormatWithMatchExceedingContentLength() throws Exception { - - String bodyText = "123 5678 01234 TEST"; - - String[] snippets = formatWithMatchExceedingContentLength(bodyText); - - assertEquals(1, snippets.length); - // LUCENE-5166: no snippet - assertEquals("123 5678 01234 TE", snippets[0]); - } - - private String[] formatWithMatchExceedingContentLength(String bodyText) throws IOException { - - int maxLength = 17; - - final Analyzer analyzer = new MockAnalyzer(random()); - - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(analyzer); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - final FieldType fieldType = new FieldType(TextField.TYPE_STORED); - fieldType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - final Field body = new Field("body", bodyText, fieldType); - - Document doc = new Document(); - doc.add(body); - - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - - Query query = new TermQuery(new Term("body", "test")); - - TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER); - assertEquals(1, topDocs.totalHits); - - XPostingsHighlighter highlighter = new XPostingsHighlighter(maxLength); - String snippets[] = highlighter.highlight("body", query, searcher, topDocs); - - - ir.close(); - dir.close(); - return snippets; - } - - // simple test highlighting last word. - public void testHighlightLastWord() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random())); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Field body = new Field("body", "", offsetsType); - Document doc = new Document(); - doc.add(body); - - body.setStringValue("This is a test"); - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - XPostingsHighlighter highlighter = new XPostingsHighlighter(); - Query query = new TermQuery(new Term("body", "test")); - TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER); - assertEquals(1, topDocs.totalHits); - String snippets[] = highlighter.highlight("body", query, searcher, topDocs); - assertEquals(1, snippets.length); - assertEquals("This is a test", snippets[0]); - - ir.close(); - dir.close(); - } - - // simple test with one sentence documents. - @Test - public void testOneSentence() throws Exception { - Directory dir = newDirectory(); - // use simpleanalyzer for more natural tokenization (else "test." is a token) - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.SIMPLE, true)); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Field body = new Field("body", "", offsetsType); - Document doc = new Document(); - doc.add(body); - - body.setStringValue("This is a test."); - iw.addDocument(doc); - body.setStringValue("Test a one sentence document."); - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - XPostingsHighlighter highlighter = new XPostingsHighlighter(); - Query query = new TermQuery(new Term("body", "test")); - TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER); - assertEquals(2, topDocs.totalHits); - String snippets[] = highlighter.highlight("body", query, searcher, topDocs); - assertEquals(2, snippets.length); - assertEquals("This is a test.", snippets[0]); - assertEquals("Test a one sentence document.", snippets[1]); - - ir.close(); - dir.close(); - } - - // simple test with multiple values that make a result longer than maxLength. - @Test - public void testMaxLengthWithMultivalue() throws Exception { - Directory dir = newDirectory(); - // use simpleanalyzer for more natural tokenization (else "test." is a token) - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.SIMPLE, true)); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Document doc = new Document(); - - for(int i = 0; i < 3 ; i++) { - Field body = new Field("body", "", offsetsType); - body.setStringValue("This is a multivalued field"); - doc.add(body); - } - - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - XPostingsHighlighter highlighter = new XPostingsHighlighter(40); - Query query = new TermQuery(new Term("body", "field")); - TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER); - assertEquals(1, topDocs.totalHits); - String snippets[] = highlighter.highlight("body", query, searcher, topDocs); - assertEquals(1, snippets.length); - assertTrue("Snippet should have maximum 40 characters plus the pre and post tags", - snippets[0].length() == (40 + "".length())); - - ir.close(); - dir.close(); - } - - @Test - public void testMultipleFields() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.SIMPLE, true)); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Field body = new Field("body", "", offsetsType); - Field title = new Field("title", "", offsetsType); - Document doc = new Document(); - doc.add(body); - doc.add(title); - - body.setStringValue("This is a test. Just a test highlighting from postings. Feel free to ignore."); - title.setStringValue("I am hoping for the best."); - iw.addDocument(doc); - body.setStringValue("Highlighting the first term. Hope it works."); - title.setStringValue("But best may not be good enough."); - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - XPostingsHighlighter highlighter = new XPostingsHighlighter(); - BooleanQuery query = new BooleanQuery(); - query.add(new TermQuery(new Term("body", "highlighting")), BooleanClause.Occur.SHOULD); - query.add(new TermQuery(new Term("title", "best")), BooleanClause.Occur.SHOULD); - TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER); - assertEquals(2, topDocs.totalHits); - Map snippets = highlighter.highlightFields(new String [] { "body", "title" }, query, searcher, topDocs); - assertEquals(2, snippets.size()); - assertEquals("Just a test highlighting from postings. ", snippets.get("body")[0]); - assertEquals("Highlighting the first term. ", snippets.get("body")[1]); - assertEquals("I am hoping for the best.", snippets.get("title")[0]); - assertEquals("But best may not be good enough.", snippets.get("title")[1]); - ir.close(); - dir.close(); - } - - @Test - public void testMultipleTerms() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random())); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Field body = new Field("body", "", offsetsType); - Document doc = new Document(); - doc.add(body); - - body.setStringValue("This is a test. Just a test highlighting from postings. Feel free to ignore."); - iw.addDocument(doc); - body.setStringValue("Highlighting the first term. Hope it works."); - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - XPostingsHighlighter highlighter = new XPostingsHighlighter(); - BooleanQuery query = new BooleanQuery(); - query.add(new TermQuery(new Term("body", "highlighting")), BooleanClause.Occur.SHOULD); - query.add(new TermQuery(new Term("body", "just")), BooleanClause.Occur.SHOULD); - query.add(new TermQuery(new Term("body", "first")), BooleanClause.Occur.SHOULD); - TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER); - assertEquals(2, topDocs.totalHits); - String snippets[] = highlighter.highlight("body", query, searcher, topDocs); - assertEquals(2, snippets.length); - assertEquals("Just a test highlighting from postings. ", snippets[0]); - assertEquals("Highlighting the first term. ", snippets[1]); - - ir.close(); - dir.close(); - } - - @Test - public void testMultiplePassages() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.SIMPLE, true)); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Field body = new Field("body", "", offsetsType); - Document doc = new Document(); - doc.add(body); - - body.setStringValue("This is a test. Just a test highlighting from postings. Feel free to ignore."); - iw.addDocument(doc); - body.setStringValue("This test is another test. Not a good sentence. Test test test test."); - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - XPostingsHighlighter highlighter = new XPostingsHighlighter(); - Query query = new TermQuery(new Term("body", "test")); - TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER); - assertEquals(2, topDocs.totalHits); - String snippets[] = highlighter.highlight("body", query, searcher, topDocs, 2); - assertEquals(2, snippets.length); - assertEquals("This is a test. Just a test highlighting from postings. ", snippets[0]); - assertEquals("This test is another test. ... Test test test test.", snippets[1]); - - ir.close(); - dir.close(); - } - - @Test - public void testUserFailedToIndexOffsets() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.SIMPLE, true)); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType positionsType = new FieldType(TextField.TYPE_STORED); - positionsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS); - Field body = new Field("body", "", positionsType); - Field title = new StringField("title", "", Field.Store.YES); - Document doc = new Document(); - doc.add(body); - doc.add(title); - - body.setStringValue("This is a test. Just a test highlighting from postings. Feel free to ignore."); - title.setStringValue("test"); - iw.addDocument(doc); - body.setStringValue("This test is another test. Not a good sentence. Test test test test."); - title.setStringValue("test"); - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - XPostingsHighlighter highlighter = new XPostingsHighlighter(); - Query query = new TermQuery(new Term("body", "test")); - TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER); - assertEquals(2, topDocs.totalHits); - try { - highlighter.highlight("body", query, searcher, topDocs, 2); - fail("did not hit expected exception"); - } catch (IllegalArgumentException iae) { - // expected - } - - try { - highlighter.highlight("title", new TermQuery(new Term("title", "test")), searcher, topDocs, 2); - fail("did not hit expected exception"); - } catch (IllegalArgumentException iae) { - // expected - } - ir.close(); - dir.close(); - } - - @Test - public void testBuddhism() throws Exception { - String text = "This eight-volume set brings together seminal papers in Buddhist studies from a vast " + - "range of academic disciplines published over the last forty years. With a new introduction " + - "by the editor, this collection is a unique and unrivalled research resource for both " + - "student and scholar. Coverage includes: - Buddhist origins; early history of Buddhism in " + - "South and Southeast Asia - early Buddhist Schools and Doctrinal History; Theravada Doctrine " + - "- the Origins and nature of Mahayana Buddhism; some Mahayana religious topics - Abhidharma " + - "and Madhyamaka - Yogacara, the Epistemological tradition, and Tathagatagarbha - Tantric " + - "Buddhism (Including China and Japan); Buddhism in Nepal and Tibet - Buddhism in South and " + - "Southeast Asia, and - Buddhism in China, East Asia, and Japan."; - Directory dir = newDirectory(); - Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, analyzer); - - FieldType positionsType = new FieldType(TextField.TYPE_STORED); - positionsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Field body = new Field("body", text, positionsType); - Document document = new Document(); - document.add(body); - iw.addDocument(document); - IndexReader ir = iw.getReader(); - iw.close(); - IndexSearcher searcher = newSearcher(ir); - PhraseQuery query = new PhraseQuery(); - query.add(new Term("body", "buddhist")); - query.add(new Term("body", "origins")); - TopDocs topDocs = searcher.search(query, 10); - assertEquals(1, topDocs.totalHits); - XPostingsHighlighter highlighter = new XPostingsHighlighter(); - String snippets[] = highlighter.highlight("body", query, searcher, topDocs, 2); - assertEquals(1, snippets.length); - assertTrue(snippets[0].contains("Buddhist origins")); - ir.close(); - dir.close(); - } - - @Test - public void testCuriousGeorge() throws Exception { - String text = "It’s the formula for success for preschoolers—Curious George and fire trucks! " + - "Curious George and the Firefighters is a story based on H. A. and Margret Rey’s " + - "popular primate and painted in the original watercolor and charcoal style. " + - "Firefighters are a famously brave lot, but can they withstand a visit from one curious monkey?"; - Directory dir = newDirectory(); - Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, analyzer); - FieldType positionsType = new FieldType(TextField.TYPE_STORED); - positionsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Field body = new Field("body", text, positionsType); - Document document = new Document(); - document.add(body); - iw.addDocument(document); - IndexReader ir = iw.getReader(); - iw.close(); - IndexSearcher searcher = newSearcher(ir); - PhraseQuery query = new PhraseQuery(); - query.add(new Term("body", "curious")); - query.add(new Term("body", "george")); - TopDocs topDocs = searcher.search(query, 10); - assertEquals(1, topDocs.totalHits); - XPostingsHighlighter highlighter = new XPostingsHighlighter(); - String snippets[] = highlighter.highlight("body", query, searcher, topDocs, 2); - assertEquals(1, snippets.length); - assertFalse(snippets[0].contains("CuriousCurious")); - ir.close(); - dir.close(); - } - - @Test - public void testCambridgeMA() throws Exception { - BufferedReader r = new BufferedReader(new InputStreamReader( - this.getClass().getResourceAsStream("CambridgeMA.utf8"), "UTF-8")); - String text = r.readLine(); - r.close(); - Directory dir = newDirectory(); - Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, analyzer); - FieldType positionsType = new FieldType(TextField.TYPE_STORED); - positionsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Field body = new Field("body", text, positionsType); - Document document = new Document(); - document.add(body); - iw.addDocument(document); - IndexReader ir = iw.getReader(); - iw.close(); - IndexSearcher searcher = newSearcher(ir); - BooleanQuery query = new BooleanQuery(); - query.add(new TermQuery(new Term("body", "porter")), BooleanClause.Occur.SHOULD); - query.add(new TermQuery(new Term("body", "square")), BooleanClause.Occur.SHOULD); - query.add(new TermQuery(new Term("body", "massachusetts")), BooleanClause.Occur.SHOULD); - TopDocs topDocs = searcher.search(query, 10); - assertEquals(1, topDocs.totalHits); - XPostingsHighlighter highlighter = new XPostingsHighlighter(Integer.MAX_VALUE-1); - String snippets[] = highlighter.highlight("body", query, searcher, topDocs, 2); - assertEquals(1, snippets.length); - assertTrue(snippets[0].contains("Square")); - assertTrue(snippets[0].contains("Porter")); - ir.close(); - dir.close(); - } - - @Test - public void testPassageRanking() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.SIMPLE, true)); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Field body = new Field("body", "", offsetsType); - Document doc = new Document(); - doc.add(body); - - body.setStringValue("This is a test. Just highlighting from postings. This is also a much sillier test. Feel free to test test test test test test test."); - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - XPostingsHighlighter highlighter = new XPostingsHighlighter(); - Query query = new TermQuery(new Term("body", "test")); - TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER); - assertEquals(1, topDocs.totalHits); - String snippets[] = highlighter.highlight("body", query, searcher, topDocs, 2); - assertEquals(1, snippets.length); - assertEquals("This is a test. ... Feel free to test test test test test test test.", snippets[0]); - - ir.close(); - dir.close(); - } - - @Test - public void testBooleanMustNot() throws Exception { - Directory dir = newDirectory(); - Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, analyzer); - FieldType positionsType = new FieldType(TextField.TYPE_STORED); - positionsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Field body = new Field("body", "This sentence has both terms. This sentence has only terms.", positionsType); - Document document = new Document(); - document.add(body); - iw.addDocument(document); - IndexReader ir = iw.getReader(); - iw.close(); - IndexSearcher searcher = newSearcher(ir); - BooleanQuery query = new BooleanQuery(); - query.add(new TermQuery(new Term("body", "terms")), BooleanClause.Occur.SHOULD); - BooleanQuery query2 = new BooleanQuery(); - query.add(query2, BooleanClause.Occur.SHOULD); - query2.add(new TermQuery(new Term("body", "both")), BooleanClause.Occur.MUST_NOT); - TopDocs topDocs = searcher.search(query, 10); - assertEquals(1, topDocs.totalHits); - XPostingsHighlighter highlighter = new XPostingsHighlighter(Integer.MAX_VALUE-1); - String snippets[] = highlighter.highlight("body", query, searcher, topDocs, 2); - assertEquals(1, snippets.length); - assertFalse(snippets[0].contains("both")); - ir.close(); - dir.close(); - } - - @Test - public void testHighlightAllText() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.SIMPLE, true)); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Field body = new Field("body", "", offsetsType); - Document doc = new Document(); - doc.add(body); - - body.setStringValue("This is a test. Just highlighting from postings. This is also a much sillier test. Feel free to test test test test test test test."); - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - XPostingsHighlighter highlighter = new XPostingsHighlighter(10000) { - @Override - protected BreakIterator getBreakIterator(String field) { - return new WholeBreakIterator(); - } - }; - Query query = new TermQuery(new Term("body", "test")); - TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER); - assertEquals(1, topDocs.totalHits); - String snippets[] = highlighter.highlight("body", query, searcher, topDocs, 2); - assertEquals(1, snippets.length); - assertEquals("This is a test. Just highlighting from postings. This is also a much sillier test. Feel free to test test test test test test test.", snippets[0]); - - ir.close(); - dir.close(); - } - - @Test - public void testSpecificDocIDs() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random())); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Field body = new Field("body", "", offsetsType); - Document doc = new Document(); - doc.add(body); - - body.setStringValue("This is a test. Just a test highlighting from postings. Feel free to ignore."); - iw.addDocument(doc); - body.setStringValue("Highlighting the first term. Hope it works."); - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - XPostingsHighlighter highlighter = new XPostingsHighlighter(); - Query query = new TermQuery(new Term("body", "highlighting")); - TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER); - assertEquals(2, topDocs.totalHits); - ScoreDoc[] hits = topDocs.scoreDocs; - int[] docIDs = new int[2]; - docIDs[0] = hits[0].doc; - docIDs[1] = hits[1].doc; - String snippets[] = highlighter.highlightFields(new String[] {"body"}, query, searcher, docIDs, new int[] { 1 }).get("body"); - assertEquals(2, snippets.length); - assertEquals("Just a test highlighting from postings. ", snippets[0]); - assertEquals("Highlighting the first term. ", snippets[1]); - - ir.close(); - dir.close(); - } - - @Test - public void testCustomFieldValueSource() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.SIMPLE, true)); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - Document doc = new Document(); - - FieldType offsetsType = new FieldType(TextField.TYPE_NOT_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - final String text = "This is a test. Just highlighting from postings. This is also a much sillier test. Feel free to test test test test test test test."; - Field body = new Field("body", text, offsetsType); - doc.add(body); - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - - XPostingsHighlighter highlighter = new XPostingsHighlighter(10000) { - @Override - protected String[][] loadFieldValues(IndexSearcher searcher, String[] fields, int[] docids, int maxLength) throws IOException { - assertThat(fields.length, equalTo(1)); - assertThat(docids.length, equalTo(1)); - String[][] contents = new String[1][1]; - contents[0][0] = text; - return contents; - } - - @Override - protected BreakIterator getBreakIterator(String field) { - return new WholeBreakIterator(); - } - }; - - Query query = new TermQuery(new Term("body", "test")); - TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER); - assertEquals(1, topDocs.totalHits); - String snippets[] = highlighter.highlight("body", query, searcher, topDocs, 2); - assertEquals(1, snippets.length); - assertEquals("This is a test. Just highlighting from postings. This is also a much sillier test. Feel free to test test test test test test test.", snippets[0]); - - ir.close(); - dir.close(); - } - - /** Make sure highlighter returns first N sentences if - * there were no hits. */ - @Test - public void testEmptyHighlights() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random())); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Document doc = new Document(); - - Field body = new Field("body", "test this is. another sentence this test has. far away is that planet.", offsetsType); - doc.add(body); - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - XPostingsHighlighter highlighter = new XPostingsHighlighter(); - Query query = new TermQuery(new Term("body", "highlighting")); - int[] docIDs = new int[] {0}; - String snippets[] = highlighter.highlightFields(new String[] {"body"}, query, searcher, docIDs, new int[] { 2 }).get("body"); - assertEquals(1, snippets.length); - assertEquals("test this is. another sentence this test has. ", snippets[0]); - - ir.close(); - dir.close(); - } - - /** Make sure highlighter we can customize how emtpy - * highlight is returned. */ - @Test - public void testCustomEmptyHighlights() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random())); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Document doc = new Document(); - - Field body = new Field("body", "test this is. another sentence this test has. far away is that planet.", offsetsType); - doc.add(body); - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - XPostingsHighlighter highlighter = new XPostingsHighlighter() { - @Override - public Passage[] getEmptyHighlight(String fieldName, BreakIterator bi, int maxPassages) { - return new Passage[0]; - } - }; - Query query = new TermQuery(new Term("body", "highlighting")); - int[] docIDs = new int[] {0}; - String snippets[] = highlighter.highlightFields(new String[] {"body"}, query, searcher, docIDs, new int[] { 2 }).get("body"); - assertEquals(1, snippets.length); - assertNull(snippets[0]); - - ir.close(); - dir.close(); - } - - /** Make sure highlighter returns whole text when there - * are no hits and BreakIterator is null. */ - @Test - public void testEmptyHighlightsWhole() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random())); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Document doc = new Document(); - - Field body = new Field("body", "test this is. another sentence this test has. far away is that planet.", offsetsType); - doc.add(body); - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - XPostingsHighlighter highlighter = new XPostingsHighlighter(10000) { - @Override - protected BreakIterator getBreakIterator(String field) { - return new WholeBreakIterator(); - } - }; - Query query = new TermQuery(new Term("body", "highlighting")); - int[] docIDs = new int[] {0}; - String snippets[] = highlighter.highlightFields(new String[] {"body"}, query, searcher, docIDs, new int[] { 2 }).get("body"); - assertEquals(1, snippets.length); - assertEquals("test this is. another sentence this test has. far away is that planet.", snippets[0]); - - ir.close(); - dir.close(); - } - - /** Make sure highlighter is OK with entirely missing - * field. */ - @Test - public void testFieldIsMissing() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random())); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Document doc = new Document(); - - Field body = new Field("body", "test this is. another sentence this test has. far away is that planet.", offsetsType); - doc.add(body); - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - XPostingsHighlighter highlighter = new XPostingsHighlighter(); - Query query = new TermQuery(new Term("bogus", "highlighting")); - int[] docIDs = new int[] {0}; - String snippets[] = highlighter.highlightFields(new String[] {"bogus"}, query, searcher, docIDs, new int[] { 2 }).get("bogus"); - assertEquals(1, snippets.length); - assertNull(snippets[0]); - - ir.close(); - dir.close(); - } - - @Test - public void testFieldIsJustSpace() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random())); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - - Document doc = new Document(); - doc.add(new Field("body", " ", offsetsType)); - doc.add(new Field("id", "id", offsetsType)); - iw.addDocument(doc); - - doc = new Document(); - doc.add(new Field("body", "something", offsetsType)); - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - XPostingsHighlighter highlighter = new XPostingsHighlighter(); - int docID = searcher.search(new TermQuery(new Term("id", "id")), 1).scoreDocs[0].doc; - - Query query = new TermQuery(new Term("body", "highlighting")); - int[] docIDs = new int[1]; - docIDs[0] = docID; - String snippets[] = highlighter.highlightFields(new String[] {"body"}, query, searcher, docIDs, new int[] { 2 }).get("body"); - assertEquals(1, snippets.length); - assertEquals(" ", snippets[0]); - - ir.close(); - dir.close(); - } - - @Test - public void testFieldIsEmptyString() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random())); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - - Document doc = new Document(); - doc.add(new Field("body", "", offsetsType)); - doc.add(new Field("id", "id", offsetsType)); - iw.addDocument(doc); - - doc = new Document(); - doc.add(new Field("body", "something", offsetsType)); - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - XPostingsHighlighter highlighter = new XPostingsHighlighter(); - int docID = searcher.search(new TermQuery(new Term("id", "id")), 1).scoreDocs[0].doc; - - Query query = new TermQuery(new Term("body", "highlighting")); - int[] docIDs = new int[1]; - docIDs[0] = docID; - String snippets[] = highlighter.highlightFields(new String[] {"body"}, query, searcher, docIDs, new int[] { 2 }).get("body"); - assertEquals(1, snippets.length); - assertNull(snippets[0]); - - ir.close(); - dir.close(); - } - - @Test - public void testMultipleDocs() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random())); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - - int numDocs = scaledRandomIntBetween(100, 1000); - for(int i=0;i snippets = highlighter.highlightFields(new String[] { "title", "body" }, query, searcher, new int[] { 0 }, new int[] { 1, 2 }); - String titleHighlight = snippets.get("title")[0]; - String bodyHighlight = snippets.get("body")[0]; - assertEquals("This is a test. ", titleHighlight); - assertEquals("This is a test. Just a test highlighting from postings. ", bodyHighlight); - ir.close(); - dir.close(); - } - - public void testEncode() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random())); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Field body = new Field("body", "", offsetsType); - Document doc = new Document(); - doc.add(body); - - body.setStringValue("This is a test. Just a test highlighting from postings. Feel free to ignore."); - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - PostingsHighlighter highlighter = new PostingsHighlighter() { - @Override - protected PassageFormatter getFormatter(String field) { - return new DefaultPassageFormatter("", "", "... ", true); - } - }; - Query query = new TermQuery(new Term("body", "highlighting")); - TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER); - assertEquals(1, topDocs.totalHits); - String snippets[] = highlighter.highlight("body", query, searcher, topDocs); - assertEquals(1, snippets.length); - assertEquals("Just a test highlighting from <i>postings</i>. ", snippets[0]); - - ir.close(); - dir.close(); - } - - /** customizing the gap separator to force a sentence break */ - public void testGapSeparator() throws Exception { - Directory dir = newDirectory(); - // use simpleanalyzer for more natural tokenization (else "test." is a token) - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.SIMPLE, true)); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Document doc = new Document(); - - Field body1 = new Field("body", "", offsetsType); - body1.setStringValue("This is a multivalued field"); - doc.add(body1); - - Field body2 = new Field("body", "", offsetsType); - body2.setStringValue("This is something different"); - doc.add(body2); - - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - PostingsHighlighter highlighter = new PostingsHighlighter() { - @Override - protected char getMultiValuedSeparator(String field) { - assert field.equals("body"); - return '\u2029'; - } - }; - Query query = new TermQuery(new Term("body", "field")); - TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER); - assertEquals(1, topDocs.totalHits); - String snippets[] = highlighter.highlight("body", query, searcher, topDocs); - assertEquals(1, snippets.length); - assertEquals("This is a multivalued field\u2029", snippets[0]); - - ir.close(); - dir.close(); - } - - // LUCENE-4906 - public void testObjectFormatter() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random())); - iwc.setMergePolicy(newLogMergePolicy()); - RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); - - FieldType offsetsType = new FieldType(TextField.TYPE_STORED); - offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); - Field body = new Field("body", "", offsetsType); - Document doc = new Document(); - doc.add(body); - - body.setStringValue("This is a test. Just a test highlighting from postings. Feel free to ignore."); - iw.addDocument(doc); - - IndexReader ir = iw.getReader(); - iw.close(); - - IndexSearcher searcher = newSearcher(ir); - XPostingsHighlighter highlighter = new XPostingsHighlighter() { - @Override - protected PassageFormatter getFormatter(String field) { - return new PassageFormatter() { - PassageFormatter defaultFormatter = new DefaultPassageFormatter(); - - @Override - public String[] format(Passage passages[], String content) { - // Just turns the String snippet into a length 2 - // array of String - return new String[] {"blah blah", defaultFormatter.format(passages, content).toString()}; - } - }; - } - }; - - Query query = new TermQuery(new Term("body", "highlighting")); - TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER); - assertEquals(1, topDocs.totalHits); - int[] docIDs = new int[1]; - docIDs[0] = topDocs.scoreDocs[0].doc; - Map snippets = highlighter.highlightFieldsAsObjects(new String[]{"body"}, query, searcher, docIDs, new int[] {1}); - Object[] bodySnippets = snippets.get("body"); - assertEquals(1, bodySnippets.length); - assertTrue(Arrays.equals(new String[] {"blah blah", "Just a test highlighting from postings. "}, (String[]) bodySnippets[0])); - - ir.close(); - dir.close(); - } -} diff --git a/src/test/java/org/elasticsearch/search/highlight/HighlighterSearchTests.java b/src/test/java/org/elasticsearch/search/highlight/HighlighterSearchTests.java index 4691cc9f76d..89c0b2dc4a7 100644 --- a/src/test/java/org/elasticsearch/search/highlight/HighlighterSearchTests.java +++ b/src/test/java/org/elasticsearch/search/highlight/HighlighterSearchTests.java @@ -21,7 +21,6 @@ package org.elasticsearch.search.highlight; import com.carrotsearch.randomizedtesting.generators.RandomPicks; import com.google.common.base.Joiner; import com.google.common.collect.Iterables; - import org.apache.lucene.util.LuceneTestCase.Slow; import org.elasticsearch.Version; import org.elasticsearch.action.index.IndexRequestBuilder; @@ -31,15 +30,9 @@ import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.settings.ImmutableSettings.Builder; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; -import org.elasticsearch.index.query.BoostableQueryBuilder; -import org.elasticsearch.index.query.QueryBuilders; -import org.elasticsearch.index.query.IdsQueryBuilder; -import org.elasticsearch.index.query.MatchQueryBuilder; +import org.elasticsearch.index.query.*; import org.elasticsearch.index.query.MatchQueryBuilder.Operator; import org.elasticsearch.index.query.MatchQueryBuilder.Type; -import org.elasticsearch.index.query.MultiMatchQueryBuilder; -import org.elasticsearch.index.query.QueryBuilder; -import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.builder.SearchSourceBuilder; @@ -56,40 +49,12 @@ import java.util.Map; import static org.elasticsearch.client.Requests.searchRequest; import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; -import static org.elasticsearch.index.query.QueryBuilders.missingQuery; -import static org.elasticsearch.index.query.QueryBuilders.typeQuery; -import static org.elasticsearch.index.query.QueryBuilders.boolQuery; -import static org.elasticsearch.index.query.QueryBuilders.boostingQuery; -import static org.elasticsearch.index.query.QueryBuilders.commonTermsQuery; -import static org.elasticsearch.index.query.QueryBuilders.constantScoreQuery; -import static org.elasticsearch.index.query.QueryBuilders.filteredQuery; -import static org.elasticsearch.index.query.QueryBuilders.fuzzyQuery; -import static org.elasticsearch.index.query.QueryBuilders.matchPhrasePrefixQuery; -import static org.elasticsearch.index.query.QueryBuilders.matchPhraseQuery; -import static org.elasticsearch.index.query.QueryBuilders.matchQuery; -import static org.elasticsearch.index.query.QueryBuilders.multiMatchQuery; -import static org.elasticsearch.index.query.QueryBuilders.prefixQuery; -import static org.elasticsearch.index.query.QueryBuilders.queryStringQuery; -import static org.elasticsearch.index.query.QueryBuilders.rangeQuery; -import static org.elasticsearch.index.query.QueryBuilders.regexpQuery; -import static org.elasticsearch.index.query.QueryBuilders.termQuery; -import static org.elasticsearch.index.query.QueryBuilders.wildcardQuery; +import static org.elasticsearch.index.query.QueryBuilders.*; import static org.elasticsearch.search.builder.SearchSourceBuilder.highlight; import static org.elasticsearch.search.builder.SearchSourceBuilder.searchSource; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertFailures; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHighlight; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNotHighlighted; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.*; import static org.elasticsearch.test.hamcrest.RegexMatcher.matches; -import static org.hamcrest.Matchers.anyOf; -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.hasKey; -import static org.hamcrest.Matchers.not; -import static org.hamcrest.Matchers.nullValue; -import static org.hamcrest.Matchers.startsWith; +import static org.hamcrest.Matchers.*; @Slow public class HighlighterSearchTests extends ElasticsearchIntegrationTest { @@ -496,10 +461,10 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { .addHighlightedField("titleTV", -1, 2) .get(); - assertHighlight(search, 0, "title", 0, equalTo("This is a test on the highlighting bug present in elasticsearch")); + assertHighlight(search, 0, "title", 0, equalTo("This is a test on the highlighting bug present in elasticsearch")); assertHighlight(search, 0, "title", 1, 2, equalTo("The bug is bugging us")); - assertHighlight(search, 0, "titleTV", 0, equalTo("This is a test on the highlighting bug present in elasticsearch")); - assertHighlight(search, 0, "titleTV", 1, 2, equalTo("The bug is bugging us")); + assertHighlight(search, 0, "titleTV", 0, equalTo("This is a test on the highlighting bug present in elasticsearch")); + assertHighlight(search, 0, "titleTV", 1, 2, equalTo("The bug is bugging us")); search = client().prepareSearch() .setQuery(matchQuery("titleTV", "highlight")) @@ -551,8 +516,9 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { logger.info("--> highlighting and searching on field*"); SearchSourceBuilder source = searchSource() - .query(termQuery("field-plain", "test")) - .highlight(highlight().field("field*").preTags("").postTags("")); + //postings hl doesn't support require_field_match, its field needs to be queried directly + .query(termQuery("field-postings", "test")) + .highlight(highlight().field("field*").preTags("").postTags("").requireFieldMatch(false)); SearchResponse searchResponse = client().search(searchRequest("test").source(source)).actionGet(); @@ -586,14 +552,14 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { assertHighlight(searchResponse, 0, "field1", 0, 1, equalTo("The quick brown fox jumps over the lazy dog")); assertFailures(client().prepareSearch("test") - .setQuery(termQuery("field1", "quick")) - .addHighlightedField(new Field("field1").preTags("").postTags("").highlighterType("plain").forceSource(true)), + .setQuery(termQuery("field1", "quick")) + .addHighlightedField(new Field("field1").preTags("").postTags("").highlighterType("plain").forceSource(true)), RestStatus.BAD_REQUEST, containsString("source is forced for fields [field1] but type [type1] has disabled _source")); assertFailures(client().prepareSearch("test") - .setQuery(termQuery("field1", "quick")) - .addHighlightedField(new Field("field1").preTags("").postTags("").highlighterType("fvh").forceSource(true)), + .setQuery(termQuery("field1", "quick")) + .addHighlightedField(new Field("field1").preTags("").postTags("").highlighterType("fvh").forceSource(true)), RestStatus.BAD_REQUEST, containsString("source is forced for fields [field1] but type [type1] has disabled _source")); @@ -1450,11 +1416,11 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { assertHighlight(response, 0, "tags", 1, 2, equalTo("here is another one that is very long tag and has the tag token near the end")); assertFailures(client().prepareSearch("test") - .setQuery(QueryBuilders.matchQuery("tags", "long tag").type(MatchQueryBuilder.Type.PHRASE)) - .addHighlightedField(new HighlightBuilder.Field("tags") - .fragmentSize(-1).numOfFragments(2).fragmenter("invalid")), - RestStatus.BAD_REQUEST, - containsString("unknown fragmenter option [invalid] for the field [tags]")); + .setQuery(QueryBuilders.matchQuery("tags", "long tag").type(MatchQueryBuilder.Type.PHRASE)) + .addHighlightedField(new HighlightBuilder.Field("tags") + .fragmentSize(-1).numOfFragments(2).fragmenter("invalid")), + RestStatus.BAD_REQUEST, + containsString("unknown fragmenter option [invalid] for the field [tags]")); } @Test @@ -1852,7 +1818,7 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { .addMapping("type1", "text", "type=string," + randomStoreField() + "term_vector=with_positions_offsets,index_options=offsets")); ensureGreen(); - String text1 = "This is the first sentence. This is the second sentence."; + String text1 = "This is the first sentence. This is the second sentence." + HighlightUtils.PARAGRAPH_SEPARATOR; String text2 = "This is the third sentence. This is the fourth sentence."; String text3 = "This is the fifth sentence"; index("test", "type1", "1", "text", new String[] {text1, text2, text3}); @@ -1913,7 +1879,7 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { logger.info("--> searching on _all, highlighting on field1"); source = searchSource() - .query(termQuery("_all", "test")) + .query(termQuery("field1", "test")) .highlight(highlight().field("field1").preTags("").postTags("")); searchResponse = client().search(searchRequest("test").source(source)).actionGet(); @@ -1922,7 +1888,7 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { logger.info("--> searching on _all, highlighting on field2"); source = searchSource() - .query(termQuery("_all", "quick")) + .query(termQuery("field2", "quick")) .highlight(highlight().field("field2").order("score").preTags("").postTags("")); searchResponse = client().search(searchRequest("test").source(source)).actionGet(); @@ -1931,7 +1897,7 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { logger.info("--> searching on _all, highlighting on field2"); source = searchSource() - .query(matchPhraseQuery("_all", "quick brown")) + .query(matchPhraseQuery("field2", "quick brown")) .highlight(highlight().field("field2").preTags("").postTags("")); searchResponse = client().search(searchRequest("test").source(source)).actionGet(); @@ -1942,7 +1908,7 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { //lets fall back to the standard highlighter then, what people would do to highlight query matches logger.info("--> searching on _all, highlighting on field2, falling back to the plain highlighter"); source = searchSource() - .query(matchPhraseQuery("_all", "quick brown")) + .query(matchPhraseQuery("field2", "quick brown")) .highlight(highlight().field("field2").preTags("").postTags("").highlighterType("highlighter")); searchResponse = client().search(searchRequest("test").source(source)).actionGet(); @@ -1961,10 +1927,8 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { SearchResponse response = client().prepareSearch("test") .setQuery(QueryBuilders.matchQuery("field1", "fox")) .addHighlightedField(new HighlightBuilder.Field("field1").preTags("<1>").postTags("").requireFieldMatch(true)) - .addHighlightedField(new HighlightBuilder.Field("field2").preTags("<2>").postTags("").requireFieldMatch(false)) .get(); assertHighlight(response, 0, "field1", 0, 1, equalTo("The quick brown <1>fox.")); - assertHighlight(response, 0, "field2", 0, 1, equalTo("The slow brown <2>fox.")); } @Test @@ -1981,8 +1945,7 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { SearchSourceBuilder source = searchSource() .query(termQuery("field1", "fox")) .highlight(highlight() - .field(new HighlightBuilder.Field("field1").numOfFragments(5).preTags("").postTags("")) - .field(new HighlightBuilder.Field("field2").numOfFragments(2).preTags("").postTags(""))); + .field(new HighlightBuilder.Field("field1").numOfFragments(5).preTags("").postTags(""))); SearchResponse searchResponse = client().search(searchRequest("test").source(source)).actionGet(); @@ -1990,9 +1953,6 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { assertHighlight(searchResponse, 0, "field1", 1, equalTo("The lazy red fox jumps over the quick dog.")); assertHighlight(searchResponse, 0, "field1", 2, 3, equalTo("The quick brown dog jumps over the lazy fox.")); - assertHighlight(searchResponse, 0, "field2", 0, equalTo("The quick brown fox jumps over the lazy dog.")); - assertHighlight(searchResponse, 0, "field2", 1, 2, equalTo("The lazy red fox jumps over the quick dog.")); - client().prepareIndex("test", "type1", "2") .setSource("field1", new String[]{"The quick brown fox jumps over the lazy dog. Second sentence not finished", "The lazy red fox jumps over the quick dog.", "The quick brown dog jumps over the lazy fox."}).get(); refresh(); @@ -2009,8 +1969,8 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { if ("1".equals(searchHit.id())) { assertHighlight(searchHit, "field1", 0, 1, equalTo("The quick brown fox jumps over the lazy dog. The lazy red fox jumps over the quick dog. The quick brown dog jumps over the lazy fox.")); } else if ("2".equals(searchHit.id())) { - assertHighlight(searchHit, "field1", 0, equalTo("The quick brown fox jumps over the lazy dog. Second sentence not finished")); - assertHighlight(searchHit, "field1", 1, equalTo("The lazy red fox jumps over the quick dog.")); + assertHighlight(searchHit, "field1", 0, 3, equalTo("The quick brown fox jumps over the lazy dog. Second sentence not finished")); + assertHighlight(searchHit, "field1", 1, 3, equalTo("The lazy red fox jumps over the quick dog.")); assertHighlight(searchHit, "field1", 2, 3, equalTo("The quick brown dog jumps over the lazy fox.")); } else { fail("Only hits with id 1 and 2 are returned"); @@ -2018,67 +1978,6 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { } } - @Test - public void testPostingsHighlighterRequireFieldMatch() throws Exception { - assertAcked(prepareCreate("test").addMapping("type1", type1PostingsffsetsMapping())); - ensureGreen(); - - client().prepareIndex("test", "type1") - .setSource("field1", "The quick brown fox jumps over the lazy dog. The lazy red fox jumps over the quick dog. The quick brown dog jumps over the lazy fox.", - "field2", "The quick brown fox jumps over the lazy dog. The lazy red fox jumps over the quick dog. The quick brown dog jumps over the lazy fox.").get(); - refresh(); - - logger.info("--> highlighting and searching on field1"); - SearchSourceBuilder source = searchSource() - .query(termQuery("field1", "fox")) - .highlight(highlight() - .field(new HighlightBuilder.Field("field1").requireFieldMatch(true).preTags("").postTags("")) - .field(new HighlightBuilder.Field("field2").requireFieldMatch(true).preTags("").postTags(""))); - - SearchResponse searchResponse = client().search(searchRequest("test").source(source)).actionGet(); - - //field2 is not returned highlighted because of the require field match option set to true - assertNotHighlighted(searchResponse, 0, "field2"); - assertHighlight(searchResponse, 0, "field1", 0, equalTo("The quick brown fox jumps over the lazy dog.")); - assertHighlight(searchResponse, 0, "field1", 1, equalTo("The lazy red fox jumps over the quick dog.")); - assertHighlight(searchResponse, 0, "field1", 2, 3, equalTo("The quick brown dog jumps over the lazy fox.")); - - logger.info("--> highlighting and searching on field1 and field2 - require field match set to false"); - source = searchSource() - .query(termQuery("field1", "fox")) - .highlight(highlight() - .field(new HighlightBuilder.Field("field1").requireFieldMatch(false).preTags("").postTags("")) - .field(new HighlightBuilder.Field("field2").requireFieldMatch(false).preTags("").postTags(""))); - - searchResponse = client().search(searchRequest("test").source(source)).actionGet(); - - assertHighlight(searchResponse, 0, "field1", 0, equalTo("The quick brown fox jumps over the lazy dog.")); - assertHighlight(searchResponse, 0, "field1", 1, equalTo("The lazy red fox jumps over the quick dog.")); - assertHighlight(searchResponse, 0, "field1", 2, 3, equalTo("The quick brown dog jumps over the lazy fox.")); - - //field2 is now returned highlighted thanks to require_field_match set to false - assertHighlight(searchResponse, 0, "field2", 0, equalTo("The quick brown fox jumps over the lazy dog.")); - assertHighlight(searchResponse, 0, "field2", 1, equalTo("The lazy red fox jumps over the quick dog.")); - assertHighlight(searchResponse, 0, "field2", 2, 3, equalTo("The quick brown dog jumps over the lazy fox.")); - logger.info("--> highlighting and searching on field1 and field2 via multi_match query"); - final MultiMatchQueryBuilder mmquery = multiMatchQuery("fox", "field1", "field2").type(RandomPicks.randomFrom(getRandom(), MultiMatchQueryBuilder.Type.values())); - source = searchSource() - .query(mmquery) - .highlight(highlight().highlightQuery(randomBoolean() ? mmquery : null) - .field(new HighlightBuilder.Field("field1").requireFieldMatch(true).preTags("").postTags("")) - .field(new HighlightBuilder.Field("field2").requireFieldMatch(true).preTags("").postTags(""))); - searchResponse = client().search(searchRequest("test").source(source)).actionGet(); - assertHitCount(searchResponse, 1l); - - assertHighlight(searchResponse, 0, "field1", 0, equalTo("The quick brown fox jumps over the lazy dog.")); - assertHighlight(searchResponse, 0, "field1", 1, equalTo("The lazy red fox jumps over the quick dog.")); - assertHighlight(searchResponse, 0, "field1", 2, 3, equalTo("The quick brown dog jumps over the lazy fox.")); - //field2 is now returned highlighted thanks to the multi_match query on both fields - assertHighlight(searchResponse, 0, "field2", 0, equalTo("The quick brown fox jumps over the lazy dog.")); - assertHighlight(searchResponse, 0, "field2", 1, equalTo("The lazy red fox jumps over the quick dog.")); - assertHighlight(searchResponse, 0, "field2", 2, 3, equalTo("The quick brown dog jumps over the lazy fox.")); - } - @Test public void testMultiMatchQueryHighlight() throws IOException { String[] highlighterTypes = new String[] {"fvh", "plain", "postings"}; @@ -2097,14 +1996,22 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { refresh(); final int iters = scaledRandomIntBetween(20, 30); for (int i = 0; i < iters; i++) { - MultiMatchQueryBuilder.Type matchQueryType = rarely() ? null : RandomPicks.randomFrom(getRandom(), MultiMatchQueryBuilder.Type.values()); + String highlighterType = rarely() ? null : RandomPicks.randomFrom(getRandom(), highlighterTypes); + MultiMatchQueryBuilder.Type[] supportedQueryTypes; + if ("postings".equals(highlighterType)) { + //phrase_prefix is not supported by postings highlighter, as it rewrites against an empty reader, the prefix will never match any term + supportedQueryTypes = new MultiMatchQueryBuilder.Type[]{MultiMatchQueryBuilder.Type.BEST_FIELDS, MultiMatchQueryBuilder.Type.CROSS_FIELDS, MultiMatchQueryBuilder.Type.MOST_FIELDS, MultiMatchQueryBuilder.Type.PHRASE}; + } else { + supportedQueryTypes = MultiMatchQueryBuilder.Type.values(); + } + MultiMatchQueryBuilder.Type matchQueryType = rarely() ? null : RandomPicks.randomFrom(getRandom(), supportedQueryTypes); final MultiMatchQueryBuilder multiMatchQueryBuilder = multiMatchQuery("the quick brown fox", "field1", "field2").type(matchQueryType); - String type = rarely() ? null : RandomPicks.randomFrom(getRandom(),highlighterTypes); + SearchSourceBuilder source = searchSource() .query(multiMatchQueryBuilder) - .highlight(highlight().highlightQuery(randomBoolean() ? multiMatchQueryBuilder : null).highlighterType(type) + .highlight(highlight().highlightQuery(randomBoolean() ? multiMatchQueryBuilder : null).highlighterType(highlighterType) .field(new Field("field1").requireFieldMatch(true).preTags("").postTags(""))); - logger.info("Running multi-match type: [" + matchQueryType + "] highlight with type: [" + type + "]"); + logger.info("Running multi-match type: [" + matchQueryType + "] highlight with type: [" + highlighterType + "]"); SearchResponse searchResponse = client().search(searchRequest("test").source(source)).actionGet(); assertHitCount(searchResponse, 1l); assertHighlight(searchResponse, 0, "field1", 0, anyOf(equalTo("The quick brown fox jumps over"), @@ -2139,16 +2046,6 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { assertThat(field1.fragments()[2].string(), equalTo("This is the second value's first sentence.")); assertThat(field1.fragments()[3].string(), equalTo("This sentence contains one match, not that short.")); assertThat(field1.fragments()[4].string(), equalTo("One sentence match here and scored lower since the text is quite long, not that appealing.")); - - //lets use now number_of_fragments = 0, so that we highlight per value without breaking them into snippets, but we sort the values by score - source = searchSource() - .query(termQuery("field1", "sentence")) - .highlight(highlight().field("field1", -1, 0).order("score")); - - searchResponse = client().search(searchRequest("test").source(source)).actionGet(); - assertHighlight(searchResponse, 0, "field1", 0, equalTo("This is the second value's first sentence. This one contains no matches. This sentence contains three sentence occurrences (sentence).")); - assertHighlight(searchResponse, 0, "field1", 1, equalTo("This sentence contains one match, not that short. This sentence contains two sentence matches. This one contains no matches.")); - assertHighlight(searchResponse, 0, "field1", 2, 3, equalTo("One sentence match here and scored lower since the text is quite long, not that appealing. This one contains no matches.")); } @Test @@ -2260,25 +2157,24 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { assertNoFailures(search); assertFailures(client().prepareSearch() - .setQuery(matchQuery("title", "this is a test")) - .addHighlightedField("title") - .setHighlighterType("postings-highlighter"), + .setQuery(matchQuery("title", "this is a test")) + .addHighlightedField("title") + .setHighlighterType("postings-highlighter"), RestStatus.BAD_REQUEST, containsString("the field [title] should be indexed with positions and offsets in the postings list to be used with postings highlighter")); - assertFailures(client().prepareSearch() - .setQuery(matchQuery("title", "this is a test")) - .addHighlightedField("title") - .setHighlighterType("postings"), + .setQuery(matchQuery("title", "this is a test")) + .addHighlightedField("title") + .setHighlighterType("postings"), RestStatus.BAD_REQUEST, containsString("the field [title] should be indexed with positions and offsets in the postings list to be used with postings highlighter")); assertFailures(client().prepareSearch() - .setQuery(matchQuery("title", "this is a test")) - .addHighlightedField("tit*") - .setHighlighterType("postings"), + .setQuery(matchQuery("title", "this is a test")) + .addHighlightedField("tit*") + .setHighlighterType("postings"), RestStatus.BAD_REQUEST, containsString("the field [title] should be indexed with positions and offsets in the postings list to be used with postings highlighter")); } @@ -2316,9 +2212,8 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { assertHighlight(searchResponse, 0, "field2", 0, 1, equalTo("The quick brown fox jumps over the lazy dog!")); } - public XContentBuilder type1PostingsffsetsMapping() throws IOException { + private static XContentBuilder type1PostingsffsetsMapping() throws IOException { return XContentFactory.jsonBuilder().startObject().startObject("type1") - .startObject("_all").field("store", "yes").field("index_options", "offsets").endObject() .startObject("properties") .startObject("field1").field("type", "string").field("index_options", "offsets").endObject() .startObject("field2").field("type", "string").field("index_options", "offsets").endObject() @@ -2326,9 +2221,6 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { .endObject().endObject(); } - private static final String[] REWRITE_METHODS = new String[]{"constant_score_auto", "scoring_boolean", "constant_score_boolean", - "constant_score_filter", "top_terms_boost_50", "top_terms_50"}; - @Test public void testPostingsHighlighterPrefixQuery() throws Exception { assertAcked(prepareCreate("test").addMapping("type1", type1PostingsffsetsMapping())); @@ -2338,7 +2230,7 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { refresh(); logger.info("--> highlighting and searching on field2"); - SearchSourceBuilder source = searchSource().query(prefixQuery("field2", "qui").rewrite(randomFrom(REWRITE_METHODS))) + SearchSourceBuilder source = searchSource().query(prefixQuery("field2", "qui")) .highlight(highlight().field("field2")); SearchResponse searchResponse = client().prepareSearch("test").setSource(source.buildAsBytes()).get(); assertHighlight(searchResponse, 0, "field2", 0, 1, equalTo("The quick brown fox jumps over the lazy dog!")); @@ -2368,7 +2260,7 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { client().prepareIndex("test", "type1").setSource("field1", "this is a test", "field2", "The quick brown fox jumps over the lazy dog! Second sentence.").get(); refresh(); logger.info("--> highlighting and searching on field2"); - SearchSourceBuilder source = searchSource().query(regexpQuery("field2", "qu[a-l]+k").rewrite(randomFrom(REWRITE_METHODS))) + SearchSourceBuilder source = searchSource().query(regexpQuery("field2", "qu[a-l]+k")) .highlight(highlight().field("field2")); SearchResponse searchResponse = client().prepareSearch("test").setSource(source.buildAsBytes()).get(); @@ -2383,13 +2275,13 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { client().prepareIndex("test", "type1").setSource("field1", "this is a test", "field2", "The quick brown fox jumps over the lazy dog! Second sentence.").get(); refresh(); logger.info("--> highlighting and searching on field2"); - SearchSourceBuilder source = searchSource().query(wildcardQuery("field2", "qui*").rewrite(randomFrom(REWRITE_METHODS))) + SearchSourceBuilder source = searchSource().query(wildcardQuery("field2", "qui*")) .highlight(highlight().field("field2")); SearchResponse searchResponse = client().prepareSearch("test").setSource(source.buildAsBytes()).get(); assertHighlight(searchResponse, 0, "field2", 0, 1, equalTo("The quick brown fox jumps over the lazy dog!")); - source = searchSource().query(wildcardQuery("field2", "qu*k").rewrite(randomFrom(REWRITE_METHODS))) + source = searchSource().query(wildcardQuery("field2", "qu*k")) .highlight(highlight().field("field2")); searchResponse = client().prepareSearch("test").setSource(source.buildAsBytes()).get(); assertHitCount(searchResponse, 1l); @@ -2420,7 +2312,7 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { client().prepareIndex("test", "type1").setSource("field1", "this is a test", "field2", "The quick brown fox jumps over the lazy dog! Second sentence.").get(); refresh(); logger.info("--> highlighting and searching on field2"); - SearchSourceBuilder source = searchSource().query(queryStringQuery("qui*").defaultField("field2").rewrite(randomFrom(REWRITE_METHODS))) + SearchSourceBuilder source = searchSource().query(queryStringQuery("qui*").defaultField("field2")) .highlight(highlight().field("field2")); SearchResponse searchResponse = client().prepareSearch("test").setSource(source.buildAsBytes()).get(); assertHighlight(searchResponse, 0, "field2", 0, 1, equalTo("The quick brown fox jumps over the lazy dog!")); @@ -2436,7 +2328,7 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { refresh(); logger.info("--> highlighting and searching on field1"); - SearchSourceBuilder source = searchSource().query(constantScoreQuery(regexpQuery("field1", "pho[a-z]+").rewrite(randomFrom(REWRITE_METHODS)))) + SearchSourceBuilder source = searchSource().query(constantScoreQuery(regexpQuery("field1", "pho[a-z]+"))) .highlight(highlight().field("field1")); SearchResponse searchResponse = client().prepareSearch("test").setSource(source.buildAsBytes()).get(); assertHighlight(searchResponse, 0, "field1", 0, 1, equalTo("The photography word will get highlighted")); @@ -2455,7 +2347,7 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { SearchSourceBuilder source = searchSource().query(boolQuery() .should(constantScoreQuery(QueryBuilders.missingQuery("field1"))) .should(matchQuery("field1", "test")) - .should(filteredQuery(queryStringQuery("field1:photo*").rewrite(randomFrom(REWRITE_METHODS)), null))) + .should(filteredQuery(queryStringQuery("field1:photo*"), null))) .highlight(highlight().field("field1")); SearchResponse searchResponse = client().prepareSearch("test").setSource(source.buildAsBytes()).get(); assertHighlight(searchResponse, 0, "field1", 0, 1, equalTo("The photography word will get highlighted")); @@ -2471,7 +2363,7 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { refresh(); logger.info("--> highlighting and searching on field1"); - SearchSourceBuilder source = searchSource().query(boolQuery().must(prefixQuery("field1", "photo").rewrite(randomFrom(REWRITE_METHODS))).should(matchQuery("field1", "test").minimumShouldMatch("0"))) + SearchSourceBuilder source = searchSource().query(boolQuery().must(prefixQuery("field1", "photo")).should(matchQuery("field1", "test").minimumShouldMatch("0"))) .highlight(highlight().field("field1")); SearchResponse searchResponse = client().prepareSearch("test").setSource(source.buildAsBytes()).get(); assertHighlight(searchResponse, 0, "field1", 0, 1, equalTo("The photography word will get highlighted")); @@ -2487,7 +2379,7 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { refresh(); logger.info("--> highlighting and searching on field1"); - SearchSourceBuilder source = searchSource().query(filteredQuery(queryStringQuery("field1:photo*").rewrite(randomFrom(REWRITE_METHODS)), missingQuery("field_null"))) + SearchSourceBuilder source = searchSource().query(filteredQuery(queryStringQuery("field1:photo*"), missingQuery("field_null"))) .highlight(highlight().field("field1")); SearchResponse searchResponse = client().prepareSearch("test").setSource(source.buildAsBytes()).get(); assertHighlight(searchResponse, 0, "field1", 0, 1, equalTo("The photography word will get highlighted")); From ae599f93a754419f34d490b389c3f3db415725da Mon Sep 17 00:00:00 2001 From: Scott Chamberlain Date: Tue, 12 May 2015 07:57:41 -0700 Subject: [PATCH 20/56] Added an R client to community clients page I maintain an R client called `elastic` at https://github.com/ropensci/elastic --- docs/community/clients.asciidoc | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/community/clients.asciidoc b/docs/community/clients.asciidoc index c92534eca7d..c99645ac6bf 100644 --- a/docs/community/clients.asciidoc +++ b/docs/community/clients.asciidoc @@ -218,3 +218,6 @@ See the {client}/net-api/current/index.html[official Elasticsearch .NET client]. === R * https://github.com/Tomesch/elasticsearch[elasticsearch] R client for Elasticsearch + +* https://github.com/ropensci/elastic[elastic]: + A general purpose R client for Elasticsearch From 9d71816cd22af91f6425da4cd60b973c9d079361 Mon Sep 17 00:00:00 2001 From: Clinton Gormley Date: Fri, 15 May 2015 21:24:59 +0200 Subject: [PATCH 21/56] Docs: Fixed explanation of AUTO fuzziness Closes #11186 --- docs/reference/api-conventions.asciidoc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/reference/api-conventions.asciidoc b/docs/reference/api-conventions.asciidoc index fc8b60a91af..25b9ac4fcea 100644 --- a/docs/reference/api-conventions.asciidoc +++ b/docs/reference/api-conventions.asciidoc @@ -219,8 +219,8 @@ the maximum allowed Levenshtein Edit Distance (or number of edits) -- generates an edit distance based on the length of the term. For lengths: -`0..1`:: must match exactly -`1..5`:: one edit allowed +`0..2`:: must match exactly +`3..5`:: one edit allowed `>5`:: two edits allowed `AUTO` should generally be the preferred value for `fuzziness`. From a843008b17efffc4ca897ebf3c0c2db96daa1b7f Mon Sep 17 00:00:00 2001 From: javanna Date: Fri, 8 May 2015 20:12:55 +0200 Subject: [PATCH 22/56] Highlighting: require_field_match set to true by default The default `false` for `require_field_match` is a bit odd and confusing for users, given that field names get ignored by default and every field gets highlighted if it contains terms extracted out of the query, regardless of which fields were queries. Changed the default to `true`, it can always be changed per request. Closes #10627 Closes #11067 --- docs/reference/migration/migrate_2_0.asciidoc | 10 ++++++ .../search/request/highlighting.asciidoc | 8 ++--- .../highlight/HighlighterParseElement.java | 2 +- .../highlight/HighlighterSearchTests.java | 32 +++++++++---------- 4 files changed, 31 insertions(+), 21 deletions(-) diff --git a/docs/reference/migration/migrate_2_0.asciidoc b/docs/reference/migration/migrate_2_0.asciidoc index c91e6e0a74f..53cacf7be16 100644 --- a/docs/reference/migration/migrate_2_0.asciidoc +++ b/docs/reference/migration/migrate_2_0.asciidoc @@ -591,8 +591,18 @@ from fielddata. [float] === Highlighting +The default value for the `require_field_match` option is `true` rather than +`false`, meaning that the highlighters will take the fields that were queried +into account by default. That means for instance that highlighting any field +when querying the `_all` field will produce no highlighted snippets by default, +given that the match was on the `_all` field only. Querying the same fields +that need to be highlighted is the cleaner solution to get highlighted snippets +back. Otherwise `require_field_match` option can be set to `false` to ignore +field names completely when highlighting. + The postings highlighter doesn't support the `require_field_match` option anymore, it will only highlight fields that were queried. The `match` query with type set to `match_phrase_prefix` is not supported by the postings highlighter. No highlighted snippets will be returned. + diff --git a/docs/reference/search/request/highlighting.asciidoc b/docs/reference/search/request/highlighting.asciidoc index 7c43fdc37db..0b5d41e683e 100644 --- a/docs/reference/search/request/highlighting.asciidoc +++ b/docs/reference/search/request/highlighting.asciidoc @@ -404,10 +404,10 @@ at the field level. [[field-match]] ==== Require Field Match -`require_field_match` can be set to `true` which will cause a field to -be highlighted only if a query matched that field. `false` means that -terms are highlighted on all requested fields regardless if the query -matches specifically on them. +`require_field_match` can be set to `false` which will cause any field to +be highlighted regardless of whether the query matched specifically on them. +The default behaviour is `true`, meaning that only fields that hold a query +match will be highlighted. [[boundary-characters]] ==== Boundary Characters diff --git a/src/main/java/org/elasticsearch/search/highlight/HighlighterParseElement.java b/src/main/java/org/elasticsearch/search/highlight/HighlighterParseElement.java index 3613327c679..802122f648d 100644 --- a/src/main/java/org/elasticsearch/search/highlight/HighlighterParseElement.java +++ b/src/main/java/org/elasticsearch/search/highlight/HighlighterParseElement.java @@ -82,7 +82,7 @@ public class HighlighterParseElement implements SearchParseElement { final SearchContextHighlight.FieldOptions.Builder globalOptionsBuilder = new SearchContextHighlight.FieldOptions.Builder() .preTags(DEFAULT_PRE_TAGS).postTags(DEFAULT_POST_TAGS).scoreOrdered(false).highlightFilter(false) - .requireFieldMatch(false).forceSource(false).fragmentCharSize(100).numberOfFragments(5) + .requireFieldMatch(true).forceSource(false).fragmentCharSize(100).numberOfFragments(5) .encoder("default").boundaryMaxScan(SimpleBoundaryScanner.DEFAULT_MAX_SCAN) .boundaryChars(SimpleBoundaryScanner.DEFAULT_BOUNDARY_CHARS) .noMatchSize(0).phraseLimit(256); diff --git a/src/test/java/org/elasticsearch/search/highlight/HighlighterSearchTests.java b/src/test/java/org/elasticsearch/search/highlight/HighlighterSearchTests.java index 89c0b2dc4a7..bc4f78bc50b 100644 --- a/src/test/java/org/elasticsearch/search/highlight/HighlighterSearchTests.java +++ b/src/test/java/org/elasticsearch/search/highlight/HighlighterSearchTests.java @@ -458,7 +458,7 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { SearchResponse search = client().prepareSearch() .setQuery(matchQuery("title", "bug")) .addHighlightedField("title", -1, 2) - .addHighlightedField("titleTV", -1, 2) + .addHighlightedField("titleTV", -1, 2).setHighlighterRequireFieldMatch(false) .get(); assertHighlight(search, 0, "title", 0, equalTo("This is a test on the highlighting bug present in elasticsearch")); @@ -490,7 +490,7 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { .query(termQuery("field1", "test")) .highlight(highlight().order("score").preTags("").postTags("").fragmentSize(1).numOfFragments(1) .field(new HighlightBuilder.Field("field1").numOfFragments(2)) - .field(new HighlightBuilder.Field("field2").preTags("").postTags("").fragmentSize(50))); + .field(new HighlightBuilder.Field("field2").preTags("").postTags("").fragmentSize(50).requireFieldMatch(false))); SearchResponse searchResponse = client().prepareSearch("test").setSource(source.buildAsBytes()).get(); @@ -603,7 +603,7 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { logger.info("--> searching on _all, highlighting on field1"); source = searchSource() .query(termQuery("_all", "test")) - .highlight(highlight().field("field1").order("score").preTags("").postTags("")); + .highlight(highlight().field("field1").order("score").preTags("").postTags("").requireFieldMatch(false)); searchResponse = client().prepareSearch("test").setSource(source.buildAsBytes()).get(); @@ -612,7 +612,7 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { logger.info("--> searching on _all, highlighting on field2"); source = searchSource() .query(termQuery("_all", "quick")) - .highlight(highlight().field("field2").order("score").preTags("").postTags("")); + .highlight(highlight().field("field2").order("score").preTags("").postTags("").requireFieldMatch(false)); searchResponse = client().prepareSearch("test").setSource(source.buildAsBytes()).get(); @@ -621,7 +621,7 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { logger.info("--> searching on _all, highlighting on field2"); source = searchSource() .query(prefixQuery("_all", "qui")) - .highlight(highlight().field("field2").order("score").preTags("").postTags("")); + .highlight(highlight().field("field2").order("score").preTags("").postTags("").requireFieldMatch(false)); searchResponse = client().prepareSearch("test").setSource(source.buildAsBytes()).get(); @@ -630,7 +630,7 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { logger.info("--> searching on _all with constant score, highlighting on field2"); source = searchSource() .query(constantScoreQuery(prefixQuery("_all", "qui"))) - .highlight(highlight().field("field2").order("score").preTags("").postTags("")); + .highlight(highlight().field("field2").order("score").preTags("").postTags("").requireFieldMatch(false)); searchResponse = client().prepareSearch("test").setSource(source.buildAsBytes()).get(); @@ -639,7 +639,7 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { logger.info("--> searching on _all with constant score, highlighting on field2"); source = searchSource() .query(boolQuery().should(constantScoreQuery(prefixQuery("_all", "qui")))) - .highlight(highlight().field("field2").order("score").preTags("").postTags("")); + .highlight(highlight().field("field2").order("score").preTags("").postTags("").requireFieldMatch(false)); searchResponse = client().prepareSearch("test").setSource(source.buildAsBytes()).get(); assertHighlight(searchResponse, 0, "field2", 0, 1, equalTo("The quick brown fox jumps over the lazy dog")); @@ -666,7 +666,7 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { logger.info("--> searching on _all, highlighting on field1"); source = searchSource() .query(termQuery("_all", "test")) - .highlight(highlight().field("field1", 100, 0).order("score").preTags("").postTags("")); + .highlight(highlight().field("field1", 100, 0).order("score").preTags("").postTags("").requireFieldMatch(false)); searchResponse = client().prepareSearch("test").setSource(source.buildAsBytes()).get(); @@ -676,7 +676,7 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { logger.info("--> searching on _all, highlighting on field2"); source = searchSource() .query(termQuery("_all", "quick")) - .highlight(highlight().field("field2", 100, 0).order("score").preTags("").postTags("")); + .highlight(highlight().field("field2", 100, 0).order("score").preTags("").postTags("").requireFieldMatch(false)); searchResponse = client().prepareSearch("test").setSource(source.buildAsBytes()).get(); @@ -686,7 +686,7 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { logger.info("--> searching on _all, highlighting on field2"); source = searchSource() .query(prefixQuery("_all", "qui")) - .highlight(highlight().field("field2", 100, 0).order("score").preTags("").postTags("")); + .highlight(highlight().field("field2", 100, 0).order("score").preTags("").postTags("").requireFieldMatch(false)); searchResponse = client().prepareSearch("test").setSource(source.buildAsBytes()).get(); @@ -1877,7 +1877,7 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { assertHighlight(searchResponse, 0, "field1", 0, 1, equalTo("this is a test")); - logger.info("--> searching on _all, highlighting on field1"); + logger.info("--> searching on field1, highlighting on field1"); source = searchSource() .query(termQuery("field1", "test")) .highlight(highlight().field("field1").preTags("").postTags("")); @@ -1886,7 +1886,7 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { assertHighlight(searchResponse, 0, "field1", 0, 1, equalTo("this is a test")); - logger.info("--> searching on _all, highlighting on field2"); + logger.info("--> searching on field2, highlighting on field2"); source = searchSource() .query(termQuery("field2", "quick")) .highlight(highlight().field("field2").order("score").preTags("").postTags("")); @@ -1895,7 +1895,7 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { assertHighlight(searchResponse, 0, "field2", 0, 1, equalTo("The quick brown fox jumps over the lazy quick dog")); - logger.info("--> searching on _all, highlighting on field2"); + logger.info("--> searching on field2, highlighting on field2"); source = searchSource() .query(matchPhraseQuery("field2", "quick brown")) .highlight(highlight().field("field2").preTags("").postTags("")); @@ -1906,10 +1906,10 @@ public class HighlighterSearchTests extends ElasticsearchIntegrationTest { assertHighlight(searchResponse, 0, "field2", 0, 1, equalTo("The quick brown fox jumps over the lazy quick dog")); //lets fall back to the standard highlighter then, what people would do to highlight query matches - logger.info("--> searching on _all, highlighting on field2, falling back to the plain highlighter"); + logger.info("--> searching on field2, highlighting on field2, falling back to the plain highlighter"); source = searchSource() - .query(matchPhraseQuery("field2", "quick brown")) - .highlight(highlight().field("field2").preTags("").postTags("").highlighterType("highlighter")); + .query(matchPhraseQuery("_all", "quick brown")) + .highlight(highlight().field("field2").preTags("").postTags("").highlighterType("highlighter").requireFieldMatch(false)); searchResponse = client().search(searchRequest("test").source(source)).actionGet(); From 217fd4443a15dbf60f61d188e779b856ed197083 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 13 May 2015 22:24:27 +0200 Subject: [PATCH 23/56] Make SearchFactory static class in InternalEngine Now that lucene provides a way to identify if the warming reader is the first initial opened reader we can detach this class from the enclosing and make it static. This is important since it might access not fully initialized members of the enclosing class since it's initialized and used during constructor invocation. --- .../index/engine/InternalEngine.java | 63 +++++++++++-------- 1 file changed, 36 insertions(+), 27 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 95e3a202819..8bddc98c16b 100644 --- a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -49,6 +49,7 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.cluster.routing.DjbHashFunction; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.lucene.LoggerInfoStream; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; @@ -64,6 +65,7 @@ import org.elasticsearch.index.merge.policy.ElasticsearchMergePolicy; import org.elasticsearch.index.merge.policy.MergePolicyProvider; import org.elasticsearch.index.merge.scheduler.MergeSchedulerProvider; import org.elasticsearch.index.search.nested.IncludeNestedDocsQuery; +import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.TranslogRecoveryPerformer; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.IndicesWarmer; @@ -143,7 +145,7 @@ public class InternalEngine extends Engine { } throttle = new IndexThrottle(); - this.searcherFactory = new SearchFactory(engineConfig); + this.searcherFactory = new SearchFactory(logger, isClosed, engineConfig); final Long committedTranslogId; try { writer = createWriter(); @@ -1050,10 +1052,19 @@ public class InternalEngine extends Engine { } /** Extended SearcherFactory that warms the segments if needed when acquiring a new searcher */ - class SearchFactory extends EngineSearcherFactory { + final static class SearchFactory extends EngineSearcherFactory { - SearchFactory(EngineConfig engineConfig) { + private final IndicesWarmer warmer; + private final ShardId shardId; + private final ESLogger logger; + private final AtomicBoolean isEngineClosed; + + SearchFactory(ESLogger logger, AtomicBoolean isEngineClosed, EngineConfig engineConfig) { super(engineConfig); + warmer = engineConfig.getWarmer(); + shardId = engineConfig.getShardId(); + this.logger = logger; + this.isEngineClosed = isEngineClosed; } @Override @@ -1065,36 +1076,34 @@ public class InternalEngine extends Engine { IndexSearcher newSearcher = null; boolean closeNewSearcher = false; try { - if (searcherManager == null) { + if (previousReader == null) { // we are starting up - no writer active so we can't acquire a searcher. newSearcher = searcher; } else { - try (final Searcher currentSearcher = acquireSearcher("search_factory")) { - // figure out the newSearcher, with only the new readers that are relevant for us - List readers = Lists.newArrayList(); - for (LeafReaderContext newReaderContext : searcher.getIndexReader().leaves()) { - if (isMergedSegment(newReaderContext.reader())) { - // merged segments are already handled by IndexWriterConfig.setMergedSegmentWarmer - continue; - } - boolean found = false; - for (LeafReaderContext currentReaderContext : currentSearcher.reader().leaves()) { - if (currentReaderContext.reader().getCoreCacheKey().equals(newReaderContext.reader().getCoreCacheKey())) { - found = true; - break; - } - } - if (!found) { - readers.add(newReaderContext.reader()); + // figure out the newSearcher, with only the new readers that are relevant for us + List readers = Lists.newArrayList(); + for (LeafReaderContext newReaderContext : reader.leaves()) { + if (isMergedSegment(newReaderContext.reader())) { + // merged segments are already handled by IndexWriterConfig.setMergedSegmentWarmer + continue; + } + boolean found = false; + for (LeafReaderContext currentReaderContext : previousReader.leaves()) { + if (currentReaderContext.reader().getCoreCacheKey().equals(newReaderContext.reader().getCoreCacheKey())) { + found = true; + break; } } - if (!readers.isEmpty()) { - // we don't want to close the inner readers, just increase ref on them - IndexReader newReader = new MultiReader(readers.toArray(new IndexReader[readers.size()]), false); - newSearcher = super.newSearcher(newReader, null); - closeNewSearcher = true; + if (!found) { + readers.add(newReaderContext.reader()); } } + if (!readers.isEmpty()) { + // we don't want to close the inner readers, just increase ref on them + IndexReader newReader = new MultiReader(readers.toArray(new IndexReader[readers.size()]), false); + newSearcher = super.newSearcher(newReader, null); + closeNewSearcher = true; + } } if (newSearcher != null) { @@ -1103,7 +1112,7 @@ public class InternalEngine extends Engine { } warmer.warmTopReader(new IndicesWarmer.WarmerContext(shardId, new Searcher("warmer", searcher))); } catch (Throwable e) { - if (isClosed.get() == false) { + if (isEngineClosed.get() == false) { logger.warn("failed to prepare/warm", e); } } finally { From 97ffb2b4fdcc811c664c0971bf2281274ac7775b Mon Sep 17 00:00:00 2001 From: Robert Muir Date: Fri, 15 May 2015 16:15:23 -0400 Subject: [PATCH 24/56] include rest tests in test-framework.jar Closes #11192 which I accidentally already closed. Squashed commit of the following: commit f23faccddc2a77a880841da4c89c494edaa2aa46 Author: Robert Muir Date: Fri May 15 16:04:55 2015 -0400 Simplify this FileUtils even more: its either from the filesystem, or the classpath, not both. Its already trying 4 different combinations of crazy paths for either of these anyway. commit c7016c8a2b5a6043e2ded4b48b160821ba196974 Author: Robert Muir Date: Fri May 15 14:21:37 2015 -0400 include rest tests in test-framework jar --- pom.xml | 4 +- .../test/rest/ElasticsearchRestTestCase.java | 81 +++++++++++++++---- .../test/rest/spec/RestSpec.java | 6 +- .../test/rest/support/FileUtils.java | 79 ++++++++++-------- .../test/rest/test/FileUtilsTests.java | 26 +++--- 5 files changed, 130 insertions(+), 66 deletions(-) diff --git a/pom.xml b/pom.xml index 4b36f6c44ef..61966eb58a5 100644 --- a/pom.xml +++ b/pom.xml @@ -787,8 +787,6 @@ org/elasticsearch/test/test/**/* - - true @@ -1519,6 +1517,7 @@ dev-tools/forbidden/all-signatures.txt ${forbidden.test.signatures} + **.SuppressForbidden test-compile @@ -1539,6 +1538,7 @@ + rest-api-spec/**/* org/elasticsearch/test/**/* org/elasticsearch/bootstrap/BootstrapForTesting.class org/elasticsearch/common/cli/CliToolTestCase.class diff --git a/src/test/java/org/elasticsearch/test/rest/ElasticsearchRestTestCase.java b/src/test/java/org/elasticsearch/test/rest/ElasticsearchRestTestCase.java index b7b207a6b11..9661e3950c6 100644 --- a/src/test/java/org/elasticsearch/test/rest/ElasticsearchRestTestCase.java +++ b/src/test/java/org/elasticsearch/test/rest/ElasticsearchRestTestCase.java @@ -23,11 +23,14 @@ import com.carrotsearch.randomizedtesting.RandomizedTest; import com.carrotsearch.randomizedtesting.annotations.TestGroup; import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite; import com.google.common.collect.Lists; + import org.apache.lucene.util.LuceneTestCase.Slow; import org.apache.lucene.util.LuceneTestCase.SuppressCodecs; import org.apache.lucene.util.LuceneTestCase.SuppressFsync; +import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.TimeUnits; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.SuppressForbidden; import org.elasticsearch.common.io.PathUtils; import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.Settings; @@ -48,9 +51,17 @@ import org.junit.BeforeClass; import org.junit.Test; import java.io.IOException; +import java.io.InputStream; import java.lang.annotation.*; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URL; +import java.nio.file.FileSystem; +import java.nio.file.FileSystems; +import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.PathMatcher; +import java.nio.file.StandardCopyOption; import java.util.*; /** @@ -158,23 +169,29 @@ public abstract class ElasticsearchRestTestCase extends ElasticsearchIntegration } private static List collectTestCandidates(int id, int count) throws RestTestParseException, IOException { - String[] paths = resolvePathsProperty(REST_TESTS_SUITE, DEFAULT_TESTS_PATH); - Map> yamlSuites = FileUtils.findYamlSuites(DEFAULT_TESTS_PATH, paths); - List testCandidates = Lists.newArrayList(); - RestTestSuiteParser restTestSuiteParser = new RestTestSuiteParser(); - //yaml suites are grouped by directory (effectively by api) - for (String api : yamlSuites.keySet()) { - List yamlFiles = Lists.newArrayList(yamlSuites.get(api)); - for (Path yamlFile : yamlFiles) { - String key = api + yamlFile.getFileName().toString(); - if (mustExecute(key, id, count)) { - RestTestSuite restTestSuite = restTestSuiteParser.parse(api, yamlFile); - for (TestSection testSection : restTestSuite.getTestSections()) { - testCandidates.add(new RestTestCandidate(restTestSuite, testSection)); + FileSystem fileSystem = getFileSystem(); + // don't make a try-with, getFileSystem returns null + // ... and you can't close() the default filesystem + try { + String[] paths = resolvePathsProperty(REST_TESTS_SUITE, DEFAULT_TESTS_PATH); + Map> yamlSuites = FileUtils.findYamlSuites(fileSystem, DEFAULT_TESTS_PATH, paths); + RestTestSuiteParser restTestSuiteParser = new RestTestSuiteParser(); + //yaml suites are grouped by directory (effectively by api) + for (String api : yamlSuites.keySet()) { + List yamlFiles = Lists.newArrayList(yamlSuites.get(api)); + for (Path yamlFile : yamlFiles) { + String key = api + yamlFile.getFileName().toString(); + if (mustExecute(key, id, count)) { + RestTestSuite restTestSuite = restTestSuiteParser.parse(api, yamlFile); + for (TestSection testSection : restTestSuite.getTestSections()) { + testCandidates.add(new RestTestCandidate(restTestSuite, testSection)); + } } } } + } finally { + IOUtils.close(fileSystem); } //sort the candidates so they will always be in the same order before being shuffled, for repeatability @@ -202,10 +219,46 @@ public abstract class ElasticsearchRestTestCase extends ElasticsearchIntegration } } + /** + * Returns a new FileSystem to read REST resources, or null if they + * are available from classpath. + */ + @SuppressForbidden(reason = "proper use of URL, hack around a JDK bug") + static FileSystem getFileSystem() throws IOException { + // REST suite handling is currently complicated, with lots of filtering and so on + // For now, to work embedded in a jar, return a ZipFileSystem over the jar contents. + URL codeLocation = FileUtils.class.getProtectionDomain().getCodeSource().getLocation(); + + if (codeLocation.getFile().endsWith(".jar")) { + try { + // hack around a bug in the zipfilesystem implementation before java 9, + // its checkWritable was incorrect and it won't work without write permissions. + // if we add the permission, it will open jars r/w, which is too scary! so copy to a safe r-w location. + Path tmp = Files.createTempFile(null, ".jar"); + try (InputStream in = codeLocation.openStream()) { + Files.copy(in, tmp, StandardCopyOption.REPLACE_EXISTING); + } + return FileSystems.newFileSystem(new URI("jar:" + tmp.toUri()), Collections.emptyMap()); + } catch (URISyntaxException e) { + throw new IOException("couldn't open zipfilesystem: ", e); + } + } else { + return null; + } + } + @BeforeClass public static void initExecutionContext() throws IOException, RestException { String[] specPaths = resolvePathsProperty(REST_TESTS_SPEC, DEFAULT_SPEC_PATH); - RestSpec restSpec = RestSpec.parseFrom(DEFAULT_SPEC_PATH, specPaths); + RestSpec restSpec = null; + FileSystem fileSystem = getFileSystem(); + // don't make a try-with, getFileSystem returns null + // ... and you can't close() the default filesystem + try { + restSpec = RestSpec.parseFrom(fileSystem, DEFAULT_SPEC_PATH, specPaths); + } finally { + IOUtils.close(fileSystem); + } validateSpec(restSpec); restTestExecutionContext = new RestTestExecutionContext(restSpec); } diff --git a/src/test/java/org/elasticsearch/test/rest/spec/RestSpec.java b/src/test/java/org/elasticsearch/test/rest/spec/RestSpec.java index 34b6315fc2e..979bacc26c2 100644 --- a/src/test/java/org/elasticsearch/test/rest/spec/RestSpec.java +++ b/src/test/java/org/elasticsearch/test/rest/spec/RestSpec.java @@ -19,12 +19,14 @@ package org.elasticsearch.test.rest.spec; import com.google.common.collect.Maps; + import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.test.rest.support.FileUtils; import java.io.IOException; import java.io.InputStream; +import java.nio.file.FileSystem; import java.nio.file.Files; import java.nio.file.Path; import java.util.Collection; @@ -54,10 +56,10 @@ public class RestSpec { /** * Parses the complete set of REST spec available under the provided directories */ - public static RestSpec parseFrom(String optionalPathPrefix, String... paths) throws IOException { + public static RestSpec parseFrom(FileSystem fileSystem, String optionalPathPrefix, String... paths) throws IOException { RestSpec restSpec = new RestSpec(); for (String path : paths) { - for (Path jsonFile : FileUtils.findJsonSpec(optionalPathPrefix, path)) { + for (Path jsonFile : FileUtils.findJsonSpec(fileSystem, optionalPathPrefix, path)) { try (InputStream stream = Files.newInputStream(jsonFile)) { XContentParser parser = JsonXContent.jsonXContent.createParser(stream); RestApi restApi = new RestApiParser().parse(parser); diff --git a/src/test/java/org/elasticsearch/test/rest/support/FileUtils.java b/src/test/java/org/elasticsearch/test/rest/support/FileUtils.java index 28c3f597f9f..5e230a6f993 100644 --- a/src/test/java/org/elasticsearch/test/rest/support/FileUtils.java +++ b/src/test/java/org/elasticsearch/test/rest/support/FileUtils.java @@ -25,17 +25,14 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.PathUtils; import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; import java.net.URL; import java.nio.file.DirectoryStream; -import java.nio.file.FileSystems; +import java.nio.file.FileSystem; import java.nio.file.FileVisitResult; import java.nio.file.Files; import java.nio.file.NoSuchFileException; import java.nio.file.NotDirectoryException; import java.nio.file.Path; -import java.nio.file.Paths; import java.nio.file.SimpleFileVisitor; import java.nio.file.attribute.BasicFileAttributes; import java.util.HashSet; @@ -53,10 +50,10 @@ public final class FileUtils { /** * Returns the json files found within the directory provided as argument. - * Files are looked up in the classpath first, then outside of it if not found. + * Files are looked up in the classpath, or optionally from {@code fileSystem} if its not null. */ - public static Set findJsonSpec(String optionalPathPrefix, String path) throws IOException { - Path dir = resolveFile(optionalPathPrefix, path, null); + public static Set findJsonSpec(FileSystem fileSystem, String optionalPathPrefix, String path) throws IOException { + Path dir = resolveFile(fileSystem, optionalPathPrefix, path, null); if (!Files.isDirectory(dir)) { throw new NotDirectoryException(path); @@ -81,37 +78,46 @@ public final class FileUtils { /** * Returns the yaml files found within the paths provided. * Each input path can either be a single file (the .yaml suffix is optional) or a directory. - * Each path is looked up in the classpath first, then outside of it if not found yet. + * Each path is looked up in the classpath, or optionally from {@code fileSystem} if its not null. */ - public static Map> findYamlSuites(final String optionalPathPrefix, final String... paths) throws IOException { + public static Map> findYamlSuites(FileSystem fileSystem, String optionalPathPrefix, final String... paths) throws IOException { Map> yamlSuites = Maps.newHashMap(); for (String path : paths) { - collectFiles(resolveFile(optionalPathPrefix, path, YAML_SUFFIX), YAML_SUFFIX, yamlSuites); + collectFiles(resolveFile(fileSystem, optionalPathPrefix, path, YAML_SUFFIX), YAML_SUFFIX, yamlSuites); } return yamlSuites; } - private static Path resolveFile(String optionalPathPrefix, String path, String optionalFileSuffix) throws IOException { - //try within classpath with and without file suffix (as it could be a single test suite) - URL resource = findResource(path, optionalFileSuffix); - if (resource == null) { - //try within classpath with optional prefix: /rest-api-spec/test (or /rest-api-spec/api) is optional - String newPath = optionalPathPrefix + "/" + path; - resource = findResource(newPath, optionalFileSuffix); - if (resource == null) { - //if it wasn't on classpath we look outside of the classpath - Path file = findFile(path, optionalFileSuffix); - if (!Files.exists(file)) { + private static Path resolveFile(FileSystem fileSystem, String optionalPathPrefix, String path, String optionalFileSuffix) throws IOException { + if (fileSystem != null) { + Path file = findFile(fileSystem, path, optionalFileSuffix); + if (!lenientExists(file)) { + // try with optional prefix: /rest-api-spec/test (or /rest-api-spec/api) is optional + String newPath = optionalPathPrefix + "/" + path; + file = findFile(fileSystem, newPath, optionalFileSuffix); + if (!lenientExists(file)) { throw new NoSuchFileException(path); } - return file; } - } - - try { - return PathUtils.get(resource.toURI()); - } catch (URISyntaxException e) { - throw new RuntimeException(e); + return file; + } else { + //try within classpath + URL resource = findResource(path, optionalFileSuffix); + if (resource == null) { + //try within classpath with optional prefix: /rest-api-spec/test (or /rest-api-spec/api) is optional + String newPath = optionalPathPrefix + "/" + path; + resource = findResource(newPath, optionalFileSuffix); + if (resource == null) { + throw new NoSuchFileException(path); + } + } + try { + return PathUtils.get(resource.toURI()); + } catch (Exception e) { + // some filesystems have REALLY useless exceptions here. + // ZipFileSystem I am looking at you. + throw new RuntimeException("couldn't retrieve URL: " + resource, e); + } } } @@ -125,11 +131,20 @@ public final class FileUtils { } return resource; } + + // used because this test "guesses" from like 4 different places from the filesystem! + private static boolean lenientExists(Path file) { + boolean exists = false; + try { + exists = Files.exists(file); + } catch (SecurityException ok) {} + return exists; + } - private static Path findFile(String path, String optionalFileSuffix) { - Path file = PathUtils.get(path); - if (!Files.exists(file)) { - file = PathUtils.get(path + optionalFileSuffix); + private static Path findFile(FileSystem fileSystem, String path, String optionalFileSuffix) { + Path file = fileSystem.getPath(path); + if (!lenientExists(file)) { + file = fileSystem.getPath(path + optionalFileSuffix); } return file; } diff --git a/src/test/java/org/elasticsearch/test/rest/test/FileUtilsTests.java b/src/test/java/org/elasticsearch/test/rest/test/FileUtilsTests.java index f35daa926a1..10db051e2b1 100644 --- a/src/test/java/org/elasticsearch/test/rest/test/FileUtilsTests.java +++ b/src/test/java/org/elasticsearch/test/rest/test/FileUtilsTests.java @@ -35,29 +35,29 @@ public class FileUtilsTests extends ElasticsearchTestCase { @Test public void testLoadSingleYamlSuite() throws Exception { - Map> yamlSuites = FileUtils.findYamlSuites("/rest-api-spec/test", "/rest-api-spec/test/get/10_basic"); + Map> yamlSuites = FileUtils.findYamlSuites(null, "/rest-api-spec/test", "/rest-api-spec/test/get/10_basic"); assertSingleFile(yamlSuites, "get", "10_basic.yaml"); //the path prefix is optional - yamlSuites = FileUtils.findYamlSuites("/rest-api-spec/test", "get/10_basic.yaml"); + yamlSuites = FileUtils.findYamlSuites(null, "/rest-api-spec/test", "get/10_basic.yaml"); assertSingleFile(yamlSuites, "get", "10_basic.yaml"); //extension .yaml is optional - yamlSuites = FileUtils.findYamlSuites("/rest-api-spec/test", "get/10_basic"); + yamlSuites = FileUtils.findYamlSuites(null, "/rest-api-spec/test", "get/10_basic"); assertSingleFile(yamlSuites, "get", "10_basic.yaml"); } @Test public void testLoadMultipleYamlSuites() throws Exception { //single directory - Map> yamlSuites = FileUtils.findYamlSuites("/rest-api-spec/test", "get"); + Map> yamlSuites = FileUtils.findYamlSuites(null, "/rest-api-spec/test", "get"); assertThat(yamlSuites, notNullValue()); assertThat(yamlSuites.size(), equalTo(1)); assertThat(yamlSuites.containsKey("get"), equalTo(true)); assertThat(yamlSuites.get("get").size(), greaterThan(1)); //multiple directories - yamlSuites = FileUtils.findYamlSuites("/rest-api-spec/test", "get", "index"); + yamlSuites = FileUtils.findYamlSuites(null, "/rest-api-spec/test", "get", "index"); assertThat(yamlSuites, notNullValue()); assertThat(yamlSuites.size(), equalTo(2)); assertThat(yamlSuites.containsKey("get"), equalTo(true)); @@ -66,7 +66,7 @@ public class FileUtilsTests extends ElasticsearchTestCase { assertThat(yamlSuites.get("index").size(), greaterThan(1)); //multiple paths, which can be both directories or yaml test suites (with optional file extension) - yamlSuites = FileUtils.findYamlSuites("/rest-api-spec/test", "indices.optimize/10_basic", "index"); + yamlSuites = FileUtils.findYamlSuites(null, "/rest-api-spec/test", "indices.optimize/10_basic", "index"); assertThat(yamlSuites, notNullValue()); assertThat(yamlSuites.size(), equalTo(2)); assertThat(yamlSuites.containsKey("indices.optimize"), equalTo(true)); @@ -81,22 +81,16 @@ public class FileUtilsTests extends ElasticsearchTestCase { Files.createFile(file); //load from directory outside of the classpath - yamlSuites = FileUtils.findYamlSuites("/rest-api-spec/test", "get/10_basic", dir.toAbsolutePath().toString()); + yamlSuites = FileUtils.findYamlSuites(dir.getFileSystem(), "/rest-api-spec/test", dir.toAbsolutePath().toString()); assertThat(yamlSuites, notNullValue()); - assertThat(yamlSuites.size(), equalTo(2)); - assertThat(yamlSuites.containsKey("get"), equalTo(true)); - assertThat(yamlSuites.get("get").size(), equalTo(1)); - assertSingleFile(yamlSuites.get("get"), "get", "10_basic.yaml"); + assertThat(yamlSuites.size(), equalTo(1)); assertThat(yamlSuites.containsKey(dir.getFileName().toString()), equalTo(true)); assertSingleFile(yamlSuites.get(dir.getFileName().toString()), dir.getFileName().toString(), file.getFileName().toString()); //load from external file (optional extension) - yamlSuites = FileUtils.findYamlSuites("/rest-api-spec/test", "get/10_basic", dir.resolve("test_loading").toAbsolutePath().toString()); + yamlSuites = FileUtils.findYamlSuites(dir.getFileSystem(), "/rest-api-spec/test", dir.resolve("test_loading").toAbsolutePath().toString()); assertThat(yamlSuites, notNullValue()); - assertThat(yamlSuites.size(), equalTo(2)); - assertThat(yamlSuites.containsKey("get"), equalTo(true)); - assertThat(yamlSuites.get("get").size(), equalTo(1)); - assertSingleFile(yamlSuites.get("get"), "get", "10_basic.yaml"); + assertThat(yamlSuites.size(), equalTo(1)); assertThat(yamlSuites.containsKey(dir.getFileName().toString()), equalTo(true)); assertSingleFile(yamlSuites.get(dir.getFileName().toString()), dir.getFileName().toString(), file.getFileName().toString()); } From fc73540aacc0e73a2fe878dbf4f25ae59bcb485e Mon Sep 17 00:00:00 2001 From: Robert Muir Date: Fri, 15 May 2015 16:15:23 -0400 Subject: [PATCH 25/56] include rest tests in test-framework.jar Closes #11192 which I accidentally already closed. Squashed commit of the following: commit f23faccddc2a77a880841da4c89c494edaa2aa46 Author: Robert Muir Date: Fri May 15 16:04:55 2015 -0400 Simplify this FileUtils even more: its either from the filesystem, or the classpath, not both. Its already trying 4 different combinations of crazy paths for either of these anyway. commit c7016c8a2b5a6043e2ded4b48b160821ba196974 Author: Robert Muir Date: Fri May 15 14:21:37 2015 -0400 include rest tests in test-framework jar --- pom.xml | 4 +- .../test/rest/ElasticsearchRestTestCase.java | 81 +++++++++++++++---- .../test/rest/spec/RestSpec.java | 6 +- .../test/rest/support/FileUtils.java | 79 ++++++++++-------- .../test/rest/test/FileUtilsTests.java | 26 +++--- 5 files changed, 130 insertions(+), 66 deletions(-) diff --git a/pom.xml b/pom.xml index 4b36f6c44ef..61966eb58a5 100644 --- a/pom.xml +++ b/pom.xml @@ -787,8 +787,6 @@ org/elasticsearch/test/test/**/* - - true @@ -1519,6 +1517,7 @@ dev-tools/forbidden/all-signatures.txt ${forbidden.test.signatures} + **.SuppressForbidden test-compile @@ -1539,6 +1538,7 @@ + rest-api-spec/**/* org/elasticsearch/test/**/* org/elasticsearch/bootstrap/BootstrapForTesting.class org/elasticsearch/common/cli/CliToolTestCase.class diff --git a/src/test/java/org/elasticsearch/test/rest/ElasticsearchRestTestCase.java b/src/test/java/org/elasticsearch/test/rest/ElasticsearchRestTestCase.java index b7b207a6b11..9661e3950c6 100644 --- a/src/test/java/org/elasticsearch/test/rest/ElasticsearchRestTestCase.java +++ b/src/test/java/org/elasticsearch/test/rest/ElasticsearchRestTestCase.java @@ -23,11 +23,14 @@ import com.carrotsearch.randomizedtesting.RandomizedTest; import com.carrotsearch.randomizedtesting.annotations.TestGroup; import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite; import com.google.common.collect.Lists; + import org.apache.lucene.util.LuceneTestCase.Slow; import org.apache.lucene.util.LuceneTestCase.SuppressCodecs; import org.apache.lucene.util.LuceneTestCase.SuppressFsync; +import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.TimeUnits; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.SuppressForbidden; import org.elasticsearch.common.io.PathUtils; import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.Settings; @@ -48,9 +51,17 @@ import org.junit.BeforeClass; import org.junit.Test; import java.io.IOException; +import java.io.InputStream; import java.lang.annotation.*; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URL; +import java.nio.file.FileSystem; +import java.nio.file.FileSystems; +import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.PathMatcher; +import java.nio.file.StandardCopyOption; import java.util.*; /** @@ -158,23 +169,29 @@ public abstract class ElasticsearchRestTestCase extends ElasticsearchIntegration } private static List collectTestCandidates(int id, int count) throws RestTestParseException, IOException { - String[] paths = resolvePathsProperty(REST_TESTS_SUITE, DEFAULT_TESTS_PATH); - Map> yamlSuites = FileUtils.findYamlSuites(DEFAULT_TESTS_PATH, paths); - List testCandidates = Lists.newArrayList(); - RestTestSuiteParser restTestSuiteParser = new RestTestSuiteParser(); - //yaml suites are grouped by directory (effectively by api) - for (String api : yamlSuites.keySet()) { - List yamlFiles = Lists.newArrayList(yamlSuites.get(api)); - for (Path yamlFile : yamlFiles) { - String key = api + yamlFile.getFileName().toString(); - if (mustExecute(key, id, count)) { - RestTestSuite restTestSuite = restTestSuiteParser.parse(api, yamlFile); - for (TestSection testSection : restTestSuite.getTestSections()) { - testCandidates.add(new RestTestCandidate(restTestSuite, testSection)); + FileSystem fileSystem = getFileSystem(); + // don't make a try-with, getFileSystem returns null + // ... and you can't close() the default filesystem + try { + String[] paths = resolvePathsProperty(REST_TESTS_SUITE, DEFAULT_TESTS_PATH); + Map> yamlSuites = FileUtils.findYamlSuites(fileSystem, DEFAULT_TESTS_PATH, paths); + RestTestSuiteParser restTestSuiteParser = new RestTestSuiteParser(); + //yaml suites are grouped by directory (effectively by api) + for (String api : yamlSuites.keySet()) { + List yamlFiles = Lists.newArrayList(yamlSuites.get(api)); + for (Path yamlFile : yamlFiles) { + String key = api + yamlFile.getFileName().toString(); + if (mustExecute(key, id, count)) { + RestTestSuite restTestSuite = restTestSuiteParser.parse(api, yamlFile); + for (TestSection testSection : restTestSuite.getTestSections()) { + testCandidates.add(new RestTestCandidate(restTestSuite, testSection)); + } } } } + } finally { + IOUtils.close(fileSystem); } //sort the candidates so they will always be in the same order before being shuffled, for repeatability @@ -202,10 +219,46 @@ public abstract class ElasticsearchRestTestCase extends ElasticsearchIntegration } } + /** + * Returns a new FileSystem to read REST resources, or null if they + * are available from classpath. + */ + @SuppressForbidden(reason = "proper use of URL, hack around a JDK bug") + static FileSystem getFileSystem() throws IOException { + // REST suite handling is currently complicated, with lots of filtering and so on + // For now, to work embedded in a jar, return a ZipFileSystem over the jar contents. + URL codeLocation = FileUtils.class.getProtectionDomain().getCodeSource().getLocation(); + + if (codeLocation.getFile().endsWith(".jar")) { + try { + // hack around a bug in the zipfilesystem implementation before java 9, + // its checkWritable was incorrect and it won't work without write permissions. + // if we add the permission, it will open jars r/w, which is too scary! so copy to a safe r-w location. + Path tmp = Files.createTempFile(null, ".jar"); + try (InputStream in = codeLocation.openStream()) { + Files.copy(in, tmp, StandardCopyOption.REPLACE_EXISTING); + } + return FileSystems.newFileSystem(new URI("jar:" + tmp.toUri()), Collections.emptyMap()); + } catch (URISyntaxException e) { + throw new IOException("couldn't open zipfilesystem: ", e); + } + } else { + return null; + } + } + @BeforeClass public static void initExecutionContext() throws IOException, RestException { String[] specPaths = resolvePathsProperty(REST_TESTS_SPEC, DEFAULT_SPEC_PATH); - RestSpec restSpec = RestSpec.parseFrom(DEFAULT_SPEC_PATH, specPaths); + RestSpec restSpec = null; + FileSystem fileSystem = getFileSystem(); + // don't make a try-with, getFileSystem returns null + // ... and you can't close() the default filesystem + try { + restSpec = RestSpec.parseFrom(fileSystem, DEFAULT_SPEC_PATH, specPaths); + } finally { + IOUtils.close(fileSystem); + } validateSpec(restSpec); restTestExecutionContext = new RestTestExecutionContext(restSpec); } diff --git a/src/test/java/org/elasticsearch/test/rest/spec/RestSpec.java b/src/test/java/org/elasticsearch/test/rest/spec/RestSpec.java index 34b6315fc2e..979bacc26c2 100644 --- a/src/test/java/org/elasticsearch/test/rest/spec/RestSpec.java +++ b/src/test/java/org/elasticsearch/test/rest/spec/RestSpec.java @@ -19,12 +19,14 @@ package org.elasticsearch.test.rest.spec; import com.google.common.collect.Maps; + import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.test.rest.support.FileUtils; import java.io.IOException; import java.io.InputStream; +import java.nio.file.FileSystem; import java.nio.file.Files; import java.nio.file.Path; import java.util.Collection; @@ -54,10 +56,10 @@ public class RestSpec { /** * Parses the complete set of REST spec available under the provided directories */ - public static RestSpec parseFrom(String optionalPathPrefix, String... paths) throws IOException { + public static RestSpec parseFrom(FileSystem fileSystem, String optionalPathPrefix, String... paths) throws IOException { RestSpec restSpec = new RestSpec(); for (String path : paths) { - for (Path jsonFile : FileUtils.findJsonSpec(optionalPathPrefix, path)) { + for (Path jsonFile : FileUtils.findJsonSpec(fileSystem, optionalPathPrefix, path)) { try (InputStream stream = Files.newInputStream(jsonFile)) { XContentParser parser = JsonXContent.jsonXContent.createParser(stream); RestApi restApi = new RestApiParser().parse(parser); diff --git a/src/test/java/org/elasticsearch/test/rest/support/FileUtils.java b/src/test/java/org/elasticsearch/test/rest/support/FileUtils.java index 28c3f597f9f..5e230a6f993 100644 --- a/src/test/java/org/elasticsearch/test/rest/support/FileUtils.java +++ b/src/test/java/org/elasticsearch/test/rest/support/FileUtils.java @@ -25,17 +25,14 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.PathUtils; import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; import java.net.URL; import java.nio.file.DirectoryStream; -import java.nio.file.FileSystems; +import java.nio.file.FileSystem; import java.nio.file.FileVisitResult; import java.nio.file.Files; import java.nio.file.NoSuchFileException; import java.nio.file.NotDirectoryException; import java.nio.file.Path; -import java.nio.file.Paths; import java.nio.file.SimpleFileVisitor; import java.nio.file.attribute.BasicFileAttributes; import java.util.HashSet; @@ -53,10 +50,10 @@ public final class FileUtils { /** * Returns the json files found within the directory provided as argument. - * Files are looked up in the classpath first, then outside of it if not found. + * Files are looked up in the classpath, or optionally from {@code fileSystem} if its not null. */ - public static Set findJsonSpec(String optionalPathPrefix, String path) throws IOException { - Path dir = resolveFile(optionalPathPrefix, path, null); + public static Set findJsonSpec(FileSystem fileSystem, String optionalPathPrefix, String path) throws IOException { + Path dir = resolveFile(fileSystem, optionalPathPrefix, path, null); if (!Files.isDirectory(dir)) { throw new NotDirectoryException(path); @@ -81,37 +78,46 @@ public final class FileUtils { /** * Returns the yaml files found within the paths provided. * Each input path can either be a single file (the .yaml suffix is optional) or a directory. - * Each path is looked up in the classpath first, then outside of it if not found yet. + * Each path is looked up in the classpath, or optionally from {@code fileSystem} if its not null. */ - public static Map> findYamlSuites(final String optionalPathPrefix, final String... paths) throws IOException { + public static Map> findYamlSuites(FileSystem fileSystem, String optionalPathPrefix, final String... paths) throws IOException { Map> yamlSuites = Maps.newHashMap(); for (String path : paths) { - collectFiles(resolveFile(optionalPathPrefix, path, YAML_SUFFIX), YAML_SUFFIX, yamlSuites); + collectFiles(resolveFile(fileSystem, optionalPathPrefix, path, YAML_SUFFIX), YAML_SUFFIX, yamlSuites); } return yamlSuites; } - private static Path resolveFile(String optionalPathPrefix, String path, String optionalFileSuffix) throws IOException { - //try within classpath with and without file suffix (as it could be a single test suite) - URL resource = findResource(path, optionalFileSuffix); - if (resource == null) { - //try within classpath with optional prefix: /rest-api-spec/test (or /rest-api-spec/api) is optional - String newPath = optionalPathPrefix + "/" + path; - resource = findResource(newPath, optionalFileSuffix); - if (resource == null) { - //if it wasn't on classpath we look outside of the classpath - Path file = findFile(path, optionalFileSuffix); - if (!Files.exists(file)) { + private static Path resolveFile(FileSystem fileSystem, String optionalPathPrefix, String path, String optionalFileSuffix) throws IOException { + if (fileSystem != null) { + Path file = findFile(fileSystem, path, optionalFileSuffix); + if (!lenientExists(file)) { + // try with optional prefix: /rest-api-spec/test (or /rest-api-spec/api) is optional + String newPath = optionalPathPrefix + "/" + path; + file = findFile(fileSystem, newPath, optionalFileSuffix); + if (!lenientExists(file)) { throw new NoSuchFileException(path); } - return file; } - } - - try { - return PathUtils.get(resource.toURI()); - } catch (URISyntaxException e) { - throw new RuntimeException(e); + return file; + } else { + //try within classpath + URL resource = findResource(path, optionalFileSuffix); + if (resource == null) { + //try within classpath with optional prefix: /rest-api-spec/test (or /rest-api-spec/api) is optional + String newPath = optionalPathPrefix + "/" + path; + resource = findResource(newPath, optionalFileSuffix); + if (resource == null) { + throw new NoSuchFileException(path); + } + } + try { + return PathUtils.get(resource.toURI()); + } catch (Exception e) { + // some filesystems have REALLY useless exceptions here. + // ZipFileSystem I am looking at you. + throw new RuntimeException("couldn't retrieve URL: " + resource, e); + } } } @@ -125,11 +131,20 @@ public final class FileUtils { } return resource; } + + // used because this test "guesses" from like 4 different places from the filesystem! + private static boolean lenientExists(Path file) { + boolean exists = false; + try { + exists = Files.exists(file); + } catch (SecurityException ok) {} + return exists; + } - private static Path findFile(String path, String optionalFileSuffix) { - Path file = PathUtils.get(path); - if (!Files.exists(file)) { - file = PathUtils.get(path + optionalFileSuffix); + private static Path findFile(FileSystem fileSystem, String path, String optionalFileSuffix) { + Path file = fileSystem.getPath(path); + if (!lenientExists(file)) { + file = fileSystem.getPath(path + optionalFileSuffix); } return file; } diff --git a/src/test/java/org/elasticsearch/test/rest/test/FileUtilsTests.java b/src/test/java/org/elasticsearch/test/rest/test/FileUtilsTests.java index f35daa926a1..10db051e2b1 100644 --- a/src/test/java/org/elasticsearch/test/rest/test/FileUtilsTests.java +++ b/src/test/java/org/elasticsearch/test/rest/test/FileUtilsTests.java @@ -35,29 +35,29 @@ public class FileUtilsTests extends ElasticsearchTestCase { @Test public void testLoadSingleYamlSuite() throws Exception { - Map> yamlSuites = FileUtils.findYamlSuites("/rest-api-spec/test", "/rest-api-spec/test/get/10_basic"); + Map> yamlSuites = FileUtils.findYamlSuites(null, "/rest-api-spec/test", "/rest-api-spec/test/get/10_basic"); assertSingleFile(yamlSuites, "get", "10_basic.yaml"); //the path prefix is optional - yamlSuites = FileUtils.findYamlSuites("/rest-api-spec/test", "get/10_basic.yaml"); + yamlSuites = FileUtils.findYamlSuites(null, "/rest-api-spec/test", "get/10_basic.yaml"); assertSingleFile(yamlSuites, "get", "10_basic.yaml"); //extension .yaml is optional - yamlSuites = FileUtils.findYamlSuites("/rest-api-spec/test", "get/10_basic"); + yamlSuites = FileUtils.findYamlSuites(null, "/rest-api-spec/test", "get/10_basic"); assertSingleFile(yamlSuites, "get", "10_basic.yaml"); } @Test public void testLoadMultipleYamlSuites() throws Exception { //single directory - Map> yamlSuites = FileUtils.findYamlSuites("/rest-api-spec/test", "get"); + Map> yamlSuites = FileUtils.findYamlSuites(null, "/rest-api-spec/test", "get"); assertThat(yamlSuites, notNullValue()); assertThat(yamlSuites.size(), equalTo(1)); assertThat(yamlSuites.containsKey("get"), equalTo(true)); assertThat(yamlSuites.get("get").size(), greaterThan(1)); //multiple directories - yamlSuites = FileUtils.findYamlSuites("/rest-api-spec/test", "get", "index"); + yamlSuites = FileUtils.findYamlSuites(null, "/rest-api-spec/test", "get", "index"); assertThat(yamlSuites, notNullValue()); assertThat(yamlSuites.size(), equalTo(2)); assertThat(yamlSuites.containsKey("get"), equalTo(true)); @@ -66,7 +66,7 @@ public class FileUtilsTests extends ElasticsearchTestCase { assertThat(yamlSuites.get("index").size(), greaterThan(1)); //multiple paths, which can be both directories or yaml test suites (with optional file extension) - yamlSuites = FileUtils.findYamlSuites("/rest-api-spec/test", "indices.optimize/10_basic", "index"); + yamlSuites = FileUtils.findYamlSuites(null, "/rest-api-spec/test", "indices.optimize/10_basic", "index"); assertThat(yamlSuites, notNullValue()); assertThat(yamlSuites.size(), equalTo(2)); assertThat(yamlSuites.containsKey("indices.optimize"), equalTo(true)); @@ -81,22 +81,16 @@ public class FileUtilsTests extends ElasticsearchTestCase { Files.createFile(file); //load from directory outside of the classpath - yamlSuites = FileUtils.findYamlSuites("/rest-api-spec/test", "get/10_basic", dir.toAbsolutePath().toString()); + yamlSuites = FileUtils.findYamlSuites(dir.getFileSystem(), "/rest-api-spec/test", dir.toAbsolutePath().toString()); assertThat(yamlSuites, notNullValue()); - assertThat(yamlSuites.size(), equalTo(2)); - assertThat(yamlSuites.containsKey("get"), equalTo(true)); - assertThat(yamlSuites.get("get").size(), equalTo(1)); - assertSingleFile(yamlSuites.get("get"), "get", "10_basic.yaml"); + assertThat(yamlSuites.size(), equalTo(1)); assertThat(yamlSuites.containsKey(dir.getFileName().toString()), equalTo(true)); assertSingleFile(yamlSuites.get(dir.getFileName().toString()), dir.getFileName().toString(), file.getFileName().toString()); //load from external file (optional extension) - yamlSuites = FileUtils.findYamlSuites("/rest-api-spec/test", "get/10_basic", dir.resolve("test_loading").toAbsolutePath().toString()); + yamlSuites = FileUtils.findYamlSuites(dir.getFileSystem(), "/rest-api-spec/test", dir.resolve("test_loading").toAbsolutePath().toString()); assertThat(yamlSuites, notNullValue()); - assertThat(yamlSuites.size(), equalTo(2)); - assertThat(yamlSuites.containsKey("get"), equalTo(true)); - assertThat(yamlSuites.get("get").size(), equalTo(1)); - assertSingleFile(yamlSuites.get("get"), "get", "10_basic.yaml"); + assertThat(yamlSuites.size(), equalTo(1)); assertThat(yamlSuites.containsKey(dir.getFileName().toString()), equalTo(true)); assertSingleFile(yamlSuites.get(dir.getFileName().toString()), dir.getFileName().toString(), file.getFileName().toString()); } From b2a0ed387841fcffd18a8c587ed5d10dba8c80b2 Mon Sep 17 00:00:00 2001 From: Robert Muir Date: Fri, 15 May 2015 16:33:24 -0400 Subject: [PATCH 26/56] allow mockito to be used in tests --- src/main/resources/org/elasticsearch/bootstrap/security.policy | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/main/resources/org/elasticsearch/bootstrap/security.policy b/src/main/resources/org/elasticsearch/bootstrap/security.policy index ef79e807cd7..51c66f9cc9e 100644 --- a/src/main/resources/org/elasticsearch/bootstrap/security.policy +++ b/src/main/resources/org/elasticsearch/bootstrap/security.policy @@ -98,4 +98,7 @@ grant { // needed by JDKESLoggerTests permission java.util.logging.LoggingPermission "control"; + + // needed by Mockito + permission java.lang.RuntimePermission "reflectionFactoryAccess"; }; From 22ae561bfd39c4941c36729bda9aabff0522bb81 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Thu, 14 May 2015 12:13:59 -0600 Subject: [PATCH 27/56] [GATEWAY] When using `recover_on_any_node`, respect Deciders When in a shared filesystem environment and recovering the primary to any node. We should respect the allocation deciders if possible (still force-allocting to another node if there aren't any "YES" decisions). The AllocationDeciders should take precedence over the shard state version when force-allocating an unassigned primary shard. --- .../gateway/GatewayAllocator.java | 85 ++++++++++++----- .../index/IndexWithShadowReplicasTests.java | 92 +++++++++++++++++-- 2 files changed, 145 insertions(+), 32 deletions(-) diff --git a/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java b/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java index f77eeb253ac..c0ae88a5102 100644 --- a/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java +++ b/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java @@ -23,8 +23,10 @@ import com.carrotsearch.hppc.ObjectLongHashMap; import com.carrotsearch.hppc.ObjectHashSet; import com.carrotsearch.hppc.cursors.ObjectCursor; import com.carrotsearch.hppc.predicates.ObjectPredicate; +import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import org.apache.lucene.util.CollectionUtil; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.FailedNodeException; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -45,14 +47,13 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.index.settings.IndexSettings; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.StoreFileMetaData; import org.elasticsearch.indices.store.TransportNodesListShardStoreMetaData; import org.elasticsearch.transport.ConnectTransportException; -import java.util.Iterator; -import java.util.Map; -import java.util.Set; +import java.util.*; import java.util.concurrent.ConcurrentMap; /** @@ -101,6 +102,15 @@ public class GatewayAllocator extends AbstractComponent { } } + /** + * Return {@code true} if the index is configured to allow shards to be + * recovered on any node + */ + private boolean recoverOnAnyNode(@IndexSettings Settings idxSettings) { + return IndexMetaData.isOnSharedFilesystem(idxSettings) && + idxSettings.getAsBoolean(IndexMetaData.SETTING_SHARED_FS_ALLOW_RECOVERY_ON_ANY_NODE, false); + } + public boolean allocateUnassigned(RoutingAllocation allocation) { boolean changed = false; DiscoveryNodes nodes = allocation.nodes(); @@ -125,11 +135,13 @@ public class GatewayAllocator extends AbstractComponent { int numberOfAllocationsFound = 0; long highestVersion = -1; - Set nodesWithHighestVersion = Sets.newHashSet(); + final Map nodesWithVersion = Maps.newHashMap(); assert !nodesState.containsKey(null); final Object[] keys = nodesState.keys; final long[] values = nodesState.values; + IndexMetaData indexMetaData = routingNodes.metaData().index(shard.index()); + Settings idxSettings = indexMetaData.settings(); for (int i = 0; i < keys.length; i++) { if (keys[i] == null) { continue; @@ -141,29 +153,63 @@ public class GatewayAllocator extends AbstractComponent { if (allocation.shouldIgnoreShardForNode(shard.shardId(), node.id())) { continue; } - if (version != -1) { + if (recoverOnAnyNode(idxSettings)) { numberOfAllocationsFound++; - if (highestVersion == -1) { - nodesWithHighestVersion.add(node); + if (version > highestVersion) { highestVersion = version; - } else { - if (version > highestVersion) { - nodesWithHighestVersion.clear(); - nodesWithHighestVersion.add(node); - highestVersion = version; - } else if (version == highestVersion) { - nodesWithHighestVersion.add(node); - } + } + // We always put the node without clearing the map + nodesWithVersion.put(node, version); + } else if (version != -1) { + numberOfAllocationsFound++; + // If we've found a new "best" candidate, clear the + // current candidates and add it + if (version > highestVersion) { + highestVersion = version; + nodesWithVersion.clear(); + nodesWithVersion.put(node, version); + } else if (version == highestVersion) { + // If the candidate is the same, add it to the + // list, but keep the current candidate + nodesWithVersion.put(node, version); } } } + // Now that we have a map of nodes to versions along with the + // number of allocations found (and not ignored), we need to sort + // it so the node with the highest version is at the beginning + List nodesWithHighestVersion = Lists.newArrayList(); + nodesWithHighestVersion.addAll(nodesWithVersion.keySet()); + CollectionUtil.timSort(nodesWithHighestVersion, new Comparator() { + @Override + public int compare(DiscoveryNode o1, DiscoveryNode o2) { + return Long.compare(nodesWithVersion.get(o2), nodesWithVersion.get(o1)); + } + }); + + if (logger.isDebugEnabled()) { + logger.debug("[{}][{}] found {} allocations of {}, highest version: [{}]", + shard.index(), shard.id(), numberOfAllocationsFound, shard, highestVersion); + } + if (logger.isTraceEnabled()) { + StringBuilder sb = new StringBuilder("["); + for (DiscoveryNode n : nodesWithHighestVersion) { + sb.append("["); + sb.append(n.getName()); + sb.append("]"); + sb.append(" -> "); + sb.append(nodesWithVersion.get(n)); + sb.append(", "); + } + sb.append("]"); + logger.trace("{} candidates for allocation: {}", shard, sb.toString()); + } // check if the counts meets the minimum set int requiredAllocation = 1; // if we restore from a repository one copy is more then enough if (shard.restoreSource() == null) { try { - IndexMetaData indexMetaData = routingNodes.metaData().index(shard.index()); String initialShards = indexMetaData.settings().get(INDEX_RECOVERY_INITIAL_SHARDS, settings.get(INDEX_RECOVERY_INITIAL_SHARDS, this.initialShards)); if ("quorum".equals(initialShards)) { if (indexMetaData.numberOfReplicas() > 1) { @@ -415,13 +461,6 @@ public class GatewayAllocator extends AbstractComponent { for (TransportNodesListGatewayStartedShards.NodeGatewayStartedShards nodeShardState : response) { long version = nodeShardState.version(); - Settings idxSettings = indexMetaData.settings(); - if (IndexMetaData.isOnSharedFilesystem(idxSettings) && - idxSettings.getAsBoolean(IndexMetaData.SETTING_SHARED_FS_ALLOW_RECOVERY_ON_ANY_NODE, false)) { - // Shared filesystems use 0 as a minimum shard state, which - // means that the shard can be allocated to any node - version = Math.max(0, version); - } // -1 version means it does not exists, which is what the API returns, and what we expect to logger.trace("[{}] on node [{}] has version [{}] of shard", shard, nodeShardState.getNode(), version); diff --git a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java index d3a1a00e98b..125046ce43f 100644 --- a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java +++ b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java @@ -52,6 +52,7 @@ import java.nio.file.Path; import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -612,20 +613,67 @@ public class IndexWithShadowReplicasTests extends ElasticsearchIntegrationTest { assertThat(hits[3].field("foo").getValue().toString(), equalTo("foo")); } + /** wait until none of the nodes have shards allocated on them */ + private void assertNoShardsOn(final List nodeList) throws Exception { + assertBusy(new Runnable() { + @Override + public void run() { + ClusterStateResponse resp = client().admin().cluster().prepareState().get(); + RoutingNodes nodes = resp.getState().getRoutingNodes(); + for (RoutingNode node : nodes) { + logger.info("--> node {} has {} shards", node.node().getName(), node.numberOfOwningShards()); + if (nodeList.contains(node.node().getName())) { + assertThat("no shards on node", node.numberOfOwningShards(), equalTo(0)); + } + } + } + }); + } + + /** wait until the node has the specified number of shards allocated on it */ + private void assertShardCountOn(final String nodeName, final int shardCount) throws Exception { + assertBusy(new Runnable() { + @Override + public void run() { + ClusterStateResponse resp = client().admin().cluster().prepareState().get(); + RoutingNodes nodes = resp.getState().getRoutingNodes(); + for (RoutingNode node : nodes) { + logger.info("--> node {} has {} shards", node.node().getName(), node.numberOfOwningShards()); + if (nodeName.equals(node.node().getName())) { + assertThat(node.numberOfOwningShards(), equalTo(shardCount)); + } + } + } + }); + } + @Test public void testIndexOnSharedFSRecoversToAnyNode() throws Exception { Settings nodeSettings = nodeSettings(); + Settings fooSettings = ImmutableSettings.builder().put(nodeSettings).put("node.affinity", "foo").build(); + Settings barSettings = ImmutableSettings.builder().put(nodeSettings).put("node.affinity", "bar").build(); - internalCluster().startNode(nodeSettings); + final Future> fooNodes = internalCluster().startNodesAsync(2, fooSettings); + final Future> barNodes = internalCluster().startNodesAsync(2, barSettings); + fooNodes.get(); + barNodes.get(); Path dataPath = createTempDir(); String IDX = "test"; + Settings includeFoo = ImmutableSettings.builder() + .put("index.routing.allocation.include.affinity", "foo") + .build(); + Settings includeBar = ImmutableSettings.builder() + .put("index.routing.allocation.include.affinity", "bar") + .build(); + Settings idxSettings = ImmutableSettings.builder() .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 5) .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) .put(IndexMetaData.SETTING_DATA_PATH, dataPath.toAbsolutePath().toString()) .put(IndexMetaData.SETTING_SHARED_FILESYSTEM, true) .put(IndexMetaData.SETTING_SHARED_FS_ALLOW_RECOVERY_ON_ANY_NODE, true) + .put(includeFoo) // start with requiring the shards on "foo" .build(); // only one node, so all primaries will end up on node1 @@ -637,17 +685,43 @@ public class IndexWithShadowReplicasTests extends ElasticsearchIntegrationTest { client().prepareIndex(IDX, "doc", "2").setSource("foo", "bar").get(); client().prepareIndex(IDX, "doc", "3").setSource("foo", "baz").get(); client().prepareIndex(IDX, "doc", "4").setSource("foo", "eggplant").get(); + flushAndRefresh(IDX); - // start a second node - internalCluster().startNode(nodeSettings); + // put shards on "bar" + client().admin().indices().prepareUpdateSettings(IDX).setSettings(includeBar).get(); - // node1 is master, stop that one, since we only have primaries, - // usually this would mean data loss, but not on shared fs! - internalCluster().stopCurrentMasterNode(); + // wait for the shards to move from "foo" nodes to "bar" nodes + assertNoShardsOn(fooNodes.get()); + // put shards back on "foo" + client().admin().indices().prepareUpdateSettings(IDX).setSettings(includeFoo).get(); + + // wait for the shards to move from "bar" nodes to "foo" nodes + assertNoShardsOn(barNodes.get()); + + // Stop a foo node + logger.info("--> stopping first 'foo' node"); + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(fooNodes.get().get(0))); + + // Ensure that the other foo node has all the shards now + assertShardCountOn(fooNodes.get().get(1), 5); + + // Assert no shards on the "bar" nodes + assertNoShardsOn(barNodes.get()); + + // Stop the second "foo" node + logger.info("--> stopping second 'foo' node"); + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(fooNodes.get().get(1))); + + // The index should still be able to be allocated (on the "bar" nodes), + // all the "foo" nodes are gone ensureGreen(IDX); - refresh(); - SearchResponse resp = client().prepareSearch(IDX).setQuery(matchAllQuery()).addFieldDataField("foo").addSort("foo", SortOrder.ASC).get(); - assertHitCount(resp, 4); + + // Start another "foo" node and make sure the index moves back + logger.info("--> starting additional 'foo' node"); + String newFooNode = internalCluster().startNode(fooSettings); + + assertShardCountOn(newFooNode, 5); + assertNoShardsOn(barNodes.get()); } } From 1038933832afd66144fbbee3e285e711c829edff Mon Sep 17 00:00:00 2001 From: Igor Motov Date: Fri, 15 May 2015 19:54:35 -0400 Subject: [PATCH 28/56] Snapshot/Restore: Fix cluster state task name for update snapshot task This commit fixes the name of the upated_snapshot task from something like "update_snapshot [org.elasticsearch.cluster.metadata.SnapshotMetaData$Entry@de00bc50]" to a more readable "update_snapshot [test-snap]" --- .../cluster/snapshots/create/TransportCreateSnapshotAction.java | 2 +- src/main/java/org/elasticsearch/snapshots/SnapshotsService.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java b/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java index 0441ec41dd2..5d757e62c35 100644 --- a/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java +++ b/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java @@ -66,7 +66,7 @@ public class TransportCreateSnapshotAction extends TransportMasterNodeOperationA @Override protected void masterOperation(final CreateSnapshotRequest request, ClusterState state, final ActionListener listener) { SnapshotsService.SnapshotRequest snapshotRequest = - new SnapshotsService.SnapshotRequest("create_snapshot[" + request.snapshot() + "]", request.snapshot(), request.repository()) + new SnapshotsService.SnapshotRequest("create_snapshot [" + request.snapshot() + "]", request.snapshot(), request.repository()) .indices(request.indices()) .indicesOptions(request.indicesOptions()) .partial(request.partial()) diff --git a/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index bc00f9c7f18..2bddf4960f5 100644 --- a/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -308,7 +308,7 @@ public class SnapshotsService extends AbstractLifecycleComponent Date: Sun, 17 May 2015 20:26:13 +0200 Subject: [PATCH 29/56] [TEST] use correct index name created for this test --- .../elasticsearch/search/suggest/SuggestSearchTests.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/test/java/org/elasticsearch/search/suggest/SuggestSearchTests.java b/src/test/java/org/elasticsearch/search/suggest/SuggestSearchTests.java index a362b435af8..ebc8fcf0713 100644 --- a/src/test/java/org/elasticsearch/search/suggest/SuggestSearchTests.java +++ b/src/test/java/org/elasticsearch/search/suggest/SuggestSearchTests.java @@ -764,9 +764,9 @@ public class SuggestSearchTests extends ElasticsearchIntegrationTest { public void testDifferentShardSize() throws Exception { createIndex("test"); ensureGreen(); - indexRandom(true, client().prepareIndex("text", "type1", "1").setSource("field1", "foobar1").setRouting("1"), - client().prepareIndex("text", "type1", "2").setSource("field1", "foobar2").setRouting("2"), - client().prepareIndex("text", "type1", "3").setSource("field1", "foobar3").setRouting("3")); + indexRandom(true, client().prepareIndex("test", "type1", "1").setSource("field1", "foobar1").setRouting("1"), + client().prepareIndex("test", "type1", "2").setSource("field1", "foobar2").setRouting("2"), + client().prepareIndex("test", "type1", "3").setSource("field1", "foobar3").setRouting("3")); Suggest suggest = searchSuggest( "foobar", termSuggestion("simple") From b16e47c5fb86ae79dca1c18d8a1c2e897e594ddc Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Mon, 18 May 2015 08:38:10 +0200 Subject: [PATCH 30/56] Tests: Mark IndexCacheableQueryTests.testCache as @AwaitsFix. --- .../elasticsearch/common/lucene/IndexCacheableQueryTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/java/org/elasticsearch/common/lucene/IndexCacheableQueryTests.java b/src/test/java/org/elasticsearch/common/lucene/IndexCacheableQueryTests.java index 5d418d185d2..d1aa163667a 100644 --- a/src/test/java/org/elasticsearch/common/lucene/IndexCacheableQueryTests.java +++ b/src/test/java/org/elasticsearch/common/lucene/IndexCacheableQueryTests.java @@ -20,7 +20,6 @@ package org.elasticsearch.common.lucene; import org.apache.lucene.document.Document; -import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.MultiReader; @@ -94,6 +93,7 @@ public class IndexCacheableQueryTests extends ElasticsearchTestCase { QueryUtils.checkUnequal(rewritten, rewritten2); } + @AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/LUCENE-6483") public void testCache() throws IOException { Directory dir = newDirectory(); LRUQueryCache cache = new LRUQueryCache(10000, Long.MAX_VALUE); From 8b8ba9a44f6c39edb40c524a58ed1fc7850b575d Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 18 May 2015 09:54:04 +0200 Subject: [PATCH 31/56] Implement toXContent on ShardOpertionFailureException ShardOperationFailureException implementations alread provide structured exception support but it's not yet exposed on the interface. This change allows nice rendering of structured REST exceptions also if searches fail on only a subset of the shards etc. Closes #11017 --- .../elasticsearch/ElasticsearchException.java | 8 +-- .../org/elasticsearch/ExceptionsHelper.java | 61 ++++++++++++++++++ .../action/ActionWriteResponse.java | 5 ++ .../action/ShardOperationFailedException.java | 14 ++++- .../action/bulk/BulkItemResponse.java | 29 ++++----- .../action/bulk/TransportBulkAction.java | 10 ++- .../action/percolate/PercolateResponse.java | 2 +- .../search/SearchPhaseExecutionException.java | 62 ++----------------- .../action/search/SearchResponse.java | 2 +- .../action/search/ShardSearchFailure.java | 3 +- .../DefaultShardOperationFailedException.java | 22 +++++++ .../clear/RestClearIndicesCacheAction.java | 2 +- .../admin/indices/flush/RestFlushAction.java | 2 +- .../indices/optimize/RestOptimizeAction.java | 2 +- .../indices/refresh/RestRefreshAction.java | 2 +- .../segments/RestIndicesSegmentsAction.java | 2 +- .../indices/stats/RestIndicesStatsAction.java | 2 +- .../indices/upgrade/RestUpgradeAction.java | 4 +- .../query/RestValidateQueryAction.java | 2 +- .../rest/action/bulk/RestBulkAction.java | 5 +- .../rest/action/count/RestCountAction.java | 2 +- .../fieldstats/RestFieldStatsAction.java | 2 +- .../action/suggest/RestSuggestAction.java | 2 +- .../rest/action/support/RestActions.java | 28 +++------ .../org/elasticsearch/snapshots/Snapshot.java | 4 +- .../elasticsearch/snapshots/SnapshotInfo.java | 4 +- .../snapshots/SnapshotShardFailure.java | 27 +++++--- 27 files changed, 181 insertions(+), 129 deletions(-) diff --git a/src/main/java/org/elasticsearch/ElasticsearchException.java b/src/main/java/org/elasticsearch/ElasticsearchException.java index a622b0f7e81..627c1b674a1 100644 --- a/src/main/java/org/elasticsearch/ElasticsearchException.java +++ b/src/main/java/org/elasticsearch/ElasticsearchException.java @@ -22,16 +22,18 @@ package org.elasticsearch; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.IndexException; import org.elasticsearch.rest.HasRestHeaders; import org.elasticsearch.rest.RestStatus; import java.io.IOException; -import java.util.List; -import java.util.Map; +import java.util.*; /** * A base class for all elasticsearch exceptions. @@ -288,6 +290,4 @@ public class ElasticsearchException extends RuntimeException implements ToXConte public String toString() { return ExceptionsHelper.detailedMessage(this).trim(); } - - } diff --git a/src/main/java/org/elasticsearch/ExceptionsHelper.java b/src/main/java/org/elasticsearch/ExceptionsHelper.java index 9c29a4dc0aa..c9f51cb5d55 100644 --- a/src/main/java/org/elasticsearch/ExceptionsHelper.java +++ b/src/main/java/org/elasticsearch/ExceptionsHelper.java @@ -22,15 +22,21 @@ package org.elasticsearch; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.IndexFormatTooNewException; import org.apache.lucene.index.IndexFormatTooOldException; +import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.logging.Loggers; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.IndexException; import org.elasticsearch.rest.RestStatus; import java.io.IOException; import java.io.PrintWriter; import java.io.StringWriter; +import java.util.ArrayList; +import java.util.HashSet; import java.util.List; +import java.util.Set; /** * @@ -214,4 +220,59 @@ public final class ExceptionsHelper { } return true; } + + + /** + * Deduplicate the failures by exception message and index. + */ + public static ShardOperationFailedException[] groupBy(ShardOperationFailedException[] failures) { + List uniqueFailures = new ArrayList<>(); + Set reasons = new HashSet<>(); + for (ShardOperationFailedException failure : failures) { + GroupBy reason = new GroupBy(failure.getCause()); + if (reasons.contains(reason) == false) { + reasons.add(reason); + uniqueFailures.add(failure); + } + } + return uniqueFailures.toArray(new ShardOperationFailedException[0]); + } + + static class GroupBy { + final String reason; + final Index index; + final Class causeType; + + public GroupBy(Throwable t) { + if (t instanceof IndexException) { + index = ((IndexException) t).index(); + } else { + index = null; + } + reason = t.getMessage(); + causeType = t.getClass(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + GroupBy groupBy = (GroupBy) o; + + if (!causeType.equals(groupBy.causeType)) return false; + if (index != null ? !index.equals(groupBy.index) : groupBy.index != null) return false; + if (reason != null ? !reason.equals(groupBy.reason) : groupBy.reason != null) return false; + + return true; + } + + @Override + public int hashCode() { + int result = reason != null ? reason.hashCode() : 0; + result = 31 * result + (index != null ? index.hashCode() : 0); + result = 31 * result + causeType.hashCode(); + return result; + } + } } diff --git a/src/main/java/org/elasticsearch/action/ActionWriteResponse.java b/src/main/java/org/elasticsearch/action/ActionWriteResponse.java index 11240cc2cbc..a63f6dcd9fa 100644 --- a/src/main/java/org/elasticsearch/action/ActionWriteResponse.java +++ b/src/main/java/org/elasticsearch/action/ActionWriteResponse.java @@ -231,6 +231,11 @@ public abstract class ActionWriteResponse extends ActionResponse { return status; } + @Override + public Throwable getCause() { + return cause; + } + /** * @return Whether this failure occurred on a primary shard. * (this only reports true for delete by query) diff --git a/src/main/java/org/elasticsearch/action/ShardOperationFailedException.java b/src/main/java/org/elasticsearch/action/ShardOperationFailedException.java index 49b3d0194af..95cda257690 100644 --- a/src/main/java/org/elasticsearch/action/ShardOperationFailedException.java +++ b/src/main/java/org/elasticsearch/action/ShardOperationFailedException.java @@ -20,16 +20,23 @@ package org.elasticsearch.action; import org.elasticsearch.common.io.stream.Streamable; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.IndexException; import org.elasticsearch.rest.RestStatus; import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; /** * An exception indicating that a failure occurred performing an operation on the shard. * * */ -public interface ShardOperationFailedException extends Streamable, Serializable { +public interface ShardOperationFailedException extends Streamable, Serializable, ToXContent { /** * The index the operation failed on. Might return null if it can't be derived. @@ -50,4 +57,9 @@ public interface ShardOperationFailedException extends Streamable, Serializable * The status of the failure. */ RestStatus status(); + + /** + * The cause of this failure + */ + Throwable getCause(); } diff --git a/src/main/java/org/elasticsearch/action/bulk/BulkItemResponse.java b/src/main/java/org/elasticsearch/action/bulk/BulkItemResponse.java index c32f02f0022..3b9d2e03ba0 100644 --- a/src/main/java/org/elasticsearch/action/bulk/BulkItemResponse.java +++ b/src/main/java/org/elasticsearch/action/bulk/BulkItemResponse.java @@ -19,6 +19,7 @@ package org.elasticsearch.action.bulk; +import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionWriteResponse; import org.elasticsearch.action.delete.DeleteResponse; @@ -27,6 +28,7 @@ import org.elasticsearch.action.update.UpdateResponse; 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; @@ -44,26 +46,17 @@ public class BulkItemResponse implements Streamable { private final String index; private final String type; private final String id; - private final String message; + private final Throwable cause; private final RestStatus status; public Failure(String index, String type, String id, Throwable t) { this.index = index; this.type = type; this.id = id; - this.message = t.toString(); + this.cause = t; this.status = ExceptionsHelper.status(t); } - - public Failure(String index, String type, String id, String message, RestStatus status) { - this.index = index; - this.type = type; - this.id = id; - this.message = message; - this.status = status; - } - /** * The index name of the action. */ @@ -89,7 +82,7 @@ public class BulkItemResponse implements Streamable { * The failure message. */ public String getMessage() { - return this.message; + return this.cause.toString(); } /** @@ -98,6 +91,10 @@ public class BulkItemResponse implements Streamable { public RestStatus getStatus() { return this.status; } + + public Throwable getCause() { + return cause; + } } private int id; @@ -265,9 +262,8 @@ public class BulkItemResponse implements Streamable { String fIndex = in.readString(); String fType = in.readString(); String fId = in.readOptionalString(); - String fMessage = in.readString(); - RestStatus status = RestStatus.readFrom(in); - failure = new Failure(fIndex, fType, fId, fMessage, status); + Throwable throwable = in.readThrowable(); + failure = new Failure(fIndex, fType, fId, throwable); } } @@ -295,8 +291,7 @@ public class BulkItemResponse implements Streamable { out.writeString(failure.getIndex()); out.writeString(failure.getType()); out.writeOptionalString(failure.getId()); - out.writeString(failure.getMessage()); - RestStatus.writeTo(out, failure.getStatus()); + out.writeThrowable(failure.getCause()); } } } diff --git a/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java b/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java index 66d3965b4f5..5ffba0598bc 100644 --- a/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java +++ b/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java @@ -284,7 +284,7 @@ public class TransportBulkAction extends HandledTransportAction uniqueFailures = new ArrayList<>(); - Set reasons = new HashSet<>(); - for (ShardSearchFailure failure : failures) { - GroupBy reason = new GroupBy(failure.getCause()); - if (reasons.contains(reason) == false) { - reasons.add(reason); - uniqueFailures.add(failure); - } - } - return uniqueFailures.toArray(new ShardSearchFailure[0]); - - } - @Override public ElasticsearchException[] guessRootCauses() { - ShardSearchFailure[] failures = groupBy(shardFailures); + ShardOperationFailedException[] failures = ExceptionsHelper.groupBy(shardFailures); List rootCauses = new ArrayList<>(failures.length); - for (ShardSearchFailure failure : failures) { + for (ShardOperationFailedException failure : failures) { ElasticsearchException[] guessRootCauses = ElasticsearchException.guessRootCauses(failure.getCause()); rootCauses.addAll(Arrays.asList(guessRootCauses)); } @@ -132,42 +120,4 @@ public class SearchPhaseExecutionException extends ElasticsearchException { public String toString() { return buildMessage(phaseName, getMessage(), shardFailures); } - - static class GroupBy { - final String reason; - final Index index; - final Class causeType; - - public GroupBy(Throwable t) { - if (t instanceof IndexException) { - index = ((IndexException) t).index(); - } else { - index = null; - } - reason = t.getMessage(); - causeType = t.getClass(); - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - GroupBy groupBy = (GroupBy) o; - - if (!causeType.equals(groupBy.causeType)) return false; - if (index != null ? !index.equals(groupBy.index) : groupBy.index != null) return false; - if (reason != null ? !reason.equals(groupBy.reason) : groupBy.reason != null) return false; - - return true; - } - - @Override - public int hashCode() { - int result = reason != null ? reason.hashCode() : 0; - result = 31 * result + (index != null ? index.hashCode() : 0); - result = 31 * result + causeType.hashCode(); - return result; - } - } } diff --git a/src/main/java/org/elasticsearch/action/search/SearchResponse.java b/src/main/java/org/elasticsearch/action/search/SearchResponse.java index b0e2f213191..769e0978a71 100644 --- a/src/main/java/org/elasticsearch/action/search/SearchResponse.java +++ b/src/main/java/org/elasticsearch/action/search/SearchResponse.java @@ -177,7 +177,7 @@ public class SearchResponse extends ActionResponse implements StatusToXContent { if (isTerminatedEarly() != null) { builder.field(Fields.TERMINATED_EARLY, isTerminatedEarly()); } - RestActions.buildBroadcastShardsHeader(builder, getTotalShards(), getSuccessfulShards(), getFailedShards(), getShardFailures()); + RestActions.buildBroadcastShardsHeader(builder, params, getTotalShards(), getSuccessfulShards(), getFailedShards(), getShardFailures()); internalResponse.toXContent(builder, params); return builder; } diff --git a/src/main/java/org/elasticsearch/action/search/ShardSearchFailure.java b/src/main/java/org/elasticsearch/action/search/ShardSearchFailure.java index c090c3e6d72..79d0de4884e 100644 --- a/src/main/java/org/elasticsearch/action/search/ShardSearchFailure.java +++ b/src/main/java/org/elasticsearch/action/search/ShardSearchFailure.java @@ -40,7 +40,7 @@ import static org.elasticsearch.search.SearchShardTarget.readSearchShardTarget; /** * Represents a failure to search on a specific shard. */ -public class ShardSearchFailure implements ShardOperationFailedException, ToXContent { +public class ShardSearchFailure implements ShardOperationFailedException { public static final ShardSearchFailure[] EMPTY_ARRAY = new ShardSearchFailure[0]; @@ -172,6 +172,7 @@ public class ShardSearchFailure implements ShardOperationFailedException, ToXCon return builder; } + @Override public Throwable getCause() { return cause; } diff --git a/src/main/java/org/elasticsearch/action/support/DefaultShardOperationFailedException.java b/src/main/java/org/elasticsearch/action/support/DefaultShardOperationFailedException.java index 3be7b7d2aee..313692d75b1 100644 --- a/src/main/java/org/elasticsearch/action/support/DefaultShardOperationFailedException.java +++ b/src/main/java/org/elasticsearch/action/support/DefaultShardOperationFailedException.java @@ -24,6 +24,8 @@ import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.index.shard.IndexShardException; import org.elasticsearch.rest.RestStatus; @@ -81,6 +83,11 @@ public class DefaultShardOperationFailedException implements ShardOperationFaile return status; } + @Override + public Throwable getCause() { + return reason; + } + public static DefaultShardOperationFailedException readShardOperationFailed(StreamInput in) throws IOException { DefaultShardOperationFailedException exp = new DefaultShardOperationFailedException(); exp.readFrom(in); @@ -114,4 +121,19 @@ public class DefaultShardOperationFailedException implements ShardOperationFaile public String toString() { return "[" + index + "][" + shardId + "] failed, reason [" + reason() + "]"; } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.field("shard", shardId()); + builder.field("index", index()); + builder.field("status", status.name()); + if (reason != null) { + builder.field("reason"); + builder.startObject(); + ElasticsearchException.toXContent(builder, params, reason); + builder.endObject(); + } + return builder; + + } } diff --git a/src/main/java/org/elasticsearch/rest/action/admin/indices/cache/clear/RestClearIndicesCacheAction.java b/src/main/java/org/elasticsearch/rest/action/admin/indices/cache/clear/RestClearIndicesCacheAction.java index e676409dbec..09560a4e2e8 100644 --- a/src/main/java/org/elasticsearch/rest/action/admin/indices/cache/clear/RestClearIndicesCacheAction.java +++ b/src/main/java/org/elasticsearch/rest/action/admin/indices/cache/clear/RestClearIndicesCacheAction.java @@ -62,7 +62,7 @@ public class RestClearIndicesCacheAction extends BaseRestHandler { @Override public RestResponse buildResponse(ClearIndicesCacheResponse response, XContentBuilder builder) throws Exception { builder.startObject(); - buildBroadcastShardsHeader(builder, response); + buildBroadcastShardsHeader(builder, request, response); builder.endObject(); return new BytesRestResponse(OK, builder); } diff --git a/src/main/java/org/elasticsearch/rest/action/admin/indices/flush/RestFlushAction.java b/src/main/java/org/elasticsearch/rest/action/admin/indices/flush/RestFlushAction.java index 6c95342cf89..97a951d2ad2 100644 --- a/src/main/java/org/elasticsearch/rest/action/admin/indices/flush/RestFlushAction.java +++ b/src/main/java/org/elasticsearch/rest/action/admin/indices/flush/RestFlushAction.java @@ -60,7 +60,7 @@ public class RestFlushAction extends BaseRestHandler { @Override public RestResponse buildResponse(FlushResponse response, XContentBuilder builder) throws Exception { builder.startObject(); - buildBroadcastShardsHeader(builder, response); + buildBroadcastShardsHeader(builder, request, response); builder.endObject(); return new BytesRestResponse(OK, builder); } diff --git a/src/main/java/org/elasticsearch/rest/action/admin/indices/optimize/RestOptimizeAction.java b/src/main/java/org/elasticsearch/rest/action/admin/indices/optimize/RestOptimizeAction.java index 74379f632c5..3ecafae993a 100644 --- a/src/main/java/org/elasticsearch/rest/action/admin/indices/optimize/RestOptimizeAction.java +++ b/src/main/java/org/elasticsearch/rest/action/admin/indices/optimize/RestOptimizeAction.java @@ -61,7 +61,7 @@ public class RestOptimizeAction extends BaseRestHandler { @Override public RestResponse buildResponse(OptimizeResponse response, XContentBuilder builder) throws Exception { builder.startObject(); - buildBroadcastShardsHeader(builder, response); + buildBroadcastShardsHeader(builder, request, response); builder.endObject(); return new BytesRestResponse(OK, builder); } diff --git a/src/main/java/org/elasticsearch/rest/action/admin/indices/refresh/RestRefreshAction.java b/src/main/java/org/elasticsearch/rest/action/admin/indices/refresh/RestRefreshAction.java index 949b82270ff..85775d55394 100644 --- a/src/main/java/org/elasticsearch/rest/action/admin/indices/refresh/RestRefreshAction.java +++ b/src/main/java/org/elasticsearch/rest/action/admin/indices/refresh/RestRefreshAction.java @@ -58,7 +58,7 @@ public class RestRefreshAction extends BaseRestHandler { @Override public RestResponse buildResponse(RefreshResponse response, XContentBuilder builder) throws Exception { builder.startObject(); - buildBroadcastShardsHeader(builder, response); + buildBroadcastShardsHeader(builder, request, response); builder.endObject(); return new BytesRestResponse(OK, builder); } diff --git a/src/main/java/org/elasticsearch/rest/action/admin/indices/segments/RestIndicesSegmentsAction.java b/src/main/java/org/elasticsearch/rest/action/admin/indices/segments/RestIndicesSegmentsAction.java index b5b2ba6e7c4..7356d1b759f 100644 --- a/src/main/java/org/elasticsearch/rest/action/admin/indices/segments/RestIndicesSegmentsAction.java +++ b/src/main/java/org/elasticsearch/rest/action/admin/indices/segments/RestIndicesSegmentsAction.java @@ -54,7 +54,7 @@ public class RestIndicesSegmentsAction extends BaseRestHandler { @Override public RestResponse buildResponse(IndicesSegmentResponse response, XContentBuilder builder) throws Exception { builder.startObject(); - buildBroadcastShardsHeader(builder, response); + buildBroadcastShardsHeader(builder, request, response); response.toXContent(builder, request); builder.endObject(); return new BytesRestResponse(OK, builder); diff --git a/src/main/java/org/elasticsearch/rest/action/admin/indices/stats/RestIndicesStatsAction.java b/src/main/java/org/elasticsearch/rest/action/admin/indices/stats/RestIndicesStatsAction.java index a71e5a3f814..0d6d71fdd90 100644 --- a/src/main/java/org/elasticsearch/rest/action/admin/indices/stats/RestIndicesStatsAction.java +++ b/src/main/java/org/elasticsearch/rest/action/admin/indices/stats/RestIndicesStatsAction.java @@ -103,7 +103,7 @@ public class RestIndicesStatsAction extends BaseRestHandler { @Override public RestResponse buildResponse(IndicesStatsResponse response, XContentBuilder builder) throws Exception { builder.startObject(); - buildBroadcastShardsHeader(builder, response); + buildBroadcastShardsHeader(builder, request, response); response.toXContent(builder, request); builder.endObject(); return new BytesRestResponse(OK, builder); diff --git a/src/main/java/org/elasticsearch/rest/action/admin/indices/upgrade/RestUpgradeAction.java b/src/main/java/org/elasticsearch/rest/action/admin/indices/upgrade/RestUpgradeAction.java index 23509582a66..1dd086eb3ce 100644 --- a/src/main/java/org/elasticsearch/rest/action/admin/indices/upgrade/RestUpgradeAction.java +++ b/src/main/java/org/elasticsearch/rest/action/admin/indices/upgrade/RestUpgradeAction.java @@ -87,7 +87,7 @@ public class RestUpgradeAction extends BaseRestHandler { }); } - void handlePost(RestRequest request, RestChannel channel, Client client) { + void handlePost(final RestRequest request, RestChannel channel, Client client) { OptimizeRequest optimizeReq = new OptimizeRequest(Strings.splitStringByCommaToArray(request.param("index"))); optimizeReq.flush(true); optimizeReq.upgrade(true); @@ -97,7 +97,7 @@ public class RestUpgradeAction extends BaseRestHandler { @Override public RestResponse buildResponse(OptimizeResponse response, XContentBuilder builder) throws Exception { builder.startObject(); - buildBroadcastShardsHeader(builder, response); + buildBroadcastShardsHeader(builder, request, response); builder.endObject(); return new BytesRestResponse(OK, builder); } diff --git a/src/main/java/org/elasticsearch/rest/action/admin/indices/validate/query/RestValidateQueryAction.java b/src/main/java/org/elasticsearch/rest/action/admin/indices/validate/query/RestValidateQueryAction.java index 4f237465bd6..67661967d56 100644 --- a/src/main/java/org/elasticsearch/rest/action/admin/indices/validate/query/RestValidateQueryAction.java +++ b/src/main/java/org/elasticsearch/rest/action/admin/indices/validate/query/RestValidateQueryAction.java @@ -84,7 +84,7 @@ public class RestValidateQueryAction extends BaseRestHandler { builder.startObject(); builder.field("valid", response.isValid()); - buildBroadcastShardsHeader(builder, response); + buildBroadcastShardsHeader(builder, request, response); if (response.getQueryExplanation() != null && !response.getQueryExplanation().isEmpty()) { builder.startArray("explanations"); diff --git a/src/main/java/org/elasticsearch/rest/action/bulk/RestBulkAction.java b/src/main/java/org/elasticsearch/rest/action/bulk/RestBulkAction.java index 1a3a1b38a6e..987178595a6 100644 --- a/src/main/java/org/elasticsearch/rest/action/bulk/RestBulkAction.java +++ b/src/main/java/org/elasticsearch/rest/action/bulk/RestBulkAction.java @@ -19,6 +19,7 @@ package org.elasticsearch.rest.action.bulk; +import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionWriteResponse; import org.elasticsearch.action.WriteConsistencyLevel; import org.elasticsearch.action.bulk.BulkItemResponse; @@ -102,7 +103,9 @@ public class RestBulkAction extends BaseRestHandler { } if (itemResponse.isFailed()) { builder.field(Fields.STATUS, itemResponse.getFailure().getStatus().getStatus()); - builder.field(Fields.ERROR, itemResponse.getFailure().getMessage()); + builder.startObject(Fields.ERROR); + ElasticsearchException.toXContent(builder, request, itemResponse.getFailure().getCause()); + builder.endObject(); } else { ActionWriteResponse.ShardInfo shardInfo = itemResponse.getResponse().getShardInfo(); shardInfo.toXContent(builder, request); diff --git a/src/main/java/org/elasticsearch/rest/action/count/RestCountAction.java b/src/main/java/org/elasticsearch/rest/action/count/RestCountAction.java index dc38db49181..677f3af4508 100644 --- a/src/main/java/org/elasticsearch/rest/action/count/RestCountAction.java +++ b/src/main/java/org/elasticsearch/rest/action/count/RestCountAction.java @@ -85,7 +85,7 @@ public class RestCountAction extends BaseRestHandler { builder.field("terminated_early", response.terminatedEarly()); } builder.field("count", response.getCount()); - buildBroadcastShardsHeader(builder, response); + buildBroadcastShardsHeader(builder, request, response); builder.endObject(); return new BytesRestResponse(response.status(), builder); diff --git a/src/main/java/org/elasticsearch/rest/action/fieldstats/RestFieldStatsAction.java b/src/main/java/org/elasticsearch/rest/action/fieldstats/RestFieldStatsAction.java index ca382f3c642..6850b8b91ec 100644 --- a/src/main/java/org/elasticsearch/rest/action/fieldstats/RestFieldStatsAction.java +++ b/src/main/java/org/elasticsearch/rest/action/fieldstats/RestFieldStatsAction.java @@ -62,7 +62,7 @@ public class RestFieldStatsAction extends BaseRestHandler { @Override public RestResponse buildResponse(FieldStatsResponse response, XContentBuilder builder) throws Exception { builder.startObject(); - buildBroadcastShardsHeader(builder, response); + buildBroadcastShardsHeader(builder, request, response); builder.startObject("indices"); for (Map.Entry> entry1 : response.getIndicesMergedFieldStats().entrySet()) { diff --git a/src/main/java/org/elasticsearch/rest/action/suggest/RestSuggestAction.java b/src/main/java/org/elasticsearch/rest/action/suggest/RestSuggestAction.java index 184a62244d1..8bf360dc36c 100644 --- a/src/main/java/org/elasticsearch/rest/action/suggest/RestSuggestAction.java +++ b/src/main/java/org/elasticsearch/rest/action/suggest/RestSuggestAction.java @@ -72,7 +72,7 @@ public class RestSuggestAction extends BaseRestHandler { public RestResponse buildResponse(SuggestResponse response, XContentBuilder builder) throws Exception { RestStatus restStatus = RestStatus.status(response.getSuccessfulShards(), response.getTotalShards(), response.getShardFailures()); builder.startObject(); - buildBroadcastShardsHeader(builder, response); + buildBroadcastShardsHeader(builder, request, response); Suggest suggest = response.getSuggest(); if (suggest != null) { suggest.toXContent(builder, request); diff --git a/src/main/java/org/elasticsearch/rest/action/support/RestActions.java b/src/main/java/org/elasticsearch/rest/action/support/RestActions.java index ee2cb77bb90..67ceb83c33b 100644 --- a/src/main/java/org/elasticsearch/rest/action/support/RestActions.java +++ b/src/main/java/org/elasticsearch/rest/action/support/RestActions.java @@ -19,16 +19,14 @@ package org.elasticsearch.rest.action.support; +import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.support.QuerySourceBuilder; import org.elasticsearch.action.support.broadcast.BroadcastOperationResponse; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.lucene.uid.Versions; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentBuilderString; -import org.elasticsearch.common.xcontent.XContentFactory; -import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.common.xcontent.*; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.QueryStringQueryBuilder; import org.elasticsearch.rest.RestRequest; @@ -62,33 +60,23 @@ public class RestActions { static final XContentBuilderString SUCCESSFUL = new XContentBuilderString("successful"); static final XContentBuilderString FAILED = new XContentBuilderString("failed"); static final XContentBuilderString FAILURES = new XContentBuilderString("failures"); - static final XContentBuilderString INDEX = new XContentBuilderString("index"); - static final XContentBuilderString SHARD = new XContentBuilderString("shard"); - static final XContentBuilderString STATUS = new XContentBuilderString("status"); - static final XContentBuilderString REASON = new XContentBuilderString("reason"); } - public static void buildBroadcastShardsHeader(XContentBuilder builder, BroadcastOperationResponse response) throws IOException { - buildBroadcastShardsHeader(builder, response.getTotalShards(), response.getSuccessfulShards(), response.getFailedShards(), response.getShardFailures()); + public static void buildBroadcastShardsHeader(XContentBuilder builder, ToXContent.Params params, BroadcastOperationResponse response) throws IOException { + buildBroadcastShardsHeader(builder, params, response.getTotalShards(), response.getSuccessfulShards(), response.getFailedShards(), response.getShardFailures()); } - public static void buildBroadcastShardsHeader(XContentBuilder builder, int total, int successful, int failed, ShardOperationFailedException[] shardFailures) throws IOException { + public static void buildBroadcastShardsHeader(XContentBuilder builder, ToXContent.Params params, int total, int successful, int failed, ShardOperationFailedException[] shardFailures) throws IOException { builder.startObject(Fields._SHARDS); builder.field(Fields.TOTAL, total); builder.field(Fields.SUCCESSFUL, successful); builder.field(Fields.FAILED, failed); if (shardFailures != null && shardFailures.length > 0) { builder.startArray(Fields.FAILURES); - for (ShardOperationFailedException shardFailure : shardFailures) { + final boolean group = params.paramAsBoolean("group_shard_failures", true); // we group by default + for (ShardOperationFailedException shardFailure : group ? ExceptionsHelper.groupBy(shardFailures) : shardFailures) { builder.startObject(); - if (shardFailure.index() != null) { - builder.field(Fields.INDEX, shardFailure.index(), XContentBuilder.FieldCaseConversion.NONE); - } - if (shardFailure.shardId() != -1) { - builder.field(Fields.SHARD, shardFailure.shardId()); - } - builder.field(Fields.STATUS, shardFailure.status().getStatus()); - builder.field(Fields.REASON, shardFailure.reason()); + shardFailure.toXContent(builder, params); builder.endObject(); } builder.endArray(); diff --git a/src/main/java/org/elasticsearch/snapshots/Snapshot.java b/src/main/java/org/elasticsearch/snapshots/Snapshot.java index 365f88a0048..3feed76c5ff 100644 --- a/src/main/java/org/elasticsearch/snapshots/Snapshot.java +++ b/src/main/java/org/elasticsearch/snapshots/Snapshot.java @@ -255,7 +255,9 @@ public class Snapshot implements Comparable, ToXContent { builder.field(Fields.SUCCESSFUL_SHARDS, successfulShards); builder.startArray(Fields.FAILURES); for (SnapshotShardFailure shardFailure : shardFailures) { - SnapshotShardFailure.toXContent(shardFailure, builder, params); + builder.startObject(); + shardFailure.toXContent(builder, params); + builder.endObject(); } builder.endArray(); builder.endObject(); diff --git a/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java b/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java index 8e09fbd2c23..1a280908f06 100644 --- a/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java +++ b/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java @@ -223,7 +223,9 @@ public class SnapshotInfo implements ToXContent, Streamable { } builder.startArray(Fields.FAILURES); for (SnapshotShardFailure shardFailure : shardFailures) { - SnapshotShardFailure.toXContent(shardFailure, builder, params); + builder.startObject(); + shardFailure.toXContent(builder, params); + builder.endObject(); } builder.endArray(); builder.startObject(Fields.SHARDS); diff --git a/src/main/java/org/elasticsearch/snapshots/SnapshotShardFailure.java b/src/main/java/org/elasticsearch/snapshots/SnapshotShardFailure.java index 83c903244a8..9edea5c8720 100644 --- a/src/main/java/org/elasticsearch/snapshots/SnapshotShardFailure.java +++ b/src/main/java/org/elasticsearch/snapshots/SnapshotShardFailure.java @@ -27,6 +27,8 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.snapshots.IndexShardSnapshotFailedException; import org.elasticsearch.rest.RestStatus; import java.io.IOException; @@ -106,6 +108,11 @@ public class SnapshotShardFailure implements ShardOperationFailedException { return status; } + @Override + public Throwable getCause() { + return new IndexShardSnapshotFailedException(new ShardId(index, shardId), reason); + } + /** * Returns node id where failure occurred * @@ -162,13 +169,7 @@ public class SnapshotShardFailure implements ShardOperationFailedException { */ public static void toXContent(SnapshotShardFailure snapshotShardFailure, XContentBuilder builder, ToXContent.Params params) throws IOException { builder.startObject(); - if (snapshotShardFailure.nodeId != null) { - builder.field("node_id", snapshotShardFailure.nodeId); - } - builder.field("index", snapshotShardFailure.index); - builder.field("reason", snapshotShardFailure.reason); - builder.field("shard_id", snapshotShardFailure.shardId); - builder.field("status", snapshotShardFailure.status.name()); + snapshotShardFailure.toXContent(builder, params); builder.endObject(); } @@ -212,4 +213,16 @@ public class SnapshotShardFailure implements ShardOperationFailedException { } return snapshotShardFailure; } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.field("index", index); + builder.field("shard_id", shardId); + builder.field("reason", reason); + if (nodeId != null) { + builder.field("node_id", nodeId); + } + builder.field("status", status.name()); + return builder; + } } From 03e5149994e38886e3c01de8303be849844d1199 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Fri, 15 May 2015 22:23:59 +0200 Subject: [PATCH 32/56] [ENGINE] Remove the ability to flush without flushing the translog This is a leftover from the times where we failed a flush when recoveries are ongoing. This code is really not needed anymore and we can luckily flush the translog all the time as well. --- .../index/engine/InternalEngine.java | 44 +++++-------------- 1 file changed, 12 insertions(+), 32 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index dee7eb2a0a6..0730ea41f5b 100644 --- a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -674,15 +674,11 @@ public class InternalEngine extends Engine { @Override public void flush() throws EngineException { - flush(true, false, false); + flush(false, false); } @Override public void flush(boolean force, boolean waitIfOngoing) throws EngineException { - flush(true, force, waitIfOngoing); - } - - private void flush(boolean commitTranslog, boolean force, boolean waitIfOngoing) throws EngineException { ensureOpen(); /* * Unfortunately the lock order is important here. We have to acquire the readlock first otherwise @@ -706,37 +702,21 @@ public class InternalEngine extends Engine { logger.trace("acquired flush lock immediately"); } try { - if (commitTranslog) { - if (flushNeeded || force) { - flushNeeded = false; - try { - translog.prepareCommit(); - logger.trace("starting commit for flush; commitTranslog=true"); - commitIndexWriter(indexWriter, translog); - logger.trace("finished commit for flush"); - translog.commit(); - // we need to refresh in order to clear older version values - refresh("version_table_flush"); - - } catch (Throwable e) { - throw new FlushFailedEngineException(shardId, e); - } - } - } else { - // note, its ok to just commit without cleaning the translog, its perfectly fine to replay a - // translog on an index that was opened on a committed point in time that is "in the future" - // of that translog - // we allow to *just* commit if there is an ongoing recovery happening... - // its ok to use this, only a flush will cause a new translogFileGeneration, and we are locked here from - // other flushes use flushLock + if (flushNeeded || force) { + flushNeeded = false; + final long translogId; try { - logger.trace("starting commit for flush; commitTranslog=false"); + translog.prepareCommit(); + logger.trace("starting commit for flush; commitTranslog=true"); commitIndexWriter(indexWriter, translog); logger.trace("finished commit for flush"); + translog.commit(); + // we need to refresh in order to clear older version values + refresh("version_table_flush"); + } catch (Throwable e) { throw new FlushFailedEngineException(shardId, e); } - } /* * we have to inc-ref the store here since if the engine is closed by a tragic event @@ -830,7 +810,7 @@ public class InternalEngine extends Engine { indexWriter.forceMerge(maxNumSegments, true /* blocks and waits for merges*/); } if (flush) { - flush(true, true, true); + flush(true, true); } if (upgrade) { logger.info("finished segment upgrade"); @@ -857,7 +837,7 @@ public class InternalEngine extends Engine { // the to a write lock when we fail the engine in this operation if (flushFirst) { logger.trace("start flush for snapshot"); - flush(false, false, true); + flush(false, true); logger.trace("finish flush for snapshot"); } try (ReleasableLock lock = readLock.acquire()) { From 6c56cedf9a960177969a6a5748fbe41080ec79ed Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 18 May 2015 10:11:07 +0200 Subject: [PATCH 33/56] Prevent MockStore where commits should be prevented on close --- .../java/org/elasticsearch/index/store/IndexStoreModule.java | 2 +- .../org/elasticsearch/index/store/CorruptedTranslogTests.java | 2 ++ src/test/java/org/elasticsearch/test/InternalTestCluster.java | 2 +- 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/store/IndexStoreModule.java b/src/main/java/org/elasticsearch/index/store/IndexStoreModule.java index c40301d560b..3d351179d21 100644 --- a/src/main/java/org/elasticsearch/index/store/IndexStoreModule.java +++ b/src/main/java/org/elasticsearch/index/store/IndexStoreModule.java @@ -32,7 +32,7 @@ public class IndexStoreModule extends AbstractModule implements SpawnModules { private final Settings settings; - public static enum Type { + public enum Type { NIOFS { @Override public boolean match(String setting) { diff --git a/src/test/java/org/elasticsearch/index/store/CorruptedTranslogTests.java b/src/test/java/org/elasticsearch/index/store/CorruptedTranslogTests.java index 5890233fe27..4692823e9a2 100644 --- a/src/test/java/org/elasticsearch/index/store/CorruptedTranslogTests.java +++ b/src/test/java/org/elasticsearch/index/store/CorruptedTranslogTests.java @@ -40,6 +40,7 @@ import org.elasticsearch.monitor.fs.FsStats; import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.elasticsearch.test.engine.MockEngineSupport; import org.elasticsearch.test.junit.annotations.TestLogging; +import org.elasticsearch.test.store.MockFSIndexStoreModule; import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.transport.TransportModule; import org.junit.Test; @@ -81,6 +82,7 @@ public class CorruptedTranslogTests extends ElasticsearchIntegrationTest { .put("index.number_of_shards", 1) .put("index.number_of_replicas", 0) .put("index.refresh_interval", "-1") + .put(IndexStoreModule.STORE_TYPE, IndexStoreModule.Type.DEFAULT) // no mock store - it commits for check-index .put(MockEngineSupport.FLUSH_ON_CLOSE_RATIO, 0.0d) // never flush - always recover from translog .put(IndexShard.INDEX_FLUSH_ON_CLOSE, false) // never flush - always recover from translog .put(TranslogConfig.INDEX_TRANSLOG_SYNC_INTERVAL, "1s") // fsync the translog every second diff --git a/src/test/java/org/elasticsearch/test/InternalTestCluster.java b/src/test/java/org/elasticsearch/test/InternalTestCluster.java index 5d8209b753e..cf0a52df92f 100644 --- a/src/test/java/org/elasticsearch/test/InternalTestCluster.java +++ b/src/test/java/org/elasticsearch/test/InternalTestCluster.java @@ -365,7 +365,7 @@ public final class InternalTestCluster extends TestCluster { .put("cluster.routing.schedule", (30 + random.nextInt(50)) + "ms") .put(SETTING_CLUSTER_NODE_SEED, seed); if (ENABLE_MOCK_MODULES && usually(random)) { - builder.put(IndexStoreModule.STORE_TYPE, MockFSIndexStoreModule.class.getName()); // no RAM dir for now! + builder.put(IndexStoreModule.STORE_TYPE, MockFSIndexStoreModule.class.getName()); builder.put(IndexShardModule.ENGINE_FACTORY, MockEngineFactory.class); builder.put(PageCacheRecyclerModule.CACHE_IMPL, MockPageCacheRecyclerModule.class.getName()); builder.put(BigArraysModule.IMPL, MockBigArraysModule.class.getName()); From 61a31dd57f2f7809de5dd2a514eb144b322731a5 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Mon, 18 May 2015 10:16:15 +0200 Subject: [PATCH 34/56] Tests: Add information about shards that are not closed when the filter cache gets closed. --- .../elasticsearch/indices/cache/filter/IndicesFilterCache.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/org/elasticsearch/indices/cache/filter/IndicesFilterCache.java b/src/main/java/org/elasticsearch/indices/cache/filter/IndicesFilterCache.java index 0ee4f07c7e2..b5d8e0a0762 100644 --- a/src/main/java/org/elasticsearch/indices/cache/filter/IndicesFilterCache.java +++ b/src/main/java/org/elasticsearch/indices/cache/filter/IndicesFilterCache.java @@ -266,7 +266,7 @@ public class IndicesFilterCache extends AbstractComponent implements QueryCache, @Override public void close() { assert shardKeyMap.size() == 0 : shardKeyMap.size(); - assert shardStats.isEmpty(); + assert shardStats.isEmpty() : shardStats.keySet(); assert stats2.isEmpty() : stats2; cache.clear(); } From 7892c7c86942350f6b4167976df76d15e5f7b797 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 18 May 2015 11:03:11 +0200 Subject: [PATCH 35/56] remove flushing before running checkindex --- .../index/store/CorruptedTranslogTests.java | 1 - .../snapshots/SharedClusterSnapshotRestoreTests.java | 2 +- .../elasticsearch/test/store/MockFSDirectoryService.java | 9 --------- 3 files changed, 1 insertion(+), 11 deletions(-) diff --git a/src/test/java/org/elasticsearch/index/store/CorruptedTranslogTests.java b/src/test/java/org/elasticsearch/index/store/CorruptedTranslogTests.java index 4692823e9a2..738f4c106b1 100644 --- a/src/test/java/org/elasticsearch/index/store/CorruptedTranslogTests.java +++ b/src/test/java/org/elasticsearch/index/store/CorruptedTranslogTests.java @@ -82,7 +82,6 @@ public class CorruptedTranslogTests extends ElasticsearchIntegrationTest { .put("index.number_of_shards", 1) .put("index.number_of_replicas", 0) .put("index.refresh_interval", "-1") - .put(IndexStoreModule.STORE_TYPE, IndexStoreModule.Type.DEFAULT) // no mock store - it commits for check-index .put(MockEngineSupport.FLUSH_ON_CLOSE_RATIO, 0.0d) // never flush - always recover from translog .put(IndexShard.INDEX_FLUSH_ON_CLOSE, false) // never flush - always recover from translog .put(TranslogConfig.INDEX_TRANSLOG_SYNC_INTERVAL, "1s") // fsync the translog every second diff --git a/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreTests.java b/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreTests.java index 37adeca328f..4ef3a153204 100644 --- a/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreTests.java +++ b/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreTests.java @@ -1556,7 +1556,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests { SnapshotStatus snapshotStatus = client.admin().cluster().prepareSnapshotStatus("test-repo").setSnapshots("test-1").get().getSnapshots().get(0); List shards = snapshotStatus.getShards(); for (SnapshotIndexShardStatus status : shards) { - assertThat(status.getStats().getProcessedFiles(), equalTo(1)); // we flush before the snapshot such that we have to process the segments_N files + assertThat(status.getStats().getProcessedFiles(), equalTo(0)); } } diff --git a/src/test/java/org/elasticsearch/test/store/MockFSDirectoryService.java b/src/test/java/org/elasticsearch/test/store/MockFSDirectoryService.java index 2a8d7cff621..00e493da1c3 100644 --- a/src/test/java/org/elasticsearch/test/store/MockFSDirectoryService.java +++ b/src/test/java/org/elasticsearch/test/store/MockFSDirectoryService.java @@ -107,16 +107,7 @@ public class MockFSDirectoryService extends FsDirectoryService { public void beforeIndexShardClosed(ShardId sid, @Nullable IndexShard indexShard, @IndexSettings Settings indexSettings) { if (indexShard != null && shardId.equals(sid)) { - logger.info("{} shard state before potentially flushing is {}", indexShard.shardId(), indexShard.state()); if (validCheckIndexStates.contains(indexShard.state()) && IndexMetaData.isOnSharedFilesystem(indexSettings) == false) { - // When the the internal engine closes we do a rollback, which removes uncommitted segments - // By doing a commit flush we perform a Lucene commit, but don't clear the translog, - // so that even in tests where don't flush we can check the integrity of the Lucene index - if (indexShard.engine().hasUncommittedChanges()) { // only if we have any changes - logger.info("{} flushing in order to run checkindex", indexShard.shardId()); - Releasables.close(indexShard.engine().snapshotIndex(true)); // Keep translog for tests that rely on replaying it - } - logger.info("{} flush finished in beforeIndexShardClosed", indexShard.shardId()); canRun = true; } } From 98640310c2adff8f8d164590faf019193201c747 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 18 May 2015 14:32:36 +0200 Subject: [PATCH 36/56] Remove dead code / unused class --- .../elasticsearch/index/engine/Engine.java | 23 +------------------ 1 file changed, 1 insertion(+), 22 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/engine/Engine.java b/src/main/java/org/elasticsearch/index/engine/Engine.java index 3f0d10c2060..c2f7f63a76a 100644 --- a/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -534,31 +534,10 @@ public abstract class Engine implements Closeable { return t; } - public static interface FailedEngineListener { + public interface FailedEngineListener { void onFailedEngine(ShardId shardId, String reason, @Nullable Throwable t); } - /** - * Recovery allow to start the recovery process. It is built of three phases. - *

- *

The first phase allows to take a snapshot of the master index. Once this - * is taken, no commit operations are effectively allowed on the index until the recovery - * phases are through. - *

- *

The seconds phase takes a snapshot of the current transaction log. - *

- *

The last phase returns the remaining transaction log. During this phase, no dirty - * operations are allowed on the index. - */ - public static interface RecoveryHandler { - - void phase1(SnapshotIndexCommit snapshot); - - void phase2(Translog.Snapshot snapshot); - - void phase3(Translog.Snapshot snapshot); - } - public static class Searcher implements Releasable { private final String source; From 8bdfd21b70568a39ebf5232e8c395f5206a7a63f Mon Sep 17 00:00:00 2001 From: Alexander Reelsen Date: Mon, 18 May 2015 14:40:54 +0200 Subject: [PATCH 37/56] Internal: Propagate headers & contexts to sub-requests Whenever a query parser (or any other component) issues another request as part of a request, the headers and the context has to be supplied as well. In order to do this, the `SearchContext` has to have those headers available, which in turn means, the shard level request needs to copy those from the original `SearchRequest` This commit introduces two new interface to supply the needed methods to work with context and headers. Closes #10979 --- .../percolate/TransportPercolateAction.java | 1 - .../TransportShardSingleOperationAction.java | 1 - ...older.java => ContextAndHeaderHolder.java} | 111 +++-- .../org/elasticsearch/common/HasContext.java | 82 ++++ .../common/HasContextAndHeaders.java | 33 ++ .../org/elasticsearch/common/HasHeaders.java | 38 ++ .../index/query/GeoShapeQueryParser.java | 16 +- .../index/query/MoreLikeThisQueryParser.java | 6 +- .../index/query/TermsQueryParser.java | 6 +- .../index/search/shape/ShapeFetchService.java | 14 +- .../percolator/PercolateContext.java | 92 +++- .../org/elasticsearch/rest/RestRequest.java | 4 +- .../elasticsearch/script/ScriptService.java | 3 +- .../search/internal/DefaultSearchContext.java | 126 +++-- .../internal/FilteredSearchContext.java | 82 +++- .../search/internal/SearchContext.java | 5 +- .../internal/ShardSearchLocalRequest.java | 13 +- .../search/internal/ShardSearchRequest.java | 5 +- .../search/suggest/SuggestParseElement.java | 1 + .../transport/TransportMessage.java | 37 +- .../elasticsearch/test/TestSearchContext.java | 75 +++ .../ContextAndHeaderTransportTests.java | 438 ++++++++++++++++++ 22 files changed, 1022 insertions(+), 167 deletions(-) rename src/main/java/org/elasticsearch/common/{ContextHolder.java => ContextAndHeaderHolder.java} (57%) create mode 100644 src/main/java/org/elasticsearch/common/HasContext.java create mode 100644 src/main/java/org/elasticsearch/common/HasContextAndHeaders.java create mode 100644 src/main/java/org/elasticsearch/common/HasHeaders.java create mode 100644 src/test/java/org/elasticsearch/transport/ContextAndHeaderTransportTests.java diff --git a/src/main/java/org/elasticsearch/action/percolate/TransportPercolateAction.java b/src/main/java/org/elasticsearch/action/percolate/TransportPercolateAction.java index 98928c09b7c..d1ee7be3b19 100644 --- a/src/main/java/org/elasticsearch/action/percolate/TransportPercolateAction.java +++ b/src/main/java/org/elasticsearch/action/percolate/TransportPercolateAction.java @@ -18,7 +18,6 @@ */ package org.elasticsearch.action.percolate; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.get.GetRequest; diff --git a/src/main/java/org/elasticsearch/action/support/single/shard/TransportShardSingleOperationAction.java b/src/main/java/org/elasticsearch/action/support/single/shard/TransportShardSingleOperationAction.java index db6260e6f85..6e7dcd224cb 100644 --- a/src/main/java/org/elasticsearch/action/support/single/shard/TransportShardSingleOperationAction.java +++ b/src/main/java/org/elasticsearch/action/support/single/shard/TransportShardSingleOperationAction.java @@ -19,7 +19,6 @@ package org.elasticsearch.action.support.single.shard; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.NoShardAvailableActionException; diff --git a/src/main/java/org/elasticsearch/common/ContextHolder.java b/src/main/java/org/elasticsearch/common/ContextAndHeaderHolder.java similarity index 57% rename from src/main/java/org/elasticsearch/common/ContextHolder.java rename to src/main/java/org/elasticsearch/common/ContextAndHeaderHolder.java index c7676e90b74..338ea6b683d 100644 --- a/src/main/java/org/elasticsearch/common/ContextHolder.java +++ b/src/main/java/org/elasticsearch/common/ContextAndHeaderHolder.java @@ -23,20 +23,21 @@ import com.carrotsearch.hppc.ObjectObjectAssociativeContainer; import com.carrotsearch.hppc.ObjectObjectHashMap; import org.elasticsearch.common.collect.ImmutableOpenMap; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + /** * */ -public class ContextHolder { +public class ContextAndHeaderHolder implements HasContextAndHeaders { private ObjectObjectHashMap context; + protected Map headers; - /** - * Attaches the given value to the context. - * - * @return The previous value that was associated with the given key in the context, or - * {@code null} if there was none. - */ @SuppressWarnings("unchecked") + @Override public final synchronized V putInContext(Object key, Object value) { if (context == null) { context = new ObjectObjectHashMap<>(2); @@ -44,9 +45,7 @@ public class ContextHolder { return (V) context.put(key, value); } - /** - * Attaches the given values to the context - */ + @Override public final synchronized void putAllInContext(ObjectObjectAssociativeContainer map) { if (map == null) { return; @@ -58,72 +57,98 @@ public class ContextHolder { } } - /** - * @return The context value that is associated with the given key - * - * @see #putInContext(Object, Object) - */ @SuppressWarnings("unchecked") + @Override public final synchronized V getFromContext(Object key) { return context != null ? (V) context.get(key) : null; } - /** - * @param defaultValue The default value that should be returned for the given key, if no - * value is currently associated with it. - * - * @return The value that is associated with the given key in the context - * - * @see #putInContext(Object, Object) - */ @SuppressWarnings("unchecked") + @Override public final synchronized V getFromContext(Object key, V defaultValue) { V value = getFromContext(key); return value == null ? defaultValue : value; } - /** - * Checks if the context contains an entry with the given key - */ + @Override public final synchronized boolean hasInContext(Object key) { return context != null && context.containsKey(key); } - /** - * @return The number of values attached in the context. - */ + @Override public final synchronized int contextSize() { return context != null ? context.size() : 0; } - /** - * Checks if the context is empty. - */ + @Override public final synchronized boolean isContextEmpty() { return context == null || context.isEmpty(); } - /** - * @return A safe immutable copy of the current context. - */ + @Override public synchronized ImmutableOpenMap getContext() { return context != null ? ImmutableOpenMap.copyOf(context) : ImmutableOpenMap.of(); } - /** - * Copies the context from the given context holder to this context holder. Any shared keys between - * the two context will be overridden by the given context holder. - */ - public synchronized void copyContextFrom(ContextHolder other) { + @Override + public synchronized void copyContextFrom(HasContext other) { + if (other == null) { + return; + } + synchronized (other) { - if (other.context == null) { + ImmutableOpenMap otherContext = other.getContext(); + if (otherContext == null) { return; } if (context == null) { - context = new ObjectObjectHashMap<>(other.context); + ObjectObjectHashMap map = new ObjectObjectHashMap<>(other.getContext().size()); + map.putAll(otherContext); + this.context = map; } else { - context.putAll(other.context); + context.putAll(otherContext); } } } + + @SuppressWarnings("unchecked") + @Override + public final T putHeader(String key, Object value) { + if (headers == null) { + headers = new HashMap<>(); + } + headers.put(key, value); + return (T) this; + } + + @SuppressWarnings("unchecked") + @Override + public final V getHeader(String key) { + return headers != null ? (V) headers.get(key) : null; + } + + @Override + public final boolean hasHeader(String key) { + return headers != null && headers.containsKey(key); + } + + @Override + public Set getHeaders() { + return headers != null ? headers.keySet() : Collections.emptySet(); + } + + @Override + public void copyHeadersFrom(HasHeaders from) { + if (from != null && from.getHeaders() != null && !from.getHeaders().isEmpty()) { + for (String headerName : from.getHeaders()) { + putHeader(headerName, from.getHeader(headerName)); + } + } + } + + @Override + public void copyContextAndHeadersFrom(HasContextAndHeaders other) { + copyContextFrom(other); + copyHeadersFrom(other); + } } diff --git a/src/main/java/org/elasticsearch/common/HasContext.java b/src/main/java/org/elasticsearch/common/HasContext.java new file mode 100644 index 00000000000..6a303e39bae --- /dev/null +++ b/src/main/java/org/elasticsearch/common/HasContext.java @@ -0,0 +1,82 @@ +/* + * 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.common; + +import com.carrotsearch.hppc.ObjectObjectAssociativeContainer; +import org.elasticsearch.common.collect.ImmutableOpenMap; + +public interface HasContext { + + /** + * Attaches the given value to the context. + * + * @return The previous value that was associated with the given key in the context, or + * {@code null} if there was none. + */ + V putInContext(Object key, Object value); + + /** + * Attaches the given values to the context + */ + void putAllInContext(ObjectObjectAssociativeContainer map); + + /** + * @return The context value that is associated with the given key + * + * @see #putInContext(Object, Object) + */ + V getFromContext(Object key); + + /** + * @param defaultValue The default value that should be returned for the given key, if no + * value is currently associated with it. + * + * @return The value that is associated with the given key in the context + * + * @see #putInContext(Object, Object) + */ + V getFromContext(Object key, V defaultValue); + + /** + * Checks if the context contains an entry with the given key + */ + boolean hasInContext(Object key); + + /** + * @return The number of values attached in the context. + */ + int contextSize(); + + /** + * Checks if the context is empty. + */ + boolean isContextEmpty(); + + /** + * @return A safe immutable copy of the current context. + */ + ImmutableOpenMap getContext(); + + /** + * Copies the context from the given context holder to this context holder. Any shared keys between + * the two context will be overridden by the given context holder. + */ + void copyContextFrom(HasContext other); +} diff --git a/src/main/java/org/elasticsearch/common/HasContextAndHeaders.java b/src/main/java/org/elasticsearch/common/HasContextAndHeaders.java new file mode 100644 index 00000000000..35bea9a7778 --- /dev/null +++ b/src/main/java/org/elasticsearch/common/HasContextAndHeaders.java @@ -0,0 +1,33 @@ +/* + * 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.common; + +/** + * marker interface + */ +public interface HasContextAndHeaders extends HasContext, HasHeaders { + + /** + * copies over the context and the headers + * @param other another object supporting headers and context + */ + void copyContextAndHeadersFrom(HasContextAndHeaders other); + +} diff --git a/src/main/java/org/elasticsearch/common/HasHeaders.java b/src/main/java/org/elasticsearch/common/HasHeaders.java new file mode 100644 index 00000000000..b296362c81c --- /dev/null +++ b/src/main/java/org/elasticsearch/common/HasHeaders.java @@ -0,0 +1,38 @@ +/* + * 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.common; + +import java.util.Set; + +/** + * + */ +public interface HasHeaders { + + V putHeader(String key, V value); + + V getHeader(String key); + + boolean hasHeader(String key); + + Set getHeaders(); + + void copyHeadersFrom(HasHeaders from); +} diff --git a/src/main/java/org/elasticsearch/index/query/GeoShapeQueryParser.java b/src/main/java/org/elasticsearch/index/query/GeoShapeQueryParser.java index ffe72547fb5..83e35e13d5e 100644 --- a/src/main/java/org/elasticsearch/index/query/GeoShapeQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/GeoShapeQueryParser.java @@ -19,15 +19,12 @@ package org.elasticsearch.index.query; -import org.apache.lucene.search.BooleanClause; -import org.apache.lucene.search.BooleanQuery; -import org.apache.lucene.search.ConstantScoreQuery; -import org.apache.lucene.search.Filter; -import org.apache.lucene.search.Query; +import org.apache.lucene.search.*; import org.apache.lucene.spatial.prefix.PrefixTreeStrategy; import org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy; import org.apache.lucene.spatial.query.SpatialArgs; import org.apache.lucene.spatial.query.SpatialOperation; +import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.geo.ShapeRelation; @@ -38,6 +35,7 @@ import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.geo.GeoShapeFieldMapper; import org.elasticsearch.index.search.shape.ShapeFetchService; +import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; @@ -116,7 +114,9 @@ public class GeoShapeQueryParser implements QueryParser { } else if (type == null) { throw new QueryParsingException(parseContext, "Type for indexed shape not provided"); } - shape = fetchService.fetch(id, type, index, shapePath); + GetRequest getRequest = new GetRequest(index, type, id); + getRequest.copyContextAndHeadersFrom(SearchContext.current()); + shape = fetchService.fetch(getRequest, shapePath); } else { throw new QueryParsingException(parseContext, "[geo_shape] query does not support [" + currentFieldName + "]"); } @@ -180,7 +180,7 @@ public class GeoShapeQueryParser implements QueryParser { public void setFetchService(@Nullable ShapeFetchService fetchService) { this.fetchService = fetchService; } - + public static SpatialArgs getArgs(ShapeBuilder shape, ShapeRelation relation) { switch(relation) { case DISJOINT: @@ -191,7 +191,7 @@ public class GeoShapeQueryParser implements QueryParser { return new SpatialArgs(SpatialOperation.IsWithin, shape.build()); default: throw new IllegalArgumentException(""); - + } } } diff --git a/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryParser.java b/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryParser.java index 4ba930cc742..c1d3792e7f6 100644 --- a/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryParser.java @@ -21,7 +21,6 @@ package org.elasticsearch.index.query; import com.google.common.collect.Lists; import com.google.common.collect.Sets; - import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.queries.TermsQuery; import org.apache.lucene.search.BooleanClause; @@ -40,6 +39,7 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.analysis.Analysis; import org.elasticsearch.index.mapper.internal.UidFieldMapper; import org.elasticsearch.index.search.morelikethis.MoreLikeThisFetchService; +import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; import java.util.ArrayList; @@ -245,6 +245,7 @@ public class MoreLikeThisQueryParser implements QueryParser { if (!likeItems.isEmpty()) { // set default index, type and fields if not specified MultiTermVectorsRequest items = likeItems; + for (TermVectorsRequest item : ignoreItems) { items.add(item); } @@ -272,7 +273,7 @@ public class MoreLikeThisQueryParser implements QueryParser { } } // fetching the items with multi-termvectors API - BooleanQuery boolQuery = new BooleanQuery(); + items.copyContextAndHeadersFrom(SearchContext.current()); MultiTermVectorsResponse responses = fetchService.fetchResponse(items); // getting the Fields for liked items @@ -286,6 +287,7 @@ public class MoreLikeThisQueryParser implements QueryParser { } } + BooleanQuery boolQuery = new BooleanQuery(); boolQuery.add(mltQuery, BooleanClause.Occur.SHOULD); // exclude the items from the search diff --git a/src/main/java/org/elasticsearch/index/query/TermsQueryParser.java b/src/main/java/org/elasticsearch/index/query/TermsQueryParser.java index b26c65459a9..d663885d5b6 100644 --- a/src/main/java/org/elasticsearch/index/query/TermsQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/TermsQueryParser.java @@ -20,7 +20,6 @@ package org.elasticsearch.index.query; import com.google.common.collect.Lists; - import org.apache.lucene.index.Term; import org.apache.lucene.queries.TermsQuery; import org.apache.lucene.search.BooleanClause.Occur; @@ -40,6 +39,7 @@ import org.elasticsearch.common.xcontent.support.XContentMapValues; import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.indices.cache.filter.terms.TermsLookup; +import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; import java.util.List; @@ -171,7 +171,9 @@ public class TermsQueryParser implements QueryParser { if (lookupId != null) { final TermsLookup lookup = new TermsLookup(lookupIndex, lookupType, lookupId, lookupRouting, lookupPath, parseContext); - final GetResponse getResponse = client.get(new GetRequest(lookup.getIndex(), lookup.getType(), lookup.getId()).preference("_local").routing(lookup.getRouting())).actionGet(); + GetRequest getRequest = new GetRequest(lookup.getIndex(), lookup.getType(), lookup.getId()).preference("_local").routing(lookup.getRouting()); + getRequest.copyContextAndHeadersFrom(SearchContext.current()); + final GetResponse getResponse = client.get(getRequest).actionGet(); if (getResponse.isExists()) { List values = XContentMapValues.extractRawValues(lookup.getPath(), getResponse.getSourceAsMap()); terms.addAll(values); diff --git a/src/main/java/org/elasticsearch/index/search/shape/ShapeFetchService.java b/src/main/java/org/elasticsearch/index/search/shape/ShapeFetchService.java index dbe42131d1e..97d08045a65 100644 --- a/src/main/java/org/elasticsearch/index/search/shape/ShapeFetchService.java +++ b/src/main/java/org/elasticsearch/index/search/shape/ShapeFetchService.java @@ -48,17 +48,17 @@ public class ShapeFetchService extends AbstractComponent { /** * Fetches the Shape with the given ID in the given type and index. * - * @param id ID of the Shape to fetch - * @param type Index type where the Shape is indexed - * @param index Index where the Shape is indexed + * @param getRequest GetRequest containing index, type and id * @param path Name or path of the field in the Shape Document where the Shape itself is located * @return Shape with the given ID * @throws IOException Can be thrown while parsing the Shape Document and extracting the Shape */ - public ShapeBuilder fetch(String id, String type, String index, String path) throws IOException { - GetResponse response = client.get(new GetRequest(index, type, id).preference("_local").operationThreaded(false)).actionGet(); + public ShapeBuilder fetch(GetRequest getRequest,String path) throws IOException { + getRequest.preference("_local"); + getRequest.operationThreaded(false); + GetResponse response = client.get(getRequest).actionGet(); if (!response.isExists()) { - throw new IllegalArgumentException("Shape with ID [" + id + "] in type [" + type + "] not found"); + throw new IllegalArgumentException("Shape with ID [" + getRequest.id() + "] in type [" + getRequest.type() + "] not found"); } String[] pathElements = Strings.splitStringToArray(path, '.'); @@ -81,7 +81,7 @@ public class ShapeFetchService extends AbstractComponent { } } } - throw new IllegalStateException("Shape with name [" + id + "] found but missing " + path + " field"); + throw new IllegalStateException("Shape with name [" + getRequest.id() + "] found but missing " + path + " field"); } finally { if (parser != null) { parser.close(); diff --git a/src/main/java/org/elasticsearch/percolator/PercolateContext.java b/src/main/java/org/elasticsearch/percolator/PercolateContext.java index 592423f63f6..407c7a2b141 100644 --- a/src/main/java/org/elasticsearch/percolator/PercolateContext.java +++ b/src/main/java/org/elasticsearch/percolator/PercolateContext.java @@ -18,8 +18,8 @@ */ package org.elasticsearch.percolator; +import com.carrotsearch.hppc.ObjectObjectAssociativeContainer; import com.google.common.collect.ImmutableList; - import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.LeafReaderContext; @@ -32,6 +32,10 @@ import org.apache.lucene.util.Counter; import org.elasticsearch.action.percolate.PercolateShardRequest; import org.elasticsearch.action.search.SearchType; import org.elasticsearch.cache.recycler.PageCacheRecycler; +import org.elasticsearch.common.HasContext; +import org.elasticsearch.common.HasContextAndHeaders; +import org.elasticsearch.common.HasHeaders; +import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.text.StringText; import org.elasticsearch.common.util.BigArrays; @@ -61,11 +65,7 @@ import org.elasticsearch.search.fetch.innerhits.InnerHitsContext; import org.elasticsearch.search.fetch.script.ScriptFieldsContext; import org.elasticsearch.search.fetch.source.FetchSourceContext; import org.elasticsearch.search.highlight.SearchContextHighlight; -import org.elasticsearch.search.internal.ContextIndexSearcher; -import org.elasticsearch.search.internal.InternalSearchHit; -import org.elasticsearch.search.internal.InternalSearchHitField; -import org.elasticsearch.search.internal.SearchContext; -import org.elasticsearch.search.internal.ShardSearchRequest; +import org.elasticsearch.search.internal.*; import org.elasticsearch.search.lookup.LeafSearchLookup; import org.elasticsearch.search.lookup.SearchLookup; import org.elasticsearch.search.query.QuerySearchResult; @@ -73,9 +73,7 @@ import org.elasticsearch.search.rescore.RescoreSearchContext; import org.elasticsearch.search.scan.ScanContext; import org.elasticsearch.search.suggest.SuggestionSearchContext; -import java.util.HashMap; -import java.util.List; -import java.util.Map; +import java.util.*; import java.util.concurrent.ConcurrentMap; /** @@ -693,4 +691,80 @@ public class PercolateContext extends SearchContext { throw new UnsupportedOperationException(); } + @Override + public V putInContext(Object key, Object value) { + assert false : "percolatocontext does not support contexts & headers"; + return null; + } + + @Override + public void putAllInContext(ObjectObjectAssociativeContainer map) { + assert false : "percolatocontext does not support contexts & headers"; + } + + @Override + public V getFromContext(Object key) { + return null; + } + + @Override + public V getFromContext(Object key, V defaultValue) { + return defaultValue; + } + + @Override + public boolean hasInContext(Object key) { + return false; + } + + @Override + public int contextSize() { + return 0; + } + + @Override + public boolean isContextEmpty() { + return true; + } + + @Override + public ImmutableOpenMap getContext() { + return ImmutableOpenMap.of(); + } + + @Override + public void copyContextFrom(HasContext other) { + assert false : "percolatocontext does not support contexts & headers"; + } + + @Override + public V putHeader(String key, V value) { + assert false : "percolatocontext does not support contexts & headers"; + return value; + } + + @Override + public V getHeader(String key) { + return null; + } + + @Override + public boolean hasHeader(String key) { + return false; + } + + @Override + public Set getHeaders() { + return Collections.EMPTY_SET; + } + + @Override + public void copyHeadersFrom(HasHeaders from) { + assert false : "percolatocontext does not support contexts & headers"; + } + + @Override + public void copyContextAndHeadersFrom(HasContextAndHeaders other) { + assert false : "percolatocontext does not support contexts & headers"; + } } diff --git a/src/main/java/org/elasticsearch/rest/RestRequest.java b/src/main/java/org/elasticsearch/rest/RestRequest.java index 6f2fd926ab9..113bfabe4ff 100644 --- a/src/main/java/org/elasticsearch/rest/RestRequest.java +++ b/src/main/java/org/elasticsearch/rest/RestRequest.java @@ -20,7 +20,7 @@ package org.elasticsearch.rest; import org.elasticsearch.common.Booleans; -import org.elasticsearch.common.ContextHolder; +import org.elasticsearch.common.ContextAndHeaderHolder; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; @@ -38,7 +38,7 @@ import static org.elasticsearch.common.unit.TimeValue.parseTimeValue; /** * */ -public abstract class RestRequest extends ContextHolder implements ToXContent.Params { +public abstract class RestRequest extends ContextAndHeaderHolder implements ToXContent.Params { public enum Method { GET, POST, PUT, DELETE, OPTIONS, HEAD diff --git a/src/main/java/org/elasticsearch/script/ScriptService.java b/src/main/java/org/elasticsearch/script/ScriptService.java index 8e363068c30..3bbec6b225e 100644 --- a/src/main/java/org/elasticsearch/script/ScriptService.java +++ b/src/main/java/org/elasticsearch/script/ScriptService.java @@ -25,7 +25,6 @@ import com.google.common.cache.CacheBuilder; import com.google.common.cache.RemovalListener; import com.google.common.cache.RemovalNotification; import com.google.common.collect.ImmutableMap; - import org.apache.lucene.util.IOUtils; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.delete.DeleteRequest; @@ -57,6 +56,7 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.env.Environment; import org.elasticsearch.index.query.TemplateQueryParser; import org.elasticsearch.script.groovy.GroovyScriptEngineService; +import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.lookup.SearchLookup; import org.elasticsearch.watcher.FileChangesListener; import org.elasticsearch.watcher.FileWatcher; @@ -288,6 +288,7 @@ public class ScriptService extends AbstractComponent implements Closeable { } scriptLang = validateScriptLanguage(scriptLang); GetRequest getRequest = new GetRequest(SCRIPT_INDEX, scriptLang, id); + getRequest.copyContextAndHeadersFrom(SearchContext.current()); GetResponse responseFields = client.get(getRequest).actionGet(); if (responseFields.isExists()) { return getScriptFromResponse(responseFields); diff --git a/src/main/java/org/elasticsearch/search/internal/DefaultSearchContext.java b/src/main/java/org/elasticsearch/search/internal/DefaultSearchContext.java index 4bbb3ec09af..e15a778767d 100644 --- a/src/main/java/org/elasticsearch/search/internal/DefaultSearchContext.java +++ b/src/main/java/org/elasticsearch/search/internal/DefaultSearchContext.java @@ -19,6 +19,7 @@ package org.elasticsearch.search.internal; +import com.carrotsearch.hppc.ObjectObjectAssociativeContainer; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; @@ -33,7 +34,11 @@ import org.apache.lucene.search.Sort; import org.apache.lucene.util.Counter; import org.elasticsearch.action.search.SearchType; import org.elasticsearch.cache.recycler.PageCacheRecycler; +import org.elasticsearch.common.HasContext; +import org.elasticsearch.common.HasContextAndHeaders; +import org.elasticsearch.common.HasHeaders; import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.common.lucene.search.function.BoostScoreFunction; @@ -72,6 +77,7 @@ import org.elasticsearch.search.suggest.SuggestionSearchContext; import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Set; /** * @@ -79,101 +85,56 @@ import java.util.List; public class DefaultSearchContext extends SearchContext { private final long id; - private final ShardSearchRequest request; - private final SearchShardTarget shardTarget; private final Counter timeEstimateCounter; - private SearchType searchType; - private final Engine.Searcher engineSearcher; - private final ScriptService scriptService; - private final PageCacheRecycler pageCacheRecycler; - private final BigArrays bigArrays; - private final IndexShard indexShard; - private final IndexService indexService; - private final ContextIndexSearcher searcher; - private final DfsSearchResult dfsResult; - private final QuerySearchResult queryResult; - private final FetchSearchResult fetchResult; - // lazy initialized only if needed private ScanContext scanContext; - private float queryBoost = 1.0f; - // timeout in millis private long timeoutInMillis = -1; - // terminate after count private int terminateAfter = DEFAULT_TERMINATE_AFTER; - - private List groupStats; - private Scroll scroll; - private boolean explain; - private boolean version = false; // by default, we don't return versions - private List fieldNames; private FieldDataFieldsContext fieldDataFields; private ScriptFieldsContext scriptFields; private FetchSourceContext fetchSourceContext; - private int from = -1; - private int size = -1; - private Sort sort; - private Float minimumScore; - private boolean trackScores = false; // when sorting, track scores as well... - private ParsedQuery originalQuery; - private Query query; - private ParsedQuery postFilter; - private Query aliasFilter; - private int[] docIdsToLoad; - private int docsIdsToLoadFrom; - private int docsIdsToLoadSize; - private SearchContextAggregations aggregations; - private SearchContextHighlight highlight; - private SuggestionSearchContext suggest; - private List rescore; - private SearchLookup searchLookup; - private boolean queryRewritten; - private volatile long keepAlive; - private ScoreDoc lastEmittedDoc; - private volatile long lastAccessTime = -1; - private InnerHitsContext innerHitsContext; public DefaultSearchContext(long id, ShardSearchRequest request, SearchShardTarget shardTarget, @@ -790,4 +751,79 @@ public class DefaultSearchContext extends SearchContext { public InnerHitsContext innerHits() { return innerHitsContext; } + + @Override + public V putInContext(Object key, Object value) { + return request.putInContext(key, value); + } + + @Override + public void putAllInContext(ObjectObjectAssociativeContainer map) { + request.putAllInContext(map); + } + + @Override + public V getFromContext(Object key) { + return request.getFromContext(key); + } + + @Override + public V getFromContext(Object key, V defaultValue) { + return request.getFromContext(key, defaultValue); + } + + @Override + public boolean hasInContext(Object key) { + return request.hasInContext(key); + } + + @Override + public int contextSize() { + return request.contextSize(); + } + + @Override + public boolean isContextEmpty() { + return request.isContextEmpty(); + } + + @Override + public ImmutableOpenMap getContext() { + return request.getContext(); + } + + @Override + public void copyContextFrom(HasContext other) { + request.copyContextFrom(other); + } + + @Override + public V putHeader(String key, V value) { + return request.putHeader(key, value); + } + + @Override + public V getHeader(String key) { + return request.getHeader(key); + } + + @Override + public boolean hasHeader(String key) { + return request.hasHeader(key); + } + + @Override + public Set getHeaders() { + return request.getHeaders(); + } + + @Override + public void copyHeadersFrom(HasHeaders from) { + request.copyHeadersFrom(from); + } + + @Override + public void copyContextAndHeadersFrom(HasContextAndHeaders other) { + request.copyContextAndHeadersFrom(other); + } } diff --git a/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java b/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java index 7fce6e96e24..2ebb7f0e337 100644 --- a/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java +++ b/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java @@ -19,23 +19,26 @@ package org.elasticsearch.search.internal; +import com.carrotsearch.hppc.ObjectObjectAssociativeContainer; import org.apache.lucene.search.Query; import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.Sort; import org.apache.lucene.util.Counter; import org.elasticsearch.action.search.SearchType; import org.elasticsearch.cache.recycler.PageCacheRecycler; +import org.elasticsearch.common.HasContext; +import org.elasticsearch.common.HasContextAndHeaders; +import org.elasticsearch.common.HasHeaders; +import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.cache.bitset.BitsetFilterCache; -import org.elasticsearch.index.cache.filter.FilterCache; import org.elasticsearch.index.fielddata.IndexFieldDataService; import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.FieldMappers; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.query.IndexQueryParserService; import org.elasticsearch.index.query.ParsedQuery; -import org.elasticsearch.index.query.ParsedQuery; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.script.ScriptService; @@ -56,6 +59,7 @@ import org.elasticsearch.search.scan.ScanContext; import org.elasticsearch.search.suggest.SuggestionSearchContext; import java.util.List; +import java.util.Set; /** */ @@ -557,4 +561,78 @@ public abstract class FilteredSearchContext extends SearchContext { return in.timeEstimateCounter(); } + @Override + public V putInContext(Object key, Object value) { + return in.putInContext(key, value); + } + + @Override + public void putAllInContext(ObjectObjectAssociativeContainer map) { + in.putAllInContext(map); + } + + @Override + public V getFromContext(Object key) { + return in.getFromContext(key); + } + + @Override + public V getFromContext(Object key, V defaultValue) { + return in.getFromContext(key, defaultValue); + } + + @Override + public boolean hasInContext(Object key) { + return in.hasInContext(key); + } + + @Override + public int contextSize() { + return in.contextSize(); + } + + @Override + public boolean isContextEmpty() { + return in.isContextEmpty(); + } + + @Override + public ImmutableOpenMap getContext() { + return in.getContext(); + } + + @Override + public void copyContextFrom(HasContext other) { + in.copyContextFrom(other); + } + + @Override + public V putHeader(String key, V value) { + return in.putHeader(key, value); + } + + @Override + public V getHeader(String key) { + return in.getHeader(key); + } + + @Override + public boolean hasHeader(String key) { + return in.hasHeader(key); + } + + @Override + public Set getHeaders() { + return in.getHeaders(); + } + + @Override + public void copyHeadersFrom(HasHeaders from) { + in.copyHeadersFrom(from); + } + + @Override + public void copyContextAndHeadersFrom(HasContextAndHeaders other) { + in.copyContextAndHeadersFrom(other); + } } diff --git a/src/main/java/org/elasticsearch/search/internal/SearchContext.java b/src/main/java/org/elasticsearch/search/internal/SearchContext.java index c60655b27ed..099ed8cff09 100644 --- a/src/main/java/org/elasticsearch/search/internal/SearchContext.java +++ b/src/main/java/org/elasticsearch/search/internal/SearchContext.java @@ -28,6 +28,9 @@ import org.apache.lucene.search.Sort; import org.apache.lucene.util.Counter; import org.elasticsearch.action.search.SearchType; import org.elasticsearch.cache.recycler.PageCacheRecycler; +import org.elasticsearch.common.HasContext; +import org.elasticsearch.common.HasContextAndHeaders; +import org.elasticsearch.common.HasHeaders; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasables; @@ -67,7 +70,7 @@ import java.util.concurrent.atomic.AtomicBoolean; /** */ -public abstract class SearchContext implements Releasable { +public abstract class SearchContext implements Releasable, HasContextAndHeaders { private static ThreadLocal current = new ThreadLocal<>(); public final static int DEFAULT_TERMINATE_AFTER = 0; diff --git a/src/main/java/org/elasticsearch/search/internal/ShardSearchLocalRequest.java b/src/main/java/org/elasticsearch/search/internal/ShardSearchLocalRequest.java index 3467252c778..c1b5ec9e7d0 100644 --- a/src/main/java/org/elasticsearch/search/internal/ShardSearchLocalRequest.java +++ b/src/main/java/org/elasticsearch/search/internal/ShardSearchLocalRequest.java @@ -22,6 +22,7 @@ package org.elasticsearch.search.internal; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchType; import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.common.ContextAndHeaderHolder; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.BytesStreamOutput; @@ -57,22 +58,15 @@ import static org.elasticsearch.search.Scroll.readScroll; * */ -public class ShardSearchLocalRequest implements ShardSearchRequest { +public class ShardSearchLocalRequest extends ContextAndHeaderHolder implements ShardSearchRequest { private String index; - private int shardId; - private int numberOfShards; - private SearchType searchType; - private Scroll scroll; - private String[] types = Strings.EMPTY_ARRAY; - private String[] filteringAliases; - private BytesReference source; private BytesReference extraSource; private BytesReference templateSource; @@ -80,7 +74,6 @@ public class ShardSearchLocalRequest implements ShardSearchRequest { private ScriptService.ScriptType templateType; private Map templateParams; private Boolean queryCache; - private long nowInMillis; ShardSearchLocalRequest() { @@ -90,7 +83,6 @@ public class ShardSearchLocalRequest implements ShardSearchRequest { String[] filteringAliases, long nowInMillis) { this(shardRouting.shardId(), numberOfShards, searchRequest.searchType(), searchRequest.source(), searchRequest.types(), searchRequest.queryCache()); - this.extraSource = searchRequest.extraSource(); this.templateSource = searchRequest.templateSource(); this.templateName = searchRequest.templateName(); @@ -99,6 +91,7 @@ public class ShardSearchLocalRequest implements ShardSearchRequest { this.scroll = searchRequest.scroll(); this.filteringAliases = filteringAliases; this.nowInMillis = nowInMillis; + copyContextAndHeadersFrom(searchRequest); } public ShardSearchLocalRequest(String[] types, long nowInMillis) { diff --git a/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java b/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java index 5fae15dddee..8c73ed6026a 100644 --- a/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java +++ b/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java @@ -20,6 +20,9 @@ package org.elasticsearch.search.internal; import org.elasticsearch.action.search.SearchType; +import org.elasticsearch.common.HasContext; +import org.elasticsearch.common.HasContextAndHeaders; +import org.elasticsearch.common.HasHeaders; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.script.ScriptService; import org.elasticsearch.search.Scroll; @@ -32,7 +35,7 @@ import java.util.Map; * It provides all the methods that the {@link org.elasticsearch.search.internal.SearchContext} needs. * Provides a cache key based on its content that can be used to cache shard level response. */ -public interface ShardSearchRequest { +public interface ShardSearchRequest extends HasContextAndHeaders { String index(); diff --git a/src/main/java/org/elasticsearch/search/suggest/SuggestParseElement.java b/src/main/java/org/elasticsearch/search/suggest/SuggestParseElement.java index cf85500cb4b..637ed3d6c48 100644 --- a/src/main/java/org/elasticsearch/search/suggest/SuggestParseElement.java +++ b/src/main/java/org/elasticsearch/search/suggest/SuggestParseElement.java @@ -51,6 +51,7 @@ public final class SuggestParseElement implements SearchParseElement { public SuggestionSearchContext parseInternal(XContentParser parser, MapperService mapperService, IndexQueryParserService queryParserService, String index, int shardId) throws IOException { SuggestionSearchContext suggestionSearchContext = new SuggestionSearchContext(); + BytesRef globalText = null; String fieldName = null; Map suggestionContexts = newHashMap(); diff --git a/src/main/java/org/elasticsearch/transport/TransportMessage.java b/src/main/java/org/elasticsearch/transport/TransportMessage.java index 5e9755013e0..5adb02d3f89 100644 --- a/src/main/java/org/elasticsearch/transport/TransportMessage.java +++ b/src/main/java/org/elasticsearch/transport/TransportMessage.java @@ -19,25 +19,20 @@ package org.elasticsearch.transport; -import org.elasticsearch.common.ContextHolder; +import org.elasticsearch.common.ContextAndHeaderHolder; 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.transport.TransportAddress; import java.io.IOException; -import java.util.Collections; import java.util.HashMap; -import java.util.Map; -import java.util.Set; /** - * The transport message is also a {@link ContextHolder context holder} that holds transient context, that is, + * The transport message is also a {@link ContextAndHeaderHolder context holder} that holds transient context, that is, * the context is not serialized with message. */ -public abstract class TransportMessage> extends ContextHolder implements Streamable { - - private Map headers; +public abstract class TransportMessage> extends ContextAndHeaderHolder implements Streamable { private TransportAddress remoteAddress; @@ -48,8 +43,8 @@ public abstract class TransportMessage> extends // create a new copy of the headers/context, since we are creating a new request // which might have its headers/context changed in the context of that specific request - if (((TransportMessage) message).headers != null) { - this.headers = new HashMap<>(((TransportMessage) message).headers); + if (message.headers != null) { + this.headers = new HashMap<>(message.headers); } copyContextFrom(message); } @@ -62,28 +57,6 @@ public abstract class TransportMessage> extends return remoteAddress; } - @SuppressWarnings("unchecked") - public final TM putHeader(String key, Object value) { - if (headers == null) { - headers = new HashMap<>(); - } - headers.put(key, value); - return (TM) this; - } - - @SuppressWarnings("unchecked") - public final V getHeader(String key) { - return headers != null ? (V) headers.get(key) : null; - } - - public final boolean hasHeader(String key) { - return headers != null && headers.containsKey(key); - } - - public Set getHeaders() { - return headers != null ? headers.keySet() : Collections.emptySet(); - } - @Override public void readFrom(StreamInput in) throws IOException { headers = in.readBoolean() ? in.readMap() : null; diff --git a/src/test/java/org/elasticsearch/test/TestSearchContext.java b/src/test/java/org/elasticsearch/test/TestSearchContext.java index 53a938a1378..42847de819b 100644 --- a/src/test/java/org/elasticsearch/test/TestSearchContext.java +++ b/src/test/java/org/elasticsearch/test/TestSearchContext.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.test; +import com.carrotsearch.hppc.ObjectObjectAssociativeContainer; import org.apache.lucene.search.Filter; import org.apache.lucene.search.Query; import org.apache.lucene.search.ScoreDoc; @@ -25,6 +26,10 @@ import org.apache.lucene.search.Sort; import org.apache.lucene.util.Counter; import org.elasticsearch.action.search.SearchType; import org.elasticsearch.cache.recycler.PageCacheRecycler; +import org.elasticsearch.common.HasContext; +import org.elasticsearch.common.HasContextAndHeaders; +import org.elasticsearch.common.HasHeaders; +import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.analysis.AnalysisService; @@ -59,7 +64,9 @@ import org.elasticsearch.search.scan.ScanContext; import org.elasticsearch.search.suggest.SuggestionSearchContext; import org.elasticsearch.threadpool.ThreadPool; +import java.util.Collections; import java.util.List; +import java.util.Set; public class TestSearchContext extends SearchContext { @@ -597,4 +604,72 @@ public class TestSearchContext extends SearchContext { throw new UnsupportedOperationException(); } + @Override + public V putInContext(Object key, Object value) { + return null; + } + + @Override + public void putAllInContext(ObjectObjectAssociativeContainer map) { + } + + @Override + public V getFromContext(Object key) { + return null; + } + + @Override + public V getFromContext(Object key, V defaultValue) { + return defaultValue; + } + + @Override + public boolean hasInContext(Object key) { + return false; + } + + @Override + public int contextSize() { + return 0; + } + + @Override + public boolean isContextEmpty() { + return true; + } + + @Override + public ImmutableOpenMap getContext() { + return ImmutableOpenMap.of(); + } + + @Override + public void copyContextFrom(HasContext other) { + } + + @Override + public V putHeader(String key, V value) { + return value; + } + + @Override + public V getHeader(String key) { + return null; + } + + @Override + public boolean hasHeader(String key) { + return false; + } + + @Override + public Set getHeaders() { + return Collections.EMPTY_SET; + } + + @Override + public void copyHeadersFrom(HasHeaders from) {} + + @Override + public void copyContextAndHeadersFrom(HasContextAndHeaders other) {} } diff --git a/src/test/java/org/elasticsearch/transport/ContextAndHeaderTransportTests.java b/src/test/java/org/elasticsearch/transport/ContextAndHeaderTransportTests.java new file mode 100644 index 00000000000..6563a9a4a27 --- /dev/null +++ b/src/test/java/org/elasticsearch/transport/ContextAndHeaderTransportTests.java @@ -0,0 +1,438 @@ +/* + * 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.transport; + +import com.carrotsearch.ant.tasks.junit4.dependencies.com.google.common.collect.ImmutableList; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClients; +import org.elasticsearch.action.*; +import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; +import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.indexedscripts.put.PutIndexedScriptRequest; +import org.elasticsearch.action.indexedscripts.put.PutIndexedScriptResponse; +import org.elasticsearch.action.percolate.PercolateResponse; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.support.ActionFilter; +import org.elasticsearch.action.termvectors.MultiTermVectorsRequest; +import org.elasticsearch.client.Client; +import org.elasticsearch.client.FilterClient; +import org.elasticsearch.common.inject.AbstractModule; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.inject.Module; +import org.elasticsearch.common.inject.PreProcessModule; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.http.HttpServerTransport; +import org.elasticsearch.index.query.*; +import org.elasticsearch.plugins.AbstractPlugin; +import org.elasticsearch.rest.RestController; +import org.elasticsearch.script.ScriptService; +import org.elasticsearch.script.groovy.GroovyScriptEngineService; +import org.elasticsearch.script.mustache.MustacheScriptEngineService; +import org.elasticsearch.test.ElasticsearchIntegrationTest; +import org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope; +import org.elasticsearch.test.rest.client.http.HttpRequestBuilder; +import org.elasticsearch.test.rest.client.http.HttpResponse; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.*; + +import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS; +import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder; +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.elasticsearch.node.Node.HTTP_ENABLED; +import static org.elasticsearch.rest.RestStatus.OK; +import static org.elasticsearch.test.ElasticsearchIntegrationTest.Scope.SUITE; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.*; +import static org.hamcrest.Matchers.*; + +@ClusterScope(scope = SUITE) +public class ContextAndHeaderTransportTests extends ElasticsearchIntegrationTest { + + private static final List requests = Collections.synchronizedList(new ArrayList()); + private String randomHeaderKey = randomAsciiOfLength(10); + private String randomHeaderValue = randomAsciiOfLength(20); + private String queryIndex = "query-" + randomAsciiOfLength(10).toLowerCase(Locale.ROOT); + private String lookupIndex = "lookup-" + randomAsciiOfLength(10).toLowerCase(Locale.ROOT); + + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return settingsBuilder() + .put(super.nodeSettings(nodeOrdinal)) + .put("plugin.types", ActionLoggingPlugin.class.getName()) + .put("script.indexed", "on") + .put(HTTP_ENABLED, true) + .build(); + } + + @Before + public void createIndices() throws Exception { + String mapping = jsonBuilder().startObject().startObject("type") + .startObject("properties") + .startObject("location").field("type", "geo_shape").endObject() + .startObject("name").field("type", "string").endObject() + .startObject("title").field("type", "string").field("analyzer", "text").endObject() + .endObject() + .endObject().endObject().string(); + + Settings settings = settingsBuilder() + .put(indexSettings()) + .put(SETTING_NUMBER_OF_SHARDS, 1) // A single shard will help to keep the tests repeatable. + .put("index.analysis.analyzer.text.tokenizer", "standard") + .putArray("index.analysis.analyzer.text.filter", "lowercase", "my_shingle") + .put("index.analysis.filter.my_shingle.type", "shingle") + .put("index.analysis.filter.my_shingle.output_unigrams", true) + .put("index.analysis.filter.my_shingle.min_shingle_size", 2) + .put("index.analysis.filter.my_shingle.max_shingle_size", 3) + .build(); + assertAcked(transportClient().admin().indices().prepareCreate(lookupIndex) + .setSettings(settings).addMapping("type", mapping)); + assertAcked(transportClient().admin().indices().prepareCreate(queryIndex) + .setSettings(settings).addMapping("type", mapping)); + ensureGreen(queryIndex, lookupIndex); + + requests.clear(); + } + + @After + public void checkAllRequestsContainHeaders() { + assertRequestsContainHeader(IndexRequest.class); + assertRequestsContainHeader(RefreshRequest.class); + + /* + for (ActionRequest request : requests) { + String msg = String.format(Locale.ROOT, "Expected request [%s] to have randomized header key set", request.getClass().getSimpleName()); + assertThat(msg, request.hasHeader(randomHeaderKey), is(true)); + assertThat(request.getHeader(randomHeaderKey).toString(), is(randomHeaderValue)); + } + */ + } + + // TODO check context as well + + @Test + public void testThatTermsLookupGetRequestContainsContextAndHeaders() throws Exception { + transportClient().prepareIndex(lookupIndex, "type", "1") + .setSource(jsonBuilder().startObject().array("followers", "foo", "bar", "baz").endObject()).get(); + transportClient().prepareIndex(queryIndex, "type", "1") + .setSource(jsonBuilder().startObject().field("username", "foo").endObject()).get(); + transportClient().admin().indices().prepareRefresh(queryIndex, lookupIndex).get(); + + TermsLookupQueryBuilder termsLookupFilterBuilder = QueryBuilders.termsLookupQuery("username").lookupIndex(lookupIndex).lookupType("type").lookupId("1").lookupPath("followers"); + BoolQueryBuilder queryBuilder = QueryBuilders.boolQuery().must(QueryBuilders.matchAllQuery()).must(termsLookupFilterBuilder); + + SearchResponse searchResponse = transportClient() + .prepareSearch(queryIndex) + .setQuery(queryBuilder) + .get(); + assertNoFailures(searchResponse); + assertHitCount(searchResponse, 1); + + assertGetRequestsContainHeaders(); + } + + @Test + public void testThatGeoShapeQueryGetRequestContainsContextAndHeaders() throws Exception { + indexRandom(false, false, + transportClient().prepareIndex(lookupIndex, "type", "1").setSource(jsonBuilder().startObject() + .field("name", "Munich Suburban Area") + .startObject("location") + .field("type", "polygon") + .startArray("coordinates").startArray() + .startArray().value(11.34).value(48.25).endArray() + .startArray().value(11.68).value(48.25).endArray() + .startArray().value(11.65).value(48.06).endArray() + .startArray().value(11.37).value(48.13).endArray() + .startArray().value(11.34).value(48.25).endArray() // close the polygon + .endArray().endArray() + .endObject() + .endObject()), + // second document + transportClient().prepareIndex(queryIndex, "type", "1").setSource(jsonBuilder().startObject() + .field("name", "Munich Center") + .startObject("location") + .field("type", "point") + .startArray("coordinates").value(11.57).value(48.13).endArray() + .endObject() + .endObject()) + ); + transportClient().admin().indices().prepareRefresh(lookupIndex, queryIndex).get(); + + GeoShapeQueryBuilder queryBuilder = QueryBuilders.geoShapeQuery("location", "1", "type") + .indexedShapeIndex(lookupIndex) + .indexedShapePath("location"); + + SearchResponse searchResponse = transportClient() + .prepareSearch(queryIndex) + .setQuery(queryBuilder) + .get(); + assertNoFailures(searchResponse); + assertHitCount(searchResponse, 1); + assertThat(requests, hasSize(greaterThan(0))); + + assertGetRequestsContainHeaders(); + } + + @Test + public void testThatMoreLikeThisQueryMultiTermVectorRequestContainsContextAndHeaders() throws Exception { + indexRandom(false, false, + transportClient().prepareIndex(lookupIndex, "type", "1") + .setSource(jsonBuilder().startObject().field("name", "Star Wars - The new republic").endObject()), + transportClient().prepareIndex(queryIndex, "type", "1") + .setSource(jsonBuilder().startObject().field("name", "Jar Jar Binks - A horrible mistake").endObject()), + transportClient().prepareIndex(queryIndex, "type", "2") + .setSource(jsonBuilder().startObject().field("name", "Star Wars - Return of the jedi").endObject())); + transportClient().admin().indices().prepareRefresh(lookupIndex, queryIndex).get(); + + MoreLikeThisQueryBuilder moreLikeThisQueryBuilder = QueryBuilders.moreLikeThisQuery("name") + .addItem(new MoreLikeThisQueryBuilder.Item(lookupIndex, "type", "1")) + .minTermFreq(1) + .minDocFreq(1); + + SearchResponse searchResponse = transportClient() + .prepareSearch(queryIndex) + .setQuery(moreLikeThisQueryBuilder) + .get(); + assertNoFailures(searchResponse); + assertHitCount(searchResponse, 1); + + assertRequestsContainHeader(MultiTermVectorsRequest.class); + } + + @Test + public void testThatPercolatingExistingDocumentGetRequestContainsContextAndHeaders() throws Exception { + indexRandom(false, + transportClient().prepareIndex(lookupIndex, ".percolator", "1") + .setSource(jsonBuilder().startObject().startObject("query").startObject("match").field("name", "star wars").endObject().endObject().endObject()), + transportClient().prepareIndex(lookupIndex, "type", "1") + .setSource(jsonBuilder().startObject().field("name", "Star Wars - The new republic").endObject()) + ); + transportClient().admin().indices().prepareRefresh(lookupIndex).get(); + + GetRequest getRequest = transportClient().prepareGet(lookupIndex, "type", "1").request(); + PercolateResponse response = transportClient().preparePercolate().setDocumentType("type").setGetRequest(getRequest).get(); + assertThat(response.getCount(), is(1l)); + + assertGetRequestsContainHeaders(); + } + + @Test + public void testThatIndexedScriptGetRequestContainsContextAndHeaders() throws Exception { + PutIndexedScriptResponse scriptResponse = transportClient().preparePutIndexedScript(GroovyScriptEngineService.NAME, "my_script", + jsonBuilder().startObject().field("script", "_score * 10").endObject().string() + ).get(); + assertThat(scriptResponse.isCreated(), is(true)); + + indexRandom(false, false, transportClient().prepareIndex(queryIndex, "type", "1") + .setSource(jsonBuilder().startObject().field("name", "Star Wars - The new republic").endObject())); + transportClient().admin().indices().prepareRefresh(queryIndex).get(); + + // custom content, not sure how to specify "script_id" otherwise in the API + XContentBuilder builder = jsonBuilder().startObject().startObject("function_score").field("boost_mode", "replace").startArray("functions") + .startObject().startObject("script_score").field("script_id", "my_script").field("lang", "groovy").endObject().endObject().endArray().endObject().endObject(); + + SearchResponse searchResponse = transportClient() + .prepareSearch(queryIndex) + .setQuery(builder) + .get(); + assertNoFailures(searchResponse); + assertHitCount(searchResponse, 1); + assertThat(searchResponse.getHits().getMaxScore(), is(10.0f)); + + assertGetRequestsContainHeaders(".scripts"); + assertRequestsContainHeader(PutIndexedScriptRequest.class); + } + + @Test + public void testThatSearchTemplatesWithIndexedTemplatesGetRequestContainsContextAndHeaders() throws Exception { + PutIndexedScriptResponse scriptResponse = transportClient().preparePutIndexedScript(MustacheScriptEngineService.NAME, "the_template", + jsonBuilder().startObject().startObject("template").startObject("query").startObject("match") + .field("name", "{{query_string}}").endObject().endObject().endObject().endObject().string() + ).get(); + assertThat(scriptResponse.isCreated(), is(true)); + + indexRandom(false, false, transportClient().prepareIndex(queryIndex, "type", "1") + .setSource(jsonBuilder().startObject().field("name", "Star Wars - The new republic").endObject())); + transportClient().admin().indices().prepareRefresh(queryIndex).get(); + + Map params = new HashMap<>(); + params.put("query_string", "star wars"); + + SearchResponse searchResponse = transportClient().prepareSearch(queryIndex) + .setTemplateName("the_template") + .setTemplateParams(params) + .setTemplateType(ScriptService.ScriptType.INDEXED) + .get(); + + assertNoFailures(searchResponse); + assertHitCount(searchResponse, 1); + + assertGetRequestsContainHeaders(".scripts"); + assertRequestsContainHeader(PutIndexedScriptRequest.class); + } + + @Test + public void testThatRelevantHttpHeadersBecomeRequestHeaders() throws Exception { + String releventHeaderName = "relevant_" + randomHeaderKey; + for (RestController restController : internalCluster().getDataNodeInstances(RestController.class)) { + restController.registerRelevantHeaders(releventHeaderName); + } + + CloseableHttpClient httpClient = HttpClients.createDefault(); + HttpResponse response = new HttpRequestBuilder(httpClient) + .httpTransport(internalCluster().getDataNodeInstance(HttpServerTransport.class)) + .addHeader(randomHeaderKey, randomHeaderValue) + .addHeader(releventHeaderName, randomHeaderValue) + .path("/" + queryIndex + "/_search") + .execute(); + + assertThat(response, hasStatus(OK)); + List searchRequests = getRequests(SearchRequest.class); + assertThat(searchRequests, hasSize(greaterThan(0))); + for (SearchRequest searchRequest : searchRequests) { + assertThat(searchRequest.hasHeader(releventHeaderName), is(true)); + // was not specified, thus is not included + assertThat(searchRequest.hasHeader(randomHeaderKey), is(false)); + } + } + + private List getRequests(Class clazz) { + List results = new ArrayList<>(); + for (ActionRequest request : requests) { + if (request.getClass().equals(clazz)) { + results.add((T) request); + } + } + + return results; + } + + private void assertRequestsContainHeader(Class clazz) { + List classRequests = getRequests(clazz); + for (ActionRequest request : classRequests) { + assertRequestContainsHeader(request); + } + } + + private void assertGetRequestsContainHeaders() { + assertGetRequestsContainHeaders(this.lookupIndex); + } + + private void assertGetRequestsContainHeaders(String index) { + List getRequests = getRequests(GetRequest.class); + assertThat(getRequests, hasSize(greaterThan(0))); + + for (GetRequest request : getRequests) { + if (!request.index().equals(index)) { + continue; + } + assertRequestContainsHeader(request); + } + } + + private void assertRequestContainsHeader(ActionRequest request) { + String msg = String.format(Locale.ROOT, "Expected header %s to be in request %s", randomHeaderKey, request.getClass().getName()); + if (request instanceof IndexRequest) { + IndexRequest indexRequest = (IndexRequest) request; + msg = String.format(Locale.ROOT, "Expected header %s to be in index request %s/%s/%s", randomHeaderKey, + indexRequest.index(), indexRequest.type(), indexRequest.id()); + } + assertThat(msg, request.hasHeader(randomHeaderKey), is(true)); + assertThat(request.getHeader(randomHeaderKey).toString(), is(randomHeaderValue)); + } + + /** + * a transport client that adds our random header + */ + private Client transportClient() { + Client transportClient = internalCluster().transportClient(); + FilterClient filterClient = new FilterClient(transportClient) { + @Override + protected > void doExecute(Action action, Request request, ActionListener listener) { + request.putHeader(randomHeaderKey, randomHeaderValue); + super.doExecute(action, request, listener); + } + }; + + return filterClient; + } + + public static class ActionLoggingPlugin extends AbstractPlugin { + + @Override + public String name() { + return "test-action-logging"; + } + + @Override + public String description() { + return "Test action logging"; + } + + @Override + public Collection> modules() { + return ImmutableList.of(ActionLoggingModule.class); + } + } + + public static class ActionLoggingModule extends AbstractModule implements PreProcessModule { + + + @Override + protected void configure() { + bind(LoggingFilter.class).asEagerSingleton(); + } + + @Override + public void processModule(Module module) { + if (module instanceof ActionModule) { + ((ActionModule)module).registerFilter(LoggingFilter.class); + } + } + } + + public static class LoggingFilter extends ActionFilter.Simple { + + @Inject + public LoggingFilter(Settings settings) { + super(settings); + } + + @Override + public int order() { + return 999; + } + + @Override + protected boolean apply(String action, ActionRequest request, ActionListener listener) { + requests.add(request); + return true; + } + + @Override + protected boolean apply(String action, ActionResponse response, ActionListener listener) { + return true; + } + } +} From 2c241e8a3622bb874d1726d6004bd4c2099be441 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Mon, 18 May 2015 14:47:22 +0200 Subject: [PATCH 38/56] Mappings: Remove the `ignore_conflicts` option. Mappings conflicts should not be ignored. If I read the history correctly, this option was added when a mapping update to an existing field was considered a conflict, even if the new mapping was exactly the same. Now that mapping updates are smart enough to detect conflicting options, we don't need an option to ignore conflicts. --- docs/reference/indices/put-mapping.asciidoc | 4 +- docs/reference/migration/migrate_2_0.asciidoc | 1 + rest-api-spec/api/indices.put_mapping.json | 4 - .../PutMappingClusterStateUpdateRequest.java | 11 --- .../mapping/put/PutMappingRequest.java | 26 +----- .../mapping/put/PutMappingRequestBuilder.java | 9 -- .../put/TransportPutMappingAction.java | 2 +- .../metadata/MetaDataMappingService.java | 4 +- .../mapping/put/RestPutMappingAction.java | 1 - .../MultiFieldsIntegrationTests.java | 88 ++++++------------- .../UpdateMappingIntegrationTests.java | 24 ----- 11 files changed, 32 insertions(+), 142 deletions(-) diff --git a/docs/reference/indices/put-mapping.asciidoc b/docs/reference/indices/put-mapping.asciidoc index 4c2f4406b8a..6de2fa22342 100644 --- a/docs/reference/indices/put-mapping.asciidoc +++ b/docs/reference/indices/put-mapping.asciidoc @@ -31,9 +31,7 @@ More information on how to define type mappings can be found in the When an existing mapping already exists under the given type, the two mapping definitions, the one already defined, and the new ones are -merged. The `ignore_conflicts` parameters can be used to control if -conflicts should be ignored or not, by default, it is set to `false` -which means conflicts are *not* ignored. +merged. If there are conflicts then the update will be rejected. The definition of conflict is really dependent on the type merged, but in general, if a different core type is defined, it is considered as a diff --git a/docs/reference/migration/migrate_2_0.asciidoc b/docs/reference/migration/migrate_2_0.asciidoc index 53cacf7be16..4c4678ef885 100644 --- a/docs/reference/migration/migrate_2_0.asciidoc +++ b/docs/reference/migration/migrate_2_0.asciidoc @@ -177,6 +177,7 @@ A `RoutingMissingException` is now thrown instead. * The setting `index.mapping.allow_type_wrapper` has been removed. Documents should always be sent without the type as the root element. * The delete mappings API has been removed. Mapping types can no longer be deleted. +* The `ignore_conflicts` option of the put mappings API has been removed. Conflicts can't be ignored anymore. ==== Removed type prefix on field names in queries Types can no longer be specified on fields within queries. Instead, specify type restrictions in the search request. diff --git a/rest-api-spec/api/indices.put_mapping.json b/rest-api-spec/api/indices.put_mapping.json index e86d2905fd1..c6b547914ef 100644 --- a/rest-api-spec/api/indices.put_mapping.json +++ b/rest-api-spec/api/indices.put_mapping.json @@ -17,10 +17,6 @@ } }, "params": { - "ignore_conflicts": { - "type" : "boolean", - "description" : "Specify whether to ignore conflicts while updating the mapping (default: false)" - }, "timeout": { "type" : "time", "description" : "Explicit operation timeout" diff --git a/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingClusterStateUpdateRequest.java b/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingClusterStateUpdateRequest.java index f0299a145c5..25aafdef23e 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingClusterStateUpdateRequest.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingClusterStateUpdateRequest.java @@ -30,8 +30,6 @@ public class PutMappingClusterStateUpdateRequest extends IndicesClusterStateUpda private String source; - private boolean ignoreConflicts = false; - PutMappingClusterStateUpdateRequest() { } @@ -53,13 +51,4 @@ public class PutMappingClusterStateUpdateRequest extends IndicesClusterStateUpda this.source = source; return this; } - - public boolean ignoreConflicts() { - return ignoreConflicts; - } - - public PutMappingClusterStateUpdateRequest ignoreConflicts(boolean ignoreConflicts) { - this.ignoreConflicts = ignoreConflicts; - return this; - } } diff --git a/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequest.java b/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequest.java index 3c751c67675..3563fd1449a 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequest.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequest.java @@ -42,8 +42,7 @@ import static org.elasticsearch.action.ValidateActions.addValidationError; * {@link org.elasticsearch.client.Requests#putMappingRequest(String...)}. *

*

If the mappings already exists, the new mappings will be merged with the new one. If there are elements - * that can't be merged are detected, the request will be rejected unless the {@link #ignoreConflicts(boolean)} - * is set. In such a case, the duplicate mappings will be rejected. + * that can't be merged are detected, the request will be rejected. * * @see org.elasticsearch.client.Requests#putMappingRequest(String...) * @see org.elasticsearch.client.IndicesAdminClient#putMapping(PutMappingRequest) @@ -64,8 +63,6 @@ public class PutMappingRequest extends AcknowledgedRequest im private String source; - private boolean ignoreConflicts = false; - PutMappingRequest() { } @@ -239,25 +236,6 @@ public class PutMappingRequest extends AcknowledgedRequest im return this; } - /** - * If there is already a mapping definition registered against the type, then it will be merged. If there are - * elements that can't be merged are detected, the request will be rejected unless the - * {@link #ignoreConflicts(boolean)} is set. In such a case, the duplicate mappings will be rejected. - */ - public boolean ignoreConflicts() { - return ignoreConflicts; - } - - /** - * If there is already a mapping definition registered against the type, then it will be merged. If there are - * elements that can't be merged are detected, the request will be rejected unless the - * {@link #ignoreConflicts(boolean)} is set. In such a case, the duplicate mappings will be rejected. - */ - public PutMappingRequest ignoreConflicts(boolean ignoreDuplicates) { - this.ignoreConflicts = ignoreDuplicates; - return this; - } - @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); @@ -266,7 +244,6 @@ public class PutMappingRequest extends AcknowledgedRequest im type = in.readOptionalString(); source = in.readString(); readTimeout(in); - ignoreConflicts = in.readBoolean(); } @Override @@ -277,6 +254,5 @@ public class PutMappingRequest extends AcknowledgedRequest im out.writeOptionalString(type); out.writeString(source); writeTimeout(out); - out.writeBoolean(ignoreConflicts); } } diff --git a/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequestBuilder.java b/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequestBuilder.java index e87edef0b8f..acf2c33c9dc 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequestBuilder.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequestBuilder.java @@ -91,13 +91,4 @@ public class PutMappingRequestBuilder extends AcknowledgedRequestBuilder() { diff --git a/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java b/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java index 5bacae63d0c..0e39e7a613d 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java @@ -382,8 +382,8 @@ public class MetaDataMappingService extends AbstractComponent { if (existingMapper != null) { // first, simulate MergeResult mergeResult = existingMapper.merge(newMapper.mapping(), true); - // if we have conflicts, and we are not supposed to ignore them, throw an exception - if (!request.ignoreConflicts() && mergeResult.hasConflicts()) { + // if we have conflicts, throw an exception + if (mergeResult.hasConflicts()) { throw new MergeMappingException(mergeResult.buildConflicts()); } } diff --git a/src/main/java/org/elasticsearch/rest/action/admin/indices/mapping/put/RestPutMappingAction.java b/src/main/java/org/elasticsearch/rest/action/admin/indices/mapping/put/RestPutMappingAction.java index 6df8edc0c82..0295009c55f 100644 --- a/src/main/java/org/elasticsearch/rest/action/admin/indices/mapping/put/RestPutMappingAction.java +++ b/src/main/java/org/elasticsearch/rest/action/admin/indices/mapping/put/RestPutMappingAction.java @@ -70,7 +70,6 @@ public class RestPutMappingAction extends BaseRestHandler { putMappingRequest.type(request.param("type")); putMappingRequest.source(request.content().toUtf8()); putMappingRequest.timeout(request.paramAsTime("timeout", putMappingRequest.timeout())); - putMappingRequest.ignoreConflicts(request.paramAsBoolean("ignore_conflicts", putMappingRequest.ignoreConflicts())); putMappingRequest.masterNodeTimeout(request.paramAsTime("master_timeout", putMappingRequest.masterNodeTimeout())); putMappingRequest.indicesOptions(IndicesOptions.fromRequest(request, putMappingRequest.indicesOptions())); client.admin().indices().putMapping(putMappingRequest, new AcknowledgedRestListener(channel)); diff --git a/src/test/java/org/elasticsearch/index/mapper/multifield/MultiFieldsIntegrationTests.java b/src/test/java/org/elasticsearch/index/mapper/multifield/MultiFieldsIntegrationTests.java index 6b2bd463ca0..3e5c29daade 100644 --- a/src/test/java/org/elasticsearch/index/mapper/multifield/MultiFieldsIntegrationTests.java +++ b/src/test/java/org/elasticsearch/index/mapper/multifield/MultiFieldsIntegrationTests.java @@ -76,7 +76,6 @@ public class MultiFieldsIntegrationTests extends ElasticsearchIntegrationTest { assertAcked( client().admin().indices().preparePutMapping("my-index").setType("my-type") .setSource(createPutMappingSource()) - .setIgnoreConflicts(true) // If updated with multi-field type, we need to ignore failures. ); getMappingsResponse = client().admin().indices().prepareGetMappings("my-index").get(); @@ -240,70 +239,35 @@ public class MultiFieldsIntegrationTests extends ElasticsearchIntegrationTest { } private XContentBuilder createTypeSource() throws IOException { - if (randomBoolean()) { - return XContentFactory.jsonBuilder().startObject().startObject("my-type") - .startObject("properties") - .startObject("title") - .field("type", "string") - .startObject("fields") - .startObject("not_analyzed") - .field("type", "string") - .field("index", "not_analyzed") - .endObject() - .endObject() - .endObject() - .endObject() - .endObject().endObject(); - } else { - return XContentFactory.jsonBuilder().startObject().startObject("my-type") - .startObject("properties") - .startObject("title") - .field("type", "multi_field") - .startObject("fields") - .startObject("title") - .field("type", "string") - .endObject() - .startObject("not_analyzed") - .field("type", "string") - .field("index", "not_analyzed") - .endObject() - .endObject() - .endObject() - .endObject() - .endObject().endObject(); - } + return XContentFactory.jsonBuilder().startObject().startObject("my-type") + .startObject("properties") + .startObject("title") + .field("type", "string") + .startObject("fields") + .startObject("not_analyzed") + .field("type", "string") + .field("index", "not_analyzed") + .endObject() + .endObject() + .endObject() + .endObject() + .endObject().endObject(); } private XContentBuilder createPutMappingSource() throws IOException { - if (randomBoolean()) { - return XContentFactory.jsonBuilder().startObject().startObject("my-type") - .startObject("properties") - .startObject("title") - .field("type", "string") - .startObject("fields") - .startObject("uncased") - .field("type", "string") - .field("analyzer", "whitespace") - .endObject() - .endObject() - .endObject() - .endObject() - .endObject().endObject(); - } else { - return XContentFactory.jsonBuilder().startObject().startObject("my-type") - .startObject("properties") - .startObject("title") - .field("type", "multi_field") - .startObject("fields") - .startObject("uncased") - .field("type", "string") - .field("analyzer", "whitespace") - .endObject() - .endObject() - .endObject() - .endObject() - .endObject().endObject(); - } + return XContentFactory.jsonBuilder().startObject().startObject("my-type") + .startObject("properties") + .startObject("title") + .field("type", "string") + .startObject("fields") + .startObject("uncased") + .field("type", "string") + .field("analyzer", "whitespace") + .endObject() + .endObject() + .endObject() + .endObject() + .endObject().endObject(); } } diff --git a/src/test/java/org/elasticsearch/indices/mapping/UpdateMappingIntegrationTests.java b/src/test/java/org/elasticsearch/indices/mapping/UpdateMappingIntegrationTests.java index dca5183a471..b6cae9322c6 100644 --- a/src/test/java/org/elasticsearch/indices/mapping/UpdateMappingIntegrationTests.java +++ b/src/test/java/org/elasticsearch/indices/mapping/UpdateMappingIntegrationTests.java @@ -165,30 +165,6 @@ public class UpdateMappingIntegrationTests extends ElasticsearchIntegrationTest .execute().actionGet(); } - /* - First regression test for https://github.com/elasticsearch/elasticsearch/issues/3381 - */ - @Test - public void updateMappingWithIgnoredConflicts() throws Exception { - - client().admin().indices().prepareCreate("test") - .setSettings( - settingsBuilder() - .put("index.number_of_shards", 2) - .put("index.number_of_replicas", 0) - ).addMapping("type", "{\"type\":{\"properties\":{\"body\":{\"type\":\"string\"}}}}") - .execute().actionGet(); - client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus().execute().actionGet(); - - PutMappingResponse putMappingResponse = client().admin().indices().preparePutMapping("test").setType("type") - .setSource("{\"type\":{\"properties\":{\"body\":{\"type\":\"integer\"}}}}") - .setIgnoreConflicts(true) - .execute().actionGet(); - - //no changes since the only one had a conflict and was ignored, we return - assertThat(putMappingResponse.isAcknowledged(), equalTo(true)); - } - /* Second regression test for https://github.com/elasticsearch/elasticsearch/issues/3381 */ From 33adbe6082280195b5ae4f3515adfb491b839d4e Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Mon, 18 May 2015 16:38:13 +0200 Subject: [PATCH 39/56] Recovery: No need to send mappings to the master node on phase 2. Now that mapping updates are synchronous, it is not necessary to send mappings to the master node during the recovery process anymore: they will already be on the master node since we ensure mappings are on the master node before indexing. --- .../indices/recovery/RecoverySource.java | 16 +- .../recovery/RecoverySourceHandler.java | 158 +----------------- .../SharedFSRecoverySourceHandler.java | 11 +- 3 files changed, 16 insertions(+), 169 deletions(-) diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java b/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java index 73c57e5ef7e..80b319d5b7a 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java @@ -21,7 +21,6 @@ package org.elasticsearch.indices.recovery; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.cluster.ClusterService; -import org.elasticsearch.cluster.action.index.MappingUpdatedAction; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.RoutingNode; import org.elasticsearch.cluster.routing.ShardRouting; @@ -40,7 +39,12 @@ import org.elasticsearch.transport.TransportChannel; import org.elasticsearch.transport.TransportRequestHandler; import org.elasticsearch.transport.TransportService; -import java.util.*; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; /** * The source recovery accepts recovery requests from other peer shards and start the recovery process from this @@ -55,7 +59,6 @@ public class RecoverySource extends AbstractComponent { private final TransportService transportService; private final IndicesService indicesService; private final RecoverySettings recoverySettings; - private final MappingUpdatedAction mappingUpdatedAction; private final ClusterService clusterService; @@ -64,11 +67,10 @@ public class RecoverySource extends AbstractComponent { @Inject public RecoverySource(Settings settings, TransportService transportService, IndicesService indicesService, - RecoverySettings recoverySettings, MappingUpdatedAction mappingUpdatedAction, ClusterService clusterService) { + RecoverySettings recoverySettings, ClusterService clusterService) { super(settings); this.transportService = transportService; this.indicesService = indicesService; - this.mappingUpdatedAction = mappingUpdatedAction; this.clusterService = clusterService; this.indicesService.indicesLifecycle().addListener(new IndicesLifecycle.Listener() { @Override @@ -116,9 +118,9 @@ public class RecoverySource extends AbstractComponent { logger.trace("[{}][{}] starting recovery to {}, mark_as_relocated {}", request.shardId().index().name(), request.shardId().id(), request.targetNode(), request.markAsRelocated()); final RecoverySourceHandler handler; if (IndexMetaData.isOnSharedFilesystem(shard.indexSettings())) { - handler = new SharedFSRecoverySourceHandler(shard, request, recoverySettings, transportService, clusterService, indicesService, mappingUpdatedAction, logger); + handler = new SharedFSRecoverySourceHandler(shard, request, recoverySettings, transportService, logger); } else { - handler = new RecoverySourceHandler(shard, request, recoverySettings, transportService, clusterService, indicesService, mappingUpdatedAction, logger); + handler = new RecoverySourceHandler(shard, request, recoverySettings, transportService, logger); } ongoingRecoveries.add(shard, handler); try { diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index d9ebe4798ee..823742404e3 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -21,6 +21,7 @@ package org.elasticsearch.indices.recovery; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; + import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.IndexFormatTooNewException; import org.apache.lucene.index.IndexFormatTooOldException; @@ -30,31 +31,19 @@ import org.apache.lucene.store.RateLimiter; import org.apache.lucene.util.ArrayUtil; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.cluster.ClusterService; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.TimeoutClusterStateUpdateTask; -import org.elasticsearch.cluster.action.index.MappingUpdatedAction; -import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.Priority; import org.elasticsearch.common.StopWatch; import org.elasticsearch.common.bytes.BytesArray; -import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.compress.CompressorFactory; import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.CancellableThreads; import org.elasticsearch.common.util.CancellableThreads.Interruptable; import org.elasticsearch.common.util.concurrent.AbstractRunnable; -import org.elasticsearch.common.util.concurrent.ConcurrentCollections; -import org.elasticsearch.index.IndexService; import org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.RecoveryEngineException; -import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.shard.IllegalIndexShardStateException; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardClosedException; @@ -62,7 +51,6 @@ import org.elasticsearch.index.shard.IndexShardState; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetaData; import org.elasticsearch.index.translog.Translog; -import org.elasticsearch.indices.IndicesService; import org.elasticsearch.transport.EmptyTransportResponseHandler; import org.elasticsearch.transport.RemoteTransportException; import org.elasticsearch.transport.TransportRequestOptions; @@ -71,7 +59,9 @@ import org.elasticsearch.transport.TransportService; import java.io.IOException; import java.util.Comparator; import java.util.List; -import java.util.concurrent.*; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -91,9 +81,6 @@ public class RecoverySourceHandler { private final StartRecoveryRequest request; private final RecoverySettings recoverySettings; private final TransportService transportService; - private final ClusterService clusterService; - private final IndexService indexService; - private final MappingUpdatedAction mappingUpdatedAction; protected final RecoveryResponse response; private final CancellableThreads cancellableThreads = new CancellableThreads() { @@ -114,18 +101,14 @@ public class RecoverySourceHandler { public RecoverySourceHandler(final IndexShard shard, final StartRecoveryRequest request, final RecoverySettings recoverySettings, - final TransportService transportService, final ClusterService clusterService, - final IndicesService indicesService, final MappingUpdatedAction mappingUpdatedAction, final ESLogger logger) { + final TransportService transportService, final ESLogger logger) { this.shard = shard; this.request = request; this.recoverySettings = recoverySettings; this.logger = logger; this.transportService = transportService; - this.clusterService = clusterService; this.indexName = this.request.shardId().index().name(); this.shardId = this.request.shardId().id(); - this.indexService = indicesService.indexServiceSafe(indexName); - this.mappingUpdatedAction = mappingUpdatedAction; this.response = new RecoveryResponse(); } @@ -490,9 +473,6 @@ public class RecoverySourceHandler { cancellableThreads.checkForCancel(); StopWatch stopWatch = new StopWatch().start(); - logger.trace("{} recovery [phase2] to {}: updating current mapping to master", request.shardId(), request.targetNode()); - // Ensure that the mappings are synced with the master node - updateMappingOnMaster(); logger.trace("{} recovery [phase2] to {}: sending transaction log operations", request.shardId(), request.targetNode()); // Send all the snapshot's translog operations to the target @@ -546,67 +526,6 @@ public class RecoverySourceHandler { indexName, shardId, request.targetNode(), stopWatch.totalTime()); } - /** - * Ensures that the mapping in the cluster state is the same as the mapping - * in our mapper service. If the mapping is not in sync, sends a request - * to update it in the cluster state and blocks until it has finished - * being updated. - */ - private void updateMappingOnMaster() { - // we test that the cluster state is in sync with our in memory mapping stored by the mapperService - // we have to do it under the "cluster state update" thread to make sure that one doesn't modify it - // while we're checking - final BlockingQueue documentMappersToUpdate = ConcurrentCollections.newBlockingQueue(); - final CountDownLatch latch = new CountDownLatch(1); - final AtomicReference mappingCheckException = new AtomicReference<>(); - - // we use immediate as this is a very light weight check and we don't wait to delay recovery - clusterService.submitStateUpdateTask("recovery_mapping_check", Priority.IMMEDIATE, new MappingUpdateTask(clusterService, indexService, recoverySettings, latch, documentMappersToUpdate, mappingCheckException, this.cancellableThreads)); - cancellableThreads.execute(new Interruptable() { - @Override - public void run() throws InterruptedException { - latch.await(); - } - }); - if (mappingCheckException.get() != null) { - logger.warn("error during mapping check, failing recovery", mappingCheckException.get()); - throw new ElasticsearchException("error during mapping check", mappingCheckException.get()); - } - if (documentMappersToUpdate.isEmpty()) { - return; - } - final CountDownLatch updatedOnMaster = new CountDownLatch(documentMappersToUpdate.size()); - MappingUpdatedAction.MappingUpdateListener listener = new MappingUpdatedAction.MappingUpdateListener() { - @Override - public void onMappingUpdate() { - updatedOnMaster.countDown(); - } - - @Override - public void onFailure(Throwable t) { - logger.debug("{} recovery to {}: failed to update mapping on master", request.shardId(), request.targetNode(), t); - updatedOnMaster.countDown(); - } - }; - for (DocumentMapper documentMapper : documentMappersToUpdate) { - mappingUpdatedAction.updateMappingOnMaster(indexService.index().getName(), documentMapper.type(), documentMapper.mapping(), recoverySettings.internalActionTimeout(), listener); - } - cancellableThreads.execute(new Interruptable() { - @Override - public void run() throws InterruptedException { - try { - if (!updatedOnMaster.await(recoverySettings.internalActionTimeout().millis(), TimeUnit.MILLISECONDS)) { - logger.debug("[{}][{}] recovery [phase2] to {}: waiting on pending mapping update timed out. waited [{}]", - indexName, shardId, request.targetNode(), recoverySettings.internalActionTimeout()); - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - logger.debug("interrupted while waiting for mapping to update on master"); - } - } - }); - } - /** * Send the given snapshot's operations to this handler's target node. *

@@ -723,71 +642,4 @@ public class RecoverySourceHandler { '}'; } - // this is a static class since we are holding an instance to the IndexShard - // on ShardRecoveryHandler which can not be GCed if the recovery is canceled - // but this task is still stuck in the queue. This can be problematic if the - // queue piles up and recoveries fail and can lead to OOM or memory pressure if lots of shards - // are created and removed. - private static class MappingUpdateTask extends TimeoutClusterStateUpdateTask { - private final CountDownLatch latch; - private final BlockingQueue documentMappersToUpdate; - private final AtomicReference mappingCheckException; - private final CancellableThreads cancellableThreads; - private ClusterService clusterService; - private IndexService indexService; - private RecoverySettings recoverySettings; - - public MappingUpdateTask(ClusterService clusterService, IndexService indexService, RecoverySettings recoverySettings, CountDownLatch latch, BlockingQueue documentMappersToUpdate, AtomicReference mappingCheckException, CancellableThreads cancellableThreads) { - this.latch = latch; - this.documentMappersToUpdate = documentMappersToUpdate; - this.mappingCheckException = mappingCheckException; - this.clusterService = clusterService; - this.indexService = indexService; - this.recoverySettings = recoverySettings; - this.cancellableThreads = cancellableThreads; - } - - @Override - public boolean runOnlyOnMaster() { - return false; - } - - @Override - public TimeValue timeout() { - return recoverySettings.internalActionTimeout(); - } - - @Override - public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { - latch.countDown(); - } - - @Override - public ClusterState execute(ClusterState currentState) throws Exception { - if (cancellableThreads.isCancelled() == false) { // no need to run this if recovery is canceled - IndexMetaData indexMetaData = clusterService.state().metaData().getIndices().get(indexService.index().getName()); - ImmutableOpenMap metaDataMappings = null; - if (indexMetaData != null) { - metaDataMappings = indexMetaData.getMappings(); - } - // default mapping should not be sent back, it can only be updated by put mapping API, and its - // a full in place replace, we don't want to override a potential update coming into it - for (DocumentMapper documentMapper : indexService.mapperService().docMappers(false)) { - - MappingMetaData mappingMetaData = metaDataMappings == null ? null : metaDataMappings.get(documentMapper.type()); - if (mappingMetaData == null || !documentMapper.refreshSource().equals(mappingMetaData.source())) { - // not on master yet in the right form - documentMappersToUpdate.add(documentMapper); - } - } - } - return currentState; - } - - @Override - public void onFailure(String source, Throwable t) { - mappingCheckException.set(t); - latch.countDown(); - } - } } diff --git a/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java b/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java index 187215ebfa9..9e80accf5e9 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java +++ b/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java @@ -19,16 +19,9 @@ package org.elasticsearch.indices.recovery; -import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.cluster.ClusterService; -import org.elasticsearch.cluster.action.index.MappingUpdatedAction; -import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.logging.ESLogger; -import org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit; -import org.elasticsearch.index.engine.RecoveryEngineException; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.translog.Translog; -import org.elasticsearch.indices.IndicesService; import org.elasticsearch.transport.TransportService; import java.io.IOException; @@ -43,8 +36,8 @@ public class SharedFSRecoverySourceHandler extends RecoverySourceHandler { private final StartRecoveryRequest request; private static final Translog.View EMPTY_VIEW = new EmptyView(); - public SharedFSRecoverySourceHandler(IndexShard shard, StartRecoveryRequest request, RecoverySettings recoverySettings, TransportService transportService, ClusterService clusterService, IndicesService indicesService, MappingUpdatedAction mappingUpdatedAction, ESLogger logger) { - super(shard, request, recoverySettings, transportService, clusterService, indicesService, mappingUpdatedAction, logger); + public SharedFSRecoverySourceHandler(IndexShard shard, StartRecoveryRequest request, RecoverySettings recoverySettings, TransportService transportService, ESLogger logger) { + super(shard, request, recoverySettings, transportService, logger); this.shard = shard; this.request = request; } From 7d1eb6e900e34a1a0567d3722edc3c7b855a697d Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 18 May 2015 16:47:24 +0200 Subject: [PATCH 40/56] [ENGINE] Close Translog if initial commit fails --- .../org/elasticsearch/index/engine/InternalEngine.java | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index dee7eb2a0a6..dbdb13f03d7 100644 --- a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -194,7 +194,15 @@ public class InternalEngine extends Engine { final Translog translog = new Translog(translogConfig); if (generation == null) { logger.debug("no translog ID present in the current generation - creating one"); - commitIndexWriter(writer, translog); + boolean success = false; + try { + commitIndexWriter(writer, translog); + success = true; + } finally { + if (success == false) { + IOUtils.closeWhileHandlingException(translog); + } + } } return translog; } From bba1528fa4664fb2d568d1c2a20580518571ba98 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 18 May 2015 16:58:19 +0200 Subject: [PATCH 41/56] [TEST] Enable MockFS on InternalEngineTests and TranslogTests --- .../elasticsearch/index/engine/InternalEngineTests.java | 8 +------- .../org/elasticsearch/index/translog/TranslogTests.java | 5 +---- 2 files changed, 2 insertions(+), 11 deletions(-) diff --git a/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 46a2129bc48..04334b68a53 100644 --- a/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -38,7 +38,6 @@ import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.Directory; import org.apache.lucene.store.MockDirectoryWrapper; import org.apache.lucene.util.IOUtils; -import org.apache.lucene.util.LuceneTestCase.SuppressFileSystems; import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.Nullable; @@ -82,7 +81,6 @@ import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.TranslogConfig; import org.elasticsearch.test.DummyShardLock; import org.elasticsearch.test.ElasticsearchTestCase; -import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.threadpool.ThreadPool; import org.hamcrest.MatcherAssert; import org.junit.After; @@ -102,8 +100,6 @@ import static org.elasticsearch.index.engine.Engine.Operation.Origin.PRIMARY; import static org.elasticsearch.index.engine.Engine.Operation.Origin.REPLICA; import static org.hamcrest.Matchers.*; -// TODO: this guy isn't ready for mock filesystems yet -@SuppressFileSystems("*") public class InternalEngineTests extends ElasticsearchTestCase { protected final ShardId shardId = new ShardId(new Index("index"), 1); @@ -1555,9 +1551,8 @@ public class InternalEngineTests extends ElasticsearchTestCase { // fake a new translog, causing the engine to point to a missing one. Translog translog = createTranslog(); long id = translog.currentFileGeneration(); - IOUtils.rm(translog.location().resolve(Translog.getFilename(id))); - // we have to re-open the translog because o.w. it will complain about commit information going backwards, which is OK as we did a fake markComitted translog.close(); + IOUtils.rm(translog.location().resolve(Translog.getFilename(id))); try { engine = createEngine(store, primaryTranslogDir); fail("engine shouldn't start without a valid translog id"); @@ -1570,7 +1565,6 @@ public class InternalEngineTests extends ElasticsearchTestCase { engine = createEngine(indexSettingsService, store, primaryTranslogDir, createMergeScheduler(indexSettingsService)); } - @TestLogging("index.translog:TRACE") public void testTranslogReplayWithFailure() throws IOException { boolean canHaveDuplicates = true; boolean autoGeneratedId = true; diff --git a/src/test/java/org/elasticsearch/index/translog/TranslogTests.java b/src/test/java/org/elasticsearch/index/translog/TranslogTests.java index 8d0e1d80808..18c53ecf16b 100644 --- a/src/test/java/org/elasticsearch/index/translog/TranslogTests.java +++ b/src/test/java/org/elasticsearch/index/translog/TranslogTests.java @@ -25,7 +25,6 @@ import org.apache.lucene.index.Term; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.ByteArrayDataOutput; import org.apache.lucene.util.IOUtils; -import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.TestUtil; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; @@ -64,7 +63,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; -import java.util.regex.Matcher; import static com.google.common.collect.Lists.newArrayList; import static org.hamcrest.Matchers.*; @@ -72,7 +70,6 @@ import static org.hamcrest.Matchers.*; /** * */ -@LuceneTestCase.SuppressFileSystems("ExtrasFS") public class TranslogTests extends ElasticsearchTestCase { protected final ShardId shardId = new ShardId(new Index("index"), 1); @@ -117,7 +114,7 @@ public class TranslogTests extends ElasticsearchTestCase { protected Translog create(Path path) throws IOException { Settings build = ImmutableSettings.settingsBuilder() - .put("index.translog.fs.type", TranslogWriter.Type.SIMPLE.name()) + .put(TranslogConfig.INDEX_TRANSLOG_FS_TYPE, TranslogWriter.Type.SIMPLE.name()) .build(); TranslogConfig translogConfig = new TranslogConfig(shardId, path, build, Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, null); return new Translog(translogConfig); From 470e8754456e2cbd042dc26156b570deea301c14 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 18 May 2015 14:19:05 +0200 Subject: [PATCH 42/56] Decrement reference even if IndexShard#postRecovery barfs This can cause a reference leak if we call `IndexShard#postRecovery` on an already closed shard. --- .../indices/recovery/RecoveryStatus.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoveryStatus.java b/src/main/java/org/elasticsearch/indices/recovery/RecoveryStatus.java index 093edffb312..1db369548d9 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoveryStatus.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoveryStatus.java @@ -188,10 +188,13 @@ public class RecoveryStatus extends AbstractRefCounted { public void markAsDone() { if (finished.compareAndSet(false, true)) { assert tempFileNames.isEmpty() : "not all temporary files are renamed"; - indexShard.postRecovery("peer recovery done"); - // release the initial reference. recovery files will be cleaned as soon as ref count goes to zero, potentially now - decRef(); - listener.onRecoveryDone(state()); + try { + indexShard.postRecovery("peer recovery done"); + } finally { + // release the initial reference. recovery files will be cleaned as soon as ref count goes to zero, potentially now + decRef(); + listener.onRecoveryDone(state()); + } } } From 8aef499aaac15eff6b070f85020f0f622f9eb99b Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 18 May 2015 17:00:47 +0200 Subject: [PATCH 43/56] review comments --- .../org/elasticsearch/indices/recovery/RecoveryStatus.java | 4 +++- .../org/elasticsearch/indices/recovery/RecoveryTarget.java | 2 +- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoveryStatus.java b/src/main/java/org/elasticsearch/indices/recovery/RecoveryStatus.java index 1db369548d9..b4d5bf6471e 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoveryStatus.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoveryStatus.java @@ -189,12 +189,14 @@ public class RecoveryStatus extends AbstractRefCounted { if (finished.compareAndSet(false, true)) { assert tempFileNames.isEmpty() : "not all temporary files are renamed"; try { + // this might still throw an exception ie. if the shard is CLOSED due to some other event. + // it's safer to decrement the reference in a try finally here. indexShard.postRecovery("peer recovery done"); } finally { // release the initial reference. recovery files will be cleaned as soon as ref count goes to zero, potentially now decRef(); - listener.onRecoveryDone(state()); } + listener.onRecoveryDone(state()); } } diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index 2c8119bfe44..0b0f65b3f4d 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -262,7 +262,7 @@ public class RecoveryTarget extends AbstractComponent { } } - public static interface RecoveryListener { + public interface RecoveryListener { void onRecoveryDone(RecoveryState state); void onRecoveryFailure(RecoveryState state, RecoveryFailedException e, boolean sendShardFailure); From a75cfb42de2a68dcafd07f7c1631a50971b095b4 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Mon, 18 May 2015 17:05:02 +0200 Subject: [PATCH 44/56] Mappings: Make DocumentMapper.refreshSource() private. This method should not be public, we should refresh the source automatically when we change mappings. --- .../index/mapper/DocumentMapper.java | 4 ++-- .../index/mapper/DocumentMapperParser.java | 5 +---- .../camelcase/CamelCaseFieldNameTests.java | 4 ---- .../index/mapper/multifield/MultiFieldTests.java | 1 - .../mapper/source/DefaultSourceMappingTests.java | 1 - .../mapper/timestamp/TimestampMappingTests.java | 4 ---- .../index/mapper/ttl/TTLMappingTests.java | 16 ++++++++-------- .../index/mapper/update/UpdateMappingTests.java | 4 ---- 8 files changed, 11 insertions(+), 28 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java b/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java index a1a8180f8dc..42e64ca975b 100644 --- a/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java +++ b/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java @@ -491,7 +491,7 @@ public class DocumentMapper implements ToXContent { return mergeResult; } - public CompressedString refreshSource() throws ElasticsearchGenerationException { + private void refreshSource() throws ElasticsearchGenerationException { try { BytesStreamOutput bStream = new BytesStreamOutput(); XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON, CompressorFactory.defaultCompressor().streamOutput(bStream)); @@ -499,7 +499,7 @@ public class DocumentMapper implements ToXContent { toXContent(builder, ToXContent.EMPTY_PARAMS); builder.endObject(); builder.close(); - return mappingSource = new CompressedString(bStream.bytes()); + mappingSource = new CompressedString(bStream.bytes()); } catch (Exception e) { throw new ElasticsearchGenerationException("failed to serialize source for type [" + type + "]", e); } diff --git a/src/main/java/org/elasticsearch/index/mapper/DocumentMapperParser.java b/src/main/java/org/elasticsearch/index/mapper/DocumentMapperParser.java index c955d6aeb6a..9084e17d60b 100644 --- a/src/main/java/org/elasticsearch/index/mapper/DocumentMapperParser.java +++ b/src/main/java/org/elasticsearch/index/mapper/DocumentMapperParser.java @@ -271,10 +271,7 @@ public class DocumentMapperParser extends AbstractIndexComponent { checkNoRemainingFields(mapping, parserContext.indexVersionCreated(), "Root mapping definition has unsupported parameters: "); - DocumentMapper documentMapper = docBuilder.build(mapperService, this); - // update the source with the generated one - documentMapper.refreshSource(); - return documentMapper; + return docBuilder.build(mapperService, this); } public static void checkNoRemainingFields(String fieldName, Map fieldNodeMap, Version indexVersionCreated) { diff --git a/src/test/java/org/elasticsearch/index/mapper/camelcase/CamelCaseFieldNameTests.java b/src/test/java/org/elasticsearch/index/mapper/camelcase/CamelCaseFieldNameTests.java index 622c4e567ac..ae4298c063d 100644 --- a/src/test/java/org/elasticsearch/index/mapper/camelcase/CamelCaseFieldNameTests.java +++ b/src/test/java/org/elasticsearch/index/mapper/camelcase/CamelCaseFieldNameTests.java @@ -26,9 +26,6 @@ import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.test.ElasticsearchSingleNodeTest; import org.junit.Test; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.nullValue; - /** * */ @@ -53,7 +50,6 @@ public class CamelCaseFieldNameTests extends ElasticsearchSingleNodeTest { assertNotNull(documentMapper.mappers().getMapper("thisIsCamelCase")); assertNull(documentMapper.mappers().getMapper("this_is_camel_case")); - documentMapper.refreshSource(); documentMapper = index.mapperService().documentMapperParser().parse(documentMapper.mappingSource().string()); assertNotNull(documentMapper.mappers().getMapper("thisIsCamelCase")); diff --git a/src/test/java/org/elasticsearch/index/mapper/multifield/MultiFieldTests.java b/src/test/java/org/elasticsearch/index/mapper/multifield/MultiFieldTests.java index fd528770d15..e7df72c3dcd 100644 --- a/src/test/java/org/elasticsearch/index/mapper/multifield/MultiFieldTests.java +++ b/src/test/java/org/elasticsearch/index/mapper/multifield/MultiFieldTests.java @@ -148,7 +148,6 @@ public class MultiFieldTests extends ElasticsearchSingleNodeTest { .addMultiField(stringField("indexed").index(true).tokenized(true)) .addMultiField(stringField("not_indexed").index(false).store(true)) )).build(indexService.mapperService(), mapperParser); - builderDocMapper.refreshSource(); String builtMapping = builderDocMapper.mappingSource().string(); // System.out.println(builtMapping); diff --git a/src/test/java/org/elasticsearch/index/mapper/source/DefaultSourceMappingTests.java b/src/test/java/org/elasticsearch/index/mapper/source/DefaultSourceMappingTests.java index 25ab2a5c6c0..f41a94ab132 100644 --- a/src/test/java/org/elasticsearch/index/mapper/source/DefaultSourceMappingTests.java +++ b/src/test/java/org/elasticsearch/index/mapper/source/DefaultSourceMappingTests.java @@ -224,7 +224,6 @@ public class DefaultSourceMappingTests extends ElasticsearchSingleNodeTest { void assertConflicts(String mapping1, String mapping2, DocumentMapperParser parser, String... conflicts) throws IOException { DocumentMapper docMapper = parser.parse(mapping1); - docMapper.refreshSource(); docMapper = parser.parse(docMapper.mappingSource().string()); MergeResult mergeResult = docMapper.merge(parser.parse(mapping2).mapping(), true); diff --git a/src/test/java/org/elasticsearch/index/mapper/timestamp/TimestampMappingTests.java b/src/test/java/org/elasticsearch/index/mapper/timestamp/TimestampMappingTests.java index f4d449ed3d0..471563af05d 100644 --- a/src/test/java/org/elasticsearch/index/mapper/timestamp/TimestampMappingTests.java +++ b/src/test/java/org/elasticsearch/index/mapper/timestamp/TimestampMappingTests.java @@ -534,7 +534,6 @@ public class TimestampMappingTests extends ElasticsearchSingleNodeTest { DocumentMapperParser parser = createIndex("test").mapperService().documentMapperParser(); DocumentMapper docMapper = parser.parse(mapping); - docMapper.refreshSource(); docMapper = parser.parse(docMapper.mappingSource().string()); assertThat(docMapper.mappingSource().string(), equalTo(mapping)); } @@ -557,7 +556,6 @@ public class TimestampMappingTests extends ElasticsearchSingleNodeTest { DocumentMapper docMapper = parser.parse(mapping); boolean tokenized = docMapper.timestampFieldMapper().fieldType().tokenized(); - docMapper.refreshSource(); docMapper = parser.parse(docMapper.mappingSource().string()); assertThat(tokenized, equalTo(docMapper.timestampFieldMapper().fieldType().tokenized())); } @@ -686,7 +684,6 @@ public class TimestampMappingTests extends ElasticsearchSingleNodeTest { void assertConflict(String mapping1, String mapping2, DocumentMapperParser parser, String conflict) throws IOException { DocumentMapper docMapper = parser.parse(mapping1); - docMapper.refreshSource(); docMapper = parser.parse(docMapper.mappingSource().string()); MergeResult mergeResult = docMapper.merge(parser.parse(mapping2).mapping(), true); assertThat(mergeResult.buildConflicts().length, equalTo(conflict == null ? 0 : 1)); @@ -744,7 +741,6 @@ public class TimestampMappingTests extends ElasticsearchSingleNodeTest { DocumentMapperParser parser = createIndex("test_doc_values").mapperService().documentMapperParser(); DocumentMapper docMapper = parser.parse(mapping); boolean docValues= docMapper.timestampFieldMapper().hasDocValues(); - docMapper.refreshSource(); docMapper = parser.parse(docMapper.mappingSource().string()); assertThat(docMapper.timestampFieldMapper().hasDocValues(), equalTo(docValues)); assertAcked(client().admin().indices().prepareDelete("test_doc_values")); diff --git a/src/test/java/org/elasticsearch/index/mapper/ttl/TTLMappingTests.java b/src/test/java/org/elasticsearch/index/mapper/ttl/TTLMappingTests.java index cbd2003bbd7..c858e9437eb 100644 --- a/src/test/java/org/elasticsearch/index/mapper/ttl/TTLMappingTests.java +++ b/src/test/java/org/elasticsearch/index/mapper/ttl/TTLMappingTests.java @@ -216,7 +216,7 @@ public class TTLMappingTests extends ElasticsearchSingleNodeTest { XContentBuilder mappingWithOnlyDefaultSet = getMappingWithOnlyTtlDefaultSet("6m"); MergeResult mergeResult = indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedString(mappingWithOnlyDefaultSet.string()), true).mapping(), false); assertFalse(mergeResult.hasConflicts()); - CompressedString mappingAfterMerge = indexService.mapperService().documentMapper("type").refreshSource(); + CompressedString mappingAfterMerge = indexService.mapperService().documentMapper("type").mappingSource(); assertThat(mappingAfterMerge, equalTo(new CompressedString("{\"type\":{\"_ttl\":{\"enabled\":true,\"default\":360000},\"properties\":{\"field\":{\"type\":\"string\"}}}}"))); } @@ -224,12 +224,12 @@ public class TTLMappingTests extends ElasticsearchSingleNodeTest { public void testMergeWithOnlyDefaultSetTtlDisabled() throws Exception { XContentBuilder mappingWithTtlEnabled = getMappingWithTtlDisabled("7d"); IndexService indexService = createIndex("testindex", ImmutableSettings.settingsBuilder().build(), "type", mappingWithTtlEnabled); - CompressedString mappingAfterCreation = indexService.mapperService().documentMapper("type").refreshSource(); + CompressedString mappingAfterCreation = indexService.mapperService().documentMapper("type").mappingSource(); assertThat(mappingAfterCreation, equalTo(new CompressedString("{\"type\":{\"_ttl\":{\"enabled\":false},\"properties\":{\"field\":{\"type\":\"string\"}}}}"))); XContentBuilder mappingWithOnlyDefaultSet = getMappingWithOnlyTtlDefaultSet("6m"); MergeResult mergeResult = indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedString(mappingWithOnlyDefaultSet.string()), true).mapping(), false); assertFalse(mergeResult.hasConflicts()); - CompressedString mappingAfterMerge = indexService.mapperService().documentMapper("type").refreshSource(); + CompressedString mappingAfterMerge = indexService.mapperService().documentMapper("type").mappingSource(); assertThat(mappingAfterMerge, equalTo(new CompressedString("{\"type\":{\"_ttl\":{\"enabled\":false},\"properties\":{\"field\":{\"type\":\"string\"}}}}"))); } @@ -244,7 +244,7 @@ public class TTLMappingTests extends ElasticsearchSingleNodeTest { MergeResult mergeResult = indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedString(mappingWithTtlDifferentDefault.string()), true).mapping(), true); assertFalse(mergeResult.hasConflicts()); // make sure simulate flag actually worked - no mappings applied - CompressedString mappingAfterMerge = indexService.mapperService().documentMapper("type").refreshSource(); + CompressedString mappingAfterMerge = indexService.mapperService().documentMapper("type").mappingSource(); assertThat(mappingAfterMerge, equalTo(mappingBeforeMerge)); client().admin().indices().prepareDelete("testindex").get(); @@ -256,7 +256,7 @@ public class TTLMappingTests extends ElasticsearchSingleNodeTest { mergeResult = indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedString(mappingWithTtlEnabled.string()), true).mapping(), true); assertFalse(mergeResult.hasConflicts()); // make sure simulate flag actually worked - no mappings applied - mappingAfterMerge = indexService.mapperService().documentMapper("type").refreshSource(); + mappingAfterMerge = indexService.mapperService().documentMapper("type").mappingSource(); assertThat(mappingAfterMerge, equalTo(mappingBeforeMerge)); client().admin().indices().prepareDelete("testindex").get(); @@ -268,7 +268,7 @@ public class TTLMappingTests extends ElasticsearchSingleNodeTest { mergeResult = indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedString(mappingWithTtlEnabled.string()), true).mapping(), true); assertFalse(mergeResult.hasConflicts()); // make sure simulate flag actually worked - no mappings applied - mappingAfterMerge = indexService.mapperService().documentMapper("type").refreshSource(); + mappingAfterMerge = indexService.mapperService().documentMapper("type").mappingSource(); assertThat(mappingAfterMerge, equalTo(mappingBeforeMerge)); client().admin().indices().prepareDelete("testindex").get(); @@ -279,7 +279,7 @@ public class TTLMappingTests extends ElasticsearchSingleNodeTest { mergeResult = indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedString(mappingWithTtlEnabled.string()), true).mapping(), false); assertFalse(mergeResult.hasConflicts()); // make sure simulate flag actually worked - mappings applied - mappingAfterMerge = indexService.mapperService().documentMapper("type").refreshSource(); + mappingAfterMerge = indexService.mapperService().documentMapper("type").mappingSource(); assertThat(mappingAfterMerge, equalTo(new CompressedString("{\"type\":{\"_ttl\":{\"enabled\":true,\"default\":604800000},\"properties\":{\"field\":{\"type\":\"string\"}}}}"))); client().admin().indices().prepareDelete("testindex").get(); @@ -289,7 +289,7 @@ public class TTLMappingTests extends ElasticsearchSingleNodeTest { mergeResult = indexService.mapperService().documentMapper("type").merge(indexService.mapperService().parse("type", new CompressedString(mappingWithTtlEnabled.string()), true).mapping(), false); assertFalse(mergeResult.hasConflicts()); // make sure simulate flag actually worked - mappings applied - mappingAfterMerge = indexService.mapperService().documentMapper("type").refreshSource(); + mappingAfterMerge = indexService.mapperService().documentMapper("type").mappingSource(); assertThat(mappingAfterMerge, equalTo(new CompressedString("{\"type\":{\"_ttl\":{\"enabled\":true,\"default\":604800000},\"properties\":{\"field\":{\"type\":\"string\"}}}}"))); } diff --git a/src/test/java/org/elasticsearch/index/mapper/update/UpdateMappingTests.java b/src/test/java/org/elasticsearch/index/mapper/update/UpdateMappingTests.java index 7c12cd14c36..9dc5900187c 100644 --- a/src/test/java/org/elasticsearch/index/mapper/update/UpdateMappingTests.java +++ b/src/test/java/org/elasticsearch/index/mapper/update/UpdateMappingTests.java @@ -126,7 +126,6 @@ public class UpdateMappingTests extends ElasticsearchSingleNodeTest { .endObject(); DocumentMapper documentMapper = indexService.mapperService().parse("type", new CompressedString(indexMapping.string()), true); assertThat(documentMapper.indexMapper().enabled(), equalTo(enabled)); - documentMapper.refreshSource(); documentMapper = indexService.mapperService().parse("type", new CompressedString(documentMapper.mappingSource().string()), true); assertThat(documentMapper.indexMapper().enabled(), equalTo(enabled)); } @@ -151,7 +150,6 @@ public class UpdateMappingTests extends ElasticsearchSingleNodeTest { assertThat(documentMapper.timestampFieldMapper().enabled(), equalTo(enabled)); assertTrue(documentMapper.timestampFieldMapper().fieldType().stored()); assertTrue(documentMapper.timestampFieldMapper().hasDocValues()); - documentMapper.refreshSource(); documentMapper = indexService.mapperService().parse("type", new CompressedString(documentMapper.mappingSource().string()), true); assertThat(documentMapper.timestampFieldMapper().enabled(), equalTo(enabled)); assertTrue(documentMapper.timestampFieldMapper().hasDocValues()); @@ -173,7 +171,6 @@ public class UpdateMappingTests extends ElasticsearchSingleNodeTest { DocumentMapper documentMapper = indexService.mapperService().parse("type", new CompressedString(indexMapping.string()), true); assertThat(documentMapper.sizeFieldMapper().enabled(), equalTo(enabled)); assertTrue(documentMapper.sizeFieldMapper().fieldType().stored()); - documentMapper.refreshSource(); documentMapper = indexService.mapperService().parse("type", new CompressedString(documentMapper.mappingSource().string()), true); assertThat(documentMapper.sizeFieldMapper().enabled(), equalTo(enabled)); } @@ -184,7 +181,6 @@ public class UpdateMappingTests extends ElasticsearchSingleNodeTest { String mapping = copyToStringFromClasspath("/org/elasticsearch/index/mapper/update/default_mapping_with_disabled_root_types.json"); DocumentMapper documentMapper = indexService.mapperService().parse("type", new CompressedString(mapping), true); assertThat(documentMapper.mappingSource().string(), equalTo(mapping)); - documentMapper.refreshSource(); documentMapper = indexService.mapperService().parse("type", new CompressedString(documentMapper.mappingSource().string()), true); assertThat(documentMapper.mappingSource().string(), equalTo(mapping)); } From 60b66a7235dc9eb30e5705a3d77096b28925664f Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 18 May 2015 11:14:00 +0200 Subject: [PATCH 45/56] [TRANSLOG] Fold Translog.View into it's only implementation --- .../index/translog/Translog.java | 73 +++++++++---------- .../recovery/RecoverySourceHandler.java | 4 +- .../SharedFSRecoverySourceHandler.java | 32 +------- 3 files changed, 39 insertions(+), 70 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/translog/Translog.java b/src/main/java/org/elasticsearch/index/translog/Translog.java index 8bc71be6576..3323a71006a 100644 --- a/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -110,7 +110,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC private volatile ScheduledFuture syncScheduler; // this is a concurrent set and is not protected by any of the locks. The main reason // is that is being accessed by two separate classes (additions & reading are done by FsTranslog, remove by FsView when closed) - private final Set outstandingViews = ConcurrentCollections.newConcurrentSet(); + private final Set outstandingViews = ConcurrentCollections.newConcurrentSet(); private BigArrays bigArrays; protected final ReleasableLock readLock; protected final ReleasableLock writeLock; @@ -121,6 +121,14 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC private final AtomicBoolean closed = new AtomicBoolean(); private final TranslogConfig config; private final String translogUUID; + private Callback onViewClose = new Callback() { + @Override + public void handle(View view) { + logger.trace("closing view starting at translog [{}]", view.minTranslogGeneration()); + outstandingViews.remove(this); + } + }; + /** @@ -475,7 +483,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC } } - private Snapshot createSnapshot(TranslogReader... translogs) { + private static Snapshot createSnapshot(TranslogReader... translogs) { Snapshot[] snapshots = new Snapshot[translogs.length]; boolean success = false; try { @@ -507,7 +515,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC translogs.add(currentCommittingTranslog.clone()); } translogs.add(current.newReaderFromWriter()); - FsView view = new FsView(translogs); + View view = new View(translogs, onViewClose); // this is safe as we know that no new translog is being made at the moment // (we hold a read lock) and the view will be notified of any future one outstandingViews.add(view); @@ -615,16 +623,18 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC * a view into the translog, capturing all translog file at the moment of creation * and updated with any future translog. */ - class FsView implements View { + public static final class View implements Closeable { + public static final Translog.View EMPTY_VIEW = new View(Collections.EMPTY_LIST, null); boolean closed; // last in this list is always FsTranslog.current final List orderedTranslogs; + private final Callback onClose; - FsView(List orderedTranslogs) { - assert orderedTranslogs.isEmpty() == false; + View(List orderedTranslogs, Callback onClose) { // clone so we can safely mutate.. this.orderedTranslogs = new ArrayList<>(orderedTranslogs); + this.onClose = onClose; } /** @@ -647,13 +657,15 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC orderedTranslogs.add(newCurrent); } - @Override + /** this smallest translog generation in this view */ public synchronized long minTranslogGeneration() { ensureOpen(); return orderedTranslogs.get(0).getGeneration(); } - @Override + /** + * The total number of operations in the view. + */ public synchronized int totalOperations() { int ops = 0; for (TranslogReader translog : orderedTranslogs) { @@ -667,7 +679,9 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC return ops; } - @Override + /** + * Returns the size in bytes of the files behind the view. + */ public synchronized long sizeInBytes() { long size = 0; for (TranslogReader translog : orderedTranslogs) { @@ -676,6 +690,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC return size; } + /** create a snapshot from this view */ public synchronized Snapshot snapshot() { ensureOpen(); return createSnapshot(orderedTranslogs.toArray(new TranslogReader[orderedTranslogs.size()])); @@ -690,15 +705,19 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC @Override public void close() { - List toClose = new ArrayList<>(); + final List toClose = new ArrayList<>(); try { synchronized (this) { if (closed == false) { - logger.trace("closing view starting at translog [{}]", minTranslogGeneration()); - closed = true; - outstandingViews.remove(this); - toClose.addAll(orderedTranslogs); - orderedTranslogs.clear(); + try { + if (onClose != null) { + onClose.handle(this); + } + } finally { + closed = true; + toClose.addAll(orderedTranslogs); + orderedTranslogs.clear(); + } } } } finally { @@ -816,27 +835,6 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC } - /** a view into the current translog that receives all operations from the moment created */ - public interface View extends Releasable { - - /** - * The total number of operations in the view. - */ - int totalOperations(); - - /** - * Returns the size in bytes of the files behind the view. - */ - long sizeInBytes(); - - /** create a snapshot from this view */ - Snapshot snapshot(); - - /** this smallest translog generation in this view */ - long minTranslogGeneration(); - - } - /** * A generic interface representing an operation performed on the transaction log. * Each is associated with a type. @@ -1666,7 +1664,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC current = createWriter(current.getGeneration() + 1); // notify all outstanding views of the new translog (no views are created now as // we hold a write lock). - for (FsView view : outstandingViews) { + for (View view : outstandingViews) { view.onNewTranslog(currentCommittingTranslog.clone(), current.newReaderFromWriter()); } IOUtils.close(oldCurrent); @@ -1759,5 +1757,4 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC } } - } diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 823742404e3..86cc680b56a 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -29,6 +29,7 @@ import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.RateLimiter; import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.IOUtils; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.common.Nullable; @@ -56,6 +57,7 @@ import org.elasticsearch.transport.RemoteTransportException; import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportService; +import java.io.Closeable; import java.io.IOException; import java.util.Comparator; import java.util.List; @@ -123,7 +125,7 @@ public class RecoverySourceHandler { try { phase1Snapshot = shard.snapshotIndex(false); } catch (Throwable e) { - Releasables.closeWhileHandlingException(translogView); + IOUtils.closeWhileHandlingException(translogView); throw new RecoveryEngineException(shard.shardId(), 1, "Snapshot failed", e); } diff --git a/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java b/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java index 9e80accf5e9..2b691f558e1 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java +++ b/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java @@ -34,7 +34,6 @@ public class SharedFSRecoverySourceHandler extends RecoverySourceHandler { private final IndexShard shard; private final StartRecoveryRequest request; - private static final Translog.View EMPTY_VIEW = new EmptyView(); public SharedFSRecoverySourceHandler(IndexShard shard, StartRecoveryRequest request, RecoverySettings recoverySettings, TransportService transportService, ESLogger logger) { super(shard, request, recoverySettings, transportService, logger); @@ -59,7 +58,7 @@ public class SharedFSRecoverySourceHandler extends RecoverySourceHandler { shard.failShard("failed to close engine (phase1)", e); } } - prepareTargetForTranslog(EMPTY_VIEW); + prepareTargetForTranslog(Translog.View.EMPTY_VIEW); finalizeRecovery(); return response; } catch (Throwable t) { @@ -88,33 +87,4 @@ public class SharedFSRecoverySourceHandler extends RecoverySourceHandler { return request.recoveryType() == RecoveryState.Type.RELOCATION && shard.routingEntry().primary(); } - /** - * An empty view since we don't recover from translog even in the shared FS case - */ - private static class EmptyView implements Translog.View { - - @Override - public int totalOperations() { - return 0; - } - - @Override - public long sizeInBytes() { - return 0; - } - - @Override - public Translog.Snapshot snapshot() { - return null; - } - - @Override - public long minTranslogGeneration() { - return 0; - } - - @Override - public void close() { - } - } } From 651c067fae82fc5a28702dea7f5e22334713cb1a Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 18 May 2015 14:10:05 +0200 Subject: [PATCH 46/56] Cleanup translog operation serialization We used to double write the translog operation which is not needed except of for recovery. This commit cuts over to a big-array based temporary serialiation and removes the crazy double writing. --- .../BufferedChecksumStreamOutput.java | 4 + .../index/translog/Translog.java | 83 +++++++++++-------- .../RecoveryTranslogOperationsRequest.java | 13 +-- .../index/translog/TranslogTests.java | 13 +-- 4 files changed, 57 insertions(+), 56 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/translog/BufferedChecksumStreamOutput.java b/src/main/java/org/elasticsearch/index/translog/BufferedChecksumStreamOutput.java index 3c3fbf1ecbb..b99a6da5f22 100644 --- a/src/main/java/org/elasticsearch/index/translog/BufferedChecksumStreamOutput.java +++ b/src/main/java/org/elasticsearch/index/translog/BufferedChecksumStreamOutput.java @@ -70,4 +70,8 @@ public final class BufferedChecksumStreamOutput extends StreamOutput { out.reset(); digest.reset(); } + + public void resetDigest() { + digest.reset(); + } } diff --git a/src/main/java/org/elasticsearch/index/translog/Translog.java b/src/main/java/org/elasticsearch/index/translog/Translog.java index 3323a71006a..aca4e61f6ed 100644 --- a/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -450,7 +450,15 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC public Location add(Operation operation) throws TranslogException { ReleasableBytesStreamOutput out = new ReleasableBytesStreamOutput(bigArrays); try { - writeOperation(out, operation); + final BufferedChecksumStreamOutput checksumStreamOutput = new BufferedChecksumStreamOutput(out); + final long start = out.position(); + out.skip(RamUsageEstimator.NUM_BYTES_INT); + writeOperationNoSize(checksumStreamOutput, operation); + long end = out.position(); + int operationSize = (int) (out.position() - RamUsageEstimator.NUM_BYTES_INT - start); + out.seek(start); + out.writeInt(operationSize); + out.seek(end); ReleasablePagedBytesReference bytes = out.bytes(); try (ReleasableLock lock = readLock.acquire()) { Location location = current.add(bytes); @@ -1546,29 +1554,17 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC } } - public static Snapshot snapshotFromStream(StreamInput input, final int numOps) { + /** + * Reads a list of operations written with {@link #writeOperations(StreamOutput, List)} + */ + public static List readOperations(StreamInput input) throws IOException { + ArrayList operations = new ArrayList<>(); + int numOps = input.readInt(); final BufferedChecksumStreamInput checksumStreamInput = new BufferedChecksumStreamInput(input); - return new Snapshot() { - int read = 0; - @Override - public int estimatedTotalOperations() { - return numOps; - } - - @Override - public Operation next() throws IOException { - if (read < numOps) { - read++; - return readOperation(checksumStreamInput); - } - return null; - } - - @Override - public void close() { - // doNothing - } - }; + for (int i = 0; i < numOps; i++) { + operations.add(readOperation(checksumStreamInput)); + } + return operations; } static Translog.Operation readOperation(BufferedChecksumStreamInput in) throws IOException { @@ -1601,24 +1597,39 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC return operation; } - public static void writeOperation(StreamOutput outStream, Translog.Operation op) throws IOException { - //TODO lets get rid of this crazy double writing here. + /** + * Writes all operations in the given iterable to the given output stream including the size of the array + * use {@link #readOperations(StreamInput)} to read it back. + */ + public static void writeOperations(StreamOutput outStream, List toWrite) throws IOException { + final ReleasableBytesStreamOutput out = new ReleasableBytesStreamOutput(BigArrays.NON_RECYCLING_INSTANCE); + try { + outStream.writeInt(toWrite.size()); + final BufferedChecksumStreamOutput checksumStreamOutput = new BufferedChecksumStreamOutput(out); + for (Operation op : toWrite) { + out.reset(); + final long start = out.position(); + out.skip(RamUsageEstimator.NUM_BYTES_INT); + writeOperationNoSize(checksumStreamOutput, op); + long end = out.position(); + int operationSize = (int) (out.position() - RamUsageEstimator.NUM_BYTES_INT - start); + out.seek(start); + out.writeInt(operationSize); + out.seek(end); + ReleasablePagedBytesReference bytes = out.bytes(); + bytes.writeTo(outStream); + } + } finally { + Releasables.close(out.bytes()); + } - // We first write to a NoopStreamOutput to get the size of the - // operation. We could write to a byte array and then send that as an - // alternative, but here we choose to use CPU over allocating new - // byte arrays. - NoopStreamOutput noopOut = new NoopStreamOutput(); - noopOut.writeByte(op.opType().id()); - op.writeTo(noopOut); - noopOut.writeInt(0); // checksum holder - int size = noopOut.getCount(); + } + public static void writeOperationNoSize(BufferedChecksumStreamOutput out, Translog.Operation op) throws IOException { // This BufferedChecksumStreamOutput remains unclosed on purpose, // because closing it closes the underlying stream, which we don't // want to do here. - BufferedChecksumStreamOutput out = new BufferedChecksumStreamOutput(outStream); - outStream.writeInt(size); // opSize is not checksummed + out.resetDigest(); out.writeByte(op.opType().id()); op.writeTo(out); long checksum = out.getChecksum(); diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoveryTranslogOperationsRequest.java b/src/main/java/org/elasticsearch/indices/recovery/RecoveryTranslogOperationsRequest.java index 8e377c729c7..b320c98568a 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoveryTranslogOperationsRequest.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoveryTranslogOperationsRequest.java @@ -70,13 +70,7 @@ class RecoveryTranslogOperationsRequest extends TransportRequest { super.readFrom(in); recoveryId = in.readLong(); shardId = ShardId.readShardId(in); - int size = in.readVInt(); - operations = Lists.newArrayListWithExpectedSize(size); - Translog.Snapshot snapshot = Translog.snapshotFromStream(in, size); - Translog.Operation next = null; - while((next = snapshot.next()) != null) { - operations.add(next); - } + operations = Translog.readOperations(in); totalTranslogOps = in.readVInt(); } @@ -85,10 +79,7 @@ class RecoveryTranslogOperationsRequest extends TransportRequest { super.writeTo(out); out.writeLong(recoveryId); shardId.writeTo(out); - out.writeVInt(operations.size()); - for (Translog.Operation operation : operations) { - Translog.writeOperation(out, operation); - } + Translog.writeOperations(out, operations); out.writeVInt(totalTranslogOps); } } diff --git a/src/test/java/org/elasticsearch/index/translog/TranslogTests.java b/src/test/java/org/elasticsearch/index/translog/TranslogTests.java index 18c53ecf16b..e1068d9e0ab 100644 --- a/src/test/java/org/elasticsearch/index/translog/TranslogTests.java +++ b/src/test/java/org/elasticsearch/index/translog/TranslogTests.java @@ -1021,23 +1021,18 @@ public class TranslogTests extends ElasticsearchTestCase { } - public void testSnapshotFromStreamInput() throws IOException { BytesStreamOutput out = new BytesStreamOutput(); List ops = newArrayList(); int translogOperations = randomIntBetween(10, 100); for (int op = 0; op < translogOperations; op++) { Translog.Create test = new Translog.Create("test", "" + op, Integer.toString(op).getBytes(Charset.forName("UTF-8"))); - Translog.writeOperation(out, test); ops.add(test); } - Translog.Snapshot snapshot = Translog.snapshotFromStream(StreamInput.wrap(out.bytes()), ops.size()); - assertEquals(ops.size(), snapshot.estimatedTotalOperations()); - for (Translog.Operation op : ops) { - assertEquals(op, snapshot.next()); - } - assertNull(snapshot.next()); - // no need to close + Translog.writeOperations(out, ops); + final List readOperations = Translog.readOperations(StreamInput.wrap(out.bytes())); + assertEquals(ops.size(), readOperations.size()); + assertEquals(ops, readOperations); } public void testLocationHashCodeEquals() throws IOException { From bc810e13206ff6fe8a0765755cb544408c821f93 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 18 May 2015 14:10:56 +0200 Subject: [PATCH 47/56] remove NoopOutputStream --- .../common/io/stream/NoopStreamOutput.java | 61 ------------------- 1 file changed, 61 deletions(-) delete mode 100644 src/main/java/org/elasticsearch/common/io/stream/NoopStreamOutput.java diff --git a/src/main/java/org/elasticsearch/common/io/stream/NoopStreamOutput.java b/src/main/java/org/elasticsearch/common/io/stream/NoopStreamOutput.java deleted file mode 100644 index c77cce8d9b7..00000000000 --- a/src/main/java/org/elasticsearch/common/io/stream/NoopStreamOutput.java +++ /dev/null @@ -1,61 +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.common.io.stream; - -import java.io.IOException; - -/** - * A non-threadsafe StreamOutput that doesn't actually write the bytes to any - * stream, it only keeps track of how many bytes have been written - */ -public final class NoopStreamOutput extends StreamOutput { - - private int count = 0; - - /** Retrieve the number of bytes that have been written */ - public int getCount() { - return count; - } - - @Override - public void writeByte(byte b) throws IOException { - count++; - } - - @Override - public void writeBytes(byte[] b, int offset, int length) throws IOException { - count += length; - } - - @Override - public void flush() throws IOException { - // no-op - } - - @Override - public void close() throws IOException { - // nothing to close - } - - @Override - public void reset() throws IOException { - count = 0; - } -} From a21e4449b7136c4a95d344d235cb8c5dafe61758 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 18 May 2015 17:04:54 +0200 Subject: [PATCH 48/56] Added additional assertion that the view to remove was actually registered. --- .../elasticsearch/index/translog/Translog.java | 18 +++++++++++++----- .../index/translog/TranslogTests.java | 3 +++ 2 files changed, 16 insertions(+), 5 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/translog/Translog.java b/src/main/java/org/elasticsearch/index/translog/Translog.java index aca4e61f6ed..af04ec50dd3 100644 --- a/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -125,7 +125,8 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC @Override public void handle(View view) { logger.trace("closing view starting at translog [{}]", view.minTranslogGeneration()); - outstandingViews.remove(this); + boolean removed = outstandingViews.remove(view); + assert removed : "View was never set but was supposed to be removed"; } }; @@ -448,18 +449,18 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC * @see org.elasticsearch.index.translog.Translog.Delete */ public Location add(Operation operation) throws TranslogException { - ReleasableBytesStreamOutput out = new ReleasableBytesStreamOutput(bigArrays); + final ReleasableBytesStreamOutput out = new ReleasableBytesStreamOutput(bigArrays); try { final BufferedChecksumStreamOutput checksumStreamOutput = new BufferedChecksumStreamOutput(out); final long start = out.position(); out.skip(RamUsageEstimator.NUM_BYTES_INT); writeOperationNoSize(checksumStreamOutput, operation); - long end = out.position(); - int operationSize = (int) (out.position() - RamUsageEstimator.NUM_BYTES_INT - start); + final long end = out.position(); + final int operationSize = (int) (end - RamUsageEstimator.NUM_BYTES_INT - start); out.seek(start); out.writeInt(operationSize); out.seek(end); - ReleasablePagedBytesReference bytes = out.bytes(); + final ReleasablePagedBytesReference bytes = out.bytes(); try (ReleasableLock lock = readLock.acquire()) { Location location = current.add(bytes); if (config.isSyncOnEachOperation()) { @@ -1768,4 +1769,11 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC } } + /** + * The number of currently open views + */ + int getNumOpenViews() { + return outstandingViews.size(); + } + } diff --git a/src/test/java/org/elasticsearch/index/translog/TranslogTests.java b/src/test/java/org/elasticsearch/index/translog/TranslogTests.java index e1068d9e0ab..7b34427dfc4 100644 --- a/src/test/java/org/elasticsearch/index/translog/TranslogTests.java +++ b/src/test/java/org/elasticsearch/index/translog/TranslogTests.java @@ -25,6 +25,7 @@ import org.apache.lucene.index.Term; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.ByteArrayDataOutput; import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.TestUtil; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; @@ -70,6 +71,7 @@ import static org.hamcrest.Matchers.*; /** * */ +@LuceneTestCase.SuppressFileSystems("ExtrasFS") public class TranslogTests extends ElasticsearchTestCase { protected final ShardId shardId = new ShardId(new Index("index"), 1); @@ -106,6 +108,7 @@ public class TranslogTests extends ElasticsearchTestCase { @After public void tearDown() throws Exception { try { + assertEquals("there are still open views", 0, translog.getNumOpenViews()); translog.close(); } finally { super.tearDown(); From 2999f53e86b01844ee60983f89459cd98af58ccb Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Mon, 18 May 2015 18:02:19 +0200 Subject: [PATCH 49/56] Tests: Mark RelocationTests.testRelocationWithBusyClusterUpdateThread as @AwaitsFix. --- src/test/java/org/elasticsearch/recovery/RelocationTests.java | 1 + 1 file changed, 1 insertion(+) diff --git a/src/test/java/org/elasticsearch/recovery/RelocationTests.java b/src/test/java/org/elasticsearch/recovery/RelocationTests.java index 8e131e99f90..3ba5ab9cee0 100644 --- a/src/test/java/org/elasticsearch/recovery/RelocationTests.java +++ b/src/test/java/org/elasticsearch/recovery/RelocationTests.java @@ -447,6 +447,7 @@ public class RelocationTests extends ElasticsearchIntegrationTest { @Test @Slow @TestLogging("cluster.service:TRACE,indices.recovery:TRACE") + @AwaitsFix(bugUrl="Fails now that we removed the mapping update from phase 2 in #11207") public void testRelocationWithBusyClusterUpdateThread() throws Exception { final String indexName = "test"; final Settings settings = ImmutableSettings.builder() From 13e5c19dcf07ff5d70fcaf2f58f8bf458c57bc72 Mon Sep 17 00:00:00 2001 From: Alexander Reelsen Date: Mon, 18 May 2015 18:14:40 +0200 Subject: [PATCH 50/56] Test: Prevent automatic refresh in ContextAndHeaderTransportTests In some cases due to calling checking `rarely()` the `indexRandom()` method can potentially flush, which creates flush requests, that miss a certain header in this test and allow the test to fail. In addition unused configuration code for this test has been removed. --- .../ContextAndHeaderTransportTests.java | 101 ++++++++---------- 1 file changed, 44 insertions(+), 57 deletions(-) diff --git a/src/test/java/org/elasticsearch/transport/ContextAndHeaderTransportTests.java b/src/test/java/org/elasticsearch/transport/ContextAndHeaderTransportTests.java index 6563a9a4a27..bc506de6bdc 100644 --- a/src/test/java/org/elasticsearch/transport/ContextAndHeaderTransportTests.java +++ b/src/test/java/org/elasticsearch/transport/ContextAndHeaderTransportTests.java @@ -92,19 +92,12 @@ public class ContextAndHeaderTransportTests extends ElasticsearchIntegrationTest .startObject("properties") .startObject("location").field("type", "geo_shape").endObject() .startObject("name").field("type", "string").endObject() - .startObject("title").field("type", "string").field("analyzer", "text").endObject() .endObject() .endObject().endObject().string(); Settings settings = settingsBuilder() .put(indexSettings()) .put(SETTING_NUMBER_OF_SHARDS, 1) // A single shard will help to keep the tests repeatable. - .put("index.analysis.analyzer.text.tokenizer", "standard") - .putArray("index.analysis.analyzer.text.filter", "lowercase", "my_shingle") - .put("index.analysis.filter.my_shingle.type", "shingle") - .put("index.analysis.filter.my_shingle.output_unigrams", true) - .put("index.analysis.filter.my_shingle.min_shingle_size", 2) - .put("index.analysis.filter.my_shingle.max_shingle_size", 3) .build(); assertAcked(transportClient().admin().indices().prepareCreate(lookupIndex) .setSettings(settings).addMapping("type", mapping)); @@ -119,18 +112,8 @@ public class ContextAndHeaderTransportTests extends ElasticsearchIntegrationTest public void checkAllRequestsContainHeaders() { assertRequestsContainHeader(IndexRequest.class); assertRequestsContainHeader(RefreshRequest.class); - - /* - for (ActionRequest request : requests) { - String msg = String.format(Locale.ROOT, "Expected request [%s] to have randomized header key set", request.getClass().getSimpleName()); - assertThat(msg, request.hasHeader(randomHeaderKey), is(true)); - assertThat(request.getHeader(randomHeaderKey).toString(), is(randomHeaderValue)); - } - */ } - // TODO check context as well - @Test public void testThatTermsLookupGetRequestContainsContextAndHeaders() throws Exception { transportClient().prepareIndex(lookupIndex, "type", "1") @@ -154,29 +137,29 @@ public class ContextAndHeaderTransportTests extends ElasticsearchIntegrationTest @Test public void testThatGeoShapeQueryGetRequestContainsContextAndHeaders() throws Exception { - indexRandom(false, false, - transportClient().prepareIndex(lookupIndex, "type", "1").setSource(jsonBuilder().startObject() - .field("name", "Munich Suburban Area") - .startObject("location") - .field("type", "polygon") - .startArray("coordinates").startArray() - .startArray().value(11.34).value(48.25).endArray() - .startArray().value(11.68).value(48.25).endArray() - .startArray().value(11.65).value(48.06).endArray() - .startArray().value(11.37).value(48.13).endArray() - .startArray().value(11.34).value(48.25).endArray() // close the polygon - .endArray().endArray() - .endObject() - .endObject()), - // second document - transportClient().prepareIndex(queryIndex, "type", "1").setSource(jsonBuilder().startObject() - .field("name", "Munich Center") - .startObject("location") - .field("type", "point") - .startArray("coordinates").value(11.57).value(48.13).endArray() - .endObject() - .endObject()) - ); + transportClient().prepareIndex(lookupIndex, "type", "1").setSource(jsonBuilder().startObject() + .field("name", "Munich Suburban Area") + .startObject("location") + .field("type", "polygon") + .startArray("coordinates").startArray() + .startArray().value(11.34).value(48.25).endArray() + .startArray().value(11.68).value(48.25).endArray() + .startArray().value(11.65).value(48.06).endArray() + .startArray().value(11.37).value(48.13).endArray() + .startArray().value(11.34).value(48.25).endArray() // close the polygon + .endArray().endArray() + .endObject() + .endObject()) + .get(); + // second document + transportClient().prepareIndex(queryIndex, "type", "1").setSource(jsonBuilder().startObject() + .field("name", "Munich Center") + .startObject("location") + .field("type", "point") + .startArray("coordinates").value(11.57).value(48.13).endArray() + .endObject() + .endObject()) + .get(); transportClient().admin().indices().prepareRefresh(lookupIndex, queryIndex).get(); GeoShapeQueryBuilder queryBuilder = QueryBuilders.geoShapeQuery("location", "1", "type") @@ -196,13 +179,15 @@ public class ContextAndHeaderTransportTests extends ElasticsearchIntegrationTest @Test public void testThatMoreLikeThisQueryMultiTermVectorRequestContainsContextAndHeaders() throws Exception { - indexRandom(false, false, - transportClient().prepareIndex(lookupIndex, "type", "1") - .setSource(jsonBuilder().startObject().field("name", "Star Wars - The new republic").endObject()), - transportClient().prepareIndex(queryIndex, "type", "1") - .setSource(jsonBuilder().startObject().field("name", "Jar Jar Binks - A horrible mistake").endObject()), - transportClient().prepareIndex(queryIndex, "type", "2") - .setSource(jsonBuilder().startObject().field("name", "Star Wars - Return of the jedi").endObject())); + transportClient().prepareIndex(lookupIndex, "type", "1") + .setSource(jsonBuilder().startObject().field("name", "Star Wars - The new republic").endObject()) + .get(); + transportClient().prepareIndex(queryIndex, "type", "1") + .setSource(jsonBuilder().startObject().field("name", "Jar Jar Binks - A horrible mistake").endObject()) + .get(); + transportClient().prepareIndex(queryIndex, "type", "2") + .setSource(jsonBuilder().startObject().field("name", "Star Wars - Return of the jedi").endObject()) + .get(); transportClient().admin().indices().prepareRefresh(lookupIndex, queryIndex).get(); MoreLikeThisQueryBuilder moreLikeThisQueryBuilder = QueryBuilders.moreLikeThisQuery("name") @@ -222,12 +207,12 @@ public class ContextAndHeaderTransportTests extends ElasticsearchIntegrationTest @Test public void testThatPercolatingExistingDocumentGetRequestContainsContextAndHeaders() throws Exception { - indexRandom(false, - transportClient().prepareIndex(lookupIndex, ".percolator", "1") - .setSource(jsonBuilder().startObject().startObject("query").startObject("match").field("name", "star wars").endObject().endObject().endObject()), - transportClient().prepareIndex(lookupIndex, "type", "1") - .setSource(jsonBuilder().startObject().field("name", "Star Wars - The new republic").endObject()) - ); + transportClient().prepareIndex(lookupIndex, ".percolator", "1") + .setSource(jsonBuilder().startObject().startObject("query").startObject("match").field("name", "star wars").endObject().endObject().endObject()) + .get(); + transportClient().prepareIndex(lookupIndex, "type", "1") + .setSource(jsonBuilder().startObject().field("name", "Star Wars - The new republic").endObject()) + .get(); transportClient().admin().indices().prepareRefresh(lookupIndex).get(); GetRequest getRequest = transportClient().prepareGet(lookupIndex, "type", "1").request(); @@ -244,8 +229,9 @@ public class ContextAndHeaderTransportTests extends ElasticsearchIntegrationTest ).get(); assertThat(scriptResponse.isCreated(), is(true)); - indexRandom(false, false, transportClient().prepareIndex(queryIndex, "type", "1") - .setSource(jsonBuilder().startObject().field("name", "Star Wars - The new republic").endObject())); + transportClient().prepareIndex(queryIndex, "type", "1") + .setSource(jsonBuilder().startObject().field("name", "Star Wars - The new republic").endObject()) + .get(); transportClient().admin().indices().prepareRefresh(queryIndex).get(); // custom content, not sure how to specify "script_id" otherwise in the API @@ -272,8 +258,9 @@ public class ContextAndHeaderTransportTests extends ElasticsearchIntegrationTest ).get(); assertThat(scriptResponse.isCreated(), is(true)); - indexRandom(false, false, transportClient().prepareIndex(queryIndex, "type", "1") - .setSource(jsonBuilder().startObject().field("name", "Star Wars - The new republic").endObject())); + transportClient().prepareIndex(queryIndex, "type", "1") + .setSource(jsonBuilder().startObject().field("name", "Star Wars - The new republic").endObject()) + .get(); transportClient().admin().indices().prepareRefresh(queryIndex).get(); Map params = new HashMap<>(); From 4b8ed1b0bf69e4d2ce187f33e70dad77b67359f3 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 18 May 2015 20:14:39 +0200 Subject: [PATCH 51/56] [TRANSLOG] Return null from read() if location generation is outdated This behavior has changed been changed rescently to throw an IAE if the translog we try to read from is already outdated. This is not the expected behavior and this commit adds back the `old` way returning `null` instead. The InternalEngine implementation will then go and ask the lucene index for the document instead. --- .../org/elasticsearch/index/translog/Translog.java | 13 +++++++++---- .../index/translog/TranslogTests.java | 14 ++++++++++++-- 2 files changed, 21 insertions(+), 6 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/translog/Translog.java b/src/main/java/org/elasticsearch/index/translog/Translog.java index af04ec50dd3..f64423290e2 100644 --- a/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -422,17 +422,22 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC /** - * Read the Operation object from the given location. + * Read the Operation object from the given location. This method will try to read the given location from + * the current or from the currently committing translog file. If the location is in a file that has already + * been closed or even removed the method will return null instead. */ public Translog.Operation read(Location location) { try (ReleasableLock lock = readLock.acquire()) { final TranslogReader reader; - if (current.getGeneration() == location.generation) { + final long currentGeneration = current.getGeneration(); + if (currentGeneration == location.generation) { reader = current; } else if (currentCommittingTranslog != null && currentCommittingTranslog.getGeneration() == location.generation) { reader = currentCommittingTranslog; + } else if (currentGeneration < location.generation) { + throw new IllegalStateException("location generation [" + location.generation + "] is greater than the current generation [" + currentGeneration + "]"); } else { - throw new IllegalStateException("Can't read from translog location" + location); + return null; } return reader.read(location); } catch (IOException e) { @@ -776,7 +781,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC public final long translogLocation; public final int size; - public Location(long generation, long translogLocation, int size) { + Location(long generation, long translogLocation, int size) { this.generation = generation; this.translogLocation = translogLocation; this.size = size; diff --git a/src/test/java/org/elasticsearch/index/translog/TranslogTests.java b/src/test/java/org/elasticsearch/index/translog/TranslogTests.java index 7b34427dfc4..340b0210ecc 100644 --- a/src/test/java/org/elasticsearch/index/translog/TranslogTests.java +++ b/src/test/java/org/elasticsearch/index/translog/TranslogTests.java @@ -165,8 +165,18 @@ public class TranslogTests extends ElasticsearchTestCase { assertThat(translog.read(loc3).getSource().source.toBytesArray(), equalTo(new BytesArray(new byte[]{3}))); translog.sync(); assertThat(translog.read(loc3).getSource().source.toBytesArray(), equalTo(new BytesArray(new byte[]{3}))); - - + translog.prepareCommit(); + assertThat(translog.read(loc3).getSource().source.toBytesArray(), equalTo(new BytesArray(new byte[]{3}))); + translog.commit(); + assertNull(translog.read(loc1)); + assertNull(translog.read(loc2)); + assertNull(translog.read(loc3)); + try { + translog.read(new Translog.Location(translog.currentFileGeneration() + 1, 17, 35)); + fail("generation is greater than the current"); + } catch (IllegalStateException ex) { + // expected + } } @Test From 400abfceaf7b7b7bfd86f0703f1a92bf2ea09b40 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 18 May 2015 12:51:00 -0400 Subject: [PATCH 52/56] detect_noop now understands null as a valid value If the source contrains a null value for a field then detect_noop should consider setting it to null again to be a noop. Closes #11208 --- .../common/xcontent/XContentHelper.java | 6 ++- .../elasticsearch/update/UpdateNoopTests.java | 39 ++++++++++++++++--- 2 files changed, 38 insertions(+), 7 deletions(-) diff --git a/src/main/java/org/elasticsearch/common/xcontent/XContentHelper.java b/src/main/java/org/elasticsearch/common/xcontent/XContentHelper.java index e92582ed555..d196d459fbd 100644 --- a/src/main/java/org/elasticsearch/common/xcontent/XContentHelper.java +++ b/src/main/java/org/elasticsearch/common/xcontent/XContentHelper.java @@ -20,7 +20,9 @@ package org.elasticsearch.common.xcontent; import com.google.common.base.Charsets; +import com.google.common.base.Objects; import com.google.common.collect.Maps; + import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.common.bytes.BytesArray; @@ -260,11 +262,11 @@ public class XContentHelper { if (modified) { continue; } - if (!checkUpdatesAreUnequal || old == null) { + if (!checkUpdatesAreUnequal) { modified = true; continue; } - modified = !old.equals(changesEntry.getValue()); + modified = !Objects.equal(old, changesEntry.getValue()); } return modified; } diff --git a/src/test/java/org/elasticsearch/update/UpdateNoopTests.java b/src/test/java/org/elasticsearch/update/UpdateNoopTests.java index e5c1ee9625e..9f4f203b299 100644 --- a/src/test/java/org/elasticsearch/update/UpdateNoopTests.java +++ b/src/test/java/org/elasticsearch/update/UpdateNoopTests.java @@ -19,7 +19,6 @@ package org.elasticsearch.update; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.update.UpdateResponse; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; @@ -42,8 +41,11 @@ public class UpdateNoopTests extends ElasticsearchIntegrationTest { updateAndCheckSource(2, fields("bar", "bir")); updateAndCheckSource(2, fields("bar", "bir")); updateAndCheckSource(3, fields("bar", "foo")); + updateAndCheckSource(4, fields("bar", null)); + updateAndCheckSource(4, fields("bar", null)); + updateAndCheckSource(5, fields("bar", "foo")); - assertEquals(2, totalNoopUpdates()); + assertEquals(3, totalNoopUpdates()); } @Test @@ -51,13 +53,22 @@ public class UpdateNoopTests extends ElasticsearchIntegrationTest { // Use random keys so we get random iteration order. String key1 = 1 + randomAsciiOfLength(3); String key2 = 2 + randomAsciiOfLength(3); + String key3 = 3 + randomAsciiOfLength(3); updateAndCheckSource(1, fields(key1, "foo", key2, "baz")); updateAndCheckSource(1, fields(key1, "foo", key2, "baz")); updateAndCheckSource(2, fields(key1, "foo", key2, "bir")); updateAndCheckSource(2, fields(key1, "foo", key2, "bir")); updateAndCheckSource(3, fields(key1, "foo", key2, "foo")); + updateAndCheckSource(4, fields(key1, "foo", key2, null)); + updateAndCheckSource(4, fields(key1, "foo", key2, null)); + updateAndCheckSource(5, fields(key1, "foo", key2, "foo")); + updateAndCheckSource(6, fields(key1, null, key2, "foo")); + updateAndCheckSource(6, fields(key1, null, key2, "foo")); + updateAndCheckSource(7, fields(key1, null, key2, null)); + updateAndCheckSource(7, fields(key1, null, key2, null)); + updateAndCheckSource(8, fields(key1, null, key2, null, key3, null)); - assertEquals(2, totalNoopUpdates()); + assertEquals(5, totalNoopUpdates()); } @Test @@ -83,6 +94,7 @@ public class UpdateNoopTests extends ElasticsearchIntegrationTest { // Use random keys so we get variable iteration order. String key1 = 1 + randomAsciiOfLength(3); String key2 = 2 + randomAsciiOfLength(3); + String key3 = 3 + randomAsciiOfLength(3); updateAndCheckSource(1, XContentFactory.jsonBuilder().startObject() .startObject("test") .field(key1, "foo") @@ -108,8 +120,24 @@ public class UpdateNoopTests extends ElasticsearchIntegrationTest { .field(key1, "foo") .field(key2, "foo") .endObject().endObject()); + updateAndCheckSource(4, XContentFactory.jsonBuilder().startObject() + .startObject("test") + .field(key1, "foo") + .field(key2, (Object) null) + .endObject().endObject()); + updateAndCheckSource(4, XContentFactory.jsonBuilder().startObject() + .startObject("test") + .field(key1, "foo") + .field(key2, (Object) null) + .endObject().endObject()); + updateAndCheckSource(5, XContentFactory.jsonBuilder().startObject() + .startObject("test") + .field(key1, "foo") + .field(key2, (Object) null) + .field(key3, (Object) null) + .endObject().endObject()); - assertEquals(2, totalNoopUpdates()); + assertEquals(3, totalNoopUpdates()); } @Test @@ -199,7 +227,7 @@ public class UpdateNoopTests extends ElasticsearchIntegrationTest { private XContentBuilder fields(Object... fields) throws IOException { assertEquals("Fields must field1, value1, field2, value2, etc", 0, fields.length % 2); - + XContentBuilder builder = XContentFactory.jsonBuilder().startObject(); for (int i = 0; i < fields.length; i += 2) { builder.field((String) fields[i], fields[i + 1]); @@ -229,6 +257,7 @@ public class UpdateNoopTests extends ElasticsearchIntegrationTest { return client().admin().indices().prepareStats("test").setIndexing(true).get().getIndex("test").getTotal().getIndexing().getTotal() .getNoopUpdateCount(); } + @Before public void setup() { createIndex("test"); From e89693408fa69dba70cbd9c3e83b69c81d6defa2 Mon Sep 17 00:00:00 2001 From: Robert Muir Date: Mon, 18 May 2015 16:37:20 -0400 Subject: [PATCH 53/56] Ensure java.io.tmpdir is created earlier in tests. When testing, each jvm gets its own tmpdir set, so it may not exist at all. A Lucene test rule ensures its created, but some tests (I am looking at you rest tests) do a bunch of file stuff in static {}, in that case because its a parameterized test. And if you try to extend it, it will fail if security manager is disabled... Currently we ensure(java.io.tmpdir) very early when tests are running under security manager, but otherwise we don't and it won't happen until the test rule fires. So just do it early always. --- .../elasticsearch/bootstrap/BootstrapForTesting.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/src/test/java/org/elasticsearch/bootstrap/BootstrapForTesting.java b/src/test/java/org/elasticsearch/bootstrap/BootstrapForTesting.java index c0444eb81be..473330bc83c 100644 --- a/src/test/java/org/elasticsearch/bootstrap/BootstrapForTesting.java +++ b/src/test/java/org/elasticsearch/bootstrap/BootstrapForTesting.java @@ -50,6 +50,16 @@ public class BootstrapForTesting { static { // just like bootstrap, initialize natives, then SM Bootstrap.initializeNatives(true, true); + + // make sure java.io.tmpdir exists always (in case code uses it in a static initializer) + Path javaTmpDir = PathUtils.get(Objects.requireNonNull(System.getProperty("java.io.tmpdir"), + "please set ${java.io.tmpdir} in pom.xml")); + try { + Security.ensureDirectoryExists(javaTmpDir); + } catch (Exception e) { + throw new RuntimeException("unable to create test temp directory", e); + } + // install security manager if requested if (systemPropertyAsBoolean("tests.security.manager", false)) { try { @@ -67,8 +77,6 @@ public class BootstrapForTesting { "please set ${m2.repository} in pom.xml")); Security.addPath(perms, m2repoDir, "read,readlink"); // java.io.tmpdir - Path javaTmpDir = PathUtils.get(Objects.requireNonNull(System.getProperty("java.io.tmpdir"), - "please set ${java.io.tmpdir} in pom.xml")); Security.addPath(perms, javaTmpDir, "read,readlink,write,delete"); // custom test config file if (Strings.hasLength(System.getProperty("tests.config"))) { From 437910e8827b8171ea96ad56dad040c136ba05aa Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Tue, 19 May 2015 10:28:35 +0200 Subject: [PATCH 54/56] Upgrade to lucene-5.2.0-snapshot-1680200. --- pom.xml | 2 +- .../elasticsearch/common/lucene/IndexCacheableQueryTests.java | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 61966eb58a5..3a9a6e43cab 100644 --- a/pom.xml +++ b/pom.xml @@ -32,7 +32,7 @@ 5.2.0 - 1678978 + 1680200 5.2.0-snapshot-${lucene.snapshot.revision} 2.1.14 auto diff --git a/src/test/java/org/elasticsearch/common/lucene/IndexCacheableQueryTests.java b/src/test/java/org/elasticsearch/common/lucene/IndexCacheableQueryTests.java index d1aa163667a..cb72ec45c22 100644 --- a/src/test/java/org/elasticsearch/common/lucene/IndexCacheableQueryTests.java +++ b/src/test/java/org/elasticsearch/common/lucene/IndexCacheableQueryTests.java @@ -93,7 +93,6 @@ public class IndexCacheableQueryTests extends ElasticsearchTestCase { QueryUtils.checkUnequal(rewritten, rewritten2); } - @AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/LUCENE-6483") public void testCache() throws IOException { Directory dir = newDirectory(); LRUQueryCache cache = new LRUQueryCache(10000, Long.MAX_VALUE); From 17d65a5f90ae8f1b0a5e08f679b376a9d84484e5 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Tue, 19 May 2015 10:37:45 +0200 Subject: [PATCH 55/56] Tests: Remove RelocationTests.testRelocationWithBusyClusterUpdateThread. It is not relevant anymore with synchronous mapping updates. --- .../recovery/RelocationTests.java | 94 ------------------- 1 file changed, 94 deletions(-) diff --git a/src/test/java/org/elasticsearch/recovery/RelocationTests.java b/src/test/java/org/elasticsearch/recovery/RelocationTests.java index 3ba5ab9cee0..2d8c3881161 100644 --- a/src/test/java/org/elasticsearch/recovery/RelocationTests.java +++ b/src/test/java/org/elasticsearch/recovery/RelocationTests.java @@ -444,100 +444,6 @@ public class RelocationTests extends ElasticsearchIntegrationTest { assertTrue(stateResponse.getState().readOnlyRoutingNodes().node(blueNodeId).isEmpty()); } - @Test - @Slow - @TestLogging("cluster.service:TRACE,indices.recovery:TRACE") - @AwaitsFix(bugUrl="Fails now that we removed the mapping update from phase 2 in #11207") - public void testRelocationWithBusyClusterUpdateThread() throws Exception { - final String indexName = "test"; - final Settings settings = ImmutableSettings.builder() - .put("gateway.type", "local") - .put(DiscoverySettings.PUBLISH_TIMEOUT, "1s") - .put("indices.recovery.internal_action_timeout", "1s").build(); - String master = internalCluster().startNode(settings); - ensureGreen(); - List nodes = internalCluster().startNodesAsync(2, settings).get(); - final String node1 = nodes.get(0); - final String node2 = nodes.get(1); - ClusterHealthResponse response = client().admin().cluster().prepareHealth().setWaitForNodes("3").get(); - assertThat(response.isTimedOut(), is(false)); - - - client().admin().indices().prepareCreate(indexName) - .setSettings( - ImmutableSettings.builder() - .put(FilterAllocationDecider.INDEX_ROUTING_INCLUDE_GROUP + "_name", node1) - .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) - ).get(); - - - List requests = new ArrayList<>(); - int numDocs = scaledRandomIntBetween(25, 250); - for (int i = 0; i < numDocs; i++) { - requests.add(client().prepareIndex(indexName, "type").setCreate(true).setSource("{}")); - } - indexRandom(true, requests); - ensureSearchable(indexName); - - // capture the incoming state indicate that the replicas have upgraded and assigned - - final CountDownLatch allReplicasAssigned = new CountDownLatch(1); - final CountDownLatch releaseClusterState = new CountDownLatch(1); - final CountDownLatch unassignedShardsAfterReplicasAssigned = new CountDownLatch(1); - try { - internalCluster().getInstance(ClusterService.class, node1).addLast(new ClusterStateListener() { - @Override - public void clusterChanged(ClusterChangedEvent event) { - ClusterState state = event.state(); - if (state.routingTable().allShards().size() == 1 || state.routingNodes().hasUnassignedShards()) { - // we have no replicas or they are not assigned yet - return; - } - - allReplicasAssigned.countDown(); - try { - releaseClusterState.await(); - } catch (InterruptedException e) { - // - } - } - - }); - - internalCluster().getInstance(ClusterService.class, master).addLast(new ClusterStateListener() { - @Override - public void clusterChanged(ClusterChangedEvent event) { - if (event.state().routingNodes().hasUnassigned() && allReplicasAssigned.getCount() == 0) { - unassignedShardsAfterReplicasAssigned.countDown(); - } - } - }); - - logger.info("--> starting replica recovery"); - // we don't expect this to be acknowledge by node1 where we block the cluster state thread - assertFalse(client().admin().indices().prepareUpdateSettings(indexName) - .setSettings(ImmutableSettings.builder() - .put(FilterAllocationDecider.INDEX_ROUTING_INCLUDE_GROUP + "_name", node1 + "," + node2) - .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1) - - ).setTimeout("200ms") - .get().isAcknowledged()); - - logger.info("--> waiting for node1 to process replica existence"); - allReplicasAssigned.await(); - logger.info("--> waiting for recovery to fail"); - unassignedShardsAfterReplicasAssigned.await(); - } finally { - logger.info("--> releasing cluster state update thread"); - releaseClusterState.countDown(); - } - logger.info("--> waiting for recovery to succeed"); - // force a move. - client().admin().cluster().prepareReroute().get(); - ensureGreen(); - } - @Test @Slow public void testCancellationCleansTempFiles() throws Exception { From 84b24e130eb0d4bf950f81d6915fe23bbc33bde4 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Mon, 18 May 2015 01:18:19 +0200 Subject: [PATCH 56/56] inner hits: Fix search hit field and term resolution --- .../index/query/HasChildQueryParser.java | 2 +- .../index/query/HasParentQueryParser.java | 2 +- .../fetch/innerhits/InnerHitsContext.java | 45 +++++++---- .../innerhits/InnerHitsParseElement.java | 2 +- .../search/innerhits/InnerHitsTests.java | 75 +++++++++++++++++++ 5 files changed, 107 insertions(+), 19 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/query/HasChildQueryParser.java b/src/main/java/org/elasticsearch/index/query/HasChildQueryParser.java index 330c99c313e..52af7afe580 100644 --- a/src/main/java/org/elasticsearch/index/query/HasChildQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/HasChildQueryParser.java @@ -142,7 +142,7 @@ public class HasChildQueryParser implements QueryParser { } if (innerHits != null) { - InnerHitsContext.ParentChildInnerHits parentChildInnerHits = new InnerHitsContext.ParentChildInnerHits(innerHits.v2(), innerQuery, null, childDocMapper); + InnerHitsContext.ParentChildInnerHits parentChildInnerHits = new InnerHitsContext.ParentChildInnerHits(innerHits.v2(), innerQuery, null, parseContext.mapperService(), childDocMapper); String name = innerHits.v1() != null ? innerHits.v1() : childType; parseContext.addInnerHits(name, parentChildInnerHits); } diff --git a/src/main/java/org/elasticsearch/index/query/HasParentQueryParser.java b/src/main/java/org/elasticsearch/index/query/HasParentQueryParser.java index ae85d3230ad..4100269b4de 100644 --- a/src/main/java/org/elasticsearch/index/query/HasParentQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/HasParentQueryParser.java @@ -150,7 +150,7 @@ public class HasParentQueryParser implements QueryParser { } if (innerHits != null) { - InnerHitsContext.ParentChildInnerHits parentChildInnerHits = new InnerHitsContext.ParentChildInnerHits(innerHits.v2(), innerQuery, null, parentDocMapper); + InnerHitsContext.ParentChildInnerHits parentChildInnerHits = new InnerHitsContext.ParentChildInnerHits(innerHits.v2(), innerQuery, null, parseContext.mapperService(), parentDocMapper); String name = innerHits.v1() != null ? innerHits.v1() : parentType; parseContext.addInnerHits(name, parentChildInnerHits); } diff --git a/src/main/java/org/elasticsearch/search/fetch/innerhits/InnerHitsContext.java b/src/main/java/org/elasticsearch/search/fetch/innerhits/InnerHitsContext.java index 13b08dd49ad..bcdd56b5c47 100644 --- a/src/main/java/org/elasticsearch/search/fetch/innerhits/InnerHitsContext.java +++ b/src/main/java/org/elasticsearch/search/fetch/innerhits/InnerHitsContext.java @@ -31,7 +31,6 @@ import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.Filter; import org.apache.lucene.search.FilteredQuery; import org.apache.lucene.search.Query; -import org.apache.lucene.search.QueryWrapperFilter; import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TopDocsCollector; @@ -45,6 +44,7 @@ import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.index.fieldvisitor.SingleFieldsVisitor; import org.elasticsearch.index.mapper.DocumentMapper; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.mapper.internal.ParentFieldMapper; import org.elasticsearch.index.mapper.internal.UidFieldMapper; @@ -53,6 +53,7 @@ import org.elasticsearch.index.query.ParsedQuery; import org.elasticsearch.search.SearchHitField; import org.elasticsearch.search.fetch.FetchSubPhase; import org.elasticsearch.search.internal.FilteredSearchContext; +import org.elasticsearch.search.internal.InternalSearchHit; import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; @@ -276,23 +277,23 @@ public final class InnerHitsContext { public static final class ParentChildInnerHits extends BaseInnerHits { + private final MapperService mapperService; private final DocumentMapper documentMapper; - public ParentChildInnerHits(SearchContext context, Query query, Map childInnerHits, DocumentMapper documentMapper) { + public ParentChildInnerHits(SearchContext context, Query query, Map childInnerHits, MapperService mapperService, DocumentMapper documentMapper) { super(context, query, childInnerHits); + this.mapperService = mapperService; this.documentMapper = documentMapper; } @Override public TopDocs topDocs(SearchContext context, FetchSubPhase.HitContext hitContext) throws IOException { - final String term; final String field; - if (documentMapper.parentFieldMapper().active()) { - // Active _parent field has been selected, so we want a children doc as inner hits. + final String term; + if (isParentHit(hitContext.hit())) { field = ParentFieldMapper.NAME; term = Uid.createUid(hitContext.hit().type(), hitContext.hit().id()); - } else { - // No active _parent field has been selected, so we want parent docs as inner hits. + } else if (isChildHit(hitContext.hit())) { field = UidFieldMapper.NAME; SearchHitField parentField = hitContext.hit().field(ParentFieldMapper.NAME); if (parentField != null) { @@ -305,16 +306,19 @@ public final class InnerHitsContext { } term = (String) fieldsVisitor.fields().get(ParentFieldMapper.NAME).get(0); } - } - Filter filter = new QueryWrapperFilter(new TermQuery(new Term(field, term))); // Only include docs that have the current hit as parent - Query typeFilter = documentMapper.typeFilter(); // Only include docs that have this inner hits type. - BooleanQuery filteredQuery = new BooleanQuery(); - filteredQuery.add(query, Occur.MUST); - filteredQuery.add(filter, Occur.FILTER); - filteredQuery.add(typeFilter, Occur.FILTER); + } else { + return Lucene.EMPTY_TOP_DOCS; + } + + BooleanQuery q = new BooleanQuery(); + q.add(query, Occur.MUST); + // Only include docs that have the current hit as parent + q.add(new TermQuery(new Term(field, term)), Occur.MUST); + // Only include docs that have this inner hits type + q.add(documentMapper.typeFilter(), Occur.MUST); if (size() == 0) { - final int count = context.searcher().count(filteredQuery); + final int count = context.searcher().count(q); return new TopDocs(count, Lucene.EMPTY_SCORE_DOCS, 0); } else { int topN = from() + size(); @@ -324,9 +328,18 @@ public final class InnerHitsContext { } else { topDocsCollector = TopScoreDocCollector.create(topN); } - context.searcher().search( filteredQuery, topDocsCollector); + context.searcher().search( q, topDocsCollector); return topDocsCollector.topDocs(from(), size()); } } + + private boolean isParentHit(InternalSearchHit hit) { + return hit.type().equals(documentMapper.parentFieldMapper().type()); + } + + private boolean isChildHit(InternalSearchHit hit) { + DocumentMapper hitDocumentMapper = mapperService.documentMapper(hit.type()); + return documentMapper.type().equals(hitDocumentMapper.parentFieldMapper().type()); + } } } diff --git a/src/main/java/org/elasticsearch/search/fetch/innerhits/InnerHitsParseElement.java b/src/main/java/org/elasticsearch/search/fetch/innerhits/InnerHitsParseElement.java index 2642b7d862a..65d6a6057fe 100644 --- a/src/main/java/org/elasticsearch/search/fetch/innerhits/InnerHitsParseElement.java +++ b/src/main/java/org/elasticsearch/search/fetch/innerhits/InnerHitsParseElement.java @@ -150,7 +150,7 @@ public class InnerHitsParseElement implements SearchParseElement { if (documentMapper == null) { throw new IllegalArgumentException("type [" + type + "] doesn't exist"); } - return new InnerHitsContext.ParentChildInnerHits(parseResult.context(), parseResult.query(), parseResult.childInnerHits(), documentMapper); + return new InnerHitsContext.ParentChildInnerHits(parseResult.context(), parseResult.query(), parseResult.childInnerHits(), parseContext.mapperService(), documentMapper); } private InnerHitsContext.NestedInnerHits parseNested(XContentParser parser, QueryParseContext parseContext, SearchContext searchContext, String nestedPath) throws Exception { diff --git a/src/test/java/org/elasticsearch/search/innerhits/InnerHitsTests.java b/src/test/java/org/elasticsearch/search/innerhits/InnerHitsTests.java index 6974f0b7c03..e2da5cf1333 100644 --- a/src/test/java/org/elasticsearch/search/innerhits/InnerHitsTests.java +++ b/src/test/java/org/elasticsearch/search/innerhits/InnerHitsTests.java @@ -26,10 +26,12 @@ import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.index.query.BoolQueryBuilder; +import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.support.QueryInnerHitBuilder; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.fetch.innerhits.InnerHitsBuilder; +import org.elasticsearch.search.sort.SortBuilders; import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.junit.Test; @@ -921,4 +923,77 @@ public class InnerHitsTests extends ElasticsearchIntegrationTest { assertThat(response.getHits().getAt(0).getInnerHits().get("comments.messages").getAt(0).getNestedIdentity().getChild(), nullValue()); } + @Test + public void testRoyals() throws Exception { + assertAcked( + prepareCreate("royals") + .addMapping("king") + .addMapping("prince", "_parent", "type=king") + .addMapping("duke", "_parent", "type=prince") + .addMapping("earl", "_parent", "type=duke") + .addMapping("baron", "_parent", "type=earl") + ); + + List requests = new ArrayList<>(); + requests.add(client().prepareIndex("royals", "king", "king").setSource("{}")); + requests.add(client().prepareIndex("royals", "prince", "prince").setParent("king").setSource("{}")); + requests.add(client().prepareIndex("royals", "duke", "duke").setParent("prince").setRouting("king").setSource("{}")); + requests.add(client().prepareIndex("royals", "earl", "earl1").setParent("duke").setRouting("king").setSource("{}")); + requests.add(client().prepareIndex("royals", "earl", "earl2").setParent("duke").setRouting("king").setSource("{}")); + requests.add(client().prepareIndex("royals", "earl", "earl3").setParent("duke").setRouting("king").setSource("{}")); + requests.add(client().prepareIndex("royals", "earl", "earl4").setParent("duke").setRouting("king").setSource("{}")); + requests.add(client().prepareIndex("royals", "baron", "baron1").setParent("earl1").setRouting("king").setSource("{}")); + requests.add(client().prepareIndex("royals", "baron", "baron2").setParent("earl2").setRouting("king").setSource("{}")); + requests.add(client().prepareIndex("royals", "baron", "baron3").setParent("earl3").setRouting("king").setSource("{}")); + requests.add(client().prepareIndex("royals", "baron", "baron4").setParent("earl4").setRouting("king").setSource("{}")); + indexRandom(true, requests); + + SearchResponse response = client().prepareSearch("royals") + .setTypes("duke") + .addInnerHit("earls", new InnerHitsBuilder.InnerHit() + .setType("earl") + .addSort(SortBuilders.fieldSort("_uid").order(SortOrder.ASC)) + .setSize(4) + .addInnerHit("barons", new InnerHitsBuilder.InnerHit().setType("baron")) + ) + .addInnerHit("princes", + new InnerHitsBuilder.InnerHit().setType("prince") + .addInnerHit("kings", new InnerHitsBuilder.InnerHit().setType("king")) + ) + .get(); + assertHitCount(response, 1); + assertThat(response.getHits().getAt(0).getId(), equalTo("duke")); + + SearchHits innerHits = response.getHits().getAt(0).getInnerHits().get("earls"); + assertThat(innerHits.getTotalHits(), equalTo(4l)); + assertThat(innerHits.getAt(0).getId(), equalTo("earl1")); + assertThat(innerHits.getAt(1).getId(), equalTo("earl2")); + assertThat(innerHits.getAt(2).getId(), equalTo("earl3")); + assertThat(innerHits.getAt(3).getId(), equalTo("earl4")); + + SearchHits innerInnerHits = innerHits.getAt(0).getInnerHits().get("barons"); + assertThat(innerInnerHits.totalHits(), equalTo(1l)); + assertThat(innerInnerHits.getAt(0).getId(), equalTo("baron1")); + + innerInnerHits = innerHits.getAt(1).getInnerHits().get("barons"); + assertThat(innerInnerHits.totalHits(), equalTo(1l)); + assertThat(innerInnerHits.getAt(0).getId(), equalTo("baron2")); + + innerInnerHits = innerHits.getAt(2).getInnerHits().get("barons"); + assertThat(innerInnerHits.totalHits(), equalTo(1l)); + assertThat(innerInnerHits.getAt(0).getId(), equalTo("baron3")); + + innerInnerHits = innerHits.getAt(3).getInnerHits().get("barons"); + assertThat(innerInnerHits.totalHits(), equalTo(1l)); + assertThat(innerInnerHits.getAt(0).getId(), equalTo("baron4")); + + innerHits = response.getHits().getAt(0).getInnerHits().get("princes"); + assertThat(innerHits.getTotalHits(), equalTo(1l)); + assertThat(innerHits.getAt(0).getId(), equalTo("prince")); + + innerInnerHits = innerHits.getAt(0).getInnerHits().get("kings"); + assertThat(innerInnerHits.totalHits(), equalTo(1l)); + assertThat(innerInnerHits.getAt(0).getId(), equalTo("king")); + } + }