From 1b1f484c2840d42b2da3a6fdd8c773d1c98f7508 Mon Sep 17 00:00:00 2001 From: Areek Zillur Date: Thu, 13 Oct 2016 14:37:08 -0400 Subject: [PATCH 001/132] Distinguish primary and replica request types in TransportWriteAction --- .../action/bulk/TransportShardBulkAction.java | 4 ++-- .../action/delete/TransportDeleteAction.java | 4 ++-- .../action/index/TransportIndexAction.java | 4 ++-- .../support/replication/TransportWriteAction.java | 15 ++++++++------- .../replication/TransportWriteActionTests.java | 4 ++-- 5 files changed, 16 insertions(+), 15 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index c7cfd6330cb..15e6a5efa31 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -66,7 +66,7 @@ import static org.elasticsearch.action.support.replication.ReplicationOperation. import static org.elasticsearch.action.support.replication.ReplicationOperation.isConflictException; /** Performs shard-level bulk (index, delete or update) operations */ -public class TransportShardBulkAction extends TransportWriteAction { +public class TransportShardBulkAction extends TransportWriteAction { public static final String ACTION_NAME = BulkAction.NAME + "[s]"; @@ -80,7 +80,7 @@ public class TransportShardBulkAction extends TransportWriteAction { +public class TransportDeleteAction extends TransportWriteAction { private final AutoCreateIndex autoCreateIndex; private final TransportCreateIndexAction createIndexAction; @@ -61,7 +61,7 @@ public class TransportDeleteAction extends TransportWriteActionallowIdGeneration: If the id is set not, should it be generated. Defaults to true. * */ -public class TransportIndexAction extends TransportWriteAction { +public class TransportIndexAction extends TransportWriteAction { private final AutoCreateIndex autoCreateIndex; private final boolean allowIdGeneration; @@ -76,7 +76,7 @@ public class TransportIndexAction extends TransportWriteAction, + ReplicaRequest extends ReplicatedWriteRequest, Response extends ReplicationResponse & WriteResponse - > extends TransportReplicationAction { + > extends TransportReplicationAction { protected TransportWriteAction(Settings settings, String actionName, TransportService transportService, ClusterService clusterService, IndicesService indicesService, ThreadPool threadPool, ShardStateAction shardStateAction, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, Supplier request, - String executor) { + Supplier replicaRequest, String executor) { super(settings, actionName, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters, - indexNameExpressionResolver, request, request, executor); + indexNameExpressionResolver, request, replicaRequest, executor); } /** @@ -68,16 +69,16 @@ public abstract class TransportWriteAction< * * @return the translog location of the {@linkplain IndexShard} after the write was completed or null if no write occurred */ - protected abstract Translog.Location onReplicaShard(Request request, IndexShard indexShard); + protected abstract Translog.Location onReplicaShard(ReplicaRequest request, IndexShard indexShard); @Override protected final WritePrimaryResult shardOperationOnPrimary(Request request, IndexShard primary) throws Exception { WriteResult result = onPrimaryShard(request, primary); - return new WritePrimaryResult(request, result.getResponse(), result.getLocation(), primary); + return new WritePrimaryResult(((ReplicaRequest) request), result.getResponse(), result.getLocation(), primary); } @Override - protected final WriteReplicaResult shardOperationOnReplica(Request request, IndexShard replica) { + protected final WriteReplicaResult shardOperationOnReplica(ReplicaRequest request, IndexShard replica) { Translog.Location location = onReplicaShard(request, replica); return new WriteReplicaResult(replica, request, location); } @@ -110,7 +111,7 @@ public abstract class TransportWriteAction< boolean finishedAsyncActions; ActionListener listener = null; - public WritePrimaryResult(Request request, Response finalResponse, + public WritePrimaryResult(ReplicaRequest request, Response finalResponse, @Nullable Translog.Location location, IndexShard indexShard) { super(request, finalResponse); diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java b/core/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java index 14afe4dee9b..2a1471fa746 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java @@ -128,12 +128,12 @@ public class TransportWriteActionTests extends ESTestCase { resultChecker.accept(listener.response, forcedRefresh); } - private class TestAction extends TransportWriteAction { + private class TestAction extends TransportWriteAction { protected TestAction() { super(Settings.EMPTY, "test", new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR, null), null, null, null, null, new ActionFilters(new HashSet<>()), new IndexNameExpressionResolver(Settings.EMPTY), TestRequest::new, - ThreadPool.Names.SAME); + TestRequest::new, ThreadPool.Names.SAME); } @Override From 415fdee8287a891b005de32c613ed0ba6badb774 Mon Sep 17 00:00:00 2001 From: Areek Zillur Date: Thu, 13 Oct 2016 20:02:36 -0400 Subject: [PATCH 002/132] Distinguish environment failures from transient operation failures for write operations Currently, we treat all write operation exceptions as equals, but in reality every write operation can cause either an environment failure (i.e. a failure that should fail the engine e.g. data corruption, lucene tragic events) or operation failure (i.e. a failure that is transient w.r.t the operation e.g. parsing exception). This change bubbles up enironment failures from the engine, after failing the engine but captures transient operation failures as part of the operation to be processed appopriately at the transport level. --- .../elasticsearch/ElasticsearchException.java | 9 +- .../engine/DeleteFailedEngineException.java | 36 --- .../elasticsearch/index/engine/Engine.java | 46 +++- .../index/engine/InternalEngine.java | 43 ++-- ...va => OperationFailedEngineException.java} | 8 +- .../index/engine/ShadowEngine.java | 4 +- .../elasticsearch/index/shard/IndexShard.java | 13 +- .../ExceptionSerializationTests.java | 10 +- .../index/engine/InternalEngineTests.java | 211 ++++++------------ .../index/shard/IndexShardTests.java | 2 +- .../indices/stats/IndexStatsIT.java | 4 +- 11 files changed, 168 insertions(+), 218 deletions(-) delete mode 100644 core/src/main/java/org/elasticsearch/index/engine/DeleteFailedEngineException.java rename core/src/main/java/org/elasticsearch/index/engine/{IndexFailedEngineException.java => OperationFailedEngineException.java} (82%) diff --git a/core/src/main/java/org/elasticsearch/ElasticsearchException.java b/core/src/main/java/org/elasticsearch/ElasticsearchException.java index 63161a0a187..0c6125c52af 100644 --- a/core/src/main/java/org/elasticsearch/ElasticsearchException.java +++ b/core/src/main/java/org/elasticsearch/ElasticsearchException.java @@ -19,7 +19,6 @@ package org.elasticsearch; -import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.action.support.replication.ReplicationOperation; import org.elasticsearch.cluster.action.shard.ShardStateAction; import org.elasticsearch.common.io.stream.StreamInput; @@ -29,6 +28,7 @@ import org.elasticsearch.common.logging.LoggerMessageFormat; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.index.Index; +import org.elasticsearch.index.engine.OperationFailedEngineException; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.transport.TcpTransport; @@ -487,8 +487,7 @@ public class ElasticsearchException extends RuntimeException implements ToXConte org.elasticsearch.index.shard.TranslogRecoveryPerformer.BatchOperationException::new, 26), SNAPSHOT_CREATION_EXCEPTION(org.elasticsearch.snapshots.SnapshotCreationException.class, org.elasticsearch.snapshots.SnapshotCreationException::new, 27), - DELETE_FAILED_ENGINE_EXCEPTION(org.elasticsearch.index.engine.DeleteFailedEngineException.class, - org.elasticsearch.index.engine.DeleteFailedEngineException::new, 28), + // 28 was DeleteFailedEngineException DOCUMENT_MISSING_EXCEPTION(org.elasticsearch.index.engine.DocumentMissingException.class, org.elasticsearch.index.engine.DocumentMissingException::new, 29), SNAPSHOT_EXCEPTION(org.elasticsearch.snapshots.SnapshotException.class, @@ -581,8 +580,8 @@ public class ElasticsearchException extends RuntimeException implements ToXConte org.elasticsearch.action.TimestampParsingException::new, 78), ROUTING_MISSING_EXCEPTION(org.elasticsearch.action.RoutingMissingException.class, org.elasticsearch.action.RoutingMissingException::new, 79), - INDEX_FAILED_ENGINE_EXCEPTION(org.elasticsearch.index.engine.IndexFailedEngineException.class, - org.elasticsearch.index.engine.IndexFailedEngineException::new, 80), + OPERATION_FAILED_ENGINE_EXCEPTION(OperationFailedEngineException.class, + OperationFailedEngineException::new, 80), INDEX_SHARD_RESTORE_FAILED_EXCEPTION(org.elasticsearch.index.snapshots.IndexShardRestoreFailedException.class, org.elasticsearch.index.snapshots.IndexShardRestoreFailedException::new, 81), REPOSITORY_EXCEPTION(org.elasticsearch.repositories.RepositoryException.class, diff --git a/core/src/main/java/org/elasticsearch/index/engine/DeleteFailedEngineException.java b/core/src/main/java/org/elasticsearch/index/engine/DeleteFailedEngineException.java deleted file mode 100644 index f3f806fb831..00000000000 --- a/core/src/main/java/org/elasticsearch/index/engine/DeleteFailedEngineException.java +++ /dev/null @@ -1,36 +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.engine; - -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.index.shard.ShardId; - -import java.io.IOException; - -public class DeleteFailedEngineException extends EngineException { - - public DeleteFailedEngineException(ShardId shardId, Engine.Delete delete, Throwable cause) { - super(shardId, "Delete failed for [" + delete.uid().text() + "]", cause); - } - - public DeleteFailedEngineException(StreamInput in) throws IOException{ - super(in); - } -} \ No newline at end of file diff --git a/core/src/main/java/org/elasticsearch/index/engine/Engine.java b/core/src/main/java/org/elasticsearch/index/engine/Engine.java index 17822a71332..8b4150dc1b4 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -77,6 +77,7 @@ import java.util.Base64; import java.util.Comparator; import java.util.HashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Objects; import java.util.concurrent.TimeUnit; @@ -277,9 +278,9 @@ public abstract class Engine implements Closeable { } } - public abstract void index(Index operation) throws EngineException; + public abstract void index(Index operation) throws OperationFailedEngineException; - public abstract void delete(Delete delete) throws EngineException; + public abstract void delete(Delete delete) throws OperationFailedEngineException; /** * Attempts to do a special commit where the given syncID is put into the commit data. The attempt @@ -767,11 +768,28 @@ public abstract class Engine implements Closeable { } public abstract static class Operation { + + /** type of operation (index, delete), subclasses use static types */ + public enum TYPE { + INDEX, DELETE; + + private final String lowercase; + + TYPE() { + this.lowercase = this.toString().toLowerCase(Locale.ROOT); + } + + public String getLowercase() { + return lowercase; + } + } + private final Term uid; private long version; private final VersionType versionType; private final Origin origin; private Translog.Location location; + private Exception failure; private final long startTime; private long endTime; @@ -818,6 +836,18 @@ public abstract class Engine implements Closeable { return this.location; } + public Exception getFailure() { + return failure; + } + + public void setFailure(Exception failure) { + this.failure = failure; + } + + public boolean hasFailure() { + return failure != null; + } + public int sizeInBytes() { if (location != null) { return location.size; @@ -853,6 +883,8 @@ public abstract class Engine implements Closeable { abstract String type(); abstract String id(); + + abstract TYPE operationType(); } public static class Index extends Operation { @@ -892,6 +924,11 @@ public abstract class Engine implements Closeable { return this.doc.id(); } + @Override + TYPE operationType() { + return TYPE.INDEX; + } + public String routing() { return this.doc.routing(); } @@ -985,6 +1022,11 @@ public abstract class Engine implements Closeable { return this.id; } + @Override + TYPE operationType() { + return TYPE.DELETE; + } + public void updateVersion(long version, boolean found) { updateVersion(version); this.found = found; diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 6336318d1e2..1a88a82e8a2 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -397,7 +397,7 @@ public class InternalEngine extends Engine { } @Override - public void index(Index index) { + public void index(Index index) throws OperationFailedEngineException { try (ReleasableLock lock = readLock.acquire()) { ensureOpen(); if (index.origin().isRecovery()) { @@ -408,13 +408,31 @@ public class InternalEngine extends Engine { innerIndex(index); } } - } catch (IllegalStateException | IOException e) { - try { - maybeFailEngine("index", e); - } catch (Exception inner) { - e.addSuppressed(inner); + } catch (Exception e) { + handleOperationFailure(index, e); + } + } + + /** + * When indexing a document into Lucene, Lucene distinguishes between environment related errors + * (like out of disk space) and document specific errors (like analysis chain problems) by setting + * the IndexWriter.getTragicEvent() value for the former. maybeFailEngine checks for these kind of + * errors and returns true if that is the case. We use that to indicate a document level failure + * and set the error in operation.setFailure. In case of environment related errors, the failure + * is bubbled up + */ + private void handleOperationFailure(final Operation operation, final Exception e) throws OperationFailedEngineException { + try { + if (maybeFailEngine(operation.operationType().getLowercase(), e)) { + throw new OperationFailedEngineException(shardId, + operation.operationType().getLowercase(), operation.type(), operation.id(), e); + } else { + operation.setFailure(e); } - throw new IndexFailedEngineException(shardId, index.type(), index.id(), e); + } catch (Exception inner) { + e.addSuppressed(inner); + throw new OperationFailedEngineException(shardId, + operation.operationType().getLowercase(), operation.type(), operation.id(), e); } } @@ -545,18 +563,13 @@ public class InternalEngine extends Engine { } @Override - public void delete(Delete delete) throws EngineException { + public void delete(Delete delete) throws OperationFailedEngineException { try (ReleasableLock lock = readLock.acquire()) { ensureOpen(); // NOTE: we don't throttle this when merges fall behind because delete-by-id does not create new segments: innerDelete(delete); - } catch (IllegalStateException | IOException e) { - try { - maybeFailEngine("delete", e); - } catch (Exception inner) { - e.addSuppressed(inner); - } - throw new DeleteFailedEngineException(shardId, delete, e); + } catch (Exception e) { + handleOperationFailure(delete, e); } maybePruneDeletedTombstones(); diff --git a/core/src/main/java/org/elasticsearch/index/engine/IndexFailedEngineException.java b/core/src/main/java/org/elasticsearch/index/engine/OperationFailedEngineException.java similarity index 82% rename from core/src/main/java/org/elasticsearch/index/engine/IndexFailedEngineException.java rename to core/src/main/java/org/elasticsearch/index/engine/OperationFailedEngineException.java index bacc786c7dc..1c080f2fc18 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/IndexFailedEngineException.java +++ b/core/src/main/java/org/elasticsearch/index/engine/OperationFailedEngineException.java @@ -26,21 +26,21 @@ import org.elasticsearch.index.shard.ShardId; import java.io.IOException; import java.util.Objects; -public class IndexFailedEngineException extends EngineException { +public class OperationFailedEngineException extends EngineException { private final String type; private final String id; - public IndexFailedEngineException(ShardId shardId, String type, String id, Throwable cause) { - super(shardId, "Index failed for [" + type + "#" + id + "]", cause); + public OperationFailedEngineException(ShardId shardId, String operationType, String type, String id, Throwable cause) { + super(shardId, operationType + " failed for [" + type + "#" + id + "]", cause); Objects.requireNonNull(type, "type must not be null"); Objects.requireNonNull(id, "id must not be null"); this.type = type; this.id = id; } - public IndexFailedEngineException(StreamInput in) throws IOException{ + public OperationFailedEngineException(StreamInput in) throws IOException{ super(in); type = in.readString(); id = in.readString(); diff --git a/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java b/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java index 3aafcaff748..e9cb2fb62af 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java @@ -106,12 +106,12 @@ public class ShadowEngine extends Engine { @Override - public void index(Index index) throws EngineException { + public void index(Index index) throws OperationFailedEngineException { throw new UnsupportedOperationException(shardId + " index operation not allowed on shadow engine"); } @Override - public void delete(Delete delete) throws EngineException { + public void delete(Delete delete) throws OperationFailedEngineException { throw new UnsupportedOperationException(shardId + " delete operation not allowed on shadow engine"); } diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 1eb21d55e22..162d3495c3e 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -555,7 +555,11 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl indexingOperationListeners.postIndex(index, e); throw e; } - indexingOperationListeners.postIndex(index, index.isCreated()); + if (index.hasFailure()) { + indexingOperationListeners.postIndex(index, index.getFailure()); + } else { + indexingOperationListeners.postIndex(index, index.isCreated()); + } } public Engine.Delete prepareDeleteOnPrimary(String type, String id, long version, VersionType versionType) { @@ -599,8 +603,11 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl indexingOperationListeners.postDelete(delete, e); throw e; } - - indexingOperationListeners.postDelete(delete); + if (delete.hasFailure()) { + indexingOperationListeners.postDelete(delete, delete.getFailure()); + } else { + indexingOperationListeners.postDelete(delete); + } } public Engine.GetResult get(Engine.Get get) { diff --git a/core/src/test/java/org/elasticsearch/ExceptionSerializationTests.java b/core/src/test/java/org/elasticsearch/ExceptionSerializationTests.java index 0eff8d7e60c..61227c3e366 100644 --- a/core/src/test/java/org/elasticsearch/ExceptionSerializationTests.java +++ b/core/src/test/java/org/elasticsearch/ExceptionSerializationTests.java @@ -51,7 +51,7 @@ import org.elasticsearch.common.xcontent.XContentLocation; import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.index.AlreadyExpiredException; import org.elasticsearch.index.Index; -import org.elasticsearch.index.engine.IndexFailedEngineException; +import org.elasticsearch.index.engine.OperationFailedEngineException; import org.elasticsearch.index.engine.RecoveryEngineException; import org.elasticsearch.index.query.QueryShardException; import org.elasticsearch.index.shard.IllegalIndexShardStateException; @@ -402,13 +402,13 @@ public class ExceptionSerializationTests extends ESTestCase { public void testIndexFailedEngineException() throws IOException { ShardId id = new ShardId("foo", "_na_", 1); - IndexFailedEngineException ex = serialize(new IndexFailedEngineException(id, "type", "id", null)); + OperationFailedEngineException ex = serialize(new OperationFailedEngineException(id, "index", "type", "id", null)); assertEquals(ex.getShardId(), new ShardId("foo", "_na_", 1)); assertEquals("type", ex.type()); assertEquals("id", ex.id()); assertNull(ex.getCause()); - ex = serialize(new IndexFailedEngineException(null, "type", "id", new NullPointerException())); + ex = serialize(new OperationFailedEngineException(null, "index", "type", "id", new NullPointerException())); assertNull(ex.getShardId()); assertEquals("type", ex.type()); assertEquals("id", ex.id()); @@ -680,7 +680,7 @@ public class ExceptionSerializationTests extends ESTestCase { ids.put(25, org.elasticsearch.script.GeneralScriptException.class); ids.put(26, org.elasticsearch.index.shard.TranslogRecoveryPerformer.BatchOperationException.class); ids.put(27, org.elasticsearch.snapshots.SnapshotCreationException.class); - ids.put(28, org.elasticsearch.index.engine.DeleteFailedEngineException.class); + ids.put(28, null); // was DeleteFailedEngineException ids.put(29, org.elasticsearch.index.engine.DocumentMissingException.class); ids.put(30, org.elasticsearch.snapshots.SnapshotException.class); ids.put(31, org.elasticsearch.indices.InvalidAliasNameException.class); @@ -732,7 +732,7 @@ public class ExceptionSerializationTests extends ESTestCase { ids.put(77, org.elasticsearch.common.util.concurrent.UncategorizedExecutionException.class); ids.put(78, org.elasticsearch.action.TimestampParsingException.class); ids.put(79, org.elasticsearch.action.RoutingMissingException.class); - ids.put(80, org.elasticsearch.index.engine.IndexFailedEngineException.class); + ids.put(80, OperationFailedEngineException.class); ids.put(81, org.elasticsearch.index.snapshots.IndexShardRestoreFailedException.class); ids.put(82, org.elasticsearch.repositories.RepositoryException.class); ids.put(83, org.elasticsearch.transport.ReceiveTimeoutTransportException.class); diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 0f64e343d98..5d010a7cfbe 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -68,11 +68,9 @@ import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.BigArrays; -import org.elasticsearch.env.Environment; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.VersionType; -import org.elasticsearch.index.analysis.AnalysisRegistry; import org.elasticsearch.index.analysis.AnalyzerScope; import org.elasticsearch.index.analysis.IndexAnalyzers; import org.elasticsearch.index.analysis.NamedAnalyzer; @@ -135,6 +133,7 @@ import java.util.concurrent.atomic.AtomicReference; import static java.util.Collections.emptyMap; import static org.elasticsearch.index.engine.Engine.Operation.Origin.PRIMARY; import static org.elasticsearch.index.engine.Engine.Operation.Origin.REPLICA; +import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.everyItem; import static org.hamcrest.Matchers.greaterThan; @@ -1080,21 +1079,15 @@ public class InternalEngineTests extends ESTestCase { assertThat(index.version(), equalTo(2L)); index = new Engine.Index(newUid("1"), doc, 1L, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, 0, -1, false); - try { - engine.index(index); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } + engine.index(index); + assertTrue(index.hasFailure()); + assertThat(index.getFailure(), instanceOf(VersionConflictEngineException.class)); // future versions should not work as well index = new Engine.Index(newUid("1"), doc, 3L, VersionType.INTERNAL, PRIMARY, 0, -1, false); - try { - engine.index(index); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } + engine.index(index); + assertTrue(index.hasFailure()); + assertThat(index.getFailure(), instanceOf(VersionConflictEngineException.class)); } public void testExternalVersioningIndexConflict() { @@ -1108,12 +1101,9 @@ public class InternalEngineTests extends ESTestCase { assertThat(index.version(), equalTo(14L)); index = new Engine.Index(newUid("1"), doc, 13, VersionType.EXTERNAL, PRIMARY, 0, -1, false); - try { - engine.index(index); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } + engine.index(index); + assertTrue(index.hasFailure()); + assertThat(index.getFailure(), instanceOf(VersionConflictEngineException.class)); } public void testVersioningIndexConflictWithFlush() { @@ -1129,21 +1119,15 @@ public class InternalEngineTests extends ESTestCase { engine.flush(); index = new Engine.Index(newUid("1"), doc, 1L, VersionType.INTERNAL, PRIMARY, 0, -1, false); - try { - engine.index(index); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } + engine.index(index); + assertTrue(index.hasFailure()); + assertThat(index.getFailure(), instanceOf(VersionConflictEngineException.class)); // future versions should not work as well index = new Engine.Index(newUid("1"), doc, 3L, VersionType.INTERNAL, PRIMARY, 0, -1, false); - try { - engine.index(index); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } + engine.index(index); + assertTrue(index.hasFailure()); + assertThat(index.getFailure(), instanceOf(VersionConflictEngineException.class)); } public void testExternalVersioningIndexConflictWithFlush() { @@ -1159,12 +1143,9 @@ public class InternalEngineTests extends ESTestCase { engine.flush(); index = new Engine.Index(newUid("1"), doc, 13, VersionType.EXTERNAL, PRIMARY, 0, -1, false); - try { - engine.index(index); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } + engine.index(index); + assertTrue(index.hasFailure()); + assertThat(index.getFailure(), instanceOf(VersionConflictEngineException.class)); } public void testForceMerge() throws IOException { @@ -1273,21 +1254,15 @@ public class InternalEngineTests extends ESTestCase { assertThat(index.version(), equalTo(2L)); Engine.Delete delete = new Engine.Delete("test", "1", newUid("1"), 1L, VersionType.INTERNAL, PRIMARY, 0, false); - try { - engine.delete(delete); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } + engine.delete(delete); + assertTrue(delete.hasFailure()); + assertThat(delete.getFailure(), instanceOf(VersionConflictEngineException.class)); // future versions should not work as well delete = new Engine.Delete("test", "1", newUid("1"), 3L, VersionType.INTERNAL, PRIMARY, 0, false); - try { - engine.delete(delete); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } + engine.delete(delete); + assertTrue(delete.hasFailure()); + assertThat(delete.getFailure(), instanceOf(VersionConflictEngineException.class)); // now actually delete delete = new Engine.Delete("test", "1", newUid("1"), 2L, VersionType.INTERNAL, PRIMARY, 0, false); @@ -1296,20 +1271,9 @@ public class InternalEngineTests extends ESTestCase { // now check if we can index to a delete doc with version index = new Engine.Index(newUid("1"), doc, 2L, VersionType.INTERNAL, PRIMARY, 0, -1, false); - try { - engine.index(index); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } - - // we shouldn't be able to create as well - Engine.Index create = new Engine.Index(newUid("1"), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false); - try { - engine.index(create); - } catch (VersionConflictEngineException e) { - // all is well - } + engine.index(index); + assertTrue(index.hasFailure()); + assertThat(index.getFailure(), instanceOf(VersionConflictEngineException.class)); } public void testVersioningDeleteConflictWithFlush() { @@ -1325,21 +1289,15 @@ public class InternalEngineTests extends ESTestCase { engine.flush(); Engine.Delete delete = new Engine.Delete("test", "1", newUid("1"), 1L, VersionType.INTERNAL, PRIMARY, 0, false); - try { - engine.delete(delete); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } + engine.delete(delete); + assertTrue(delete.hasFailure()); + assertThat(delete.getFailure(), instanceOf(VersionConflictEngineException.class)); // future versions should not work as well delete = new Engine.Delete("test", "1", newUid("1"), 3L, VersionType.INTERNAL, PRIMARY, 0, false); - try { - engine.delete(delete); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } + engine.delete(delete); + assertTrue(delete.hasFailure()); + assertThat(delete.getFailure(), instanceOf(VersionConflictEngineException.class)); engine.flush(); @@ -1352,20 +1310,9 @@ public class InternalEngineTests extends ESTestCase { // now check if we can index to a delete doc with version index = new Engine.Index(newUid("1"), doc, 2L, VersionType.INTERNAL, PRIMARY, 0, -1, false); - try { - engine.index(index); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } - - // we shouldn't be able to create as well - Engine.Index create = new Engine.Index(newUid("1"), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false); - try { - engine.index(create); - } catch (VersionConflictEngineException e) { - // all is well - } + engine.index(index); + assertTrue(index.hasFailure()); + assertThat(index.getFailure(), instanceOf(VersionConflictEngineException.class)); } public void testVersioningCreateExistsException() { @@ -1375,12 +1322,9 @@ public class InternalEngineTests extends ESTestCase { assertThat(create.version(), equalTo(1L)); create = new Engine.Index(newUid("1"), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false); - try { - engine.index(create); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } + engine.index(create); + assertTrue(create.hasFailure()); + assertThat(create.getFailure(), instanceOf(VersionConflictEngineException.class)); } public void testVersioningCreateExistsExceptionWithFlush() { @@ -1392,12 +1336,9 @@ public class InternalEngineTests extends ESTestCase { engine.flush(); create = new Engine.Index(newUid("1"), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false); - try { - engine.index(create); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } + engine.index(create); + assertTrue(create.hasFailure()); + assertThat(create.getFailure(), instanceOf(VersionConflictEngineException.class)); } public void testVersioningReplicaConflict1() { @@ -1417,22 +1358,16 @@ public class InternalEngineTests extends ESTestCase { // now, the old one should not work index = new Engine.Index(newUid("1"), doc, 1L, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false); - try { - replicaEngine.index(index); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } + replicaEngine.index(index); + assertTrue(index.hasFailure()); + assertThat(index.getFailure(), instanceOf(VersionConflictEngineException.class)); // second version on replica should fail as well - try { - index = new Engine.Index(newUid("1"), doc, 2L - , VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false); - replicaEngine.index(index); - assertThat(index.version(), equalTo(2L)); - } catch (VersionConflictEngineException e) { - // all is well - } + index = new Engine.Index(newUid("1"), doc, 2L + , VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false); + replicaEngine.index(index); + assertThat(index.version(), equalTo(2L)); + assertThat(index.getFailure(), instanceOf(VersionConflictEngineException.class)); } public void testVersioningReplicaConflict2() { @@ -1464,23 +1399,17 @@ public class InternalEngineTests extends ESTestCase { assertThat(delete.version(), equalTo(3L)); // second time delete with same version should fail - try { - delete = new Engine.Delete("test", "1", newUid("1"), 3L - , VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, false); - replicaEngine.delete(delete); - fail("excepted VersionConflictEngineException to be thrown"); - } catch (VersionConflictEngineException e) { - // all is well - } + delete = new Engine.Delete("test", "1", newUid("1"), 3L + , VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, false); + replicaEngine.delete(delete); + assertTrue(delete.hasFailure()); + assertThat(delete.getFailure(), instanceOf(VersionConflictEngineException.class)); // now do the second index on the replica, it should fail - try { - index = new Engine.Index(newUid("1"), doc, 2L, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false); - replicaEngine.index(index); - fail("excepted VersionConflictEngineException to be thrown"); - } catch (VersionConflictEngineException e) { - // all is well - } + index = new Engine.Index(newUid("1"), doc, 2L, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false); + replicaEngine.index(index); + assertTrue(index.hasFailure()); + assertThat(index.getFailure(), instanceOf(VersionConflictEngineException.class)); } public void testBasicCreatedFlag() { @@ -1636,24 +1565,20 @@ public class InternalEngineTests extends ESTestCase { assertThat(getResult.exists(), equalTo(false)); // Try to index uid=1 with a too-old version, should fail: - try { - engine.index(new Engine.Index(newUid("1"), doc, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false)); - fail("did not hit expected exception"); - } catch (VersionConflictEngineException vcee) { - // expected - } + Engine.Index index = new Engine.Index(newUid("1"), doc, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false); + engine.index(index); + assertTrue(index.hasFailure()); + assertThat(index.getFailure(), instanceOf(VersionConflictEngineException.class)); // Get should still not find the document getResult = engine.get(new Engine.Get(true, newUid("1"))); assertThat(getResult.exists(), equalTo(false)); // Try to index uid=2 with a too-old version, should fail: - try { - engine.index(new Engine.Index(newUid("2"), doc, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false)); - fail("did not hit expected exception"); - } catch (VersionConflictEngineException vcee) { - // expected - } + Engine.Index index1 = new Engine.Index(newUid("2"), doc, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false); + engine.index(index1); + assertTrue(index1.hasFailure()); + assertThat(index1.getFailure(), instanceOf(VersionConflictEngineException.class)); // Get should not find the document getResult = engine.get(new Engine.Get(true, newUid("2"))); diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 335cffba493..b531aae0532 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -359,7 +359,7 @@ public class IndexShardTests extends IndexShardTestCase { try { indexShard.acquirePrimaryOperationLock(null, ThreadPool.Names.INDEX); fail("shard shouldn't accept primary ops"); - } catch (IllegalStateException ignored) { + } catch (ShardNotFoundException ignored) { } } diff --git a/core/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java b/core/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java index b95d872a61e..7d23ad37828 100644 --- a/core/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java +++ b/core/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java @@ -489,8 +489,8 @@ public class IndexStatsIT extends ESIntegTestCase { } catch (VersionConflictEngineException e) {} stats = client().admin().indices().prepareStats().setTypes("type1", "type2").execute().actionGet(); - assertThat(stats.getIndex("test1").getTotal().getIndexing().getTotal().getIndexFailedCount(), equalTo(2L)); - assertThat(stats.getIndex("test2").getTotal().getIndexing().getTotal().getIndexFailedCount(), equalTo(1L)); + assertThat(stats.getIndex("test1").getPrimaries().getIndexing().getTotal().getIndexFailedCount(), equalTo(2L)); + assertThat(stats.getIndex("test2").getPrimaries().getIndexing().getTotal().getIndexFailedCount(), equalTo(1L)); assertThat(stats.getPrimaries().getIndexing().getTypeStats().get("type1").getIndexFailedCount(), equalTo(1L)); assertThat(stats.getPrimaries().getIndexing().getTypeStats().get("type2").getIndexFailedCount(), equalTo(1L)); assertThat(stats.getTotal().getIndexing().getTotal().getIndexFailedCount(), equalTo(3L)); From 71dc4178b9225659abcfe37b363895b986b84271 Mon Sep 17 00:00:00 2001 From: Areek Zillur Date: Thu, 13 Oct 2016 22:42:27 -0400 Subject: [PATCH 003/132] Handle transient write failure in transport replication action --- .../action/bulk/TransportShardBulkAction.java | 211 ++++++++++-------- .../action/delete/TransportDeleteAction.java | 41 ++-- .../action/index/TransportIndexAction.java | 65 ++++-- .../replication/ReplicationOperation.java | 37 +-- .../TransportReplicationAction.java | 73 +++--- .../replication/TransportWriteAction.java | 104 +++++++-- .../elasticsearch/index/shard/IndexShard.java | 3 +- .../TransportWriteActionTests.java | 9 +- .../ESIndexLevelReplicationTestCase.java | 10 +- 9 files changed, 345 insertions(+), 208 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 15e6a5efa31..3021361b23e 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -21,13 +21,10 @@ package org.elasticsearch.action.bulk; import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.logging.log4j.util.Supplier; -import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.delete.DeleteResponse; -import org.elasticsearch.action.delete.TransportDeleteAction; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.index.TransportIndexAction; @@ -50,11 +47,9 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.VersionType; -import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.VersionConflictEngineException; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.translog.Translog; -import org.elasticsearch.index.translog.Translog.Location; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportRequestOptions; @@ -62,6 +57,8 @@ import org.elasticsearch.transport.TransportService; import java.util.Map; +import static org.elasticsearch.action.delete.TransportDeleteAction.*; +import static org.elasticsearch.action.index.TransportIndexAction.executeIndexRequestOnPrimary; import static org.elasticsearch.action.support.replication.ReplicationOperation.ignoreReplicaException; import static org.elasticsearch.action.support.replication.ReplicationOperation.isConflictException; @@ -102,7 +99,7 @@ public class TransportShardBulkAction extends TransportWriteAction onPrimaryShard(BulkShardRequest request, IndexShard primary) throws Exception { + protected PrimaryOperationResult onPrimaryShard(BulkShardRequest request, IndexShard primary) throws Exception { final IndexMetaData metaData = primary.indexSettings().getIndexMetaData(); long[] preVersions = new long[request.items().length]; @@ -118,30 +115,77 @@ public class TransportShardBulkAction extends TransportWriteAction(response, location); + return new PrimaryOperationResult<>(response, location); } /** Executes bulk item requests and handles request execution exceptions */ - private Translog.Location executeBulkItemRequest(IndexMetaData metaData, IndexShard indexShard, + private Translog.Location executeBulkItemRequest(IndexMetaData metaData, IndexShard primary, BulkShardRequest request, long[] preVersions, VersionType[] preVersionTypes, - Translog.Location location, int requestIndex) { + Translog.Location location, int requestIndex) throws Exception { preVersions[requestIndex] = request.items()[requestIndex].request().version(); preVersionTypes[requestIndex] = request.items()[requestIndex].request().versionType(); DocWriteRequest.OpType opType = request.items()[requestIndex].request().opType(); try { - WriteResult writeResult = innerExecuteBulkItemRequest(metaData, indexShard, - request, requestIndex); - if (writeResult.getLocation() != null) { - location = locationToSync(location, writeResult.getLocation()); - } else { - assert writeResult.getResponse().getResult() == DocWriteResponse.Result.NOOP - : "only noop operation can have null next operation"; + DocWriteRequest itemRequest = request.items()[requestIndex].request(); + final PrimaryOperationResult primaryOperationResult; + switch (itemRequest.opType()) { + case CREATE: + case INDEX: + primaryOperationResult = executeIndexRequestOnPrimary(((IndexRequest) itemRequest), primary, mappingUpdatedAction); + break; + case UPDATE: + int maxAttempts = ((UpdateRequest) itemRequest).retryOnConflict(); + PrimaryOperationResult shardUpdateOperation = null; + for (int attemptCount = 0; attemptCount <= maxAttempts; attemptCount++) { + shardUpdateOperation = shardUpdateOperation(metaData, primary, request, requestIndex, ((UpdateRequest) itemRequest)); + if (shardUpdateOperation.success() + || shardUpdateOperation.getFailure() instanceof VersionConflictEngineException == false) { + break; + } + } + if (shardUpdateOperation == null) { + throw new IllegalStateException("version conflict exception should bubble up on last attempt"); + } + primaryOperationResult = shardUpdateOperation; + break; + case DELETE: + primaryOperationResult = executeDeleteRequestOnPrimary(((DeleteRequest) itemRequest), primary); + break; + default: throw new IllegalStateException("unexpected opType [" + itemRequest.opType() + "] found"); + } + if (primaryOperationResult.success()) { + if (primaryOperationResult.getLocation() != null) { + location = locationToSync(location, primaryOperationResult.getLocation()); + } else { + assert primaryOperationResult.getResponse().getResult() == DocWriteResponse.Result.NOOP + : "only noop operation can have null next operation"; + } + // update the bulk item request because update request execution can mutate the bulk item request + BulkItemRequest item = request.items()[requestIndex]; + // add the response + setResponse(item, new BulkItemResponse(item.id(), opType, primaryOperationResult.getResponse())); + } else { + BulkItemRequest item = request.items()[requestIndex]; + DocWriteRequest docWriteRequest = item.request(); + Exception failure = primaryOperationResult.getFailure(); + if (isConflictException(failure)) { + logger.trace((Supplier) () -> new ParameterizedMessage("{} failed to execute bulk item ({}) {}", + request.shardId(), docWriteRequest.opType().getLowercase(), request), failure); + } else { + logger.debug((Supplier) () -> new ParameterizedMessage("{} failed to execute bulk item ({}) {}", + request.shardId(), docWriteRequest.opType().getLowercase(), request), failure); + } + // if its a conflict failure, and we already executed the request on a primary (and we execute it + // again, due to primary relocation and only processing up to N bulk items when the shard gets closed) + // then just use the response we got from the successful execution + if (item.getPrimaryResponse() != null && isConflictException(failure)) { + setResponse(item, item.getPrimaryResponse()); + } else { + setResponse(item, new BulkItemResponse(item.id(), docWriteRequest.opType(), + new BulkItemResponse.Failure(request.index(), docWriteRequest.type(), docWriteRequest.id(), failure))); + } } - // update the bulk item request because update request execution can mutate the bulk item request - BulkItemRequest item = request.items()[requestIndex]; - // add the response - setResponse(item, new BulkItemResponse(item.id(), opType, writeResult.getResponse())); } catch (Exception e) { // rethrow the failure if we are going to retry on primary and let parent failure to handle it if (retryPrimaryException(e)) { @@ -151,59 +195,16 @@ public class TransportShardBulkAction extends TransportWriteAction) () -> new ParameterizedMessage("{} failed to execute bulk item ({}) {}", - request.shardId(), docWriteRequest.opType().getLowercase(), request), e); - } else { - logger.debug((Supplier) () -> new ParameterizedMessage("{} failed to execute bulk item ({}) {}", - request.shardId(), docWriteRequest.opType().getLowercase(), request), e); - } - // if its a conflict failure, and we already executed the request on a primary (and we execute it - // again, due to primary relocation and only processing up to N bulk items when the shard gets closed) - // then just use the response we got from the successful execution - if (item.getPrimaryResponse() != null && isConflictException(e)) { - setResponse(item, item.getPrimaryResponse()); - } else { - setResponse(item, new BulkItemResponse(item.id(), docWriteRequest.opType(), - new BulkItemResponse.Failure(request.index(), docWriteRequest.type(), docWriteRequest.id(), e))); + throw e; } + // TODO: maybe this assert is too strict, we can still get environment failures while executing write operations + assert false : "unexpected exception: " + e.getMessage() + " class:" + e.getClass().getSimpleName(); } assert request.items()[requestIndex].getPrimaryResponse() != null; assert preVersionTypes[requestIndex] != null; return location; } - private WriteResult innerExecuteBulkItemRequest(IndexMetaData metaData, IndexShard indexShard, - BulkShardRequest request, int requestIndex) throws Exception { - DocWriteRequest itemRequest = request.items()[requestIndex].request(); - switch (itemRequest.opType()) { - case CREATE: - case INDEX: - return TransportIndexAction.executeIndexRequestOnPrimary(((IndexRequest) itemRequest), indexShard, mappingUpdatedAction); - case UPDATE: - int maxAttempts = ((UpdateRequest) itemRequest).retryOnConflict(); - for (int attemptCount = 0; attemptCount <= maxAttempts; attemptCount++) { - try { - return shardUpdateOperation(metaData, indexShard, request, requestIndex, ((UpdateRequest) itemRequest)); - } catch (Exception e) { - final Throwable cause = ExceptionsHelper.unwrapCause(e); - if (attemptCount == maxAttempts // bubble up exception when we run out of attempts - || (cause instanceof VersionConflictEngineException) == false) { // or when exception is not a version conflict - throw e; - } - } - } - throw new IllegalStateException("version conflict exception should bubble up on last attempt"); - case DELETE: - return TransportDeleteAction.executeDeleteRequestOnPrimary(((DeleteRequest) itemRequest), indexShard); - default: throw new IllegalStateException("unexpected opType [" + itemRequest.opType() + "] found"); - } - } - private void setResponse(BulkItemRequest request, BulkItemResponse response) { request.setPrimaryResponse(response); if (response.isFailed()) { @@ -218,50 +219,62 @@ public class TransportShardBulkAction extends TransportWriteAction shardUpdateOperation(IndexMetaData metaData, IndexShard indexShard, + private PrimaryOperationResult shardUpdateOperation(IndexMetaData metaData, IndexShard primary, BulkShardRequest request, int requestIndex, UpdateRequest updateRequest) throws Exception { - // Todo: capture read version conflicts, missing documents and malformed script errors in the write result due to get request - UpdateHelper.Result translate = updateHelper.prepare(updateRequest, indexShard, threadPool::estimatedTimeInMillis); + final UpdateHelper.Result translate; + try { + translate = updateHelper.prepare(updateRequest, primary, threadPool::estimatedTimeInMillis); + } catch (Exception e) { + return new PrimaryOperationResult<>(e); + } switch (translate.getResponseResult()) { case CREATED: case UPDATED: IndexRequest indexRequest = translate.action(); MappingMetaData mappingMd = metaData.mappingOrDefault(indexRequest.type()); indexRequest.process(mappingMd, allowIdGeneration, request.index()); - WriteResult writeResult = TransportIndexAction.executeIndexRequestOnPrimary(indexRequest, indexShard, mappingUpdatedAction); - BytesReference indexSourceAsBytes = indexRequest.source(); - IndexResponse indexResponse = writeResult.getResponse(); - UpdateResponse update = new UpdateResponse(indexResponse.getShardInfo(), indexResponse.getShardId(), indexResponse.getType(), indexResponse.getId(), indexResponse.getVersion(), indexResponse.getResult()); - if ((updateRequest.fetchSource() != null && updateRequest.fetchSource().fetchSource()) || - (updateRequest.fields() != null && updateRequest.fields().length > 0)) { - Tuple> sourceAndContent = XContentHelper.convertToMap(indexSourceAsBytes, true); - update.setGetResult(updateHelper.extractGetResult(updateRequest, request.index(), indexResponse.getVersion(), sourceAndContent.v2(), sourceAndContent.v1(), indexSourceAsBytes)); + PrimaryOperationResult writeResult = executeIndexRequestOnPrimary(indexRequest, primary, mappingUpdatedAction); + if (writeResult.success()) { + BytesReference indexSourceAsBytes = indexRequest.source(); + IndexResponse indexResponse = writeResult.getResponse(); + UpdateResponse update = new UpdateResponse(indexResponse.getShardInfo(), indexResponse.getShardId(), indexResponse.getType(), indexResponse.getId(), indexResponse.getVersion(), indexResponse.getResult()); + if ((updateRequest.fetchSource() != null && updateRequest.fetchSource().fetchSource()) || + (updateRequest.fields() != null && updateRequest.fields().length > 0)) { + Tuple> sourceAndContent = XContentHelper.convertToMap(indexSourceAsBytes, true); + update.setGetResult(updateHelper.extractGetResult(updateRequest, request.index(), indexResponse.getVersion(), sourceAndContent.v2(), sourceAndContent.v1(), indexSourceAsBytes)); + } + // Replace the update request to the translated index request to execute on the replica. + request.items()[requestIndex] = new BulkItemRequest(request.items()[requestIndex].id(), indexRequest); + return new PrimaryOperationResult<>(update, writeResult.getLocation()); + } else { + return writeResult; } - // Replace the update request to the translated index request to execute on the replica. - request.items()[requestIndex] = new BulkItemRequest(request.items()[requestIndex].id(), indexRequest); - return new WriteResult<>(update, writeResult.getLocation()); case DELETED: DeleteRequest deleteRequest = translate.action(); - WriteResult deleteResult = TransportDeleteAction.executeDeleteRequestOnPrimary(deleteRequest, indexShard); - DeleteResponse response = deleteResult.getResponse(); - UpdateResponse deleteUpdateResponse = new UpdateResponse(response.getShardInfo(), response.getShardId(), response.getType(), response.getId(), response.getVersion(), response.getResult()); - deleteUpdateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, request.index(), response.getVersion(), translate.updatedSourceAsMap(), translate.updateSourceContentType(), null)); - // Replace the update request to the translated delete request to execute on the replica. - request.items()[requestIndex] = new BulkItemRequest(request.items()[requestIndex].id(), deleteRequest); - return new WriteResult<>(deleteUpdateResponse, deleteResult.getLocation()); + PrimaryOperationResult deleteResult = executeDeleteRequestOnPrimary(deleteRequest, primary); + if (deleteResult.success()) { + DeleteResponse response = deleteResult.getResponse(); + UpdateResponse deleteUpdateResponse = new UpdateResponse(response.getShardInfo(), response.getShardId(), response.getType(), response.getId(), response.getVersion(), response.getResult()); + deleteUpdateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, request.index(), response.getVersion(), translate.updatedSourceAsMap(), translate.updateSourceContentType(), null)); + // Replace the update request to the translated delete request to execute on the replica. + request.items()[requestIndex] = new BulkItemRequest(request.items()[requestIndex].id(), deleteRequest); + return new PrimaryOperationResult<>(deleteUpdateResponse, deleteResult.getLocation()); + } else { + return deleteResult; + } case NOOP: BulkItemRequest item = request.items()[requestIndex]; - indexShard.noopUpdate(updateRequest.type()); + primary.noopUpdate(updateRequest.type()); item.setIgnoreOnReplica(); // no need to go to the replica - return new WriteResult<>(translate.action(), null); + return new PrimaryOperationResult<>(translate.action(), null); default: throw new IllegalStateException("Illegal update operation " + translate.getResponseResult()); } } @Override - protected Location onReplicaShard(BulkShardRequest request, IndexShard indexShard) { + protected ReplicaOperationResult onReplicaShard(BulkShardRequest request, IndexShard replica) throws Exception { Translog.Location location = null; for (int i = 0; i < request.items().length; i++) { BulkItemRequest item = request.items()[i]; @@ -269,20 +282,28 @@ public class TransportShardBulkAction extends TransportWriteAction onPrimaryShard(DeleteRequest request, IndexShard indexShard) { - return executeDeleteRequestOnPrimary(request, indexShard); + protected PrimaryOperationResult onPrimaryShard(DeleteRequest request, IndexShard primary) { + return executeDeleteRequestOnPrimary(request, primary); } @Override - protected Location onReplicaShard(DeleteRequest request, IndexShard indexShard) { - return executeDeleteRequestOnReplica(request, indexShard).getTranslogLocation(); + protected ReplicaOperationResult onReplicaShard(DeleteRequest request, IndexShard replica) { + return executeDeleteRequestOnReplica(request, replica); } - public static WriteResult executeDeleteRequestOnPrimary(DeleteRequest request, IndexShard indexShard) { - Engine.Delete delete = indexShard.prepareDeleteOnPrimary(request.type(), request.id(), request.version(), request.versionType()); - indexShard.delete(delete); - // update the request with the version so it will go to the replicas - request.versionType(delete.versionType().versionTypeForReplicationAndRecovery()); - request.version(delete.version()); + public static PrimaryOperationResult executeDeleteRequestOnPrimary(DeleteRequest request, IndexShard primary) { + Engine.Delete delete = primary.prepareDeleteOnPrimary(request.type(), request.id(), request.version(), request.versionType()); + primary.delete(delete); + if (delete.hasFailure()) { + return new PrimaryOperationResult<>(delete.getFailure()); + } else { + // update the request with the version so it will go to the replicas + request.versionType(delete.versionType().versionTypeForReplicationAndRecovery()); + request.version(delete.version()); - assert request.versionType().validateVersionForWrites(request.version()); - DeleteResponse response = new DeleteResponse(indexShard.shardId(), request.type(), request.id(), delete.version(), delete.found()); - return new WriteResult<>(response, delete.getTranslogLocation()); + assert request.versionType().validateVersionForWrites(request.version()); + DeleteResponse response = new DeleteResponse(primary.shardId(), request.type(), request.id(), delete.version(), delete.found()); + return new PrimaryOperationResult<>(response, delete.getTranslogLocation()); + } } - public static Engine.Delete executeDeleteRequestOnReplica(DeleteRequest request, IndexShard indexShard) { - Engine.Delete delete = indexShard.prepareDeleteOnReplica(request.type(), request.id(), request.version(), request.versionType()); - indexShard.delete(delete); - return delete; + public static ReplicaOperationResult executeDeleteRequestOnReplica(DeleteRequest request, IndexShard replica) { + Engine.Delete delete = replica.prepareDeleteOnReplica(request.type(), request.id(), request.version(), request.versionType()); + replica.delete(delete); + return delete.hasFailure() + ? new ReplicaOperationResult(delete.getFailure()) + : new ReplicaOperationResult(delete.getTranslogLocation()); } } diff --git a/core/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java b/core/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java index 9f091be5863..a4dc10b11c0 100644 --- a/core/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java +++ b/core/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java @@ -39,11 +39,11 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.index.translog.Translog.Location; import org.elasticsearch.indices.IndexAlreadyExistsException; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.tasks.Task; @@ -140,48 +140,68 @@ public class TransportIndexAction extends TransportWriteAction onPrimaryShard(IndexRequest request, IndexShard indexShard) throws Exception { - return executeIndexRequestOnPrimary(request, indexShard, mappingUpdatedAction); + protected PrimaryOperationResult onPrimaryShard(IndexRequest request, IndexShard primary) throws Exception { + return executeIndexRequestOnPrimary(request, primary, mappingUpdatedAction); } @Override - protected Location onReplicaShard(IndexRequest request, IndexShard indexShard) { - return executeIndexRequestOnReplica(request, indexShard).getTranslogLocation(); + protected ReplicaOperationResult onReplicaShard(IndexRequest request, IndexShard replica) { + return executeIndexRequestOnReplica(request, replica); } /** * Execute the given {@link IndexRequest} on a replica shard, throwing a * {@link RetryOnReplicaException} if the operation needs to be re-tried. */ - public static Engine.Index executeIndexRequestOnReplica(IndexRequest request, IndexShard indexShard) { + public static ReplicaOperationResult executeIndexRequestOnReplica(IndexRequest request, IndexShard indexShard) { final ShardId shardId = indexShard.shardId(); SourceToParse sourceToParse = SourceToParse.source(SourceToParse.Origin.REPLICA, shardId.getIndexName(), request.type(), request.id(), request.source()) .routing(request.routing()).parent(request.parent()).timestamp(request.timestamp()).ttl(request.ttl()); - final Engine.Index operation = indexShard.prepareIndexOnReplica(sourceToParse, request.version(), request.versionType(), request.getAutoGeneratedTimestamp(), request.isRetry()); + final Engine.Index operation; + try { + operation = indexShard.prepareIndexOnReplica(sourceToParse, request.version(), request.versionType(), request.getAutoGeneratedTimestamp(), request.isRetry()); + } catch (MapperParsingException | IllegalArgumentException e) { + return new ReplicaOperationResult(e); + } Mapping update = operation.parsedDoc().dynamicMappingsUpdate(); if (update != null) { throw new RetryOnReplicaException(shardId, "Mappings are not available on the replica yet, triggered update: " + update); } indexShard.index(operation); - return operation; + if (operation.hasFailure()) { + return new ReplicaOperationResult(operation.getFailure()); + } else { + return new ReplicaOperationResult(operation.getTranslogLocation()); + } } /** Utility method to prepare an index operation on primary shards */ - public static Engine.Index prepareIndexOperationOnPrimary(IndexRequest request, IndexShard indexShard) { + static Engine.Index prepareIndexOperationOnPrimary(IndexRequest request, IndexShard indexShard) { SourceToParse sourceToParse = SourceToParse.source(SourceToParse.Origin.PRIMARY, request.index(), request.type(), request.id(), request.source()) .routing(request.routing()).parent(request.parent()).timestamp(request.timestamp()).ttl(request.ttl()); return indexShard.prepareIndexOnPrimary(sourceToParse, request.version(), request.versionType(), request.getAutoGeneratedTimestamp(), request.isRetry()); } - public static WriteResult executeIndexRequestOnPrimary(IndexRequest request, IndexShard indexShard, + public static PrimaryOperationResult executeIndexRequestOnPrimary(IndexRequest request, IndexShard indexShard, MappingUpdatedAction mappingUpdatedAction) throws Exception { - Engine.Index operation = prepareIndexOperationOnPrimary(request, indexShard); + Engine.Index operation; + try { + operation = prepareIndexOperationOnPrimary(request, indexShard); + } catch (MapperParsingException | IllegalArgumentException e) { + return new PrimaryOperationResult<>(e); + } Mapping update = operation.parsedDoc().dynamicMappingsUpdate(); final ShardId shardId = indexShard.shardId(); if (update != null) { - mappingUpdatedAction.updateMappingOnMaster(shardId.getIndex(), request.type(), update); - operation = prepareIndexOperationOnPrimary(request, indexShard); + try { + // can throw timeout exception when updating mappings or ISE for attempting to update default mappings + // which are bubbled up + mappingUpdatedAction.updateMappingOnMaster(shardId.getIndex(), request.type(), update); + operation = prepareIndexOperationOnPrimary(request, indexShard); + } catch (MapperParsingException | IllegalArgumentException e) { + return new PrimaryOperationResult<>(e); + } update = operation.parsedDoc().dynamicMappingsUpdate(); if (update != null) { throw new ReplicationOperation.RetryOnPrimaryException(shardId, @@ -189,16 +209,19 @@ public class TransportIndexAction extends TransportWriteAction(operation.getFailure()); + } else { + // update the version on request so it will happen on the replicas + final long version = operation.version(); + request.version(version); + request.versionType(request.versionType().versionTypeForReplicationAndRecovery()); - // update the version on request so it will happen on the replicas - final long version = operation.version(); - request.version(version); - request.versionType(request.versionType().versionTypeForReplicationAndRecovery()); + assert request.versionType().validateVersionForWrites(request.version()); - assert request.versionType().validateVersionForWrites(request.version()); - - IndexResponse response = new IndexResponse(shardId, request.type(), request.id(), request.version(), operation.isCreated()); - return new WriteResult<>(response, operation.getTranslogLocation()); + IndexResponse response = new IndexResponse(shardId, request.type(), request.id(), request.version(), operation.isCreated()); + return new PrimaryOperationResult<>(response, operation.getTranslogLocation()); + } } } diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java b/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java index d541ef6a35c..c049336bafc 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java @@ -32,6 +32,7 @@ import org.elasticsearch.cluster.routing.AllocationId; import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.common.Nullable; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.index.engine.VersionConflictEngineException; @@ -112,22 +113,24 @@ public class ReplicationOperation< pendingActions.incrementAndGet(); primaryResult = primary.perform(request); final ReplicaRequest replicaRequest = primaryResult.replicaRequest(); - assert replicaRequest.primaryTerm() > 0 : "replicaRequest doesn't have a primary term"; - if (logger.isTraceEnabled()) { - logger.trace("[{}] op [{}] completed on primary for request [{}]", primaryId, opType, request); + if (replicaRequest != null) { + assert replicaRequest.primaryTerm() > 0 : "replicaRequest doesn't have a primary term"; + if (logger.isTraceEnabled()) { + logger.trace("[{}] op [{}] completed on primary for request [{}]", primaryId, opType, request); + } + + // we have to get a new state after successfully indexing into the primary in order to honour recovery semantics. + // we have to make sure that every operation indexed into the primary after recovery start will also be replicated + // to the recovery target. If we use an old cluster state, we may miss a relocation that has started since then. + ClusterState clusterState = clusterStateSupplier.get(); + final List shards = getShards(primaryId, clusterState); + Set inSyncAllocationIds = getInSyncAllocationIds(primaryId, clusterState); + + markUnavailableShardsAsStale(replicaRequest, inSyncAllocationIds, shards); + + performOnReplicas(replicaRequest, shards); } - // we have to get a new state after successfully indexing into the primary in order to honour recovery semantics. - // we have to make sure that every operation indexed into the primary after recovery start will also be replicated - // to the recovery target. If we use an old cluster state, we may miss a relocation that has started since then. - ClusterState clusterState = clusterStateSupplier.get(); - final List shards = getShards(primaryId, clusterState); - Set inSyncAllocationIds = getInSyncAllocationIds(primaryId, clusterState); - - markUnavailableShardsAsStale(replicaRequest, inSyncAllocationIds, shards); - - performOnReplicas(replicaRequest, shards); - successfulShards.incrementAndGet(); decPendingAndFinishIfNeeded(); } @@ -419,7 +422,11 @@ public class ReplicationOperation< public interface PrimaryResult> { - R replicaRequest(); + /** + * @return null if no operation needs to be sent to a replica + * (for example when the operation failed on the primary due to a parsing exception) + */ + @Nullable R replicaRequest(); void setShardInfo(ReplicationResponse.ShardInfo shardInfo); } diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index 740a003ffa8..983132ac787 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -178,7 +178,7 @@ public abstract class TransportReplicationAction< * @param shardRequest the request to the replica shard * @param replica the replica shard to perform the operation on */ - protected abstract ReplicaResult shardOperationOnReplica(ReplicaRequest shardRequest, IndexShard replica); + protected abstract ReplicaResult shardOperationOnReplica(ReplicaRequest shardRequest, IndexShard replica) throws Exception; /** * Cluster level block to check before request execution @@ -206,8 +206,13 @@ public abstract class TransportReplicationAction< } protected boolean retryPrimaryException(final Throwable e) { - return e.getClass() == ReplicationOperation.RetryOnPrimaryException.class - || TransportActions.isShardNotAvailableException(e); + boolean retry = e.getClass() == ReplicationOperation.RetryOnPrimaryException.class + || TransportActions.isShardNotAvailableException(e); + if (retry) { + assert e instanceof ElasticsearchException + : "expected all retry on primary exception to be ElasticsearchException instances, found: " + e.getClass(); + } + return retry; } class OperationTransportHandler implements TransportRequestHandler { @@ -310,17 +315,10 @@ public abstract class TransportReplicationAction< final IndexMetaData indexMetaData = clusterService.state().getMetaData().index(request.shardId().getIndex()); final boolean executeOnReplicas = (indexMetaData == null) || shouldExecuteReplication(indexMetaData.getSettings()); final ActionListener listener = createResponseListener(primaryShardReference); - createReplicatedOperation(request, new ActionListener() { - @Override - public void onResponse(PrimaryResult result) { - result.respond(listener); - } - - @Override - public void onFailure(Exception e) { - listener.onFailure(e); - } - }, primaryShardReference, executeOnReplicas).execute(); + createReplicatedOperation(request, + ActionListener.wrap(result -> result.respond(listener), listener::onFailure), + primaryShardReference, executeOnReplicas) + .execute(); } } catch (Exception e) { Releasables.closeWhileHandlingException(primaryShardReference); // release shard operation lock before responding to caller @@ -376,11 +374,19 @@ public abstract class TransportReplicationAction< protected class PrimaryResult implements ReplicationOperation.PrimaryResult { final ReplicaRequest replicaRequest; - final Response finalResponse; + final Response finalResponseIfSuccessful; + final Exception finalFailure; - public PrimaryResult(ReplicaRequest replicaRequest, Response finalResponse) { + public PrimaryResult(ReplicaRequest replicaRequest, Response finalResponseIfSuccessful) { this.replicaRequest = replicaRequest; - this.finalResponse = finalResponse; + this.finalResponseIfSuccessful = finalResponseIfSuccessful; + this.finalFailure = null; + } + + public PrimaryResult(Exception finalFailure) { + this.replicaRequest = null; + this.finalResponseIfSuccessful = null; + this.finalFailure = finalFailure; } @Override @@ -390,22 +396,37 @@ public abstract class TransportReplicationAction< @Override public void setShardInfo(ReplicationResponse.ShardInfo shardInfo) { - finalResponse.setShardInfo(shardInfo); + if (finalResponseIfSuccessful != null) { + finalResponseIfSuccessful.setShardInfo(shardInfo); + } } public void respond(ActionListener listener) { - listener.onResponse(finalResponse); + if (finalResponseIfSuccessful != null) { + listener.onResponse(finalResponseIfSuccessful); + } else { + listener.onFailure(finalFailure); + } } } protected class ReplicaResult { - /** - * Public constructor so subclasses can call it. - */ - public ReplicaResult() {} + final Exception finalFailure; + + public ReplicaResult() { + this.finalFailure = null; + } + + public ReplicaResult(Exception finalFailure) { + this.finalFailure = finalFailure; + } public void respond(ActionListener listener) { - listener.onResponse(TransportResponse.Empty.INSTANCE); + if (finalFailure == null) { + listener.onResponse(TransportResponse.Empty.INSTANCE); + } else { + listener.onFailure(finalFailure); + } } } @@ -902,7 +923,9 @@ public abstract class TransportReplicationAction< @Override public PrimaryResult perform(Request request) throws Exception { PrimaryResult result = shardOperationOnPrimary(request, indexShard); - result.replicaRequest().primaryTerm(indexShard.getPrimaryTerm()); + if (result.replicaRequest() != null) { + result.replicaRequest().primaryTerm(indexShard.getPrimaryTerm()); + } return result; } diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java b/core/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java index 68ae7dffa02..6176889d159 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java @@ -60,47 +60,94 @@ public abstract class TransportWriteAction< } /** - * Called on the primary with a reference to the {@linkplain IndexShard} to modify. + * Called on the primary with a reference to the primary {@linkplain IndexShard} to modify. */ - protected abstract WriteResult onPrimaryShard(Request request, IndexShard indexShard) throws Exception; + protected abstract PrimaryOperationResult onPrimaryShard(Request request, IndexShard primary) throws Exception; /** - * Called once per replica with a reference to the {@linkplain IndexShard} to modify. + * Called once per replica with a reference to the replica {@linkplain IndexShard} to modify. * - * @return the translog location of the {@linkplain IndexShard} after the write was completed or null if no write occurred + * @return the result of the replication operation containing either the translog location of the {@linkplain IndexShard} + * after the write was completed or a failure if the operation failed */ - protected abstract Translog.Location onReplicaShard(ReplicaRequest request, IndexShard indexShard); + protected abstract ReplicaOperationResult onReplicaShard(ReplicaRequest request, IndexShard replica) throws Exception; @Override protected final WritePrimaryResult shardOperationOnPrimary(Request request, IndexShard primary) throws Exception { - WriteResult result = onPrimaryShard(request, primary); - return new WritePrimaryResult(((ReplicaRequest) request), result.getResponse(), result.getLocation(), primary); + final PrimaryOperationResult result = onPrimaryShard(request, primary); + return result.success() + ? new WritePrimaryResult((ReplicaRequest) request, result.getResponse(), result.getLocation(), primary) + : new WritePrimaryResult(result.getFailure()); } @Override - protected final WriteReplicaResult shardOperationOnReplica(ReplicaRequest request, IndexShard replica) { - Translog.Location location = onReplicaShard(request, replica); - return new WriteReplicaResult(replica, request, location); + protected final WriteReplicaResult shardOperationOnReplica(ReplicaRequest request, IndexShard replica) throws Exception { + final ReplicaOperationResult result = onReplicaShard(request, replica); + return result.success() + ? new WriteReplicaResult(request, result.getLocation(), replica) + : new WriteReplicaResult(result.getFailure()); + } + + abstract static class OperationWriteResult { + private final Translog.Location location; + private final Exception failure; + + protected OperationWriteResult(@Nullable Location location) { + this.location = location; + this.failure = null; + } + + protected OperationWriteResult(Exception failure) { + this.location = null; + this.failure = failure; + } + + public Translog.Location getLocation() { + return location; + } + + public Exception getFailure() { + return failure; + } + + public boolean success() { + return failure == null; + } } /** - * Simple result from a write action. Write actions have static method to return these so they can integrate with bulk. + * Simple result from a primary write action (includes response). + * Write actions have static method to return these so they can integrate with bulk. */ - public static class WriteResult { + public static class PrimaryOperationResult extends OperationWriteResult { private final Response response; - private final Translog.Location location; - public WriteResult(Response response, @Nullable Location location) { + public PrimaryOperationResult(Response response, @Nullable Location location) { + super(location); this.response = response; - this.location = location; + } + + public PrimaryOperationResult(Exception failure) { + super(failure); + this.response = null; } public Response getResponse() { return response; } + } - public Translog.Location getLocation() { - return location; + /** + * Simple result from a replica write action. Write actions have static method to return these so they can integrate with bulk. + */ + public static class ReplicaOperationResult extends OperationWriteResult { + + public ReplicaOperationResult(@Nullable Location location) { + super(location); + } + + public ReplicaOperationResult(Exception failure) { + super(failure); } } @@ -112,14 +159,18 @@ public abstract class TransportWriteAction< ActionListener listener = null; public WritePrimaryResult(ReplicaRequest request, Response finalResponse, - @Nullable Translog.Location location, - IndexShard indexShard) { + @Nullable Location location, IndexShard primary) { super(request, finalResponse); /* * We call this before replication because this might wait for a refresh and that can take a while. This way we wait for the * refresh in parallel on the primary and on the replica. */ - new AsyncAfterWriteAction(indexShard, request, location, this, logger).run(); + new AsyncAfterWriteAction(primary, request, location, this, logger).run(); + } + + public WritePrimaryResult(Exception failure) { + super(failure); + this.finishedAsyncActions = true; } @Override @@ -148,7 +199,9 @@ public abstract class TransportWriteAction< @Override public synchronized void onSuccess(boolean forcedRefresh) { - finalResponse.setForcedRefresh(forcedRefresh); + if (finalResponseIfSuccessful != null) { + finalResponseIfSuccessful.setForcedRefresh(forcedRefresh); + } finishedAsyncActions = true; respondIfPossible(null); } @@ -161,8 +214,13 @@ public abstract class TransportWriteAction< boolean finishedAsyncActions; private ActionListener listener; - public WriteReplicaResult(IndexShard indexShard, ReplicatedWriteRequest request, Translog.Location location) { - new AsyncAfterWriteAction(indexShard, request, location, this, logger).run(); + public WriteReplicaResult(ReplicaRequest request, Location location, IndexShard replica) { + new AsyncAfterWriteAction(replica, request, location, this, logger).run(); + } + + public WriteReplicaResult(Exception finalFailure) { + super(finalFailure); + this.finishedAsyncActions = true; } @Override diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 162d3495c3e..0eb8d1b076e 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1124,7 +1124,8 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl private void verifyPrimary() { if (shardRouting.primary() == false) { - throw new IllegalStateException("shard is not a primary " + shardRouting); + // TODO throw a more appropriate exception + throw new ShardNotFoundException(shardRouting.shardId(), "shard is not a primary anymore"); } } diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java b/core/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java index 2a1471fa746..c92bba093db 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java @@ -28,7 +28,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.translog.Translog; -import org.elasticsearch.index.translog.Translog.Location; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -137,13 +136,13 @@ public class TransportWriteActionTests extends ESTestCase { } @Override - protected WriteResult onPrimaryShard(TestRequest request, IndexShard indexShard) throws Exception { - return new WriteResult<>(new TestResponse(), location); + protected PrimaryOperationResult onPrimaryShard(TestRequest request, IndexShard primary) throws Exception { + return new PrimaryOperationResult<>(new TestResponse(), location); } @Override - protected Location onReplicaShard(TestRequest request, IndexShard indexShard) { - return location; + protected ReplicaOperationResult onReplicaShard(TestRequest request, IndexShard replica) { + return new ReplicaOperationResult(location); } @Override diff --git a/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java b/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java index 9ccfd7243a5..31b8f61bc30 100644 --- a/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java +++ b/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java @@ -25,7 +25,6 @@ import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexResponse; -import org.elasticsearch.action.index.TransportIndexAction; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.replication.ReplicationOperation; import org.elasticsearch.action.support.replication.ReplicationRequest; @@ -40,7 +39,6 @@ import org.elasticsearch.cluster.routing.ShardRoutingHelper; import org.elasticsearch.common.collect.Iterators; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.Index; -import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardTestCase; @@ -65,6 +63,8 @@ import java.util.function.Consumer; import java.util.stream.Collectors; import java.util.stream.StreamSupport; +import static org.elasticsearch.action.index.TransportIndexAction.executeIndexRequestOnPrimary; +import static org.elasticsearch.action.index.TransportIndexAction.executeIndexRequestOnReplica; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; @@ -365,7 +365,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase @Override protected PrimaryResult performOnPrimary(IndexShard primary, IndexRequest request) throws Exception { - TransportWriteAction.WriteResult result = TransportIndexAction.executeIndexRequestOnPrimary(request, primary, + TransportWriteAction.PrimaryOperationResult result = executeIndexRequestOnPrimary(request, primary, null); request.primaryTerm(primary.getPrimaryTerm()); TransportWriteActionTestHelper.performPostWriteActions(primary, request, result.getLocation(), logger); @@ -374,8 +374,8 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase @Override protected void performOnReplica(IndexRequest request, IndexShard replica) { - Engine.Index index = TransportIndexAction.executeIndexRequestOnReplica(request, replica); - TransportWriteActionTestHelper.performPostWriteActions(replica, request, index.getTranslogLocation(), logger); + TransportWriteAction.ReplicaOperationResult index = executeIndexRequestOnReplica(request, replica); + TransportWriteActionTestHelper.performPostWriteActions(replica, request, index.getLocation(), logger); } } } From 1bdeada8aa9e699d5c864d7257fea926426ce474 Mon Sep 17 00:00:00 2001 From: Areek Zillur Date: Fri, 14 Oct 2016 17:50:43 -0400 Subject: [PATCH 004/132] Generify index shard method to execute engine write operation Now index and delete methods in index shard share code for indexing stats. This commit collapses seperate methods for index and delete operations into a generic execute method for performing engine write operations. As an added benefit, this commit cleans up the interface for indexing operation listener making it more simple and concise to use. --- .../action/bulk/TransportShardBulkAction.java | 3 +- .../action/delete/TransportDeleteAction.java | 4 +- .../action/index/TransportIndexAction.java | 22 +-- .../elasticsearch/index/IndexingSlowLog.java | 29 ++-- .../elasticsearch/index/engine/Engine.java | 19 ++- .../elasticsearch/index/shard/IndexShard.java | 61 +++------ .../shard/IndexingOperationListener.java | 125 ++++-------------- .../index/shard/InternalIndexingStats.java | 98 +++++++------- .../indices/IndexingMemoryController.java | 18 +-- .../elasticsearch/index/IndexModuleTests.java | 9 +- .../index/mapper/TextFieldMapperTests.java | 4 +- .../index/shard/IndexShardIT.java | 19 +-- .../index/shard/IndexShardTests.java | 102 +++++++------- .../shard/IndexingOperationListenerTests.java | 80 +++++------ .../index/reindex/CancelTests.java | 19 +-- .../index/shard/IndexShardTestCase.java | 4 +- 16 files changed, 243 insertions(+), 373 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 3021361b23e..7401fc6f63c 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -57,7 +57,8 @@ import org.elasticsearch.transport.TransportService; import java.util.Map; -import static org.elasticsearch.action.delete.TransportDeleteAction.*; +import static org.elasticsearch.action.delete.TransportDeleteAction.executeDeleteRequestOnPrimary; +import static org.elasticsearch.action.delete.TransportDeleteAction.executeDeleteRequestOnReplica; import static org.elasticsearch.action.index.TransportIndexAction.executeIndexRequestOnPrimary; import static org.elasticsearch.action.support.replication.ReplicationOperation.ignoreReplicaException; import static org.elasticsearch.action.support.replication.ReplicationOperation.isConflictException; diff --git a/core/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java b/core/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java index 3cfb2930447..5b83f288b07 100644 --- a/core/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java +++ b/core/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java @@ -129,7 +129,7 @@ public class TransportDeleteAction extends TransportWriteAction executeDeleteRequestOnPrimary(DeleteRequest request, IndexShard primary) { Engine.Delete delete = primary.prepareDeleteOnPrimary(request.type(), request.id(), request.version(), request.versionType()); - primary.delete(delete); + primary.execute(delete); if (delete.hasFailure()) { return new PrimaryOperationResult<>(delete.getFailure()); } else { @@ -145,7 +145,7 @@ public class TransportDeleteAction extends TransportWriteAction executeIndexRequestOnPrimary(IndexRequest request, IndexShard indexShard, + public static PrimaryOperationResult executeIndexRequestOnPrimary(IndexRequest request, IndexShard primary, MappingUpdatedAction mappingUpdatedAction) throws Exception { Engine.Index operation; try { - operation = prepareIndexOperationOnPrimary(request, indexShard); + operation = prepareIndexOperationOnPrimary(request, primary); } catch (MapperParsingException | IllegalArgumentException e) { return new PrimaryOperationResult<>(e); } Mapping update = operation.parsedDoc().dynamicMappingsUpdate(); - final ShardId shardId = indexShard.shardId(); + final ShardId shardId = primary.shardId(); if (update != null) { try { // can throw timeout exception when updating mappings or ISE for attempting to update default mappings // which are bubbled up mappingUpdatedAction.updateMappingOnMaster(shardId.getIndex(), request.type(), update); - operation = prepareIndexOperationOnPrimary(request, indexShard); + operation = prepareIndexOperationOnPrimary(request, primary); } catch (MapperParsingException | IllegalArgumentException e) { return new PrimaryOperationResult<>(e); } @@ -208,7 +208,7 @@ public class TransportIndexAction extends TransportWriteAction(operation.getFailure()); } else { diff --git a/core/src/main/java/org/elasticsearch/index/IndexingSlowLog.java b/core/src/main/java/org/elasticsearch/index/IndexingSlowLog.java index 513e87878d6..d2ebc4e78de 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexingSlowLog.java +++ b/core/src/main/java/org/elasticsearch/index/IndexingSlowLog.java @@ -133,23 +133,20 @@ public final class IndexingSlowLog implements IndexingOperationListener { this.reformat = reformat; } - @Override - public void postIndex(Engine.Index index, boolean created) { - final long took = index.endTime() - index.startTime(); - postIndexing(index.parsedDoc(), took); - } - - - private void postIndexing(ParsedDocument doc, long tookInNanos) { - if (indexWarnThreshold >= 0 && tookInNanos > indexWarnThreshold) { - indexLogger.warn("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog)); - } else if (indexInfoThreshold >= 0 && tookInNanos > indexInfoThreshold) { - indexLogger.info("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog)); - } else if (indexDebugThreshold >= 0 && tookInNanos > indexDebugThreshold) { - indexLogger.debug("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog)); - } else if (indexTraceThreshold >= 0 && tookInNanos > indexTraceThreshold) { - indexLogger.trace("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog)); + public void postOperation(Engine.Operation operation) { + if (operation.operationType() == Engine.Operation.TYPE.INDEX) { + final long tookInNanos = operation.endTime() - operation.startTime(); + ParsedDocument doc = ((Engine.Index) operation).parsedDoc(); + if (indexWarnThreshold >= 0 && tookInNanos > indexWarnThreshold) { + indexLogger.warn("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog)); + } else if (indexInfoThreshold >= 0 && tookInNanos > indexInfoThreshold) { + indexLogger.info("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog)); + } else if (indexDebugThreshold >= 0 && tookInNanos > indexDebugThreshold) { + indexLogger.debug("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog)); + } else if (indexTraceThreshold >= 0 && tookInNanos > indexTraceThreshold) { + indexLogger.trace("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog)); + } } } diff --git a/core/src/main/java/org/elasticsearch/index/engine/Engine.java b/core/src/main/java/org/elasticsearch/index/engine/Engine.java index 8b4150dc1b4..cfb2532280e 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -880,11 +880,13 @@ public abstract class Engine implements Closeable { return this.endTime; } - abstract String type(); + public abstract String type(); abstract String id(); - abstract TYPE operationType(); + public abstract TYPE operationType(); + + public abstract String toString(); } public static class Index extends Operation { @@ -925,7 +927,7 @@ public abstract class Engine implements Closeable { } @Override - TYPE operationType() { + public TYPE operationType() { return TYPE.INDEX; } @@ -989,6 +991,10 @@ public abstract class Engine implements Closeable { return isRetry; } + @Override + public String toString() { + return "index [{" + type() + "}][{" + id()+ "}] [{" + docs() + "}]"; + } } public static class Delete extends Operation { @@ -1023,10 +1029,15 @@ public abstract class Engine implements Closeable { } @Override - TYPE operationType() { + public TYPE operationType() { return TYPE.DELETE; } + @Override + public String toString() { + return "delete [{"+ uid().text() +"}]"; + } + public void updateVersion(long version, boolean found) { updateVersion(version); this.found = found; diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 0eb8d1b076e..adc1fa876cb 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -536,29 +536,36 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl return new Engine.Index(uid, doc, version, versionType, origin, startTime, autoGeneratedIdTimestamp, isRetry); } - public void index(Engine.Index index) { - ensureWriteAllowed(index); + public void execute(Engine.Operation operation) { + ensureWriteAllowed(operation); Engine engine = getEngine(); - index(engine, index); + execute(engine, operation); } - private void index(Engine engine, Engine.Index index) { + private void execute(Engine engine, Engine.Operation operation) { active.set(true); - index = indexingOperationListeners.preIndex(index); + indexingOperationListeners.preOperation(operation); try { if (logger.isTraceEnabled()) { - logger.trace("index [{}][{}]{}", index.type(), index.id(), index.docs()); + logger.trace(operation.toString()); } - engine.index(index); - index.endTime(System.nanoTime()); + switch (operation.operationType()) { + case INDEX: + engine.index(((Engine.Index) operation)); + break; + case DELETE: + engine.delete(((Engine.Delete) operation)); + break; + } + operation.endTime(System.nanoTime()); } catch (Exception e) { - indexingOperationListeners.postIndex(index, e); + indexingOperationListeners.postOperation(operation, e); throw e; } - if (index.hasFailure()) { - indexingOperationListeners.postIndex(index, index.getFailure()); + if (operation.hasFailure()) { + indexingOperationListeners.postOperation(operation, operation.getFailure()); } else { - indexingOperationListeners.postIndex(index, index.isCreated()); + indexingOperationListeners.postOperation(operation); } } @@ -584,32 +591,6 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl return new Engine.Delete(type, id, uid, version, versionType, origin, startTime, false); } - public void delete(Engine.Delete delete) { - ensureWriteAllowed(delete); - Engine engine = getEngine(); - delete(engine, delete); - } - - private void delete(Engine engine, Engine.Delete delete) { - active.set(true); - delete = indexingOperationListeners.preDelete(delete); - try { - if (logger.isTraceEnabled()) { - logger.trace("delete [{}]", delete.uid().text()); - } - engine.delete(delete); - delete.endTime(System.nanoTime()); - } catch (Exception e) { - indexingOperationListeners.postDelete(delete, e); - throw e; - } - if (delete.hasFailure()) { - indexingOperationListeners.postDelete(delete, delete.getFailure()); - } else { - indexingOperationListeners.postDelete(delete); - } - } - public Engine.GetResult get(Engine.Get get) { readAllowed(); return getEngine().get(get, this::acquireSearcher); @@ -1841,12 +1822,12 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl @Override protected void index(Engine engine, Engine.Index engineIndex) { - IndexShard.this.index(engine, engineIndex); + IndexShard.this.execute(engine, engineIndex); } @Override protected void delete(Engine engine, Engine.Delete engineDelete) { - IndexShard.this.delete(engine, engineDelete); + IndexShard.this.execute(engine, engineDelete); } } } diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java b/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java index 042ddec924e..ec0fab2629f 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java @@ -25,49 +25,19 @@ import org.elasticsearch.index.engine.Engine; import java.util.List; -/** - * An indexing listener for indexing, delete, events. - */ +/** An engine operation listener for index and delete execution. */ public interface IndexingOperationListener { - /** - * Called before the indexing occurs. - */ - default Engine.Index preIndex(Engine.Index operation) { - return operation; - } + /** Called before executing index or delete operation */ + default void preOperation(Engine.Operation operation) {} - /** - * Called after the indexing operation occurred. - */ - default void postIndex(Engine.Index index, boolean created) {} + /** Called after executing index or delete operation */ + default void postOperation(Engine.Operation operation) {} - /** - * Called after the indexing operation occurred with exception. - */ - default void postIndex(Engine.Index index, Exception ex) {} + /** Called after index or delete operation failed with exception */ + default void postOperation(Engine.Operation operation, Exception ex) {} - /** - * Called before the delete occurs. - */ - default Engine.Delete preDelete(Engine.Delete delete) { - return delete; - } - - - /** - * Called after the delete operation occurred. - */ - default void postDelete(Engine.Delete delete) {} - - /** - * Called after the delete operation occurred with exception. - */ - default void postDelete(Engine.Delete delete, Exception ex) {} - - /** - * A Composite listener that multiplexes calls to each of the listeners methods. - */ + /** A Composite listener that multiplexes calls to each of the listeners methods. */ final class CompositeListener implements IndexingOperationListener{ private final List listeners; private final Logger logger; @@ -78,79 +48,40 @@ public interface IndexingOperationListener { } @Override - public Engine.Index preIndex(Engine.Index operation) { + public void preOperation(Engine.Operation operation) { assert operation != null; for (IndexingOperationListener listener : listeners) { try { - listener.preIndex(operation); + listener.preOperation(operation); } catch (Exception e) { - logger.warn((Supplier) () -> new ParameterizedMessage("preIndex listener [{}] failed", listener), e); - } - } - return operation; - } - - @Override - public void postIndex(Engine.Index index, boolean created) { - assert index != null; - for (IndexingOperationListener listener : listeners) { - try { - listener.postIndex(index, created); - } catch (Exception e) { - logger.warn((Supplier) () -> new ParameterizedMessage("postIndex listener [{}] failed", listener), e); + logger.warn((Supplier) () -> new ParameterizedMessage("preOperation listener [{}] failed", listener), e); } } } @Override - public void postIndex(Engine.Index index, Exception ex) { - assert index != null && ex != null; + public void postOperation(Engine.Operation operation) { + assert operation != null; for (IndexingOperationListener listener : listeners) { try { - listener.postIndex(index, ex); + listener.postOperation(operation); + } catch (Exception e) { + logger.warn((Supplier) () -> new ParameterizedMessage("postOperation listener [{}] failed", listener), e); + } + } + } + + @Override + public void postOperation(Engine.Operation operation, Exception ex) { + assert operation != null && ex != null; + for (IndexingOperationListener listener : listeners) { + try { + listener.postOperation(operation, ex); } catch (Exception inner) { inner.addSuppressed(ex); - logger.warn((Supplier) () -> new ParameterizedMessage("postIndex listener [{}] failed", listener), inner); - } - } - } - - @Override - public Engine.Delete preDelete(Engine.Delete delete) { - assert delete != null; - for (IndexingOperationListener listener : listeners) { - try { - listener.preDelete(delete); - } catch (Exception e) { - logger.warn((Supplier) () -> new ParameterizedMessage("preDelete listener [{}] failed", listener), e); - } - } - return delete; - } - - @Override - public void postDelete(Engine.Delete delete) { - assert delete != null; - for (IndexingOperationListener listener : listeners) { - try { - listener.postDelete(delete); - } catch (Exception e) { - logger.warn((Supplier) () -> new ParameterizedMessage("postDelete listener [{}] failed", listener), e); - } - } - } - - @Override - public void postDelete(Engine.Delete delete, Exception ex) { - assert delete != null && ex != null; - for (IndexingOperationListener listener : listeners) { - try { - listener.postDelete(delete, ex); - } catch (Exception inner) { - inner.addSuppressed(ex); - logger.warn((Supplier) () -> new ParameterizedMessage("postDelete listener [{}] failed", listener), inner); + logger.warn((Supplier) () -> new ParameterizedMessage("postOperation listener [{}] failed", listener), inner); } } } } -} +} \ No newline at end of file diff --git a/core/src/main/java/org/elasticsearch/index/shard/InternalIndexingStats.java b/core/src/main/java/org/elasticsearch/index/shard/InternalIndexingStats.java index f62b8f7fe3c..cd1b1526e0c 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/InternalIndexingStats.java +++ b/core/src/main/java/org/elasticsearch/index/shard/InternalIndexingStats.java @@ -65,63 +65,60 @@ final class InternalIndexingStats implements IndexingOperationListener { } @Override - public Engine.Index preIndex(Engine.Index operation) { + public void preOperation(Engine.Operation operation) { if (!operation.origin().isRecovery()) { - totalStats.indexCurrent.inc(); - typeStats(operation.type()).indexCurrent.inc(); - } - return operation; - } - - @Override - public void postIndex(Engine.Index index, boolean created) { - if (!index.origin().isRecovery()) { - long took = index.endTime() - index.startTime(); - totalStats.indexMetric.inc(took); - totalStats.indexCurrent.dec(); - StatsHolder typeStats = typeStats(index.type()); - typeStats.indexMetric.inc(took); - typeStats.indexCurrent.dec(); + StatsHolder statsHolder = typeStats(operation.type()); + switch (operation.operationType()) { + case INDEX: + totalStats.indexCurrent.inc(); + statsHolder.indexCurrent.inc(); + break; + case DELETE: + totalStats.deleteCurrent.inc(); + statsHolder.deleteCurrent.inc(); + break; + } } } @Override - public void postIndex(Engine.Index index, Exception ex) { - if (!index.origin().isRecovery()) { - totalStats.indexCurrent.dec(); - typeStats(index.type()).indexCurrent.dec(); - totalStats.indexFailed.inc(); - typeStats(index.type()).indexFailed.inc(); + public void postOperation(Engine.Operation operation) { + if (!operation.origin().isRecovery()) { + long took = operation.endTime() - operation.startTime(); + StatsHolder typeStats = typeStats(operation.type()); + switch (operation.operationType()) { + case INDEX: + totalStats.indexMetric.inc(took); + totalStats.indexCurrent.dec(); + typeStats.indexMetric.inc(took); + typeStats.indexCurrent.dec(); + break; + case DELETE: + totalStats.deleteMetric.inc(took); + totalStats.deleteCurrent.dec(); + typeStats.deleteMetric.inc(took); + typeStats.deleteCurrent.dec(); + break; + } } } @Override - public Engine.Delete preDelete(Engine.Delete delete) { - if (!delete.origin().isRecovery()) { - totalStats.deleteCurrent.inc(); - typeStats(delete.type()).deleteCurrent.inc(); - } - return delete; - - } - - @Override - public void postDelete(Engine.Delete delete) { - if (!delete.origin().isRecovery()) { - long took = delete.endTime() - delete.startTime(); - totalStats.deleteMetric.inc(took); - totalStats.deleteCurrent.dec(); - StatsHolder typeStats = typeStats(delete.type()); - typeStats.deleteMetric.inc(took); - typeStats.deleteCurrent.dec(); - } - } - - @Override - public void postDelete(Engine.Delete delete, Exception ex) { - if (!delete.origin().isRecovery()) { - totalStats.deleteCurrent.dec(); - typeStats(delete.type()).deleteCurrent.dec(); + public void postOperation(Engine.Operation operation, Exception ex) { + if (!operation.origin().isRecovery()) { + StatsHolder statsHolder = typeStats(operation.type()); + switch (operation.operationType()) { + case INDEX: + totalStats.indexCurrent.dec(); + statsHolder.indexCurrent.dec(); + totalStats.indexFailed.inc(); + statsHolder.indexFailed.inc(); + break; + case DELETE: + totalStats.deleteCurrent.dec(); + statsHolder.deleteCurrent.dec(); + break; + } } } @@ -158,10 +155,5 @@ final class InternalIndexingStats implements IndexingOperationListener { deleteMetric.count(), TimeUnit.NANOSECONDS.toMillis(deleteMetric.sum()), deleteCurrent.count(), noopUpdates.count(), isThrottled, TimeUnit.MILLISECONDS.toMillis(currentThrottleMillis)); } - - void clear() { - indexMetric.clear(); - deleteMetric.clear(); - } } } diff --git a/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java b/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java index 3b4258a8bdf..da5a9b7c28e 100644 --- a/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java +++ b/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java @@ -189,11 +189,6 @@ public class IndexingMemoryController extends AbstractComponent implements Index statusChecker.run(); } - /** called by IndexShard to record that this many bytes were written to translog */ - public void bytesWritten(int bytes) { - statusChecker.bytesWritten(bytes); - } - /** Asks this shard to throttle indexing to one thread */ protected void activateThrottling(IndexShard shard) { shard.activateThrottling(); @@ -205,17 +200,8 @@ public class IndexingMemoryController extends AbstractComponent implements Index } @Override - public void postIndex(Engine.Index index, boolean created) { - recordOperationBytes(index); - } - - @Override - public void postDelete(Engine.Delete delete) { - recordOperationBytes(delete); - } - - private void recordOperationBytes(Engine.Operation op) { - bytesWritten(op.sizeInBytes()); + public void postOperation(Engine.Operation operation) { + statusChecker.bytesWritten(operation.sizeInBytes()); } private static final class ShardAndBytesUsed implements Comparable { diff --git a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java index 5e9d1ffaf9e..1cf56c50234 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java +++ b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java @@ -233,9 +233,10 @@ public class IndexModuleTests extends ESTestCase { AtomicBoolean executed = new AtomicBoolean(false); IndexingOperationListener listener = new IndexingOperationListener() { @Override - public Engine.Index preIndex(Engine.Index operation) { - executed.set(true); - return operation; + public void preOperation(Engine.Operation operation) { + if (operation.operationType() == Engine.Operation.TYPE.INDEX) { + executed.set(true); + } } }; module.addIndexOperationListener(listener); @@ -251,7 +252,7 @@ public class IndexModuleTests extends ESTestCase { Engine.Index index = new Engine.Index(new Term("_uid", "1"), null); for (IndexingOperationListener l : indexService.getIndexOperationListeners()) { - l.preIndex(index); + l.preOperation(index); } assertTrue(executed.get()); indexService.close("simon says", false); diff --git a/core/src/test/java/org/elasticsearch/index/mapper/TextFieldMapperTests.java b/core/src/test/java/org/elasticsearch/index/mapper/TextFieldMapperTests.java index 846d2c56669..7313b7ec9bf 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/TextFieldMapperTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/TextFieldMapperTests.java @@ -219,7 +219,7 @@ public class TextFieldMapperTests extends ESSingleNodeTestCase { assertEquals("b", fields[1].stringValue()); IndexShard shard = indexService.getShard(0); - shard.index(new Engine.Index(new Term("_uid", "1"), doc)); + shard.execute(new Engine.Index(new Term("_uid", "1"), doc)); shard.refresh("test"); try (Engine.Searcher searcher = shard.acquireSearcher("test")) { LeafReader leaf = searcher.getDirectoryReader().leaves().get(0).reader(); @@ -258,7 +258,7 @@ public class TextFieldMapperTests extends ESSingleNodeTestCase { assertEquals("b", fields[1].stringValue()); IndexShard shard = indexService.getShard(0); - shard.index(new Engine.Index(new Term("_uid", "1"), doc)); + shard.execute(new Engine.Index(new Term("_uid", "1"), doc)); shard.refresh("test"); try (Engine.Searcher searcher = shard.acquireSearcher("test")) { LeafReader leaf = searcher.getDirectoryReader().leaves().get(0).reader(); diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java index 2248ff156ac..900bea75724 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java @@ -321,7 +321,7 @@ public class IndexShardIT extends ESSingleNodeTestCase { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, new ParseContext.Document(), new BytesArray(new byte[]{1}), null); Engine.Index index = new Engine.Index(new Term("_uid", "1"), doc); - shard.index(index); + shard.execute(index); assertTrue(shard.shouldFlush()); assertEquals(2, shard.getEngine().getTranslog().totalOperations()); client().prepareIndex("test", "test", "2").setSource("{}").setRefreshPolicy(randomBoolean() ? IMMEDIATE : NONE).get(); @@ -406,23 +406,8 @@ public class IndexShardIT extends ESSingleNodeTestCase { AtomicReference shardRef = new AtomicReference<>(); List failures = new ArrayList<>(); IndexingOperationListener listener = new IndexingOperationListener() { - @Override - public void postIndex(Engine.Index index, boolean created) { - try { - assertNotNull(shardRef.get()); - // this is all IMC needs to do - check current memory and refresh - assertTrue(shardRef.get().getIndexBufferRAMBytesUsed() > 0); - shardRef.get().refresh("test"); - } catch (Exception e) { - failures.add(e); - throw e; - } - } - - - @Override - public void postDelete(Engine.Delete delete) { + public void postOperation(Engine.Operation operation) { try { assertNotNull(shardRef.get()); // this is all IMC needs to do - check current memory and refresh diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index b531aae0532..99ab4e03a9c 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -560,40 +560,43 @@ public class IndexShardTests extends IndexShardTestCase { shard.close("simon says", true); shard = reinitShard(shard, new IndexingOperationListener() { @Override - public Engine.Index preIndex(Engine.Index operation) { - preIndex.incrementAndGet(); - return operation; - } - - @Override - public void postIndex(Engine.Index index, boolean created) { - if (created) { - postIndexCreate.incrementAndGet(); - } else { - postIndexUpdate.incrementAndGet(); + public void preOperation(Engine.Operation operation) { + switch (operation.operationType()) { + case INDEX: + preIndex.incrementAndGet(); + break; + case DELETE: + preDelete.incrementAndGet(); + break; } } @Override - public void postIndex(Engine.Index index, Exception ex) { - postIndexException.incrementAndGet(); + public void postOperation(Engine.Operation operation) { + switch (operation.operationType()) { + case INDEX: + if (((Engine.Index) operation).isCreated()) { + postIndexCreate.incrementAndGet(); + } else { + postIndexUpdate.incrementAndGet(); + } + break; + case DELETE: + postDelete.incrementAndGet(); + break; + } } @Override - public Engine.Delete preDelete(Engine.Delete delete) { - preDelete.incrementAndGet(); - return delete; - } - - @Override - public void postDelete(Engine.Delete delete) { - postDelete.incrementAndGet(); - } - - @Override - public void postDelete(Engine.Delete delete, Exception ex) { - postDeleteException.incrementAndGet(); - + public void postOperation(Engine.Operation operation, Exception ex) { + switch (operation.operationType()) { + case INDEX: + postIndexException.incrementAndGet(); + break; + case DELETE: + postDeleteException.incrementAndGet(); + break; + } } }); recoveryShardFromStore(shard); @@ -601,7 +604,7 @@ public class IndexShardTests extends IndexShardTestCase { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, new ParseContext.Document(), new BytesArray(new byte[]{1}), null); Engine.Index index = new Engine.Index(new Term("_uid", "1"), doc); - shard.index(index); + shard.execute(index); assertEquals(1, preIndex.get()); assertEquals(1, postIndexCreate.get()); assertEquals(0, postIndexUpdate.get()); @@ -610,7 +613,7 @@ public class IndexShardTests extends IndexShardTestCase { assertEquals(0, postDelete.get()); assertEquals(0, postDeleteException.get()); - shard.index(index); + shard.execute(index); assertEquals(2, preIndex.get()); assertEquals(1, postIndexCreate.get()); assertEquals(1, postIndexUpdate.get()); @@ -620,7 +623,7 @@ public class IndexShardTests extends IndexShardTestCase { assertEquals(0, postDeleteException.get()); Engine.Delete delete = new Engine.Delete("test", "1", new Term("_uid", "1")); - shard.delete(delete); + shard.execute(delete); assertEquals(2, preIndex.get()); assertEquals(1, postIndexCreate.get()); @@ -634,7 +637,7 @@ public class IndexShardTests extends IndexShardTestCase { shard.state = IndexShardState.STARTED; // It will generate exception try { - shard.index(index); + shard.execute(index); fail(); } catch (IllegalIndexShardStateException e) { @@ -648,7 +651,7 @@ public class IndexShardTests extends IndexShardTestCase { assertEquals(1, postDelete.get()); assertEquals(0, postDeleteException.get()); try { - shard.delete(delete); + shard.execute(delete); fail(); } catch (IllegalIndexShardStateException e) { @@ -1123,26 +1126,27 @@ public class IndexShardTests extends IndexShardTestCase { final AtomicInteger postDelete = new AtomicInteger(); IndexingOperationListener listener = new IndexingOperationListener() { @Override - public Engine.Index preIndex(Engine.Index operation) { - preIndex.incrementAndGet(); - return operation; + public void preOperation(Engine.Operation operation) { + switch (operation.operationType()) { + case INDEX: + preIndex.incrementAndGet(); + break; + case DELETE: + preDelete.incrementAndGet(); + break; + } } @Override - public void postIndex(Engine.Index index, boolean created) { - postIndex.incrementAndGet(); - } - - @Override - public Engine.Delete preDelete(Engine.Delete delete) { - preDelete.incrementAndGet(); - return delete; - } - - @Override - public void postDelete(Engine.Delete delete) { - postDelete.incrementAndGet(); - + public void postOperation(Engine.Operation operation) { + switch (operation.operationType()) { + case INDEX: + postIndex.incrementAndGet(); + break; + case DELETE: + postDelete.incrementAndGet(); + break; + } } }; final IndexShard newShard = reinitShard(shard, listener); diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java index d1cf8b32f58..bb652c6630b 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java @@ -40,63 +40,55 @@ public class IndexingOperationListenerTests extends ESTestCase{ AtomicInteger postDeleteException = new AtomicInteger(); IndexingOperationListener listener = new IndexingOperationListener() { @Override - public Engine.Index preIndex(Engine.Index operation) { - preIndex.incrementAndGet(); - return operation; + public void preOperation(Engine.Operation operation) { + switch (operation.operationType()) { + case INDEX: + preIndex.incrementAndGet(); + break; + case DELETE: + preDelete.incrementAndGet(); + break; + } } @Override - public void postIndex(Engine.Index index, boolean created) { - postIndex.incrementAndGet(); + public void postOperation(Engine.Operation operation) { + switch (operation.operationType()) { + case INDEX: + postIndex.incrementAndGet(); + break; + case DELETE: + postDelete.incrementAndGet(); + break; + } } @Override - public void postIndex(Engine.Index index, Exception ex) { - postIndexException.incrementAndGet(); - } - - @Override - public Engine.Delete preDelete(Engine.Delete delete) { - preDelete.incrementAndGet(); - return delete; - } - - @Override - public void postDelete(Engine.Delete delete) { - postDelete.incrementAndGet(); - } - - @Override - public void postDelete(Engine.Delete delete, Exception ex) { - postDeleteException.incrementAndGet(); + public void postOperation(Engine.Operation operation, Exception ex) { + switch (operation.operationType()) { + case INDEX: + postIndexException.incrementAndGet(); + break; + case DELETE: + postDeleteException.incrementAndGet(); + break; + } } }; IndexingOperationListener throwingListener = new IndexingOperationListener() { @Override - public Engine.Index preIndex(Engine.Index operation) { + public void preOperation(Engine.Operation operation) { throw new RuntimeException(); } @Override - public void postIndex(Engine.Index index, boolean created) { - throw new RuntimeException(); } - - @Override - public void postIndex(Engine.Index index, Exception ex) { - throw new RuntimeException(); } - - @Override - public Engine.Delete preDelete(Engine.Delete delete) { + public void postOperation(Engine.Operation operation) { throw new RuntimeException(); } @Override - public void postDelete(Engine.Delete delete) { - throw new RuntimeException(); } - - @Override - public void postDelete(Engine.Delete delete, Exception ex) { + public void postOperation(Engine.Operation operation, Exception ex) { throw new RuntimeException(); } }; @@ -111,7 +103,7 @@ public class IndexingOperationListenerTests extends ESTestCase{ IndexingOperationListener.CompositeListener compositeListener = new IndexingOperationListener.CompositeListener(indexingOperationListeners, logger); Engine.Delete delete = new Engine.Delete("test", "1", new Term("_uid", "1")); Engine.Index index = new Engine.Index(new Term("_uid", "1"), null); - compositeListener.postDelete(delete); + compositeListener.postOperation(delete); assertEquals(0, preIndex.get()); assertEquals(0, postIndex.get()); assertEquals(0, postIndexException.get()); @@ -119,7 +111,7 @@ public class IndexingOperationListenerTests extends ESTestCase{ assertEquals(2, postDelete.get()); assertEquals(0, postDeleteException.get()); - compositeListener.postDelete(delete, new RuntimeException()); + compositeListener.postOperation(delete, new RuntimeException()); assertEquals(0, preIndex.get()); assertEquals(0, postIndex.get()); assertEquals(0, postIndexException.get()); @@ -127,7 +119,7 @@ public class IndexingOperationListenerTests extends ESTestCase{ assertEquals(2, postDelete.get()); assertEquals(2, postDeleteException.get()); - compositeListener.preDelete(delete); + compositeListener.preOperation(delete); assertEquals(0, preIndex.get()); assertEquals(0, postIndex.get()); assertEquals(0, postIndexException.get()); @@ -135,7 +127,7 @@ public class IndexingOperationListenerTests extends ESTestCase{ assertEquals(2, postDelete.get()); assertEquals(2, postDeleteException.get()); - compositeListener.postIndex(index, false); + compositeListener.postOperation(index); assertEquals(0, preIndex.get()); assertEquals(2, postIndex.get()); assertEquals(0, postIndexException.get()); @@ -143,7 +135,7 @@ public class IndexingOperationListenerTests extends ESTestCase{ assertEquals(2, postDelete.get()); assertEquals(2, postDeleteException.get()); - compositeListener.postIndex(index, new RuntimeException()); + compositeListener.postOperation(index, new RuntimeException()); assertEquals(0, preIndex.get()); assertEquals(2, postIndex.get()); assertEquals(2, postIndexException.get()); @@ -151,7 +143,7 @@ public class IndexingOperationListenerTests extends ESTestCase{ assertEquals(2, postDelete.get()); assertEquals(2, postDeleteException.get()); - compositeListener.preIndex(index); + compositeListener.preOperation(index); assertEquals(2, preIndex.get()); assertEquals(2, postIndex.get()); assertEquals(2, postIndexException.get()); diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/CancelTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/CancelTests.java index 8da47f1eeaf..dd1250999ba 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/CancelTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/CancelTests.java @@ -197,25 +197,14 @@ public class CancelTests extends ReindexTestCase { } public static class BlockingOperationListener implements IndexingOperationListener { - @Override - public Engine.Index preIndex(Engine.Index index) { - return preCheck(index, index.type()); - } - - @Override - public Engine.Delete preDelete(Engine.Delete delete) { - return preCheck(delete, delete.type()); - } - - private T preCheck(T operation, String type) { - if ((TYPE.equals(type) == false) || (operation.origin() != Origin.PRIMARY)) { - return operation; + public void preOperation(Engine.Operation operation) { + if ((TYPE.equals(operation.type()) == false) || (operation.origin() != Origin.PRIMARY)) { + return; } - try { if (ALLOWED_OPERATIONS.tryAcquire(30, TimeUnit.SECONDS)) { - return operation; + return; } } catch (InterruptedException e) { throw new RuntimeException(e); diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index fbb87d9f8d1..c520e3fbbfc 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -451,7 +451,7 @@ public abstract class IndexShardTestCase extends ESTestCase { SourceToParse.source(SourceToParse.Origin.PRIMARY, shard.shardId().getIndexName(), type, id, new BytesArray(source)), 1, VersionType.EXTERNAL, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); } - shard.index(index); + shard.execute(index); return index; } @@ -462,7 +462,7 @@ public abstract class IndexShardTestCase extends ESTestCase { } else { delete = shard.prepareDeleteOnPrimary(type, id, 1, VersionType.EXTERNAL); } - shard.delete(delete); + shard.execute(delete); return delete; } From 26f51187069990315c5832023c1b7e369fc9f674 Mon Sep 17 00:00:00 2001 From: Areek Zillur Date: Sat, 15 Oct 2016 13:31:15 -0400 Subject: [PATCH 005/132] remove declaring unchecked exception due to engine write operations --- .../elasticsearch/action/bulk/TransportShardBulkAction.java | 4 ++-- core/src/main/java/org/elasticsearch/index/engine/Engine.java | 4 ++-- .../java/org/elasticsearch/index/engine/InternalEngine.java | 4 ++-- .../java/org/elasticsearch/index/engine/ShadowEngine.java | 4 ++-- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 7401fc6f63c..cbedfb895d8 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -27,7 +27,6 @@ import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexResponse; -import org.elasticsearch.action.index.TransportIndexAction; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.replication.TransportWriteAction; import org.elasticsearch.action.support.replication.ReplicationResponse.ShardInfo; @@ -60,6 +59,7 @@ import java.util.Map; import static org.elasticsearch.action.delete.TransportDeleteAction.executeDeleteRequestOnPrimary; import static org.elasticsearch.action.delete.TransportDeleteAction.executeDeleteRequestOnReplica; import static org.elasticsearch.action.index.TransportIndexAction.executeIndexRequestOnPrimary; +import static org.elasticsearch.action.index.TransportIndexAction.executeIndexRequestOnReplica; import static org.elasticsearch.action.support.replication.ReplicationOperation.ignoreReplicaException; import static org.elasticsearch.action.support.replication.ReplicationOperation.isConflictException; @@ -288,7 +288,7 @@ public class TransportShardBulkAction extends TransportWriteAction Date: Sat, 15 Oct 2016 15:20:38 -0400 Subject: [PATCH 006/132] Documentation for handling engine write failures --- .../index/engine/InternalEngine.java | 43 +++++++++++-------- 1 file changed, 26 insertions(+), 17 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 18b9ad1a869..7c023b55bc0 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -57,6 +57,7 @@ import org.elasticsearch.common.util.concurrent.KeyedLock; import org.elasticsearch.common.util.concurrent.ReleasableLock; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.VersionType; +import org.elasticsearch.index.engine.Engine.Operation; import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.merge.MergeStats; import org.elasticsearch.index.merge.OnGoingMerge; @@ -414,25 +415,33 @@ public class InternalEngine extends Engine { } /** - * When indexing a document into Lucene, Lucene distinguishes between environment related errors - * (like out of disk space) and document specific errors (like analysis chain problems) by setting - * the IndexWriter.getTragicEvent() value for the former. maybeFailEngine checks for these kind of - * errors and returns true if that is the case. We use that to indicate a document level failure - * and set the error in operation.setFailure. In case of environment related errors, the failure - * is bubbled up + * Handle failures executing write operations, distinguish persistent engine (environment) failures + * from document (request) specific failures. + * Write failures that fail the engine as a side-effect, are thrown wrapped in {@link OperationFailedEngineException} + * and document specific failures are captured through {@link Operation#setFailure(Exception)} to be handled + * at the transport level. */ - private void handleOperationFailure(final Operation operation, final Exception e) throws OperationFailedEngineException { + private void handleOperationFailure(final Operation operation, final Exception failure) { + boolean isEnvironmentFailure; try { - if (maybeFailEngine(operation.operationType().getLowercase(), e)) { - throw new OperationFailedEngineException(shardId, - operation.operationType().getLowercase(), operation.type(), operation.id(), e); - } else { - operation.setFailure(e); - } + // When indexing a document into Lucene, Lucene distinguishes between environment related errors + // (like out of disk space) and document specific errors (like analysis chain problems) by setting + // the IndexWriter.getTragicEvent() value for the former. maybeFailEngine checks for these kind of + // errors and returns true if that is the case. We use that to indicate a document level failure + // and set the error in operation.setFailure. In case of environment related errors, the failure + // is bubbled up + isEnvironmentFailure = (failure instanceof IllegalStateException || failure instanceof IOException) + && maybeFailEngine(operation.operationType().getLowercase(), failure); } catch (Exception inner) { - e.addSuppressed(inner); - throw new OperationFailedEngineException(shardId, - operation.operationType().getLowercase(), operation.type(), operation.id(), e); + // we failed checking whether the failure can fail the engine, treat it as a persistent engine failure + isEnvironmentFailure = true; + failure.addSuppressed(inner); + } + if (isEnvironmentFailure) { + throw new OperationFailedEngineException(shardId, operation.operationType().getLowercase(), + operation.type(), operation.id(), failure); + } else { + operation.setFailure(failure); } } @@ -540,7 +549,7 @@ public class InternalEngine extends Engine { } } - private long updateVersion(Engine.Operation op, long currentVersion, long expectedVersion) { + private long updateVersion(Operation op, long currentVersion, long expectedVersion) { final long updatedVersion = op.versionType().updateVersion(currentVersion, expectedVersion); op.updateVersion(updatedVersion); return updatedVersion; From 63c07282920b41576dc8fedaaf0add30caa86051 Mon Sep 17 00:00:00 2001 From: Areek Zillur Date: Thu, 20 Oct 2016 20:53:39 -0400 Subject: [PATCH 007/132] Simplify TransportWriteAction request handling This commit reduces classes to handle write operation results in TransportWriteAction, this comes at the cost of handling write operations in TransportShardBulkAction. Now parsing, mapping failures (which happen before executing engine write operation) are communicated via a failure operation type while transient operation failures are set on the index/delete operations. --- .../flush/TransportShardFlushAction.java | 4 +- .../refresh/TransportShardRefreshAction.java | 4 +- .../action/bulk/TransportShardBulkAction.java | 208 ++++++++++-------- .../action/delete/TransportDeleteAction.java | 34 +-- .../action/index/TransportIndexAction.java | 84 +++---- .../TransportReplicationAction.java | 12 +- .../replication/TransportWriteAction.java | 123 +++-------- .../elasticsearch/index/engine/Engine.java | 45 +++- .../elasticsearch/index/shard/IndexShard.java | 11 +- .../TransportReplicationActionTests.java | 10 +- .../TransportWriteActionTests.java | 8 +- .../ESIndexLevelReplicationTestCase.java | 15 +- .../indices/stats/IndexStatsIT.java | 2 +- .../index/shard/IndexShardTestCase.java | 4 +- 14 files changed, 279 insertions(+), 285 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/admin/indices/flush/TransportShardFlushAction.java b/core/src/main/java/org/elasticsearch/action/admin/indices/flush/TransportShardFlushAction.java index 1ec7186393f..84e5a4df408 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/indices/flush/TransportShardFlushAction.java +++ b/core/src/main/java/org/elasticsearch/action/admin/indices/flush/TransportShardFlushAction.java @@ -54,14 +54,14 @@ public class TransportShardFlushAction extends TransportReplicationAction onPrimaryShard(BulkShardRequest request, IndexShard primary) throws Exception { + protected WritePrimaryResult onPrimaryShard(BulkShardRequest request, IndexShard primary) throws Exception { final IndexMetaData metaData = primary.indexSettings().getIndexMetaData(); long[] preVersions = new long[request.items().length]; @@ -116,7 +117,7 @@ public class TransportShardBulkAction extends TransportWriteAction(response, location); + return new WritePrimaryResult(request, response, location, null, primary); } /** Executes bulk item requests and handles request execution exceptions */ @@ -128,48 +129,117 @@ public class TransportShardBulkAction extends TransportWriteAction primaryOperationResult; + final Engine.Operation operation; + final DocWriteResponse response; switch (itemRequest.opType()) { case CREATE: case INDEX: - primaryOperationResult = executeIndexRequestOnPrimary(((IndexRequest) itemRequest), primary, mappingUpdatedAction); + final IndexRequest indexRequest = (IndexRequest) itemRequest; + operation = executeIndexRequestOnPrimary(indexRequest, primary, mappingUpdatedAction); + response = operation.hasFailure() ? null + : new IndexResponse(primary.shardId(), indexRequest.type(), indexRequest.id(), + operation.version(), ((Engine.Index) operation).isCreated()); break; case UPDATE: - int maxAttempts = ((UpdateRequest) itemRequest).retryOnConflict(); - PrimaryOperationResult shardUpdateOperation = null; + Engine.Operation updateOperation = null; + UpdateResponse updateResponse = null; + UpdateRequest updateRequest = (UpdateRequest) itemRequest; + int maxAttempts = updateRequest.retryOnConflict(); for (int attemptCount = 0; attemptCount <= maxAttempts; attemptCount++) { - shardUpdateOperation = shardUpdateOperation(metaData, primary, request, requestIndex, ((UpdateRequest) itemRequest)); - if (shardUpdateOperation.success() - || shardUpdateOperation.getFailure() instanceof VersionConflictEngineException == false) { - break; + try { + // translate and execute operation + UpdateHelper.Result translate = updateHelper.prepare(updateRequest, primary, threadPool::estimatedTimeInMillis); + updateOperation = shardUpdateOperation(metaData, primary, request, updateRequest, translate); + if (updateOperation == null) { + // this is a noop operation + updateResponse = translate.action(); + } else { + if (updateOperation.hasFailure() == false) { + // enrich update response + switch (updateOperation.operationType()) { + case INDEX: + IndexRequest updateIndexRequest = translate.action(); + final IndexResponse indexResponse = new IndexResponse(primary.shardId(), + updateIndexRequest.type(), updateIndexRequest.id(), + updateOperation.version(), ((Engine.Index) updateOperation).isCreated()); + BytesReference indexSourceAsBytes = updateIndexRequest.source(); + updateResponse = new UpdateResponse(indexResponse.getShardInfo(), + indexResponse.getShardId(), indexResponse.getType(), indexResponse.getId(), + indexResponse.getVersion(), indexResponse.getResult()); + if ((updateRequest.fetchSource() != null && updateRequest.fetchSource().fetchSource()) || + (updateRequest.fields() != null && updateRequest.fields().length > 0)) { + Tuple> sourceAndContent = + XContentHelper.convertToMap(indexSourceAsBytes, true); + updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, request.index(), + indexResponse.getVersion(), sourceAndContent.v2(), sourceAndContent.v1(), indexSourceAsBytes)); + } + // Replace the update request to the translated index request to execute on the replica. + request.items()[requestIndex] = new BulkItemRequest(request.items()[requestIndex].id(), updateIndexRequest); + break; + case DELETE: + DeleteRequest updateDeleteRequest = translate.action(); + DeleteResponse deleteResponse = new DeleteResponse(primary.shardId(), + updateDeleteRequest.type(), updateDeleteRequest.id(), + updateOperation.version(), ((Engine.Delete) updateOperation).found()); + updateResponse = new UpdateResponse(deleteResponse.getShardInfo(), + deleteResponse.getShardId(), deleteResponse.getType(), deleteResponse.getId(), + deleteResponse.getVersion(), deleteResponse.getResult()); + updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, + request.index(), deleteResponse.getVersion(), translate.updatedSourceAsMap(), + translate.updateSourceContentType(), null)); + // Replace the update request to the translated delete request to execute on the replica. + request.items()[requestIndex] = new BulkItemRequest(request.items()[requestIndex].id(), updateDeleteRequest); + break; + case FAILURE: + break; + } + } else { + // version conflict exception, retry + if (updateOperation.getFailure() instanceof VersionConflictEngineException) { + continue; + } + } + } + break; // out of retry loop + } catch (Exception failure) { + // set to a failure operation + updateOperation = new Engine.Failure(updateRequest.type(), updateRequest.id(), updateRequest.version(), + updateRequest.versionType(), Engine.Operation.Origin.PRIMARY, System.nanoTime(), failure); + break; // out of retry loop } } - if (shardUpdateOperation == null) { - throw new IllegalStateException("version conflict exception should bubble up on last attempt"); - } - primaryOperationResult = shardUpdateOperation; + operation = updateOperation; + response = updateResponse; break; case DELETE: - primaryOperationResult = executeDeleteRequestOnPrimary(((DeleteRequest) itemRequest), primary); + final DeleteRequest deleteRequest = (DeleteRequest) itemRequest; + operation = executeDeleteRequestOnPrimary(deleteRequest, primary); + response = operation.hasFailure() ? null : + new DeleteResponse(request.shardId(), deleteRequest.type(), deleteRequest.id(), + operation.version(), ((Engine.Delete) operation).found()); break; default: throw new IllegalStateException("unexpected opType [" + itemRequest.opType() + "] found"); } - if (primaryOperationResult.success()) { - if (primaryOperationResult.getLocation() != null) { - location = locationToSync(location, primaryOperationResult.getLocation()); + // set item response and handle failures + if (operation == null // in case of a noop update operation + || operation.hasFailure() == false) { + if (operation == null) { + assert response != null; + assert response.getResult() == DocWriteResponse.Result.NOOP + : "only noop operation can have null operation"; } else { - assert primaryOperationResult.getResponse().getResult() == DocWriteResponse.Result.NOOP - : "only noop operation can have null next operation"; + location = locationToSync(location, operation.getTranslogLocation()); } // update the bulk item request because update request execution can mutate the bulk item request BulkItemRequest item = request.items()[requestIndex]; // add the response - setResponse(item, new BulkItemResponse(item.id(), opType, primaryOperationResult.getResponse())); + item.setPrimaryResponse(new BulkItemResponse(item.id(), opType, response)); } else { BulkItemRequest item = request.items()[requestIndex]; DocWriteRequest docWriteRequest = item.request(); - Exception failure = primaryOperationResult.getFailure(); + Exception failure = operation.getFailure(); if (isConflictException(failure)) { logger.trace((Supplier) () -> new ParameterizedMessage("{} failed to execute bulk item ({}) {}", request.shardId(), docWriteRequest.opType().getLowercase(), request), failure); @@ -180,10 +250,8 @@ public class TransportShardBulkAction extends TransportWriteAction shardUpdateOperation(IndexMetaData metaData, IndexShard primary, - BulkShardRequest request, - int requestIndex, UpdateRequest updateRequest) + /** Executes update request, delegating to a index or delete operation after translation */ + private Engine.Operation shardUpdateOperation(IndexMetaData metaData, IndexShard primary, BulkShardRequest request, + UpdateRequest updateRequest, UpdateHelper.Result translate) throws Exception { - final UpdateHelper.Result translate; - try { - translate = updateHelper.prepare(updateRequest, primary, threadPool::estimatedTimeInMillis); - } catch (Exception e) { - return new PrimaryOperationResult<>(e); - } switch (translate.getResponseResult()) { case CREATED: case UPDATED: IndexRequest indexRequest = translate.action(); MappingMetaData mappingMd = metaData.mappingOrDefault(indexRequest.type()); indexRequest.process(mappingMd, allowIdGeneration, request.index()); - PrimaryOperationResult writeResult = executeIndexRequestOnPrimary(indexRequest, primary, mappingUpdatedAction); - if (writeResult.success()) { - BytesReference indexSourceAsBytes = indexRequest.source(); - IndexResponse indexResponse = writeResult.getResponse(); - UpdateResponse update = new UpdateResponse(indexResponse.getShardInfo(), indexResponse.getShardId(), indexResponse.getType(), indexResponse.getId(), indexResponse.getVersion(), indexResponse.getResult()); - if ((updateRequest.fetchSource() != null && updateRequest.fetchSource().fetchSource()) || - (updateRequest.fields() != null && updateRequest.fields().length > 0)) { - Tuple> sourceAndContent = XContentHelper.convertToMap(indexSourceAsBytes, true); - update.setGetResult(updateHelper.extractGetResult(updateRequest, request.index(), indexResponse.getVersion(), sourceAndContent.v2(), sourceAndContent.v1(), indexSourceAsBytes)); - } - // Replace the update request to the translated index request to execute on the replica. - request.items()[requestIndex] = new BulkItemRequest(request.items()[requestIndex].id(), indexRequest); - return new PrimaryOperationResult<>(update, writeResult.getLocation()); - } else { - return writeResult; - } + return executeIndexRequestOnPrimary(indexRequest, primary, mappingUpdatedAction); case DELETED: - DeleteRequest deleteRequest = translate.action(); - PrimaryOperationResult deleteResult = executeDeleteRequestOnPrimary(deleteRequest, primary); - if (deleteResult.success()) { - DeleteResponse response = deleteResult.getResponse(); - UpdateResponse deleteUpdateResponse = new UpdateResponse(response.getShardInfo(), response.getShardId(), response.getType(), response.getId(), response.getVersion(), response.getResult()); - deleteUpdateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, request.index(), response.getVersion(), translate.updatedSourceAsMap(), translate.updateSourceContentType(), null)); - // Replace the update request to the translated delete request to execute on the replica. - request.items()[requestIndex] = new BulkItemRequest(request.items()[requestIndex].id(), deleteRequest); - return new PrimaryOperationResult<>(deleteUpdateResponse, deleteResult.getLocation()); - } else { - return deleteResult; - } + return executeDeleteRequestOnPrimary(translate.action(), primary); case NOOP: - BulkItemRequest item = request.items()[requestIndex]; primary.noopUpdate(updateRequest.type()); - item.setIgnoreOnReplica(); // no need to go to the replica - return new PrimaryOperationResult<>(translate.action(), null); + return null; default: throw new IllegalStateException("Illegal update operation " + translate.getResponseResult()); } } @Override - protected ReplicaOperationResult onReplicaShard(BulkShardRequest request, IndexShard replica) throws Exception { + protected WriteReplicaResult onReplicaShard(BulkShardRequest request, IndexShard replica) throws Exception { Translog.Location location = null; for (int i = 0; i < request.items().length; i++) { BulkItemRequest item = request.items()[i]; @@ -283,27 +311,27 @@ public class TransportShardBulkAction extends TransportWriteAction listener) { ClusterState state = clusterService.state(); if (autoCreateIndex.shouldAutoCreate(request.index(), state)) { - createIndexAction.execute(task, new CreateIndexRequest().index(request.index()).cause("auto(delete api)").masterNodeTimeout(request.timeout()), new ActionListener() { + CreateIndexRequest createIndexRequest = new CreateIndexRequest() + .index(request.index()) + .cause("auto(delete api)") + .masterNodeTimeout(request.timeout()); + createIndexAction.execute(task, createIndexRequest, new ActionListener() { @Override public void onResponse(CreateIndexResponse result) { innerExecute(task, request, listener); @@ -118,36 +122,34 @@ public class TransportDeleteAction extends TransportWriteAction onPrimaryShard(DeleteRequest request, IndexShard primary) { - return executeDeleteRequestOnPrimary(request, primary); + protected WritePrimaryResult onPrimaryShard(DeleteRequest request, IndexShard primary) { + final Engine.Delete operation = executeDeleteRequestOnPrimary(request, primary); + final DeleteResponse response = operation.hasFailure() ? null : + new DeleteResponse(primary.shardId(), request.type(), request.id(), operation.version(), operation.found()); + return new WritePrimaryResult(request, response, operation.getTranslogLocation(), operation.getFailure(), primary); } @Override - protected ReplicaOperationResult onReplicaShard(DeleteRequest request, IndexShard replica) { - return executeDeleteRequestOnReplica(request, replica); + protected WriteReplicaResult onReplicaShard(DeleteRequest request, IndexShard replica) { + final Engine.Operation operation = executeDeleteRequestOnReplica(request, replica); + return new WriteReplicaResult(request, operation.getTranslogLocation(), operation.getFailure(), replica); } - public static PrimaryOperationResult executeDeleteRequestOnPrimary(DeleteRequest request, IndexShard primary) { + public static Engine.Delete executeDeleteRequestOnPrimary(DeleteRequest request, IndexShard primary) { Engine.Delete delete = primary.prepareDeleteOnPrimary(request.type(), request.id(), request.version(), request.versionType()); primary.execute(delete); - if (delete.hasFailure()) { - return new PrimaryOperationResult<>(delete.getFailure()); - } else { + if (delete.hasFailure() == false) { // update the request with the version so it will go to the replicas request.versionType(delete.versionType().versionTypeForReplicationAndRecovery()); request.version(delete.version()); - assert request.versionType().validateVersionForWrites(request.version()); - DeleteResponse response = new DeleteResponse(primary.shardId(), request.type(), request.id(), delete.version(), delete.found()); - return new PrimaryOperationResult<>(response, delete.getTranslogLocation()); } + return delete; } - public static ReplicaOperationResult executeDeleteRequestOnReplica(DeleteRequest request, IndexShard replica) { + public static Engine.Delete executeDeleteRequestOnReplica(DeleteRequest request, IndexShard replica) { Engine.Delete delete = replica.prepareDeleteOnReplica(request.type(), request.id(), request.version(), request.versionType()); replica.execute(delete); - return delete.hasFailure() - ? new ReplicaOperationResult(delete.getFailure()) - : new ReplicaOperationResult(delete.getTranslogLocation()); + return delete; } } diff --git a/core/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java b/core/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java index 3090c19ed18..0b0f194a767 100644 --- a/core/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java +++ b/core/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java @@ -39,7 +39,6 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.engine.Engine; -import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.shard.IndexShard; @@ -140,88 +139,77 @@ public class TransportIndexAction extends TransportWriteAction onPrimaryShard(IndexRequest request, IndexShard primary) throws Exception { - return executeIndexRequestOnPrimary(request, primary, mappingUpdatedAction); + protected WritePrimaryResult onPrimaryShard(IndexRequest request, IndexShard primary) throws Exception { + final Engine.Operation operation = executeIndexRequestOnPrimary(request, primary, mappingUpdatedAction); + final IndexResponse response = operation.hasFailure() ? null : + new IndexResponse(primary.shardId(), request.type(), request.id(), operation.version(), + ((Engine.Index) operation).isCreated()); + return new WritePrimaryResult(request, response, operation.getTranslogLocation(), operation.getFailure(), primary); } @Override - protected ReplicaOperationResult onReplicaShard(IndexRequest request, IndexShard replica) { - return executeIndexRequestOnReplica(request, replica); + protected WriteReplicaResult onReplicaShard(IndexRequest request, IndexShard replica) { + final Engine.Operation operation = executeIndexRequestOnReplica(request, replica); + return new WriteReplicaResult(request, operation.getTranslogLocation(), operation.getFailure(), replica); } /** * Execute the given {@link IndexRequest} on a replica shard, throwing a * {@link RetryOnReplicaException} if the operation needs to be re-tried. */ - public static ReplicaOperationResult executeIndexRequestOnReplica(IndexRequest request, IndexShard replica) { + public static Engine.Operation executeIndexRequestOnReplica(IndexRequest request, IndexShard replica) { final ShardId shardId = replica.shardId(); SourceToParse sourceToParse = SourceToParse.source(SourceToParse.Origin.REPLICA, shardId.getIndexName(), request.type(), request.id(), request.source()) .routing(request.routing()).parent(request.parent()).timestamp(request.timestamp()).ttl(request.ttl()); - final Engine.Index operation; - try { - operation = replica.prepareIndexOnReplica(sourceToParse, request.version(), request.versionType(), request.getAutoGeneratedTimestamp(), request.isRetry()); - } catch (MapperParsingException | IllegalArgumentException e) { - return new ReplicaOperationResult(e); - } - Mapping update = operation.parsedDoc().dynamicMappingsUpdate(); - if (update != null) { - throw new RetryOnReplicaException(shardId, "Mappings are not available on the replica yet, triggered update: " + update); - } - replica.execute(operation); - if (operation.hasFailure()) { - return new ReplicaOperationResult(operation.getFailure()); - } else { - return new ReplicaOperationResult(operation.getTranslogLocation()); + final Engine.Operation operation; + operation = replica.prepareIndexOnReplica(sourceToParse, request.version(), request.versionType(), request.getAutoGeneratedTimestamp(), request.isRetry()); + if (operation.hasFailure() == false) { + Mapping update = ((Engine.Index) operation).parsedDoc().dynamicMappingsUpdate(); + if (update != null) { + throw new RetryOnReplicaException(shardId, "Mappings are not available on the replica yet, triggered update: " + update); + } + replica.execute(operation); } + return operation; } /** Utility method to prepare an index operation on primary shards */ - static Engine.Index prepareIndexOperationOnPrimary(IndexRequest request, IndexShard primary) { + static Engine.Operation prepareIndexOperationOnPrimary(IndexRequest request, IndexShard primary) { SourceToParse sourceToParse = SourceToParse.source(SourceToParse.Origin.PRIMARY, request.index(), request.type(), request.id(), request.source()) .routing(request.routing()).parent(request.parent()).timestamp(request.timestamp()).ttl(request.ttl()); return primary.prepareIndexOnPrimary(sourceToParse, request.version(), request.versionType(), request.getAutoGeneratedTimestamp(), request.isRetry()); } - public static PrimaryOperationResult executeIndexRequestOnPrimary(IndexRequest request, IndexShard primary, + public static Engine.Operation executeIndexRequestOnPrimary(IndexRequest request, IndexShard primary, MappingUpdatedAction mappingUpdatedAction) throws Exception { - Engine.Index operation; - try { - operation = prepareIndexOperationOnPrimary(request, primary); - } catch (MapperParsingException | IllegalArgumentException e) { - return new PrimaryOperationResult<>(e); - } - Mapping update = operation.parsedDoc().dynamicMappingsUpdate(); - final ShardId shardId = primary.shardId(); - if (update != null) { - try { + Engine.Operation operation = prepareIndexOperationOnPrimary(request, primary); + if (operation.hasFailure() == false) { + Mapping update = ((Engine.Index) operation).parsedDoc().dynamicMappingsUpdate(); + final ShardId shardId = primary.shardId(); + if (update != null) { // can throw timeout exception when updating mappings or ISE for attempting to update default mappings // which are bubbled up mappingUpdatedAction.updateMappingOnMaster(shardId.getIndex(), request.type(), update); operation = prepareIndexOperationOnPrimary(request, primary); - } catch (MapperParsingException | IllegalArgumentException e) { - return new PrimaryOperationResult<>(e); - } - update = operation.parsedDoc().dynamicMappingsUpdate(); - if (update != null) { - throw new ReplicationOperation.RetryOnPrimaryException(shardId, - "Dynamic mappings are not available on the node that holds the primary yet"); + if (operation.hasFailure() == false) { + update = ((Engine.Index) operation).parsedDoc().dynamicMappingsUpdate(); + if (update != null) { + throw new ReplicationOperation.RetryOnPrimaryException(shardId, + "Dynamic mappings are not available on the node that holds the primary yet"); + } + } } } - primary.execute(operation); - if (operation.hasFailure()) { - return new PrimaryOperationResult<>(operation.getFailure()); - } else { + if (operation.hasFailure() == false) { + primary.execute(operation); // update the version on request so it will happen on the replicas final long version = operation.version(); request.version(version); request.versionType(request.versionType().versionTypeForReplicationAndRecovery()); - assert request.versionType().validateVersionForWrites(request.version()); - - IndexResponse response = new IndexResponse(shardId, request.type(), request.id(), request.version(), operation.isCreated()); - return new PrimaryOperationResult<>(response, operation.getTranslogLocation()); } + return operation; } } diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index 983132ac787..314e126e918 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -377,15 +377,9 @@ public abstract class TransportReplicationAction< final Response finalResponseIfSuccessful; final Exception finalFailure; - public PrimaryResult(ReplicaRequest replicaRequest, Response finalResponseIfSuccessful) { + public PrimaryResult(ReplicaRequest replicaRequest, Response finalResponseIfSuccessful, Exception finalFailure) { this.replicaRequest = replicaRequest; this.finalResponseIfSuccessful = finalResponseIfSuccessful; - this.finalFailure = null; - } - - public PrimaryResult(Exception finalFailure) { - this.replicaRequest = null; - this.finalResponseIfSuccessful = null; this.finalFailure = finalFailure; } @@ -413,10 +407,6 @@ public abstract class TransportReplicationAction< protected class ReplicaResult { final Exception finalFailure; - public ReplicaResult() { - this.finalFailure = null; - } - public ReplicaResult(Exception finalFailure) { this.finalFailure = finalFailure; } diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java b/core/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java index 6176889d159..feac9066357 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java @@ -62,7 +62,7 @@ public abstract class TransportWriteAction< /** * Called on the primary with a reference to the primary {@linkplain IndexShard} to modify. */ - protected abstract PrimaryOperationResult onPrimaryShard(Request request, IndexShard primary) throws Exception; + protected abstract WritePrimaryResult onPrimaryShard(Request request, IndexShard primary) throws Exception; /** * Called once per replica with a reference to the replica {@linkplain IndexShard} to modify. @@ -70,107 +70,39 @@ public abstract class TransportWriteAction< * @return the result of the replication operation containing either the translog location of the {@linkplain IndexShard} * after the write was completed or a failure if the operation failed */ - protected abstract ReplicaOperationResult onReplicaShard(ReplicaRequest request, IndexShard replica) throws Exception; + protected abstract WriteReplicaResult onReplicaShard(ReplicaRequest request, IndexShard replica) throws Exception; @Override protected final WritePrimaryResult shardOperationOnPrimary(Request request, IndexShard primary) throws Exception { - final PrimaryOperationResult result = onPrimaryShard(request, primary); - return result.success() - ? new WritePrimaryResult((ReplicaRequest) request, result.getResponse(), result.getLocation(), primary) - : new WritePrimaryResult(result.getFailure()); + return onPrimaryShard(request, primary); } @Override protected final WriteReplicaResult shardOperationOnReplica(ReplicaRequest request, IndexShard replica) throws Exception { - final ReplicaOperationResult result = onReplicaShard(request, replica); - return result.success() - ? new WriteReplicaResult(request, result.getLocation(), replica) - : new WriteReplicaResult(result.getFailure()); - } - - abstract static class OperationWriteResult { - private final Translog.Location location; - private final Exception failure; - - protected OperationWriteResult(@Nullable Location location) { - this.location = location; - this.failure = null; - } - - protected OperationWriteResult(Exception failure) { - this.location = null; - this.failure = failure; - } - - public Translog.Location getLocation() { - return location; - } - - public Exception getFailure() { - return failure; - } - - public boolean success() { - return failure == null; - } - } - - /** - * Simple result from a primary write action (includes response). - * Write actions have static method to return these so they can integrate with bulk. - */ - public static class PrimaryOperationResult extends OperationWriteResult { - private final Response response; - - public PrimaryOperationResult(Response response, @Nullable Location location) { - super(location); - this.response = response; - } - - public PrimaryOperationResult(Exception failure) { - super(failure); - this.response = null; - } - - public Response getResponse() { - return response; - } - } - - /** - * Simple result from a replica write action. Write actions have static method to return these so they can integrate with bulk. - */ - public static class ReplicaOperationResult extends OperationWriteResult { - - public ReplicaOperationResult(@Nullable Location location) { - super(location); - } - - public ReplicaOperationResult(Exception failure) { - super(failure); - } + return onReplicaShard(request, replica); } /** * Result of taking the action on the primary. */ - class WritePrimaryResult extends PrimaryResult implements RespondingWriteResult { + protected class WritePrimaryResult extends PrimaryResult implements RespondingWriteResult { boolean finishedAsyncActions; ActionListener listener = null; - public WritePrimaryResult(ReplicaRequest request, Response finalResponse, - @Nullable Location location, IndexShard primary) { - super(request, finalResponse); - /* - * We call this before replication because this might wait for a refresh and that can take a while. This way we wait for the - * refresh in parallel on the primary and on the replica. - */ - new AsyncAfterWriteAction(primary, request, location, this, logger).run(); - } - - public WritePrimaryResult(Exception failure) { - super(failure); - this.finishedAsyncActions = true; + public WritePrimaryResult(ReplicaRequest request, @Nullable Response finalResponse, + @Nullable Location location, @Nullable Exception operationFailure, + IndexShard primary) { + super(request, finalResponse, operationFailure); + assert operationFailure != null ^ finalResponse != null; + if (operationFailure != null) { + this.finishedAsyncActions = true; + } else { + /* + * We call this before replication because this might wait for a refresh and that can take a while. + * This way we wait for the refresh in parallel on the primary and on the replica. + */ + new AsyncAfterWriteAction(primary, request, location, this, logger).run(); + } } @Override @@ -210,17 +142,18 @@ public abstract class TransportWriteAction< /** * Result of taking the action on the replica. */ - class WriteReplicaResult extends ReplicaResult implements RespondingWriteResult { + protected class WriteReplicaResult extends ReplicaResult implements RespondingWriteResult { boolean finishedAsyncActions; private ActionListener listener; - public WriteReplicaResult(ReplicaRequest request, Location location, IndexShard replica) { - new AsyncAfterWriteAction(replica, request, location, this, logger).run(); - } - - public WriteReplicaResult(Exception finalFailure) { - super(finalFailure); - this.finishedAsyncActions = true; + public WriteReplicaResult(ReplicaRequest request, @Nullable Location location, + @Nullable Exception operationFailure, IndexShard replica) { + super(operationFailure); + if (operationFailure != null) { + this.finishedAsyncActions = true; + } else { + new AsyncAfterWriteAction(replica, request, location, this, logger).run(); + } } @Override diff --git a/core/src/main/java/org/elasticsearch/index/engine/Engine.java b/core/src/main/java/org/elasticsearch/index/engine/Engine.java index 246c1afae1d..11e9a375c78 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -771,7 +771,7 @@ public abstract class Engine implements Closeable { /** type of operation (index, delete), subclasses use static types */ public enum TYPE { - INDEX, DELETE; + INDEX, DELETE, FAILURE; private final String lowercase; @@ -1051,7 +1051,50 @@ public abstract class Engine implements Closeable { protected int estimatedSizeInBytes() { return (uid().field().length() + uid().text().length()) * 2 + 20; } + } + public static class Failure extends Operation { + + private final String type; + private final String id; + + public Failure(String type, String id, long version, VersionType versionType, Origin origin, + long startTime, Exception failure) { + super(null, version, versionType, origin, startTime); + this.type = type; + this.id = id; + setFailure(failure); + } + + @Override + public Term uid() { + throw new UnsupportedOperationException("failure operation doesn't have uid"); + } + + @Override + protected int estimatedSizeInBytes() { + return 0; + } + + @Override + public String type() { + return type; + } + + @Override + protected String id() { + return id; + } + + @Override + public TYPE operationType() { + return TYPE.FAILURE; + } + + @Override + public String toString() { + return "failure [{" + type() + "}][{" + id()+ "}]"; + } } public static class Get { diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index adc1fa876cb..cfb7414c904 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -93,6 +93,7 @@ import org.elasticsearch.index.get.ShardGetService; import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.DocumentMapperForType; import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.SourceToParse; @@ -499,24 +500,30 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl return previousState; } - public Engine.Index prepareIndexOnPrimary(SourceToParse source, long version, VersionType versionType, long autoGeneratedIdTimestamp, + public Engine.Operation prepareIndexOnPrimary(SourceToParse source, long version, VersionType versionType, long autoGeneratedIdTimestamp, boolean isRetry) { try { verifyPrimary(); return prepareIndex(docMapper(source.type()), source, version, versionType, Engine.Operation.Origin.PRIMARY, autoGeneratedIdTimestamp, isRetry); + } catch (MapperParsingException | IllegalArgumentException e) { + return new Engine.Failure(source.type(), source.id(), version, versionType, Engine.Operation.Origin.PRIMARY, + System.nanoTime(), e); } catch (Exception e) { verifyNotClosed(e); throw e; } } - public Engine.Index prepareIndexOnReplica(SourceToParse source, long version, VersionType versionType, long autoGeneratedIdTimestamp, + public Engine.Operation prepareIndexOnReplica(SourceToParse source, long version, VersionType versionType, long autoGeneratedIdTimestamp, boolean isRetry) { try { verifyReplicationTarget(); return prepareIndex(docMapper(source.type()), source, version, versionType, Engine.Operation.Origin.REPLICA, autoGeneratedIdTimestamp, isRetry); + } catch (MapperParsingException | IllegalArgumentException e) { + return new Engine.Failure(source.type(), source.id(), version, versionType, Engine.Operation.Origin.PRIMARY, + System.nanoTime(), e); } catch (Exception e) { verifyNotClosed(e); throw e; diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java b/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java index 3deae74f455..1642f5154f2 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java @@ -718,7 +718,7 @@ public class TransportReplicationActionTests extends ESTestCase { if (throwException) { throw new ElasticsearchException("simulated"); } - return new ReplicaResult(); + return new ReplicaResult(null); } }; final Action.ReplicaOperationTransportHandler replicaOperationTransportHandler = action.new ReplicaOperationTransportHandler(); @@ -837,7 +837,7 @@ public class TransportReplicationActionTests extends ESTestCase { if (throwException.get()) { throw new RetryOnReplicaException(shardId, "simulation"); } - return new ReplicaResult(); + return new ReplicaResult(null); } }; final Action.ReplicaOperationTransportHandler replicaOperationTransportHandler = action.new ReplicaOperationTransportHandler(); @@ -961,13 +961,13 @@ public class TransportReplicationActionTests extends ESTestCase { protected PrimaryResult shardOperationOnPrimary(Request shardRequest, IndexShard primary) throws Exception { boolean executedBefore = shardRequest.processedOnPrimary.getAndSet(true); assert executedBefore == false : "request has already been executed on the primary"; - return new PrimaryResult(shardRequest, new Response()); + return new PrimaryResult(shardRequest, new Response(), null); } @Override protected ReplicaResult shardOperationOnReplica(Request request, IndexShard replica) { request.processedOnReplicas.incrementAndGet(); - return new ReplicaResult(); + return new ReplicaResult(null); } @Override @@ -1053,7 +1053,7 @@ public class TransportReplicationActionTests extends ESTestCase { @Override public void execute() throws Exception { - this.resultListener.onResponse(action.new PrimaryResult(null, new Response())); + this.resultListener.onResponse(action.new PrimaryResult(null, new Response(), null)); } } diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java b/core/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java index c92bba093db..15aed84e1f6 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java @@ -136,13 +136,13 @@ public class TransportWriteActionTests extends ESTestCase { } @Override - protected PrimaryOperationResult onPrimaryShard(TestRequest request, IndexShard primary) throws Exception { - return new PrimaryOperationResult<>(new TestResponse(), location); + protected WritePrimaryResult onPrimaryShard(TestRequest request, IndexShard primary) throws Exception { + return new WritePrimaryResult(request, new TestResponse(), location, null, primary); } @Override - protected ReplicaOperationResult onReplicaShard(TestRequest request, IndexShard replica) { - return new ReplicaOperationResult(location); + protected WriteReplicaResult onReplicaShard(TestRequest request, IndexShard replica) { + return new WriteReplicaResult(request, location, null, replica); } @Override diff --git a/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java b/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java index 31b8f61bc30..caf7fdf335b 100644 --- a/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java +++ b/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java @@ -39,6 +39,7 @@ import org.elasticsearch.cluster.routing.ShardRoutingHelper; import org.elasticsearch.common.collect.Iterators; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.Index; +import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardTestCase; @@ -365,17 +366,19 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase @Override protected PrimaryResult performOnPrimary(IndexShard primary, IndexRequest request) throws Exception { - TransportWriteAction.PrimaryOperationResult result = executeIndexRequestOnPrimary(request, primary, - null); + final Engine.Operation operation = executeIndexRequestOnPrimary(request, primary, + null); request.primaryTerm(primary.getPrimaryTerm()); - TransportWriteActionTestHelper.performPostWriteActions(primary, request, result.getLocation(), logger); - return new PrimaryResult(request, result.getResponse()); + TransportWriteActionTestHelper.performPostWriteActions(primary, request, operation.getTranslogLocation(), logger); + IndexResponse response = new IndexResponse(primary.shardId(), request.type(), request.id(), operation.version(), + ((Engine.Index) operation).isCreated()); + return new PrimaryResult(request, response); } @Override protected void performOnReplica(IndexRequest request, IndexShard replica) { - TransportWriteAction.ReplicaOperationResult index = executeIndexRequestOnReplica(request, replica); - TransportWriteActionTestHelper.performPostWriteActions(replica, request, index.getLocation(), logger); + final Engine.Operation operation = executeIndexRequestOnReplica(request, replica); + TransportWriteActionTestHelper.performPostWriteActions(replica, request, operation.getTranslogLocation(), logger); } } } diff --git a/core/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java b/core/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java index 7d23ad37828..a3f22817dad 100644 --- a/core/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java +++ b/core/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java @@ -493,7 +493,7 @@ public class IndexStatsIT extends ESIntegTestCase { assertThat(stats.getIndex("test2").getPrimaries().getIndexing().getTotal().getIndexFailedCount(), equalTo(1L)); assertThat(stats.getPrimaries().getIndexing().getTypeStats().get("type1").getIndexFailedCount(), equalTo(1L)); assertThat(stats.getPrimaries().getIndexing().getTypeStats().get("type2").getIndexFailedCount(), equalTo(1L)); - assertThat(stats.getTotal().getIndexing().getTotal().getIndexFailedCount(), equalTo(3L)); + assertThat(stats.getPrimaries().getIndexing().getTotal().getIndexFailedCount(), equalTo(3L)); } public void testMergeStats() { diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index c520e3fbbfc..d86c08bbb05 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -441,7 +441,7 @@ public abstract class IndexShardTestCase extends ESTestCase { } protected Engine.Index indexDoc(IndexShard shard, String type, String id, String source) { - final Engine.Index index; + final Engine.Operation index; if (shard.routingEntry().primary()) { index = shard.prepareIndexOnPrimary( SourceToParse.source(SourceToParse.Origin.PRIMARY, shard.shardId().getIndexName(), type, id, new BytesArray(source)), @@ -452,7 +452,7 @@ public abstract class IndexShardTestCase extends ESTestCase { 1, VersionType.EXTERNAL, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); } shard.execute(index); - return index; + return ((Engine.Index) index); } protected Engine.Delete deleteDoc(IndexShard shard, String type, String id) { From 4396348e9e2f2ca6b6c90e60a1f46e0cf4517222 Mon Sep 17 00:00:00 2001 From: Areek Zillur Date: Thu, 20 Oct 2016 22:00:46 -0400 Subject: [PATCH 008/132] fix internal engine unit tests --- .../action/bulk/TransportShardBulkAction.java | 270 ++++++++++-------- .../action/index/TransportIndexAction.java | 13 +- .../index/engine/InternalEngineTests.java | 20 +- 3 files changed, 162 insertions(+), 141 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 6914fc00f56..b765e19a200 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -143,77 +143,10 @@ public class TransportShardBulkAction extends TransportWriteAction 0)) { - Tuple> sourceAndContent = - XContentHelper.convertToMap(indexSourceAsBytes, true); - updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, request.index(), - indexResponse.getVersion(), sourceAndContent.v2(), sourceAndContent.v1(), indexSourceAsBytes)); - } - // replace the update request to the translated index request to execute on the replica. - request.items()[requestIndex] = new BulkItemRequest(request.items()[requestIndex].id(), updateIndexRequest); - break; - case DELETE: - DeleteRequest updateDeleteRequest = translate.action(); - DeleteResponse deleteResponse = new DeleteResponse(primary.shardId(), - updateDeleteRequest.type(), updateDeleteRequest.id(), - updateOperation.version(), ((Engine.Delete) updateOperation).found()); - updateResponse = new UpdateResponse(deleteResponse.getShardInfo(), - deleteResponse.getShardId(), deleteResponse.getType(), deleteResponse.getId(), - deleteResponse.getVersion(), deleteResponse.getResult()); - updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, - request.index(), deleteResponse.getVersion(), translate.updatedSourceAsMap(), - translate.updateSourceContentType(), null)); - // replace the update request to the translated delete request to execute on the replica. - request.items()[requestIndex] = new BulkItemRequest(request.items()[requestIndex].id(), updateDeleteRequest); - break; - case FAILURE: - break; - } - } else { - // version conflict exception, retry - if (updateOperation.getFailure() instanceof VersionConflictEngineException) { - continue; - } - } - } - break; // out of retry loop - } - operation = updateOperation; - response = updateResponse; + UpdateResultHolder updateResultHolder = executeUpdateRequest(((UpdateRequest) itemRequest), + primary, metaData, request, requestIndex); + operation = updateResultHolder.operation; + response = updateResultHolder.response; break; case DELETE: final DeleteRequest deleteRequest = (DeleteRequest) itemRequest; @@ -224,21 +157,17 @@ public class TransportShardBulkAction extends TransportWriteActionrequestIndex for replicas to + * execute translated update request (NOOP update is an exception). NOOP updates are + * indicated by returning a null operation in {@link UpdateResultHolder} + * */ + private UpdateResultHolder executeUpdateRequest(UpdateRequest updateRequest, IndexShard primary, + IndexMetaData metaData, BulkShardRequest request, + int requestIndex) throws Exception { + Engine.Operation updateOperation = null; + UpdateResponse updateResponse = null; + int maxAttempts = updateRequest.retryOnConflict(); + for (int attemptCount = 0; attemptCount <= maxAttempts; attemptCount++) { + final UpdateHelper.Result translate; + // translate update request + try { + translate = updateHelper.prepare(updateRequest, primary, threadPool::estimatedTimeInMillis); + } catch (Exception failure) { + // we may fail translating a update to index or delete operation + updateOperation = new Engine.Failure(updateRequest.type(), updateRequest.id(), updateRequest.version(), + updateRequest.versionType(), Engine.Operation.Origin.PRIMARY, System.nanoTime(), failure); + break; // out of retry loop + } + // execute translated update request + switch (translate.getResponseResult()) { + case CREATED: + case UPDATED: + IndexRequest indexRequest = translate.action(); + MappingMetaData mappingMd = metaData.mappingOrDefault(indexRequest.type()); + indexRequest.process(mappingMd, allowIdGeneration, request.index()); + updateOperation = executeIndexRequestOnPrimary(indexRequest, primary, mappingUpdatedAction); + break; + case DELETED: + updateOperation = executeDeleteRequestOnPrimary(translate.action(), primary); + break; + case NOOP: + primary.noopUpdate(updateRequest.type()); + break; + default: throw new IllegalStateException("Illegal update operation " + translate.getResponseResult()); + } + if (updateOperation == null) { + // this is a noop operation + updateResponse = translate.action(); + } else { + if (updateOperation.hasFailure() == false) { + // enrich update response and + // set translated update (index/delete) request for replica execution in bulk items + switch (updateOperation.operationType()) { + case INDEX: + IndexRequest updateIndexRequest = translate.action(); + final IndexResponse indexResponse = new IndexResponse(primary.shardId(), + updateIndexRequest.type(), updateIndexRequest.id(), + updateOperation.version(), ((Engine.Index) updateOperation).isCreated()); + BytesReference indexSourceAsBytes = updateIndexRequest.source(); + updateResponse = new UpdateResponse(indexResponse.getShardInfo(), + indexResponse.getShardId(), indexResponse.getType(), indexResponse.getId(), + indexResponse.getVersion(), indexResponse.getResult()); + if ((updateRequest.fetchSource() != null && updateRequest.fetchSource().fetchSource()) || + (updateRequest.fields() != null && updateRequest.fields().length > 0)) { + Tuple> sourceAndContent = + XContentHelper.convertToMap(indexSourceAsBytes, true); + updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, request.index(), + indexResponse.getVersion(), sourceAndContent.v2(), sourceAndContent.v1(), indexSourceAsBytes)); + } + // replace the update request to the translated index request to execute on the replica. + request.items()[requestIndex] = new BulkItemRequest(request.items()[requestIndex].id(), updateIndexRequest); + break; + case DELETE: + DeleteRequest updateDeleteRequest = translate.action(); + DeleteResponse deleteResponse = new DeleteResponse(primary.shardId(), + updateDeleteRequest.type(), updateDeleteRequest.id(), + updateOperation.version(), ((Engine.Delete) updateOperation).found()); + updateResponse = new UpdateResponse(deleteResponse.getShardInfo(), + deleteResponse.getShardId(), deleteResponse.getType(), deleteResponse.getId(), + deleteResponse.getVersion(), deleteResponse.getResult()); + updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, + request.index(), deleteResponse.getVersion(), translate.updatedSourceAsMap(), + translate.updateSourceContentType(), null)); + // replace the update request to the translated delete request to execute on the replica. + request.items()[requestIndex] = new BulkItemRequest(request.items()[requestIndex].id(), updateDeleteRequest); + break; + case FAILURE: + break; + } + } else { + // version conflict exception, retry + if (updateOperation.getFailure() instanceof VersionConflictEngineException) { + continue; + } + } + } + break; // out of retry loop + } + return new UpdateResultHolder(updateOperation, updateResponse); + } + @Override protected WriteReplicaResult onReplicaShard(BulkShardRequest request, IndexShard replica) throws Exception { Translog.Location location = null; for (int i = 0; i < request.items().length; i++) { BulkItemRequest item = request.items()[i]; - if (item == null || item.isIgnoreOnReplica()) { - continue; - } - DocWriteRequest docWriteRequest = item.request(); - final Engine.Operation operation; - try { - switch (docWriteRequest.opType()) { - case CREATE: - case INDEX: - operation = executeIndexRequestOnReplica(((IndexRequest) docWriteRequest), replica); - break; - case DELETE: - operation = executeDeleteRequestOnReplica(((DeleteRequest) docWriteRequest), replica); - break; - default: throw new IllegalStateException("Unexpected request operation type on replica: " - + docWriteRequest.opType().getLowercase()); - } - if (operation.hasFailure()) { - // check if any transient write operation failures should be bubbled up - Exception failure = operation.getFailure(); - if (!ignoreReplicaException(failure)) { - throw failure; + if (item.isIgnoreOnReplica() == false) { + DocWriteRequest docWriteRequest = item.request(); + final Engine.Operation operation; + try { + switch (docWriteRequest.opType()) { + case CREATE: + case INDEX: + operation = executeIndexRequestOnReplica(((IndexRequest) docWriteRequest), replica); + break; + case DELETE: + operation = executeDeleteRequestOnReplica(((DeleteRequest) docWriteRequest), replica); + break; + default: + throw new IllegalStateException("Unexpected request operation type on replica: " + + docWriteRequest.opType().getLowercase()); + } + if (operation.hasFailure()) { + // check if any transient write operation failures should be bubbled up + Exception failure = operation.getFailure(); + if (!ignoreReplicaException(failure)) { + throw failure; + } + } else { + location = locationToSync(location, operation.getTranslogLocation()); + } + } catch (Exception e) { + // if its not an ignore replica failure, we need to make sure to bubble up the failure + // so we will fail the shard + if (!ignoreReplicaException(e)) { + throw e; } - } else { - location = locationToSync(location, operation.getTranslogLocation()); - } - } catch (Exception e) { - // if its not an ignore replica failure, we need to make sure to bubble up the failure - // so we will fail the shard - if (!ignoreReplicaException(e)) { - throw e; } } } diff --git a/core/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java b/core/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java index 894b95b5c2e..d514a321f14 100644 --- a/core/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java +++ b/core/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java @@ -190,7 +190,13 @@ public class TransportIndexAction extends TransportWriteAction Date: Fri, 21 Oct 2016 14:48:50 -0400 Subject: [PATCH 009/132] cleanup and improve documentation for TWA --- .../action/bulk/TransportShardBulkAction.java | 4 ++-- .../action/delete/TransportDeleteAction.java | 4 ++-- .../action/index/TransportIndexAction.java | 6 ++--- .../replication/TransportWriteAction.java | 22 ++++++++----------- .../TransportWriteActionTests.java | 14 ++++++------ 5 files changed, 23 insertions(+), 27 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index b765e19a200..d97fc47af25 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -101,7 +101,7 @@ public class TransportShardBulkAction extends TransportWriteAction, @@ -61,26 +62,21 @@ public abstract class TransportWriteAction< /** * Called on the primary with a reference to the primary {@linkplain IndexShard} to modify. + * + * @return the result of the operation on primary, including current translog location and operation response and failure + * async refresh is performed on the primary shard according to the Request refresh policy */ - protected abstract WritePrimaryResult onPrimaryShard(Request request, IndexShard primary) throws Exception; + @Override + protected abstract WritePrimaryResult shardOperationOnPrimary(Request request, IndexShard primary) throws Exception; /** * Called once per replica with a reference to the replica {@linkplain IndexShard} to modify. * - * @return the result of the replication operation containing either the translog location of the {@linkplain IndexShard} - * after the write was completed or a failure if the operation failed + * @return the result of the operation on replica, including current translog location and operation response and failure + * async refresh is performed on the replica shard according to the ReplicaRequest refresh policy */ - protected abstract WriteReplicaResult onReplicaShard(ReplicaRequest request, IndexShard replica) throws Exception; - @Override - protected final WritePrimaryResult shardOperationOnPrimary(Request request, IndexShard primary) throws Exception { - return onPrimaryShard(request, primary); - } - - @Override - protected final WriteReplicaResult shardOperationOnReplica(ReplicaRequest request, IndexShard replica) throws Exception { - return onReplicaShard(request, replica); - } + protected abstract WriteReplicaResult shardOperationOnReplica(ReplicaRequest request, IndexShard replica) throws Exception; /** * Result of taking the action on the primary. diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java b/core/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java index 15aed84e1f6..66251c92abf 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java @@ -136,19 +136,19 @@ public class TransportWriteActionTests extends ESTestCase { } @Override - protected WritePrimaryResult onPrimaryShard(TestRequest request, IndexShard primary) throws Exception { + protected TestResponse newResponseInstance() { + return new TestResponse(); + } + + @Override + protected WritePrimaryResult shardOperationOnPrimary(TestRequest request, IndexShard primary) throws Exception { return new WritePrimaryResult(request, new TestResponse(), location, null, primary); } @Override - protected WriteReplicaResult onReplicaShard(TestRequest request, IndexShard replica) { + protected WriteReplicaResult shardOperationOnReplica(TestRequest request, IndexShard replica) throws Exception { return new WriteReplicaResult(request, location, null, replica); } - - @Override - protected TestResponse newResponseInstance() { - return new TestResponse(); - } } private static class TestRequest extends ReplicatedWriteRequest { From 1587a77ffd36e7017141e6e42e01090e9d928ef1 Mon Sep 17 00:00:00 2001 From: Areek Zillur Date: Tue, 25 Oct 2016 09:11:16 -0400 Subject: [PATCH 010/132] Revert "Generify index shard method to execute engine write operation" This reverts commit 1bdeada8aa9e699d5c864d7257fea926426ce474. --- .../action/bulk/TransportShardBulkAction.java | 80 ++-- .../action/delete/TransportDeleteAction.java | 27 +- .../action/index/TransportIndexAction.java | 98 ++--- .../elasticsearch/index/IndexingSlowLog.java | 29 +- .../elasticsearch/index/engine/Engine.java | 249 +++++------- .../index/engine/InternalEngine.java | 112 +++--- .../index/engine/ShadowEngine.java | 4 +- .../elasticsearch/index/shard/IndexShard.java | 76 ++-- .../shard/IndexingOperationListener.java | 125 ++++-- .../index/shard/InternalIndexingStats.java | 98 ++--- .../shard/TranslogRecoveryPerformer.java | 2 +- .../indices/IndexingMemoryController.java | 18 +- .../elasticsearch/index/IndexModuleTests.java | 9 +- .../index/engine/InternalEngineTests.java | 376 +++++++++--------- .../index/mapper/TextFieldMapperTests.java | 4 +- .../ESIndexLevelReplicationTestCase.java | 12 +- .../index/shard/IndexShardIT.java | 19 +- .../index/shard/IndexShardTests.java | 102 +++-- .../shard/IndexingOperationListenerTests.java | 80 ++-- .../index/shard/RefreshListenersTests.java | 29 +- .../index/reindex/CancelTests.java | 19 +- .../index/shard/IndexShardTestCase.java | 4 +- 22 files changed, 842 insertions(+), 730 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index d97fc47af25..0b27b4c037c 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -57,8 +57,7 @@ import org.elasticsearch.transport.TransportService; import java.util.Map; -import static org.elasticsearch.action.delete.TransportDeleteAction.executeDeleteRequestOnPrimary; -import static org.elasticsearch.action.delete.TransportDeleteAction.executeDeleteRequestOnReplica; +import static org.elasticsearch.action.delete.TransportDeleteAction.*; import static org.elasticsearch.action.index.TransportIndexAction.executeIndexRequestOnPrimary; import static org.elasticsearch.action.index.TransportIndexAction.executeIndexRequestOnReplica; import static org.elasticsearch.action.support.replication.ReplicationOperation.ignoreReplicaException; @@ -125,44 +124,44 @@ public class TransportShardBulkAction extends TransportWriteAction) () -> new ParameterizedMessage("{} failed to execute bulk item ({}) {}", request.shardId(), docWriteRequest.opType().getLowercase(), request), failure); @@ -214,11 +213,11 @@ public class TransportShardBulkAction extends TransportWriteAction= 0 && tookInNanos > indexWarnThreshold) { - indexLogger.warn("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog)); - } else if (indexInfoThreshold >= 0 && tookInNanos > indexInfoThreshold) { - indexLogger.info("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog)); - } else if (indexDebugThreshold >= 0 && tookInNanos > indexDebugThreshold) { - indexLogger.debug("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog)); - } else if (indexTraceThreshold >= 0 && tookInNanos > indexTraceThreshold) { - indexLogger.trace("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog)); - } + public void postIndex(Engine.Index index, boolean created) { + final long took = index.endTime() - index.startTime(); + postIndexing(index.parsedDoc(), took); + } + + + private void postIndexing(ParsedDocument doc, long tookInNanos) { + if (indexWarnThreshold >= 0 && tookInNanos > indexWarnThreshold) { + indexLogger.warn("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog)); + } else if (indexInfoThreshold >= 0 && tookInNanos > indexInfoThreshold) { + indexLogger.info("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog)); + } else if (indexDebugThreshold >= 0 && tookInNanos > indexDebugThreshold) { + indexLogger.debug("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog)); + } else if (indexTraceThreshold >= 0 && tookInNanos > indexTraceThreshold) { + indexLogger.trace("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog)); } } diff --git a/core/src/main/java/org/elasticsearch/index/engine/Engine.java b/core/src/main/java/org/elasticsearch/index/engine/Engine.java index 11e9a375c78..137775e7a84 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -278,9 +278,105 @@ public abstract class Engine implements Closeable { } } - public abstract void index(Index operation); + public abstract IndexResult index(Index operation); - public abstract void delete(Delete delete); + public abstract DeleteResult delete(Delete delete); + + public abstract static class Result { + private final Operation.TYPE operationType; + private final Translog.Location location; + private final long version; + private final Exception failure; + private final long took; + private final int estimatedSizeInBytes; + + private Result(Operation.TYPE operationType, Translog.Location location, Exception failure, + long version, long took, int estimatedSizeInBytes) { + this.operationType = operationType; + this.location = location; + this.failure = failure; + this.version = version; + this.took = took; + this.estimatedSizeInBytes = estimatedSizeInBytes; + } + + protected Result(Operation.TYPE operationType, Translog.Location location, + long version, long took, int estimatedSizeInBytes) { + this(operationType, location, null, version, took, estimatedSizeInBytes); + } + + protected Result(Operation.TYPE operationType, Exception failure, + long version, long took, int estimatedSizeInBytes) { + this(operationType, null, failure, version, took, estimatedSizeInBytes); + } + + public boolean hasFailure() { + return failure != null; + } + + public long getVersion() { + return version; + } + + public Translog.Location getLocation() { + return location; + } + + public Exception getFailure() { + return failure; + } + + public long getTook() { + return took; + } + + public Operation.TYPE getOperationType() { + return operationType; + } + + public int getSizeInBytes() { + if (location != null) { + return location.size; + } + return estimatedSizeInBytes; + } + } + + public static class IndexResult extends Result { + private final boolean created; + + public IndexResult(Translog.Location location, long version, boolean created, long took, int estimatedSizeInBytes) { + super(Operation.TYPE.INDEX, location, version, took, estimatedSizeInBytes); + this.created = created; + } + + public IndexResult(Exception failure, long version, long took, int estimatedSizeInBytes) { + super(Operation.TYPE.INDEX, failure, version, took, estimatedSizeInBytes); + this.created = false; + } + + public boolean isCreated() { + return created; + } + } + + public static class DeleteResult extends Result { + private final boolean found; + + public DeleteResult(Translog.Location location, long version, boolean found, long took, int estimatedSizeInBytes) { + super(Operation.TYPE.DELETE, location, version, took, estimatedSizeInBytes); + this.found = found; + } + + DeleteResult(Exception failure, long version, long took, int estimatedSizeInBytes) { + super(Operation.TYPE.DELETE, failure, version, took, estimatedSizeInBytes); + this.found = false; + } + + public boolean isFound() { + return found; + } + } /** * Attempts to do a special commit where the given syncID is put into the commit data. The attempt @@ -771,7 +867,7 @@ public abstract class Engine implements Closeable { /** type of operation (index, delete), subclasses use static types */ public enum TYPE { - INDEX, DELETE, FAILURE; + INDEX, DELETE; private final String lowercase; @@ -785,13 +881,10 @@ public abstract class Engine implements Closeable { } private final Term uid; - private long version; + private final long version; private final VersionType versionType; private final Origin origin; - private Translog.Location location; - private Exception failure; private final long startTime; - private long endTime; public Operation(Term uid, long version, VersionType versionType, Origin origin, long startTime) { this.uid = uid; @@ -824,39 +917,7 @@ public abstract class Engine implements Closeable { return this.version; } - public void updateVersion(long version) { - this.version = version; - } - - public void setTranslogLocation(Translog.Location location) { - this.location = location; - } - - public Translog.Location getTranslogLocation() { - return this.location; - } - - public Exception getFailure() { - return failure; - } - - public void setFailure(Exception failure) { - this.failure = failure; - } - - public boolean hasFailure() { - return failure != null; - } - - public int sizeInBytes() { - if (location != null) { - return location.size; - } else { - return estimatedSizeInBytes(); - } - } - - protected abstract int estimatedSizeInBytes(); + public abstract int estimatedSizeInBytes(); public VersionType versionType() { return this.versionType; @@ -869,24 +930,11 @@ public abstract class Engine implements Closeable { return this.startTime; } - public void endTime(long endTime) { - this.endTime = endTime; - } - - /** - * Returns operation end time in nanoseconds. - */ - public long endTime() { - return this.endTime; - } - public abstract String type(); abstract String id(); - public abstract TYPE operationType(); - - public abstract String toString(); + abstract TYPE operationType(); } public static class Index extends Operation { @@ -894,7 +942,6 @@ public abstract class Engine implements Closeable { private final ParsedDocument doc; private final long autoGeneratedIdTimestamp; private final boolean isRetry; - private boolean created; public Index(Term uid, ParsedDocument doc, long version, VersionType versionType, Origin origin, long startTime, long autoGeneratedIdTimestamp, boolean isRetry) { @@ -927,7 +974,7 @@ public abstract class Engine implements Closeable { } @Override - public TYPE operationType() { + TYPE operationType() { return TYPE.INDEX; } @@ -943,12 +990,6 @@ public abstract class Engine implements Closeable { return this.doc.ttl(); } - @Override - public void updateVersion(long version) { - super.updateVersion(version); - this.doc.version().setLongValue(version); - } - public String parent() { return this.doc.parent(); } @@ -961,16 +1002,8 @@ public abstract class Engine implements Closeable { return this.doc.source(); } - public boolean isCreated() { - return created; - } - - public void setCreated(boolean created) { - this.created = created; - } - @Override - protected int estimatedSizeInBytes() { + public int estimatedSizeInBytes() { return (id().length() + type().length()) * 2 + source().length() + 12; } @@ -991,31 +1024,25 @@ public abstract class Engine implements Closeable { return isRetry; } - @Override - public String toString() { - return "index [{" + type() + "}][{" + id()+ "}] [{" + docs() + "}]"; - } } public static class Delete extends Operation { private final String type; private final String id; - private boolean found; - public Delete(String type, String id, Term uid, long version, VersionType versionType, Origin origin, long startTime, boolean found) { + public Delete(String type, String id, Term uid, long version, VersionType versionType, Origin origin, long startTime) { super(uid, version, versionType, origin, startTime); this.type = type; this.id = id; - this.found = found; } public Delete(String type, String id, Term uid) { - this(type, id, uid, Versions.MATCH_ANY, VersionType.INTERNAL, Origin.PRIMARY, System.nanoTime(), false); + this(type, id, uid, Versions.MATCH_ANY, VersionType.INTERNAL, Origin.PRIMARY, System.nanoTime()); } public Delete(Delete template, VersionType versionType) { - this(template.type(), template.id(), template.uid(), template.version(), versionType, template.origin(), template.startTime(), template.found()); + this(template.type(), template.id(), template.uid(), template.version(), versionType, template.origin(), template.startTime()); } @Override @@ -1029,74 +1056,16 @@ public abstract class Engine implements Closeable { } @Override - public TYPE operationType() { + TYPE operationType() { return TYPE.DELETE; } @Override - public String toString() { - return "delete [{"+ uid().text() +"}]"; - } - - public void updateVersion(long version, boolean found) { - updateVersion(version); - this.found = found; - } - - public boolean found() { - return this.found; - } - - @Override - protected int estimatedSizeInBytes() { + public int estimatedSizeInBytes() { return (uid().field().length() + uid().text().length()) * 2 + 20; } } - public static class Failure extends Operation { - - private final String type; - private final String id; - - public Failure(String type, String id, long version, VersionType versionType, Origin origin, - long startTime, Exception failure) { - super(null, version, versionType, origin, startTime); - this.type = type; - this.id = id; - setFailure(failure); - } - - @Override - public Term uid() { - throw new UnsupportedOperationException("failure operation doesn't have uid"); - } - - @Override - protected int estimatedSizeInBytes() { - return 0; - } - - @Override - public String type() { - return type; - } - - @Override - protected String id() { - return id; - } - - @Override - public TYPE operationType() { - return TYPE.FAILURE; - } - - @Override - public String toString() { - return "failure [{" + type() + "}][{" + id()+ "}]"; - } - } - public static class Get { private final boolean realtime; private final Term uid; diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 6e9bfc0e91c..7ec35d73a38 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -57,7 +57,7 @@ import org.elasticsearch.common.util.concurrent.KeyedLock; import org.elasticsearch.common.util.concurrent.ReleasableLock; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.VersionType; -import org.elasticsearch.index.engine.Engine.Operation; +import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.merge.MergeStats; import org.elasticsearch.index.merge.OnGoingMerge; @@ -394,44 +394,47 @@ public class InternalEngine extends Engine { VersionValue apply(long updatedVersion, long time); } - private void maybeAddToTranslog( + private Translog.Location maybeAddToTranslog( final T op, final long updatedVersion, final Function toTranslogOp, final VersionValueSupplier toVersionValue) throws IOException { + Translog.Location location = null; if (op.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY) { - final Translog.Location translogLocation = translog.add(toTranslogOp.apply(op)); - op.setTranslogLocation(translogLocation); + location = translog.add(toTranslogOp.apply(op)); } versionMap.putUnderLock(op.uid().bytes(), toVersionValue.apply(updatedVersion, engineConfig.getThreadPool().estimatedTimeInMillis())); - + return location; } @Override - public void index(Index index) { + public IndexResult index(Index index) { + IndexResult result; try (ReleasableLock lock = readLock.acquire()) { ensureOpen(); if (index.origin().isRecovery()) { // Don't throttle recovery operations - innerIndex(index); + result = innerIndex(index); } else { try (Releasable r = throttle.acquireThrottle()) { - innerIndex(index); + result = innerIndex(index); } } } catch (Exception e) { - handleOperationFailure(index, e); + Exception transientOperationFailure = handleOperationFailure(index, e); + result = new IndexResult(transientOperationFailure, index.version(), index.startTime() - System.nanoTime()); } + return result; } /** * Handle failures executing write operations, distinguish persistent engine (environment) failures * from document (request) specific failures. * Write failures that fail the engine as a side-effect, are thrown wrapped in {@link OperationFailedEngineException} - * and document specific failures are captured through {@link Operation#setFailure(Exception)} to be handled + * and document specific failures are returned to be set on the {@link Engine.Result} to be handled * at the transport level. */ - private void handleOperationFailure(final Operation operation, final Exception failure) { + private Exception handleOperationFailure(final Operation operation, final Exception failure) { boolean isEnvironmentFailure; try { // When indexing a document into Lucene, Lucene distinguishes between environment related errors @@ -451,7 +454,7 @@ public class InternalEngine extends Engine { throw new OperationFailedEngineException(shardId, operation.operationType().getLowercase(), operation.type(), operation.id(), failure); } else { - operation.setFailure(failure); + return failure; } } @@ -479,7 +482,9 @@ public class InternalEngine extends Engine { return false; } - private void innerIndex(Index index) throws IOException { + private IndexResult innerIndex(Index index) throws IOException { + final Translog.Location location; + final long updatedVersion; try (Releasable ignored = acquireLock(index.uid())) { lastWriteNanos = index.startTime(); /* if we have an autoGeneratedID that comes into the engine we can potentially optimize @@ -544,54 +549,52 @@ public class InternalEngine extends Engine { } final long expectedVersion = index.version(); if (checkVersionConflict(index, currentVersion, expectedVersion, deleted)) { - index.setCreated(false); - return; - } - final long updatedVersion = updateVersion(index, currentVersion, expectedVersion); - index.setCreated(deleted); - if (currentVersion == Versions.NOT_FOUND && forceUpdateDocument == false) { - // document does not exists, we can optimize for create - index(index, indexWriter); + // skip index operation because of version conflict on recovery + return new IndexResult(null, expectedVersion, false, index.startTime() - System.nanoTime()); } else { - update(index, indexWriter); + updatedVersion = index.versionType().updateVersion(currentVersion, expectedVersion); + index.parsedDoc().version().setLongValue(updatedVersion); + if (currentVersion == Versions.NOT_FOUND && forceUpdateDocument == false) { + // document does not exists, we can optimize for create + index(index.docs(), indexWriter); + } else { + update(index.uid(), index.docs(), indexWriter); + } + location = maybeAddToTranslog(index, updatedVersion, Translog.Index::new, NEW_VERSION_VALUE); + return new IndexResult(location, updatedVersion, deleted, index.startTime() - System.nanoTime()); } - maybeAddToTranslog(index, updatedVersion, Translog.Index::new, NEW_VERSION_VALUE); } } - private long updateVersion(Operation op, long currentVersion, long expectedVersion) { - final long updatedVersion = op.versionType().updateVersion(currentVersion, expectedVersion); - op.updateVersion(updatedVersion); - return updatedVersion; - } - - private static void index(final Index index, final IndexWriter indexWriter) throws IOException { - if (index.docs().size() > 1) { - indexWriter.addDocuments(index.docs()); + private static void index(final List docs, final IndexWriter indexWriter) throws IOException { + if (docs.size() > 1) { + indexWriter.addDocuments(docs); } else { - indexWriter.addDocument(index.docs().get(0)); + indexWriter.addDocument(docs.get(0)); } } - private static void update(final Index index, final IndexWriter indexWriter) throws IOException { - if (index.docs().size() > 1) { - indexWriter.updateDocuments(index.uid(), index.docs()); + private static void update(final Term uid, final List docs, final IndexWriter indexWriter) throws IOException { + if (docs.size() > 1) { + indexWriter.updateDocuments(uid, docs); } else { - indexWriter.updateDocument(index.uid(), index.docs().get(0)); + indexWriter.updateDocument(uid, docs.get(0)); } } @Override - public void delete(Delete delete) { + public DeleteResult delete(Delete delete) { + DeleteResult result; try (ReleasableLock lock = readLock.acquire()) { ensureOpen(); // NOTE: we don't throttle this when merges fall behind because delete-by-id does not create new segments: - innerDelete(delete); + result = innerDelete(delete); } catch (Exception e) { - handleOperationFailure(delete, e); + Exception transientOperationFailure = handleOperationFailure(delete, e); + result = new DeleteResult(transientOperationFailure, delete.version(), delete.startTime() - System.nanoTime()); } - maybePruneDeletedTombstones(); + return result; } private void maybePruneDeletedTombstones() { @@ -602,7 +605,10 @@ public class InternalEngine extends Engine { } } - private void innerDelete(Delete delete) throws IOException { + private DeleteResult innerDelete(Delete delete) throws IOException { + final Translog.Location location; + final long updatedVersion; + final boolean found; try (Releasable ignored = acquireLock(delete.uid())) { lastWriteNanos = delete.startTime(); final long currentVersion; @@ -618,19 +624,19 @@ public class InternalEngine extends Engine { } final long expectedVersion = delete.version(); - if (checkVersionConflict(delete, currentVersion, expectedVersion, deleted)) return; - - final long updatedVersion = updateVersion(delete, currentVersion, expectedVersion); - - final boolean found = deleteIfFound(delete, currentVersion, deleted, versionValue); - - delete.updateVersion(updatedVersion, found); - - maybeAddToTranslog(delete, updatedVersion, Translog.Delete::new, DeleteVersionValue::new); + if (checkVersionConflict(delete, currentVersion, expectedVersion, deleted)) { + // skip executing delete because of version conflict on recovery + return new DeleteResult(null, expectedVersion, true, delete.startTime() - System.nanoTime()); + } else { + updatedVersion = delete.versionType().updateVersion(currentVersion, expectedVersion); + found = deleteIfFound(delete.uid(), currentVersion, deleted, versionValue); + location = maybeAddToTranslog(delete, updatedVersion, Translog.Delete::new, DeleteVersionValue::new); + return new DeleteResult(location, updatedVersion, found, delete.startTime() - System.nanoTime()); + } } } - private boolean deleteIfFound(Delete delete, long currentVersion, boolean deleted, VersionValue versionValue) throws IOException { + private boolean deleteIfFound(Term uid, long currentVersion, boolean deleted, VersionValue versionValue) throws IOException { final boolean found; if (currentVersion == Versions.NOT_FOUND) { // doc does not exist and no prior deletes @@ -640,7 +646,7 @@ public class InternalEngine extends Engine { found = false; } else { // we deleted a currently existing document - indexWriter.deleteDocuments(delete.uid()); + indexWriter.deleteDocuments(uid); found = true; } return found; diff --git a/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java b/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java index 317d6f89408..d84f03e83dd 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java @@ -106,12 +106,12 @@ public class ShadowEngine extends Engine { @Override - public void index(Index index) { + public IndexResult index(Index index) { throw new UnsupportedOperationException(shardId + " index operation not allowed on shadow engine"); } @Override - public void delete(Delete delete) { + public DeleteResult delete(Delete delete) { throw new UnsupportedOperationException(shardId + " delete operation not allowed on shadow engine"); } diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index cfb7414c904..442adc98196 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -93,7 +93,6 @@ import org.elasticsearch.index.get.ShardGetService; import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.DocumentMapperForType; import org.elasticsearch.index.mapper.MappedFieldType; -import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.SourceToParse; @@ -500,30 +499,24 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl return previousState; } - public Engine.Operation prepareIndexOnPrimary(SourceToParse source, long version, VersionType versionType, long autoGeneratedIdTimestamp, + public Engine.Index prepareIndexOnPrimary(SourceToParse source, long version, VersionType versionType, long autoGeneratedIdTimestamp, boolean isRetry) { try { verifyPrimary(); return prepareIndex(docMapper(source.type()), source, version, versionType, Engine.Operation.Origin.PRIMARY, autoGeneratedIdTimestamp, isRetry); - } catch (MapperParsingException | IllegalArgumentException e) { - return new Engine.Failure(source.type(), source.id(), version, versionType, Engine.Operation.Origin.PRIMARY, - System.nanoTime(), e); } catch (Exception e) { verifyNotClosed(e); throw e; } } - public Engine.Operation prepareIndexOnReplica(SourceToParse source, long version, VersionType versionType, long autoGeneratedIdTimestamp, + public Engine.Index prepareIndexOnReplica(SourceToParse source, long version, VersionType versionType, long autoGeneratedIdTimestamp, boolean isRetry) { try { verifyReplicationTarget(); return prepareIndex(docMapper(source.type()), source, version, versionType, Engine.Operation.Origin.REPLICA, autoGeneratedIdTimestamp, isRetry); - } catch (MapperParsingException | IllegalArgumentException e) { - return new Engine.Failure(source.type(), source.id(), version, versionType, Engine.Operation.Origin.PRIMARY, - System.nanoTime(), e); } catch (Exception e) { verifyNotClosed(e); throw e; @@ -543,37 +536,31 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl return new Engine.Index(uid, doc, version, versionType, origin, startTime, autoGeneratedIdTimestamp, isRetry); } - public void execute(Engine.Operation operation) { - ensureWriteAllowed(operation); + public Engine.IndexResult index(Engine.Index index) { + ensureWriteAllowed(index); Engine engine = getEngine(); - execute(engine, operation); + return index(engine, index); } - private void execute(Engine engine, Engine.Operation operation) { + private Engine.IndexResult index(Engine engine, Engine.Index index) { active.set(true); - indexingOperationListeners.preOperation(operation); + final Engine.IndexResult result; + index = indexingOperationListeners.preIndex(index); try { if (logger.isTraceEnabled()) { - logger.trace(operation.toString()); + logger.trace("index [{}][{}]{}", index.type(), index.id(), index.docs()); } - switch (operation.operationType()) { - case INDEX: - engine.index(((Engine.Index) operation)); - break; - case DELETE: - engine.delete(((Engine.Delete) operation)); - break; - } - operation.endTime(System.nanoTime()); + result = engine.index(index); } catch (Exception e) { - indexingOperationListeners.postOperation(operation, e); + indexingOperationListeners.postIndex(index, e); throw e; } - if (operation.hasFailure()) { - indexingOperationListeners.postOperation(operation, operation.getFailure()); + if (result.hasFailure()) { + indexingOperationListeners.postIndex(index, result.getFailure()); } else { - indexingOperationListeners.postOperation(operation); + indexingOperationListeners.postIndex(index, result); } + return result; } public Engine.Delete prepareDeleteOnPrimary(String type, String id, long version, VersionType versionType) { @@ -595,7 +582,34 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl static Engine.Delete prepareDelete(String type, String id, Term uid, long version, VersionType versionType, Engine.Operation.Origin origin) { long startTime = System.nanoTime(); - return new Engine.Delete(type, id, uid, version, versionType, origin, startTime, false); + return new Engine.Delete(type, id, uid, version, versionType, origin, startTime); + } + + public Engine.DeleteResult delete(Engine.Delete delete) { + ensureWriteAllowed(delete); + Engine engine = getEngine(); + return delete(engine, delete); + } + + private Engine.DeleteResult delete(Engine engine, Engine.Delete delete) { + active.set(true); + final Engine.DeleteResult result; + delete = indexingOperationListeners.preDelete(delete); + try { + if (logger.isTraceEnabled()) { + logger.trace("delete [{}]", delete.uid().text()); + } + result = engine.delete(delete); + } catch (Exception e) { + indexingOperationListeners.postDelete(delete, e); + throw e; + } + if (result.hasFailure()) { + indexingOperationListeners.postDelete(delete, result.getFailure()); + } else { + indexingOperationListeners.postDelete(delete, result); + } + return result; } public Engine.GetResult get(Engine.Get get) { @@ -1829,12 +1843,12 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl @Override protected void index(Engine engine, Engine.Index engineIndex) { - IndexShard.this.execute(engine, engineIndex); + IndexShard.this.index(engine, engineIndex); } @Override protected void delete(Engine engine, Engine.Delete engineDelete) { - IndexShard.this.execute(engine, engineDelete); + IndexShard.this.delete(engine, engineDelete); } } } diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java b/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java index ec0fab2629f..042ddec924e 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java @@ -25,19 +25,49 @@ import org.elasticsearch.index.engine.Engine; import java.util.List; -/** An engine operation listener for index and delete execution. */ +/** + * An indexing listener for indexing, delete, events. + */ public interface IndexingOperationListener { - /** Called before executing index or delete operation */ - default void preOperation(Engine.Operation operation) {} + /** + * Called before the indexing occurs. + */ + default Engine.Index preIndex(Engine.Index operation) { + return operation; + } - /** Called after executing index or delete operation */ - default void postOperation(Engine.Operation operation) {} + /** + * Called after the indexing operation occurred. + */ + default void postIndex(Engine.Index index, boolean created) {} - /** Called after index or delete operation failed with exception */ - default void postOperation(Engine.Operation operation, Exception ex) {} + /** + * Called after the indexing operation occurred with exception. + */ + default void postIndex(Engine.Index index, Exception ex) {} - /** A Composite listener that multiplexes calls to each of the listeners methods. */ + /** + * Called before the delete occurs. + */ + default Engine.Delete preDelete(Engine.Delete delete) { + return delete; + } + + + /** + * Called after the delete operation occurred. + */ + default void postDelete(Engine.Delete delete) {} + + /** + * Called after the delete operation occurred with exception. + */ + default void postDelete(Engine.Delete delete, Exception ex) {} + + /** + * A Composite listener that multiplexes calls to each of the listeners methods. + */ final class CompositeListener implements IndexingOperationListener{ private final List listeners; private final Logger logger; @@ -48,40 +78,79 @@ public interface IndexingOperationListener { } @Override - public void preOperation(Engine.Operation operation) { + public Engine.Index preIndex(Engine.Index operation) { assert operation != null; for (IndexingOperationListener listener : listeners) { try { - listener.preOperation(operation); + listener.preIndex(operation); } catch (Exception e) { - logger.warn((Supplier) () -> new ParameterizedMessage("preOperation listener [{}] failed", listener), e); + logger.warn((Supplier) () -> new ParameterizedMessage("preIndex listener [{}] failed", listener), e); + } + } + return operation; + } + + @Override + public void postIndex(Engine.Index index, boolean created) { + assert index != null; + for (IndexingOperationListener listener : listeners) { + try { + listener.postIndex(index, created); + } catch (Exception e) { + logger.warn((Supplier) () -> new ParameterizedMessage("postIndex listener [{}] failed", listener), e); } } } @Override - public void postOperation(Engine.Operation operation) { - assert operation != null; + public void postIndex(Engine.Index index, Exception ex) { + assert index != null && ex != null; for (IndexingOperationListener listener : listeners) { try { - listener.postOperation(operation); - } catch (Exception e) { - logger.warn((Supplier) () -> new ParameterizedMessage("postOperation listener [{}] failed", listener), e); - } - } - } - - @Override - public void postOperation(Engine.Operation operation, Exception ex) { - assert operation != null && ex != null; - for (IndexingOperationListener listener : listeners) { - try { - listener.postOperation(operation, ex); + listener.postIndex(index, ex); } catch (Exception inner) { inner.addSuppressed(ex); - logger.warn((Supplier) () -> new ParameterizedMessage("postOperation listener [{}] failed", listener), inner); + logger.warn((Supplier) () -> new ParameterizedMessage("postIndex listener [{}] failed", listener), inner); + } + } + } + + @Override + public Engine.Delete preDelete(Engine.Delete delete) { + assert delete != null; + for (IndexingOperationListener listener : listeners) { + try { + listener.preDelete(delete); + } catch (Exception e) { + logger.warn((Supplier) () -> new ParameterizedMessage("preDelete listener [{}] failed", listener), e); + } + } + return delete; + } + + @Override + public void postDelete(Engine.Delete delete) { + assert delete != null; + for (IndexingOperationListener listener : listeners) { + try { + listener.postDelete(delete); + } catch (Exception e) { + logger.warn((Supplier) () -> new ParameterizedMessage("postDelete listener [{}] failed", listener), e); + } + } + } + + @Override + public void postDelete(Engine.Delete delete, Exception ex) { + assert delete != null && ex != null; + for (IndexingOperationListener listener : listeners) { + try { + listener.postDelete(delete, ex); + } catch (Exception inner) { + inner.addSuppressed(ex); + logger.warn((Supplier) () -> new ParameterizedMessage("postDelete listener [{}] failed", listener), inner); } } } } -} \ No newline at end of file +} diff --git a/core/src/main/java/org/elasticsearch/index/shard/InternalIndexingStats.java b/core/src/main/java/org/elasticsearch/index/shard/InternalIndexingStats.java index cd1b1526e0c..f62b8f7fe3c 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/InternalIndexingStats.java +++ b/core/src/main/java/org/elasticsearch/index/shard/InternalIndexingStats.java @@ -65,60 +65,63 @@ final class InternalIndexingStats implements IndexingOperationListener { } @Override - public void preOperation(Engine.Operation operation) { + public Engine.Index preIndex(Engine.Index operation) { if (!operation.origin().isRecovery()) { - StatsHolder statsHolder = typeStats(operation.type()); - switch (operation.operationType()) { - case INDEX: - totalStats.indexCurrent.inc(); - statsHolder.indexCurrent.inc(); - break; - case DELETE: - totalStats.deleteCurrent.inc(); - statsHolder.deleteCurrent.inc(); - break; - } + totalStats.indexCurrent.inc(); + typeStats(operation.type()).indexCurrent.inc(); + } + return operation; + } + + @Override + public void postIndex(Engine.Index index, boolean created) { + if (!index.origin().isRecovery()) { + long took = index.endTime() - index.startTime(); + totalStats.indexMetric.inc(took); + totalStats.indexCurrent.dec(); + StatsHolder typeStats = typeStats(index.type()); + typeStats.indexMetric.inc(took); + typeStats.indexCurrent.dec(); } } @Override - public void postOperation(Engine.Operation operation) { - if (!operation.origin().isRecovery()) { - long took = operation.endTime() - operation.startTime(); - StatsHolder typeStats = typeStats(operation.type()); - switch (operation.operationType()) { - case INDEX: - totalStats.indexMetric.inc(took); - totalStats.indexCurrent.dec(); - typeStats.indexMetric.inc(took); - typeStats.indexCurrent.dec(); - break; - case DELETE: - totalStats.deleteMetric.inc(took); - totalStats.deleteCurrent.dec(); - typeStats.deleteMetric.inc(took); - typeStats.deleteCurrent.dec(); - break; - } + public void postIndex(Engine.Index index, Exception ex) { + if (!index.origin().isRecovery()) { + totalStats.indexCurrent.dec(); + typeStats(index.type()).indexCurrent.dec(); + totalStats.indexFailed.inc(); + typeStats(index.type()).indexFailed.inc(); } } @Override - public void postOperation(Engine.Operation operation, Exception ex) { - if (!operation.origin().isRecovery()) { - StatsHolder statsHolder = typeStats(operation.type()); - switch (operation.operationType()) { - case INDEX: - totalStats.indexCurrent.dec(); - statsHolder.indexCurrent.dec(); - totalStats.indexFailed.inc(); - statsHolder.indexFailed.inc(); - break; - case DELETE: - totalStats.deleteCurrent.dec(); - statsHolder.deleteCurrent.dec(); - break; - } + public Engine.Delete preDelete(Engine.Delete delete) { + if (!delete.origin().isRecovery()) { + totalStats.deleteCurrent.inc(); + typeStats(delete.type()).deleteCurrent.inc(); + } + return delete; + + } + + @Override + public void postDelete(Engine.Delete delete) { + if (!delete.origin().isRecovery()) { + long took = delete.endTime() - delete.startTime(); + totalStats.deleteMetric.inc(took); + totalStats.deleteCurrent.dec(); + StatsHolder typeStats = typeStats(delete.type()); + typeStats.deleteMetric.inc(took); + typeStats.deleteCurrent.dec(); + } + } + + @Override + public void postDelete(Engine.Delete delete, Exception ex) { + if (!delete.origin().isRecovery()) { + totalStats.deleteCurrent.dec(); + typeStats(delete.type()).deleteCurrent.dec(); } } @@ -155,5 +158,10 @@ final class InternalIndexingStats implements IndexingOperationListener { deleteMetric.count(), TimeUnit.NANOSECONDS.toMillis(deleteMetric.sum()), deleteCurrent.count(), noopUpdates.count(), isThrottled, TimeUnit.MILLISECONDS.toMillis(currentThrottleMillis)); } + + void clear() { + indexMetric.clear(); + deleteMetric.clear(); + } } } diff --git a/core/src/main/java/org/elasticsearch/index/shard/TranslogRecoveryPerformer.java b/core/src/main/java/org/elasticsearch/index/shard/TranslogRecoveryPerformer.java index 64ae0c77007..5e5d2a84131 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/TranslogRecoveryPerformer.java +++ b/core/src/main/java/org/elasticsearch/index/shard/TranslogRecoveryPerformer.java @@ -170,7 +170,7 @@ public class TranslogRecoveryPerformer { logger.trace("[translog] recover [delete] op of [{}][{}]", uid.type(), uid.id()); } final Engine.Delete engineDelete = new Engine.Delete(uid.type(), uid.id(), delete.uid(), delete.version(), - delete.versionType().versionTypeForReplicationAndRecovery(), origin, System.nanoTime(), false); + delete.versionType().versionTypeForReplicationAndRecovery(), origin, System.nanoTime()); delete(engine, engineDelete); break; default: diff --git a/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java b/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java index da5a9b7c28e..3b4258a8bdf 100644 --- a/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java +++ b/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java @@ -189,6 +189,11 @@ public class IndexingMemoryController extends AbstractComponent implements Index statusChecker.run(); } + /** called by IndexShard to record that this many bytes were written to translog */ + public void bytesWritten(int bytes) { + statusChecker.bytesWritten(bytes); + } + /** Asks this shard to throttle indexing to one thread */ protected void activateThrottling(IndexShard shard) { shard.activateThrottling(); @@ -200,8 +205,17 @@ public class IndexingMemoryController extends AbstractComponent implements Index } @Override - public void postOperation(Engine.Operation operation) { - statusChecker.bytesWritten(operation.sizeInBytes()); + public void postIndex(Engine.Index index, boolean created) { + recordOperationBytes(index); + } + + @Override + public void postDelete(Engine.Delete delete) { + recordOperationBytes(delete); + } + + private void recordOperationBytes(Engine.Operation op) { + bytesWritten(op.sizeInBytes()); } private static final class ShardAndBytesUsed implements Comparable { diff --git a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java index 1cf56c50234..5e9d1ffaf9e 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java +++ b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java @@ -233,10 +233,9 @@ public class IndexModuleTests extends ESTestCase { AtomicBoolean executed = new AtomicBoolean(false); IndexingOperationListener listener = new IndexingOperationListener() { @Override - public void preOperation(Engine.Operation operation) { - if (operation.operationType() == Engine.Operation.TYPE.INDEX) { - executed.set(true); - } + public Engine.Index preIndex(Engine.Index operation) { + executed.set(true); + return operation; } }; module.addIndexOperationListener(listener); @@ -252,7 +251,7 @@ public class IndexModuleTests extends ESTestCase { Engine.Index index = new Engine.Index(new Term("_uid", "1"), null); for (IndexingOperationListener l : indexService.getIndexOperationListeners()) { - l.preOperation(index); + l.preIndex(index); } assertTrue(executed.get()); indexService.close("simon says", false); diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 70f326a3ff0..0b40d4cea52 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -334,11 +334,11 @@ public class InternalEngineTests extends ESTestCase { // create two docs and refresh ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null); Engine.Index first = new Engine.Index(newUid("1"), doc); - engine.index(first); + Engine.IndexResult firstResult = engine.index(first); ParsedDocument doc2 = testParsedDocument("2", "2", "test", null, -1, -1, testDocumentWithTextField(), B_2, null); Engine.Index second = new Engine.Index(newUid("2"), doc2); - engine.index(second); - assertThat(second.getTranslogLocation(), greaterThan(first.getTranslogLocation())); + Engine.IndexResult secondResult = engine.index(second); + assertThat(secondResult.getLocation(), greaterThan(firstResult.getLocation())); engine.refresh("test"); segments = engine.segments(false); @@ -628,7 +628,7 @@ public class InternalEngineTests extends ESTestCase { operations.add(operation); initialEngine.index(operation); } else { - final Engine.Delete operation = new Engine.Delete("test", "1", newUid("test#1"), i, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), false); + final Engine.Delete operation = new Engine.Delete("test", "1", newUid("test#1"), i, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime()); operations.add(operation); initialEngine.delete(operation); } @@ -1039,82 +1039,82 @@ public class InternalEngineTests extends ESTestCase { public void testVersioningNewCreate() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); Engine.Index create = new Engine.Index(newUid("1"), doc, Versions.MATCH_DELETED); - engine.index(create); - assertThat(create.version(), equalTo(1L)); + Engine.IndexResult indexResult = engine.index(create); + assertThat(indexResult.getVersion(), equalTo(1L)); - create = new Engine.Index(newUid("1"), doc, create.version(), create.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false); - replicaEngine.index(create); - assertThat(create.version(), equalTo(1L)); + create = new Engine.Index(newUid("1"), doc, indexResult.getVersion(), create.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false); + indexResult = replicaEngine.index(create); + assertThat(indexResult.getVersion(), equalTo(1L)); } public void testVersioningNewIndex() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); Engine.Index index = new Engine.Index(newUid("1"), doc); - engine.index(index); - assertThat(index.version(), equalTo(1L)); + Engine.IndexResult indexResult = engine.index(index); + assertThat(indexResult.getVersion(), equalTo(1L)); - index = new Engine.Index(newUid("1"), doc, index.version(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false); - replicaEngine.index(index); - assertThat(index.version(), equalTo(1L)); + index = new Engine.Index(newUid("1"), doc, indexResult.getVersion(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false); + indexResult = replicaEngine.index(index); + assertThat(indexResult.getVersion(), equalTo(1L)); } public void testExternalVersioningNewIndex() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); Engine.Index index = new Engine.Index(newUid("1"), doc, 12, VersionType.EXTERNAL, PRIMARY, 0, -1, false); - engine.index(index); - assertThat(index.version(), equalTo(12L)); + Engine.IndexResult indexResult = engine.index(index); + assertThat(indexResult.getVersion(), equalTo(12L)); - index = new Engine.Index(newUid("1"), doc, index.version(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false); - replicaEngine.index(index); - assertThat(index.version(), equalTo(12L)); + index = new Engine.Index(newUid("1"), doc, indexResult.getVersion(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false); + indexResult = replicaEngine.index(index); + assertThat(indexResult.getVersion(), equalTo(12L)); } public void testVersioningIndexConflict() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); Engine.Index index = new Engine.Index(newUid("1"), doc); - engine.index(index); - assertThat(index.version(), equalTo(1L)); + Engine.IndexResult indexResult = engine.index(index); + assertThat(indexResult.getVersion(), equalTo(1L)); index = new Engine.Index(newUid("1"), doc); - engine.index(index); - assertThat(index.version(), equalTo(2L)); + indexResult = engine.index(index); + assertThat(indexResult.getVersion(), equalTo(2L)); index = new Engine.Index(newUid("1"), doc, 1L, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, 0, -1, false); - engine.index(index); - assertTrue(index.hasFailure()); - assertThat(index.getFailure(), instanceOf(VersionConflictEngineException.class)); + indexResult = engine.index(index); + assertTrue(indexResult.hasFailure()); + assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); // future versions should not work as well index = new Engine.Index(newUid("1"), doc, 3L, VersionType.INTERNAL, PRIMARY, 0, -1, false); - engine.index(index); - assertTrue(index.hasFailure()); - assertThat(index.getFailure(), instanceOf(VersionConflictEngineException.class)); + indexResult = engine.index(index); + assertTrue(indexResult.hasFailure()); + assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); } public void testExternalVersioningIndexConflict() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); Engine.Index index = new Engine.Index(newUid("1"), doc, 12, VersionType.EXTERNAL, PRIMARY, 0, -1, false); - engine.index(index); - assertThat(index.version(), equalTo(12L)); + Engine.IndexResult indexResult = engine.index(index); + assertThat(indexResult.getVersion(), equalTo(12L)); index = new Engine.Index(newUid("1"), doc, 14, VersionType.EXTERNAL, PRIMARY, 0, -1, false); - engine.index(index); - assertThat(index.version(), equalTo(14L)); + indexResult = engine.index(index); + assertThat(indexResult.getVersion(), equalTo(14L)); index = new Engine.Index(newUid("1"), doc, 13, VersionType.EXTERNAL, PRIMARY, 0, -1, false); - engine.index(index); - assertTrue(index.hasFailure()); - assertThat(index.getFailure(), instanceOf(VersionConflictEngineException.class)); + indexResult = engine.index(index); + assertTrue(indexResult.hasFailure()); + assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); } public void testForceVersioningNotAllowedExceptForOlderIndices() throws Exception { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); Engine.Index index = new Engine.Index(newUid("1"), doc, 42, VersionType.FORCE, PRIMARY, 0, -1, false); - engine.index(index); - assertTrue(index.hasFailure()); - assertThat(index.getFailure(), instanceOf(IllegalArgumentException.class)); - assertThat(index.getFailure().getMessage(), containsString("version type [FORCE] may not be used for indices created after 6.0")); + Engine.IndexResult indexResult = engine.index(index); + assertTrue(indexResult.hasFailure()); + assertThat(indexResult.getFailure(), instanceOf(IllegalArgumentException.class)); + assertThat(indexResult.getFailure().getMessage(), containsString("version type [FORCE] may not be used for indices created after 6.0")); IndexSettings oldIndexSettings = IndexSettingsModule.newIndexSettings("test", Settings.builder() .put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_5_0_0_beta1) @@ -1122,58 +1122,58 @@ public class InternalEngineTests extends ESTestCase { try (Store store = createStore(); Engine engine = createEngine(oldIndexSettings, store, createTempDir(), NoMergePolicy.INSTANCE)) { index = new Engine.Index(newUid("1"), doc, 84, VersionType.FORCE, PRIMARY, 0, -1, false); - engine.index(index); - assertTrue(index.hasFailure()); - assertThat(index.getFailure(), instanceOf(IllegalArgumentException.class)); - assertThat(index.getFailure().getMessage(), containsString("version type [FORCE] may not be used for non-translog operations")); + Engine.IndexResult result = engine.index(index); + assertTrue(result.hasFailure()); + assertThat(result.getFailure(), instanceOf(IllegalArgumentException.class)); + assertThat(result.getFailure().getMessage(), containsString("version type [FORCE] may not be used for non-translog operations")); index = new Engine.Index(newUid("1"), doc, 84, VersionType.FORCE, Engine.Operation.Origin.LOCAL_TRANSLOG_RECOVERY, 0, -1, false); - engine.index(index); - assertThat(index.version(), equalTo(84L)); + result = engine.index(index); + assertThat(result.getVersion(), equalTo(84L)); } } public void testVersioningIndexConflictWithFlush() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); Engine.Index index = new Engine.Index(newUid("1"), doc); - engine.index(index); - assertThat(index.version(), equalTo(1L)); + Engine.IndexResult indexResult = engine.index(index); + assertThat(indexResult.getVersion(), equalTo(1L)); index = new Engine.Index(newUid("1"), doc); - engine.index(index); - assertThat(index.version(), equalTo(2L)); + indexResult = engine.index(index); + assertThat(indexResult.getVersion(), equalTo(2L)); engine.flush(); index = new Engine.Index(newUid("1"), doc, 1L, VersionType.INTERNAL, PRIMARY, 0, -1, false); - engine.index(index); - assertTrue(index.hasFailure()); - assertThat(index.getFailure(), instanceOf(VersionConflictEngineException.class)); + indexResult = engine.index(index); + assertTrue(indexResult.hasFailure()); + assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); // future versions should not work as well index = new Engine.Index(newUid("1"), doc, 3L, VersionType.INTERNAL, PRIMARY, 0, -1, false); - engine.index(index); - assertTrue(index.hasFailure()); - assertThat(index.getFailure(), instanceOf(VersionConflictEngineException.class)); + indexResult = engine.index(index); + assertTrue(indexResult.hasFailure()); + assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); } public void testExternalVersioningIndexConflictWithFlush() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); Engine.Index index = new Engine.Index(newUid("1"), doc, 12, VersionType.EXTERNAL, PRIMARY, 0, -1, false); - engine.index(index); - assertThat(index.version(), equalTo(12L)); + Engine.IndexResult indexResult = engine.index(index); + assertThat(indexResult.getVersion(), equalTo(12L)); index = new Engine.Index(newUid("1"), doc, 14, VersionType.EXTERNAL, PRIMARY, 0, -1, false); - engine.index(index); - assertThat(index.version(), equalTo(14L)); + indexResult = engine.index(index); + assertThat(indexResult.getVersion(), equalTo(14L)); engine.flush(); index = new Engine.Index(newUid("1"), doc, 13, VersionType.EXTERNAL, PRIMARY, 0, -1, false); - engine.index(index); - assertTrue(index.hasFailure()); - assertThat(index.getFailure(), instanceOf(VersionConflictEngineException.class)); + indexResult = engine.index(index); + assertTrue(indexResult.hasFailure()); + assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); } public void testForceMerge() throws IOException { @@ -1274,202 +1274,202 @@ public class InternalEngineTests extends ESTestCase { public void testVersioningDeleteConflict() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); Engine.Index index = new Engine.Index(newUid("1"), doc); - engine.index(index); - assertThat(index.version(), equalTo(1L)); + Engine.IndexResult indexResult = engine.index(index); + assertThat(indexResult.getVersion(), equalTo(1L)); index = new Engine.Index(newUid("1"), doc); - engine.index(index); - assertThat(index.version(), equalTo(2L)); + indexResult = engine.index(index); + assertThat(indexResult.getVersion(), equalTo(2L)); - Engine.Delete delete = new Engine.Delete("test", "1", newUid("1"), 1L, VersionType.INTERNAL, PRIMARY, 0, false); - engine.delete(delete); - assertTrue(delete.hasFailure()); - assertThat(delete.getFailure(), instanceOf(VersionConflictEngineException.class)); + Engine.Delete delete = new Engine.Delete("test", "1", newUid("1"), 1L, VersionType.INTERNAL, PRIMARY, 0); + Engine.DeleteResult result = engine.delete(delete); + assertTrue(result.hasFailure()); + assertThat(result.getFailure(), instanceOf(VersionConflictEngineException.class)); // future versions should not work as well - delete = new Engine.Delete("test", "1", newUid("1"), 3L, VersionType.INTERNAL, PRIMARY, 0, false); - engine.delete(delete); - assertTrue(delete.hasFailure()); - assertThat(delete.getFailure(), instanceOf(VersionConflictEngineException.class)); + delete = new Engine.Delete("test", "1", newUid("1"), 3L, VersionType.INTERNAL, PRIMARY, 0); + result = engine.delete(delete); + assertTrue(result.hasFailure()); + assertThat(result.getFailure(), instanceOf(VersionConflictEngineException.class)); // now actually delete - delete = new Engine.Delete("test", "1", newUid("1"), 2L, VersionType.INTERNAL, PRIMARY, 0, false); - engine.delete(delete); - assertThat(delete.version(), equalTo(3L)); + delete = new Engine.Delete("test", "1", newUid("1"), 2L, VersionType.INTERNAL, PRIMARY, 0); + result = engine.delete(delete); + assertThat(result.getVersion(), equalTo(3L)); // now check if we can index to a delete doc with version index = new Engine.Index(newUid("1"), doc, 2L, VersionType.INTERNAL, PRIMARY, 0, -1, false); - engine.index(index); - assertTrue(index.hasFailure()); - assertThat(index.getFailure(), instanceOf(VersionConflictEngineException.class)); + indexResult = engine.index(index); + assertTrue(indexResult.hasFailure()); + assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); } public void testVersioningDeleteConflictWithFlush() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); Engine.Index index = new Engine.Index(newUid("1"), doc); - engine.index(index); - assertThat(index.version(), equalTo(1L)); + Engine.IndexResult indexResult = engine.index(index); + assertThat(indexResult.getFailure(), equalTo(1L)); index = new Engine.Index(newUid("1"), doc); - engine.index(index); - assertThat(index.version(), equalTo(2L)); + indexResult = engine.index(index); + assertThat(indexResult.getVersion(), equalTo(2L)); engine.flush(); - Engine.Delete delete = new Engine.Delete("test", "1", newUid("1"), 1L, VersionType.INTERNAL, PRIMARY, 0, false); - engine.delete(delete); - assertTrue(delete.hasFailure()); - assertThat(delete.getFailure(), instanceOf(VersionConflictEngineException.class)); + Engine.Delete delete = new Engine.Delete("test", "1", newUid("1"), 1L, VersionType.INTERNAL, PRIMARY, 0); + Engine.DeleteResult deleteResult = engine.delete(delete); + assertTrue(deleteResult.hasFailure()); + assertThat(deleteResult.getFailure(), instanceOf(VersionConflictEngineException.class)); // future versions should not work as well - delete = new Engine.Delete("test", "1", newUid("1"), 3L, VersionType.INTERNAL, PRIMARY, 0, false); - engine.delete(delete); - assertTrue(delete.hasFailure()); - assertThat(delete.getFailure(), instanceOf(VersionConflictEngineException.class)); + delete = new Engine.Delete("test", "1", newUid("1"), 3L, VersionType.INTERNAL, PRIMARY, 0); + deleteResult = engine.delete(delete); + assertTrue(deleteResult.hasFailure()); + assertThat(deleteResult.getFailure(), instanceOf(VersionConflictEngineException.class)); engine.flush(); // now actually delete - delete = new Engine.Delete("test", "1", newUid("1"), 2L, VersionType.INTERNAL, PRIMARY, 0, false); - engine.delete(delete); - assertThat(delete.version(), equalTo(3L)); + delete = new Engine.Delete("test", "1", newUid("1"), 2L, VersionType.INTERNAL, PRIMARY, 0); + deleteResult = engine.delete(delete); + assertThat(deleteResult.getVersion(), equalTo(3L)); engine.flush(); // now check if we can index to a delete doc with version index = new Engine.Index(newUid("1"), doc, 2L, VersionType.INTERNAL, PRIMARY, 0, -1, false); - engine.index(index); - assertTrue(index.hasFailure()); - assertThat(index.getFailure(), instanceOf(VersionConflictEngineException.class)); + indexResult = engine.index(index); + assertTrue(indexResult.hasFailure()); + assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); } public void testVersioningCreateExistsException() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); Engine.Index create = new Engine.Index(newUid("1"), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false); - engine.index(create); - assertThat(create.version(), equalTo(1L)); + Engine.IndexResult indexResult = engine.index(create); + assertThat(indexResult.getVersion(), equalTo(1L)); create = new Engine.Index(newUid("1"), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false); - engine.index(create); - assertTrue(create.hasFailure()); - assertThat(create.getFailure(), instanceOf(VersionConflictEngineException.class)); + indexResult = engine.index(create); + assertTrue(indexResult.hasFailure()); + assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); } public void testVersioningCreateExistsExceptionWithFlush() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); Engine.Index create = new Engine.Index(newUid("1"), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false); - engine.index(create); - assertThat(create.version(), equalTo(1L)); + Engine.IndexResult indexResult = engine.index(create); + assertThat(indexResult.getVersion(), equalTo(1L)); engine.flush(); create = new Engine.Index(newUid("1"), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false); - engine.index(create); - assertTrue(create.hasFailure()); - assertThat(create.getFailure(), instanceOf(VersionConflictEngineException.class)); + indexResult = engine.index(create); + assertTrue(indexResult.hasFailure()); + assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); } public void testVersioningReplicaConflict1() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); Engine.Index index = new Engine.Index(newUid("1"), doc); - engine.index(index); - assertThat(index.version(), equalTo(1L)); + Engine.IndexResult indexResult = engine.index(index); + assertThat(indexResult.getVersion(), equalTo(1L)); index = new Engine.Index(newUid("1"), doc); - engine.index(index); - assertThat(index.version(), equalTo(2L)); + indexResult = engine.index(index); + assertThat(indexResult.getVersion(), equalTo(2L)); // apply the second index to the replica, should work fine - index = new Engine.Index(newUid("1"), doc, index.version(), VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false); - replicaEngine.index(index); - assertThat(index.version(), equalTo(2L)); + index = new Engine.Index(newUid("1"), doc, indexResult.getVersion(), VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false); + indexResult = replicaEngine.index(index); + assertThat(indexResult.getVersion(), equalTo(2L)); // now, the old one should not work index = new Engine.Index(newUid("1"), doc, 1L, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false); - replicaEngine.index(index); - assertTrue(index.hasFailure()); - assertThat(index.getFailure(), instanceOf(VersionConflictEngineException.class)); + indexResult = replicaEngine.index(index); + assertTrue(indexResult.hasFailure()); + assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); // second version on replica should fail as well index = new Engine.Index(newUid("1"), doc, 2L , VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false); - replicaEngine.index(index); - assertThat(index.version(), equalTo(2L)); - assertThat(index.getFailure(), instanceOf(VersionConflictEngineException.class)); + indexResult = replicaEngine.index(index); + assertThat(indexResult.getVersion(), equalTo(2L)); + assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); } public void testVersioningReplicaConflict2() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); Engine.Index index = new Engine.Index(newUid("1"), doc); - engine.index(index); - assertThat(index.version(), equalTo(1L)); + Engine.IndexResult indexResult = engine.index(index); + assertThat(indexResult.getVersion(), equalTo(1L)); // apply the first index to the replica, should work fine index = new Engine.Index(newUid("1"), doc, 1L , VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false); - replicaEngine.index(index); - assertThat(index.version(), equalTo(1L)); + indexResult = replicaEngine.index(index); + assertThat(indexResult.getVersion(), equalTo(1L)); // index it again index = new Engine.Index(newUid("1"), doc); - engine.index(index); - assertThat(index.version(), equalTo(2L)); + indexResult = engine.index(index); + assertThat(indexResult.getVersion(), equalTo(2L)); // now delete it Engine.Delete delete = new Engine.Delete("test", "1", newUid("1")); - engine.delete(delete); - assertThat(delete.version(), equalTo(3L)); + Engine.DeleteResult deleteResult = engine.delete(delete); + assertThat(deleteResult.getVersion(), equalTo(3L)); // apply the delete on the replica (skipping the second index) delete = new Engine.Delete("test", "1", newUid("1"), 3L - , VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, false); - replicaEngine.delete(delete); - assertThat(delete.version(), equalTo(3L)); + , VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); + deleteResult = replicaEngine.delete(delete); + assertThat(deleteResult.getVersion(), equalTo(3L)); // second time delete with same version should fail delete = new Engine.Delete("test", "1", newUid("1"), 3L - , VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, false); - replicaEngine.delete(delete); - assertTrue(delete.hasFailure()); - assertThat(delete.getFailure(), instanceOf(VersionConflictEngineException.class)); + , VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); + deleteResult = replicaEngine.delete(delete); + assertTrue(deleteResult.hasFailure()); + assertThat(deleteResult.getFailure(), instanceOf(VersionConflictEngineException.class)); // now do the second index on the replica, it should fail index = new Engine.Index(newUid("1"), doc, 2L, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false); - replicaEngine.index(index); - assertTrue(index.hasFailure()); - assertThat(index.getFailure(), instanceOf(VersionConflictEngineException.class)); + indexResult = replicaEngine.index(index); + assertTrue(indexResult.hasFailure()); + assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); } public void testBasicCreatedFlag() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); Engine.Index index = new Engine.Index(newUid("1"), doc); - engine.index(index); - assertTrue(index.isCreated()); + Engine.IndexResult indexResult = engine.index(index); + assertTrue(indexResult.isCreated()); index = new Engine.Index(newUid("1"), doc); - engine.index(index); - assertFalse(index.isCreated()); + indexResult = engine.index(index); + assertFalse(indexResult.isCreated()); engine.delete(new Engine.Delete(null, "1", newUid("1"))); index = new Engine.Index(newUid("1"), doc); - engine.index(index); - assertTrue(index.isCreated()); + indexResult = engine.index(index); + assertTrue(indexResult.isCreated()); } public void testCreatedFlagAfterFlush() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); Engine.Index index = new Engine.Index(newUid("1"), doc); - engine.index(index); - assertTrue(index.isCreated()); + Engine.IndexResult indexResult = engine.index(index); + assertTrue(indexResult.isCreated()); engine.delete(new Engine.Delete(null, "1", newUid("1"))); engine.flush(); index = new Engine.Index(newUid("1"), doc); - engine.index(index); - assertTrue(index.isCreated()); + indexResult = engine.index(index); + assertTrue(indexResult.isCreated()); } private static class MockAppender extends AbstractAppender { @@ -1572,7 +1572,7 @@ public class InternalEngineTests extends ESTestCase { engine.index(new Engine.Index(newUid("1"), doc, 1, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false)); // Delete document we just added: - engine.delete(new Engine.Delete("test", "1", newUid("1"), 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), false)); + engine.delete(new Engine.Delete("test", "1", newUid("1"), 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime())); // Get should not find the document Engine.GetResult getResult = engine.get(new Engine.Get(true, newUid("1"))); @@ -1586,7 +1586,7 @@ public class InternalEngineTests extends ESTestCase { } // Delete non-existent document - engine.delete(new Engine.Delete("test", "2", newUid("2"), 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), false)); + engine.delete(new Engine.Delete("test", "2", newUid("2"), 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime())); // Get should not find the document (we never indexed uid=2): getResult = engine.get(new Engine.Get(true, newUid("2"))); @@ -1594,9 +1594,9 @@ public class InternalEngineTests extends ESTestCase { // Try to index uid=1 with a too-old version, should fail: Engine.Index index = new Engine.Index(newUid("1"), doc, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false); - engine.index(index); - assertTrue(index.hasFailure()); - assertThat(index.getFailure(), instanceOf(VersionConflictEngineException.class)); + Engine.IndexResult indexResult = engine.index(index); + assertTrue(indexResult.hasFailure()); + assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); // Get should still not find the document getResult = engine.get(new Engine.Get(true, newUid("1"))); @@ -1604,9 +1604,9 @@ public class InternalEngineTests extends ESTestCase { // Try to index uid=2 with a too-old version, should fail: Engine.Index index1 = new Engine.Index(newUid("2"), doc, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false); - engine.index(index1); - assertTrue(index1.hasFailure()); - assertThat(index1.getFailure(), instanceOf(VersionConflictEngineException.class)); + indexResult = engine.index(index1); + assertTrue(indexResult.hasFailure()); + assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); // Get should not find the document getResult = engine.get(new Engine.Get(true, newUid("2"))); @@ -1702,8 +1702,8 @@ public class InternalEngineTests extends ESTestCase { 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.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); - engine.index(firstIndexRequest); - assertThat(firstIndexRequest.version(), equalTo(1L)); + Engine.IndexResult indexResult = engine.index(firstIndexRequest); + assertThat(indexResult.getVersion(), equalTo(1L)); } engine.refresh("test"); try (Engine.Searcher searcher = engine.acquireSearcher("test")) { @@ -1752,8 +1752,8 @@ public class InternalEngineTests extends ESTestCase { 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.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); - engine.index(firstIndexRequest); - assertThat(firstIndexRequest.version(), equalTo(1L)); + Engine.IndexResult indexResult = engine.index(firstIndexRequest); + assertThat(indexResult.getVersion(), equalTo(1L)); } engine.refresh("test"); try (Engine.Searcher searcher = engine.acquireSearcher("test")) { @@ -1842,8 +1842,8 @@ public class InternalEngineTests extends ESTestCase { for (int i = 0; i < numExtraDocs; i++) { ParsedDocument doc = testParsedDocument("extra" + Integer.toString(i), "extra" + Integer.toString(i), "test", null, -1, -1, testDocument(), new BytesArray("{}"), null); Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); - engine.index(firstIndexRequest); - assertThat(firstIndexRequest.version(), equalTo(1L)); + Engine.IndexResult indexResult = engine.index(firstIndexRequest); + assertThat(indexResult.getVersion(), equalTo(1L)); } engine.refresh("test"); try (Engine.Searcher searcher = engine.acquireSearcher("test")) { @@ -1871,8 +1871,8 @@ public class InternalEngineTests extends ESTestCase { 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.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); - engine.index(firstIndexRequest); - assertThat(firstIndexRequest.version(), equalTo(1L)); + Engine.IndexResult indexResult = engine.index(firstIndexRequest); + assertThat(indexResult.getVersion(), equalTo(1L)); } engine.refresh("test"); try (Engine.Searcher searcher = engine.acquireSearcher("test")) { @@ -2165,25 +2165,25 @@ public class InternalEngineTests extends ESTestCase { Engine.Index operation = randomAppendOnly(1, doc, false); Engine.Index retry = randomAppendOnly(1, doc, true); if (randomBoolean()) { - engine.index(operation); + Engine.IndexResult indexResult = engine.index(operation); assertFalse(engine.indexWriterHasDeletions()); assertEquals(0, engine.getNumVersionLookups()); - assertNotNull(operation.getTranslogLocation()); - engine.index(retry); + assertNotNull(indexResult.getLocation()); + Engine.IndexResult retryResult = engine.index(retry); assertTrue(engine.indexWriterHasDeletions()); assertEquals(0, engine.getNumVersionLookups()); - assertNotNull(retry.getTranslogLocation()); - assertTrue(retry.getTranslogLocation().compareTo(operation.getTranslogLocation()) > 0); + assertNotNull(retryResult.getLocation()); + assertTrue(retryResult.getLocation().compareTo(indexResult.getLocation()) > 0); } else { - engine.index(retry); + Engine.IndexResult retryResult = engine.index(retry); assertTrue(engine.indexWriterHasDeletions()); assertEquals(0, engine.getNumVersionLookups()); - assertNotNull(retry.getTranslogLocation()); - engine.index(operation); + assertNotNull(retryResult.getLocation()); + Engine.IndexResult indexResult = engine.index(operation); assertTrue(engine.indexWriterHasDeletions()); assertEquals(0, engine.getNumVersionLookups()); - assertNotNull(retry.getTranslogLocation()); - assertTrue(retry.getTranslogLocation().compareTo(operation.getTranslogLocation()) < 0); + assertNotNull(retryResult.getLocation()); + assertTrue(retryResult.getLocation().compareTo(indexResult.getLocation()) < 0); } engine.refresh("test"); @@ -2194,17 +2194,17 @@ public class InternalEngineTests extends ESTestCase { operation = randomAppendOnly(1, doc, false); retry = randomAppendOnly(1, doc, true); if (randomBoolean()) { - engine.index(operation); - assertNotNull(operation.getTranslogLocation()); - engine.index(retry); - assertNotNull(retry.getTranslogLocation()); - assertTrue(retry.getTranslogLocation().compareTo(operation.getTranslogLocation()) > 0); + Engine.IndexResult indexResult = engine.index(operation); + assertNotNull(indexResult.getLocation()); + Engine.IndexResult retryResult = engine.index(retry); + assertNotNull(retryResult.getLocation()); + assertTrue(retryResult.getLocation().compareTo(indexResult.getLocation()) > 0); } else { - engine.index(retry); - assertNotNull(retry.getTranslogLocation()); - engine.index(operation); - assertNotNull(retry.getTranslogLocation()); - assertTrue(retry.getTranslogLocation().compareTo(operation.getTranslogLocation()) < 0); + Engine.IndexResult retryResult = engine.index(retry); + assertNotNull(retryResult.getLocation()); + Engine.IndexResult indexResult = engine.index(operation); + assertNotNull(retryResult.getLocation()); + assertTrue(retryResult.getLocation().compareTo(indexResult.getLocation()) < 0); } engine.refresh("test"); @@ -2265,8 +2265,8 @@ public class InternalEngineTests extends ESTestCase { isRetry = false; Engine.Index secondIndexRequest = new Engine.Index(newUid("1"), doc, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry); - engine.index(secondIndexRequest); - assertTrue(secondIndexRequest.isCreated()); + Engine.IndexResult indexResult = engine.index(secondIndexRequest); + assertTrue(indexResult.isCreated()); engine.refresh("test"); try (Engine.Searcher searcher = engine.acquireSearcher("test")) { TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), 10); diff --git a/core/src/test/java/org/elasticsearch/index/mapper/TextFieldMapperTests.java b/core/src/test/java/org/elasticsearch/index/mapper/TextFieldMapperTests.java index 7313b7ec9bf..846d2c56669 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/TextFieldMapperTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/TextFieldMapperTests.java @@ -219,7 +219,7 @@ public class TextFieldMapperTests extends ESSingleNodeTestCase { assertEquals("b", fields[1].stringValue()); IndexShard shard = indexService.getShard(0); - shard.execute(new Engine.Index(new Term("_uid", "1"), doc)); + shard.index(new Engine.Index(new Term("_uid", "1"), doc)); shard.refresh("test"); try (Engine.Searcher searcher = shard.acquireSearcher("test")) { LeafReader leaf = searcher.getDirectoryReader().leaves().get(0).reader(); @@ -258,7 +258,7 @@ public class TextFieldMapperTests extends ESSingleNodeTestCase { assertEquals("b", fields[1].stringValue()); IndexShard shard = indexService.getShard(0); - shard.execute(new Engine.Index(new Term("_uid", "1"), doc)); + shard.index(new Engine.Index(new Term("_uid", "1"), doc)); shard.refresh("test"); try (Engine.Searcher searcher = shard.acquireSearcher("test")) { LeafReader leaf = searcher.getDirectoryReader().leaves().get(0).reader(); diff --git a/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java b/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java index caf7fdf335b..89487c7aa0c 100644 --- a/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java +++ b/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java @@ -366,19 +366,19 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase @Override protected PrimaryResult performOnPrimary(IndexShard primary, IndexRequest request) throws Exception { - final Engine.Operation operation = executeIndexRequestOnPrimary(request, primary, + final Engine.IndexResult indexResult = executeIndexRequestOnPrimary(request, primary, null); request.primaryTerm(primary.getPrimaryTerm()); - TransportWriteActionTestHelper.performPostWriteActions(primary, request, operation.getTranslogLocation(), logger); - IndexResponse response = new IndexResponse(primary.shardId(), request.type(), request.id(), operation.version(), - ((Engine.Index) operation).isCreated()); + TransportWriteActionTestHelper.performPostWriteActions(primary, request, indexResult.getLocation(), logger); + IndexResponse response = new IndexResponse(primary.shardId(), request.type(), request.id(), indexResult.getVersion(), + indexResult.isCreated()); return new PrimaryResult(request, response); } @Override protected void performOnReplica(IndexRequest request, IndexShard replica) { - final Engine.Operation operation = executeIndexRequestOnReplica(request, replica); - TransportWriteActionTestHelper.performPostWriteActions(replica, request, operation.getTranslogLocation(), logger); + final Engine.IndexResult result = executeIndexRequestOnReplica(request, replica); + TransportWriteActionTestHelper.performPostWriteActions(replica, request, result.getLocation(), logger); } } } diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java index 900bea75724..2248ff156ac 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java @@ -321,7 +321,7 @@ public class IndexShardIT extends ESSingleNodeTestCase { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, new ParseContext.Document(), new BytesArray(new byte[]{1}), null); Engine.Index index = new Engine.Index(new Term("_uid", "1"), doc); - shard.execute(index); + shard.index(index); assertTrue(shard.shouldFlush()); assertEquals(2, shard.getEngine().getTranslog().totalOperations()); client().prepareIndex("test", "test", "2").setSource("{}").setRefreshPolicy(randomBoolean() ? IMMEDIATE : NONE).get(); @@ -406,8 +406,23 @@ public class IndexShardIT extends ESSingleNodeTestCase { AtomicReference shardRef = new AtomicReference<>(); List failures = new ArrayList<>(); IndexingOperationListener listener = new IndexingOperationListener() { + @Override - public void postOperation(Engine.Operation operation) { + public void postIndex(Engine.Index index, boolean created) { + try { + assertNotNull(shardRef.get()); + // this is all IMC needs to do - check current memory and refresh + assertTrue(shardRef.get().getIndexBufferRAMBytesUsed() > 0); + shardRef.get().refresh("test"); + } catch (Exception e) { + failures.add(e); + throw e; + } + } + + + @Override + public void postDelete(Engine.Delete delete) { try { assertNotNull(shardRef.get()); // this is all IMC needs to do - check current memory and refresh diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 8ab0566158e..9c6f21c6958 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -558,43 +558,40 @@ public class IndexShardTests extends IndexShardTestCase { shard.close("simon says", true); shard = reinitShard(shard, new IndexingOperationListener() { @Override - public void preOperation(Engine.Operation operation) { - switch (operation.operationType()) { - case INDEX: - preIndex.incrementAndGet(); - break; - case DELETE: - preDelete.incrementAndGet(); - break; + public Engine.Index preIndex(Engine.Index operation) { + preIndex.incrementAndGet(); + return operation; + } + + @Override + public void postIndex(Engine.Index index, boolean created) { + if (created) { + postIndexCreate.incrementAndGet(); + } else { + postIndexUpdate.incrementAndGet(); } } @Override - public void postOperation(Engine.Operation operation) { - switch (operation.operationType()) { - case INDEX: - if (((Engine.Index) operation).isCreated()) { - postIndexCreate.incrementAndGet(); - } else { - postIndexUpdate.incrementAndGet(); - } - break; - case DELETE: - postDelete.incrementAndGet(); - break; - } + public void postIndex(Engine.Index index, Exception ex) { + postIndexException.incrementAndGet(); } @Override - public void postOperation(Engine.Operation operation, Exception ex) { - switch (operation.operationType()) { - case INDEX: - postIndexException.incrementAndGet(); - break; - case DELETE: - postDeleteException.incrementAndGet(); - break; - } + public Engine.Delete preDelete(Engine.Delete delete) { + preDelete.incrementAndGet(); + return delete; + } + + @Override + public void postDelete(Engine.Delete delete) { + postDelete.incrementAndGet(); + } + + @Override + public void postDelete(Engine.Delete delete, Exception ex) { + postDeleteException.incrementAndGet(); + } }); recoveryShardFromStore(shard); @@ -602,7 +599,7 @@ public class IndexShardTests extends IndexShardTestCase { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, new ParseContext.Document(), new BytesArray(new byte[]{1}), null); Engine.Index index = new Engine.Index(new Term("_uid", "1"), doc); - shard.execute(index); + shard.index(index); assertEquals(1, preIndex.get()); assertEquals(1, postIndexCreate.get()); assertEquals(0, postIndexUpdate.get()); @@ -611,7 +608,7 @@ public class IndexShardTests extends IndexShardTestCase { assertEquals(0, postDelete.get()); assertEquals(0, postDeleteException.get()); - shard.execute(index); + shard.index(index); assertEquals(2, preIndex.get()); assertEquals(1, postIndexCreate.get()); assertEquals(1, postIndexUpdate.get()); @@ -621,7 +618,7 @@ public class IndexShardTests extends IndexShardTestCase { assertEquals(0, postDeleteException.get()); Engine.Delete delete = new Engine.Delete("test", "1", new Term("_uid", "1")); - shard.execute(delete); + shard.delete(delete); assertEquals(2, preIndex.get()); assertEquals(1, postIndexCreate.get()); @@ -635,7 +632,7 @@ public class IndexShardTests extends IndexShardTestCase { shard.state = IndexShardState.STARTED; // It will generate exception try { - shard.execute(index); + shard.index(index); fail(); } catch (IllegalIndexShardStateException e) { @@ -649,7 +646,7 @@ public class IndexShardTests extends IndexShardTestCase { assertEquals(1, postDelete.get()); assertEquals(0, postDeleteException.get()); try { - shard.execute(delete); + shard.delete(delete); fail(); } catch (IllegalIndexShardStateException e) { @@ -1124,27 +1121,26 @@ public class IndexShardTests extends IndexShardTestCase { final AtomicInteger postDelete = new AtomicInteger(); IndexingOperationListener listener = new IndexingOperationListener() { @Override - public void preOperation(Engine.Operation operation) { - switch (operation.operationType()) { - case INDEX: - preIndex.incrementAndGet(); - break; - case DELETE: - preDelete.incrementAndGet(); - break; - } + public Engine.Index preIndex(Engine.Index operation) { + preIndex.incrementAndGet(); + return operation; } @Override - public void postOperation(Engine.Operation operation) { - switch (operation.operationType()) { - case INDEX: - postIndex.incrementAndGet(); - break; - case DELETE: - postDelete.incrementAndGet(); - break; - } + public void postIndex(Engine.Index index, boolean created) { + postIndex.incrementAndGet(); + } + + @Override + public Engine.Delete preDelete(Engine.Delete delete) { + preDelete.incrementAndGet(); + return delete; + } + + @Override + public void postDelete(Engine.Delete delete) { + postDelete.incrementAndGet(); + } }; final IndexShard newShard = reinitShard(shard, listener); diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java index bb652c6630b..d1cf8b32f58 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java @@ -40,55 +40,63 @@ public class IndexingOperationListenerTests extends ESTestCase{ AtomicInteger postDeleteException = new AtomicInteger(); IndexingOperationListener listener = new IndexingOperationListener() { @Override - public void preOperation(Engine.Operation operation) { - switch (operation.operationType()) { - case INDEX: - preIndex.incrementAndGet(); - break; - case DELETE: - preDelete.incrementAndGet(); - break; - } + public Engine.Index preIndex(Engine.Index operation) { + preIndex.incrementAndGet(); + return operation; } @Override - public void postOperation(Engine.Operation operation) { - switch (operation.operationType()) { - case INDEX: - postIndex.incrementAndGet(); - break; - case DELETE: - postDelete.incrementAndGet(); - break; - } + public void postIndex(Engine.Index index, boolean created) { + postIndex.incrementAndGet(); } @Override - public void postOperation(Engine.Operation operation, Exception ex) { - switch (operation.operationType()) { - case INDEX: - postIndexException.incrementAndGet(); - break; - case DELETE: - postDeleteException.incrementAndGet(); - break; - } + public void postIndex(Engine.Index index, Exception ex) { + postIndexException.incrementAndGet(); + } + + @Override + public Engine.Delete preDelete(Engine.Delete delete) { + preDelete.incrementAndGet(); + return delete; + } + + @Override + public void postDelete(Engine.Delete delete) { + postDelete.incrementAndGet(); + } + + @Override + public void postDelete(Engine.Delete delete, Exception ex) { + postDeleteException.incrementAndGet(); } }; IndexingOperationListener throwingListener = new IndexingOperationListener() { @Override - public void preOperation(Engine.Operation operation) { + public Engine.Index preIndex(Engine.Index operation) { throw new RuntimeException(); } @Override - public void postOperation(Engine.Operation operation) { + public void postIndex(Engine.Index index, boolean created) { + throw new RuntimeException(); } + + @Override + public void postIndex(Engine.Index index, Exception ex) { + throw new RuntimeException(); } + + @Override + public Engine.Delete preDelete(Engine.Delete delete) { throw new RuntimeException(); } @Override - public void postOperation(Engine.Operation operation, Exception ex) { + public void postDelete(Engine.Delete delete) { + throw new RuntimeException(); } + + @Override + public void postDelete(Engine.Delete delete, Exception ex) { throw new RuntimeException(); } }; @@ -103,7 +111,7 @@ public class IndexingOperationListenerTests extends ESTestCase{ IndexingOperationListener.CompositeListener compositeListener = new IndexingOperationListener.CompositeListener(indexingOperationListeners, logger); Engine.Delete delete = new Engine.Delete("test", "1", new Term("_uid", "1")); Engine.Index index = new Engine.Index(new Term("_uid", "1"), null); - compositeListener.postOperation(delete); + compositeListener.postDelete(delete); assertEquals(0, preIndex.get()); assertEquals(0, postIndex.get()); assertEquals(0, postIndexException.get()); @@ -111,7 +119,7 @@ public class IndexingOperationListenerTests extends ESTestCase{ assertEquals(2, postDelete.get()); assertEquals(0, postDeleteException.get()); - compositeListener.postOperation(delete, new RuntimeException()); + compositeListener.postDelete(delete, new RuntimeException()); assertEquals(0, preIndex.get()); assertEquals(0, postIndex.get()); assertEquals(0, postIndexException.get()); @@ -119,7 +127,7 @@ public class IndexingOperationListenerTests extends ESTestCase{ assertEquals(2, postDelete.get()); assertEquals(2, postDeleteException.get()); - compositeListener.preOperation(delete); + compositeListener.preDelete(delete); assertEquals(0, preIndex.get()); assertEquals(0, postIndex.get()); assertEquals(0, postIndexException.get()); @@ -127,7 +135,7 @@ public class IndexingOperationListenerTests extends ESTestCase{ assertEquals(2, postDelete.get()); assertEquals(2, postDeleteException.get()); - compositeListener.postOperation(index); + compositeListener.postIndex(index, false); assertEquals(0, preIndex.get()); assertEquals(2, postIndex.get()); assertEquals(0, postIndexException.get()); @@ -135,7 +143,7 @@ public class IndexingOperationListenerTests extends ESTestCase{ assertEquals(2, postDelete.get()); assertEquals(2, postDeleteException.get()); - compositeListener.postOperation(index, new RuntimeException()); + compositeListener.postIndex(index, new RuntimeException()); assertEquals(0, preIndex.get()); assertEquals(2, postIndex.get()); assertEquals(2, postIndexException.get()); @@ -143,7 +151,7 @@ public class IndexingOperationListenerTests extends ESTestCase{ assertEquals(2, postDelete.get()); assertEquals(2, postDeleteException.get()); - compositeListener.preOperation(index); + compositeListener.preIndex(index); assertEquals(2, preIndex.get()); assertEquals(2, postIndex.get()); assertEquals(2, postIndexException.get()); diff --git a/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java b/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java index 05147d4a72a..f0f53d9fdc9 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java @@ -137,14 +137,14 @@ public class RefreshListenersTests extends ESTestCase { public void testTooMany() throws Exception { assertFalse(listeners.refreshNeeded()); - Engine.Index index = index("1"); + Engine.IndexResult index = index("1"); // Fill the listener slots List nonForcedListeners = new ArrayList<>(maxListeners); for (int i = 0; i < maxListeners; i++) { DummyRefreshListener listener = new DummyRefreshListener(); nonForcedListeners.add(listener); - listeners.addOrNotify(index.getTranslogLocation(), listener); + listeners.addOrNotify(index.getLocation(), listener); assertTrue(listeners.refreshNeeded()); } @@ -155,7 +155,7 @@ public class RefreshListenersTests extends ESTestCase { // Add one more listener which should cause a refresh. DummyRefreshListener forcingListener = new DummyRefreshListener(); - listeners.addOrNotify(index.getTranslogLocation(), forcingListener); + listeners.addOrNotify(index.getLocation(), forcingListener); assertTrue("Forced listener wasn't forced?", forcingListener.forcedRefresh.get()); forcingListener.assertNoError(); @@ -168,7 +168,7 @@ public class RefreshListenersTests extends ESTestCase { } public void testAfterRefresh() throws Exception { - Engine.Index index = index("1"); + Engine.IndexResult index = index("1"); engine.refresh("I said so"); if (randomBoolean()) { index(randomFrom("1" /* same document */, "2" /* different document */)); @@ -178,7 +178,7 @@ public class RefreshListenersTests extends ESTestCase { } DummyRefreshListener listener = new DummyRefreshListener(); - assertTrue(listeners.addOrNotify(index.getTranslogLocation(), listener)); + assertTrue(listeners.addOrNotify(index.getLocation(), listener)); assertFalse(listener.forcedRefresh.get()); listener.assertNoError(); } @@ -198,9 +198,9 @@ public class RefreshListenersTests extends ESTestCase { refresher.start(); try { for (int i = 0; i < 1000; i++) { - Engine.Index index = index("1"); + Engine.IndexResult index = index("1"); DummyRefreshListener listener = new DummyRefreshListener(); - boolean immediate = listeners.addOrNotify(index.getTranslogLocation(), listener); + boolean immediate = listeners.addOrNotify(index.getLocation(), listener); if (immediate) { assertNotNull(listener.forcedRefresh.get()); } else { @@ -234,18 +234,18 @@ public class RefreshListenersTests extends ESTestCase { for (int iteration = 1; iteration <= 50; iteration++) { try { String testFieldValue = String.format(Locale.ROOT, "%s%04d", threadId, iteration); - Engine.Index index = index(threadId, testFieldValue); - assertEquals(iteration, index.version()); + Engine.IndexResult index = index(threadId, testFieldValue); + assertEquals(iteration, index.getVersion()); DummyRefreshListener listener = new DummyRefreshListener(); - listeners.addOrNotify(index.getTranslogLocation(), listener); + listeners.addOrNotify(index.getLocation(), listener); assertBusy(() -> assertNotNull("listener never called", listener.forcedRefresh.get())); if (threadCount < maxListeners) { assertFalse(listener.forcedRefresh.get()); } listener.assertNoError(); - Engine.Get get = new Engine.Get(false, index.uid()); + Engine.Get get = new Engine.Get(false, new Term("_uid", "test:"+threadId)); try (Engine.GetResult getResult = engine.get(get)) { assertTrue("document not found", getResult.exists()); assertEquals(iteration, getResult.version()); @@ -267,11 +267,11 @@ public class RefreshListenersTests extends ESTestCase { refresher.cancel(); } - private Engine.Index index(String id) { + private Engine.IndexResult index(String id) { return index(id, "test"); } - private Engine.Index index(String id, String testFieldValue) { + private Engine.IndexResult index(String id, String testFieldValue) { String type = "test"; String uid = type + ":" + id; Document document = new Document(); @@ -283,8 +283,7 @@ public class RefreshListenersTests extends ESTestCase { BytesReference source = new BytesArray(new byte[] { 1 }); ParsedDocument doc = new ParsedDocument(versionField, id, type, null, -1, -1, Arrays.asList(document), source, null); Engine.Index index = new Engine.Index(new Term("_uid", uid), doc); - engine.index(index); - return index; + return engine.index(index); } private static class DummyRefreshListener implements Consumer { diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/CancelTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/CancelTests.java index dd1250999ba..8da47f1eeaf 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/CancelTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/CancelTests.java @@ -197,14 +197,25 @@ public class CancelTests extends ReindexTestCase { } public static class BlockingOperationListener implements IndexingOperationListener { + @Override - public void preOperation(Engine.Operation operation) { - if ((TYPE.equals(operation.type()) == false) || (operation.origin() != Origin.PRIMARY)) { - return; + public Engine.Index preIndex(Engine.Index index) { + return preCheck(index, index.type()); + } + + @Override + public Engine.Delete preDelete(Engine.Delete delete) { + return preCheck(delete, delete.type()); + } + + private T preCheck(T operation, String type) { + if ((TYPE.equals(type) == false) || (operation.origin() != Origin.PRIMARY)) { + return operation; } + try { if (ALLOWED_OPERATIONS.tryAcquire(30, TimeUnit.SECONDS)) { - return; + return operation; } } catch (InterruptedException e) { throw new RuntimeException(e); diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index d86c08bbb05..c73f22fa73b 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -451,7 +451,7 @@ public abstract class IndexShardTestCase extends ESTestCase { SourceToParse.source(SourceToParse.Origin.PRIMARY, shard.shardId().getIndexName(), type, id, new BytesArray(source)), 1, VersionType.EXTERNAL, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); } - shard.execute(index); + shard.index(index); return ((Engine.Index) index); } @@ -462,7 +462,7 @@ public abstract class IndexShardTestCase extends ESTestCase { } else { delete = shard.prepareDeleteOnPrimary(type, id, 1, VersionType.EXTERNAL); } - shard.execute(delete); + shard.delete(delete); return delete; } From 1aee578aa18d258105635a3cc4314936b8697378 Mon Sep 17 00:00:00 2001 From: Areek Zillur Date: Tue, 25 Oct 2016 09:12:39 -0400 Subject: [PATCH 011/132] add operation result as a parameter to postIndex/delete in indexing operation listener --- .../action/bulk/TransportShardBulkAction.java | 3 ++- .../elasticsearch/index/IndexingSlowLog.java | 11 +++------- .../index/engine/InternalEngine.java | 16 ++++++++------ .../shard/IndexingOperationListener.java | 12 +++++------ .../index/shard/InternalIndexingStats.java | 8 +++---- .../indices/IndexingMemoryController.java | 18 +++++++--------- .../index/shard/IndexShardIT.java | 4 ++-- .../index/shard/IndexShardTests.java | 10 ++++----- .../shard/IndexingOperationListenerTests.java | 21 +++++++++++-------- .../index/shard/IndexShardTestCase.java | 4 ++-- 10 files changed, 53 insertions(+), 54 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 0b27b4c037c..24367fae90a 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -242,7 +242,8 @@ public class TransportShardBulkAction extends TransportWriteAction= 0 && tookInNanos > indexWarnThreshold) { indexLogger.warn("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog)); } else if (indexInfoThreshold >= 0 && tookInNanos > indexInfoThreshold) { diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 7ec35d73a38..d02097141b1 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -422,7 +422,8 @@ public class InternalEngine extends Engine { } } catch (Exception e) { Exception transientOperationFailure = handleOperationFailure(index, e); - result = new IndexResult(transientOperationFailure, index.version(), index.startTime() - System.nanoTime()); + result = new IndexResult(transientOperationFailure, index.version(), + index.startTime() - System.nanoTime(), index.estimatedSizeInBytes()); } return result; } @@ -550,7 +551,7 @@ public class InternalEngine extends Engine { final long expectedVersion = index.version(); if (checkVersionConflict(index, currentVersion, expectedVersion, deleted)) { // skip index operation because of version conflict on recovery - return new IndexResult(null, expectedVersion, false, index.startTime() - System.nanoTime()); + return new IndexResult(null, expectedVersion, false, index.startTime() - System.nanoTime(), index.estimatedSizeInBytes()); } else { updatedVersion = index.versionType().updateVersion(currentVersion, expectedVersion); index.parsedDoc().version().setLongValue(updatedVersion); @@ -561,7 +562,7 @@ public class InternalEngine extends Engine { update(index.uid(), index.docs(), indexWriter); } location = maybeAddToTranslog(index, updatedVersion, Translog.Index::new, NEW_VERSION_VALUE); - return new IndexResult(location, updatedVersion, deleted, index.startTime() - System.nanoTime()); + return new IndexResult(location, updatedVersion, deleted, index.startTime() - System.nanoTime(), index.estimatedSizeInBytes()); } } } @@ -591,7 +592,8 @@ public class InternalEngine extends Engine { result = innerDelete(delete); } catch (Exception e) { Exception transientOperationFailure = handleOperationFailure(delete, e); - result = new DeleteResult(transientOperationFailure, delete.version(), delete.startTime() - System.nanoTime()); + result = new DeleteResult(transientOperationFailure, delete.version(), + delete.startTime() - System.nanoTime(), delete.estimatedSizeInBytes()); } maybePruneDeletedTombstones(); return result; @@ -626,12 +628,14 @@ public class InternalEngine extends Engine { final long expectedVersion = delete.version(); if (checkVersionConflict(delete, currentVersion, expectedVersion, deleted)) { // skip executing delete because of version conflict on recovery - return new DeleteResult(null, expectedVersion, true, delete.startTime() - System.nanoTime()); + return new DeleteResult(null, expectedVersion, true, + delete.startTime() - System.nanoTime(), delete.estimatedSizeInBytes()); } else { updatedVersion = delete.versionType().updateVersion(currentVersion, expectedVersion); found = deleteIfFound(delete.uid(), currentVersion, deleted, versionValue); location = maybeAddToTranslog(delete, updatedVersion, Translog.Delete::new, DeleteVersionValue::new); - return new DeleteResult(location, updatedVersion, found, delete.startTime() - System.nanoTime()); + return new DeleteResult(location, updatedVersion, found, + delete.startTime() - System.nanoTime(), delete.estimatedSizeInBytes()); } } } diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java b/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java index 042ddec924e..0e605954248 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java @@ -40,7 +40,7 @@ public interface IndexingOperationListener { /** * Called after the indexing operation occurred. */ - default void postIndex(Engine.Index index, boolean created) {} + default void postIndex(Engine.Index index, Engine.IndexResult result) {} /** * Called after the indexing operation occurred with exception. @@ -58,7 +58,7 @@ public interface IndexingOperationListener { /** * Called after the delete operation occurred. */ - default void postDelete(Engine.Delete delete) {} + default void postDelete(Engine.Delete delete, Engine.DeleteResult result) {} /** * Called after the delete operation occurred with exception. @@ -91,11 +91,11 @@ public interface IndexingOperationListener { } @Override - public void postIndex(Engine.Index index, boolean created) { + public void postIndex(Engine.Index index, Engine.IndexResult result) { assert index != null; for (IndexingOperationListener listener : listeners) { try { - listener.postIndex(index, created); + listener.postIndex(index, result); } catch (Exception e) { logger.warn((Supplier) () -> new ParameterizedMessage("postIndex listener [{}] failed", listener), e); } @@ -129,11 +129,11 @@ public interface IndexingOperationListener { } @Override - public void postDelete(Engine.Delete delete) { + public void postDelete(Engine.Delete delete, Engine.DeleteResult result) { assert delete != null; for (IndexingOperationListener listener : listeners) { try { - listener.postDelete(delete); + listener.postDelete(delete, result); } catch (Exception e) { logger.warn((Supplier) () -> new ParameterizedMessage("postDelete listener [{}] failed", listener), e); } diff --git a/core/src/main/java/org/elasticsearch/index/shard/InternalIndexingStats.java b/core/src/main/java/org/elasticsearch/index/shard/InternalIndexingStats.java index f62b8f7fe3c..39a415ca8eb 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/InternalIndexingStats.java +++ b/core/src/main/java/org/elasticsearch/index/shard/InternalIndexingStats.java @@ -74,9 +74,9 @@ final class InternalIndexingStats implements IndexingOperationListener { } @Override - public void postIndex(Engine.Index index, boolean created) { + public void postIndex(Engine.Index index, Engine.IndexResult result) { if (!index.origin().isRecovery()) { - long took = index.endTime() - index.startTime(); + long took = result.getTook(); totalStats.indexMetric.inc(took); totalStats.indexCurrent.dec(); StatsHolder typeStats = typeStats(index.type()); @@ -106,9 +106,9 @@ final class InternalIndexingStats implements IndexingOperationListener { } @Override - public void postDelete(Engine.Delete delete) { + public void postDelete(Engine.Delete delete, Engine.DeleteResult result) { if (!delete.origin().isRecovery()) { - long took = delete.endTime() - delete.startTime(); + long took = result.getTook(); totalStats.deleteMetric.inc(took); totalStats.deleteCurrent.dec(); StatsHolder typeStats = typeStats(delete.type()); diff --git a/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java b/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java index 3b4258a8bdf..25b133f390d 100644 --- a/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java +++ b/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java @@ -189,11 +189,6 @@ public class IndexingMemoryController extends AbstractComponent implements Index statusChecker.run(); } - /** called by IndexShard to record that this many bytes were written to translog */ - public void bytesWritten(int bytes) { - statusChecker.bytesWritten(bytes); - } - /** Asks this shard to throttle indexing to one thread */ protected void activateThrottling(IndexShard shard) { shard.activateThrottling(); @@ -205,17 +200,18 @@ public class IndexingMemoryController extends AbstractComponent implements Index } @Override - public void postIndex(Engine.Index index, boolean created) { - recordOperationBytes(index); + public void postIndex(Engine.Index index, Engine.IndexResult result) { + recordOperationBytes(result); } @Override - public void postDelete(Engine.Delete delete) { - recordOperationBytes(delete); + public void postDelete(Engine.Delete delete, Engine.DeleteResult result) { + recordOperationBytes(result); } - private void recordOperationBytes(Engine.Operation op) { - bytesWritten(op.sizeInBytes()); + /** called by IndexShard to record that this many bytes were written to translog */ + private void recordOperationBytes(Engine.Result result) { + statusChecker.bytesWritten(result.getSizeInBytes()); } private static final class ShardAndBytesUsed implements Comparable { diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java index 2248ff156ac..9cf6594e42c 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java @@ -408,7 +408,7 @@ public class IndexShardIT extends ESSingleNodeTestCase { IndexingOperationListener listener = new IndexingOperationListener() { @Override - public void postIndex(Engine.Index index, boolean created) { + public void postIndex(Engine.Index index, Engine.IndexResult result) { try { assertNotNull(shardRef.get()); // this is all IMC needs to do - check current memory and refresh @@ -422,7 +422,7 @@ public class IndexShardIT extends ESSingleNodeTestCase { @Override - public void postDelete(Engine.Delete delete) { + public void postDelete(Engine.Delete delete, Engine.DeleteResult result) { try { assertNotNull(shardRef.get()); // this is all IMC needs to do - check current memory and refresh diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 9c6f21c6958..2b417c10ae7 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -564,8 +564,8 @@ public class IndexShardTests extends IndexShardTestCase { } @Override - public void postIndex(Engine.Index index, boolean created) { - if (created) { + public void postIndex(Engine.Index index, Engine.IndexResult result) { + if (result.isCreated()) { postIndexCreate.incrementAndGet(); } else { postIndexUpdate.incrementAndGet(); @@ -584,7 +584,7 @@ public class IndexShardTests extends IndexShardTestCase { } @Override - public void postDelete(Engine.Delete delete) { + public void postDelete(Engine.Delete delete, Engine.DeleteResult result) { postDelete.incrementAndGet(); } @@ -1127,7 +1127,7 @@ public class IndexShardTests extends IndexShardTestCase { } @Override - public void postIndex(Engine.Index index, boolean created) { + public void postIndex(Engine.Index index, Engine.IndexResult result) { postIndex.incrementAndGet(); } @@ -1138,7 +1138,7 @@ public class IndexShardTests extends IndexShardTestCase { } @Override - public void postDelete(Engine.Delete delete) { + public void postDelete(Engine.Delete delete, Engine.DeleteResult result) { postDelete.incrementAndGet(); } diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java index d1cf8b32f58..15b40e4e09c 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java @@ -46,7 +46,7 @@ public class IndexingOperationListenerTests extends ESTestCase{ } @Override - public void postIndex(Engine.Index index, boolean created) { + public void postIndex(Engine.Index index, Engine.IndexResult result) { postIndex.incrementAndGet(); } @@ -62,7 +62,7 @@ public class IndexingOperationListenerTests extends ESTestCase{ } @Override - public void postDelete(Engine.Delete delete) { + public void postDelete(Engine.Delete delete, Engine.DeleteResult result) { postDelete.incrementAndGet(); } @@ -79,12 +79,14 @@ public class IndexingOperationListenerTests extends ESTestCase{ } @Override - public void postIndex(Engine.Index index, boolean created) { - throw new RuntimeException(); } + public void postIndex(Engine.Index index, Engine.IndexResult result) { + throw new RuntimeException(); + } @Override public void postIndex(Engine.Index index, Exception ex) { - throw new RuntimeException(); } + throw new RuntimeException(); + } @Override public Engine.Delete preDelete(Engine.Delete delete) { @@ -92,8 +94,9 @@ public class IndexingOperationListenerTests extends ESTestCase{ } @Override - public void postDelete(Engine.Delete delete) { - throw new RuntimeException(); } + public void postDelete(Engine.Delete delete, Engine.DeleteResult result) { + throw new RuntimeException(); + } @Override public void postDelete(Engine.Delete delete, Exception ex) { @@ -111,7 +114,7 @@ public class IndexingOperationListenerTests extends ESTestCase{ IndexingOperationListener.CompositeListener compositeListener = new IndexingOperationListener.CompositeListener(indexingOperationListeners, logger); Engine.Delete delete = new Engine.Delete("test", "1", new Term("_uid", "1")); Engine.Index index = new Engine.Index(new Term("_uid", "1"), null); - compositeListener.postDelete(delete); + compositeListener.postDelete(delete, new Engine.DeleteResult(null, 1, true, 0, 0)); assertEquals(0, preIndex.get()); assertEquals(0, postIndex.get()); assertEquals(0, postIndexException.get()); @@ -135,7 +138,7 @@ public class IndexingOperationListenerTests extends ESTestCase{ assertEquals(2, postDelete.get()); assertEquals(2, postDeleteException.get()); - compositeListener.postIndex(index, false); + compositeListener.postIndex(index, new Engine.IndexResult(null, 0, false, 0, 0)); assertEquals(0, preIndex.get()); assertEquals(2, postIndex.get()); assertEquals(0, postIndexException.get()); diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index c73f22fa73b..fbb87d9f8d1 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -441,7 +441,7 @@ public abstract class IndexShardTestCase extends ESTestCase { } protected Engine.Index indexDoc(IndexShard shard, String type, String id, String source) { - final Engine.Operation index; + final Engine.Index index; if (shard.routingEntry().primary()) { index = shard.prepareIndexOnPrimary( SourceToParse.source(SourceToParse.Origin.PRIMARY, shard.shardId().getIndexName(), type, id, new BytesArray(source)), @@ -452,7 +452,7 @@ public abstract class IndexShardTestCase extends ESTestCase { 1, VersionType.EXTERNAL, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); } shard.index(index); - return ((Engine.Index) index); + return index; } protected Engine.Delete deleteDoc(IndexShard shard, String type, String id) { From 168946ad5a7ea4a483c3423f15e2aa430a168306 Mon Sep 17 00:00:00 2001 From: Areek Zillur Date: Tue, 25 Oct 2016 09:22:49 -0400 Subject: [PATCH 012/132] Improve documentation for handling write operation failure --- .../index/engine/InternalEngine.java | 34 +++++++++---------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index d02097141b1..806027d3ef7 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -421,22 +421,22 @@ public class InternalEngine extends Engine { } } } catch (Exception e) { - Exception transientOperationFailure = handleOperationFailure(index, e); - result = new IndexResult(transientOperationFailure, index.version(), + Exception documentFailure = extractDocumentFailure(index, e); + result = new IndexResult(documentFailure, index.version(), index.startTime() - System.nanoTime(), index.estimatedSizeInBytes()); } return result; } /** - * Handle failures executing write operations, distinguish persistent engine (environment) failures - * from document (request) specific failures. - * Write failures that fail the engine as a side-effect, are thrown wrapped in {@link OperationFailedEngineException} - * and document specific failures are returned to be set on the {@link Engine.Result} to be handled - * at the transport level. + * Inspects exception thrown when executing index or delete operations + * + * @return failure if the failure is a document specific failure (e.g. analysis chain failure) + * @throws OperationFailedEngineException if the failure caused the engine to fail + * (e.g. out of disk, lucene tragic event) */ - private Exception handleOperationFailure(final Operation operation, final Exception failure) { - boolean isEnvironmentFailure; + private Exception extractDocumentFailure(final Operation operation, final Exception failure) { + boolean isDocumentFailure; try { // When indexing a document into Lucene, Lucene distinguishes between environment related errors // (like out of disk space) and document specific errors (like analysis chain problems) by setting @@ -444,18 +444,18 @@ public class InternalEngine extends Engine { // errors and returns true if that is the case. We use that to indicate a document level failure // and set the error in operation.setFailure. In case of environment related errors, the failure // is bubbled up - isEnvironmentFailure = (failure instanceof IllegalStateException || failure instanceof IOException) - && maybeFailEngine(operation.operationType().getLowercase(), failure); + isDocumentFailure = !((failure instanceof IllegalStateException || failure instanceof IOException) + && maybeFailEngine(operation.operationType().getLowercase(), failure)); } catch (Exception inner) { // we failed checking whether the failure can fail the engine, treat it as a persistent engine failure - isEnvironmentFailure = true; + isDocumentFailure = false; failure.addSuppressed(inner); } - if (isEnvironmentFailure) { + if (isDocumentFailure) { + return failure; + } else { throw new OperationFailedEngineException(shardId, operation.operationType().getLowercase(), operation.type(), operation.id(), failure); - } else { - return failure; } } @@ -591,8 +591,8 @@ public class InternalEngine extends Engine { // NOTE: we don't throttle this when merges fall behind because delete-by-id does not create new segments: result = innerDelete(delete); } catch (Exception e) { - Exception transientOperationFailure = handleOperationFailure(delete, e); - result = new DeleteResult(transientOperationFailure, delete.version(), + Exception documentFailure = extractDocumentFailure(delete, e); + result = new DeleteResult(documentFailure, delete.version(), delete.startTime() - System.nanoTime(), delete.estimatedSizeInBytes()); } maybePruneDeletedTombstones(); From bb785483ae41e30a756a17f3fb6cdb5d06a1ad6c Mon Sep 17 00:00:00 2001 From: Areek Zillur Date: Tue, 25 Oct 2016 09:33:04 -0400 Subject: [PATCH 013/132] cleanup indexing operation listener --- .../elasticsearch/index/shard/IndexShard.java | 12 ++--- .../shard/IndexingOperationListener.java | 47 +++++-------------- .../index/shard/InternalIndexingStats.java | 7 +-- .../elasticsearch/index/IndexModuleTests.java | 3 +- .../index/shard/IndexShardTests.java | 12 ++--- .../shard/IndexingOperationListenerTests.java | 10 ++-- .../index/reindex/CancelTests.java | 14 +++--- 7 files changed, 37 insertions(+), 68 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 442adc98196..9413fb5bcc6 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -536,16 +536,16 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl return new Engine.Index(uid, doc, version, versionType, origin, startTime, autoGeneratedIdTimestamp, isRetry); } - public Engine.IndexResult index(Engine.Index index) { + public Engine.IndexResult index(final Engine.Index index) { ensureWriteAllowed(index); Engine engine = getEngine(); return index(engine, index); } - private Engine.IndexResult index(Engine engine, Engine.Index index) { + private Engine.IndexResult index(final Engine engine, final Engine.Index index) { active.set(true); final Engine.IndexResult result; - index = indexingOperationListeners.preIndex(index); + indexingOperationListeners.preIndex(index); try { if (logger.isTraceEnabled()) { logger.trace("index [{}][{}]{}", index.type(), index.id(), index.docs()); @@ -585,16 +585,16 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl return new Engine.Delete(type, id, uid, version, versionType, origin, startTime); } - public Engine.DeleteResult delete(Engine.Delete delete) { + public Engine.DeleteResult delete(final Engine.Delete delete) { ensureWriteAllowed(delete); Engine engine = getEngine(); return delete(engine, delete); } - private Engine.DeleteResult delete(Engine engine, Engine.Delete delete) { + private Engine.DeleteResult delete(final Engine engine, final Engine.Delete delete) { active.set(true); final Engine.DeleteResult result; - delete = indexingOperationListeners.preDelete(delete); + indexingOperationListeners.preDelete(delete); try { if (logger.isTraceEnabled()) { logger.trace("delete [{}]", delete.uid().text()); diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java b/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java index 0e605954248..9c870859650 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java @@ -30,44 +30,25 @@ import java.util.List; */ public interface IndexingOperationListener { - /** - * Called before the indexing occurs. - */ - default Engine.Index preIndex(Engine.Index operation) { - return operation; - } + /** Called before the indexing occurs */ + default void preIndex(Engine.Index operation) {} - /** - * Called after the indexing operation occurred. - */ + /** Called after the indexing operation occurred */ default void postIndex(Engine.Index index, Engine.IndexResult result) {} - /** - * Called after the indexing operation occurred with exception. - */ + /** Called after the indexing operation occurred with exception */ default void postIndex(Engine.Index index, Exception ex) {} - /** - * Called before the delete occurs. - */ - default Engine.Delete preDelete(Engine.Delete delete) { - return delete; - } + /** Called before the delete occurs */ + default void preDelete(Engine.Delete delete) {} - - /** - * Called after the delete operation occurred. - */ + /** Called after the delete operation occurred */ default void postDelete(Engine.Delete delete, Engine.DeleteResult result) {} - /** - * Called after the delete operation occurred with exception. - */ + /** Called after the delete operation occurred with exception */ default void postDelete(Engine.Delete delete, Exception ex) {} - /** - * A Composite listener that multiplexes calls to each of the listeners methods. - */ + /** A Composite listener that multiplexes calls to each of the listeners methods */ final class CompositeListener implements IndexingOperationListener{ private final List listeners; private final Logger logger; @@ -78,7 +59,7 @@ public interface IndexingOperationListener { } @Override - public Engine.Index preIndex(Engine.Index operation) { + public void preIndex(Engine.Index operation) { assert operation != null; for (IndexingOperationListener listener : listeners) { try { @@ -87,12 +68,11 @@ public interface IndexingOperationListener { logger.warn((Supplier) () -> new ParameterizedMessage("preIndex listener [{}] failed", listener), e); } } - return operation; } @Override public void postIndex(Engine.Index index, Engine.IndexResult result) { - assert index != null; + assert index != null && result != null; for (IndexingOperationListener listener : listeners) { try { listener.postIndex(index, result); @@ -116,7 +96,7 @@ public interface IndexingOperationListener { } @Override - public Engine.Delete preDelete(Engine.Delete delete) { + public void preDelete(Engine.Delete delete) { assert delete != null; for (IndexingOperationListener listener : listeners) { try { @@ -125,12 +105,11 @@ public interface IndexingOperationListener { logger.warn((Supplier) () -> new ParameterizedMessage("preDelete listener [{}] failed", listener), e); } } - return delete; } @Override public void postDelete(Engine.Delete delete, Engine.DeleteResult result) { - assert delete != null; + assert delete != null && result != null; for (IndexingOperationListener listener : listeners) { try { listener.postDelete(delete, result); diff --git a/core/src/main/java/org/elasticsearch/index/shard/InternalIndexingStats.java b/core/src/main/java/org/elasticsearch/index/shard/InternalIndexingStats.java index 39a415ca8eb..61cfe2fc4b5 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/InternalIndexingStats.java +++ b/core/src/main/java/org/elasticsearch/index/shard/InternalIndexingStats.java @@ -65,12 +65,11 @@ final class InternalIndexingStats implements IndexingOperationListener { } @Override - public Engine.Index preIndex(Engine.Index operation) { + public void preIndex(Engine.Index operation) { if (!operation.origin().isRecovery()) { totalStats.indexCurrent.inc(); typeStats(operation.type()).indexCurrent.inc(); } - return operation; } @Override @@ -96,13 +95,11 @@ final class InternalIndexingStats implements IndexingOperationListener { } @Override - public Engine.Delete preDelete(Engine.Delete delete) { + public void preDelete(Engine.Delete delete) { if (!delete.origin().isRecovery()) { totalStats.deleteCurrent.inc(); typeStats(delete.type()).deleteCurrent.inc(); } - return delete; - } @Override diff --git a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java index 5e9d1ffaf9e..8dcad1b8158 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java +++ b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java @@ -233,9 +233,8 @@ public class IndexModuleTests extends ESTestCase { AtomicBoolean executed = new AtomicBoolean(false); IndexingOperationListener listener = new IndexingOperationListener() { @Override - public Engine.Index preIndex(Engine.Index operation) { + public void preIndex(Engine.Index operation) { executed.set(true); - return operation; } }; module.addIndexOperationListener(listener); diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 2b417c10ae7..901dcc3b724 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -558,9 +558,8 @@ public class IndexShardTests extends IndexShardTestCase { shard.close("simon says", true); shard = reinitShard(shard, new IndexingOperationListener() { @Override - public Engine.Index preIndex(Engine.Index operation) { + public void preIndex(Engine.Index operation) { preIndex.incrementAndGet(); - return operation; } @Override @@ -578,9 +577,8 @@ public class IndexShardTests extends IndexShardTestCase { } @Override - public Engine.Delete preDelete(Engine.Delete delete) { + public void preDelete(Engine.Delete delete) { preDelete.incrementAndGet(); - return delete; } @Override @@ -1121,9 +1119,8 @@ public class IndexShardTests extends IndexShardTestCase { final AtomicInteger postDelete = new AtomicInteger(); IndexingOperationListener listener = new IndexingOperationListener() { @Override - public Engine.Index preIndex(Engine.Index operation) { + public void preIndex(Engine.Index operation) { preIndex.incrementAndGet(); - return operation; } @Override @@ -1132,9 +1129,8 @@ public class IndexShardTests extends IndexShardTestCase { } @Override - public Engine.Delete preDelete(Engine.Delete delete) { + public void preDelete(Engine.Delete delete) { preDelete.incrementAndGet(); - return delete; } @Override diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java index 15b40e4e09c..d3cf1640baa 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java @@ -40,9 +40,8 @@ public class IndexingOperationListenerTests extends ESTestCase{ AtomicInteger postDeleteException = new AtomicInteger(); IndexingOperationListener listener = new IndexingOperationListener() { @Override - public Engine.Index preIndex(Engine.Index operation) { + public void preIndex(Engine.Index operation) { preIndex.incrementAndGet(); - return operation; } @Override @@ -56,9 +55,8 @@ public class IndexingOperationListenerTests extends ESTestCase{ } @Override - public Engine.Delete preDelete(Engine.Delete delete) { + public void preDelete(Engine.Delete delete) { preDelete.incrementAndGet(); - return delete; } @Override @@ -74,7 +72,7 @@ public class IndexingOperationListenerTests extends ESTestCase{ IndexingOperationListener throwingListener = new IndexingOperationListener() { @Override - public Engine.Index preIndex(Engine.Index operation) { + public void preIndex(Engine.Index operation) { throw new RuntimeException(); } @@ -89,7 +87,7 @@ public class IndexingOperationListenerTests extends ESTestCase{ } @Override - public Engine.Delete preDelete(Engine.Delete delete) { + public void preDelete(Engine.Delete delete) { throw new RuntimeException(); } diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/CancelTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/CancelTests.java index 8da47f1eeaf..21b3a1aa7a6 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/CancelTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/CancelTests.java @@ -199,23 +199,23 @@ public class CancelTests extends ReindexTestCase { public static class BlockingOperationListener implements IndexingOperationListener { @Override - public Engine.Index preIndex(Engine.Index index) { - return preCheck(index, index.type()); + public void preIndex(Engine.Index index) { + preCheck(index, index.type()); } @Override - public Engine.Delete preDelete(Engine.Delete delete) { - return preCheck(delete, delete.type()); + public void preDelete(Engine.Delete delete) { + preCheck(delete, delete.type()); } - private T preCheck(T operation, String type) { + private void preCheck(Engine.Operation operation, String type) { if ((TYPE.equals(type) == false) || (operation.origin() != Origin.PRIMARY)) { - return operation; + return; } try { if (ALLOWED_OPERATIONS.tryAcquire(30, TimeUnit.SECONDS)) { - return operation; + return; } } catch (InterruptedException e) { throw new RuntimeException(e); From 64a897e5f2ad0274e04db7831bcc8c5334bfee1a Mon Sep 17 00:00:00 2001 From: Areek Zillur Date: Tue, 25 Oct 2016 09:58:14 -0400 Subject: [PATCH 014/132] add setters for translog location and took in engine operation result --- .../action/bulk/TransportShardBulkAction.java | 2 +- .../action/index/TransportIndexAction.java | 8 +-- .../elasticsearch/index/engine/Engine.java | 57 +++++++++++------- .../index/engine/InternalEngine.java | 59 +++++++++---------- .../index/translog/Translog.java | 8 +-- .../shard/IndexingOperationListenerTests.java | 4 +- 6 files changed, 75 insertions(+), 63 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 24367fae90a..61b9669f9a1 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -243,7 +243,7 @@ public class TransportShardBulkAction extends TransportWriteAction new VersionValue(u); - - @FunctionalInterface - private interface VersionValueSupplier { - VersionValue apply(long updatedVersion, long time); - } - - private Translog.Location maybeAddToTranslog( - final T op, - final long updatedVersion, - final Function toTranslogOp, - final VersionValueSupplier toVersionValue) throws IOException { - Translog.Location location = null; - if (op.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY) { - location = translog.add(toTranslogOp.apply(op)); - } - versionMap.putUnderLock(op.uid().bytes(), toVersionValue.apply(updatedVersion, engineConfig.getThreadPool().estimatedTimeInMillis())); - return location; - } - @Override public IndexResult index(Index index) { IndexResult result; @@ -423,7 +403,7 @@ public class InternalEngine extends Engine { } catch (Exception e) { Exception documentFailure = extractDocumentFailure(index, e); result = new IndexResult(documentFailure, index.version(), - index.startTime() - System.nanoTime(), index.estimatedSizeInBytes()); + index.estimatedSizeInBytes()); } return result; } @@ -551,7 +531,7 @@ public class InternalEngine extends Engine { final long expectedVersion = index.version(); if (checkVersionConflict(index, currentVersion, expectedVersion, deleted)) { // skip index operation because of version conflict on recovery - return new IndexResult(null, expectedVersion, false, index.startTime() - System.nanoTime(), index.estimatedSizeInBytes()); + return new IndexResult(expectedVersion, false, index.estimatedSizeInBytes()); } else { updatedVersion = index.versionType().updateVersion(currentVersion, expectedVersion); index.parsedDoc().version().setLongValue(updatedVersion); @@ -561,8 +541,17 @@ public class InternalEngine extends Engine { } else { update(index.uid(), index.docs(), indexWriter); } - location = maybeAddToTranslog(index, updatedVersion, Translog.Index::new, NEW_VERSION_VALUE); - return new IndexResult(location, updatedVersion, deleted, index.startTime() - System.nanoTime(), index.estimatedSizeInBytes()); + IndexResult indexResult = new IndexResult(updatedVersion, deleted, index.estimatedSizeInBytes()); + if (index.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY) { + location = translog.add(new Translog.Index(index, indexResult)); + } else { + location = null; + } + versionMap.putUnderLock(index.uid().bytes(), new VersionValue(updatedVersion)); + indexResult.setLocation(location); + indexResult.setTook(index.startTime() - System.nanoTime()); + indexResult.freeze(); + return indexResult; } } } @@ -593,7 +582,7 @@ public class InternalEngine extends Engine { } catch (Exception e) { Exception documentFailure = extractDocumentFailure(delete, e); result = new DeleteResult(documentFailure, delete.version(), - delete.startTime() - System.nanoTime(), delete.estimatedSizeInBytes()); + delete.estimatedSizeInBytes()); } maybePruneDeletedTombstones(); return result; @@ -628,14 +617,24 @@ public class InternalEngine extends Engine { final long expectedVersion = delete.version(); if (checkVersionConflict(delete, currentVersion, expectedVersion, deleted)) { // skip executing delete because of version conflict on recovery - return new DeleteResult(null, expectedVersion, true, - delete.startTime() - System.nanoTime(), delete.estimatedSizeInBytes()); + return new DeleteResult(expectedVersion, true, + delete.estimatedSizeInBytes()); } else { updatedVersion = delete.versionType().updateVersion(currentVersion, expectedVersion); found = deleteIfFound(delete.uid(), currentVersion, deleted, versionValue); - location = maybeAddToTranslog(delete, updatedVersion, Translog.Delete::new, DeleteVersionValue::new); - return new DeleteResult(location, updatedVersion, found, - delete.startTime() - System.nanoTime(), delete.estimatedSizeInBytes()); + DeleteResult deleteResult = new DeleteResult(updatedVersion, found, + delete.estimatedSizeInBytes()); + if (delete.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY) { + location = translog.add(new Translog.Delete(delete, deleteResult)); + } else { + location = null; + } + versionMap.putUnderLock(delete.uid().bytes(), + new DeleteVersionValue(updatedVersion, engineConfig.getThreadPool().estimatedTimeInMillis())); + deleteResult.setLocation(location); + deleteResult.setTook(delete.startTime() - System.nanoTime()); + deleteResult.freeze(); + return deleteResult; } } } diff --git a/core/src/main/java/org/elasticsearch/index/translog/Translog.java b/core/src/main/java/org/elasticsearch/index/translog/Translog.java index 056716a29bd..9cf60dbe422 100644 --- a/core/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/core/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -830,13 +830,13 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC } } - public Index(Engine.Index index) { + public Index(Engine.Index index, Engine.IndexResult indexResult) { this.id = index.id(); this.type = index.type(); this.source = index.source(); this.routing = index.routing(); this.parent = index.parent(); - this.version = index.version(); + this.version = indexResult.getVersion(); this.timestamp = index.timestamp(); this.ttl = index.ttl(); this.versionType = index.versionType(); @@ -994,9 +994,9 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC assert versionType.validateVersionForWrites(this.version); } - public Delete(Engine.Delete delete) { + public Delete(Engine.Delete delete, Engine.DeleteResult deleteResult) { this.uid = delete.uid(); - this.version = delete.version(); + this.version = deleteResult.getVersion(); this.versionType = delete.versionType(); } diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java index d3cf1640baa..742fadc9a80 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java @@ -112,7 +112,7 @@ public class IndexingOperationListenerTests extends ESTestCase{ IndexingOperationListener.CompositeListener compositeListener = new IndexingOperationListener.CompositeListener(indexingOperationListeners, logger); Engine.Delete delete = new Engine.Delete("test", "1", new Term("_uid", "1")); Engine.Index index = new Engine.Index(new Term("_uid", "1"), null); - compositeListener.postDelete(delete, new Engine.DeleteResult(null, 1, true, 0, 0)); + compositeListener.postDelete(delete, new Engine.DeleteResult(1, true, 0)); assertEquals(0, preIndex.get()); assertEquals(0, postIndex.get()); assertEquals(0, postIndexException.get()); @@ -136,7 +136,7 @@ public class IndexingOperationListenerTests extends ESTestCase{ assertEquals(2, postDelete.get()); assertEquals(2, postDeleteException.get()); - compositeListener.postIndex(index, new Engine.IndexResult(null, 0, false, 0, 0)); + compositeListener.postIndex(index, new Engine.IndexResult(0, false, 0)); assertEquals(0, preIndex.get()); assertEquals(2, postIndex.get()); assertEquals(0, postIndexException.get()); From 1ad1e2730da91a7ab90a525acae1c8e03c7a46d1 Mon Sep 17 00:00:00 2001 From: Areek Zillur Date: Tue, 25 Oct 2016 10:00:44 -0400 Subject: [PATCH 015/132] fix wildcard import --- .../elasticsearch/action/bulk/TransportShardBulkAction.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 61b9669f9a1..50db6adefc9 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -57,7 +57,8 @@ import org.elasticsearch.transport.TransportService; import java.util.Map; -import static org.elasticsearch.action.delete.TransportDeleteAction.*; +import static org.elasticsearch.action.delete.TransportDeleteAction.executeDeleteRequestOnPrimary; +import static org.elasticsearch.action.delete.TransportDeleteAction.executeDeleteRequestOnReplica; import static org.elasticsearch.action.index.TransportIndexAction.executeIndexRequestOnPrimary; import static org.elasticsearch.action.index.TransportIndexAction.executeIndexRequestOnReplica; import static org.elasticsearch.action.support.replication.ReplicationOperation.ignoreReplicaException; From 7a6f56a69236ff3503e1190f810899d414bc9cc9 Mon Sep 17 00:00:00 2001 From: Areek Zillur Date: Tue, 25 Oct 2016 10:22:32 -0400 Subject: [PATCH 016/132] fix tests --- .../index/engine/InternalEngineTests.java | 54 ++++++++++--------- .../index/engine/ShadowEngineTests.java | 4 +- 2 files changed, 30 insertions(+), 28 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 0b40d4cea52..a4df773983a 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -106,6 +106,7 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.IndexSettingsModule; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; +import org.hamcrest.CoreMatchers; import org.hamcrest.MatcherAssert; import org.junit.After; import org.junit.Before; @@ -1308,7 +1309,7 @@ public class InternalEngineTests extends ESTestCase { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); Engine.Index index = new Engine.Index(newUid("1"), doc); Engine.IndexResult indexResult = engine.index(index); - assertThat(indexResult.getFailure(), equalTo(1L)); + assertThat(indexResult.getVersion(), equalTo(1L)); index = new Engine.Index(newUid("1"), doc); indexResult = engine.index(index); @@ -1914,17 +1915,17 @@ public class InternalEngineTests extends ESTestCase { String uuidValue = "test#" + Integer.toString(randomId); ParsedDocument doc = testParsedDocument(uuidValue, Integer.toString(randomId), "test", null, -1, -1, testDocument(), new BytesArray("{}"), null); Engine.Index firstIndexRequest = new Engine.Index(newUid(uuidValue), doc, 1, VersionType.EXTERNAL, PRIMARY, System.nanoTime(), -1, false); - engine.index(firstIndexRequest); - assertThat(firstIndexRequest.version(), equalTo(1L)); + Engine.IndexResult indexResult = engine.index(firstIndexRequest); + assertThat(indexResult.getVersion(), equalTo(1L)); if (flush) { engine.flush(); } doc = testParsedDocument(uuidValue, Integer.toString(randomId), "test", null, -1, -1, testDocument(), new BytesArray("{}"), null); Engine.Index idxRequest = new Engine.Index(newUid(uuidValue), doc, 2, VersionType.EXTERNAL, PRIMARY, System.nanoTime(), -1, false); - engine.index(idxRequest); + Engine.IndexResult result = engine.index(idxRequest); engine.refresh("test"); - assertThat(idxRequest.version(), equalTo(2L)); + assertThat(result.getVersion(), equalTo(2L)); try (Engine.Searcher searcher = engine.acquireSearcher("test")) { TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), numDocs + 1); assertThat(topDocs.totalHits, equalTo(numDocs + 1)); @@ -1990,8 +1991,8 @@ public class InternalEngineTests extends ESTestCase { 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.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); - engine.index(firstIndexRequest); - assertThat(firstIndexRequest.version(), equalTo(1L)); + Engine.IndexResult index = engine.index(firstIndexRequest); + assertThat(index.getVersion(), equalTo(1L)); } engine.refresh("test"); try (Engine.Searcher searcher = engine.acquireSearcher("test")) { @@ -2138,8 +2139,8 @@ public class InternalEngineTests extends ESTestCase { 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.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); - engine.index(firstIndexRequest); - assertThat(firstIndexRequest.version(), equalTo(1L)); + Engine.IndexResult indexResult = engine.index(firstIndexRequest); + assertThat(indexResult.getVersion(), equalTo(1L)); } DocsStats docStats = engine.getDocStats(); assertEquals(numDocs, docStats.getCount()); @@ -2148,8 +2149,8 @@ public class InternalEngineTests extends ESTestCase { ParsedDocument doc = testParsedDocument(Integer.toString(0), Integer.toString(0), "test", null, -1, -1, testDocument(), new BytesArray("{}"), null); Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(0)), doc, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); - engine.index(firstIndexRequest); - assertThat(firstIndexRequest.version(), equalTo(2L)); + Engine.IndexResult index = engine.index(firstIndexRequest); + assertThat(index.getVersion(), equalTo(2L)); engine.flush(); // flush - buffered deletes are not counted docStats = engine.getDocStats(); assertEquals(1, docStats.getDeleted()); @@ -2222,25 +2223,26 @@ public class InternalEngineTests extends ESTestCase { long autoGeneratedIdTimestamp = 0; Engine.Index index = new Engine.Index(newUid("1"), doc, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry); - engine.index(index); - assertThat(index.version(), equalTo(1L)); + Engine.IndexResult indexResult = engine.index(index); + assertThat(indexResult.getVersion(), equalTo(1L)); - index = new Engine.Index(newUid("1"), doc, index.version(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry); - replicaEngine.index(index); - assertThat(index.version(), equalTo(1L)); + index = new Engine.Index(newUid("1"), doc, indexResult.getVersion(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry); + indexResult = replicaEngine.index(index); + assertThat(indexResult.getVersion(), equalTo(1L)); isRetry = true; index = new Engine.Index(newUid("1"), doc, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry); - engine.index(index); - assertThat(index.version(), equalTo(1L)); + indexResult = engine.index(index); + assertThat(indexResult.getVersion(), equalTo(1L)); engine.refresh("test"); try (Engine.Searcher searcher = engine.acquireSearcher("test")) { TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), 10); assertEquals(1, topDocs.totalHits); } - index = new Engine.Index(newUid("1"), doc, index.version(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry); - replicaEngine.index(index); + index = new Engine.Index(newUid("1"), doc, indexResult.getVersion(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry); + indexResult = replicaEngine.index(index); + assertThat(indexResult.hasFailure(), equalTo(false)); replicaEngine.refresh("test"); try (Engine.Searcher searcher = replicaEngine.acquireSearcher("test")) { TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), 10); @@ -2256,12 +2258,12 @@ public class InternalEngineTests extends ESTestCase { Engine.Index firstIndexRequest = new Engine.Index(newUid("1"), doc, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry); - engine.index(firstIndexRequest); - assertThat(firstIndexRequest.version(), equalTo(1L)); + Engine.IndexResult result = engine.index(firstIndexRequest); + assertThat(result.getVersion(), equalTo(1L)); - Engine.Index firstIndexRequestReplica = new Engine.Index(newUid("1"), doc, firstIndexRequest.version(), firstIndexRequest.versionType().versionTypeForReplicationAndRecovery(), REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry); - replicaEngine.index(firstIndexRequestReplica); - assertThat(firstIndexRequestReplica.version(), equalTo(1L)); + Engine.Index firstIndexRequestReplica = new Engine.Index(newUid("1"), doc, result.getVersion(), firstIndexRequest.versionType().versionTypeForReplicationAndRecovery(), REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry); + Engine.IndexResult indexReplicaResult = replicaEngine.index(firstIndexRequestReplica); + assertThat(indexReplicaResult.getVersion(), equalTo(1L)); isRetry = false; Engine.Index secondIndexRequest = new Engine.Index(newUid("1"), doc, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry); @@ -2273,7 +2275,7 @@ public class InternalEngineTests extends ESTestCase { assertEquals(1, topDocs.totalHits); } - Engine.Index secondIndexRequestReplica = new Engine.Index(newUid("1"), doc, firstIndexRequest.version(), firstIndexRequest.versionType().versionTypeForReplicationAndRecovery(), REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry); + Engine.Index secondIndexRequestReplica = new Engine.Index(newUid("1"), doc, result.getVersion(), firstIndexRequest.versionType().versionTypeForReplicationAndRecovery(), REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry); replicaEngine.index(secondIndexRequestReplica); replicaEngine.refresh("test"); try (Engine.Searcher searcher = replicaEngine.acquireSearcher("test")) { diff --git a/core/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java index 6dea774f258..9e5b8107965 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java @@ -988,8 +988,8 @@ public class ShadowEngineTests extends ESTestCase { 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.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); - primaryEngine.index(firstIndexRequest); - assertThat(firstIndexRequest.version(), equalTo(1L)); + Engine.IndexResult indexResult = primaryEngine.index(firstIndexRequest); + assertThat(indexResult.getVersion(), equalTo(1L)); } DocsStats docStats = primaryEngine.getDocStats(); assertEquals(numDocs, docStats.getCount()); From c237263ad1f3c40730a06aa37be4dd4092640aa1 Mon Sep 17 00:00:00 2001 From: Areek Zillur Date: Tue, 25 Oct 2016 10:42:09 -0400 Subject: [PATCH 017/132] fix computing took for write operation result --- .../java/org/elasticsearch/index/engine/InternalEngine.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 4979042e04e..1b14e435b7e 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -549,7 +549,7 @@ public class InternalEngine extends Engine { } versionMap.putUnderLock(index.uid().bytes(), new VersionValue(updatedVersion)); indexResult.setLocation(location); - indexResult.setTook(index.startTime() - System.nanoTime()); + indexResult.setTook(System.nanoTime() - index.startTime()); indexResult.freeze(); return indexResult; } @@ -632,7 +632,7 @@ public class InternalEngine extends Engine { versionMap.putUnderLock(delete.uid().bytes(), new DeleteVersionValue(updatedVersion, engineConfig.getThreadPool().estimatedTimeInMillis())); deleteResult.setLocation(location); - deleteResult.setTook(delete.startTime() - System.nanoTime()); + deleteResult.setTook(System.nanoTime() - delete.startTime()); deleteResult.freeze(); return deleteResult; } From 65832b987f3070485213fa9533bdc9e9ce7c714b Mon Sep 17 00:00:00 2001 From: Areek Zillur Date: Wed, 26 Oct 2016 11:23:09 -0400 Subject: [PATCH 018/132] Revert "cleanup indexing operation listener" This reverts commit bb785483ae41e30a756a17f3fb6cdb5d06a1ad6c. --- .../elasticsearch/index/shard/IndexShard.java | 12 ++--- .../shard/IndexingOperationListener.java | 47 ++++++++++++++----- .../index/shard/InternalIndexingStats.java | 7 ++- .../elasticsearch/index/IndexModuleTests.java | 3 +- .../index/shard/IndexShardTests.java | 12 +++-- .../shard/IndexingOperationListenerTests.java | 10 ++-- .../index/reindex/CancelTests.java | 14 +++--- 7 files changed, 68 insertions(+), 37 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 9413fb5bcc6..442adc98196 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -536,16 +536,16 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl return new Engine.Index(uid, doc, version, versionType, origin, startTime, autoGeneratedIdTimestamp, isRetry); } - public Engine.IndexResult index(final Engine.Index index) { + public Engine.IndexResult index(Engine.Index index) { ensureWriteAllowed(index); Engine engine = getEngine(); return index(engine, index); } - private Engine.IndexResult index(final Engine engine, final Engine.Index index) { + private Engine.IndexResult index(Engine engine, Engine.Index index) { active.set(true); final Engine.IndexResult result; - indexingOperationListeners.preIndex(index); + index = indexingOperationListeners.preIndex(index); try { if (logger.isTraceEnabled()) { logger.trace("index [{}][{}]{}", index.type(), index.id(), index.docs()); @@ -585,16 +585,16 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl return new Engine.Delete(type, id, uid, version, versionType, origin, startTime); } - public Engine.DeleteResult delete(final Engine.Delete delete) { + public Engine.DeleteResult delete(Engine.Delete delete) { ensureWriteAllowed(delete); Engine engine = getEngine(); return delete(engine, delete); } - private Engine.DeleteResult delete(final Engine engine, final Engine.Delete delete) { + private Engine.DeleteResult delete(Engine engine, Engine.Delete delete) { active.set(true); final Engine.DeleteResult result; - indexingOperationListeners.preDelete(delete); + delete = indexingOperationListeners.preDelete(delete); try { if (logger.isTraceEnabled()) { logger.trace("delete [{}]", delete.uid().text()); diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java b/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java index 9c870859650..0e605954248 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java @@ -30,25 +30,44 @@ import java.util.List; */ public interface IndexingOperationListener { - /** Called before the indexing occurs */ - default void preIndex(Engine.Index operation) {} + /** + * Called before the indexing occurs. + */ + default Engine.Index preIndex(Engine.Index operation) { + return operation; + } - /** Called after the indexing operation occurred */ + /** + * Called after the indexing operation occurred. + */ default void postIndex(Engine.Index index, Engine.IndexResult result) {} - /** Called after the indexing operation occurred with exception */ + /** + * Called after the indexing operation occurred with exception. + */ default void postIndex(Engine.Index index, Exception ex) {} - /** Called before the delete occurs */ - default void preDelete(Engine.Delete delete) {} + /** + * Called before the delete occurs. + */ + default Engine.Delete preDelete(Engine.Delete delete) { + return delete; + } - /** Called after the delete operation occurred */ + + /** + * Called after the delete operation occurred. + */ default void postDelete(Engine.Delete delete, Engine.DeleteResult result) {} - /** Called after the delete operation occurred with exception */ + /** + * Called after the delete operation occurred with exception. + */ default void postDelete(Engine.Delete delete, Exception ex) {} - /** A Composite listener that multiplexes calls to each of the listeners methods */ + /** + * A Composite listener that multiplexes calls to each of the listeners methods. + */ final class CompositeListener implements IndexingOperationListener{ private final List listeners; private final Logger logger; @@ -59,7 +78,7 @@ public interface IndexingOperationListener { } @Override - public void preIndex(Engine.Index operation) { + public Engine.Index preIndex(Engine.Index operation) { assert operation != null; for (IndexingOperationListener listener : listeners) { try { @@ -68,11 +87,12 @@ public interface IndexingOperationListener { logger.warn((Supplier) () -> new ParameterizedMessage("preIndex listener [{}] failed", listener), e); } } + return operation; } @Override public void postIndex(Engine.Index index, Engine.IndexResult result) { - assert index != null && result != null; + assert index != null; for (IndexingOperationListener listener : listeners) { try { listener.postIndex(index, result); @@ -96,7 +116,7 @@ public interface IndexingOperationListener { } @Override - public void preDelete(Engine.Delete delete) { + public Engine.Delete preDelete(Engine.Delete delete) { assert delete != null; for (IndexingOperationListener listener : listeners) { try { @@ -105,11 +125,12 @@ public interface IndexingOperationListener { logger.warn((Supplier) () -> new ParameterizedMessage("preDelete listener [{}] failed", listener), e); } } + return delete; } @Override public void postDelete(Engine.Delete delete, Engine.DeleteResult result) { - assert delete != null && result != null; + assert delete != null; for (IndexingOperationListener listener : listeners) { try { listener.postDelete(delete, result); diff --git a/core/src/main/java/org/elasticsearch/index/shard/InternalIndexingStats.java b/core/src/main/java/org/elasticsearch/index/shard/InternalIndexingStats.java index 61cfe2fc4b5..39a415ca8eb 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/InternalIndexingStats.java +++ b/core/src/main/java/org/elasticsearch/index/shard/InternalIndexingStats.java @@ -65,11 +65,12 @@ final class InternalIndexingStats implements IndexingOperationListener { } @Override - public void preIndex(Engine.Index operation) { + public Engine.Index preIndex(Engine.Index operation) { if (!operation.origin().isRecovery()) { totalStats.indexCurrent.inc(); typeStats(operation.type()).indexCurrent.inc(); } + return operation; } @Override @@ -95,11 +96,13 @@ final class InternalIndexingStats implements IndexingOperationListener { } @Override - public void preDelete(Engine.Delete delete) { + public Engine.Delete preDelete(Engine.Delete delete) { if (!delete.origin().isRecovery()) { totalStats.deleteCurrent.inc(); typeStats(delete.type()).deleteCurrent.inc(); } + return delete; + } @Override diff --git a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java index 8dcad1b8158..5e9d1ffaf9e 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java +++ b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java @@ -233,8 +233,9 @@ public class IndexModuleTests extends ESTestCase { AtomicBoolean executed = new AtomicBoolean(false); IndexingOperationListener listener = new IndexingOperationListener() { @Override - public void preIndex(Engine.Index operation) { + public Engine.Index preIndex(Engine.Index operation) { executed.set(true); + return operation; } }; module.addIndexOperationListener(listener); diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 901dcc3b724..2b417c10ae7 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -558,8 +558,9 @@ public class IndexShardTests extends IndexShardTestCase { shard.close("simon says", true); shard = reinitShard(shard, new IndexingOperationListener() { @Override - public void preIndex(Engine.Index operation) { + public Engine.Index preIndex(Engine.Index operation) { preIndex.incrementAndGet(); + return operation; } @Override @@ -577,8 +578,9 @@ public class IndexShardTests extends IndexShardTestCase { } @Override - public void preDelete(Engine.Delete delete) { + public Engine.Delete preDelete(Engine.Delete delete) { preDelete.incrementAndGet(); + return delete; } @Override @@ -1119,8 +1121,9 @@ public class IndexShardTests extends IndexShardTestCase { final AtomicInteger postDelete = new AtomicInteger(); IndexingOperationListener listener = new IndexingOperationListener() { @Override - public void preIndex(Engine.Index operation) { + public Engine.Index preIndex(Engine.Index operation) { preIndex.incrementAndGet(); + return operation; } @Override @@ -1129,8 +1132,9 @@ public class IndexShardTests extends IndexShardTestCase { } @Override - public void preDelete(Engine.Delete delete) { + public Engine.Delete preDelete(Engine.Delete delete) { preDelete.incrementAndGet(); + return delete; } @Override diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java index 742fadc9a80..563c6a7eabf 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java @@ -40,8 +40,9 @@ public class IndexingOperationListenerTests extends ESTestCase{ AtomicInteger postDeleteException = new AtomicInteger(); IndexingOperationListener listener = new IndexingOperationListener() { @Override - public void preIndex(Engine.Index operation) { + public Engine.Index preIndex(Engine.Index operation) { preIndex.incrementAndGet(); + return operation; } @Override @@ -55,8 +56,9 @@ public class IndexingOperationListenerTests extends ESTestCase{ } @Override - public void preDelete(Engine.Delete delete) { + public Engine.Delete preDelete(Engine.Delete delete) { preDelete.incrementAndGet(); + return delete; } @Override @@ -72,7 +74,7 @@ public class IndexingOperationListenerTests extends ESTestCase{ IndexingOperationListener throwingListener = new IndexingOperationListener() { @Override - public void preIndex(Engine.Index operation) { + public Engine.Index preIndex(Engine.Index operation) { throw new RuntimeException(); } @@ -87,7 +89,7 @@ public class IndexingOperationListenerTests extends ESTestCase{ } @Override - public void preDelete(Engine.Delete delete) { + public Engine.Delete preDelete(Engine.Delete delete) { throw new RuntimeException(); } diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/CancelTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/CancelTests.java index 21b3a1aa7a6..8da47f1eeaf 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/CancelTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/CancelTests.java @@ -199,23 +199,23 @@ public class CancelTests extends ReindexTestCase { public static class BlockingOperationListener implements IndexingOperationListener { @Override - public void preIndex(Engine.Index index) { - preCheck(index, index.type()); + public Engine.Index preIndex(Engine.Index index) { + return preCheck(index, index.type()); } @Override - public void preDelete(Engine.Delete delete) { - preCheck(delete, delete.type()); + public Engine.Delete preDelete(Engine.Delete delete) { + return preCheck(delete, delete.type()); } - private void preCheck(Engine.Operation operation, String type) { + private T preCheck(T operation, String type) { if ((TYPE.equals(type) == false) || (operation.origin() != Origin.PRIMARY)) { - return; + return operation; } try { if (ALLOWED_OPERATIONS.tryAcquire(30, TimeUnit.SECONDS)) { - return; + return operation; } } catch (InterruptedException e) { throw new RuntimeException(e); From a3fcfe819622f6610188de0af7fa75ef406767d5 Mon Sep 17 00:00:00 2001 From: Areek Zillur Date: Wed, 26 Oct 2016 12:07:32 -0400 Subject: [PATCH 019/132] add constructor overloads for primary result --- .../admin/indices/flush/TransportShardFlushAction.java | 4 ++-- .../indices/refresh/TransportShardRefreshAction.java | 4 ++-- .../replication/TransportReplicationAction.java | 9 +++++++++ .../replication/TransportReplicationActionTests.java | 10 +++++----- 4 files changed, 18 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/admin/indices/flush/TransportShardFlushAction.java b/core/src/main/java/org/elasticsearch/action/admin/indices/flush/TransportShardFlushAction.java index 84e5a4df408..1ec7186393f 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/indices/flush/TransportShardFlushAction.java +++ b/core/src/main/java/org/elasticsearch/action/admin/indices/flush/TransportShardFlushAction.java @@ -54,14 +54,14 @@ public class TransportShardFlushAction extends TransportReplicationAction listener) { if (finalFailure == null) { listener.onResponse(TransportResponse.Empty.INSTANCE); diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java b/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java index 1642f5154f2..3deae74f455 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java @@ -718,7 +718,7 @@ public class TransportReplicationActionTests extends ESTestCase { if (throwException) { throw new ElasticsearchException("simulated"); } - return new ReplicaResult(null); + return new ReplicaResult(); } }; final Action.ReplicaOperationTransportHandler replicaOperationTransportHandler = action.new ReplicaOperationTransportHandler(); @@ -837,7 +837,7 @@ public class TransportReplicationActionTests extends ESTestCase { if (throwException.get()) { throw new RetryOnReplicaException(shardId, "simulation"); } - return new ReplicaResult(null); + return new ReplicaResult(); } }; final Action.ReplicaOperationTransportHandler replicaOperationTransportHandler = action.new ReplicaOperationTransportHandler(); @@ -961,13 +961,13 @@ public class TransportReplicationActionTests extends ESTestCase { protected PrimaryResult shardOperationOnPrimary(Request shardRequest, IndexShard primary) throws Exception { boolean executedBefore = shardRequest.processedOnPrimary.getAndSet(true); assert executedBefore == false : "request has already been executed on the primary"; - return new PrimaryResult(shardRequest, new Response(), null); + return new PrimaryResult(shardRequest, new Response()); } @Override protected ReplicaResult shardOperationOnReplica(Request request, IndexShard replica) { request.processedOnReplicas.incrementAndGet(); - return new ReplicaResult(null); + return new ReplicaResult(); } @Override @@ -1053,7 +1053,7 @@ public class TransportReplicationActionTests extends ESTestCase { @Override public void execute() throws Exception { - this.resultListener.onResponse(action.new PrimaryResult(null, new Response(), null)); + this.resultListener.onResponse(action.new PrimaryResult(null, new Response())); } } From fa3ee6b996c16063e2de0dbcf080829f35cfa693 Mon Sep 17 00:00:00 2001 From: Areek Zillur Date: Wed, 26 Oct 2016 12:30:46 -0400 Subject: [PATCH 020/132] Incorporate feedback --- .../elasticsearch/ElasticsearchException.java | 4 +- .../action/bulk/TransportShardBulkAction.java | 163 +++++++++--------- .../action/delete/TransportDeleteAction.java | 4 +- .../action/index/TransportIndexAction.java | 14 +- .../TransportReplicationAction.java | 12 +- .../replication/TransportWriteAction.java | 5 +- .../elasticsearch/index/engine/Engine.java | 57 +++--- .../index/engine/InternalEngine.java | 67 ++++--- .../OperationFailedEngineException.java | 63 ------- .../elasticsearch/index/shard/IndexShard.java | 3 +- .../indices/IndexingMemoryController.java | 14 +- .../ExceptionSerializationTests.java | 18 +- .../ESIndexLevelReplicationTestCase.java | 5 +- .../index/shard/IndexShardTests.java | 2 +- .../shard/IndexingOperationListenerTests.java | 4 +- 15 files changed, 181 insertions(+), 254 deletions(-) delete mode 100644 core/src/main/java/org/elasticsearch/index/engine/OperationFailedEngineException.java diff --git a/core/src/main/java/org/elasticsearch/ElasticsearchException.java b/core/src/main/java/org/elasticsearch/ElasticsearchException.java index 0c6125c52af..e414b241a23 100644 --- a/core/src/main/java/org/elasticsearch/ElasticsearchException.java +++ b/core/src/main/java/org/elasticsearch/ElasticsearchException.java @@ -28,7 +28,6 @@ import org.elasticsearch.common.logging.LoggerMessageFormat; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.index.Index; -import org.elasticsearch.index.engine.OperationFailedEngineException; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.transport.TcpTransport; @@ -580,8 +579,7 @@ public class ElasticsearchException extends RuntimeException implements ToXConte org.elasticsearch.action.TimestampParsingException::new, 78), ROUTING_MISSING_EXCEPTION(org.elasticsearch.action.RoutingMissingException.class, org.elasticsearch.action.RoutingMissingException::new, 79), - OPERATION_FAILED_ENGINE_EXCEPTION(OperationFailedEngineException.class, - OperationFailedEngineException::new, 80), + // 80 used to be for IndexFailedEngineException, removed in 6.0 INDEX_SHARD_RESTORE_FAILED_EXCEPTION(org.elasticsearch.index.snapshots.IndexShardRestoreFailedException.class, org.elasticsearch.index.snapshots.IndexShardRestoreFailedException::new, 81), REPOSITORY_EXCEPTION(org.elasticsearch.repositories.RepositoryException.class, diff --git a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 50db6adefc9..6324c684a6f 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -48,6 +48,7 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.VersionConflictEngineException; +import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.IndicesService; @@ -133,44 +134,49 @@ public class TransportShardBulkAction extends TransportWriteAction) () -> new ParameterizedMessage("{} failed to execute bulk item ({}) {}", @@ -182,20 +188,14 @@ public class TransportShardBulkAction extends TransportWriteAction 0)) { - Tuple> sourceAndContent = - XContentHelper.convertToMap(indexSourceAsBytes, true); - updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, request.index(), - indexResponse.getVersion(), sourceAndContent.v2(), sourceAndContent.v1(), indexSourceAsBytes)); - } - // replace the update request to the translated index request to execute on the replica. - request.items()[requestIndex] = new BulkItemRequest(request.items()[requestIndex].id(), updateIndexRequest); - break; - case DELETE: - DeleteRequest updateDeleteRequest = translate.action(); - DeleteResponse deleteResponse = new DeleteResponse(primary.shardId(), - updateDeleteRequest.type(), updateDeleteRequest.id(), - updateOperationResult.getVersion(), ((Engine.DeleteResult) updateOperationResult).isFound()); - updateResponse = new UpdateResponse(deleteResponse.getShardInfo(), - deleteResponse.getShardId(), deleteResponse.getType(), deleteResponse.getId(), - deleteResponse.getVersion(), deleteResponse.getResult()); - updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, - request.index(), deleteResponse.getVersion(), translate.updatedSourceAsMap(), - translate.updateSourceContentType(), null)); - // replace the update request to the translated delete request to execute on the replica. - request.items()[requestIndex] = new BulkItemRequest(request.items()[requestIndex].id(), updateDeleteRequest); - break; - } - } else { - // version conflict exception, retry - if (updateOperationResult.getFailure() instanceof VersionConflictEngineException) { - continue; - } + break; // out of retry loop + } else if (updateOperationResult.hasFailure() == false) { + // enrich update response and + // set translated update (index/delete) request for replica execution in bulk items + switch (updateOperationResult.getOperationType()) { + case INDEX: + IndexRequest updateIndexRequest = translate.action(); + final IndexResponse indexResponse = new IndexResponse(primary.shardId(), + updateIndexRequest.type(), updateIndexRequest.id(), + updateOperationResult.getVersion(), ((Engine.IndexResult) updateOperationResult).isCreated()); + BytesReference indexSourceAsBytes = updateIndexRequest.source(); + updateResponse = new UpdateResponse(indexResponse.getShardInfo(), + indexResponse.getShardId(), indexResponse.getType(), indexResponse.getId(), + indexResponse.getVersion(), indexResponse.getResult()); + if ((updateRequest.fetchSource() != null && updateRequest.fetchSource().fetchSource()) || + (updateRequest.fields() != null && updateRequest.fields().length > 0)) { + Tuple> sourceAndContent = + XContentHelper.convertToMap(indexSourceAsBytes, true); + updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, request.index(), + indexResponse.getVersion(), sourceAndContent.v2(), sourceAndContent.v1(), indexSourceAsBytes)); + } + // set translated request as replica request + replicaRequest = new BulkItemRequest(request.items()[requestIndex].id(), updateIndexRequest); + break; + case DELETE: + DeleteRequest updateDeleteRequest = translate.action(); + DeleteResponse deleteResponse = new DeleteResponse(primary.shardId(), + updateDeleteRequest.type(), updateDeleteRequest.id(), + updateOperationResult.getVersion(), ((Engine.DeleteResult) updateOperationResult).isFound()); + updateResponse = new UpdateResponse(deleteResponse.getShardInfo(), + deleteResponse.getShardId(), deleteResponse.getType(), deleteResponse.getId(), + deleteResponse.getVersion(), deleteResponse.getResult()); + updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, + request.index(), deleteResponse.getVersion(), translate.updatedSourceAsMap(), + translate.updateSourceContentType(), null)); + // set translated request as replica request + replicaRequest = new BulkItemRequest(request.items()[requestIndex].id(), updateDeleteRequest); + break; } + // successful operation + break; // out of retry loop + } else if (updateOperationResult.getFailure() instanceof VersionConflictEngineException == false) { + // not a version conflict exception + break; // out of retry loop } - break; // out of retry loop } - return new UpdateResultHolder(updateOperationResult, updateResponse); + return new UpdateResultHolder(replicaRequest, updateOperationResult, updateResponse); } @Override @@ -342,11 +342,14 @@ public class TransportShardBulkAction extends TransportWriteAction { @@ -378,7 +373,12 @@ public abstract class TransportReplicationAction< final Response finalResponseIfSuccessful; final Exception finalFailure; + /** + * Result of executing a primary operation + * expects finalResponseIfSuccessful or finalFailure to be not-null + */ public PrimaryResult(ReplicaRequest replicaRequest, Response finalResponseIfSuccessful, Exception finalFailure) { + assert finalFailure != null ^ finalResponseIfSuccessful != null : "either a response or a failure has to be not null"; this.replicaRequest = replicaRequest; this.finalResponseIfSuccessful = finalResponseIfSuccessful; this.finalFailure = finalFailure; diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java b/core/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java index eb6073d7ad4..acc512f1a07 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java @@ -89,7 +89,6 @@ public abstract class TransportWriteAction< @Nullable Location location, @Nullable Exception operationFailure, IndexShard primary) { super(request, finalResponse, operationFailure); - assert operationFailure != null ^ finalResponse != null; if (operationFailure != null) { this.finishedAsyncActions = true; } else { @@ -127,9 +126,7 @@ public abstract class TransportWriteAction< @Override public synchronized void onSuccess(boolean forcedRefresh) { - if (finalResponseIfSuccessful != null) { - finalResponseIfSuccessful.setForcedRefresh(forcedRefresh); - } + finalResponseIfSuccessful.setForcedRefresh(forcedRefresh); finishedAsyncActions = true; respondIfPossible(null); } diff --git a/core/src/main/java/org/elasticsearch/index/engine/Engine.java b/core/src/main/java/org/elasticsearch/index/engine/Engine.java index 1f935e8ac50..0cd1a3022b1 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -282,43 +282,50 @@ public abstract class Engine implements Closeable { public abstract DeleteResult delete(Delete delete); + /** + * Base class for index and delete operation results + * Holds result meta data (e.g. translog location, updated version) + * for an executed write {@link Operation} + **/ public abstract static class Result { private final Operation.TYPE operationType; private final long version; private final Exception failure; - private final int estimatedSizeInBytes; - private Translog.Location location; + private Translog.Location translogLocation; private long took; private boolean freeze; - protected Result(Operation.TYPE operationType, Exception failure, - long version, int estimatedSizeInBytes) { + protected Result(Operation.TYPE operationType, Exception failure, long version) { this.operationType = operationType; this.failure = failure; this.version = version; - this.estimatedSizeInBytes = estimatedSizeInBytes; } - protected Result(Operation.TYPE operationType, long version, int estimatedSizeInBytes) { - this(operationType, null, version, estimatedSizeInBytes); + protected Result(Operation.TYPE operationType, long version) { + this(operationType, null, version); } + /** whether the operation had failure */ public boolean hasFailure() { return failure != null; } + /** get the updated document version */ public long getVersion() { return version; } - public Translog.Location getLocation() { - return location; + /** get the translog location after executing the operation */ + public Translog.Location getTranslogLocation() { + return translogLocation; } + /** get document failure while executing the operation {@code null} in case of no failure */ public Exception getFailure() { return failure; } + /** get total time in nanoseconds */ public long getTook() { return took; } @@ -327,22 +334,24 @@ public abstract class Engine implements Closeable { return operationType; } + /** get size of the translog operation if translog location has been set */ public int getSizeInBytes() { - if (location != null) { - return location.size; + if (translogLocation != null) { + return translogLocation.size; + } else { + throw new IllegalStateException("result has null location, use Operation#estimatedSizeInBytes instead"); } - return estimatedSizeInBytes; } - public void setLocation(Translog.Location location) { + void setTranslogLocation(Translog.Location translogLocation) { if (freeze == false) { - this.location = location; + this.translogLocation = translogLocation; } else { throw new IllegalStateException("result is already frozen"); } } - public void setTook(long took) { + void setTook(long took) { if (freeze == false) { this.took = took; } else { @@ -350,7 +359,7 @@ public abstract class Engine implements Closeable { } } - public void freeze() { + void freeze() { this.freeze = true; } } @@ -358,13 +367,13 @@ public abstract class Engine implements Closeable { public static class IndexResult extends Result { private final boolean created; - public IndexResult(long version, boolean created, int estimatedSizeInBytes) { - super(Operation.TYPE.INDEX, version, estimatedSizeInBytes); + public IndexResult(long version, boolean created) { + super(Operation.TYPE.INDEX, version); this.created = created; } - public IndexResult(Exception failure, long version, int estimatedSizeInBytes) { - super(Operation.TYPE.INDEX, failure, version, estimatedSizeInBytes); + public IndexResult(Exception failure, long version) { + super(Operation.TYPE.INDEX, failure, version); this.created = false; } @@ -376,13 +385,13 @@ public abstract class Engine implements Closeable { public static class DeleteResult extends Result { private final boolean found; - public DeleteResult(long version, boolean found, int estimatedSizeInBytes) { - super(Operation.TYPE.DELETE, version, estimatedSizeInBytes); + public DeleteResult(long version, boolean found) { + super(Operation.TYPE.DELETE, version); this.found = found; } - public DeleteResult(Exception failure, long version, int estimatedSizeInBytes) { - super(Operation.TYPE.DELETE, failure, version, estimatedSizeInBytes); + public DeleteResult(Exception failure, long version) { + super(Operation.TYPE.DELETE, failure, version); this.found = false; } diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 1b14e435b7e..8330da04682 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -41,6 +41,7 @@ import org.apache.lucene.store.LockObtainFailedException; 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.ExceptionsHelper; import org.elasticsearch.Version; import org.elasticsearch.action.index.IndexRequest; @@ -401,9 +402,7 @@ public class InternalEngine extends Engine { } } } catch (Exception e) { - Exception documentFailure = extractDocumentFailure(index, e); - result = new IndexResult(documentFailure, index.version(), - index.estimatedSizeInBytes()); + result = new IndexResult(checkIfDocumentFailureOrThrow(index, e), index.version()); } return result; } @@ -412,10 +411,12 @@ public class InternalEngine extends Engine { * Inspects exception thrown when executing index or delete operations * * @return failure if the failure is a document specific failure (e.g. analysis chain failure) - * @throws OperationFailedEngineException if the failure caused the engine to fail + * @throws ElasticsearchException if the failure caused the engine to fail * (e.g. out of disk, lucene tragic event) + * + * Note: pkg-private for testing */ - private Exception extractDocumentFailure(final Operation operation, final Exception failure) { + final Exception checkIfDocumentFailureOrThrow(final Operation operation, final Exception failure) { boolean isDocumentFailure; try { // When indexing a document into Lucene, Lucene distinguishes between environment related errors @@ -434,8 +435,9 @@ public class InternalEngine extends Engine { if (isDocumentFailure) { return failure; } else { - throw new OperationFailedEngineException(shardId, operation.operationType().getLowercase(), - operation.type(), operation.id(), failure); + ElasticsearchException exception = new ElasticsearchException(failure); + exception.setShard(shardId); + throw exception; } } @@ -529,9 +531,10 @@ public class InternalEngine extends Engine { } } final long expectedVersion = index.version(); + final IndexResult indexResult; if (checkVersionConflict(index, currentVersion, expectedVersion, deleted)) { // skip index operation because of version conflict on recovery - return new IndexResult(expectedVersion, false, index.estimatedSizeInBytes()); + indexResult = new IndexResult(expectedVersion, false); } else { updatedVersion = index.versionType().updateVersion(currentVersion, expectedVersion); index.parsedDoc().version().setLongValue(updatedVersion); @@ -541,18 +544,16 @@ public class InternalEngine extends Engine { } else { update(index.uid(), index.docs(), indexWriter); } - IndexResult indexResult = new IndexResult(updatedVersion, deleted, index.estimatedSizeInBytes()); - if (index.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY) { - location = translog.add(new Translog.Index(index, indexResult)); - } else { - location = null; - } + indexResult = new IndexResult(updatedVersion, deleted); + location = index.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY + ? translog.add(new Translog.Index(index, indexResult)) + : null; versionMap.putUnderLock(index.uid().bytes(), new VersionValue(updatedVersion)); - indexResult.setLocation(location); - indexResult.setTook(System.nanoTime() - index.startTime()); - indexResult.freeze(); - return indexResult; + indexResult.setTranslogLocation(location); } + indexResult.setTook(System.nanoTime() - index.startTime()); + indexResult.freeze(); + return indexResult; } } @@ -580,9 +581,7 @@ public class InternalEngine extends Engine { // NOTE: we don't throttle this when merges fall behind because delete-by-id does not create new segments: result = innerDelete(delete); } catch (Exception e) { - Exception documentFailure = extractDocumentFailure(delete, e); - result = new DeleteResult(documentFailure, delete.version(), - delete.estimatedSizeInBytes()); + result = new DeleteResult(checkIfDocumentFailureOrThrow(delete, e), delete.version()); } maybePruneDeletedTombstones(); return result; @@ -615,27 +614,24 @@ public class InternalEngine extends Engine { } final long expectedVersion = delete.version(); + final DeleteResult deleteResult; if (checkVersionConflict(delete, currentVersion, expectedVersion, deleted)) { // skip executing delete because of version conflict on recovery - return new DeleteResult(expectedVersion, true, - delete.estimatedSizeInBytes()); + deleteResult = new DeleteResult(expectedVersion, true); } else { updatedVersion = delete.versionType().updateVersion(currentVersion, expectedVersion); found = deleteIfFound(delete.uid(), currentVersion, deleted, versionValue); - DeleteResult deleteResult = new DeleteResult(updatedVersion, found, - delete.estimatedSizeInBytes()); - if (delete.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY) { - location = translog.add(new Translog.Delete(delete, deleteResult)); - } else { - location = null; - } + deleteResult = new DeleteResult(updatedVersion, found); + location = delete.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY + ? translog.add(new Translog.Delete(delete, deleteResult)) + : null; versionMap.putUnderLock(delete.uid().bytes(), new DeleteVersionValue(updatedVersion, engineConfig.getThreadPool().estimatedTimeInMillis())); - deleteResult.setLocation(location); - deleteResult.setTook(System.nanoTime() - delete.startTime()); - deleteResult.freeze(); - return deleteResult; + deleteResult.setTranslogLocation(location); } + deleteResult.setTook(System.nanoTime() - delete.startTime()); + deleteResult.freeze(); + return deleteResult; } } @@ -1117,7 +1113,8 @@ public class InternalEngine extends Engine { } } - private IndexWriter createWriter(boolean create) throws IOException { + // pkg-private for testing + IndexWriter createWriter(boolean create) throws IOException { try { final IndexWriterConfig iwc = new IndexWriterConfig(engineConfig.getAnalyzer()); iwc.setCommitOnClose(false); // we by default don't commit on close diff --git a/core/src/main/java/org/elasticsearch/index/engine/OperationFailedEngineException.java b/core/src/main/java/org/elasticsearch/index/engine/OperationFailedEngineException.java deleted file mode 100644 index 1c080f2fc18..00000000000 --- a/core/src/main/java/org/elasticsearch/index/engine/OperationFailedEngineException.java +++ /dev/null @@ -1,63 +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.engine; - -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.index.shard.ShardId; - -import java.io.IOException; -import java.util.Objects; - -public class OperationFailedEngineException extends EngineException { - - private final String type; - - private final String id; - - public OperationFailedEngineException(ShardId shardId, String operationType, String type, String id, Throwable cause) { - super(shardId, operationType + " failed for [" + type + "#" + id + "]", cause); - Objects.requireNonNull(type, "type must not be null"); - Objects.requireNonNull(id, "id must not be null"); - this.type = type; - this.id = id; - } - - public OperationFailedEngineException(StreamInput in) throws IOException{ - super(in); - type = in.readString(); - id = in.readString(); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeString(type); - out.writeString(id); - } - - public String type() { - return this.type; - } - - public String id() { - return this.id; - } -} \ No newline at end of file diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 442adc98196..8a5a7cc6b44 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1126,8 +1126,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl private void verifyPrimary() { if (shardRouting.primary() == false) { - // TODO throw a more appropriate exception - throw new ShardNotFoundException(shardRouting.shardId(), "shard is not a primary anymore"); + throw new IllegalStateException("shard is not a primary " + shardRouting); } } diff --git a/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java b/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java index 25b133f390d..5b6bdb05cf1 100644 --- a/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java +++ b/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java @@ -201,17 +201,23 @@ public class IndexingMemoryController extends AbstractComponent implements Index @Override public void postIndex(Engine.Index index, Engine.IndexResult result) { - recordOperationBytes(result); + recordOperationBytes(index, result); } @Override public void postDelete(Engine.Delete delete, Engine.DeleteResult result) { - recordOperationBytes(result); + recordOperationBytes(delete, result); } /** called by IndexShard to record that this many bytes were written to translog */ - private void recordOperationBytes(Engine.Result result) { - statusChecker.bytesWritten(result.getSizeInBytes()); + private void recordOperationBytes(Engine.Operation operation, Engine.Result result) { + final int sizeInBytes; + if (result.getTranslogLocation() != null) { + sizeInBytes = result.getSizeInBytes(); + } else { + sizeInBytes = operation.estimatedSizeInBytes(); + } + statusChecker.bytesWritten(sizeInBytes); } private static final class ShardAndBytesUsed implements Comparable { diff --git a/core/src/test/java/org/elasticsearch/ExceptionSerializationTests.java b/core/src/test/java/org/elasticsearch/ExceptionSerializationTests.java index 61227c3e366..1292b06496f 100644 --- a/core/src/test/java/org/elasticsearch/ExceptionSerializationTests.java +++ b/core/src/test/java/org/elasticsearch/ExceptionSerializationTests.java @@ -51,7 +51,6 @@ import org.elasticsearch.common.xcontent.XContentLocation; import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.index.AlreadyExpiredException; import org.elasticsearch.index.Index; -import org.elasticsearch.index.engine.OperationFailedEngineException; import org.elasticsearch.index.engine.RecoveryEngineException; import org.elasticsearch.index.query.QueryShardException; import org.elasticsearch.index.shard.IllegalIndexShardStateException; @@ -400,21 +399,6 @@ public class ExceptionSerializationTests extends ESTestCase { assertEquals("TIMESTAMP", ex.timestamp()); } - public void testIndexFailedEngineException() throws IOException { - ShardId id = new ShardId("foo", "_na_", 1); - OperationFailedEngineException ex = serialize(new OperationFailedEngineException(id, "index", "type", "id", null)); - assertEquals(ex.getShardId(), new ShardId("foo", "_na_", 1)); - assertEquals("type", ex.type()); - assertEquals("id", ex.id()); - assertNull(ex.getCause()); - - ex = serialize(new OperationFailedEngineException(null, "index", "type", "id", new NullPointerException())); - assertNull(ex.getShardId()); - assertEquals("type", ex.type()); - assertEquals("id", ex.id()); - assertTrue(ex.getCause() instanceof NullPointerException); - } - public void testAliasesMissingException() throws IOException { AliasesNotFoundException ex = serialize(new AliasesNotFoundException("one", "two", "three")); assertEquals("aliases [one, two, three] missing", ex.getMessage()); @@ -732,7 +716,7 @@ public class ExceptionSerializationTests extends ESTestCase { ids.put(77, org.elasticsearch.common.util.concurrent.UncategorizedExecutionException.class); ids.put(78, org.elasticsearch.action.TimestampParsingException.class); ids.put(79, org.elasticsearch.action.RoutingMissingException.class); - ids.put(80, OperationFailedEngineException.class); + ids.put(80, null); // was IndexFailedEngineException, removed in 6.0 ids.put(81, org.elasticsearch.index.snapshots.IndexShardRestoreFailedException.class); ids.put(82, org.elasticsearch.repositories.RepositoryException.class); ids.put(83, org.elasticsearch.transport.ReceiveTimeoutTransportException.class); diff --git a/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java b/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java index 89487c7aa0c..cb9eb5a85a4 100644 --- a/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java +++ b/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java @@ -29,7 +29,6 @@ import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.replication.ReplicationOperation; import org.elasticsearch.action.support.replication.ReplicationRequest; import org.elasticsearch.action.support.replication.ReplicationResponse; -import org.elasticsearch.action.support.replication.TransportWriteAction; import org.elasticsearch.action.support.replication.TransportWriteActionTestHelper; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -369,7 +368,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase final Engine.IndexResult indexResult = executeIndexRequestOnPrimary(request, primary, null); request.primaryTerm(primary.getPrimaryTerm()); - TransportWriteActionTestHelper.performPostWriteActions(primary, request, indexResult.getLocation(), logger); + TransportWriteActionTestHelper.performPostWriteActions(primary, request, indexResult.getTranslogLocation(), logger); IndexResponse response = new IndexResponse(primary.shardId(), request.type(), request.id(), indexResult.getVersion(), indexResult.isCreated()); return new PrimaryResult(request, response); @@ -378,7 +377,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase @Override protected void performOnReplica(IndexRequest request, IndexShard replica) { final Engine.IndexResult result = executeIndexRequestOnReplica(request, replica); - TransportWriteActionTestHelper.performPostWriteActions(replica, request, result.getLocation(), logger); + TransportWriteActionTestHelper.performPostWriteActions(replica, request, result.getTranslogLocation(), logger); } } } diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 2b417c10ae7..dada068446a 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -359,7 +359,7 @@ public class IndexShardTests extends IndexShardTestCase { try { indexShard.acquirePrimaryOperationLock(null, ThreadPool.Names.INDEX); fail("shard shouldn't accept primary ops"); - } catch (ShardNotFoundException ignored) { + } catch (IllegalStateException ignored) { } } diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java index 563c6a7eabf..a29be0cd69a 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java @@ -114,7 +114,7 @@ public class IndexingOperationListenerTests extends ESTestCase{ IndexingOperationListener.CompositeListener compositeListener = new IndexingOperationListener.CompositeListener(indexingOperationListeners, logger); Engine.Delete delete = new Engine.Delete("test", "1", new Term("_uid", "1")); Engine.Index index = new Engine.Index(new Term("_uid", "1"), null); - compositeListener.postDelete(delete, new Engine.DeleteResult(1, true, 0)); + compositeListener.postDelete(delete, new Engine.DeleteResult(1, true)); assertEquals(0, preIndex.get()); assertEquals(0, postIndex.get()); assertEquals(0, postIndexException.get()); @@ -138,7 +138,7 @@ public class IndexingOperationListenerTests extends ESTestCase{ assertEquals(2, postDelete.get()); assertEquals(2, postDeleteException.get()); - compositeListener.postIndex(index, new Engine.IndexResult(0, false, 0)); + compositeListener.postIndex(index, new Engine.IndexResult(0, false)); assertEquals(0, preIndex.get()); assertEquals(2, postIndex.get()); assertEquals(0, postIndexException.get()); From 7fb44a3ab63cfa502b2120fe9b1958b9bfa9630c Mon Sep 17 00:00:00 2001 From: Areek Zillur Date: Wed, 26 Oct 2016 20:21:54 -0400 Subject: [PATCH 021/132] add tests --- .../TransportWriteActionTests.java | 54 ++++++++- .../index/engine/InternalEngineTests.java | 110 +++++++++++++++--- .../index/shard/RefreshListenersTests.java | 10 +- 3 files changed, 149 insertions(+), 25 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java b/core/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java index 66251c92abf..712d1ebe993 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java @@ -70,6 +70,7 @@ public class TransportWriteActionTests extends ESTestCase { CapturingActionListener listener = new CapturingActionListener<>(); responder.accept(result, listener); assertNotNull(listener.response); + assertNull(listener.failure); verify(indexShard, never()).refresh(any()); verify(indexShard, never()).addRefreshListener(any(), any()); } @@ -91,6 +92,7 @@ public class TransportWriteActionTests extends ESTestCase { CapturingActionListener listener = new CapturingActionListener<>(); responder.accept(result, listener); assertNotNull(listener.response); + assertNull(listener.failure); responseChecker.accept(listener.response); verify(indexShard).refresh("refresh_flag_index"); verify(indexShard, never()).addRefreshListener(any(), any()); @@ -124,15 +126,46 @@ public class TransportWriteActionTests extends ESTestCase { boolean forcedRefresh = randomBoolean(); refreshListener.getValue().accept(forcedRefresh); assertNotNull(listener.response); + assertNull(listener.failure); resultChecker.accept(listener.response, forcedRefresh); } + public void testDocumentFailureInShardOperationOnPrimary() throws Exception { + handleDocumentFailure(new TestAction(true, true), TestAction::shardOperationOnPrimary, TestAction.WritePrimaryResult::respond); + } + + public void testDocumentFailureInShardOperationOnReplica() throws Exception { + handleDocumentFailure(new TestAction(randomBoolean(), true), TestAction::shardOperationOnReplica, + TestAction.WriteReplicaResult::respond); + } + + private void handleDocumentFailure(TestAction testAction, + ThrowingTriFunction action, + BiConsumer> responder) + throws Exception { + TestRequest request = new TestRequest(); + Result result = action.apply(testAction, request, indexShard); + CapturingActionListener listener = new CapturingActionListener<>(); + responder.accept(result, listener); + assertNull(listener.response); + assertNotNull(listener.failure); + } + private class TestAction extends TransportWriteAction { + + private final boolean withDocumentFailureOnPrimary; + private final boolean withDocumentFailureOnReplica; + protected TestAction() { + this(false, false); + } + protected TestAction(boolean withDocumentFailureOnPrimary, boolean withDocumentFailureOnReplica) { super(Settings.EMPTY, "test", new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR, null), null, null, null, null, new ActionFilters(new HashSet<>()), new IndexNameExpressionResolver(Settings.EMPTY), TestRequest::new, TestRequest::new, ThreadPool.Names.SAME); + this.withDocumentFailureOnPrimary = withDocumentFailureOnPrimary; + this.withDocumentFailureOnReplica = withDocumentFailureOnReplica; } @Override @@ -142,12 +175,24 @@ public class TransportWriteActionTests extends ESTestCase { @Override protected WritePrimaryResult shardOperationOnPrimary(TestRequest request, IndexShard primary) throws Exception { - return new WritePrimaryResult(request, new TestResponse(), location, null, primary); + final WritePrimaryResult primaryResult; + if (withDocumentFailureOnPrimary) { + primaryResult = new WritePrimaryResult(request, null, null, new RuntimeException("simulated"), primary); + } else { + primaryResult = new WritePrimaryResult(request, new TestResponse(), location, null, primary); + } + return primaryResult; } @Override protected WriteReplicaResult shardOperationOnReplica(TestRequest request, IndexShard replica) throws Exception { - return new WriteReplicaResult(request, location, null, replica); + final WriteReplicaResult replicaResult; + if (withDocumentFailureOnReplica) { + replicaResult = new WriteReplicaResult(request, null, new RuntimeException("simulated"), replica); + } else { + replicaResult = new WriteReplicaResult(request, location, null, replica); + } + return replicaResult; } } @@ -168,6 +213,7 @@ public class TransportWriteActionTests extends ESTestCase { private static class CapturingActionListener implements ActionListener { private R response; + private Exception failure; @Override public void onResponse(R response) { @@ -175,8 +221,8 @@ public class TransportWriteActionTests extends ESTestCase { } @Override - public void onFailure(Exception e) { - throw new RuntimeException(e); + public void onFailure(Exception failure) { + this.failure = failure; } } diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index a4df773983a..be4b7bbdefe 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -31,8 +31,11 @@ import org.apache.lucene.codecs.Codec; import org.apache.lucene.document.Field; import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.document.TextField; +import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy; import org.apache.lucene.index.LiveIndexWriterConfig; import org.apache.lucene.index.LogByteSizeMergePolicy; @@ -106,7 +109,6 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.IndexSettingsModule; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; -import org.hamcrest.CoreMatchers; import org.hamcrest.MatcherAssert; import org.junit.After; import org.junit.Before; @@ -130,6 +132,7 @@ import java.util.concurrent.CyclicBarrier; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; import static java.util.Collections.emptyMap; import static org.elasticsearch.index.engine.Engine.Operation.Origin.PRIMARY; @@ -279,12 +282,21 @@ public class InternalEngineTests extends ESTestCase { } protected InternalEngine createEngine(Store store, Path translogPath) throws IOException { - return createEngine(defaultSettings, store, translogPath, newMergePolicy()); + return createEngine(defaultSettings, store, translogPath, newMergePolicy(), null); } protected InternalEngine createEngine(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy) throws IOException { + return createEngine(indexSettings, store, translogPath, mergePolicy, null); + + } + protected InternalEngine createEngine(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy, Supplier indexWriterSupplier) throws IOException { EngineConfig config = config(indexSettings, store, translogPath, mergePolicy, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, null); - InternalEngine internalEngine = new InternalEngine(config); + InternalEngine internalEngine = new InternalEngine(config) { + @Override + IndexWriter createWriter(boolean create) throws IOException { + return (indexWriterSupplier != null) ? indexWriterSupplier.get() : super.createWriter(create); + } + }; if (config.getOpenMode() == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG) { internalEngine.recoverFromTranslog(); } @@ -339,7 +351,7 @@ public class InternalEngineTests extends ESTestCase { ParsedDocument doc2 = testParsedDocument("2", "2", "test", null, -1, -1, testDocumentWithTextField(), B_2, null); Engine.Index second = new Engine.Index(newUid("2"), doc2); Engine.IndexResult secondResult = engine.index(second); - assertThat(secondResult.getLocation(), greaterThan(firstResult.getLocation())); + assertThat(secondResult.getTranslogLocation(), greaterThan(firstResult.getTranslogLocation())); engine.refresh("test"); segments = engine.segments(false); @@ -2134,6 +2146,72 @@ public class InternalEngineTests extends ESTestCase { } } + public void testCheckDocumentFailure() throws Exception { + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null); + Exception documentFailure = engine.checkIfDocumentFailureOrThrow(new Engine.Index(newUid("1"), doc), new IOException("simulated document failure")); + assertThat(documentFailure, instanceOf(IOException.class)); + try { + engine.checkIfDocumentFailureOrThrow(new Engine.Index(newUid("1"), doc), new CorruptIndexException("simulated environment failure", "")); + fail("expected exception to be thrown"); + } catch (ElasticsearchException envirnomentException) { + assertThat(envirnomentException.getShardId(), equalTo(engine.shardId)); + assertThat(envirnomentException.getCause().getMessage(), containsString("simulated environment failure")); + } + } + + private static class ThrowingIndexWriter extends IndexWriter { + private boolean throwDocumentFailure; + + public ThrowingIndexWriter(Directory d, IndexWriterConfig conf) throws IOException { + super(d, conf); + } + + @Override + public long addDocument(Iterable doc) throws IOException { + if (throwDocumentFailure) { + throw new IOException("simulated"); + } else { + return super.addDocument(doc); + } + } + + @Override + public long deleteDocuments(Term... terms) throws IOException { + if (throwDocumentFailure) { + throw new IOException("simulated"); + } else { + return super.deleteDocuments(terms); + } + } + + public void setThrowDocumentFailure(boolean throwDocumentFailure) { + this.throwDocumentFailure = throwDocumentFailure; + } + } + + public void testHandleDocumentFailure() throws Exception { + try (Store store = createStore()) { + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null); + ThrowingIndexWriter throwingIndexWriter = new ThrowingIndexWriter(store.directory(), new IndexWriterConfig()); + try (Engine engine = createEngine(defaultSettings, store, createTempDir(), NoMergePolicy.INSTANCE, () -> throwingIndexWriter)) { + // test document failure while indexing + throwingIndexWriter.setThrowDocumentFailure(true); + Engine.IndexResult indexResult = engine.index(randomAppendOnly(1, doc, false)); + assertNotNull(indexResult.getFailure()); + + throwingIndexWriter.setThrowDocumentFailure(false); + indexResult = engine.index(randomAppendOnly(1, doc, false)); + assertNull(indexResult.getFailure()); + + // test document failure while deleting + throwingIndexWriter.setThrowDocumentFailure(true); + Engine.DeleteResult deleteResult = engine.delete(new Engine.Delete("test", "", newUid("1"))); + assertNotNull(deleteResult.getFailure()); + } + } + + } + public void testDocStats() throws IOException { final int numDocs = randomIntBetween(2, 10); // at least 2 documents otherwise we don't see any deletes below for (int i = 0; i < numDocs; i++) { @@ -2169,22 +2247,22 @@ public class InternalEngineTests extends ESTestCase { Engine.IndexResult indexResult = engine.index(operation); assertFalse(engine.indexWriterHasDeletions()); assertEquals(0, engine.getNumVersionLookups()); - assertNotNull(indexResult.getLocation()); + assertNotNull(indexResult.getTranslogLocation()); Engine.IndexResult retryResult = engine.index(retry); assertTrue(engine.indexWriterHasDeletions()); assertEquals(0, engine.getNumVersionLookups()); - assertNotNull(retryResult.getLocation()); - assertTrue(retryResult.getLocation().compareTo(indexResult.getLocation()) > 0); + assertNotNull(retryResult.getTranslogLocation()); + assertTrue(retryResult.getTranslogLocation().compareTo(indexResult.getTranslogLocation()) > 0); } else { Engine.IndexResult retryResult = engine.index(retry); assertTrue(engine.indexWriterHasDeletions()); assertEquals(0, engine.getNumVersionLookups()); - assertNotNull(retryResult.getLocation()); + assertNotNull(retryResult.getTranslogLocation()); Engine.IndexResult indexResult = engine.index(operation); assertTrue(engine.indexWriterHasDeletions()); assertEquals(0, engine.getNumVersionLookups()); - assertNotNull(retryResult.getLocation()); - assertTrue(retryResult.getLocation().compareTo(indexResult.getLocation()) < 0); + assertNotNull(retryResult.getTranslogLocation()); + assertTrue(retryResult.getTranslogLocation().compareTo(indexResult.getTranslogLocation()) < 0); } engine.refresh("test"); @@ -2196,16 +2274,16 @@ public class InternalEngineTests extends ESTestCase { retry = randomAppendOnly(1, doc, true); if (randomBoolean()) { Engine.IndexResult indexResult = engine.index(operation); - assertNotNull(indexResult.getLocation()); + assertNotNull(indexResult.getTranslogLocation()); Engine.IndexResult retryResult = engine.index(retry); - assertNotNull(retryResult.getLocation()); - assertTrue(retryResult.getLocation().compareTo(indexResult.getLocation()) > 0); + assertNotNull(retryResult.getTranslogLocation()); + assertTrue(retryResult.getTranslogLocation().compareTo(indexResult.getTranslogLocation()) > 0); } else { Engine.IndexResult retryResult = engine.index(retry); - assertNotNull(retryResult.getLocation()); + assertNotNull(retryResult.getTranslogLocation()); Engine.IndexResult indexResult = engine.index(operation); - assertNotNull(retryResult.getLocation()); - assertTrue(retryResult.getLocation().compareTo(indexResult.getLocation()) < 0); + assertNotNull(retryResult.getTranslogLocation()); + assertTrue(retryResult.getTranslogLocation().compareTo(indexResult.getTranslogLocation()) < 0); } engine.refresh("test"); diff --git a/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java b/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java index f0f53d9fdc9..94c0407f700 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java @@ -144,7 +144,7 @@ public class RefreshListenersTests extends ESTestCase { for (int i = 0; i < maxListeners; i++) { DummyRefreshListener listener = new DummyRefreshListener(); nonForcedListeners.add(listener); - listeners.addOrNotify(index.getLocation(), listener); + listeners.addOrNotify(index.getTranslogLocation(), listener); assertTrue(listeners.refreshNeeded()); } @@ -155,7 +155,7 @@ public class RefreshListenersTests extends ESTestCase { // Add one more listener which should cause a refresh. DummyRefreshListener forcingListener = new DummyRefreshListener(); - listeners.addOrNotify(index.getLocation(), forcingListener); + listeners.addOrNotify(index.getTranslogLocation(), forcingListener); assertTrue("Forced listener wasn't forced?", forcingListener.forcedRefresh.get()); forcingListener.assertNoError(); @@ -178,7 +178,7 @@ public class RefreshListenersTests extends ESTestCase { } DummyRefreshListener listener = new DummyRefreshListener(); - assertTrue(listeners.addOrNotify(index.getLocation(), listener)); + assertTrue(listeners.addOrNotify(index.getTranslogLocation(), listener)); assertFalse(listener.forcedRefresh.get()); listener.assertNoError(); } @@ -200,7 +200,7 @@ public class RefreshListenersTests extends ESTestCase { for (int i = 0; i < 1000; i++) { Engine.IndexResult index = index("1"); DummyRefreshListener listener = new DummyRefreshListener(); - boolean immediate = listeners.addOrNotify(index.getLocation(), listener); + boolean immediate = listeners.addOrNotify(index.getTranslogLocation(), listener); if (immediate) { assertNotNull(listener.forcedRefresh.get()); } else { @@ -238,7 +238,7 @@ public class RefreshListenersTests extends ESTestCase { assertEquals(iteration, index.getVersion()); DummyRefreshListener listener = new DummyRefreshListener(); - listeners.addOrNotify(index.getLocation(), listener); + listeners.addOrNotify(index.getTranslogLocation(), listener); assertBusy(() -> assertNotNull("listener never called", listener.forcedRefresh.get())); if (threadCount < maxListeners) { assertFalse(listener.forcedRefresh.get()); From 947a17ee3722966768c18e2152263b04aa109e2f Mon Sep 17 00:00:00 2001 From: Areek Zillur Date: Wed, 26 Oct 2016 23:49:03 -0400 Subject: [PATCH 022/132] cleanup operation listener handling of failure in results --- .../elasticsearch/index/IndexingSlowLog.java | 22 ++++++------ .../elasticsearch/index/shard/IndexShard.java | 12 ++----- .../index/shard/InternalIndexingStats.java | 36 +++++++++++-------- .../indices/IndexingMemoryController.java | 14 ++++---- .../index/shard/IndexShardTests.java | 17 ++++++--- .../shard/IndexingOperationListenerTests.java | 12 +++++-- 6 files changed, 67 insertions(+), 46 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/IndexingSlowLog.java b/core/src/main/java/org/elasticsearch/index/IndexingSlowLog.java index 7217ad7284c..215013bf246 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexingSlowLog.java +++ b/core/src/main/java/org/elasticsearch/index/IndexingSlowLog.java @@ -135,16 +135,18 @@ public final class IndexingSlowLog implements IndexingOperationListener { @Override public void postIndex(Engine.Index indexOperation, Engine.IndexResult result) { - final ParsedDocument doc = indexOperation.parsedDoc(); - final long tookInNanos = result.getTook(); - if (indexWarnThreshold >= 0 && tookInNanos > indexWarnThreshold) { - indexLogger.warn("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog)); - } else if (indexInfoThreshold >= 0 && tookInNanos > indexInfoThreshold) { - indexLogger.info("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog)); - } else if (indexDebugThreshold >= 0 && tookInNanos > indexDebugThreshold) { - indexLogger.debug("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog)); - } else if (indexTraceThreshold >= 0 && tookInNanos > indexTraceThreshold) { - indexLogger.trace("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog)); + if (result.hasFailure() == false) { + final ParsedDocument doc = indexOperation.parsedDoc(); + final long tookInNanos = result.getTook(); + if (indexWarnThreshold >= 0 && tookInNanos > indexWarnThreshold) { + indexLogger.warn("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog)); + } else if (indexInfoThreshold >= 0 && tookInNanos > indexInfoThreshold) { + indexLogger.info("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog)); + } else if (indexDebugThreshold >= 0 && tookInNanos > indexDebugThreshold) { + indexLogger.debug("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog)); + } else if (indexTraceThreshold >= 0 && tookInNanos > indexTraceThreshold) { + indexLogger.trace("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog)); + } } } diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 8a5a7cc6b44..0de912ee2b0 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -555,11 +555,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl indexingOperationListeners.postIndex(index, e); throw e; } - if (result.hasFailure()) { - indexingOperationListeners.postIndex(index, result.getFailure()); - } else { - indexingOperationListeners.postIndex(index, result); - } + indexingOperationListeners.postIndex(index, result); return result; } @@ -604,11 +600,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl indexingOperationListeners.postDelete(delete, e); throw e; } - if (result.hasFailure()) { - indexingOperationListeners.postDelete(delete, result.getFailure()); - } else { - indexingOperationListeners.postDelete(delete, result); - } + indexingOperationListeners.postDelete(delete, result); return result; } diff --git a/core/src/main/java/org/elasticsearch/index/shard/InternalIndexingStats.java b/core/src/main/java/org/elasticsearch/index/shard/InternalIndexingStats.java index 39a415ca8eb..32868a7368a 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/InternalIndexingStats.java +++ b/core/src/main/java/org/elasticsearch/index/shard/InternalIndexingStats.java @@ -75,13 +75,17 @@ final class InternalIndexingStats implements IndexingOperationListener { @Override public void postIndex(Engine.Index index, Engine.IndexResult result) { - if (!index.origin().isRecovery()) { - long took = result.getTook(); - totalStats.indexMetric.inc(took); - totalStats.indexCurrent.dec(); - StatsHolder typeStats = typeStats(index.type()); - typeStats.indexMetric.inc(took); - typeStats.indexCurrent.dec(); + if (result.hasFailure() == false) { + if (!index.origin().isRecovery()) { + long took = result.getTook(); + totalStats.indexMetric.inc(took); + totalStats.indexCurrent.dec(); + StatsHolder typeStats = typeStats(index.type()); + typeStats.indexMetric.inc(took); + typeStats.indexCurrent.dec(); + } + } else { + postIndex(index, result.getFailure()); } } @@ -107,13 +111,17 @@ final class InternalIndexingStats implements IndexingOperationListener { @Override public void postDelete(Engine.Delete delete, Engine.DeleteResult result) { - if (!delete.origin().isRecovery()) { - long took = result.getTook(); - totalStats.deleteMetric.inc(took); - totalStats.deleteCurrent.dec(); - StatsHolder typeStats = typeStats(delete.type()); - typeStats.deleteMetric.inc(took); - typeStats.deleteCurrent.dec(); + if (result.hasFailure() == false) { + if (!delete.origin().isRecovery()) { + long took = result.getTook(); + totalStats.deleteMetric.inc(took); + totalStats.deleteCurrent.dec(); + StatsHolder typeStats = typeStats(delete.type()); + typeStats.deleteMetric.inc(took); + typeStats.deleteCurrent.dec(); + } + } else { + postDelete(delete, result.getFailure()); } } diff --git a/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java b/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java index 5b6bdb05cf1..dd950806b6f 100644 --- a/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java +++ b/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java @@ -211,13 +211,15 @@ public class IndexingMemoryController extends AbstractComponent implements Index /** called by IndexShard to record that this many bytes were written to translog */ private void recordOperationBytes(Engine.Operation operation, Engine.Result result) { - final int sizeInBytes; - if (result.getTranslogLocation() != null) { - sizeInBytes = result.getSizeInBytes(); - } else { - sizeInBytes = operation.estimatedSizeInBytes(); + if (result.hasFailure() == false) { + final int sizeInBytes; + if (result.getTranslogLocation() != null) { + sizeInBytes = result.getSizeInBytes(); + } else { + sizeInBytes = operation.estimatedSizeInBytes(); + } + statusChecker.bytesWritten(sizeInBytes); } - statusChecker.bytesWritten(sizeInBytes); } private static final class ShardAndBytesUsed implements Comparable { diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index dada068446a..c3c4454f8a0 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -115,6 +115,7 @@ import java.util.function.BiConsumer; import static java.util.Collections.emptyMap; import static java.util.Collections.emptySet; import static org.elasticsearch.common.lucene.Lucene.cleanLuceneIndex; +import static org.elasticsearch.common.lucene.Lucene.readScoreDoc; import static org.elasticsearch.common.xcontent.ToXContent.EMPTY_PARAMS; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.hamcrest.Matchers.containsString; @@ -565,10 +566,14 @@ public class IndexShardTests extends IndexShardTestCase { @Override public void postIndex(Engine.Index index, Engine.IndexResult result) { - if (result.isCreated()) { - postIndexCreate.incrementAndGet(); + if (result.hasFailure() == false) { + if (result.isCreated()) { + postIndexCreate.incrementAndGet(); + } else { + postIndexUpdate.incrementAndGet(); + } } else { - postIndexUpdate.incrementAndGet(); + postIndex(index, result.getFailure()); } } @@ -585,7 +590,11 @@ public class IndexShardTests extends IndexShardTestCase { @Override public void postDelete(Engine.Delete delete, Engine.DeleteResult result) { - postDelete.incrementAndGet(); + if (result.hasFailure() == false) { + postDelete.incrementAndGet(); + } else { + postDelete(delete, result.getFailure()); + } } @Override diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java index a29be0cd69a..b810175fcd8 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java @@ -47,7 +47,11 @@ public class IndexingOperationListenerTests extends ESTestCase{ @Override public void postIndex(Engine.Index index, Engine.IndexResult result) { - postIndex.incrementAndGet(); + if (result.hasFailure() == false) { + postIndex.incrementAndGet(); + } else { + postIndex(index, result.getFailure()); + } } @Override @@ -63,7 +67,11 @@ public class IndexingOperationListenerTests extends ESTestCase{ @Override public void postDelete(Engine.Delete delete, Engine.DeleteResult result) { - postDelete.incrementAndGet(); + if (result.hasFailure() == false) { + postDelete.incrementAndGet(); + } else { + postDelete(delete, result.getFailure()); + } } @Override From 118913b553c970b21f93c441d81e5d4b409b73d0 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 27 Oct 2016 11:00:04 +0200 Subject: [PATCH 023/132] [TEST] Fix testRolloverConditionsNotMet to expect correct rollover index name PR #21138 changed the target index name even if _rollover conditions are not met but missed to adapt this test. --- .../elasticsearch/action/admin/indices/rollover/RolloverIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/java/org/elasticsearch/action/admin/indices/rollover/RolloverIT.java b/core/src/test/java/org/elasticsearch/action/admin/indices/rollover/RolloverIT.java index d5bc16207f5..af5909005ae 100644 --- a/core/src/test/java/org/elasticsearch/action/admin/indices/rollover/RolloverIT.java +++ b/core/src/test/java/org/elasticsearch/action/admin/indices/rollover/RolloverIT.java @@ -130,7 +130,7 @@ public class RolloverIT extends ESIntegTestCase { final RolloverResponse response = client().admin().indices().prepareRolloverIndex("test_alias") .addMaxIndexAgeCondition(TimeValue.timeValueHours(4)).get(); assertThat(response.getOldIndex(), equalTo("test_index-0")); - assertThat(response.getNewIndex(), equalTo("test_index-0")); + assertThat(response.getNewIndex(), equalTo("test_index-000001")); assertThat(response.isDryRun(), equalTo(false)); assertThat(response.isRolledOver(), equalTo(false)); assertThat(response.getConditionStatus().size(), equalTo(1)); From 952097b1c063e566851eb4aea1af28cb1acaf9c2 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 27 Oct 2016 12:02:00 +0200 Subject: [PATCH 024/132] [TEST] Fix testDelayShards to wait for master to remove stopped node This test failed when the node that was shutting down was not yet removed from the cluster state on the master. The cluster allocation explain API will not see any unassigned shards until the node shutting down is removed from the cluster state. --- .../allocation/ClusterAllocationExplainIT.java | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplainIT.java b/core/src/test/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplainIT.java index 97c1a20c33f..23fdf3499b2 100644 --- a/core/src/test/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplainIT.java +++ b/core/src/test/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplainIT.java @@ -19,7 +19,6 @@ package org.elasticsearch.action.admin.cluster.allocation; -import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse; import org.elasticsearch.action.admin.indices.shards.IndicesShardStoresResponse; import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.client.Requests; @@ -31,7 +30,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.junit.annotations.TestLogging; -import java.util.List; import java.util.Map; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; @@ -47,14 +45,11 @@ public final class ClusterAllocationExplainIT extends ESIntegTestCase { @TestLogging("_root:DEBUG") public void testDelayShards() throws Exception { logger.info("--> starting 3 nodes"); - List nodes = internalCluster().startNodesAsync(3).get(); + internalCluster().startNodesAsync(3).get(); // Wait for all 3 nodes to be up logger.info("--> waiting for 3 nodes to be up"); - assertBusy(() -> { - NodesStatsResponse resp = client().admin().cluster().prepareNodesStats().get(); - assertThat(resp.getNodes().size(), equalTo(3)); - }); + ensureStableCluster(3); logger.info("--> creating 'test' index"); assertAcked(prepareCreate("test").setSettings(Settings.builder() @@ -66,7 +61,8 @@ public final class ClusterAllocationExplainIT extends ESIntegTestCase { logger.info("--> stopping a random node"); assertTrue(internalCluster().stopRandomDataNode()); - ensureYellow("test"); + logger.info("--> waiting for the master to remove the stopped node from the cluster state"); + ensureStableCluster(2); ClusterAllocationExplainResponse resp = client().admin().cluster().prepareAllocationExplain().useAnyUnassignedShard().get(); ClusterAllocationExplanation cae = resp.getExplanation(); From f3e578f9428ba91dd44aedc86e1fe23510fb7446 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 27 Oct 2016 13:48:17 +0200 Subject: [PATCH 025/132] Stop delaying existing requests after network delay rule is cleared (#21129) The network disruption type "network delay" continues delaying existing requests even after the disruption has been cleared. This commit ensures that the requests get to execute right after the delay rule is cleared. --- .../DiscoveryWithServiceDisruptionsIT.java | 20 ++++++-- .../test/disruption/NetworkDisruption.java | 13 +++++ .../test/transport/MockTransportService.java | 51 ++++++++++++++++--- 3 files changed, 75 insertions(+), 9 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java b/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java index 2ea675ab3f7..2ca9f6f3384 100644 --- a/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java +++ b/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java @@ -153,11 +153,25 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase { return 1; } + private boolean disableBeforeIndexDeletion; + + @Override + public void setDisruptionScheme(ServiceDisruptionScheme scheme) { + if (scheme instanceof NetworkDisruption && + ((NetworkDisruption) scheme).getNetworkLinkDisruptionType() instanceof NetworkUnresponsive) { + // the network unresponsive disruption may leave operations in flight + // this is because this disruption scheme swallows requests by design + // as such, these operations will never be marked as finished + disableBeforeIndexDeletion = true; + } + super.setDisruptionScheme(scheme); + } + @Override protected void beforeIndexDeletion() { - // some test may leave operations in flight - // this is because the disruption schemes swallow requests by design - // as such, these operations will never be marked as finished + if (disableBeforeIndexDeletion == false) { + super.beforeIndexDeletion(); + } } private List startCluster(int numberOfNodes) throws ExecutionException, InterruptedException { diff --git a/test/framework/src/main/java/org/elasticsearch/test/disruption/NetworkDisruption.java b/test/framework/src/main/java/org/elasticsearch/test/disruption/NetworkDisruption.java index f7094d8ae9f..de57eee6937 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/disruption/NetworkDisruption.java +++ b/test/framework/src/main/java/org/elasticsearch/test/disruption/NetworkDisruption.java @@ -58,6 +58,14 @@ public class NetworkDisruption implements ServiceDisruptionScheme { this.networkLinkDisruptionType = networkLinkDisruptionType; } + public DisruptedLinks getDisruptedLinks() { + return disruptedLinks; + } + + public NetworkLinkDisruptionType getNetworkLinkDisruptionType() { + return networkLinkDisruptionType; + } + @Override public void applyToCluster(InternalTestCluster cluster) { this.cluster = cluster; @@ -143,6 +151,11 @@ public class NetworkDisruption implements ServiceDisruptionScheme { return (MockTransportService) cluster.getInstance(TransportService.class, node); } + @Override + public String toString() { + return "network disruption (disruption type: " + networkLinkDisruptionType + ", disrupted links: " + disruptedLinks + ")"; + } + /** * Represents a set of nodes with connections between nodes that are to be disrupted */ diff --git a/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java b/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java index cef12bc930a..c63f968011e 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java +++ b/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java @@ -58,9 +58,12 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Queue; import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.atomic.AtomicBoolean; /** * A mock transport service that allows to simulate different network topology failures. @@ -95,7 +98,7 @@ public final class MockTransportService extends TransportService { /** * Build the service. - * + * * @param clusterSettings if non null the the {@linkplain TransportService} will register with the {@link ClusterSettings} for settings * updates for {@link #TRACE_LOG_EXCLUDE_SETTING} and {@link #TRACE_LOG_INCLUDE_SETTING}. */ @@ -142,7 +145,10 @@ public final class MockTransportService extends TransportService { * Clears the rule associated with the provided transport address. */ public void clearRule(TransportAddress transportAddress) { - transport().transports.remove(transportAddress); + Transport transport = transport().transports.remove(transportAddress); + if (transport instanceof ClearableTransport) { + ((ClearableTransport) transport).clearRule(); + } } /** @@ -292,7 +298,8 @@ public final class MockTransportService extends TransportService { public void addUnresponsiveRule(TransportAddress transportAddress, final TimeValue duration) { final long startTime = System.currentTimeMillis(); - addDelegate(transportAddress, new DelegateTransport(original) { + addDelegate(transportAddress, new ClearableTransport(original) { + private final Queue requestsToSendWhenCleared = new ConcurrentLinkedQueue<>(); TimeValue getDelay() { return new TimeValue(duration.millis() - (System.currentTimeMillis() - startTime)); @@ -362,7 +369,9 @@ public final class MockTransportService extends TransportService { final TransportRequest clonedRequest = reg.newRequest(); clonedRequest.readFrom(bStream.bytes().streamInput()); - threadPool.schedule(delay, ThreadPool.Names.GENERIC, new AbstractRunnable() { + Runnable runnable = new AbstractRunnable() { + AtomicBoolean requestSent = new AtomicBoolean(); + @Override public void onFailure(Exception e) { logger.debug("failed to send delayed request", e); @@ -370,9 +379,22 @@ public final class MockTransportService extends TransportService { @Override protected void doRun() throws IOException { - original.sendRequest(node, requestId, action, clonedRequest, options); + if (requestSent.compareAndSet(false, true)) { + original.sendRequest(node, requestId, action, clonedRequest, options); + } } - }); + }; + + // store the request to send it once the rule is cleared. + requestsToSendWhenCleared.add(runnable); + + threadPool.schedule(delay, ThreadPool.Names.GENERIC, runnable); + } + + + @Override + public void clearRule() { + requestsToSendWhenCleared.forEach(Runnable::run); } }); } @@ -555,6 +577,23 @@ public final class MockTransportService extends TransportService { } } + /** + * The delegate transport instances defined in this class mock various kinds of disruption types. This subclass adds a method + * {@link #clearRule()} so that when the disruptions are cleared (see {@link #clearRule(TransportService)}) this gives the + * disruption a possibility to run clean-up actions. + */ + public abstract static class ClearableTransport extends DelegateTransport { + + public ClearableTransport(Transport transport) { + super(transport); + } + + /** + * Called by {@link #clearRule(TransportService)} + */ + public abstract void clearRule(); + } + List activeTracers = new CopyOnWriteArrayList<>(); From 9944a594b10d5cfa34fcf2fa8ed45ca6d52345fe Mon Sep 17 00:00:00 2001 From: markharwood Date: Tue, 25 Oct 2016 15:00:54 +0100 Subject: [PATCH 026/132] Aggregations fix: scripted heuristics for scoring significant_terms aggs were not thread safe when running local to the coordinating node. New code spawns an object for each shard search execution rather than sharing a common instance which is not thread safe. Closes #18120 --- .../significant/InternalSignificantTerms.java | 4 +- .../SignificantTermsAggregationBuilder.java | 4 +- .../heuristics/ScriptHeuristic.java | 71 ++++++++++--------- .../heuristics/SignificanceHeuristic.java | 20 ++++-- 4 files changed, 57 insertions(+), 42 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/InternalSignificantTerms.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/InternalSignificantTerms.java index cd18386da1e..cdd1f8d19a7 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/InternalSignificantTerms.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/InternalSignificantTerms.java @@ -197,13 +197,13 @@ public abstract class InternalSignificantTerms ordered = new BucketSignificancePriorityQueue<>(size); for (Map.Entry> entry : buckets.entrySet()) { List sameTermBuckets = entry.getValue(); final B b = sameTermBuckets.get(0).reduce(sameTermBuckets, reduceContext); - b.updateScore(getSignificanceHeuristic()); + b.updateScore(heuristic); if ((b.score > 0) && (b.subsetDf >= minDocCount)) { ordered.insertWithOverflow(b); } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregationBuilder.java index 1cf422ae50a..5af538965d1 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregationBuilder.java @@ -217,9 +217,9 @@ public class SignificantTermsAggregationBuilder extends ValuesSourceAggregationB @Override protected ValuesSourceAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig config, AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { - this.significanceHeuristic.initialize(context.searchContext()); + SignificanceHeuristic executionHeuristic = this.significanceHeuristic.rewrite(context.searchContext()); return new SignificantTermsAggregatorFactory(name, type, config, includeExclude, executionHint, filterBuilder, - bucketCountThresholds, significanceHeuristic, context, parent, subFactoriesBuilder, metaData); + bucketCountThresholds, executionHeuristic, context, parent, subFactoriesBuilder, metaData); } @Override diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/heuristics/ScriptHeuristic.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/heuristics/ScriptHeuristic.java index c854b036b00..748adb67ce5 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/heuristics/ScriptHeuristic.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/heuristics/ScriptHeuristic.java @@ -24,7 +24,6 @@ package org.elasticsearch.search.aggregations.bucket.significant.heuristics; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.logging.ESLoggerFactory; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.query.QueryShardException; @@ -43,18 +42,41 @@ import java.util.Objects; public class ScriptHeuristic extends SignificanceHeuristic { public static final String NAME = "script_heuristic"; - private final LongAccessor subsetSizeHolder; - private final LongAccessor supersetSizeHolder; - private final LongAccessor subsetDfHolder; - private final LongAccessor supersetDfHolder; private final Script script; - ExecutableScript executableScript = null; + + // This class holds an executable form of the script with private variables ready for execution + // on a single search thread. + static class ExecutableScriptHeuristic extends ScriptHeuristic { + private final LongAccessor subsetSizeHolder; + private final LongAccessor supersetSizeHolder; + private final LongAccessor subsetDfHolder; + private final LongAccessor supersetDfHolder; + private final ExecutableScript executableScript; + + ExecutableScriptHeuristic(Script script, ExecutableScript executableScript){ + super(script); + subsetSizeHolder = new LongAccessor(); + supersetSizeHolder = new LongAccessor(); + subsetDfHolder = new LongAccessor(); + supersetDfHolder = new LongAccessor(); + this.executableScript = executableScript; + executableScript.setNextVar("_subset_freq", subsetDfHolder); + executableScript.setNextVar("_subset_size", subsetSizeHolder); + executableScript.setNextVar("_superset_freq", supersetDfHolder); + executableScript.setNextVar("_superset_size", supersetSizeHolder); + } + + @Override + public double getScore(long subsetFreq, long subsetSize, long supersetFreq, long supersetSize) { + subsetSizeHolder.value = subsetSize; + supersetSizeHolder.value = supersetSize; + subsetDfHolder.value = subsetFreq; + supersetDfHolder.value = supersetFreq; + return ((Number) executableScript.run()).doubleValue(); + } + } public ScriptHeuristic(Script script) { - subsetSizeHolder = new LongAccessor(); - supersetSizeHolder = new LongAccessor(); - subsetDfHolder = new LongAccessor(); - supersetDfHolder = new LongAccessor(); this.script = script; } @@ -71,22 +93,15 @@ public class ScriptHeuristic extends SignificanceHeuristic { } @Override - public void initialize(InternalAggregation.ReduceContext context) { - initialize(context.scriptService().executable(script, ScriptContext.Standard.AGGS, Collections.emptyMap())); + public SignificanceHeuristic rewrite(InternalAggregation.ReduceContext context) { + return new ExecutableScriptHeuristic(script, context.scriptService().executable(script, ScriptContext.Standard.AGGS, Collections.emptyMap())); } @Override - public void initialize(SearchContext context) { - initialize(context.getQueryShardContext().getExecutableScript(script, ScriptContext.Standard.AGGS, Collections.emptyMap())); + public SignificanceHeuristic rewrite(SearchContext context) { + return new ExecutableScriptHeuristic(script, context.getQueryShardContext().getExecutableScript(script, ScriptContext.Standard.AGGS, Collections.emptyMap())); } - public void initialize(ExecutableScript executableScript) { - executableScript.setNextVar("_subset_freq", subsetDfHolder); - executableScript.setNextVar("_subset_size", subsetSizeHolder); - executableScript.setNextVar("_superset_freq", supersetDfHolder); - executableScript.setNextVar("_superset_size", supersetSizeHolder); - this.executableScript = executableScript; - } /** * Calculates score with a script @@ -99,19 +114,7 @@ public class ScriptHeuristic extends SignificanceHeuristic { */ @Override public double getScore(long subsetFreq, long subsetSize, long supersetFreq, long supersetSize) { - if (executableScript == null) { - //In tests, wehn calling assertSearchResponse(..) the response is streamed one additional time with an arbitrary version, see assertVersionSerializable(..). - // Now, for version before 1.5.0 the score is computed after streaming the response but for scripts the script does not exists yet. - // assertSearchResponse() might therefore fail although there is no problem. - // This should be replaced by an exception in 2.0. - ESLoggerFactory.getLogger("script heuristic").warn("cannot compute score - script has not been initialized yet."); - return 0; - } - subsetSizeHolder.value = subsetSize; - supersetSizeHolder.value = supersetSize; - subsetDfHolder.value = subsetFreq; - supersetDfHolder.value = supersetFreq; - return ((Number) executableScript.run()).doubleValue(); + throw new UnsupportedOperationException("This scoring heuristic must have 'rewrite' called on it to provide a version ready for use"); } @Override diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/heuristics/SignificanceHeuristic.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/heuristics/SignificanceHeuristic.java index db9711c1a8d..7b6cf699741 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/heuristics/SignificanceHeuristic.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/heuristics/SignificanceHeuristic.java @@ -50,11 +50,23 @@ public abstract class SignificanceHeuristic implements NamedWriteable, ToXConten } } - public void initialize(InternalAggregation.ReduceContext reduceContext) { - + /** + * Provides a hook for subclasses to provide a version of the heuristic + * prepared for execution on data on the coordinating node. + * @param reduceContext the reduce context on the coordinating node + * @return a version of this heuristic suitable for execution + */ + public SignificanceHeuristic rewrite(InternalAggregation.ReduceContext reduceContext) { + return this; } - public void initialize(SearchContext context) { - + /** + * Provides a hook for subclasses to provide a version of the heuristic + * prepared for execution on data on a shard. + * @param context the search context on the data node + * @return a version of this heuristic suitable for execution + */ + public SignificanceHeuristic rewrite(SearchContext context) { + return this; } } From 97cc426a890fa220a926d50faee463cf2cecb3a3 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Thu, 27 Oct 2016 13:38:40 +0200 Subject: [PATCH 027/132] Fix bwc cluster formation in order to run BWC tests against a mixed version cluster (#21145) This fixes our cluster formation task to run REST tests against a mixed version cluster. Yet, due to some limitations in our test framework `indices.rollover` tests are currently disabled for the BWC case since they select the current master as the merge node which happens to be a BWC node and we can't relocate all shards to it since the primaries are on a higher version node. This will be fixed in a followup. Closes #21142 Note: This has been cherry-picked from 5.0 and fixes several rest tests as well as a BWC break in `OsStats.java` --- .../gradle/test/ClusterFormationTasks.groovy | 10 +++---- .../org/elasticsearch/monitor/os/OsStats.java | 13 +++++++-- .../java/org/elasticsearch/VersionTests.java | 2 ++ .../test/cat.aliases/10_basic.yaml | 4 +++ .../test/cat.indices/10_basic.yaml | 4 +++ .../test/cat.repositories/10_basic.yaml | 4 ++- .../test/cat.shards/10_basic.yaml | 3 ++ .../test/cat.templates/10_basic.yaml | 28 ++++++++++++++----- .../rest-api-spec/test/get/90_versions.yaml | 1 + .../test/indices.rollover/10_basic.yaml | 24 +++++++++++++--- .../test/indices.shrink/10_basic.yaml | 6 ++++ .../parser/ClientYamlTestSectionParser.java | 9 +++--- 12 files changed, 84 insertions(+), 24 deletions(-) diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy index 957e845aa57..ce47ac6c541 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy @@ -73,8 +73,8 @@ class ClusterFormationTasks { } // this is our current version distribution configuration we use for all kinds of REST tests etc. String distroConfigName = "${task.name}_elasticsearchDistro" - Configuration distro = project.configurations.create(distroConfigName) - configureDistributionDependency(project, config.distribution, distro, VersionProperties.elasticsearch) + Configuration currentDistro = project.configurations.create(distroConfigName) + configureDistributionDependency(project, config.distribution, currentDistro, VersionProperties.elasticsearch) if (config.bwcVersion != null && config.numBwcNodes > 0) { // if we have a cluster that has a BWC cluster we also need to configure a dependency on the BWC version // this version uses the same distribution etc. and only differs in the version we depend on. @@ -85,11 +85,11 @@ class ClusterFormationTasks { } configureDistributionDependency(project, config.distribution, project.configurations.elasticsearchBwcDistro, config.bwcVersion) } - - for (int i = 0; i < config.numNodes; ++i) { + for (int i = 0; i < config.numNodes; i++) { // we start N nodes and out of these N nodes there might be M bwc nodes. // for each of those nodes we might have a different configuratioon String elasticsearchVersion = VersionProperties.elasticsearch + Configuration distro = currentDistro if (i < config.numBwcNodes) { elasticsearchVersion = config.bwcVersion distro = project.configurations.elasticsearchBwcDistro @@ -252,7 +252,7 @@ class ClusterFormationTasks { 'path.repo' : "${node.sharedDir}/repo", 'path.shared_data' : "${node.sharedDir}/", // Define a node attribute so we can test that it exists - 'node.attr.testattr' : 'test', + 'node.attr.testattr' : 'test', 'repositories.url.allowed_urls': 'http://snapshot.test*' ] esConfig['node.max_local_storage_nodes'] = node.config.numNodes diff --git a/core/src/main/java/org/elasticsearch/monitor/os/OsStats.java b/core/src/main/java/org/elasticsearch/monitor/os/OsStats.java index fa3c6aa861d..aec443280d0 100644 --- a/core/src/main/java/org/elasticsearch/monitor/os/OsStats.java +++ b/core/src/main/java/org/elasticsearch/monitor/os/OsStats.java @@ -19,6 +19,7 @@ package org.elasticsearch.monitor.os; +import org.elasticsearch.Version; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; @@ -31,7 +32,7 @@ import java.util.Arrays; import java.util.Objects; public class OsStats implements Writeable, ToXContent { - + public static final Version V_5_1_0 = Version.fromId(5010099); private final long timestamp; private final Cpu cpu; private final Mem mem; @@ -51,7 +52,11 @@ public class OsStats implements Writeable, ToXContent { this.cpu = new Cpu(in); this.mem = new Mem(in); this.swap = new Swap(in); - this.cgroup = in.readOptionalWriteable(Cgroup::new); + if (in.getVersion().onOrAfter(V_5_1_0)) { + this.cgroup = in.readOptionalWriteable(Cgroup::new); + } else { + this.cgroup = null; + } } @Override @@ -60,7 +65,9 @@ public class OsStats implements Writeable, ToXContent { cpu.writeTo(out); mem.writeTo(out); swap.writeTo(out); - out.writeOptionalWriteable(cgroup); + if (out.getVersion().onOrAfter(V_5_1_0)) { + out.writeOptionalWriteable(cgroup); + } } public long getTimestamp() { diff --git a/core/src/test/java/org/elasticsearch/VersionTests.java b/core/src/test/java/org/elasticsearch/VersionTests.java index cc2f000fbae..9dcbd8d1441 100644 --- a/core/src/test/java/org/elasticsearch/VersionTests.java +++ b/core/src/test/java/org/elasticsearch/VersionTests.java @@ -23,6 +23,7 @@ import org.elasticsearch.action.ShardValidateQueryRequestTests; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.monitor.os.OsStats; import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.VersionUtils; @@ -271,6 +272,7 @@ public class VersionTests extends ESTestCase { assertUnknownVersion(V_20_0_0_UNRELEASED); expectThrows(AssertionError.class, () -> assertUnknownVersion(Version.CURRENT)); assertUnknownVersion(AliasFilter.V_5_1_0); // once we released 5.1.0 and it's added to Version.java we need to remove this constant + assertUnknownVersion(OsStats.V_5_1_0); // once we released 5.1.0 and it's added to Version.java we need to remove this constant // once we released 5.0.0 and it's added to Version.java we need to remove this constant assertUnknownVersion(ShardValidateQueryRequestTests.V_5_0_0); } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.aliases/10_basic.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.aliases/10_basic.yaml index 63670061b6d..fc7eb456892 100755 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.aliases/10_basic.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.aliases/10_basic.yaml @@ -219,6 +219,10 @@ --- "Alias sorting": + - skip: + version: " - 5.0.99" + reason: sorting was introduced in 5.1.0 + - do: indices.create: index: test_index diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.indices/10_basic.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.indices/10_basic.yaml index 0b3cdba46a4..3e900132273 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.indices/10_basic.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.indices/10_basic.yaml @@ -160,6 +160,10 @@ --- "Test cat indices sort": + - skip: + version: " - 5.0.99" + reason: sorting was introduced in 5.1.0 + - do: indices.create: index: foo diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.repositories/10_basic.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.repositories/10_basic.yaml index 2345df9732e..6d83274726e 100755 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.repositories/10_basic.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.repositories/10_basic.yaml @@ -46,7 +46,9 @@ --- "Test cat repositories sort": - + - skip: + version: " - 5.0.99" + reason: sorting was introduced in 5.1.0 - do: snapshot.create_repository: repository: test_cat_repo_1 diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.shards/10_basic.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.shards/10_basic.yaml index fc077ba6529..fbdaff6cbd9 100755 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.shards/10_basic.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.shards/10_basic.yaml @@ -227,6 +227,9 @@ --- "Test cat shards sort": + - skip: + version: " - 5.0.99" + reason: sorting was introduced in 5.1.0 - do: indices.create: diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.templates/10_basic.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.templates/10_basic.yaml index 28a7e1f3bee..eb651f6b157 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.templates/10_basic.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.templates/10_basic.yaml @@ -1,6 +1,8 @@ --- "Help": - + - skip: + version: " - 5.0.99" + reason: templates were introduced in 5.1.0 - do: cat.templates: help: true @@ -15,7 +17,9 @@ --- "No templates": - + - skip: + version: " - 5.0.99" + reason: templates were introduced in 5.1.0 - do: cat.templates: {} @@ -26,7 +30,9 @@ --- "Normal templates": - + - skip: + version: " - 5.0.99" + reason: templates were introduced in 5.1.0 - do: indices.put_template: name: test @@ -72,7 +78,9 @@ --- "Filtered templates": - + - skip: + version: " - 5.0.99" + reason: templates were introduced in 5.1.0 - do: indices.put_template: name: test @@ -111,7 +119,9 @@ --- "Column headers": - + - skip: + version: " - 5.0.99" + reason: templates were introduced in 5.1.0 - do: indices.put_template: name: test @@ -145,7 +155,9 @@ --- "Select columns": - + - skip: + version: " - 5.0.99" + reason: templates were introduced in 5.1.0 - do: indices.put_template: name: test @@ -176,7 +188,9 @@ --- "Sort templates": - + - skip: + version: " - 5.0.99" + reason: templates were introduced in 5.1.0 - do: indices.put_template: name: test diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/get/90_versions.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/get/90_versions.yaml index e255ce510ed..c6631b83b18 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/get/90_versions.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/get/90_versions.yaml @@ -86,3 +86,4 @@ id: 1 version: 1 version_type: external_gte + diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.rollover/10_basic.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.rollover/10_basic.yaml index e986d3e931a..98af719bbe0 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.rollover/10_basic.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.rollover/10_basic.yaml @@ -71,6 +71,22 @@ - match: { hits.total: 1 } - match: { hits.hits.0._index: "logs-000002"} +--- +"Rollover no condition matched": + - skip: + version: " - 5.0.0" + reason: bug fixed in 5.0.1 + + # create index with alias + - do: + indices.create: + index: logs-1 + wait_for_active_shards: 1 + body: + aliases: + logs_index: {} + logs_search: {} + # run again and verify results without rolling over - do: indices.rollover: @@ -78,11 +94,11 @@ wait_for_active_shards: 1 body: conditions: - max_docs: 100 + max_docs: 1 - - match: { old_index: logs-000002 } - - match: { new_index: logs-000003 } + - match: { old_index: logs-1 } + - match: { new_index: logs-000002 } - match: { rolled_over: false } - match: { dry_run: false } - - match: { conditions: { "[max_docs: 100]": false } } + - match: { conditions: { "[max_docs: 1]": false } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.shrink/10_basic.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.shrink/10_basic.yaml index 9569728ce7d..d793816ed5d 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.shrink/10_basic.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.shrink/10_basic.yaml @@ -1,5 +1,11 @@ --- "Shrink index via API": + - skip: + version: " - 5.0.0" + reason: this doesn't work yet with BWC tests since the master is from the old verion + # TODO we need to fix this for BWC tests to make sure we get a node that all shards can allocate on + # today if we run BWC tests and we select the master as a _shrink node but since primaries are allocated + # on the newer version nodes this fails... # creates an index with one document. # relocates all it's shards to one node # shrinks it into a new index with a single shard diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/parser/ClientYamlTestSectionParser.java b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/parser/ClientYamlTestSectionParser.java index b6e8ad6c0f4..b6b6adfd037 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/parser/ClientYamlTestSectionParser.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/parser/ClientYamlTestSectionParser.java @@ -36,16 +36,17 @@ public class ClientYamlTestSectionParser implements ClientYamlTestFragmentParser try { parser.nextToken(); testSection.setSkipSection(parseContext.parseSkipSection()); - + while ( parser.currentToken() != XContentParser.Token.END_ARRAY) { parseContext.advanceToFieldName(); testSection.addExecutableSection(parseContext.parseExecutableSection()); } - + parser.nextToken(); - assert parser.currentToken() == XContentParser.Token.END_OBJECT; + assert parser.currentToken() == XContentParser.Token.END_OBJECT : "malformed section [" + testSection.getName() + "] expected " + + XContentParser.Token.END_OBJECT + " but was " + parser.currentToken(); parser.nextToken(); - + return testSection; } catch (Exception e) { throw new ClientYamlTestParseException("Error parsing test named [" + testSection.getName() + "]", e); From 2f883fcb85b1e76d20c03d09f5200354c403d465 Mon Sep 17 00:00:00 2001 From: Areek Zillur Date: Thu, 27 Oct 2016 16:38:15 -0400 Subject: [PATCH 028/132] Rethrow original exception when it fails the engine during write operations --- .../elasticsearch/index/engine/InternalEngine.java | 13 ++++++++----- .../index/engine/InternalEngineTests.java | 5 ++--- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 8330da04682..833eb92463a 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -411,8 +411,7 @@ public class InternalEngine extends Engine { * Inspects exception thrown when executing index or delete operations * * @return failure if the failure is a document specific failure (e.g. analysis chain failure) - * @throws ElasticsearchException if the failure caused the engine to fail - * (e.g. out of disk, lucene tragic event) + * or throws Exception if the failure caused the engine to fail (e.g. out of disk, lucene tragic event) * * Note: pkg-private for testing */ @@ -435,12 +434,16 @@ public class InternalEngine extends Engine { if (isDocumentFailure) { return failure; } else { - ElasticsearchException exception = new ElasticsearchException(failure); - exception.setShard(shardId); - throw exception; + rethrow(failure); + return null; } } + @SuppressWarnings("unchecked") + static void rethrow(Throwable t) throws T { + throw (T) t; + } + private boolean canOptimizeAddDocument(Index index) { if (index.getAutoGeneratedIdTimestamp() != IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP) { assert index.getAutoGeneratedIdTimestamp() >= 0 : "autoGeneratedIdTimestamp must be positive but was: " diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index be4b7bbdefe..bb2b7fe3aed 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -2153,9 +2153,8 @@ public class InternalEngineTests extends ESTestCase { try { engine.checkIfDocumentFailureOrThrow(new Engine.Index(newUid("1"), doc), new CorruptIndexException("simulated environment failure", "")); fail("expected exception to be thrown"); - } catch (ElasticsearchException envirnomentException) { - assertThat(envirnomentException.getShardId(), equalTo(engine.shardId)); - assertThat(envirnomentException.getCause().getMessage(), containsString("simulated environment failure")); + } catch (Exception envirnomentException) { + assertThat(envirnomentException.getMessage(), containsString("simulated environment failure")); } } From 557506ba6e806ec01f22928bc02f2e626b09725a Mon Sep 17 00:00:00 2001 From: Ali Beyad Date: Thu, 27 Oct 2016 16:44:04 -0400 Subject: [PATCH 029/132] Updates the versions the vagrant tests can upgrade from to 5.0.0 (#21151) --- qa/vagrant/versions | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/qa/vagrant/versions b/qa/vagrant/versions index 654a95a3a25..0062ac97180 100644 --- a/qa/vagrant/versions +++ b/qa/vagrant/versions @@ -1 +1 @@ -6.0.0-alpha1-SNAPSHOT +5.0.0 From 9cbbddb6dcc13cad4ea2439f7e5e11ebcb7019ad Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Fri, 28 Oct 2016 09:11:57 +0200 Subject: [PATCH 030/132] Add support for `quote_field_suffix` to `simple_query_string`. (#21060) Closes #18641 --- .../index/query/SimpleQueryParser.java | 42 +++- .../index/query/SimpleQueryStringBuilder.java | 33 ++- .../java/org/elasticsearch/VersionTests.java | 2 + .../index/query/SimpleQueryParserTests.java | 74 ++++++ .../query/SimpleQueryStringBuilderTests.java | 8 + docs/reference/how-to.asciidoc | 2 + docs/reference/how-to/recipes.asciidoc | 225 ++++++++++++++++++ .../query-dsl/query-string-query.asciidoc | 5 + .../simple-query-string-query.asciidoc | 5 + 9 files changed, 384 insertions(+), 12 deletions(-) create mode 100644 docs/reference/how-to/recipes.asciidoc diff --git a/core/src/main/java/org/elasticsearch/index/query/SimpleQueryParser.java b/core/src/main/java/org/elasticsearch/index/query/SimpleQueryParser.java index 151e924ad16..e763b78c7f8 100644 --- a/core/src/main/java/org/elasticsearch/index/query/SimpleQueryParser.java +++ b/core/src/main/java/org/elasticsearch/index/query/SimpleQueryParser.java @@ -120,9 +120,18 @@ public class SimpleQueryParser extends org.apache.lucene.queryparser.simple.Simp bq.setDisableCoord(true); for (Map.Entry entry : weights.entrySet()) { try { - Query q = createPhraseQuery(entry.getKey(), text, slop); + String field = entry.getKey(); + if (settings.quoteFieldSuffix() != null) { + String quoteField = field + settings.quoteFieldSuffix(); + MappedFieldType quotedFieldType = context.fieldMapper(quoteField); + if (quotedFieldType != null) { + field = quoteField; + } + } + Float boost = entry.getValue(); + Query q = createPhraseQuery(field, text, slop); if (q != null) { - bq.add(wrapWithBoost(q, entry.getValue()), BooleanClause.Occur.SHOULD); + bq.add(wrapWithBoost(q, boost), BooleanClause.Occur.SHOULD); } } catch (RuntimeException e) { rethrowUnlessLenient(e); @@ -256,6 +265,8 @@ public class SimpleQueryParser extends org.apache.lucene.queryparser.simple.Simp private boolean lenient = SimpleQueryStringBuilder.DEFAULT_LENIENT; /** Specifies whether wildcards should be analyzed. */ private boolean analyzeWildcard = SimpleQueryStringBuilder.DEFAULT_ANALYZE_WILDCARD; + /** Specifies a suffix, if any, to apply to field names for phrase matching. */ + private String quoteFieldSuffix = null; /** * Generates default {@link Settings} object (uses ROOT locale, does @@ -264,13 +275,6 @@ public class SimpleQueryParser extends org.apache.lucene.queryparser.simple.Simp public Settings() { } - public Settings(Locale locale, Boolean lowercaseExpandedTerms, Boolean lenient, Boolean analyzeWildcard) { - this.locale = locale; - this.lowercaseExpandedTerms = lowercaseExpandedTerms; - this.lenient = lenient; - this.analyzeWildcard = analyzeWildcard; - } - /** Specifies the locale to use for parsing, Locale.ROOT by default. */ public void locale(Locale locale) { this.locale = (locale != null) ? locale : SimpleQueryStringBuilder.DEFAULT_LOCALE; @@ -314,12 +318,27 @@ public class SimpleQueryParser extends org.apache.lucene.queryparser.simple.Simp return analyzeWildcard; } + /** + * Set the suffix to append to field names for phrase matching. + */ + public void quoteFieldSuffix(String suffix) { + this.quoteFieldSuffix = suffix; + } + + /** + * Return the suffix to append for phrase matching, or {@code null} if + * no suffix should be appended. + */ + public String quoteFieldSuffix() { + return quoteFieldSuffix; + } + @Override public int hashCode() { // checking the return value of toLanguageTag() for locales only. // For further reasoning see // https://issues.apache.org/jira/browse/LUCENE-4021 - return Objects.hash(locale.toLanguageTag(), lowercaseExpandedTerms, lenient, analyzeWildcard); + return Objects.hash(locale.toLanguageTag(), lowercaseExpandedTerms, lenient, analyzeWildcard, quoteFieldSuffix); } @Override @@ -338,7 +357,8 @@ public class SimpleQueryParser extends org.apache.lucene.queryparser.simple.Simp return (Objects.equals(locale.toLanguageTag(), other.locale.toLanguageTag()) && Objects.equals(lowercaseExpandedTerms, other.lowercaseExpandedTerms) && Objects.equals(lenient, other.lenient) - && Objects.equals(analyzeWildcard, other.analyzeWildcard)); + && Objects.equals(analyzeWildcard, other.analyzeWildcard) + && Objects.equals(quoteFieldSuffix, other.quoteFieldSuffix)); } } } diff --git a/core/src/main/java/org/elasticsearch/index/query/SimpleQueryStringBuilder.java b/core/src/main/java/org/elasticsearch/index/query/SimpleQueryStringBuilder.java index fbe5964f2a0..15c0e0f4f15 100644 --- a/core/src/main/java/org/elasticsearch/index/query/SimpleQueryStringBuilder.java +++ b/core/src/main/java/org/elasticsearch/index/query/SimpleQueryStringBuilder.java @@ -22,6 +22,7 @@ package org.elasticsearch.index.query; import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.Query; +import org.elasticsearch.Version; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.Strings; @@ -94,6 +95,8 @@ public class SimpleQueryStringBuilder extends AbstractQueryBuilder assertUnknownVersion(Version.CURRENT)); assertUnknownVersion(AliasFilter.V_5_1_0); // once we released 5.1.0 and it's added to Version.java we need to remove this constant assertUnknownVersion(OsStats.V_5_1_0); // once we released 5.1.0 and it's added to Version.java we need to remove this constant + assertUnknownVersion(SimpleQueryStringBuilder.V_5_1_0_UNRELEASED); // once we released 5.0.0 and it's added to Version.java we need to remove this constant assertUnknownVersion(ShardValidateQueryRequestTests.V_5_0_0); } diff --git a/core/src/test/java/org/elasticsearch/index/query/SimpleQueryParserTests.java b/core/src/test/java/org/elasticsearch/index/query/SimpleQueryParserTests.java index 8511ad6d9c7..253bc95ffb8 100644 --- a/core/src/test/java/org/elasticsearch/index/query/SimpleQueryParserTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/SimpleQueryParserTests.java @@ -28,14 +28,47 @@ import org.apache.lucene.search.SynonymQuery; import org.apache.lucene.search.PrefixQuery; import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.BooleanClause; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.mapper.ContentPath; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.Mapper; +import org.elasticsearch.index.mapper.MockFieldMapper; +import org.elasticsearch.index.mapper.TextFieldMapper; +import org.elasticsearch.indices.query.IndicesQueriesRegistry; +import org.elasticsearch.search.SearchModule; import org.elasticsearch.test.ESTestCase; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.settings.Settings; +import java.util.Collections; import java.util.HashMap; import java.util.Map; +import static java.util.Collections.emptyList; import static org.hamcrest.Matchers.equalTo; public class SimpleQueryParserTests extends ESTestCase { + + private static IndicesQueriesRegistry indicesQueriesRegistry; + + /** + * setup for the whole base test class + */ + @BeforeClass + public static void init() { + SearchModule searchModule = new SearchModule(Settings.EMPTY, false, emptyList()); + indicesQueriesRegistry = searchModule.getQueryParserRegistry(); + } + + @AfterClass + public static void afterClass() throws Exception { + indicesQueriesRegistry = null; + } + private static class MockSimpleQueryParser extends SimpleQueryParser { public MockSimpleQueryParser(Analyzer analyzer, Map weights, int flags, Settings settings) { super(analyzer, weights, flags, settings, null); @@ -106,4 +139,45 @@ public class SimpleQueryParserTests extends ESTestCase { } } + public void testQuoteFieldSuffix() { + SimpleQueryParser.Settings sqpSettings = new SimpleQueryParser.Settings(); + sqpSettings.quoteFieldSuffix(".quote"); + + Settings indexSettings = Settings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetaData.SETTING_INDEX_UUID, "some_uuid") + .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) + .build(); + IndexMetaData indexState = IndexMetaData.builder("index").settings(indexSettings).build(); + IndexSettings settings = new IndexSettings(indexState, Settings.EMPTY); + QueryShardContext mockShardContext = new QueryShardContext(0, settings, null, null, null, null, null, indicesQueriesRegistry, + null, null, null, System::currentTimeMillis) { + @Override + public MappedFieldType fieldMapper(String name) { + return new MockFieldMapper.FakeFieldType(); + } + }; + + SimpleQueryParser parser = new SimpleQueryParser(new StandardAnalyzer(), + Collections.singletonMap("foo", 1f), -1, sqpSettings, mockShardContext); + assertEquals(new TermQuery(new Term("foo", "bar")), parser.parse("bar")); + assertEquals(new TermQuery(new Term("foo.quote", "bar")), parser.parse("\"bar\"")); + + // Now check what happens if foo.quote does not exist + mockShardContext = new QueryShardContext(0, settings, null, null, null, null, null, indicesQueriesRegistry, + null, null, null, System::currentTimeMillis) { + @Override + public MappedFieldType fieldMapper(String name) { + if (name.equals("foo.quote")) { + return null; + } + return new MockFieldMapper.FakeFieldType(); + } + }; + parser = new SimpleQueryParser(new StandardAnalyzer(), + Collections.singletonMap("foo", 1f), -1, sqpSettings, mockShardContext); + assertEquals(new TermQuery(new Term("foo", "bar")), parser.parse("bar")); + assertEquals(new TermQuery(new Term("foo", "bar")), parser.parse("\"bar\"")); + } } diff --git a/core/src/test/java/org/elasticsearch/index/query/SimpleQueryStringBuilderTests.java b/core/src/test/java/org/elasticsearch/index/query/SimpleQueryStringBuilderTests.java index 07f3f389c27..77257cdce6a 100644 --- a/core/src/test/java/org/elasticsearch/index/query/SimpleQueryStringBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/SimpleQueryStringBuilderTests.java @@ -19,6 +19,8 @@ package org.elasticsearch.index.query; +import com.carrotsearch.randomizedtesting.generators.RandomStrings; + import org.apache.lucene.index.Term; import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.BooleanQuery; @@ -27,6 +29,7 @@ import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.TermQuery; +import org.apache.lucene.util.TestUtil; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.test.AbstractQueryTestCase; @@ -71,6 +74,9 @@ public class SimpleQueryStringBuilderTests extends AbstractQueryTestCase flagSet = new HashSet<>(); int size = randomIntBetween(0, SimpleQueryStringFlag.values().length); @@ -334,6 +340,7 @@ public class SimpleQueryStringBuilderTests extends AbstractQueryTestCase> in order to have the same +content indexed in two different ways: + +[source,js] +-------------------------------------------------- +PUT index +{ + "settings": { + "analysis": { + "analyzer": { + "english_exact": { + "tokenizer": "standard", + "filter": [ + "lowercase" + ] + } + } + } + }, + "mappings": { + "type": { + "properties": { + "body": { + "type": "text", + "analyzer": "english", + "fields": { + "exact": { + "type": "text", + "analyzer": "english_exact" + } + } + } + } + } + } +} + +PUT index/type/1 +{ + "body": "Ski resort" +} + +PUT index/type/2 +{ + "body": "A pair of skis" +} + +POST index/_refresh +-------------------------------------------------- +// CONSOLE + +With such a setup, searching for `ski` on `body` would return both documents: + +[source,js] +-------------------------------------------------- +GET index/_search +{ + "query": { + "simple_query_string": { + "fields": [ "body" ], + "query": "ski" + } + } +} +-------------------------------------------------- +// CONSOLE +// TEST[continued] + +[source,js] +-------------------------------------------------- +{ + "took": 2, + "timed_out": false, + "_shards": { + "total": 5, + "successful": 5, + "failed": 0 + }, + "hits": { + "total": 2, + "max_score": 0.25811607, + "hits": [ + { + "_index": "index", + "_type": "type", + "_id": "2", + "_score": 0.25811607, + "_source": { + "body": "A pair of skis" + } + }, + { + "_index": "index", + "_type": "type", + "_id": "1", + "_score": 0.25811607, + "_source": { + "body": "Ski resort" + } + } + ] + } +} +-------------------------------------------------- +// TESTRESPONSE[s/"took": 2,/"took": "$body.took",/] + +On the other hand, searching for `ski` on `body.exact` would only return +document `1` since the analysis chain of `body.exact` does not perform +stemming. + +[source,js] +-------------------------------------------------- +GET index/_search +{ + "query": { + "simple_query_string": { + "fields": [ "body.exact" ], + "query": "ski" + } + } +} +-------------------------------------------------- +// CONSOLE +// TEST[continued] + +[source,js] +-------------------------------------------------- +{ + "took": 1, + "timed_out": false, + "_shards": { + "total": 5, + "successful": 5, + "failed": 0 + }, + "hits": { + "total": 1, + "max_score": 0.25811607, + "hits": [ + { + "_index": "index", + "_type": "type", + "_id": "1", + "_score": 0.25811607, + "_source": { + "body": "Ski resort" + } + } + ] + } +} +-------------------------------------------------- +// TESTRESPONSE[s/"took": 1,/"took": "$body.took",/] + +This is not something that is easy to expose to end users, as we would need to +have a way to figure out whether they are looking for an exact match or not and +redirect to the appropriate field accordingly. Also what to do if only parts of +the query need to be matched exactly while other parts should still take +stemming into account? + +Fortunately, the `query_string` and `simple_query_string` queries have a feature +that allows to solve exactly this problem: `quote_field_suffix`. It allows to +tell Elasticsearch that words that appear in between quotes should be redirected +to a different field, see below: + +[source,js] +-------------------------------------------------- +GET index/_search +{ + "query": { + "simple_query_string": { + "fields": [ "body" ], + "quote_field_suffix": ".exact", + "query": "\"ski\"" + } + } +} +-------------------------------------------------- +// CONSOLE +// TEST[continued] + +[source,js] +-------------------------------------------------- +{ + "took": 2, + "timed_out": false, + "_shards": { + "total": 5, + "successful": 5, + "failed": 0 + }, + "hits": { + "total": 1, + "max_score": 0.25811607, + "hits": [ + { + "_index": "index", + "_type": "type", + "_id": "1", + "_score": 0.25811607, + "_source": { + "body": "Ski resort" + } + } + ] + } +} +-------------------------------------------------- +// TESTRESPONSE[s/"took": 2,/"took": "$body.took",/] + +In that case, since `ski` was in-between quotes, it was searched on the +`body.exact` field due to the `quote_field_suffix` parameter, so only document +`1` matched. This allows users to mix exact search with stemmed search as they +like. + diff --git a/docs/reference/query-dsl/query-string-query.asciidoc b/docs/reference/query-dsl/query-string-query.asciidoc index 60477d6e28a..fe113b0a289 100644 --- a/docs/reference/query-dsl/query-string-query.asciidoc +++ b/docs/reference/query-dsl/query-string-query.asciidoc @@ -85,6 +85,11 @@ Defaults to `ROOT`. |`time_zone` | Time Zone to be applied to any range query related to dates. See also http://www.joda.org/joda-time/apidocs/org/joda/time/DateTimeZone.html[JODA timezone]. + +|`quote_field_suffix` | A suffix to append to fields for quoted parts of +the query string. This allows to use a field that has a different analysis chain +for exact matching. Look <> for a +comprehensive example. |======================================================================= When a multi term query is being generated, one can control how it gets diff --git a/docs/reference/query-dsl/simple-query-string-query.asciidoc b/docs/reference/query-dsl/simple-query-string-query.asciidoc index 796f2517fea..59e3f727fe0 100644 --- a/docs/reference/query-dsl/simple-query-string-query.asciidoc +++ b/docs/reference/query-dsl/simple-query-string-query.asciidoc @@ -63,6 +63,11 @@ Defaults to `ROOT`. document to be returned. See the <> documentation for the full list of options. + +|`quote_field_suffix` | A suffix to append to fields for quoted parts of +the query string. This allows to use a field that has a different analysis chain +for exact matching. Look <> for a +comprehensive example. |======================================================================= [float] From 43dbf9c7b6baaa1f9a4cc01a87d40ef50bd95d13 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Fri, 28 Oct 2016 12:18:47 +0200 Subject: [PATCH 031/132] Use all available hosts in REST tests and allow for real master election (#21161) Today we only use a single node to send requests to when we run REST tests. In some cases we have more than one node (ie. in the BWC case) where we should send requests to all nodes in a round-robin fashion. This change passes all available node endpoints to the rest test. Additionally, this change adds the setting of `discovery.zen.minimum_master_nodes` to the cluster formation forcing the nodes to wait for all other nodes until the cluster is formed. This allows for a more realistic master election and allows all master eligable nodes to become master while before always the first node in the cluster became the master. This also adds logging to each test run to log the master nodes version and the minimum node version in the cluster to help debugging BWC test failures. --- .../gradle/test/ClusterFormationTasks.groovy | 6 +++ .../gradle/test/RestIntegTestTask.groovy | 4 +- .../test/rest/yaml/ClientYamlTestClient.java | 46 +++++++++++-------- 3 files changed, 35 insertions(+), 21 deletions(-) diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy index ce47ac6c541..56d074bee2f 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy @@ -255,6 +255,12 @@ class ClusterFormationTasks { 'node.attr.testattr' : 'test', 'repositories.url.allowed_urls': 'http://snapshot.test*' ] + // we set min master nodes to the total number of nodes in the cluster and + // basically skip initial state recovery to allow the cluster to form using a realistic master election + // this means all nodes must be up, join the seed node and do a master election. This will also allow new and + // old nodes in the BWC case to become the master + esConfig['discovery.zen.minimum_master_nodes'] = node.config.numNodes + esConfig['discovery.initial_state_timeout'] = '0s' // don't wait for state.. just start up quickly esConfig['node.max_local_storage_nodes'] = node.config.numNodes esConfig['http.port'] = node.config.httpPort esConfig['transport.tcp.port'] = node.config.transportPort diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/RestIntegTestTask.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/RestIntegTestTask.groovy index d50937408e7..51bccb4fe75 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/RestIntegTestTask.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/RestIntegTestTask.groovy @@ -55,7 +55,9 @@ public class RestIntegTestTask extends RandomizedTestingTask { parallelism = '1' include('**/*IT.class') systemProperty('tests.rest.load_packaged', 'false') - systemProperty('tests.rest.cluster', "${-> nodes[0].httpUri()}") + // we pass all nodes to the rest cluster to allow the clients to round-robin between them + // this is more realistic than just talking to a single node + systemProperty('tests.rest.cluster', "${-> nodes.collect{it.httpUri()}.join(",")}") systemProperty('tests.config.dir', "${-> nodes[0].confDir}") // TODO: our "client" qa tests currently use the rest-test plugin. instead they should have their own plugin // that sets up the test cluster and passes this transport uri instead of http uri. Until then, we pass diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestClient.java b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestClient.java index 27756b1d852..8a427a6d41f 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestClient.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestClient.java @@ -31,6 +31,7 @@ import org.elasticsearch.client.Response; import org.elasticsearch.client.ResponseException; import org.elasticsearch.client.RestClient; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.test.rest.yaml.restspec.ClientYamlSuiteRestApi; @@ -66,34 +67,39 @@ public class ClientYamlTestClient { assert hosts.size() > 0; this.restSpec = restSpec; this.restClient = restClient; - this.esVersion = readAndCheckVersion(hosts); + Tuple versionTuple = readMasterAndMinNodeVersion(); + this.esVersion = versionTuple.v1(); + Version masterVersion = versionTuple.v2(); + // this will be logged in each test such that if something fails we get it in the logs for each test + logger.info("initializing client, minimum es version: [{}] master version: [{}] hosts: {}", esVersion, masterVersion, hosts); } - private Version readAndCheckVersion(List hosts) throws IOException { - ClientYamlSuiteRestApi restApi = restApi("info"); - assert restApi.getPaths().size() == 1; - assert restApi.getMethods().size() == 1; - - String version = null; - for (HttpHost ignored : hosts) { - //we don't really use the urls here, we rely on the client doing round-robin to touch all the nodes in the cluster - String method = restApi.getMethods().get(0); - String endpoint = restApi.getPaths().get(0); - Response response = restClient.performRequest(method, endpoint); - ClientYamlTestResponse restTestResponse = new ClientYamlTestResponse(response); - Object latestVersion = restTestResponse.evaluate("version.number"); - if (latestVersion == null) { - throw new RuntimeException("elasticsearch version not found in the response"); + private Tuple readMasterAndMinNodeVersion() throws IOException { + // we simply go to the _cat/nodes API and parse all versions in the cluster + Response response = restClient.performRequest("GET", "/_cat/nodes", Collections.singletonMap("h", "version,master")); + ClientYamlTestResponse restTestResponse = new ClientYamlTestResponse(response); + String nodesCatResponse = restTestResponse.getBodyAsString(); + String[] split = nodesCatResponse.split("\n"); + Version version = null; + Version masterVersion = null; + for (String perNode : split) { + final String[] versionAndMaster = perNode.split(" "); + assert versionAndMaster.length == 2 : "invalid line: " + perNode + " length: " + versionAndMaster.length; + final Version currentVersion = Version.fromString(versionAndMaster[0]); + final boolean master = versionAndMaster[1].trim().equals("*"); + if (master) { + assert masterVersion == null; + masterVersion = currentVersion; } if (version == null) { - version = latestVersion.toString(); + version = currentVersion; } else { - if (!latestVersion.equals(version)) { - throw new IllegalArgumentException("provided nodes addresses run different elasticsearch versions"); + if (version.onOrAfter(currentVersion)) { + version = currentVersion; } } } - return Version.fromString(version); + return new Tuple<>(version, masterVersion); } public Version getEsVersion() { From b3cc54cf0d8dbec37b8bd1587475e530f085437f Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Fri, 28 Oct 2016 14:47:15 +0200 Subject: [PATCH 032/132] Upgrade to lucene-6.3.0-snapshot-ed102d6 (#21150) Lucene 6.3 is expected to be released in the next weeks so it'd be good to give it some integration testing. I had to upgrade randomized-testing too so that both Lucene and Elasticsearch are on the same version. --- .../gradle/doc/DocsTestPlugin.groovy | 2 +- buildSrc/version.properties | 4 +-- .../client/RequestLoggerTests.java | 10 +++--- .../client/RestClientMultipleHostsTests.java | 14 ++++---- .../sniff/ElasticsearchHostsSnifferTests.java | 16 +++++----- .../client/sniff/SnifferBuilderTests.java | 12 +++---- .../main/java/org/elasticsearch/Version.java | 2 +- .../xcontent/support/XContentMapValues.java | 6 ++-- .../elasticsearch/bootstrap/security.policy | 4 +-- .../bootstrap/test-framework.policy | 4 +-- .../node/tasks/CancellableTasksTests.java | 6 ++-- .../transport/FailAndRetryMockTransport.java | 6 +--- .../index/query/RandomQueryBuilder.java | 8 ++--- .../blobstore/SlicedInputStreamTests.java | 4 +-- .../test/geo/RandomShapeGenerator.java | 14 ++++---- .../lucene-analyzers-common-6.2.0.jar.sha1 | 1 - ...ers-common-6.3.0-snapshot-ed102d6.jar.sha1 | 1 + .../lucene-backward-codecs-6.2.0.jar.sha1 | 1 - ...ard-codecs-6.3.0-snapshot-ed102d6.jar.sha1 | 1 + .../licenses/lucene-core-6.2.0.jar.sha1 | 1 - ...ucene-core-6.3.0-snapshot-ed102d6.jar.sha1 | 1 + .../licenses/lucene-grouping-6.2.0.jar.sha1 | 1 - ...e-grouping-6.3.0-snapshot-ed102d6.jar.sha1 | 1 + .../lucene-highlighter-6.2.0.jar.sha1 | 1 - ...ighlighter-6.3.0-snapshot-ed102d6.jar.sha1 | 1 + .../licenses/lucene-join-6.2.0.jar.sha1 | 1 - ...ucene-join-6.3.0-snapshot-ed102d6.jar.sha1 | 1 + .../licenses/lucene-memory-6.2.0.jar.sha1 | 1 - ...ene-memory-6.3.0-snapshot-ed102d6.jar.sha1 | 1 + .../licenses/lucene-misc-6.2.0.jar.sha1 | 1 - ...ucene-misc-6.3.0-snapshot-ed102d6.jar.sha1 | 1 + .../licenses/lucene-queries-6.2.0.jar.sha1 | 1 - ...ne-queries-6.3.0-snapshot-ed102d6.jar.sha1 | 1 + .../lucene-queryparser-6.2.0.jar.sha1 | 1 - ...ueryparser-6.3.0-snapshot-ed102d6.jar.sha1 | 1 + .../licenses/lucene-sandbox-6.2.0.jar.sha1 | 1 - ...ne-sandbox-6.3.0-snapshot-ed102d6.jar.sha1 | 1 + .../licenses/lucene-spatial-6.2.0.jar.sha1 | 1 - ...ne-spatial-6.3.0-snapshot-ed102d6.jar.sha1 | 1 + .../lucene-spatial-extras-6.2.0.jar.sha1 | 1 - ...ial-extras-6.3.0-snapshot-ed102d6.jar.sha1 | 1 + .../licenses/lucene-spatial3d-6.2.0.jar.sha1 | 1 - ...-spatial3d-6.3.0-snapshot-ed102d6.jar.sha1 | 1 + .../licenses/lucene-suggest-6.2.0.jar.sha1 | 1 - ...ne-suggest-6.3.0-snapshot-ed102d6.jar.sha1 | 1 + docs/plugins/index.asciidoc | 2 +- docs/reference/index.asciidoc | 2 +- .../lucene-expressions-6.2.0.jar.sha1 | 1 - ...xpressions-6.3.0-snapshot-ed102d6.jar.sha1 | 1 + .../lucene-analyzers-icu-6.2.0.jar.sha1 | 1 - ...lyzers-icu-6.3.0-snapshot-ed102d6.jar.sha1 | 1 + .../lucene-analyzers-kuromoji-6.2.0.jar.sha1 | 1 - ...s-kuromoji-6.3.0-snapshot-ed102d6.jar.sha1 | 1 + .../lucene-analyzers-phonetic-6.2.0.jar.sha1 | 1 - ...s-phonetic-6.3.0-snapshot-ed102d6.jar.sha1 | 1 + .../lucene-analyzers-smartcn-6.2.0.jar.sha1 | 1 - ...rs-smartcn-6.3.0-snapshot-ed102d6.jar.sha1 | 1 + .../lucene-analyzers-stempel-6.2.0.jar.sha1 | 1 - ...rs-stempel-6.3.0-snapshot-ed102d6.jar.sha1 | 1 + .../ingest/RandomDocumentPicks.java | 18 +++++------ .../elasticsearch/test/BackgroundIndexer.java | 6 ++-- .../elasticsearch/test/ESIntegTestCase.java | 14 ++++---- .../org/elasticsearch/test/ESTestCase.java | 4 +-- .../test/InternalTestCluster.java | 32 +++++++++---------- 64 files changed, 114 insertions(+), 118 deletions(-) delete mode 100644 distribution/licenses/lucene-analyzers-common-6.2.0.jar.sha1 create mode 100644 distribution/licenses/lucene-analyzers-common-6.3.0-snapshot-ed102d6.jar.sha1 delete mode 100644 distribution/licenses/lucene-backward-codecs-6.2.0.jar.sha1 create mode 100644 distribution/licenses/lucene-backward-codecs-6.3.0-snapshot-ed102d6.jar.sha1 delete mode 100644 distribution/licenses/lucene-core-6.2.0.jar.sha1 create mode 100644 distribution/licenses/lucene-core-6.3.0-snapshot-ed102d6.jar.sha1 delete mode 100644 distribution/licenses/lucene-grouping-6.2.0.jar.sha1 create mode 100644 distribution/licenses/lucene-grouping-6.3.0-snapshot-ed102d6.jar.sha1 delete mode 100644 distribution/licenses/lucene-highlighter-6.2.0.jar.sha1 create mode 100644 distribution/licenses/lucene-highlighter-6.3.0-snapshot-ed102d6.jar.sha1 delete mode 100644 distribution/licenses/lucene-join-6.2.0.jar.sha1 create mode 100644 distribution/licenses/lucene-join-6.3.0-snapshot-ed102d6.jar.sha1 delete mode 100644 distribution/licenses/lucene-memory-6.2.0.jar.sha1 create mode 100644 distribution/licenses/lucene-memory-6.3.0-snapshot-ed102d6.jar.sha1 delete mode 100644 distribution/licenses/lucene-misc-6.2.0.jar.sha1 create mode 100644 distribution/licenses/lucene-misc-6.3.0-snapshot-ed102d6.jar.sha1 delete mode 100644 distribution/licenses/lucene-queries-6.2.0.jar.sha1 create mode 100644 distribution/licenses/lucene-queries-6.3.0-snapshot-ed102d6.jar.sha1 delete mode 100644 distribution/licenses/lucene-queryparser-6.2.0.jar.sha1 create mode 100644 distribution/licenses/lucene-queryparser-6.3.0-snapshot-ed102d6.jar.sha1 delete mode 100644 distribution/licenses/lucene-sandbox-6.2.0.jar.sha1 create mode 100644 distribution/licenses/lucene-sandbox-6.3.0-snapshot-ed102d6.jar.sha1 delete mode 100644 distribution/licenses/lucene-spatial-6.2.0.jar.sha1 create mode 100644 distribution/licenses/lucene-spatial-6.3.0-snapshot-ed102d6.jar.sha1 delete mode 100644 distribution/licenses/lucene-spatial-extras-6.2.0.jar.sha1 create mode 100644 distribution/licenses/lucene-spatial-extras-6.3.0-snapshot-ed102d6.jar.sha1 delete mode 100644 distribution/licenses/lucene-spatial3d-6.2.0.jar.sha1 create mode 100644 distribution/licenses/lucene-spatial3d-6.3.0-snapshot-ed102d6.jar.sha1 delete mode 100644 distribution/licenses/lucene-suggest-6.2.0.jar.sha1 create mode 100644 distribution/licenses/lucene-suggest-6.3.0-snapshot-ed102d6.jar.sha1 delete mode 100644 modules/lang-expression/licenses/lucene-expressions-6.2.0.jar.sha1 create mode 100644 modules/lang-expression/licenses/lucene-expressions-6.3.0-snapshot-ed102d6.jar.sha1 delete mode 100644 plugins/analysis-icu/licenses/lucene-analyzers-icu-6.2.0.jar.sha1 create mode 100644 plugins/analysis-icu/licenses/lucene-analyzers-icu-6.3.0-snapshot-ed102d6.jar.sha1 delete mode 100644 plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-6.2.0.jar.sha1 create mode 100644 plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-6.3.0-snapshot-ed102d6.jar.sha1 delete mode 100644 plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-6.2.0.jar.sha1 create mode 100644 plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-6.3.0-snapshot-ed102d6.jar.sha1 delete mode 100644 plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-6.2.0.jar.sha1 create mode 100644 plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-6.3.0-snapshot-ed102d6.jar.sha1 delete mode 100644 plugins/analysis-stempel/licenses/lucene-analyzers-stempel-6.2.0.jar.sha1 create mode 100644 plugins/analysis-stempel/licenses/lucene-analyzers-stempel-6.3.0-snapshot-ed102d6.jar.sha1 diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/doc/DocsTestPlugin.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/doc/DocsTestPlugin.groovy index 11bdbd19525..a46a7bda374 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/doc/DocsTestPlugin.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/doc/DocsTestPlugin.groovy @@ -38,7 +38,7 @@ public class DocsTestPlugin extends RestTestPlugin { * the last released version for docs. */ '\\{version\\}': VersionProperties.elasticsearch.replace('-SNAPSHOT', ''), - '\\{lucene_version\\}' : VersionProperties.lucene, + '\\{lucene_version\\}' : VersionProperties.lucene.replaceAll('-snapshot-\\w+$', ''), ] Task listSnippets = project.tasks.create('listSnippets', SnippetsTask) listSnippets.group 'Docs' diff --git a/buildSrc/version.properties b/buildSrc/version.properties index c630ca3ebb8..7677e8e24bb 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -1,5 +1,5 @@ elasticsearch = 6.0.0-alpha1 -lucene = 6.2.0 +lucene = 6.3.0-snapshot-ed102d6 # optional dependencies spatial4j = 0.6 @@ -11,7 +11,7 @@ slf4j = 1.6.2 jna = 4.2.2 # test dependencies -randomizedrunner = 2.3.2 +randomizedrunner = 2.4.0 junit = 4.11 httpclient = 4.5.2 httpcore = 4.4.5 diff --git a/client/rest/src/test/java/org/elasticsearch/client/RequestLoggerTests.java b/client/rest/src/test/java/org/elasticsearch/client/RequestLoggerTests.java index 789f2bf6f6d..17c2a158ea8 100644 --- a/client/rest/src/test/java/org/elasticsearch/client/RequestLoggerTests.java +++ b/client/rest/src/test/java/org/elasticsearch/client/RequestLoggerTests.java @@ -19,7 +19,7 @@ package org.elasticsearch.client; -import com.carrotsearch.randomizedtesting.generators.RandomInts; +import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import org.apache.http.HttpEntity; import org.apache.http.HttpEntityEnclosingRequest; import org.apache.http.HttpHost; @@ -62,7 +62,7 @@ public class RequestLoggerTests extends RestClientTestCase { } HttpRequestBase request; - int requestType = RandomInts.randomIntBetween(getRandom(), 0, 7); + int requestType = RandomNumbers.randomIntBetween(getRandom(), 0, 7); switch(requestType) { case 0: request = new HttpGetWithEntity(uri); @@ -99,7 +99,7 @@ public class RequestLoggerTests extends RestClientTestCase { expected += " -d '" + requestBody + "'"; HttpEntityEnclosingRequest enclosingRequest = (HttpEntityEnclosingRequest) request; HttpEntity entity; - switch(RandomInts.randomIntBetween(getRandom(), 0, 3)) { + switch(RandomNumbers.randomIntBetween(getRandom(), 0, 3)) { case 0: entity = new StringEntity(requestBody, StandardCharsets.UTF_8); break; @@ -128,12 +128,12 @@ public class RequestLoggerTests extends RestClientTestCase { public void testTraceResponse() throws IOException { ProtocolVersion protocolVersion = new ProtocolVersion("HTTP", 1, 1); - int statusCode = RandomInts.randomIntBetween(getRandom(), 200, 599); + int statusCode = RandomNumbers.randomIntBetween(getRandom(), 200, 599); String reasonPhrase = "REASON"; BasicStatusLine statusLine = new BasicStatusLine(protocolVersion, statusCode, reasonPhrase); String expected = "# " + statusLine.toString(); BasicHttpResponse httpResponse = new BasicHttpResponse(statusLine); - int numHeaders = RandomInts.randomIntBetween(getRandom(), 0, 3); + int numHeaders = RandomNumbers.randomIntBetween(getRandom(), 0, 3); for (int i = 0; i < numHeaders; i++) { httpResponse.setHeader("header" + i, "value"); expected += "\n# header" + i + ": value"; diff --git a/client/rest/src/test/java/org/elasticsearch/client/RestClientMultipleHostsTests.java b/client/rest/src/test/java/org/elasticsearch/client/RestClientMultipleHostsTests.java index 049a216936f..90ee4431009 100644 --- a/client/rest/src/test/java/org/elasticsearch/client/RestClientMultipleHostsTests.java +++ b/client/rest/src/test/java/org/elasticsearch/client/RestClientMultipleHostsTests.java @@ -19,7 +19,7 @@ package org.elasticsearch.client; -import com.carrotsearch.randomizedtesting.generators.RandomInts; +import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import org.apache.http.Header; import org.apache.http.HttpHost; import org.apache.http.HttpResponse; @@ -95,7 +95,7 @@ public class RestClientMultipleHostsTests extends RestClientTestCase { return null; } }); - int numHosts = RandomInts.randomIntBetween(getRandom(), 2, 5); + int numHosts = RandomNumbers.randomIntBetween(getRandom(), 2, 5); httpHosts = new HttpHost[numHosts]; for (int i = 0; i < numHosts; i++) { httpHosts[i] = new HttpHost("localhost", 9200 + i); @@ -105,7 +105,7 @@ public class RestClientMultipleHostsTests extends RestClientTestCase { } public void testRoundRobinOkStatusCodes() throws IOException { - int numIters = RandomInts.randomIntBetween(getRandom(), 1, 5); + int numIters = RandomNumbers.randomIntBetween(getRandom(), 1, 5); for (int i = 0; i < numIters; i++) { Set hostsSet = new HashSet<>(); Collections.addAll(hostsSet, httpHosts); @@ -121,7 +121,7 @@ public class RestClientMultipleHostsTests extends RestClientTestCase { } public void testRoundRobinNoRetryErrors() throws IOException { - int numIters = RandomInts.randomIntBetween(getRandom(), 1, 5); + int numIters = RandomNumbers.randomIntBetween(getRandom(), 1, 5); for (int i = 0; i < numIters; i++) { Set hostsSet = new HashSet<>(); Collections.addAll(hostsSet, httpHosts); @@ -198,7 +198,7 @@ public class RestClientMultipleHostsTests extends RestClientTestCase { assertEquals("every host should have been used but some weren't: " + hostsSet, 0, hostsSet.size()); } - int numIters = RandomInts.randomIntBetween(getRandom(), 2, 5); + int numIters = RandomNumbers.randomIntBetween(getRandom(), 2, 5); for (int i = 1; i <= numIters; i++) { //check that one different host is resurrected at each new attempt Set hostsSet = new HashSet<>(); @@ -228,7 +228,7 @@ public class RestClientMultipleHostsTests extends RestClientTestCase { if (getRandom().nextBoolean()) { //mark one host back alive through a successful request and check that all requests after that are sent to it HttpHost selectedHost = null; - int iters = RandomInts.randomIntBetween(getRandom(), 2, 10); + int iters = RandomNumbers.randomIntBetween(getRandom(), 2, 10); for (int y = 0; y < iters; y++) { int statusCode = randomErrorNoRetryStatusCode(getRandom()); Response response; @@ -269,7 +269,7 @@ public class RestClientMultipleHostsTests extends RestClientTestCase { } private static String randomErrorRetryEndpoint() { - switch(RandomInts.randomIntBetween(getRandom(), 0, 3)) { + switch(RandomNumbers.randomIntBetween(getRandom(), 0, 3)) { case 0: return "/" + randomErrorRetryStatusCode(getRandom()); case 1: diff --git a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/ElasticsearchHostsSnifferTests.java b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/ElasticsearchHostsSnifferTests.java index a926cabb87d..aeb0620134b 100644 --- a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/ElasticsearchHostsSnifferTests.java +++ b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/ElasticsearchHostsSnifferTests.java @@ -19,7 +19,7 @@ package org.elasticsearch.client.sniff; -import com.carrotsearch.randomizedtesting.generators.RandomInts; +import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import com.carrotsearch.randomizedtesting.generators.RandomPicks; import com.carrotsearch.randomizedtesting.generators.RandomStrings; import com.fasterxml.jackson.core.JsonFactory; @@ -69,7 +69,7 @@ public class ElasticsearchHostsSnifferTests extends RestClientTestCase { @Before public void startHttpServer() throws IOException { - this.sniffRequestTimeout = RandomInts.randomIntBetween(getRandom(), 1000, 10000); + this.sniffRequestTimeout = RandomNumbers.randomIntBetween(getRandom(), 1000, 10000); this.scheme = RandomPicks.randomFrom(getRandom(), ElasticsearchHostsSniffer.Scheme.values()); if (rarely()) { this.sniffResponse = SniffResponse.buildFailure(); @@ -101,7 +101,7 @@ public class ElasticsearchHostsSnifferTests extends RestClientTestCase { assertEquals(e.getMessage(), "scheme cannot be null"); } try { - new ElasticsearchHostsSniffer(restClient, RandomInts.randomIntBetween(getRandom(), Integer.MIN_VALUE, 0), + new ElasticsearchHostsSniffer(restClient, RandomNumbers.randomIntBetween(getRandom(), Integer.MIN_VALUE, 0), ElasticsearchHostsSniffer.Scheme.HTTP); fail("should have failed"); } catch (IllegalArgumentException e) { @@ -175,7 +175,7 @@ public class ElasticsearchHostsSnifferTests extends RestClientTestCase { } private static SniffResponse buildSniffResponse(ElasticsearchHostsSniffer.Scheme scheme) throws IOException { - int numNodes = RandomInts.randomIntBetween(getRandom(), 1, 5); + int numNodes = RandomNumbers.randomIntBetween(getRandom(), 1, 5); List hosts = new ArrayList<>(numNodes); JsonFactory jsonFactory = new JsonFactory(); StringWriter writer = new StringWriter(); @@ -205,7 +205,7 @@ public class ElasticsearchHostsSnifferTests extends RestClientTestCase { boolean isHttpEnabled = rarely() == false; if (isHttpEnabled) { String host = "host" + i; - int port = RandomInts.randomIntBetween(getRandom(), 9200, 9299); + int port = RandomNumbers.randomIntBetween(getRandom(), 9200, 9299); HttpHost httpHost = new HttpHost(host, port, scheme.toString()); hosts.add(httpHost); generator.writeObjectFieldStart("http"); @@ -228,7 +228,7 @@ public class ElasticsearchHostsSnifferTests extends RestClientTestCase { } if (getRandom().nextBoolean()) { String[] roles = {"master", "data", "ingest"}; - int numRoles = RandomInts.randomIntBetween(getRandom(), 0, 3); + int numRoles = RandomNumbers.randomIntBetween(getRandom(), 0, 3); Set nodeRoles = new HashSet<>(numRoles); for (int j = 0; j < numRoles; j++) { String role; @@ -242,7 +242,7 @@ public class ElasticsearchHostsSnifferTests extends RestClientTestCase { } generator.writeEndArray(); } - int numAttributes = RandomInts.randomIntBetween(getRandom(), 0, 3); + int numAttributes = RandomNumbers.randomIntBetween(getRandom(), 0, 3); Map attributes = new HashMap<>(numAttributes); for (int j = 0; j < numAttributes; j++) { attributes.put("attr" + j, "value" + j); @@ -291,6 +291,6 @@ public class ElasticsearchHostsSnifferTests extends RestClientTestCase { } private static int randomErrorResponseCode() { - return RandomInts.randomIntBetween(getRandom(), 400, 599); + return RandomNumbers.randomIntBetween(getRandom(), 400, 599); } } diff --git a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferBuilderTests.java b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferBuilderTests.java index b0c387d733a..9a7359e9c72 100644 --- a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferBuilderTests.java +++ b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferBuilderTests.java @@ -19,7 +19,7 @@ package org.elasticsearch.client.sniff; -import com.carrotsearch.randomizedtesting.generators.RandomInts; +import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import org.apache.http.HttpHost; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestClientTestCase; @@ -31,7 +31,7 @@ import static org.junit.Assert.fail; public class SnifferBuilderTests extends RestClientTestCase { public void testBuild() throws Exception { - int numNodes = RandomInts.randomIntBetween(getRandom(), 1, 5); + int numNodes = RandomNumbers.randomIntBetween(getRandom(), 1, 5); HttpHost[] hosts = new HttpHost[numNodes]; for (int i = 0; i < numNodes; i++) { hosts[i] = new HttpHost("localhost", 9200 + i); @@ -46,14 +46,14 @@ public class SnifferBuilderTests extends RestClientTestCase { } try { - Sniffer.builder(client).setSniffIntervalMillis(RandomInts.randomIntBetween(getRandom(), Integer.MIN_VALUE, 0)); + Sniffer.builder(client).setSniffIntervalMillis(RandomNumbers.randomIntBetween(getRandom(), Integer.MIN_VALUE, 0)); fail("should have failed"); } catch(IllegalArgumentException e) { assertEquals("sniffIntervalMillis must be greater than 0", e.getMessage()); } try { - Sniffer.builder(client).setSniffAfterFailureDelayMillis(RandomInts.randomIntBetween(getRandom(), Integer.MIN_VALUE, 0)); + Sniffer.builder(client).setSniffAfterFailureDelayMillis(RandomNumbers.randomIntBetween(getRandom(), Integer.MIN_VALUE, 0)); fail("should have failed"); } catch(IllegalArgumentException e) { assertEquals("sniffAfterFailureDelayMillis must be greater than 0", e.getMessage()); @@ -74,10 +74,10 @@ public class SnifferBuilderTests extends RestClientTestCase { SnifferBuilder builder = Sniffer.builder(client); if (getRandom().nextBoolean()) { - builder.setSniffIntervalMillis(RandomInts.randomIntBetween(getRandom(), 1, Integer.MAX_VALUE)); + builder.setSniffIntervalMillis(RandomNumbers.randomIntBetween(getRandom(), 1, Integer.MAX_VALUE)); } if (getRandom().nextBoolean()) { - builder.setSniffAfterFailureDelayMillis(RandomInts.randomIntBetween(getRandom(), 1, Integer.MAX_VALUE)); + builder.setSniffAfterFailureDelayMillis(RandomNumbers.randomIntBetween(getRandom(), 1, Integer.MAX_VALUE)); } if (getRandom().nextBoolean()) { builder.setHostsSniffer(new MockHostsSniffer()); diff --git a/core/src/main/java/org/elasticsearch/Version.java b/core/src/main/java/org/elasticsearch/Version.java index 0d8c3b72672..219eaa920d5 100644 --- a/core/src/main/java/org/elasticsearch/Version.java +++ b/core/src/main/java/org/elasticsearch/Version.java @@ -90,7 +90,7 @@ public class Version { public static final int V_5_0_0_rc1_ID = 5000051; public static final Version V_5_0_0_rc1 = new Version(V_5_0_0_rc1_ID, org.apache.lucene.util.Version.LUCENE_6_2_0); public static final int V_6_0_0_alpha1_ID = 6000001; - public static final Version V_6_0_0_alpha1 = new Version(V_6_0_0_alpha1_ID, org.apache.lucene.util.Version.LUCENE_6_2_0); + public static final Version V_6_0_0_alpha1 = new Version(V_6_0_0_alpha1_ID, org.apache.lucene.util.Version.LUCENE_6_3_0); public static final Version CURRENT = V_6_0_0_alpha1; /* NOTE: don't add unreleased version to this list except of the version assigned to CURRENT. diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/support/XContentMapValues.java b/core/src/main/java/org/elasticsearch/common/xcontent/support/XContentMapValues.java index c745c7d6c0c..a94bf63e270 100644 --- a/core/src/main/java/org/elasticsearch/common/xcontent/support/XContentMapValues.java +++ b/core/src/main/java/org/elasticsearch/common/xcontent/support/XContentMapValues.java @@ -185,8 +185,8 @@ public class XContentMapValues { // we want all sub properties to match as soon as an object matches return (map) -> filter(map, - include, include.getInitialState(), - exclude, exclude.getInitialState(), + include, 0, + exclude, 0, matchAllAutomaton); } @@ -237,7 +237,7 @@ public class XContentMapValues { // the object matched, so consider that the include matches every inner property // we only care about excludes now subIncludeAutomaton = matchAllAutomaton; - subIncludeState = includeAutomaton.getInitialState(); + subIncludeState = 0; } } diff --git a/core/src/main/resources/org/elasticsearch/bootstrap/security.policy b/core/src/main/resources/org/elasticsearch/bootstrap/security.policy index cbd1f93491b..310d485f98d 100644 --- a/core/src/main/resources/org/elasticsearch/bootstrap/security.policy +++ b/core/src/main/resources/org/elasticsearch/bootstrap/security.policy @@ -31,7 +31,7 @@ grant codeBase "${codebase.securesm-1.1.jar}" { //// Very special jar permissions: //// These are dangerous permissions that we don't want to grant to everything. -grant codeBase "${codebase.lucene-core-6.2.0.jar}" { +grant codeBase "${codebase.lucene-core-6.3.0-snapshot-ed102d6.jar}" { // needed to allow MMapDirectory's "unmap hack" (die unmap hack, die) // java 8 package permission java.lang.RuntimePermission "accessClassInPackage.sun.misc"; @@ -42,7 +42,7 @@ grant codeBase "${codebase.lucene-core-6.2.0.jar}" { permission java.lang.RuntimePermission "accessDeclaredMembers"; }; -grant codeBase "${codebase.lucene-misc-6.2.0.jar}" { +grant codeBase "${codebase.lucene-misc-6.3.0-snapshot-ed102d6.jar}" { // needed to allow shard shrinking to use hard-links if possible via lucenes HardlinkCopyDirectoryWrapper permission java.nio.file.LinkPermission "hard"; }; diff --git a/core/src/main/resources/org/elasticsearch/bootstrap/test-framework.policy b/core/src/main/resources/org/elasticsearch/bootstrap/test-framework.policy index 43f6b62c3c3..2c1a963e296 100644 --- a/core/src/main/resources/org/elasticsearch/bootstrap/test-framework.policy +++ b/core/src/main/resources/org/elasticsearch/bootstrap/test-framework.policy @@ -33,7 +33,7 @@ grant codeBase "${codebase.securemock-1.2.jar}" { permission java.lang.reflect.ReflectPermission "suppressAccessChecks"; }; -grant codeBase "${codebase.lucene-test-framework-6.2.0.jar}" { +grant codeBase "${codebase.lucene-test-framework-6.3.0-snapshot-ed102d6.jar}" { // needed by RamUsageTester permission java.lang.reflect.ReflectPermission "suppressAccessChecks"; // needed for testing hardlinks in StoreRecoveryTests since we install MockFS @@ -42,7 +42,7 @@ grant codeBase "${codebase.lucene-test-framework-6.2.0.jar}" { permission java.lang.RuntimePermission "accessDeclaredMembers"; }; -grant codeBase "${codebase.randomizedtesting-runner-2.3.2.jar}" { +grant codeBase "${codebase.randomizedtesting-runner-2.4.0.jar}" { // optionally needed for access to private test methods (e.g. beforeClass) permission java.lang.reflect.ReflectPermission "suppressAccessChecks"; // needed to fail tests on uncaught exceptions from other threads diff --git a/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/CancellableTasksTests.java b/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/CancellableTasksTests.java index be2a83af42c..6d0a0824490 100644 --- a/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/CancellableTasksTests.java +++ b/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/CancellableTasksTests.java @@ -19,7 +19,7 @@ package org.elasticsearch.action.admin.cluster.node.tasks; import com.carrotsearch.randomizedtesting.RandomizedContext; -import com.carrotsearch.randomizedtesting.generators.RandomInts; +import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest; import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksResponse; @@ -380,9 +380,9 @@ public class CancellableTasksTests extends TaskManagerTestCase { // Introduce an additional pseudo random repeatable race conditions String delayName = RandomizedContext.current().getRunnerSeedAsString() + ":" + nodeId + ":" + name; Random random = new Random(delayName.hashCode()); - if (RandomInts.randomIntBetween(random, 0, 10) < 1) { + if (RandomNumbers.randomIntBetween(random, 0, 10) < 1) { try { - Thread.sleep(RandomInts.randomIntBetween(random, 20, 50)); + Thread.sleep(RandomNumbers.randomIntBetween(random, 20, 50)); } catch (InterruptedException ex) { Thread.currentThread().interrupt(); } diff --git a/core/src/test/java/org/elasticsearch/client/transport/FailAndRetryMockTransport.java b/core/src/test/java/org/elasticsearch/client/transport/FailAndRetryMockTransport.java index 9d2c176dffb..bc771f5721d 100644 --- a/core/src/test/java/org/elasticsearch/client/transport/FailAndRetryMockTransport.java +++ b/core/src/test/java/org/elasticsearch/client/transport/FailAndRetryMockTransport.java @@ -19,10 +19,7 @@ package org.elasticsearch.client.transport; -import com.carrotsearch.randomizedtesting.generators.RandomInts; -import org.elasticsearch.Version; import org.elasticsearch.action.admin.cluster.node.liveness.LivenessResponse; -import org.elasticsearch.action.admin.cluster.node.liveness.TransportLivenessAction; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.component.Lifecycle; @@ -37,7 +34,6 @@ import org.elasticsearch.transport.TransportRequest; import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportResponseHandler; -import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportServiceAdapter; import java.io.IOException; @@ -83,7 +79,7 @@ abstract class FailAndRetryMockTransport imp //once nodes are connected we'll just return errors for each sendRequest call triedNodes.add(node); - if (RandomInts.randomInt(random, 100) > 10) { + if (random.nextInt(100) > 10) { connectTransportExceptions.incrementAndGet(); throw new ConnectTransportException(node, "node not available"); } else { diff --git a/core/src/test/java/org/elasticsearch/index/query/RandomQueryBuilder.java b/core/src/test/java/org/elasticsearch/index/query/RandomQueryBuilder.java index c244273d13a..d16b225ab69 100644 --- a/core/src/test/java/org/elasticsearch/index/query/RandomQueryBuilder.java +++ b/core/src/test/java/org/elasticsearch/index/query/RandomQueryBuilder.java @@ -19,7 +19,7 @@ package org.elasticsearch.index.query; -import com.carrotsearch.randomizedtesting.generators.RandomInts; +import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import com.carrotsearch.randomizedtesting.generators.RandomStrings; import org.elasticsearch.test.AbstractQueryTestCase; @@ -38,7 +38,7 @@ public class RandomQueryBuilder { * @return a random {@link QueryBuilder} */ public static QueryBuilder createQuery(Random r) { - switch (RandomInts.randomIntBetween(r, 0, 3)) { + switch (RandomNumbers.randomIntBetween(r, 0, 3)) { case 0: return new MatchAllQueryBuilderTests().createTestQueryBuilder(); case 1: @@ -61,7 +61,7 @@ public class RandomQueryBuilder { // for now, only use String Rangequeries for MultiTerm test, numeric and date makes little sense // see issue #12123 for discussion MultiTermQueryBuilder multiTermQueryBuilder; - switch(RandomInts.randomIntBetween(r, 0, 3)) { + switch(RandomNumbers.randomIntBetween(r, 0, 3)) { case 0: RangeQueryBuilder stringRangeQuery = new RangeQueryBuilder(AbstractQueryTestCase.STRING_FIELD_NAME); stringRangeQuery.from("a" + RandomStrings.randomAsciiOfLengthBetween(r, 1, 10)); @@ -81,7 +81,7 @@ public class RandomQueryBuilder { throw new UnsupportedOperationException(); } if (r.nextBoolean()) { - multiTermQueryBuilder.boost(2.0f / RandomInts.randomIntBetween(r, 1, 20)); + multiTermQueryBuilder.boost(2.0f / RandomNumbers.randomIntBetween(r, 1, 20)); } return multiTermQueryBuilder; } diff --git a/core/src/test/java/org/elasticsearch/index/snapshots/blobstore/SlicedInputStreamTests.java b/core/src/test/java/org/elasticsearch/index/snapshots/blobstore/SlicedInputStreamTests.java index 4bd8ba9cb3e..74b61047ace 100644 --- a/core/src/test/java/org/elasticsearch/index/snapshots/blobstore/SlicedInputStreamTests.java +++ b/core/src/test/java/org/elasticsearch/index/snapshots/blobstore/SlicedInputStreamTests.java @@ -18,7 +18,7 @@ */ package org.elasticsearch.index.snapshots.blobstore; -import com.carrotsearch.randomizedtesting.generators.RandomInts; +import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import org.elasticsearch.test.ESTestCase; import java.io.ByteArrayInputStream; @@ -111,7 +111,7 @@ public class SlicedInputStreamTests extends ESTestCase { } private byte[] randomBytes(Random random) { - int length = RandomInts.randomIntBetween(random, 1, 10); + int length = RandomNumbers.randomIntBetween(random, 1, 10); byte[] data = new byte[length]; random.nextBytes(data); return data; diff --git a/core/src/test/java/org/elasticsearch/test/geo/RandomShapeGenerator.java b/core/src/test/java/org/elasticsearch/test/geo/RandomShapeGenerator.java index eeebe8cbcdc..e18ba0fe322 100644 --- a/core/src/test/java/org/elasticsearch/test/geo/RandomShapeGenerator.java +++ b/core/src/test/java/org/elasticsearch/test/geo/RandomShapeGenerator.java @@ -19,7 +19,7 @@ package org.elasticsearch.test.geo; -import com.carrotsearch.randomizedtesting.generators.RandomInts; +import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import com.vividsolutions.jts.algorithm.ConvexHull; import com.vividsolutions.jts.geom.Coordinate; import com.vividsolutions.jts.geom.Geometry; @@ -61,7 +61,7 @@ public class RandomShapeGenerator extends RandomGeoGenerator { private static final ShapeType[] types = values(); public static ShapeType randomType(Random r) { - return types[RandomInts.randomIntBetween(r, 0, types.length - 1)]; + return types[RandomNumbers.randomIntBetween(r, 0, types.length - 1)]; } } @@ -115,7 +115,7 @@ public class RandomShapeGenerator extends RandomGeoGenerator { throws InvalidShapeException { if (numGeometries <= 0) { // cap geometry collection at 4 shapes (to save test time) - numGeometries = RandomInts.randomIntBetween(r, 2, 4); + numGeometries = RandomNumbers.randomIntBetween(r, 2, 4); } if (nearPoint == null) { @@ -187,7 +187,7 @@ public class RandomShapeGenerator extends RandomGeoGenerator { // for random testing having a maximum number of 10 points for a line string is more than sufficient // if this number gets out of hand, the number of self intersections for a linestring can become // (n^2-n)/2 and computing the relation intersection matrix will become NP-Hard - int numPoints = RandomInts.randomIntBetween(r, 3, 10); + int numPoints = RandomNumbers.randomIntBetween(r, 3, 10); CoordinatesBuilder coordinatesBuilder = new CoordinatesBuilder(); for (int i=0; i= 90; + return r.nextInt(100) >= 90; } private static Range xRandomRange(Random r, double near, Range bounds) { diff --git a/distribution/licenses/lucene-analyzers-common-6.2.0.jar.sha1 b/distribution/licenses/lucene-analyzers-common-6.2.0.jar.sha1 deleted file mode 100644 index 57aec3f4ac2..00000000000 --- a/distribution/licenses/lucene-analyzers-common-6.2.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -d254d52dd394b5079129f3d5f3bf4f2d44a5936e \ No newline at end of file diff --git a/distribution/licenses/lucene-analyzers-common-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-analyzers-common-6.3.0-snapshot-ed102d6.jar.sha1 new file mode 100644 index 00000000000..7947a57e315 --- /dev/null +++ b/distribution/licenses/lucene-analyzers-common-6.3.0-snapshot-ed102d6.jar.sha1 @@ -0,0 +1 @@ +6a96375ee285920400266dfc411342f2e3d3d623 \ No newline at end of file diff --git a/distribution/licenses/lucene-backward-codecs-6.2.0.jar.sha1 b/distribution/licenses/lucene-backward-codecs-6.2.0.jar.sha1 deleted file mode 100644 index 04aefc62f61..00000000000 --- a/distribution/licenses/lucene-backward-codecs-6.2.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -b625bb21456b3c0d1e5e431bced125cb060c1abd \ No newline at end of file diff --git a/distribution/licenses/lucene-backward-codecs-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-backward-codecs-6.3.0-snapshot-ed102d6.jar.sha1 new file mode 100644 index 00000000000..e395661bace --- /dev/null +++ b/distribution/licenses/lucene-backward-codecs-6.3.0-snapshot-ed102d6.jar.sha1 @@ -0,0 +1 @@ +aa0403f90ac23f50667477b314175e58175c8067 \ No newline at end of file diff --git a/distribution/licenses/lucene-core-6.2.0.jar.sha1 b/distribution/licenses/lucene-core-6.2.0.jar.sha1 deleted file mode 100644 index 2d74124e624..00000000000 --- a/distribution/licenses/lucene-core-6.2.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -849ee62525a294416802be2cacc66c80352f6f13 \ No newline at end of file diff --git a/distribution/licenses/lucene-core-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-core-6.3.0-snapshot-ed102d6.jar.sha1 new file mode 100644 index 00000000000..0ac4d342184 --- /dev/null +++ b/distribution/licenses/lucene-core-6.3.0-snapshot-ed102d6.jar.sha1 @@ -0,0 +1 @@ +6510e4e4af2b90a2d1e2e8c27e92120667c0bff8 \ No newline at end of file diff --git a/distribution/licenses/lucene-grouping-6.2.0.jar.sha1 b/distribution/licenses/lucene-grouping-6.2.0.jar.sha1 deleted file mode 100644 index 6ba525a038f..00000000000 --- a/distribution/licenses/lucene-grouping-6.2.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -9527fedfd5acc624b2bb3f862bd99fb8f470b053 \ No newline at end of file diff --git a/distribution/licenses/lucene-grouping-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-grouping-6.3.0-snapshot-ed102d6.jar.sha1 new file mode 100644 index 00000000000..fddadafd79c --- /dev/null +++ b/distribution/licenses/lucene-grouping-6.3.0-snapshot-ed102d6.jar.sha1 @@ -0,0 +1 @@ +16fb10b3b006a4a7cb2ed9b436e59c91dc9edf47 \ No newline at end of file diff --git a/distribution/licenses/lucene-highlighter-6.2.0.jar.sha1 b/distribution/licenses/lucene-highlighter-6.2.0.jar.sha1 deleted file mode 100644 index c258e3fb850..00000000000 --- a/distribution/licenses/lucene-highlighter-6.2.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -7ca342372a3f45e32bbd21cecaa757e38eccb8a5 \ No newline at end of file diff --git a/distribution/licenses/lucene-highlighter-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-highlighter-6.3.0-snapshot-ed102d6.jar.sha1 new file mode 100644 index 00000000000..2e9d4b99abd --- /dev/null +++ b/distribution/licenses/lucene-highlighter-6.3.0-snapshot-ed102d6.jar.sha1 @@ -0,0 +1 @@ +47a3c05e1be7f593a0860c744c752807fb4d44fb \ No newline at end of file diff --git a/distribution/licenses/lucene-join-6.2.0.jar.sha1 b/distribution/licenses/lucene-join-6.2.0.jar.sha1 deleted file mode 100644 index 01989e96a58..00000000000 --- a/distribution/licenses/lucene-join-6.2.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -da0b8de98511abd4fe9c7d48a353d17854c5ed46 \ No newline at end of file diff --git a/distribution/licenses/lucene-join-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-join-6.3.0-snapshot-ed102d6.jar.sha1 new file mode 100644 index 00000000000..4cf7214aa44 --- /dev/null +++ b/distribution/licenses/lucene-join-6.3.0-snapshot-ed102d6.jar.sha1 @@ -0,0 +1 @@ +2bbe25506f997056555c8bfba64382663433517e \ No newline at end of file diff --git a/distribution/licenses/lucene-memory-6.2.0.jar.sha1 b/distribution/licenses/lucene-memory-6.2.0.jar.sha1 deleted file mode 100644 index b8a4a87efe2..00000000000 --- a/distribution/licenses/lucene-memory-6.2.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -bc9e075b1ee051c8e5246c237c38d8e71dab8a66 \ No newline at end of file diff --git a/distribution/licenses/lucene-memory-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-memory-6.3.0-snapshot-ed102d6.jar.sha1 new file mode 100644 index 00000000000..ea50210ceab --- /dev/null +++ b/distribution/licenses/lucene-memory-6.3.0-snapshot-ed102d6.jar.sha1 @@ -0,0 +1 @@ +76a1c370acfa5ad74beef243512a1dda64d01d79 \ No newline at end of file diff --git a/distribution/licenses/lucene-misc-6.2.0.jar.sha1 b/distribution/licenses/lucene-misc-6.2.0.jar.sha1 deleted file mode 100644 index f4e081865ad..00000000000 --- a/distribution/licenses/lucene-misc-6.2.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -94ddde6312566a4da4a50a88e453b6c82c759b41 \ No newline at end of file diff --git a/distribution/licenses/lucene-misc-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-misc-6.3.0-snapshot-ed102d6.jar.sha1 new file mode 100644 index 00000000000..85a2e7bfd79 --- /dev/null +++ b/distribution/licenses/lucene-misc-6.3.0-snapshot-ed102d6.jar.sha1 @@ -0,0 +1 @@ +ce459756eb7210c1bf7cd2a0cb25c7c5660de224 \ No newline at end of file diff --git a/distribution/licenses/lucene-queries-6.2.0.jar.sha1 b/distribution/licenses/lucene-queries-6.2.0.jar.sha1 deleted file mode 100644 index f7270a23afe..00000000000 --- a/distribution/licenses/lucene-queries-6.2.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -dce47238f78e3e97d91dc6fefa9f46f07866bc2b \ No newline at end of file diff --git a/distribution/licenses/lucene-queries-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-queries-6.3.0-snapshot-ed102d6.jar.sha1 new file mode 100644 index 00000000000..83e1c72fa8a --- /dev/null +++ b/distribution/licenses/lucene-queries-6.3.0-snapshot-ed102d6.jar.sha1 @@ -0,0 +1 @@ +1c27529566f2b8ab8856e8b0a4779ad2071588fc \ No newline at end of file diff --git a/distribution/licenses/lucene-queryparser-6.2.0.jar.sha1 b/distribution/licenses/lucene-queryparser-6.2.0.jar.sha1 deleted file mode 100644 index 8e95aa600ec..00000000000 --- a/distribution/licenses/lucene-queryparser-6.2.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -17ef728ac15e668bfa1105321611548424637645 \ No newline at end of file diff --git a/distribution/licenses/lucene-queryparser-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-queryparser-6.3.0-snapshot-ed102d6.jar.sha1 new file mode 100644 index 00000000000..3a2cbe25abf --- /dev/null +++ b/distribution/licenses/lucene-queryparser-6.3.0-snapshot-ed102d6.jar.sha1 @@ -0,0 +1 @@ +b5a9d1e55751dc193879c22a48823a863492bbe2 \ No newline at end of file diff --git a/distribution/licenses/lucene-sandbox-6.2.0.jar.sha1 b/distribution/licenses/lucene-sandbox-6.2.0.jar.sha1 deleted file mode 100644 index 1f34be3033d..00000000000 --- a/distribution/licenses/lucene-sandbox-6.2.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -520183f7b9aba77a26e224760c420a3844b0631a \ No newline at end of file diff --git a/distribution/licenses/lucene-sandbox-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-sandbox-6.3.0-snapshot-ed102d6.jar.sha1 new file mode 100644 index 00000000000..b83229a5e93 --- /dev/null +++ b/distribution/licenses/lucene-sandbox-6.3.0-snapshot-ed102d6.jar.sha1 @@ -0,0 +1 @@ +8d73ab3c889d386b3a594941992a5d942c57513a \ No newline at end of file diff --git a/distribution/licenses/lucene-spatial-6.2.0.jar.sha1 b/distribution/licenses/lucene-spatial-6.2.0.jar.sha1 deleted file mode 100644 index 22e81792e40..00000000000 --- a/distribution/licenses/lucene-spatial-6.2.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -8dba929b66927b936fbc76103b109ad6c824daee \ No newline at end of file diff --git a/distribution/licenses/lucene-spatial-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-spatial-6.3.0-snapshot-ed102d6.jar.sha1 new file mode 100644 index 00000000000..cfff1364101 --- /dev/null +++ b/distribution/licenses/lucene-spatial-6.3.0-snapshot-ed102d6.jar.sha1 @@ -0,0 +1 @@ +85e74df9bc722f9a858775496cad7f35dce65aa0 \ No newline at end of file diff --git a/distribution/licenses/lucene-spatial-extras-6.2.0.jar.sha1 b/distribution/licenses/lucene-spatial-extras-6.2.0.jar.sha1 deleted file mode 100644 index d5e8f379d78..00000000000 --- a/distribution/licenses/lucene-spatial-extras-6.2.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3b5a6ef5cd90c0218a72e9e2f7e60104be2447da \ No newline at end of file diff --git a/distribution/licenses/lucene-spatial-extras-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-spatial-extras-6.3.0-snapshot-ed102d6.jar.sha1 new file mode 100644 index 00000000000..f3c30ce0275 --- /dev/null +++ b/distribution/licenses/lucene-spatial-extras-6.3.0-snapshot-ed102d6.jar.sha1 @@ -0,0 +1 @@ +cc12ac933b39ef33e186e77f7f6af6bdc99f9108 \ No newline at end of file diff --git a/distribution/licenses/lucene-spatial3d-6.2.0.jar.sha1 b/distribution/licenses/lucene-spatial3d-6.2.0.jar.sha1 deleted file mode 100644 index d0ce5275a26..00000000000 --- a/distribution/licenses/lucene-spatial3d-6.2.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -fcdb0567725722c5145149d1502848b6a96ec18d \ No newline at end of file diff --git a/distribution/licenses/lucene-spatial3d-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-spatial3d-6.3.0-snapshot-ed102d6.jar.sha1 new file mode 100644 index 00000000000..32f9ef0d6aa --- /dev/null +++ b/distribution/licenses/lucene-spatial3d-6.3.0-snapshot-ed102d6.jar.sha1 @@ -0,0 +1 @@ +8dfc7a42839db1e4ef69dbfc23d9ef2fb6c43ff6 \ No newline at end of file diff --git a/distribution/licenses/lucene-suggest-6.2.0.jar.sha1 b/distribution/licenses/lucene-suggest-6.2.0.jar.sha1 deleted file mode 100644 index 39392ad1158..00000000000 --- a/distribution/licenses/lucene-suggest-6.2.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3d9d526c51f483d27f425c75d7e56bc0849242d6 \ No newline at end of file diff --git a/distribution/licenses/lucene-suggest-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-suggest-6.3.0-snapshot-ed102d6.jar.sha1 new file mode 100644 index 00000000000..1cde0aa3992 --- /dev/null +++ b/distribution/licenses/lucene-suggest-6.3.0-snapshot-ed102d6.jar.sha1 @@ -0,0 +1 @@ +0b065269e76aa5cf56fa30dcb0cb4641974b6bed \ No newline at end of file diff --git a/docs/plugins/index.asciidoc b/docs/plugins/index.asciidoc index 8b1c3dd7726..ec1954a86a8 100644 --- a/docs/plugins/index.asciidoc +++ b/docs/plugins/index.asciidoc @@ -3,7 +3,7 @@ :ref: https://www.elastic.co/guide/en/elasticsearch/reference/master :guide: https://www.elastic.co/guide :version: 6.0.0-alpha1 -:lucene_version: 6.2.0 +:lucene_version: 6.3.0 :plugin_url: https://artifacts.elastic.co/downloads/elasticsearch-plugins [[intro]] diff --git a/docs/reference/index.asciidoc b/docs/reference/index.asciidoc index 26181a0fd80..93b25802bd3 100644 --- a/docs/reference/index.asciidoc +++ b/docs/reference/index.asciidoc @@ -9,7 +9,7 @@ release-state can be: released | prerelease | unreleased ////////// :release-state: unreleased -:lucene_version: 6.2.0 +:lucene_version: 6.3.4 :branch: master :jdk: 1.8.0_73 :defguide: https://www.elastic.co/guide/en/elasticsearch/guide/master diff --git a/modules/lang-expression/licenses/lucene-expressions-6.2.0.jar.sha1 b/modules/lang-expression/licenses/lucene-expressions-6.2.0.jar.sha1 deleted file mode 100644 index 205aaae6e66..00000000000 --- a/modules/lang-expression/licenses/lucene-expressions-6.2.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -99764b20aba5443f8a181f7015a806443c589844 \ No newline at end of file diff --git a/modules/lang-expression/licenses/lucene-expressions-6.3.0-snapshot-ed102d6.jar.sha1 b/modules/lang-expression/licenses/lucene-expressions-6.3.0-snapshot-ed102d6.jar.sha1 new file mode 100644 index 00000000000..5499a2d2ebf --- /dev/null +++ b/modules/lang-expression/licenses/lucene-expressions-6.3.0-snapshot-ed102d6.jar.sha1 @@ -0,0 +1 @@ +3ad2da8a2c48520f05723050252b131c96cc3b7c \ No newline at end of file diff --git a/plugins/analysis-icu/licenses/lucene-analyzers-icu-6.2.0.jar.sha1 b/plugins/analysis-icu/licenses/lucene-analyzers-icu-6.2.0.jar.sha1 deleted file mode 100644 index 2a734f79a3f..00000000000 --- a/plugins/analysis-icu/licenses/lucene-analyzers-icu-6.2.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -68de5f298090b92aa9a803eb4f5aed0c9104e685 \ No newline at end of file diff --git a/plugins/analysis-icu/licenses/lucene-analyzers-icu-6.3.0-snapshot-ed102d6.jar.sha1 b/plugins/analysis-icu/licenses/lucene-analyzers-icu-6.3.0-snapshot-ed102d6.jar.sha1 new file mode 100644 index 00000000000..296a0b911fc --- /dev/null +++ b/plugins/analysis-icu/licenses/lucene-analyzers-icu-6.3.0-snapshot-ed102d6.jar.sha1 @@ -0,0 +1 @@ +2794258f69dd65abe1d991013b6ba98ae8a2ab60 \ No newline at end of file diff --git a/plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-6.2.0.jar.sha1 b/plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-6.2.0.jar.sha1 deleted file mode 100644 index 749cb8ecde8..00000000000 --- a/plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-6.2.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -17ee76df332c0342a172790472b777086487a299 \ No newline at end of file diff --git a/plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-6.3.0-snapshot-ed102d6.jar.sha1 b/plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-6.3.0-snapshot-ed102d6.jar.sha1 new file mode 100644 index 00000000000..12c5e58189c --- /dev/null +++ b/plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-6.3.0-snapshot-ed102d6.jar.sha1 @@ -0,0 +1 @@ +7e325a21b6bde7a86861bf380b40aeaafabe8c90 \ No newline at end of file diff --git a/plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-6.2.0.jar.sha1 b/plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-6.2.0.jar.sha1 deleted file mode 100644 index 359173e0084..00000000000 --- a/plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-6.2.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -8d2a6b8679563d9f044eb1cee580282b20d8e149 \ No newline at end of file diff --git a/plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-6.3.0-snapshot-ed102d6.jar.sha1 b/plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-6.3.0-snapshot-ed102d6.jar.sha1 new file mode 100644 index 00000000000..2618809af12 --- /dev/null +++ b/plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-6.3.0-snapshot-ed102d6.jar.sha1 @@ -0,0 +1 @@ +746a7888f48f3be86f7a31d95a1ca1e033cf0c3f \ No newline at end of file diff --git a/plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-6.2.0.jar.sha1 b/plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-6.2.0.jar.sha1 deleted file mode 100644 index 66e339bfa2f..00000000000 --- a/plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-6.2.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -ba3fd99d1cf47d31b82817accdb199fc7a8d838d \ No newline at end of file diff --git a/plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-6.3.0-snapshot-ed102d6.jar.sha1 b/plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-6.3.0-snapshot-ed102d6.jar.sha1 new file mode 100644 index 00000000000..6b89f1e5386 --- /dev/null +++ b/plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-6.3.0-snapshot-ed102d6.jar.sha1 @@ -0,0 +1 @@ +fec95fc0b5bb2e0855a9c9be3dc2982c0ab0fa16 \ No newline at end of file diff --git a/plugins/analysis-stempel/licenses/lucene-analyzers-stempel-6.2.0.jar.sha1 b/plugins/analysis-stempel/licenses/lucene-analyzers-stempel-6.2.0.jar.sha1 deleted file mode 100644 index 5cfb071f3a3..00000000000 --- a/plugins/analysis-stempel/licenses/lucene-analyzers-stempel-6.2.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -09d2a759a765f73e2e7becbc560411469c464cfa \ No newline at end of file diff --git a/plugins/analysis-stempel/licenses/lucene-analyzers-stempel-6.3.0-snapshot-ed102d6.jar.sha1 b/plugins/analysis-stempel/licenses/lucene-analyzers-stempel-6.3.0-snapshot-ed102d6.jar.sha1 new file mode 100644 index 00000000000..f6e72856fd9 --- /dev/null +++ b/plugins/analysis-stempel/licenses/lucene-analyzers-stempel-6.3.0-snapshot-ed102d6.jar.sha1 @@ -0,0 +1 @@ +8663b181f65dadffa6f0f42e56a7ab36ff447fd4 \ No newline at end of file diff --git a/test/framework/src/main/java/org/elasticsearch/ingest/RandomDocumentPicks.java b/test/framework/src/main/java/org/elasticsearch/ingest/RandomDocumentPicks.java index 3be4cd3edf4..efdf10d5a5d 100644 --- a/test/framework/src/main/java/org/elasticsearch/ingest/RandomDocumentPicks.java +++ b/test/framework/src/main/java/org/elasticsearch/ingest/RandomDocumentPicks.java @@ -19,7 +19,7 @@ package org.elasticsearch.ingest; -import com.carrotsearch.randomizedtesting.generators.RandomInts; +import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import com.carrotsearch.randomizedtesting.generators.RandomPicks; import com.carrotsearch.randomizedtesting.generators.RandomStrings; @@ -41,7 +41,7 @@ public final class RandomDocumentPicks { * path to refer to a field name using the dot notation. */ public static String randomFieldName(Random random) { - int numLevels = RandomInts.randomIntBetween(random, 1, 5); + int numLevels = RandomNumbers.randomIntBetween(random, 1, 5); String fieldName = ""; for (int i = 0; i < numLevels; i++) { if (i > 0) { @@ -169,7 +169,7 @@ public final class RandomDocumentPicks { } private static Object randomFieldValue(Random random, int currentDepth) { - switch(RandomInts.randomIntBetween(random, 0, 9)) { + switch(RandomNumbers.randomIntBetween(random, 0, 9)) { case 0: return randomString(random); case 1: @@ -180,28 +180,28 @@ public final class RandomDocumentPicks { return random.nextDouble(); case 4: List stringList = new ArrayList<>(); - int numStringItems = RandomInts.randomIntBetween(random, 1, 10); + int numStringItems = RandomNumbers.randomIntBetween(random, 1, 10); for (int j = 0; j < numStringItems; j++) { stringList.add(randomString(random)); } return stringList; case 5: List intList = new ArrayList<>(); - int numIntItems = RandomInts.randomIntBetween(random, 1, 10); + int numIntItems = RandomNumbers.randomIntBetween(random, 1, 10); for (int j = 0; j < numIntItems; j++) { intList.add(random.nextInt()); } return intList; case 6: List booleanList = new ArrayList<>(); - int numBooleanItems = RandomInts.randomIntBetween(random, 1, 10); + int numBooleanItems = RandomNumbers.randomIntBetween(random, 1, 10); for (int j = 0; j < numBooleanItems; j++) { booleanList.add(random.nextBoolean()); } return booleanList; case 7: List doubleList = new ArrayList<>(); - int numDoubleItems = RandomInts.randomIntBetween(random, 1, 10); + int numDoubleItems = RandomNumbers.randomIntBetween(random, 1, 10); for (int j = 0; j < numDoubleItems; j++) { doubleList.add(random.nextDouble()); } @@ -211,7 +211,7 @@ public final class RandomDocumentPicks { addRandomFields(random, newNode, ++currentDepth); return newNode; case 9: - byte[] byteArray = new byte[RandomInts.randomIntBetween(random, 1, 1024)]; + byte[] byteArray = new byte[RandomNumbers.randomIntBetween(random, 1, 1024)]; random.nextBytes(byteArray); return byteArray; default: @@ -230,7 +230,7 @@ public final class RandomDocumentPicks { if (currentDepth > 5) { return; } - int numFields = RandomInts.randomIntBetween(random, 1, 10); + int numFields = RandomNumbers.randomIntBetween(random, 1, 10); for (int i = 0; i < numFields; i++) { String fieldName = randomLeafFieldName(random); Object fieldValue = randomFieldValue(random, currentDepth); diff --git a/test/framework/src/main/java/org/elasticsearch/test/BackgroundIndexer.java b/test/framework/src/main/java/org/elasticsearch/test/BackgroundIndexer.java index 3c5f105e4d1..b739099cff0 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/BackgroundIndexer.java +++ b/test/framework/src/main/java/org/elasticsearch/test/BackgroundIndexer.java @@ -18,7 +18,7 @@ package org.elasticsearch.test;/* */ import com.carrotsearch.randomizedtesting.RandomizedTest; -import com.carrotsearch.randomizedtesting.generators.RandomInts; +import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import com.carrotsearch.randomizedtesting.generators.RandomStrings; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; @@ -208,10 +208,10 @@ public class BackgroundIndexer implements AutoCloseable { } private XContentBuilder generateSource(long id, Random random) throws IOException { - int contentLength = RandomInts.randomIntBetween(random, minFieldSize, maxFieldSize); + int contentLength = RandomNumbers.randomIntBetween(random, minFieldSize, maxFieldSize); StringBuilder text = new StringBuilder(contentLength); while (text.length() < contentLength) { - int tokenLength = RandomInts.randomIntBetween(random, 1, Math.min(contentLength - text.length(), 10)); + int tokenLength = RandomNumbers.randomIntBetween(random, 1, Math.min(contentLength - text.length(), 10)); text.append(" ").append(RandomStrings.randomRealisticUnicodeOfCodepointLength(random, tokenLength)); } XContentBuilder builder = XContentFactory.smileBuilder(); diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index 1f2f01cc352..bde3ebaa85c 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -20,7 +20,7 @@ package org.elasticsearch.test; import com.carrotsearch.randomizedtesting.RandomizedContext; import com.carrotsearch.randomizedtesting.annotations.TestGroup; -import com.carrotsearch.randomizedtesting.generators.RandomInts; +import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import com.carrotsearch.randomizedtesting.generators.RandomPicks; import org.apache.http.HttpHost; import org.apache.lucene.util.IOUtils; @@ -433,7 +433,7 @@ public abstract class ESIntegTestCase extends ESTestCase { if (randomBoolean()) { // keep this low so we don't stall tests - builder.put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), RandomInts.randomIntBetween(random, 1, 15) + "ms"); + builder.put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), RandomNumbers.randomIntBetween(random, 1, 15) + "ms"); } return builder; @@ -446,8 +446,8 @@ public abstract class ESIntegTestCase extends ESTestCase { } switch (random.nextInt(4)) { case 3: - final int maxThreadCount = RandomInts.randomIntBetween(random, 1, 4); - final int maxMergeCount = RandomInts.randomIntBetween(random, maxThreadCount, maxThreadCount + 4); + final int maxThreadCount = RandomNumbers.randomIntBetween(random, 1, 4); + final int maxMergeCount = RandomNumbers.randomIntBetween(random, maxThreadCount, maxThreadCount + 4); builder.put(MergeSchedulerConfig.MAX_MERGE_COUNT_SETTING.getKey(), maxMergeCount); builder.put(MergeSchedulerConfig.MAX_THREAD_COUNT_SETTING.getKey(), maxThreadCount); break; @@ -458,7 +458,7 @@ public abstract class ESIntegTestCase extends ESTestCase { private static Settings.Builder setRandomIndexTranslogSettings(Random random, Settings.Builder builder) { if (random.nextBoolean()) { - builder.put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), new ByteSizeValue(RandomInts.randomIntBetween(random, 1, 300), ByteSizeUnit.MB)); + builder.put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), new ByteSizeValue(RandomNumbers.randomIntBetween(random, 1, 300), ByteSizeUnit.MB)); } if (random.nextBoolean()) { builder.put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), new ByteSizeValue(1, ByteSizeUnit.PB)); // just don't flush @@ -468,14 +468,14 @@ public abstract class ESIntegTestCase extends ESTestCase { } if (random.nextBoolean()) { - builder.put(IndexSettings.INDEX_TRANSLOG_SYNC_INTERVAL_SETTING.getKey(), RandomInts.randomIntBetween(random, 100, 5000), TimeUnit.MILLISECONDS); + builder.put(IndexSettings.INDEX_TRANSLOG_SYNC_INTERVAL_SETTING.getKey(), RandomNumbers.randomIntBetween(random, 100, 5000), TimeUnit.MILLISECONDS); } return builder; } private TestCluster buildWithPrivateContext(final Scope scope, final long seed) throws Exception { - return RandomizedContext.current().runWithPrivateRandomness(new com.carrotsearch.randomizedtesting.Randomness(seed), new Callable() { + return RandomizedContext.current().runWithPrivateRandomness(seed, new Callable() { @Override public TestCluster call() throws Exception { return buildTestCluster(scope, seed); diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java index 9f6f17b80df..f75774a7364 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java @@ -25,7 +25,7 @@ import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope; import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope.Scope; import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite; import com.carrotsearch.randomizedtesting.generators.CodepointSetGenerator; -import com.carrotsearch.randomizedtesting.generators.RandomInts; +import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import com.carrotsearch.randomizedtesting.generators.RandomPicks; import com.carrotsearch.randomizedtesting.generators.RandomStrings; import com.carrotsearch.randomizedtesting.rules.TestRuleAdapter; @@ -288,7 +288,7 @@ public abstract class ESTestCase extends LuceneTestCase { * @see #scaledRandomIntBetween(int, int) */ public static int randomIntBetween(int min, int max) { - return RandomInts.randomIntBetween(random(), min, max); + return RandomNumbers.randomIntBetween(random(), min, max); } /** diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java index ef86c45b59f..360a94c0a6d 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -21,7 +21,7 @@ package org.elasticsearch.test; import com.carrotsearch.randomizedtesting.RandomizedTest; import com.carrotsearch.randomizedtesting.SeedUtils; import com.carrotsearch.randomizedtesting.SysGlobals; -import com.carrotsearch.randomizedtesting.generators.RandomInts; +import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import com.carrotsearch.randomizedtesting.generators.RandomPicks; import com.carrotsearch.randomizedtesting.generators.RandomStrings; import org.apache.logging.log4j.Logger; @@ -249,7 +249,7 @@ public final class InternalTestCluster extends TestCluster { boolean useDedicatedMasterNodes = randomlyAddDedicatedMasters ? random.nextBoolean() : false; - this.numSharedDataNodes = RandomInts.randomIntBetween(random, minNumDataNodes, maxNumDataNodes); + this.numSharedDataNodes = RandomNumbers.randomIntBetween(random, minNumDataNodes, maxNumDataNodes); assert this.numSharedDataNodes >= 0; if (numSharedDataNodes == 0) { @@ -267,7 +267,7 @@ public final class InternalTestCluster extends TestCluster { this.numSharedDedicatedMasterNodes = 0; } if (numClientNodes < 0) { - this.numSharedCoordOnlyNodes = RandomInts.randomIntBetween(random, DEFAULT_MIN_NUM_CLIENT_NODES, DEFAULT_MAX_NUM_CLIENT_NODES); + this.numSharedCoordOnlyNodes = RandomNumbers.randomIntBetween(random, DEFAULT_MIN_NUM_CLIENT_NODES, DEFAULT_MAX_NUM_CLIENT_NODES); } else { this.numSharedCoordOnlyNodes = numClientNodes; } @@ -321,14 +321,14 @@ public final class InternalTestCluster extends TestCluster { // Some tests make use of scripting quite a bit, so increase the limit for integration tests builder.put(ScriptService.SCRIPT_MAX_COMPILATIONS_PER_MINUTE.getKey(), 1000); if (TEST_NIGHTLY) { - builder.put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_INCOMING_RECOVERIES_SETTING.getKey(), RandomInts.randomIntBetween(random, 5, 10)); - builder.put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_OUTGOING_RECOVERIES_SETTING.getKey(), RandomInts.randomIntBetween(random, 5, 10)); + builder.put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_INCOMING_RECOVERIES_SETTING.getKey(), RandomNumbers.randomIntBetween(random, 5, 10)); + builder.put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_OUTGOING_RECOVERIES_SETTING.getKey(), RandomNumbers.randomIntBetween(random, 5, 10)); } else if (random.nextInt(100) <= 90) { - builder.put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_INCOMING_RECOVERIES_SETTING.getKey(), RandomInts.randomIntBetween(random, 2, 5)); - builder.put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_OUTGOING_RECOVERIES_SETTING.getKey(), RandomInts.randomIntBetween(random, 2, 5)); + builder.put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_INCOMING_RECOVERIES_SETTING.getKey(), RandomNumbers.randomIntBetween(random, 2, 5)); + builder.put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_OUTGOING_RECOVERIES_SETTING.getKey(), RandomNumbers.randomIntBetween(random, 2, 5)); } // always reduce this - it can make tests really slow - builder.put(RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_STATE_SYNC_SETTING.getKey(), TimeValue.timeValueMillis(RandomInts.randomIntBetween(random, 20, 50))); + builder.put(RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_STATE_SYNC_SETTING.getKey(), TimeValue.timeValueMillis(RandomNumbers.randomIntBetween(random, 20, 50))); defaultSettings = builder.build(); executor = EsExecutors.newScaling("test runner", 0, Integer.MAX_VALUE, 0, TimeUnit.SECONDS, EsExecutors.daemonThreadFactory("test_" + clusterName), new ThreadContext(Settings.EMPTY)); } @@ -396,7 +396,7 @@ public final class InternalTestCluster extends TestCluster { } if (random.nextBoolean()) { - builder.put(MappingUpdatedAction.INDICES_MAPPING_DYNAMIC_TIMEOUT_SETTING.getKey(), new TimeValue(RandomInts.randomIntBetween(random, 10, 30), TimeUnit.SECONDS)); + builder.put(MappingUpdatedAction.INDICES_MAPPING_DYNAMIC_TIMEOUT_SETTING.getKey(), new TimeValue(RandomNumbers.randomIntBetween(random, 10, 30), TimeUnit.SECONDS)); } if (random.nextInt(10) == 0) { @@ -406,9 +406,9 @@ public final class InternalTestCluster extends TestCluster { if (random.nextBoolean()) { if (random.nextInt(10) == 0) { // do something crazy slow here - builder.put(IndexStoreConfig.INDICES_STORE_THROTTLE_MAX_BYTES_PER_SEC_SETTING.getKey(), new ByteSizeValue(RandomInts.randomIntBetween(random, 1, 10), ByteSizeUnit.MB)); + builder.put(IndexStoreConfig.INDICES_STORE_THROTTLE_MAX_BYTES_PER_SEC_SETTING.getKey(), new ByteSizeValue(RandomNumbers.randomIntBetween(random, 1, 10), ByteSizeUnit.MB)); } else { - builder.put(IndexStoreConfig.INDICES_STORE_THROTTLE_MAX_BYTES_PER_SEC_SETTING.getKey(), new ByteSizeValue(RandomInts.randomIntBetween(random, 10, 200), ByteSizeUnit.MB)); + builder.put(IndexStoreConfig.INDICES_STORE_THROTTLE_MAX_BYTES_PER_SEC_SETTING.getKey(), new ByteSizeValue(RandomNumbers.randomIntBetween(random, 10, 200), ByteSizeUnit.MB)); } } if (random.nextBoolean()) { @@ -417,21 +417,21 @@ public final class InternalTestCluster extends TestCluster { if (random.nextBoolean()) { if (random.nextInt(10) == 0) { // do something crazy slow here - builder.put(RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING.getKey(), new ByteSizeValue(RandomInts.randomIntBetween(random, 1, 10), ByteSizeUnit.MB)); + builder.put(RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING.getKey(), new ByteSizeValue(RandomNumbers.randomIntBetween(random, 1, 10), ByteSizeUnit.MB)); } else { - builder.put(RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING.getKey(), new ByteSizeValue(RandomInts.randomIntBetween(random, 10, 200), ByteSizeUnit.MB)); + builder.put(RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING.getKey(), new ByteSizeValue(RandomNumbers.randomIntBetween(random, 10, 200), ByteSizeUnit.MB)); } } if (random.nextBoolean()) { - builder.put(TcpTransport.PING_SCHEDULE.getKey(), RandomInts.randomIntBetween(random, 100, 2000) + "ms"); + builder.put(TcpTransport.PING_SCHEDULE.getKey(), RandomNumbers.randomIntBetween(random, 100, 2000) + "ms"); } if (random.nextBoolean()) { - builder.put(ScriptService.SCRIPT_CACHE_SIZE_SETTING.getKey(), RandomInts.randomIntBetween(random, 0, 2000)); + builder.put(ScriptService.SCRIPT_CACHE_SIZE_SETTING.getKey(), RandomNumbers.randomIntBetween(random, 0, 2000)); } if (random.nextBoolean()) { - builder.put(ScriptService.SCRIPT_CACHE_EXPIRE_SETTING.getKey(), TimeValue.timeValueMillis(RandomInts.randomIntBetween(random, 750, 10000000)).getStringRep()); + builder.put(ScriptService.SCRIPT_CACHE_EXPIRE_SETTING.getKey(), TimeValue.timeValueMillis(RandomNumbers.randomIntBetween(random, 750, 10000000)).getStringRep()); } return builder.build(); From bafedc93fa4b2e005b168410497ac8c58c75d61c Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Fri, 28 Oct 2016 14:50:46 +0200 Subject: [PATCH 033/132] Fix Lucene version in the docs. --- docs/reference/index.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/index.asciidoc b/docs/reference/index.asciidoc index 93b25802bd3..317500b474f 100644 --- a/docs/reference/index.asciidoc +++ b/docs/reference/index.asciidoc @@ -9,7 +9,7 @@ release-state can be: released | prerelease | unreleased ////////// :release-state: unreleased -:lucene_version: 6.3.4 +:lucene_version: 6.3.0 :branch: master :jdk: 1.8.0_73 :defguide: https://www.elastic.co/guide/en/elasticsearch/guide/master From 326a665509b66a0fe15135e1dd4b9a486b97433e Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Fri, 28 Oct 2016 15:21:43 +0200 Subject: [PATCH 034/132] [TEST] Only set 0s state timeout if we have more than one node --- .../elasticsearch/gradle/test/ClusterFormationTasks.groovy | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy index 56d074bee2f..d3b08f7339a 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy @@ -259,8 +259,10 @@ class ClusterFormationTasks { // basically skip initial state recovery to allow the cluster to form using a realistic master election // this means all nodes must be up, join the seed node and do a master election. This will also allow new and // old nodes in the BWC case to become the master - esConfig['discovery.zen.minimum_master_nodes'] = node.config.numNodes - esConfig['discovery.initial_state_timeout'] = '0s' // don't wait for state.. just start up quickly + if (node.config.numNodes > 1) { + esConfig['discovery.zen.minimum_master_nodes'] = node.config.numNodes + esConfig['discovery.initial_state_timeout'] = '0s' // don't wait for state.. just start up quickly + } esConfig['node.max_local_storage_nodes'] = node.config.numNodes esConfig['http.port'] = node.config.httpPort esConfig['transport.tcp.port'] = node.config.transportPort From 733d74229b20653479e02dce660bd6402edbc8ba Mon Sep 17 00:00:00 2001 From: Ali Beyad Date: Fri, 28 Oct 2016 09:40:04 -0400 Subject: [PATCH 035/132] Revert "Updates vagrant tests upgrade version number" (#21171) --- qa/vagrant/versions | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/qa/vagrant/versions b/qa/vagrant/versions index 0062ac97180..654a95a3a25 100644 --- a/qa/vagrant/versions +++ b/qa/vagrant/versions @@ -1 +1 @@ -5.0.0 +6.0.0-alpha1-SNAPSHOT From bce34e4f1e8c2d78d6a7c6265c212589b70b32f8 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Fri, 28 Oct 2016 09:47:47 -0400 Subject: [PATCH 036/132] Disable out of date versions exception We are upgrading from out of date versions in our tests right now and we can't fix that because the current versions to upgrade from aren't in maven central. We'll resolve the resolution issue soon, but for now let's get the build green. --- qa/vagrant/build.gradle | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/qa/vagrant/build.gradle b/qa/vagrant/build.gradle index bc27de7f624..1923a8b6a1e 100644 --- a/qa/vagrant/build.gradle +++ b/qa/vagrant/build.gradle @@ -153,7 +153,8 @@ task verifyPackagingTestUpgradeFromVersions { Set versions = getVersions() Set actualVersions = new HashSet<>(Arrays.asList(upgradeFromVersions)) if (!versions.equals(actualVersions)) { - throw new GradleException("out-of-date versions [" + actualVersions + "], expected [" + versions + "]; run gradle updatePackagingTestUpgradeFromVersions") + println("out-of-date versions [" + actualVersions + "], expected [" + versions + "]; run gradle updatePackagingTestUpgradeFromVersions") + // throw new GradleException("out-of-date versions [" + actualVersions + "], expected [" + versions + "]; run gradle updatePackagingTestUpgradeFromVersions") } } } From 36c86c2b0017242e8abbcc4253c717b1bc4400d4 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Fri, 28 Oct 2016 16:03:21 +0200 Subject: [PATCH 037/132] Fix `_cat/store` sort test to use a reliable sort value using the store size is not reliable since it's cached and might be stale depending on the timing when the cache entry was generated. --- .../rest-api-spec/test/cat.shards/10_basic.yaml | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.shards/10_basic.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.shards/10_basic.yaml index fbdaff6cbd9..5529d4f5799 100755 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.shards/10_basic.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.shards/10_basic.yaml @@ -256,11 +256,11 @@ - do: cat.shards: - h: [index, store] - s: [store] + h: [index, docs] + s: [docs] - - match: + - match: # don't use the store here it's cached and might be stale $body: | - /^ foo \s+ (\d+|\d+[.]\d+)(kb|b)\n - bar \s+ (\d+|\d+[.]\d+)(kb|b)\n + /^ foo \s+ 0\n + bar \s+ 1\n $/ From 9598616dfe3e2719082e6f98851588a3609b0f19 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Fri, 28 Oct 2016 16:18:41 +0200 Subject: [PATCH 038/132] Fallback to '/' info call to fetch cluster version The `_cat/nodes` API might not be available in all clusters for instance if they have authorization enabled. This change falls back to the previously used method of using the '/' endpoint to fetch the nodes version, this is best effort and will emit a warning. --- .../test/rest/yaml/ClientYamlTestClient.java | 79 +++++++++++++------ 1 file changed, 56 insertions(+), 23 deletions(-) diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestClient.java b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestClient.java index 8a427a6d41f..62b09b3ecbf 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestClient.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestClient.java @@ -67,39 +67,72 @@ public class ClientYamlTestClient { assert hosts.size() > 0; this.restSpec = restSpec; this.restClient = restClient; - Tuple versionTuple = readMasterAndMinNodeVersion(); + Tuple versionTuple = readMasterAndMinNodeVersion(hosts.size()); this.esVersion = versionTuple.v1(); Version masterVersion = versionTuple.v2(); // this will be logged in each test such that if something fails we get it in the logs for each test logger.info("initializing client, minimum es version: [{}] master version: [{}] hosts: {}", esVersion, masterVersion, hosts); } - private Tuple readMasterAndMinNodeVersion() throws IOException { - // we simply go to the _cat/nodes API and parse all versions in the cluster - Response response = restClient.performRequest("GET", "/_cat/nodes", Collections.singletonMap("h", "version,master")); - ClientYamlTestResponse restTestResponse = new ClientYamlTestResponse(response); - String nodesCatResponse = restTestResponse.getBodyAsString(); - String[] split = nodesCatResponse.split("\n"); - Version version = null; - Version masterVersion = null; - for (String perNode : split) { - final String[] versionAndMaster = perNode.split(" "); - assert versionAndMaster.length == 2 : "invalid line: " + perNode + " length: " + versionAndMaster.length; - final Version currentVersion = Version.fromString(versionAndMaster[0]); - final boolean master = versionAndMaster[1].trim().equals("*"); - if (master) { - assert masterVersion == null; - masterVersion = currentVersion; - } - if (version == null) { - version = currentVersion; - } else { - if (version.onOrAfter(currentVersion)) { + private Tuple readMasterAndMinNodeVersion(int numHosts) throws IOException { + try { + // we simply go to the _cat/nodes API and parse all versions in the cluster + Response response = restClient.performRequest("GET", "/_cat/nodes", Collections.singletonMap("h", "version,master")); + ClientYamlTestResponse restTestResponse = new ClientYamlTestResponse(response); + String nodesCatResponse = restTestResponse.getBodyAsString(); + String[] split = nodesCatResponse.split("\n"); + Version version = null; + Version masterVersion = null; + for (String perNode : split) { + final String[] versionAndMaster = perNode.split(" "); + assert versionAndMaster.length == 2 : "invalid line: " + perNode + " length: " + versionAndMaster.length; + final Version currentVersion = Version.fromString(versionAndMaster[0]); + final boolean master = versionAndMaster[1].trim().equals("*"); + if (master) { + assert masterVersion == null; + masterVersion = currentVersion; + } + if (version == null) { version = currentVersion; + } else if (version.onOrAfter(currentVersion)) { + version = currentVersion; } } + return new Tuple<>(version, masterVersion); + } catch (ResponseException ex) { + if (ex.getResponse().getStatusLine().getStatusCode() == 403) { + logger.warn("Fallback to simple info '/' request, _cat/nodes is not authorized"); + final Version version = readAndCheckVersion(numHosts); + return new Tuple<>(version, version); + } + throw ex; } - return new Tuple<>(version, masterVersion); + } + + private Version readAndCheckVersion(int numHosts) throws IOException { + ClientYamlSuiteRestApi restApi = restApi("info"); + assert restApi.getPaths().size() == 1; + assert restApi.getMethods().size() == 1; + Version version = null; + for (int i = 0; i < numHosts; i++) { + //we don't really use the urls here, we rely on the client doing round-robin to touch all the nodes in the cluster + String method = restApi.getMethods().get(0); + String endpoint = restApi.getPaths().get(0); + Response response = restClient.performRequest(method, endpoint); + ClientYamlTestResponse restTestResponse = new ClientYamlTestResponse(response); + + Object latestVersion = restTestResponse.evaluate("version.number"); + if (latestVersion == null) { + throw new RuntimeException("elasticsearch version not found in the response"); + } + final Version currentVersion = Version.fromString(restTestResponse.evaluate("version.number").toString()); + if (version == null) { + version = currentVersion; + } else if (version.onOrAfter(currentVersion)) { + version = currentVersion; + } + } + return version; } public Version getEsVersion() { From 9e3eacec357a7d79199d755fab628560449aff8e Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Fri, 28 Oct 2016 10:29:29 -0400 Subject: [PATCH 039/132] Fix generation of versions to upgrade from Vagrant tests use a static list of dependencies to upgrade from and we weren't including 5.0.0 deps in that list. Also when the list was incorrect we weren't sorting the "current" list so it was difficult to read. Also adds 2.4.1 to the list but *doesn't* add 5.0.0 because we still can't resolve it yet. We still only print an error when the list is wrong but don't abort the build. We'll abort the build once we've fixed resolution for 5.0.0 and we can re-add it. --- qa/vagrant/build.gradle | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/qa/vagrant/build.gradle b/qa/vagrant/build.gradle index 1923a8b6a1e..edad98b5fbd 100644 --- a/qa/vagrant/build.gradle +++ b/qa/vagrant/build.gradle @@ -125,10 +125,7 @@ Set getVersions() { new URL('https://repo1.maven.org/maven2/org/elasticsearch/elasticsearch/maven-metadata.xml').openStream().withStream { s -> xml = new XmlParser().parse(s) } - - // List all N-1 releases from maven central - int major = Integer.parseInt(project.version.substring(0, project.version.indexOf('.'))) - 1 - Set versions = new TreeSet<>(xml.versioning.versions.version.collect { it.text() }.findAll { it ==~ /$major\.\d\.\d/ }) + Set versions = new TreeSet<>(xml.versioning.versions.version.collect { it.text() }.findAll { it ==~ /[5]\.\d\.\d/ }) if (versions.isEmpty() == false) { return versions; } @@ -151,7 +148,7 @@ task verifyPackagingTestUpgradeFromVersions { String maybeUpdateFromVersions = System.getProperty("tests.packaging.upgrade.from.versions", null) if (maybeUpdateFromVersions == null) { Set versions = getVersions() - Set actualVersions = new HashSet<>(Arrays.asList(upgradeFromVersions)) + Set actualVersions = new TreeSet<>(Arrays.asList(upgradeFromVersions)) if (!versions.equals(actualVersions)) { println("out-of-date versions [" + actualVersions + "], expected [" + versions + "]; run gradle updatePackagingTestUpgradeFromVersions") // throw new GradleException("out-of-date versions [" + actualVersions + "], expected [" + versions + "]; run gradle updatePackagingTestUpgradeFromVersions") From 75ee2bb61d235a8ff898bce63bf2646e4e73e627 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Fri, 28 Oct 2016 18:46:54 +0200 Subject: [PATCH 040/132] Update resiliency page for the release of v5 (#21177) --- docs/resiliency/index.asciidoc | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/docs/resiliency/index.asciidoc b/docs/resiliency/index.asciidoc index 47ca68e00f5..0ded9530e0e 100644 --- a/docs/resiliency/index.asciidoc +++ b/docs/resiliency/index.asciidoc @@ -153,10 +153,10 @@ The new tests are run continuously in our testing farm and are passing. We are a that no failures are found. -== Unreleased +== Completed [float] -=== Port Jepsen tests dealing with loss of acknowledged writes to our testing framework (STATUS: UNRELEASED, V5.0.0) +=== Port Jepsen tests dealing with loss of acknowledged writes to our testing framework (STATUS: DONE, V5.0.0) We have increased our test coverage to include scenarios tested by Jepsen that demonstrate loss of acknowledged writes, as described in the Elasticsearch related blogs. We make heavy use of randomization to expand on the scenarios that can be tested and to introduce @@ -167,7 +167,7 @@ where the `testAckedIndexing` test was specifically added to check that we don't [float] -=== Loss of documents during network partition (STATUS: UNRELEASED, v5.0.0) +=== Loss of documents during network partition (STATUS: DONE, v5.0.0) If a network partition separates a node from the master, there is some window of time before the node detects it. The length of the window is dependent on the type of the partition. This window is extremely small if a socket is broken. More adversarial partitions, for example, silently dropping requests without breaking the socket can take longer (up to 3x30s using current defaults). @@ -175,7 +175,7 @@ If the node hosts a primary shard at the moment of partition, and ends up being To prevent this situation, the primary needs to wait for the master to acknowledge replica shard failures before acknowledging the write to the client. {GIT}14252[#14252] [float] -=== Safe primary relocations (STATUS: UNRELEASED, v5.0.0) +=== Safe primary relocations (STATUS: DONE, v5.0.0) When primary relocation completes, a cluster state is propagated that deactivates the old primary and marks the new primary as active. As cluster state changes are not applied synchronously on all nodes, there can be a time interval where the relocation target has processed the @@ -189,7 +189,7 @@ on the relocation target, each of the nodes believes the other to be the active chasing the primary being quickly sent back and forth between the nodes, potentially making them both go OOM. {GIT}12573[#12573] [float] -=== Do not allow stale shards to automatically be promoted to primary (STATUS: UNRELEASED, v5.0.0) +=== Do not allow stale shards to automatically be promoted to primary (STATUS: DONE, v5.0.0) In some scenarios, after the loss of all valid copies, a stale replica shard can be automatically assigned as a primary, preferring old data to no data at all ({GIT}14671[#14671]). This can lead to a loss of acknowledged writes if the valid copies are not lost but are rather @@ -199,7 +199,7 @@ for one of the good shard copies to reappear. In case where all good copies are stale shard copy. [float] -=== Make index creation resilient to index closing and full cluster crashes (STATUS: UNRELEASED, v5.0.0) +=== Make index creation resilient to index closing and full cluster crashes (STATUS: DONE, v5.0.0) Recovering an index requires a quorum (with an exception for 2) of shard copies to be available to allocate a primary. This means that a primary cannot be assigned if the cluster dies before enough shards have been allocated ({GIT}9126[#9126]). The same happens if an index @@ -211,7 +211,7 @@ shard will be allocated upon reopening the index. [float] -=== Use two phase commit for Cluster State publishing (STATUS: UNRELEASED, v5.0.0) +=== Use two phase commit for Cluster State publishing (STATUS: DONE, v5.0.0) A master node in Elasticsearch continuously https://www.elastic.co/guide/en/elasticsearch/reference/current/modules-discovery-zen.html#fault-detection[monitors the cluster nodes] and removes any node from the cluster that doesn't respond to its pings in a timely @@ -225,8 +225,6 @@ a new phase to cluster state publishing where the proposed cluster state is sent but is not yet committed. Only once enough nodes (`discovery.zen.minimum_master_nodes`) actively acknowledge the change, it is committed and commit messages are sent to the nodes. See {GIT}13062[#13062]. -== Completed - [float] === Wait on incoming joins before electing local node as master (STATUS: DONE, v2.0.0) From 267a7b427b044efada3fb4c94aa1c495b83842dc Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Fri, 28 Oct 2016 13:23:45 -0400 Subject: [PATCH 041/132] Fix the package upgrade tests for 5.0.0 (#21178) 5.0+'s deb and rpm packages aren't pushed to maven cnertral so instead we have to download them from artifacts.elastic.org. --- qa/vagrant/build.gradle | 19 +++++++++++++++---- qa/vagrant/ivy.xml | 6 ++++++ qa/vagrant/versions | 2 +- 3 files changed, 22 insertions(+), 5 deletions(-) create mode 100644 qa/vagrant/ivy.xml diff --git a/qa/vagrant/build.gradle b/qa/vagrant/build.gradle index edad98b5fbd..78730f7a9f4 100644 --- a/qa/vagrant/build.gradle +++ b/qa/vagrant/build.gradle @@ -90,7 +90,18 @@ configurations { } repositories { - mavenCentral() + mavenCentral() // Try maven central first, it'll have releases before 5.0.0 + /* Setup a repository that tries to download from + https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-[revision].[ext] + which should work for 5.0.0+. This isn't a real ivy repository so we have + to fake the ivy.xml file locally but it gets the job done. */ + ivy { + url "https://artifacts.elastic.co/downloads/" + layout "pattern", { + artifact "elasticsearch/elasticsearch-[revision].[ext]" + ivy "file:$projectDir/ivy.xml" + } + } } dependencies { @@ -150,8 +161,8 @@ task verifyPackagingTestUpgradeFromVersions { Set versions = getVersions() Set actualVersions = new TreeSet<>(Arrays.asList(upgradeFromVersions)) if (!versions.equals(actualVersions)) { - println("out-of-date versions [" + actualVersions + "], expected [" + versions + "]; run gradle updatePackagingTestUpgradeFromVersions") - // throw new GradleException("out-of-date versions [" + actualVersions + "], expected [" + versions + "]; run gradle updatePackagingTestUpgradeFromVersions") + throw new GradleException("out-of-date versions " + actualVersions + + ", expected " + versions + "; run gradle updatePackagingTestUpgradeFromVersions") } } } @@ -245,7 +256,7 @@ for (String box : availableBoxes) { Task up = tasks.create("vagrant${boxTask}#up", VagrantCommandTask) { boxName box - /* Its important that we try to reprovision the box even if it already + /* It's important that we try to reprovision the box even if it already exists. That way updates to the vagrant configuration take automatically. That isn't to say that the updates will always be compatible. Its ok to just destroy the boxes if they get busted but that is a manual step diff --git a/qa/vagrant/ivy.xml b/qa/vagrant/ivy.xml new file mode 100644 index 00000000000..63afcc52bcf --- /dev/null +++ b/qa/vagrant/ivy.xml @@ -0,0 +1,6 @@ + + + + diff --git a/qa/vagrant/versions b/qa/vagrant/versions index 654a95a3a25..0062ac97180 100644 --- a/qa/vagrant/versions +++ b/qa/vagrant/versions @@ -1 +1 @@ -6.0.0-alpha1-SNAPSHOT +5.0.0 From b9691d15aed2a3822d861a77d80e64af90f0f91a Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Fri, 28 Oct 2016 20:22:24 +0200 Subject: [PATCH 042/132] IndexWithShadowReplicasIT.testReplicaToPrimaryPromotion should wait for node leave to be processed --- .../org/elasticsearch/index/IndexWithShadowReplicasIT.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasIT.java b/core/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasIT.java index 7bae67f5b2f..a335a42edb6 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasIT.java +++ b/core/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasIT.java @@ -291,13 +291,14 @@ public class IndexWithShadowReplicasIT extends ESIntegTestCase { assertThat(gResp2.getSource().get("foo"), equalTo("bar")); // Node1 has the primary, now node2 has the replica - String node2 = internalCluster().startNode(nodeSettings); + internalCluster().startNode(nodeSettings); ensureGreen(IDX); client().admin().cluster().prepareHealth().setWaitForNodes("2").get(); flushAndRefresh(IDX); logger.info("--> stopping node1 [{}]", node1); internalCluster().stopRandomNode(InternalTestCluster.nameFilter(node1)); + ensureClusterSizeConsistency(); // wait for the new node to be elected and process the node leave ensureYellow(IDX); logger.info("--> performing query"); From f11a217546587c2b41147b5eb2b378913517886f Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Fri, 28 Oct 2016 17:02:24 -0400 Subject: [PATCH 043/132] Rewrite vagrant upgrade test's dependency resolution This is cleaner and it doesn't fail in Jenkins. I'm not sure why the old way failed in jenkins but not locally, but computers are basically magic so I take what I can get. Here is a jenkins failure: https://elasticsearch-ci.elastic.co/job/elastic+elasticsearch+master+packaging-tests/238/consoleFull --- qa/vagrant/build.gradle | 12 ++++-------- qa/vagrant/ivy.xml | 6 ------ 2 files changed, 4 insertions(+), 14 deletions(-) delete mode 100644 qa/vagrant/ivy.xml diff --git a/qa/vagrant/build.gradle b/qa/vagrant/build.gradle index 78730f7a9f4..f263adbadc8 100644 --- a/qa/vagrant/build.gradle +++ b/qa/vagrant/build.gradle @@ -92,15 +92,11 @@ configurations { repositories { mavenCentral() // Try maven central first, it'll have releases before 5.0.0 /* Setup a repository that tries to download from - https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-[revision].[ext] - which should work for 5.0.0+. This isn't a real ivy repository so we have - to fake the ivy.xml file locally but it gets the job done. */ + https://artifacts.elastic.co/downloads/elasticsearch/[module]-[revision].[ext] + which should work for 5.0.0+. This isn't a real ivy repository but gradle + is fine with that */ ivy { - url "https://artifacts.elastic.co/downloads/" - layout "pattern", { - artifact "elasticsearch/elasticsearch-[revision].[ext]" - ivy "file:$projectDir/ivy.xml" - } + artifactPattern "https://artifacts.elastic.co/downloads/elasticsearch/[module]-[revision].[ext]" } } diff --git a/qa/vagrant/ivy.xml b/qa/vagrant/ivy.xml deleted file mode 100644 index 63afcc52bcf..00000000000 --- a/qa/vagrant/ivy.xml +++ /dev/null @@ -1,6 +0,0 @@ - - - - From d731a330aa708cdf6c652348aca2ac2e0635c280 Mon Sep 17 00:00:00 2001 From: Igor Motov Date: Fri, 28 Oct 2016 11:24:50 -1000 Subject: [PATCH 044/132] Tests: Add addtional logging to SearchCancellationIT tests --- .../search/SearchCancellationIT.java | 21 ++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/search/SearchCancellationIT.java b/core/src/test/java/org/elasticsearch/search/SearchCancellationIT.java index f9b344fc264..53653392e6e 100644 --- a/core/src/test/java/org/elasticsearch/search/SearchCancellationIT.java +++ b/core/src/test/java/org/elasticsearch/search/SearchCancellationIT.java @@ -28,8 +28,11 @@ import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchScrollAction; import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.PluginsService; import org.elasticsearch.plugins.ScriptPlugin; @@ -58,6 +61,9 @@ import static org.hamcrest.Matchers.hasSize; @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.SUITE) public class SearchCancellationIT extends ESIntegTestCase { + private static final ToXContent.Params FORMAT_PARAMS = new ToXContent.MapParams(Collections.singletonMap("pretty", "false")); + + @Override protected Collection> nodePlugins() { return Collections.singleton(ScriptedBlockPlugin.class); @@ -65,15 +71,17 @@ public class SearchCancellationIT extends ESIntegTestCase { @Override protected Settings nodeSettings(int nodeOrdinal) { - return Settings.builder().put(SearchService.LOW_LEVEL_CANCELLATION_SETTING.getKey(), randomBoolean()).build(); + boolean lowLevelCancellation = randomBoolean(); + logger.info("Using lowLevelCancellation: {}", lowLevelCancellation); + return Settings.builder().put(SearchService.LOW_LEVEL_CANCELLATION_SETTING.getKey(), lowLevelCancellation).build(); } private void indexTestData() { - for (int i = 0; i < 10; i++) { + for (int i = 0; i < 5; i++) { // Make sure we have a few segments BulkRequestBuilder bulkRequestBuilder = client().prepareBulk().setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); - for(int j=0; j<10; j++) { - bulkRequestBuilder.add(client().prepareIndex("test", "type", Integer.toString(i*10 + j)).setSource("field", "value")); + for (int j = 0; j < 20; j++) { + bulkRequestBuilder.add(client().prepareIndex("test", "type", Integer.toString(i * 5 + j)).setSource("field", "value")); } assertNoFailures(bulkRequestBuilder.get()); } @@ -145,6 +153,7 @@ public class SearchCancellationIT extends ESIntegTestCase { awaitForBlock(plugins); cancelSearch(SearchAction.NAME); disableBlocks(plugins); + logger.info("Segments {}", XContentHelper.toString(client().admin().indices().prepareSegments("test").get(), FORMAT_PARAMS)); ensureSearchWasCancelled(searchResponse); } @@ -162,6 +171,7 @@ public class SearchCancellationIT extends ESIntegTestCase { awaitForBlock(plugins); cancelSearch(SearchAction.NAME); disableBlocks(plugins); + logger.info("Segments {}", XContentHelper.toString(client().admin().indices().prepareSegments("test").get(), FORMAT_PARAMS)); ensureSearchWasCancelled(searchResponse); } @@ -222,7 +232,7 @@ public class SearchCancellationIT extends ESIntegTestCase { disableBlocks(plugins); SearchResponse response = ensureSearchWasCancelled(scrollResponse); - if (response != null){ + if (response != null) { // The response didn't fail completely - update scroll id scrollId = response.getScrollId(); } @@ -285,6 +295,7 @@ public class SearchCancellationIT extends ESIntegTestCase { public class NativeTestScriptedBlock extends AbstractSearchScript { @Override public Object run() { + Loggers.getLogger(SearchCancellationIT.class).info("Blocking on the document {}", doc().get("_uid")); hits.incrementAndGet(); try { awaitBusy(() -> shouldBlock.get() == false); From 3a7a218e8f349c89131ede0403ec33b588c42fc1 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Thu, 20 Oct 2016 16:15:47 -0400 Subject: [PATCH 045/132] Support negative array ofsets in painless Adds support for indexing into lists and arrays with negative indexes meaning "counting from the back". So for if `x = ["cat", "dog", "chicken"]` then `x[-1] == "chicken"`. This adds an extra branch to every array and list access but some performance testing makes it look like the branch predictor successfully predicts the branch every time so there isn't a in execution time for this feature when the index is positive. When the index is negative performance testing showed the runtime is the same as writing `x[x.length - 1]`, again, presumably thanks to the branch predictor. Those performance metrics were calculated for lists and arrays but `def`s get roughly the same treatment though instead of inlining the test they need to make a invoke dynamic so we don't screw up maps. Closes #20870 --- .../scripting/painless-syntax.asciidoc | 21 +++- .../java/org/elasticsearch/painless/Def.java | 86 ++++++++++++++ .../elasticsearch/painless/DefBootstrap.java | 10 +- .../elasticsearch/painless/MethodWriter.java | 1 - .../painless/WriterConstants.java | 5 + .../painless/node/AStoreable.java | 20 ++++ .../painless/node/PSubBrace.java | 7 +- .../painless/node/PSubDefArray.java | 17 ++- .../painless/node/PSubListShortcut.java | 15 +-- .../painless/ArrayLikeObjectTestCase.java | 105 ++++++++++++++++++ .../elasticsearch/painless/ArrayTests.java | 48 ++++---- .../org/elasticsearch/painless/ListTests.java | 70 ++++++++++++ .../org/elasticsearch/painless/MapTests.java | 45 ++++++++ 13 files changed, 399 insertions(+), 51 deletions(-) create mode 100644 modules/lang-painless/src/test/java/org/elasticsearch/painless/ArrayLikeObjectTestCase.java create mode 100644 modules/lang-painless/src/test/java/org/elasticsearch/painless/ListTests.java create mode 100644 modules/lang-painless/src/test/java/org/elasticsearch/painless/MapTests.java diff --git a/docs/reference/modules/scripting/painless-syntax.asciidoc b/docs/reference/modules/scripting/painless-syntax.asciidoc index 1191facc369..8f280de9946 100644 --- a/docs/reference/modules/scripting/painless-syntax.asciidoc +++ b/docs/reference/modules/scripting/painless-syntax.asciidoc @@ -28,6 +28,23 @@ String constants can be declared with single quotes, to avoid escaping horrors w def mystring = 'foo'; --------------------------------------------------------- +[float] +[[painless-arrays]] +==== Arrays + +Arrays can be subscripted starting from `0` for traditional array access or with +negative numbers to starting from the back of the array. So the following +returns `2`. + +[source,painless] +--------------------------------------------------------- +int[] x = new int[5]; +x[0]++; +x[-5]++; +return x[0]; +--------------------------------------------------------- + + [float] [[painless-lists]] ==== List @@ -39,11 +56,13 @@ Lists can be created explicitly (e.g. `new ArrayList()`) or initialized similar def list = [1,2,3]; --------------------------------------------------------- -Lists can also be accessed similar to arrays: they support subscript and `.length`: +Lists can also be accessed similar to arrays. They support `.length` and +subscripts, including negative subscripts to read from the back of the list: [source,painless] --------------------------------------------------------- def list = [1,2,3]; +list[-1] = 5 return list[0] --------------------------------------------------------- diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/Def.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/Def.java index cd761d0ad44..6de116da0e9 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/Def.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/Def.java @@ -109,6 +109,10 @@ public final class Def { private static final MethodHandle LIST_SET; /** pointer to Iterable.iterator() */ private static final MethodHandle ITERATOR; + /** pointer to {@link Def#mapIndexNormalize}. */ + private static final MethodHandle MAP_INDEX_NORMALIZE; + /** pointer to {@link Def#listIndexNormalize}. */ + private static final MethodHandle LIST_INDEX_NORMALIZE; /** factory for arraylength MethodHandle (intrinsic) from Java 9 */ private static final MethodHandle JAVA9_ARRAY_LENGTH_MH_FACTORY; @@ -121,6 +125,10 @@ public final class Def { LIST_GET = lookup.findVirtual(List.class, "get", MethodType.methodType(Object.class, int.class)); LIST_SET = lookup.findVirtual(List.class, "set", MethodType.methodType(Object.class, int.class, Object.class)); ITERATOR = lookup.findVirtual(Iterable.class, "iterator", MethodType.methodType(Iterator.class)); + MAP_INDEX_NORMALIZE = lookup.findStatic(Def.class, "mapIndexNormalize", + MethodType.methodType(Object.class, Map.class, Object.class)); + LIST_INDEX_NORMALIZE = lookup.findStatic(Def.class, "listIndexNormalize", + MethodType.methodType(int.class, List.class, int.class)); } catch (final ReflectiveOperationException roe) { throw new AssertionError(roe); } @@ -522,6 +530,26 @@ public final class Def { "for class [" + receiverClass.getCanonicalName() + "]."); } + /** + * Returns a method handle to normalize the index into an array. This is what makes lists and arrays stored in {@code def} support + * negative offsets. + * @param receiverClass Class of the array to store the value in + * @return a MethodHandle that accepts the receiver as first argument, the index as second argument, and returns the normalized index + * to use with array loads and array stores + */ + static MethodHandle lookupIndexNormalize(Class receiverClass) { + if (receiverClass.isArray()) { + return ArrayIndexNormalizeHelper.arrayIndexNormalizer(receiverClass); + } else if (Map.class.isAssignableFrom(receiverClass)) { + // noop so that mymap[key] doesn't do funny things with negative keys + return MAP_INDEX_NORMALIZE; + } else if (List.class.isAssignableFrom(receiverClass)) { + return LIST_INDEX_NORMALIZE; + } + throw new IllegalArgumentException("Attempting to address a non-array-like type " + + "[" + receiverClass.getCanonicalName() + "] as an array."); + } + /** * Returns a method handle to do an array store. * @param receiverClass Class of the array to store the value in @@ -814,4 +842,62 @@ public final class Def { return ((Number)value).doubleValue(); } } + + /** + * "Normalizes" the index into a {@code Map} by making no change to the index. + */ + public static Object mapIndexNormalize(final Map value, Object index) { + return index; + } + + /** + * "Normalizes" the idnex into a {@code List} by flipping negative indexes around so they are "from the end" of the list. + */ + public static int listIndexNormalize(final List value, int index) { + return index >= 0 ? index : value.size() + index; + } + + /** + * Methods to normalize array indices to support negative indices into arrays stored in {@code def}s. + */ + @SuppressWarnings("unused") // normalizeIndex() methods are are actually used, javac just does not know :) + private static final class ArrayIndexNormalizeHelper { + private static final Lookup PRIV_LOOKUP = MethodHandles.lookup(); + + private static final Map,MethodHandle> ARRAY_TYPE_MH_MAPPING = Collections.unmodifiableMap( + Stream.of(boolean[].class, byte[].class, short[].class, int[].class, long[].class, + char[].class, float[].class, double[].class, Object[].class) + .collect(Collectors.toMap(Function.identity(), type -> { + try { + return PRIV_LOOKUP.findStatic(PRIV_LOOKUP.lookupClass(), "normalizeIndex", + MethodType.methodType(int.class, type, int.class)); + } catch (ReflectiveOperationException e) { + throw new AssertionError(e); + } + })) + ); + + private static final MethodHandle OBJECT_ARRAY_MH = ARRAY_TYPE_MH_MAPPING.get(Object[].class); + + static int normalizeIndex(final boolean[] array, final int index) { return index >= 0 ? index : index + array.length; } + static int normalizeIndex(final byte[] array, final int index) { return index >= 0 ? index : index + array.length; } + static int normalizeIndex(final short[] array, final int index) { return index >= 0 ? index : index + array.length; } + static int normalizeIndex(final int[] array, final int index) { return index >= 0 ? index : index + array.length; } + static int normalizeIndex(final long[] array, final int index) { return index >= 0 ? index : index + array.length; } + static int normalizeIndex(final char[] array, final int index) { return index >= 0 ? index : index + array.length; } + static int normalizeIndex(final float[] array, final int index) { return index >= 0 ? index : index + array.length; } + static int normalizeIndex(final double[] array, final int index) { return index >= 0 ? index : index + array.length; } + static int normalizeIndex(final Object[] array, final int index) { return index >= 0 ? index : index + array.length; } + + static MethodHandle arrayIndexNormalizer(Class arrayType) { + if (!arrayType.isArray()) { + throw new IllegalArgumentException("type must be an array"); + } + return (ARRAY_TYPE_MH_MAPPING.containsKey(arrayType)) ? + ARRAY_TYPE_MH_MAPPING.get(arrayType) : + OBJECT_ARRAY_MH.asType(OBJECT_ARRAY_MH.type().changeParameterType(0, arrayType)); + } + + private ArrayIndexNormalizeHelper() {} + } } diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/DefBootstrap.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/DefBootstrap.java index 9640629cb87..307316efdf4 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/DefBootstrap.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/DefBootstrap.java @@ -32,9 +32,10 @@ import java.lang.invoke.WrongMethodTypeException; /** * Painless invokedynamic bootstrap for the call site. *

- * Has 7 flavors (passed as static bootstrap parameters): dynamic method call, + * Has 11 flavors (passed as static bootstrap parameters): dynamic method call, * dynamic field load (getter), and dynamic field store (setter), dynamic array load, - * dynamic array store, iterator, and method reference. + * dynamic array store, iterator, method reference, unary operator, binary operator, + * shift operator, and dynamic array index normalize. *

* When a new type is encountered at the call site, we lookup from the appropriate * whitelist, and cache with a guard. If we encounter too many types, we stop caching. @@ -69,6 +70,8 @@ public final class DefBootstrap { public static final int BINARY_OPERATOR = 8; /** static bootstrap parameter indicating a shift operator, e.g. foo >> bar */ public static final int SHIFT_OPERATOR = 9; + /** static bootstrap parameter indicating a request to normalize an index for array-like-access */ + public static final int INDEX_NORMALIZE = 10; // constants for the flags parameter of operators /** @@ -152,6 +155,8 @@ public final class DefBootstrap { return Def.lookupIterator(receiver); case REFERENCE: return Def.lookupReference(lookup, (String) args[0], receiver, name); + case INDEX_NORMALIZE: + return Def.lookupIndexNormalize(receiver); default: throw new AssertionError(); } } @@ -448,6 +453,7 @@ public final class DefBootstrap { case ARRAY_LOAD: case ARRAY_STORE: case ITERATOR: + case INDEX_NORMALIZE: if (args.length > 0) { throw new BootstrapMethodError("Illegal static bootstrap parameters for flavor: " + flavor); } diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/MethodWriter.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/MethodWriter.java index 43fd54c51a4..7e56bf49156 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/MethodWriter.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/MethodWriter.java @@ -30,7 +30,6 @@ import org.objectweb.asm.commons.Method; import java.util.ArrayDeque; import java.util.ArrayList; -import java.util.Arrays; import java.util.BitSet; import java.util.Deque; import java.util.List; diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/WriterConstants.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/WriterConstants.java index c546207b1ee..684f9a59ee2 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/WriterConstants.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/WriterConstants.java @@ -32,6 +32,7 @@ import java.lang.invoke.MethodHandle; import java.lang.invoke.MethodHandles; import java.lang.invoke.MethodType; import java.util.BitSet; +import java.util.Collection; import java.util.Iterator; import java.util.Map; import java.util.Objects; @@ -112,6 +113,7 @@ public final class WriterConstants { public static final Method DEF_TO_LONG_EXPLICIT = getAsmMethod(long.class , "DefTolongExplicit" , Object.class); public static final Method DEF_TO_FLOAT_EXPLICIT = getAsmMethod(float.class , "DefTofloatExplicit" , Object.class); public static final Method DEF_TO_DOUBLE_EXPLICIT = getAsmMethod(double.class , "DefTodoubleExplicit", Object.class); + public static final Type DEF_ARRAY_LENGTH_METHOD_TYPE = Type.getMethodType(Type.INT_TYPE, Definition.DEF_TYPE.type); /** invokedynamic bootstrap for lambda expression/method references */ public static final MethodType LAMBDA_BOOTSTRAP_TYPE = @@ -158,6 +160,9 @@ public final class WriterConstants { public static final Type OBJECTS_TYPE = Type.getType(Objects.class); public static final Method EQUALS = getAsmMethod(boolean.class, "equals", Object.class, Object.class); + public static final Type COLLECTION_TYPE = Type.getType(Collection.class); + public static final Method COLLECTION_SIZE = getAsmMethod(int.class, "size"); + private static Method getAsmMethod(final Class rtype, final String name, final Class... ptypes) { return new Method(name, MethodType.methodType(rtype, ptypes).toMethodDescriptorString()); } diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/AStoreable.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/AStoreable.java index 71b8ccd4da1..3cff6bab08e 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/AStoreable.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/AStoreable.java @@ -23,8 +23,11 @@ import org.elasticsearch.painless.Definition.Type; import org.elasticsearch.painless.Globals; import org.elasticsearch.painless.Location; import org.elasticsearch.painless.MethodWriter; +import org.objectweb.asm.Label; +import org.objectweb.asm.Opcodes; import java.util.Objects; +import java.util.function.Consumer; /** * The super class for an expression that can store a value in local memory. @@ -100,4 +103,21 @@ abstract class AStoreable extends AExpression { * Called to store a storabable to local memory. */ abstract void store(MethodWriter writer, Globals globals); + + /** + * Writes the opcodes to flip a negative array index (meaning slots from the end of the array) into a 0-based one (meaning slots from + * the start of the array). + */ + static void writeIndexFlip(MethodWriter writer, Consumer writeGetLength) { + Label noFlip = new Label(); + // Everywhere when it says 'array' below that could also be a list + // The stack after each instruction: array, unnormalized_index + writer.dup(); // array, unnormalized_index, unnormalized_index + writer.ifZCmp(Opcodes.IFGE, noFlip); // array, unnormalized_index + writer.swap(); // negative_index, array + writer.dupX1(); // array, negative_index, array + writeGetLength.accept(writer); // array, negative_index, length + writer.visitInsn(Opcodes.IADD); // array, noralized_index + writer.mark(noFlip); // array, noralized_index + } } diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/PSubBrace.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/PSubBrace.java index 45b3ef88cd1..a6fb3cefbb1 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/PSubBrace.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/PSubBrace.java @@ -60,10 +60,8 @@ final class PSubBrace extends AStoreable { @Override void write(MethodWriter writer, Globals globals) { - if (!write) { - setup(writer, globals); - load(writer, globals); - } + setup(writer, globals); + load(writer, globals); } @Override @@ -84,6 +82,7 @@ final class PSubBrace extends AStoreable { @Override void setup(MethodWriter writer, Globals globals) { index.write(writer, globals); + writeIndexFlip(writer, MethodWriter::arrayLength); } @Override diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/PSubDefArray.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/PSubDefArray.java index 2153897a000..2776fffec61 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/PSubDefArray.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/PSubDefArray.java @@ -34,7 +34,6 @@ import java.util.Set; * Represents an array load/store or shortcut on a def type. (Internal only.) */ final class PSubDefArray extends AStoreable { - private AExpression index; PSubDefArray(Location location, AExpression index) { @@ -59,13 +58,8 @@ final class PSubDefArray extends AStoreable { @Override void write(MethodWriter writer, Globals globals) { - index.write(writer, globals); - - writer.writeDebugInfo(location); - - org.objectweb.asm.Type methodType = - org.objectweb.asm.Type.getMethodType(actual.type, Definition.DEF_TYPE.type, index.actual.type); - writer.invokeDefCall("arrayLoad", methodType, DefBootstrap.ARRAY_LOAD); + setup(writer, globals); + load(writer, globals); } @Override @@ -85,7 +79,12 @@ final class PSubDefArray extends AStoreable { @Override void setup(MethodWriter writer, Globals globals) { - index.write(writer, globals); + // Current stack: def + writer.dup(); // def, def + index.write(writer, globals); // def, def, unnormalized_index + org.objectweb.asm.Type methodType = org.objectweb.asm.Type.getMethodType( + index.actual.type, Definition.DEF_TYPE.type, index.actual.type); + writer.invokeDefCall("normalizeIndex", methodType, DefBootstrap.INDEX_NORMALIZE); // def, normalized_index } @Override diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/PSubListShortcut.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/PSubListShortcut.java index c13f8235821..5b8396f72d3 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/PSubListShortcut.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/PSubListShortcut.java @@ -28,6 +28,7 @@ import org.elasticsearch.painless.Globals; import org.elasticsearch.painless.Locals; import org.elasticsearch.painless.Location; import org.elasticsearch.painless.MethodWriter; +import org.elasticsearch.painless.WriterConstants; import java.util.Objects; import java.util.Set; @@ -87,15 +88,8 @@ final class PSubListShortcut extends AStoreable { @Override void write(MethodWriter writer, Globals globals) { - index.write(writer, globals); - - writer.writeDebugInfo(location); - - getter.write(writer); - - if (!getter.rtn.clazz.equals(getter.handle.type().returnType())) { - writer.checkCast(getter.rtn.type); - } + setup(writer, globals); + load(writer, globals); } @Override @@ -116,6 +110,9 @@ final class PSubListShortcut extends AStoreable { @Override void setup(MethodWriter writer, Globals globals) { index.write(writer, globals); + writeIndexFlip(writer, w -> { + w.invokeInterface(WriterConstants.COLLECTION_TYPE, WriterConstants.COLLECTION_SIZE); + }); } @Override diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/ArrayLikeObjectTestCase.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/ArrayLikeObjectTestCase.java new file mode 100644 index 00000000000..1be3c024710 --- /dev/null +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/ArrayLikeObjectTestCase.java @@ -0,0 +1,105 @@ +/* + * 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.painless; + +import org.elasticsearch.common.Nullable; +import org.hamcrest.Matcher; + +import static java.util.Collections.singletonMap; + +/** + * Superclass for testing array-like objects (arrays and lists). + */ +public abstract class ArrayLikeObjectTestCase extends ScriptTestCase { + /** + * Build the string for declaring the variable holding the array-like-object to test. So {@code int[]} for arrays and {@code List} for + * lists. + */ + protected abstract String declType(String valueType); + /** + * Build the string for calling the constructor for the array-like-object to test. So {@code new int[5]} for arrays and + * {@code [0, 0, 0, 0, 0]} or {@code [null, null, null, null, null]} for lists. + */ + protected abstract String valueCtorCall(String valueType, int size); + /** + * The type of the exception thrown by out of bounds accesses; + */ + protected abstract Matcher outOfBoundsExceptionMatcher(int index, int size); + + private void arrayLoadStoreTestCase(boolean declareAsDef, String valueType, Object val, @Nullable Number valPlusOne) { + String declType = declareAsDef ? "def" : declType(valueType); + String valueCtorCall = valueCtorCall(valueType, 5); + String decl = declType + " x = " + valueCtorCall; + assertEquals(5, exec(decl + "; return x.length", true)); + assertEquals(val, exec(decl + "; x[ 0] = params.val; return x[ 0];", singletonMap("val", val), true)); + assertEquals(val, exec(decl + "; x[ 0] = params.val; return x[-5];", singletonMap("val", val), true)); + assertEquals(val, exec(decl + "; x[-5] = params.val; return x[-5];", singletonMap("val", val), true)); + + expectOutOfBounds( 6, decl + "; return x[ 6]", val); + expectOutOfBounds(-1, decl + "; return x[-6]", val); + expectOutOfBounds( 6, decl + "; x[ 6] = params.val; return 0", val); + expectOutOfBounds(-1, decl + "; x[-6] = params.val; return 0", val); + + if (valPlusOne != null) { + assertEquals(val, exec(decl + "; x[0] = params.val; x[ 0] = x[ 0]++; return x[0];", singletonMap("val", val), true)); + assertEquals(val, exec(decl + "; x[0] = params.val; x[ 0] = x[-5]++; return x[0];", singletonMap("val", val), true)); + assertEquals(valPlusOne, exec(decl + "; x[0] = params.val; x[ 0] = ++x[ 0]; return x[0];", singletonMap("val", val), true)); + assertEquals(valPlusOne, exec(decl + "; x[0] = params.val; x[ 0] = ++x[-5]; return x[0];", singletonMap("val", val), true)); + assertEquals(valPlusOne, exec(decl + "; x[0] = params.val; x[ 0]++ ; return x[0];", singletonMap("val", val), true)); + assertEquals(valPlusOne, exec(decl + "; x[0] = params.val; x[-5]++ ; return x[0];", singletonMap("val", val), true)); + assertEquals(valPlusOne, exec(decl + "; x[0] = params.val; x[ 0] += 1 ; return x[0];", singletonMap("val", val), true)); + assertEquals(valPlusOne, exec(decl + "; x[0] = params.val; x[-5] += 1 ; return x[0];", singletonMap("val", val), true)); + + expectOutOfBounds( 6, decl + "; return x[ 6]++", val); + expectOutOfBounds(-1, decl + "; return x[-6]++", val); + expectOutOfBounds( 6, decl + "; return ++x[ 6]", val); + expectOutOfBounds(-1, decl + "; return ++x[-6]", val); + expectOutOfBounds( 6, decl + "; x[ 6] += 1; return 0", val); + expectOutOfBounds(-1, decl + "; x[-6] += 1; return 0", val); + } + } + + private void expectOutOfBounds(int index, String script, Object val) { + IndexOutOfBoundsException e = expectScriptThrows(IndexOutOfBoundsException.class, + () -> exec(script, singletonMap("val", val), true)); + try { + assertThat(e, outOfBoundsExceptionMatcher(index, 5)); + } catch (AssertionError ae) { + ae.addSuppressed(e); // Mark the exception we are testing as suppressed so we get its stack trace. If it has one :( + throw ae; + } + } + + public void testInts() { arrayLoadStoreTestCase(false, "int", 5, 6); } + public void testIntsInDef() { arrayLoadStoreTestCase(true, "int", 5, 6); } + public void testLongs() { arrayLoadStoreTestCase(false, "long", 5L, 6L); } + public void testLongsInDef() { arrayLoadStoreTestCase(true, "long", 5L, 6L); } + public void testShorts() { arrayLoadStoreTestCase(false, "short", (short) 5, (short) 6); } + public void testShortsInDef() { arrayLoadStoreTestCase(true, "short", (short) 5, (short) 6); } + public void testBytes() { arrayLoadStoreTestCase(false, "byte", (byte) 5, (byte) 6); } + public void testBytesInDef() { arrayLoadStoreTestCase(true, "byte", (byte) 5, (byte) 6); } + public void testFloats() { arrayLoadStoreTestCase(false, "float", 5.0f, 6.0f); } + public void testFloatsInDef() { arrayLoadStoreTestCase(true, "float", 5.0f, 6.0f); } + public void testDoubles() { arrayLoadStoreTestCase(false, "double", 5.0d, 6.0d); } + public void testDoublesInDef() { arrayLoadStoreTestCase(true, "double", 5.0d, 6.0d); } + public void testStrings() { arrayLoadStoreTestCase(false, "String", "cat", null); } + public void testStringsInDef() { arrayLoadStoreTestCase(true, "String", "cat", null); } + public void testDef() { arrayLoadStoreTestCase(true, "def", 5, null); } +} diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/ArrayTests.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/ArrayTests.java index acacc613ab3..350f27f44fc 100644 --- a/modules/lang-painless/src/test/java/org/elasticsearch/painless/ArrayTests.java +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/ArrayTests.java @@ -19,11 +19,32 @@ package org.elasticsearch.painless; +import org.hamcrest.Matcher; + import java.lang.invoke.MethodHandle; import java.lang.invoke.MethodType; -/** Tests for or operator across all types */ -public class ArrayTests extends ScriptTestCase { +import static org.hamcrest.Matchers.both; +import static org.hamcrest.Matchers.hasToString; +import static org.hamcrest.Matchers.instanceOf; + +/** Tests for working with arrays. */ +public class ArrayTests extends ArrayLikeObjectTestCase { + @Override + protected String declType(String valueType) { + return valueType + "[]"; + } + + @Override + protected String valueCtorCall(String valueType, int size) { + return "new " + valueType + "[" + size + "]"; + } + + @Override + protected Matcher outOfBoundsExceptionMatcher(int index, int size) { + return both(instanceOf(ArrayIndexOutOfBoundsException.class)) + .and(hasToString("java.lang.ArrayIndexOutOfBoundsException: " + index)); + } public void testArrayLengthHelper() throws Throwable { assertArrayLength(2, new int[2]); @@ -45,29 +66,6 @@ public class ArrayTests extends ScriptTestCase { .invokeExact(array)); } - public void testArrayLoadStoreInt() { - assertEquals(5, exec("def x = new int[5]; return x.length")); - assertEquals(5, exec("def x = new int[4]; x[0] = 5; return x[0];")); - } - - public void testArrayLoadStoreString() { - assertEquals(5, exec("def x = new String[5]; return x.length")); - assertEquals("foobar", exec("def x = new String[4]; x[0] = 'foobar'; return x[0];")); - } - - public void testArrayLoadStoreDef() { - assertEquals(5, exec("def x = new def[5]; return x.length")); - assertEquals(5, exec("def x = new def[4]; x[0] = 5; return x[0];")); - } - - public void testArrayCompoundInt() { - assertEquals(6, exec("int[] x = new int[5]; x[0] = 5; x[0]++; return x[0];")); - } - - public void testArrayCompoundDef() { - assertEquals(6, exec("def x = new int[5]; x[0] = 5; x[0]++; return x[0];")); - } - public void testJacksCrazyExpression1() { assertEquals(1, exec("int x; def[] y = new def[1]; x = y[0] = 1; return x;")); } diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/ListTests.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/ListTests.java new file mode 100644 index 00000000000..7b57e1d0d0a --- /dev/null +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/ListTests.java @@ -0,0 +1,70 @@ +/* + * 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.painless; + +import org.hamcrest.Matcher; + +import java.util.Arrays; + +import static org.hamcrest.Matchers.both; +import static org.hamcrest.Matchers.either; +import static org.hamcrest.Matchers.hasToString; +import static org.hamcrest.Matchers.instanceOf; + +/** Tests for working with lists. */ +public class ListTests extends ArrayLikeObjectTestCase { + @Override + protected String declType(String valueType) { + return "List"; + } + + @Override + protected String valueCtorCall(String valueType, int size) { + String[] fill = new String[size]; + Arrays.fill(fill, fillValue(valueType)); + return "[" + String.join(",", fill) + "]"; + } + + private String fillValue(String valueType) { + switch (valueType) { + case "int": return "0"; + case "long": return "0L"; + case "short": return "(short) 0"; + case "byte": return "(byte) 0"; + case "float": return "0.0f"; + case "double": return "0.0"; // Double is implicit for decimal constants + default: return null; + } + } + + @Override + protected Matcher outOfBoundsExceptionMatcher(int index, int size) { + if (index > size) { + return hasToString("java.lang.IndexOutOfBoundsException: Index: " + index + ", Size: " + size); + } else { + Matcher m = both(instanceOf(ArrayIndexOutOfBoundsException.class)) + .and(hasToString("java.lang.ArrayIndexOutOfBoundsException: " + index)); + // If we set -XX:-OmitStackTraceInFastThrow we wouldn't need this + m = either(m).or(instanceOf(ArrayIndexOutOfBoundsException.class)); + return m; + } + } + +} diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/MapTests.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/MapTests.java new file mode 100644 index 00000000000..034213e74be --- /dev/null +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/MapTests.java @@ -0,0 +1,45 @@ +/* + * 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.painless; + +import static java.util.Collections.singletonMap; + +/** Tests for working with maps. */ +public class MapTests extends ScriptTestCase { + private void mapAccessesTestCase(String listType) { + Object val = randomFrom("test", 1, 1.3, new Object()); + String decl = listType + " x = ['a': 1, 'b': 2, 0: 2, -5: 'slot', 123.1: 12]"; + assertEquals(5, exec(decl + "; return x.size()")); + assertEquals(2, exec(decl + "; return x[0];", true)); + assertEquals(1, exec(decl + "; return x['a'];", true)); + assertEquals(12, exec(decl + "; return x[123.1];", true)); + assertEquals(val, exec(decl + "; x[ 0] = params.val; return x[ 0];", singletonMap("val", val), true)); + assertEquals("slot", exec(decl + "; x[ 0] = params.val; return x[-5];", singletonMap("val", val), true)); + assertEquals(val, exec(decl + "; x[-5] = params.val; return x[-5];", singletonMap("val", val), true)); + } + + public void testMapInDefAccesses() { + mapAccessesTestCase("def"); + } + + public void testMapAccesses() { + mapAccessesTestCase("Map"); + } +} From 1bbd3c5400ebda40e299c1ba60ce51166eeed392 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Sat, 29 Oct 2016 22:19:48 -0400 Subject: [PATCH 046/132] Fix painless's out of bounds assertions in java 9 Java 9's exception message when lists have an out of bounds index is much better than java 8 but the painless code asserted on the java 8 message. Now it'll accept either. I'm tempted to weaken the assertion but I like asserting that the message is readable. --- .../painless/ArrayLikeObjectTestCase.java | 6 ++-- .../elasticsearch/painless/ArrayTests.java | 9 ++---- .../org/elasticsearch/painless/ListTests.java | 28 +++++++++++-------- 3 files changed, 22 insertions(+), 21 deletions(-) diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/ArrayLikeObjectTestCase.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/ArrayLikeObjectTestCase.java index 1be3c024710..69b40f141e2 100644 --- a/modules/lang-painless/src/test/java/org/elasticsearch/painless/ArrayLikeObjectTestCase.java +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/ArrayLikeObjectTestCase.java @@ -39,9 +39,9 @@ public abstract class ArrayLikeObjectTestCase extends ScriptTestCase { */ protected abstract String valueCtorCall(String valueType, int size); /** - * The type of the exception thrown by out of bounds accesses; + * Matcher for the message of the out of bounds exceptions thrown for too negative or too positive offsets. */ - protected abstract Matcher outOfBoundsExceptionMatcher(int index, int size); + protected abstract Matcher outOfBoundsExceptionMessageMatcher(int index, int size); private void arrayLoadStoreTestCase(boolean declareAsDef, String valueType, Object val, @Nullable Number valPlusOne) { String declType = declareAsDef ? "def" : declType(valueType); @@ -80,7 +80,7 @@ public abstract class ArrayLikeObjectTestCase extends ScriptTestCase { IndexOutOfBoundsException e = expectScriptThrows(IndexOutOfBoundsException.class, () -> exec(script, singletonMap("val", val), true)); try { - assertThat(e, outOfBoundsExceptionMatcher(index, 5)); + assertThat(e.getMessage(), outOfBoundsExceptionMessageMatcher(index, 5)); } catch (AssertionError ae) { ae.addSuppressed(e); // Mark the exception we are testing as suppressed so we get its stack trace. If it has one :( throw ae; diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/ArrayTests.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/ArrayTests.java index 350f27f44fc..fe2ee1683bb 100644 --- a/modules/lang-painless/src/test/java/org/elasticsearch/painless/ArrayTests.java +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/ArrayTests.java @@ -24,9 +24,7 @@ import org.hamcrest.Matcher; import java.lang.invoke.MethodHandle; import java.lang.invoke.MethodType; -import static org.hamcrest.Matchers.both; -import static org.hamcrest.Matchers.hasToString; -import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.equalTo; /** Tests for working with arrays. */ public class ArrayTests extends ArrayLikeObjectTestCase { @@ -41,9 +39,8 @@ public class ArrayTests extends ArrayLikeObjectTestCase { } @Override - protected Matcher outOfBoundsExceptionMatcher(int index, int size) { - return both(instanceOf(ArrayIndexOutOfBoundsException.class)) - .and(hasToString("java.lang.ArrayIndexOutOfBoundsException: " + index)); + protected Matcher outOfBoundsExceptionMessageMatcher(int index, int size) { + return equalTo(Integer.toString(index)); } public void testArrayLengthHelper() throws Throwable { diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/ListTests.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/ListTests.java index 7b57e1d0d0a..79112d8f58e 100644 --- a/modules/lang-painless/src/test/java/org/elasticsearch/painless/ListTests.java +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/ListTests.java @@ -23,10 +23,9 @@ import org.hamcrest.Matcher; import java.util.Arrays; -import static org.hamcrest.Matchers.both; -import static org.hamcrest.Matchers.either; -import static org.hamcrest.Matchers.hasToString; -import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.anyOf; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.nullValue; /** Tests for working with lists. */ public class ListTests extends ArrayLikeObjectTestCase { @@ -55,15 +54,20 @@ public class ListTests extends ArrayLikeObjectTestCase { } @Override - protected Matcher outOfBoundsExceptionMatcher(int index, int size) { - if (index > size) { - return hasToString("java.lang.IndexOutOfBoundsException: Index: " + index + ", Size: " + size); + protected Matcher outOfBoundsExceptionMessageMatcher(int index, int size) { + if ("1.8".equals(Runtime.class.getPackage().getSpecificationVersion())) { + // 1.8 and below aren't as clean as 1.9+ + if (index > size) { + return equalTo("Index: " + index + ", Size: " + size); + } else { + Matcher m = equalTo(Integer.toString(index)); + // If we set -XX:-OmitStackTraceInFastThrow we wouldn't need this + m = anyOf(m, nullValue()); + return m; + } } else { - Matcher m = both(instanceOf(ArrayIndexOutOfBoundsException.class)) - .and(hasToString("java.lang.ArrayIndexOutOfBoundsException: " + index)); - // If we set -XX:-OmitStackTraceInFastThrow we wouldn't need this - m = either(m).or(instanceOf(ArrayIndexOutOfBoundsException.class)); - return m; + // Starting with 1.9 it gets nicer + return equalTo("Index " + index + " out-of-bounds for length " + size); } } From b5f86f6f0544a9ad9fa4ab88bf7397cf361d696a Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Sun, 30 Oct 2016 00:09:09 -0400 Subject: [PATCH 047/132] Add note to getting-started doc about CONSOLE Link to the documentation for CONSOLE in getting-started so the layout of the snippet makes some sense. Relates to #21158 --- docs/reference/getting-started.asciidoc | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docs/reference/getting-started.asciidoc b/docs/reference/getting-started.asciidoc index d6b1aa12daf..5e7830ce90c 100755 --- a/docs/reference/getting-started.asciidoc +++ b/docs/reference/getting-started.asciidoc @@ -198,7 +198,10 @@ Now that we have our node (and cluster) up and running, the next step is to unde Let's start with a basic health check, which we can use to see how our cluster is doing. We'll be using curl to do this but you can use any tool that allows you to make HTTP/REST calls. Let's assume that we are still on the same node where we started Elasticsearch on and open another command shell window. -To check the cluster health, we will be using the <>. Remember previously that our node HTTP endpoint is available at port `9200`: +To check the cluster health, we will be using the <>. You can +run the command below in https://www.elastic.co/guide/en/kibana/{branch}/console-kibana.html[Kibana's Console] +by clicking "VIEW IN CONSOLE" or with `curl` by clicking the "COPY AS CURL" +link below and pasting the into a terminal. [source,js] -------------------------------------------------- From e7cfe101e4c1e71edf9089280a86f51f8c1ceb6b Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Mon, 31 Oct 2016 13:43:55 +0100 Subject: [PATCH 048/132] Retrying replication requests on replica doesn't call `onRetry` (#21189) Replication request may arrive at a replica before the replica's node has processed a required mapping update. In these cases the TransportReplicationAction will retry the request once a new cluster state arrives. Sadly that retry logic failed to call `ReplicationRequest#onRetry`, causing duplicates in the append only use case. This commit fixes this and also the test which missed the check. I also added an assertion which would have helped finding the source of the duplicates. This was discovered by https://elasticsearch-ci.elastic.co/job/elastic+elasticsearch+master+multijob-unix-compatibility/os=opensuse/174/ Relates #20211 --- .../TransportReplicationAction.java | 1 + .../index/engine/DeleteVersionValue.java | 8 ++++++ .../index/engine/InternalEngine.java | 27 +++++++++++++++++-- .../index/engine/VersionValue.java | 7 +++++ .../TransportReplicationActionTests.java | 7 +++-- 5 files changed, 46 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index 740a003ffa8..4f44f680b46 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -481,6 +481,7 @@ public abstract class TransportReplicationAction< transportReplicaAction, request), e); + request.onRetry(); final ThreadContext.StoredContext context = threadPool.getThreadContext().newStoredContext(); observer.waitForNextChange(new ClusterStateObserver.Listener() { @Override diff --git a/core/src/main/java/org/elasticsearch/index/engine/DeleteVersionValue.java b/core/src/main/java/org/elasticsearch/index/engine/DeleteVersionValue.java index baacc4b240d..b145a86e43d 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/DeleteVersionValue.java +++ b/core/src/main/java/org/elasticsearch/index/engine/DeleteVersionValue.java @@ -48,4 +48,12 @@ class DeleteVersionValue extends VersionValue { public long ramBytesUsed() { return BASE_RAM_BYTES_USED; } + + @Override + public String toString() { + return "DeleteVersionValue{" + + "version=" + version() + ", " + + "time=" + time + + '}'; + } } diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 3cc1aa26195..c450e5214c1 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -35,6 +35,7 @@ import org.apache.lucene.index.Term; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.SearcherFactory; import org.apache.lucene.search.SearcherManager; +import org.apache.lucene.search.TermQuery; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.Directory; import org.apache.lucene.store.LockObtainFailedException; @@ -484,7 +485,8 @@ public class InternalEngine extends Engine { // if anything is fishy here ie. there is a retry we go and force updateDocument below so we are updating the document in the // lucene index without checking the version map but we still do the version check final boolean forceUpdateDocument; - if (canOptimizeAddDocument(index)) { + final boolean canOptimizeAddDocument = canOptimizeAddDocument(index); + if (canOptimizeAddDocument) { long deOptimizeTimestamp = maxUnsafeAutoIdTimestamp.get(); if (index.isRetry()) { forceUpdateDocument = true; @@ -523,7 +525,8 @@ public class InternalEngine extends Engine { final long updatedVersion = updateVersion(index, currentVersion, expectedVersion); index.setCreated(deleted); if (currentVersion == Versions.NOT_FOUND && forceUpdateDocument == false) { - // document does not exists, we can optimize for create + // document does not exists, we can optimize for create, but double check if assertions are running + assert assertDocDoesNotExist(index, canOptimizeAddDocument == false); index(index, indexWriter); } else { update(index, indexWriter); @@ -532,6 +535,26 @@ public class InternalEngine extends Engine { } } + /** + * Asserts that the doc in the index operation really doesn't exist + */ + private boolean assertDocDoesNotExist(final Index index, final boolean allowDeleted) throws IOException { + final VersionValue versionValue = versionMap.getUnderLock(index.uid()); + if (versionValue != null) { + if (versionValue.delete() == false || allowDeleted == false) { + throw new AssertionError("doc [" + index.type() + "][" + index.id() + "] exists in version map (version " + versionValue + ")"); + } + } else { + try (final Searcher searcher = acquireSearcher("assert doc doesn't exist")) { + final long docsWithId = searcher.searcher().count(new TermQuery(index.uid())); + if (docsWithId > 0) { + throw new AssertionError("doc [" + index.type() + "][" + index.id() + "] exists [" + docsWithId + "] times in index"); + } + } + } + return true; + } + private long updateVersion(Engine.Operation op, long currentVersion, long expectedVersion) { final long updatedVersion = op.versionType().updateVersion(currentVersion, expectedVersion); op.updateVersion(updatedVersion); diff --git a/core/src/main/java/org/elasticsearch/index/engine/VersionValue.java b/core/src/main/java/org/elasticsearch/index/engine/VersionValue.java index 662c88df5d9..5258b270091 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/VersionValue.java +++ b/core/src/main/java/org/elasticsearch/index/engine/VersionValue.java @@ -57,4 +57,11 @@ class VersionValue implements Accountable { public Collection getChildResources() { return Collections.emptyList(); } + + @Override + public String toString() { + return "VersionValue{" + + "version=" + version + + '}'; + } } diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java b/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java index 3deae74f455..1caac899005 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java @@ -867,8 +867,11 @@ public class TransportReplicationActionTests extends ESTestCase { final CapturingTransport.CapturedRequest capturedRequest = capturedRequests.get(0); assertThat(capturedRequest.action, equalTo("testActionWithExceptions[r]")); assertThat(capturedRequest.request, instanceOf(TransportReplicationAction.ConcreteShardRequest.class)); - assertThat(((TransportReplicationAction.ConcreteShardRequest) capturedRequest.request).getRequest(), equalTo(request)); - assertThat(((TransportReplicationAction.ConcreteShardRequest) capturedRequest.request).getTargetAllocationID(), + final TransportReplicationAction.ConcreteShardRequest concreteShardRequest = + (TransportReplicationAction.ConcreteShardRequest) capturedRequest.request; + assertThat(concreteShardRequest.getRequest(), equalTo(request)); + assertThat(concreteShardRequest.getRequest().isRetrySet.get(), equalTo(true)); + assertThat(concreteShardRequest.getTargetAllocationID(), equalTo(replica.allocationId().getId())); } From 37228f924a9f32adcda8a97a7efd915f787d835a Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Mon, 31 Oct 2016 14:04:18 +0100 Subject: [PATCH 049/132] [TEST] Use assertBusy to check assertMaster property in presence of a low publish timeout The assertion assertMaster checks if all nodes have each other in the cluster state and the correct master set. It is usually called after a disruption has been healed and ensureStableCluster been called. In presence of a low publish timeout of 1s in this test class, publishing might not be fully done even after ensureStableCluster returns. This commit adds an assertBusy to assertMaster so that the node has a bit more time to apply the cluster state from the master, even if it's a bit slow. --- .../DiscoveryWithServiceDisruptionsIT.java | 28 ++++++++----------- 1 file changed, 11 insertions(+), 17 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java b/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java index 2ca9f6f3384..5af2e1c6624 100644 --- a/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java +++ b/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java @@ -761,15 +761,7 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase { oldMasterNodeSteppedDown.await(30, TimeUnit.SECONDS); // Make sure that the end state is consistent on all nodes: assertDiscoveryCompleted(nodes); - // Use assertBusy(...) because the unfrozen node may take a while to actually join the cluster. - // The assertDiscoveryCompleted(...) can't know if all nodes have the old master node in all of the local cluster states - assertBusy(new Runnable() { - @Override - public void run() { - assertMaster(newMasterNode, nodes); - } - }); - + assertMaster(newMasterNode, nodes); assertThat(masters.size(), equalTo(2)); for (Map.Entry>> entry : masters.entrySet()) { @@ -1366,14 +1358,16 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase { }, 10, TimeUnit.SECONDS); } - private void assertMaster(String masterNode, List nodes) { - for (String node : nodes) { - ClusterState state = getNodeClusterState(node); - String failMsgSuffix = "cluster_state:\n" + state.prettyPrint(); - assertThat("wrong node count on [" + node + "]. " + failMsgSuffix, state.nodes().getSize(), equalTo(nodes.size())); - String otherMasterNodeName = state.nodes().getMasterNode() != null ? state.nodes().getMasterNode().getName() : null; - assertThat("wrong master on node [" + node + "]. " + failMsgSuffix, otherMasterNodeName, equalTo(masterNode)); - } + private void assertMaster(String masterNode, List nodes) throws Exception { + assertBusy(() -> { + for (String node : nodes) { + ClusterState state = getNodeClusterState(node); + String failMsgSuffix = "cluster_state:\n" + state.prettyPrint(); + assertThat("wrong node count on [" + node + "]. " + failMsgSuffix, state.nodes().getSize(), equalTo(nodes.size())); + String otherMasterNodeName = state.nodes().getMasterNode() != null ? state.nodes().getMasterNode().getName() : null; + assertThat("wrong master on node [" + node + "]. " + failMsgSuffix, otherMasterNodeName, equalTo(masterNode)); + } + }); } private void assertDiscoveryCompleted(List nodes) throws InterruptedException { From d7d5909e69982287a550b5427747ea610d9d6805 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Mon, 31 Oct 2016 15:09:43 +0100 Subject: [PATCH 050/132] Disconnect from newly added nodes if cluster state publishing fails (#21197) Before publishing a cluster state the master connects to the nodes that are added in the cluster state. When publishing fails, however, it does not disconnect from these nodes, leaving NodeConnectionsService out of sync with the currently applied cluster state. --- .../cluster/NodeConnectionsService.java | 9 +-- .../cluster/service/ClusterService.java | 6 +- .../cluster/NodeConnectionsServiceTests.java | 10 +-- .../cluster/service/ClusterServiceTests.java | 70 ++++++++++++++++++- .../test/ClusterServiceUtils.java | 5 +- 5 files changed, 85 insertions(+), 15 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/cluster/NodeConnectionsService.java b/core/src/main/java/org/elasticsearch/cluster/NodeConnectionsService.java index 6a9d0ae160f..94333c10dde 100644 --- a/core/src/main/java/org/elasticsearch/cluster/NodeConnectionsService.java +++ b/core/src/main/java/org/elasticsearch/cluster/NodeConnectionsService.java @@ -36,6 +36,7 @@ import org.elasticsearch.discovery.zen.NodesFaultDetection; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.util.List; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ScheduledFuture; @@ -75,10 +76,10 @@ public class NodeConnectionsService extends AbstractLifecycleComponent { this.reconnectInterval = NodeConnectionsService.CLUSTER_NODE_RECONNECT_INTERVAL_SETTING.get(settings); } - public void connectToAddedNodes(ClusterChangedEvent event) { + public void connectToNodes(List addedNodes) { // TODO: do this in parallel (and wait) - for (final DiscoveryNode node : event.nodesDelta().addedNodes()) { + for (final DiscoveryNode node : addedNodes) { try (Releasable ignored = nodeLocks.acquire(node)) { Integer current = nodes.put(node, 0); assert current == null : "node " + node + " was added in event but already in internal nodes"; @@ -87,8 +88,8 @@ public class NodeConnectionsService extends AbstractLifecycleComponent { } } - public void disconnectFromRemovedNodes(ClusterChangedEvent event) { - for (final DiscoveryNode node : event.nodesDelta().removedNodes()) { + public void disconnectFromNodes(List removedNodes) { + for (final DiscoveryNode node : removedNodes) { try (Releasable ignored = nodeLocks.acquire(node)) { Integer current = nodes.remove(node); assert current != null : "node " + node + " was removed in event but not in internal nodes"; diff --git a/core/src/main/java/org/elasticsearch/cluster/service/ClusterService.java b/core/src/main/java/org/elasticsearch/cluster/service/ClusterService.java index 38d42a0b0f9..fe2adf1f4c6 100644 --- a/core/src/main/java/org/elasticsearch/cluster/service/ClusterService.java +++ b/core/src/main/java/org/elasticsearch/cluster/service/ClusterService.java @@ -671,7 +671,7 @@ public class ClusterService extends AbstractLifecycleComponent { } } - nodeConnectionsService.connectToAddedNodes(clusterChangedEvent); + nodeConnectionsService.connectToNodes(clusterChangedEvent.nodesDelta().addedNodes()); // if we are the master, publish the new state to all nodes // we publish here before we send a notification to all the listeners, since if it fails @@ -686,6 +686,8 @@ public class ClusterService extends AbstractLifecycleComponent { (Supplier) () -> new ParameterizedMessage( "failing [{}]: failed to commit cluster state version [{}]", tasksSummary, version), t); + // ensure that list of connected nodes in NodeConnectionsService is in-sync with the nodes of the current cluster state + nodeConnectionsService.disconnectFromNodes(clusterChangedEvent.nodesDelta().addedNodes()); proccessedListeners.forEach(task -> task.listener.onFailure(task.source, t)); return; } @@ -711,7 +713,7 @@ public class ClusterService extends AbstractLifecycleComponent { } } - nodeConnectionsService.disconnectFromRemovedNodes(clusterChangedEvent); + nodeConnectionsService.disconnectFromNodes(clusterChangedEvent.nodesDelta().removedNodes()); newClusterState.status(ClusterState.ClusterStateStatus.APPLIED); diff --git a/core/src/test/java/org/elasticsearch/cluster/NodeConnectionsServiceTests.java b/core/src/test/java/org/elasticsearch/cluster/NodeConnectionsServiceTests.java index 599b62b1ee2..863349e897a 100644 --- a/core/src/test/java/org/elasticsearch/cluster/NodeConnectionsServiceTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/NodeConnectionsServiceTests.java @@ -84,19 +84,19 @@ public class NodeConnectionsServiceTests extends ESTestCase { ClusterState current = clusterStateFromNodes(Collections.emptyList()); ClusterChangedEvent event = new ClusterChangedEvent("test", clusterStateFromNodes(randomSubsetOf(nodes)), current); - service.connectToAddedNodes(event); + service.connectToNodes(event.nodesDelta().addedNodes()); assertConnected(event.nodesDelta().addedNodes()); - service.disconnectFromRemovedNodes(event); + service.disconnectFromNodes(event.nodesDelta().removedNodes()); assertConnectedExactlyToNodes(event.state()); current = event.state(); event = new ClusterChangedEvent("test", clusterStateFromNodes(randomSubsetOf(nodes)), current); - service.connectToAddedNodes(event); + service.connectToNodes(event.nodesDelta().addedNodes()); assertConnected(event.nodesDelta().addedNodes()); - service.disconnectFromRemovedNodes(event); + service.disconnectFromNodes(event.nodesDelta().removedNodes()); assertConnectedExactlyToNodes(event.state()); } @@ -110,7 +110,7 @@ public class NodeConnectionsServiceTests extends ESTestCase { transport.randomConnectionExceptions = true; - service.connectToAddedNodes(event); + service.connectToNodes(event.nodesDelta().addedNodes()); for (int i = 0; i < 3; i++) { // simulate disconnects diff --git a/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceTests.java b/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceTests.java index c12b54e71ef..a39bcf38391 100644 --- a/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceTests.java @@ -41,6 +41,8 @@ import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.set.Sets; +import org.elasticsearch.discovery.Discovery; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.MockLogAppender; import org.elasticsearch.test.junit.annotations.TestLogging; @@ -119,12 +121,12 @@ public class ClusterServiceTests extends ESTestCase { emptySet(), Version.CURRENT)); timedClusterService.setNodeConnectionsService(new NodeConnectionsService(Settings.EMPTY, null, null) { @Override - public void connectToAddedNodes(ClusterChangedEvent event) { + public void connectToNodes(List addedNodes) { // skip } @Override - public void disconnectFromRemovedNodes(ClusterChangedEvent event) { + public void disconnectFromNodes(List removedNodes) { // skip } }); @@ -970,6 +972,70 @@ public class ClusterServiceTests extends ESTestCase { mockAppender.assertAllExpectationsMatched(); } + public void testDisconnectFromNewlyAddedNodesIfClusterStatePublishingFails() throws InterruptedException { + TimedClusterService timedClusterService = new TimedClusterService(Settings.builder().put("cluster.name", + "ClusterServiceTests").build(), new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), + threadPool); + timedClusterService.setLocalNode(new DiscoveryNode("node1", buildNewFakeTransportAddress(), emptyMap(), + emptySet(), Version.CURRENT)); + Set currentNodes = Collections.synchronizedSet(new HashSet<>()); + currentNodes.add(timedClusterService.localNode()); + timedClusterService.setNodeConnectionsService(new NodeConnectionsService(Settings.EMPTY, null, null) { + @Override + public void connectToNodes(List addedNodes) { + currentNodes.addAll(addedNodes); + } + + @Override + public void disconnectFromNodes(List removedNodes) { + currentNodes.removeAll(removedNodes); + } + }); + AtomicBoolean failToCommit = new AtomicBoolean(); + timedClusterService.setClusterStatePublisher((event, ackListener) -> { + if (failToCommit.get()) { + throw new Discovery.FailedToCommitClusterStateException("just to test this"); + } + }); + timedClusterService.start(); + ClusterState state = timedClusterService.state(); + final DiscoveryNodes nodes = state.nodes(); + final DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(nodes) + .masterNodeId(nodes.getLocalNodeId()); + state = ClusterState.builder(state).blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK) + .nodes(nodesBuilder).build(); + setState(timedClusterService, state); + + assertThat(currentNodes, equalTo(Sets.newHashSet(timedClusterService.state().getNodes()))); + + final CountDownLatch latch = new CountDownLatch(1); + + // try to add node when cluster state publishing fails + failToCommit.set(true); + timedClusterService.submitStateUpdateTask("test", new ClusterStateUpdateTask() { + @Override + public ClusterState execute(ClusterState currentState) throws Exception { + DiscoveryNode newNode = new DiscoveryNode("node2", buildNewFakeTransportAddress(), emptyMap(), + emptySet(), Version.CURRENT); + return ClusterState.builder(currentState).nodes(DiscoveryNodes.builder(currentState.nodes()).add(newNode)).build(); + } + + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + latch.countDown(); + } + + @Override + public void onFailure(String source, Exception e) { + latch.countDown(); + } + }); + + latch.await(); + assertThat(currentNodes, equalTo(Sets.newHashSet(timedClusterService.state().getNodes()))); + timedClusterService.close(); + } + private static class SimpleTask { private final int id; diff --git a/test/framework/src/main/java/org/elasticsearch/test/ClusterServiceUtils.java b/test/framework/src/main/java/org/elasticsearch/test/ClusterServiceUtils.java index f247c56636c..3e3896dfc2c 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ClusterServiceUtils.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ClusterServiceUtils.java @@ -34,6 +34,7 @@ import org.elasticsearch.threadpool.ThreadPool; import java.util.Arrays; import java.util.Collections; import java.util.HashSet; +import java.util.List; import java.util.concurrent.CountDownLatch; import static junit.framework.TestCase.fail; @@ -53,12 +54,12 @@ public class ClusterServiceUtils { clusterService.setLocalNode(localNode); clusterService.setNodeConnectionsService(new NodeConnectionsService(Settings.EMPTY, null, null) { @Override - public void connectToAddedNodes(ClusterChangedEvent event) { + public void connectToNodes(List addedNodes) { // skip } @Override - public void disconnectFromRemovedNodes(ClusterChangedEvent event) { + public void disconnectFromNodes(List removedNodes) { // skip } }); From 44af41ed55bf8d7ffcb88eaf96d86f9527d30a70 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 31 Oct 2016 10:10:37 -0400 Subject: [PATCH 051/132] Make explicit missing settings for Windows service When installing the Windows service, certain settings like the minimum heap, maximum heap and thread stack size setting must be set. While there is an error message making mention of this fact, the error message is not explicit exactly what setting needs to be set. This commit makes these settings explicit. Relates #21200 --- .../src/main/resources/bin/elasticsearch-service.bat | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/distribution/src/main/resources/bin/elasticsearch-service.bat b/distribution/src/main/resources/bin/elasticsearch-service.bat index 609b8bda846..f2aa5c3e3de 100644 --- a/distribution/src/main/resources/bin/elasticsearch-service.bat +++ b/distribution/src/main/resources/bin/elasticsearch-service.bat @@ -209,15 +209,15 @@ for %%a in ("%ES_JAVA_OPTS:;=","%") do ( @endlocal & set JVM_MS=%JVM_MS% & set JVM_MX=%JVM_MX% & set JVM_SS=%JVM_SS% if "%JVM_MS%" == "" ( - echo minimum heap size not set; configure via %ES_JVM_OPTIONS% or ES_JAVA_OPTS + echo minimum heap size not set; configure using -Xms via %ES_JVM_OPTIONS% or ES_JAVA_OPTS goto:eof ) if "%JVM_MX%" == "" ( - echo maximum heap size not set; configure via %ES_JVM_OPTIONS% or ES_JAVA_OPTS + echo maximum heap size not set; configure using -Xmx via %ES_JVM_OPTIONS% or ES_JAVA_OPTS goto:eof ) if "%JVM_SS%" == "" ( - echo thread stack size not set; configure via %ES_JVM_OPTIONS% or ES_JAVA_OPTS + echo thread stack size not set; configure using -Xss via %ES_JVM_OPTIONS% or ES_JAVA_OPTS goto:eof ) From e96fc3aa9f895a575e6026ee029945176ef3dd35 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 31 Oct 2016 15:27:22 +0100 Subject: [PATCH 052/132] Add a flag to use minimim_master_nodes on the integ test cluster (#21173) Setting `discovery.initial_state_timeout: 0s` to make `discovery.zen.minimum_master_nodes: N` work reliably can cause issues in clusters that rely on state recovery once the cluster is available. This change makes the use or `discovery.zen.minimum_master_nodes` optional for clusters where this behavior is desirable. --- .../gradle/test/ClusterConfiguration.groovy | 13 +++++++++++-- .../gradle/test/ClusterFormationTasks.groovy | 2 +- 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterConfiguration.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterConfiguration.groovy index 48183a07721..36828f1cb86 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterConfiguration.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterConfiguration.groovy @@ -62,6 +62,15 @@ class ClusterConfiguration { @Input boolean debug = false + /** + * if true each node will be configured with discovery.zen.minimum_master_nodes set + * to the total number of nodes in the cluster. This will also cause that each node has `0s` state recovery + * timeout which can lead to issues if for instance an existing clusterstate is expected to be recovered + * before any tests start + */ + @Input + boolean useMinimumMasterNodes = true + @Input String jvmArgs = "-Xms" + System.getProperty('tests.heap.size', '512m') + " " + "-Xmx" + System.getProperty('tests.heap.size', '512m') + @@ -95,11 +104,11 @@ class ClusterConfiguration { @Input Closure waitCondition = { NodeInfo node, AntBuilder ant -> File tmpFile = new File(node.cwd, 'wait.success') - ant.echo("==> [${new Date()}] checking health: http://${node.httpUri()}/_cluster/health?wait_for_nodes=>=${numNodes}") + ant.echo("==> [${new Date()}] checking health: http://${node.httpUri()}/_cluster/health?wait_for_nodes=>=${numNodes}&wait_for_status=yellow") // checking here for wait_for_nodes to be >= the number of nodes because its possible // this cluster is attempting to connect to nodes created by another task (same cluster name), // so there will be more nodes in that case in the cluster state - ant.get(src: "http://${node.httpUri()}/_cluster/health?wait_for_nodes=>=${numNodes}", + ant.get(src: "http://${node.httpUri()}/_cluster/health?wait_for_nodes=>=${numNodes}&wait_for_status=yellow", dest: tmpFile.toString(), ignoreerrors: true, // do not fail on error, so logging buffers can be flushed by the wait task retries: 10) diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy index d3b08f7339a..2095c892f50 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy @@ -259,7 +259,7 @@ class ClusterFormationTasks { // basically skip initial state recovery to allow the cluster to form using a realistic master election // this means all nodes must be up, join the seed node and do a master election. This will also allow new and // old nodes in the BWC case to become the master - if (node.config.numNodes > 1) { + if (node.config.useMinimumMasterNodes && node.config.numNodes > 1) { esConfig['discovery.zen.minimum_master_nodes'] = node.config.numNodes esConfig['discovery.initial_state_timeout'] = '0s' // don't wait for state.. just start up quickly } From 7738af27e86b9dd037be5ffe0741434d1dadad06 Mon Sep 17 00:00:00 2001 From: Stanislav Mamontov Date: Mon, 31 Oct 2016 18:13:21 +0300 Subject: [PATCH 053/132] Fix malformed JSON in Delete API example (#21168) Obviously, there should be "result": "deleted" instead of "result: deleted" --- docs/reference/docs/delete.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/docs/delete.asciidoc b/docs/reference/docs/delete.asciidoc index a130e66a191..dd5c1de1485 100644 --- a/docs/reference/docs/delete.asciidoc +++ b/docs/reference/docs/delete.asciidoc @@ -26,7 +26,7 @@ The result of the above delete operation is: "_type" : "tweet", "_id" : "1", "_version" : 2, - "result: deleted" + "result": "deleted" } -------------------------------------------------- From 6a8bad8a06aeebf4acaf2db0270e02b2939237d5 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Mon, 31 Oct 2016 09:15:36 -0600 Subject: [PATCH 054/132] [DOCS] Document all date formats (#21164) Resolves #21046 --- docs/reference/mapping/params/format.asciidoc | 27 ++++++++++--------- 1 file changed, 15 insertions(+), 12 deletions(-) diff --git a/docs/reference/mapping/params/format.asciidoc b/docs/reference/mapping/params/format.asciidoc index 905a7ffe90a..3619017141f 100644 --- a/docs/reference/mapping/params/format.asciidoc +++ b/docs/reference/mapping/params/format.asciidoc @@ -147,17 +147,18 @@ The following tables lists all the defaults ISO formats supported: `date_hour` or `strict_date_hour`:: - A formatter that combines a full date and two digit hour of day. + A formatter that combines a full date and two digit hour of day: + `yyyy-MM-dd'T'HH`. `date_hour_minute` or `strict_date_hour_minute`:: A formatter that combines a full date, two digit hour of day, and two - digit minute of hour. + digit minute of hour: `yyyy-MM-dd'T'HH:mm`. `date_hour_minute_second` or `strict_date_hour_minute_second`:: A formatter that combines a full date, two digit hour of day, two digit - minute of hour, and two digit second of minute. + minute of hour, and two digit second of minute: `yyyy-MM-dd'T'HH:mm:ss`. `date_hour_minute_second_fraction` or `strict_date_hour_minute_second_fraction`:: @@ -183,16 +184,17 @@ The following tables lists all the defaults ISO formats supported: `hour` or `strict_hour`:: - A formatter for a two digit hour of day. + A formatter for a two digit hour of day: `HH` `hour_minute` or `strict_hour_minute`:: - A formatter for a two digit hour of day and two digit minute of hour. + A formatter for a two digit hour of day and two digit minute of hour: + `HH:mm`. `hour_minute_second` or `strict_hour_minute_second`:: A formatter for a two digit hour of day, two digit minute of hour, and two - digit second of minute. + digit second of minute: `HH:mm:ss`. `hour_minute_second_fraction` or `strict_hour_minute_second_fraction`:: @@ -258,27 +260,28 @@ The following tables lists all the defaults ISO formats supported: `weekyear` or `strict_weekyear`:: - A formatter for a four digit weekyear. + A formatter for a four digit weekyear: `xxxx`. `weekyear_week` or `strict_weekyear_week`:: - A formatter for a four digit weekyear and two digit week of weekyear. + A formatter for a four digit weekyear and two digit week of weekyear: + `xxxx-'W'ww`. `weekyear_week_day` or `strict_weekyear_week_day`:: A formatter for a four digit weekyear, two digit week of weekyear, and one - digit day of week. + digit day of week: `xxxx-'W'ww-e`. `year` or `strict_year`:: - A formatter for a four digit year. + A formatter for a four digit year: `yyyy`. `year_month` or `strict_year_month`:: - A formatter for a four digit year and two digit month of year. + A formatter for a four digit year and two digit month of year: `yyyy-MM`. `year_month_day` or `strict_year_month_day`:: A formatter for a four digit year, two digit month of year, and two digit - day of month. + day of month: `yyyy-MM-dd`. From a23ded6a94697adad66b79fcb61e64ddc965926c Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Mon, 31 Oct 2016 16:18:58 +0100 Subject: [PATCH 055/132] [TEST] Fix NullPointerException in AzureStorageServiceMock Makes the code safe against concurrent modifications of the underlying hashmap. --- .../cloud/azure/storage/AzureStorageServiceMock.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/plugins/repository-azure/src/test/java/org/elasticsearch/cloud/azure/storage/AzureStorageServiceMock.java b/plugins/repository-azure/src/test/java/org/elasticsearch/cloud/azure/storage/AzureStorageServiceMock.java index 51b5eae57ae..ba2011c276e 100644 --- a/plugins/repository-azure/src/test/java/org/elasticsearch/cloud/azure/storage/AzureStorageServiceMock.java +++ b/plugins/repository-azure/src/test/java/org/elasticsearch/cloud/azure/storage/AzureStorageServiceMock.java @@ -43,7 +43,7 @@ import java.util.concurrent.ConcurrentHashMap; */ public class AzureStorageServiceMock extends AbstractComponent implements AzureStorageService { - protected Map blobs = new ConcurrentHashMap<>(); + protected final Map blobs = new ConcurrentHashMap<>(); public AzureStorageServiceMock() { super(Settings.EMPTY); @@ -94,7 +94,7 @@ public class AzureStorageServiceMock extends AbstractComponent implements AzureS @Override public Map listBlobsByPrefix(String account, LocationMode mode, String container, String keyPath, String prefix) { MapBuilder blobsBuilder = MapBuilder.newMapBuilder(); - for (String blobName : blobs.keySet()) { + blobs.forEach((String blobName, ByteArrayOutputStream bos) -> { final String checkBlob; if (keyPath != null && !keyPath.isEmpty()) { // strip off key path from the beginning of the blob name @@ -103,9 +103,9 @@ public class AzureStorageServiceMock extends AbstractComponent implements AzureS checkBlob = blobName; } if (prefix == null || startsWithIgnoreCase(checkBlob, prefix)) { - blobsBuilder.put(blobName, new PlainBlobMetaData(checkBlob, blobs.get(blobName).size())); + blobsBuilder.put(blobName, new PlainBlobMetaData(checkBlob, bos.size())); } - } + }); return blobsBuilder.immutableMap(); } From a9b0b977039aee7a5870503cd79624149e45efb2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Thu, 27 Oct 2016 22:53:52 +0200 Subject: [PATCH 056/132] Expose Lucenes Ukrainian analyzer Since Lucene 6.2. the UkrainianMorfologikAnalyzer is available through the lucene-analyzers-morfologik jar. This change exposes it to be used as an elasticsearch plugin. --- plugins/analysis-ukrainian/build.gradle | 39 ++ .../licenses/lucene-LICENSE.txt | 475 ++++++++++++++++++ .../licenses/lucene-NOTICE.txt | 191 +++++++ ...lucene-analyzers-morfologik-6.2.0.jar.sha1 | 1 + .../licenses/morfologik-fsa-2.1.0.jar.sha1 | 1 + .../morfologik-stemming-2.1.0.jar.sha1 | 1 + .../analysis/UkrainianAnalyzerProvider.java | 47 ++ .../ukrainian/AnalysisUkrainianPlugin.java | 39 ++ .../SimpleUkrainianAnalyzerTests.java | 56 +++ .../analysis/UkrainianAnalysisTests.java | 43 ++ .../UkrainianClientYamlTestSuiteIT.java | 42 ++ .../test/analysis_ukrainian/10_basic.yaml | 18 + .../test/analysis_ukrainian/20_search.yaml | 32 ++ settings.gradle | 1 + 14 files changed, 986 insertions(+) create mode 100644 plugins/analysis-ukrainian/build.gradle create mode 100644 plugins/analysis-ukrainian/licenses/lucene-LICENSE.txt create mode 100644 plugins/analysis-ukrainian/licenses/lucene-NOTICE.txt create mode 100644 plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-6.2.0.jar.sha1 create mode 100644 plugins/analysis-ukrainian/licenses/morfologik-fsa-2.1.0.jar.sha1 create mode 100644 plugins/analysis-ukrainian/licenses/morfologik-stemming-2.1.0.jar.sha1 create mode 100644 plugins/analysis-ukrainian/src/main/java/org/elasticsearch/index/analysis/UkrainianAnalyzerProvider.java create mode 100644 plugins/analysis-ukrainian/src/main/java/org/elasticsearch/plugin/analysis/ukrainian/AnalysisUkrainianPlugin.java create mode 100644 plugins/analysis-ukrainian/src/test/java/org/elasticsearch/index/analysis/SimpleUkrainianAnalyzerTests.java create mode 100644 plugins/analysis-ukrainian/src/test/java/org/elasticsearch/index/analysis/UkrainianAnalysisTests.java create mode 100644 plugins/analysis-ukrainian/src/test/java/org/elasticsearch/index/analysis/UkrainianClientYamlTestSuiteIT.java create mode 100644 plugins/analysis-ukrainian/src/test/resources/rest-api-spec/test/analysis_ukrainian/10_basic.yaml create mode 100644 plugins/analysis-ukrainian/src/test/resources/rest-api-spec/test/analysis_ukrainian/20_search.yaml diff --git a/plugins/analysis-ukrainian/build.gradle b/plugins/analysis-ukrainian/build.gradle new file mode 100644 index 00000000000..b3c5473a2ff --- /dev/null +++ b/plugins/analysis-ukrainian/build.gradle @@ -0,0 +1,39 @@ +/* + * 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. + */ + +esplugin { + description 'The Ukrainian Analysis plugin integrates the Lucene UkrainianMorfologikAnalyzer into elasticsearch.' + classname 'org.elasticsearch.plugin.analysis.ukrainian.AnalysisUkrainianPlugin' +} + +dependencies { + compile "org.apache.lucene:lucene-analyzers-morfologik:${versions.lucene}" + compile "org.carrot2:morfologik-stemming:2.1.0" + compile "org.carrot2:morfologik-fsa:2.1.0" +} + +dependencyLicenses { + mapping from: /lucene-.*/, to: 'lucene' + mapping from: /morfologik-.*/, to: 'lucene' +} + +thirdPartyAudit.excludes = [ + // we don't use the morfologik-fsa polish stemmer + 'morfologik.stemming.polish.PolishStemmer' +] diff --git a/plugins/analysis-ukrainian/licenses/lucene-LICENSE.txt b/plugins/analysis-ukrainian/licenses/lucene-LICENSE.txt new file mode 100644 index 00000000000..28b134f5f8e --- /dev/null +++ b/plugins/analysis-ukrainian/licenses/lucene-LICENSE.txt @@ -0,0 +1,475 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + + + +Some code in core/src/java/org/apache/lucene/util/UnicodeUtil.java was +derived from unicode conversion examples available at +http://www.unicode.org/Public/PROGRAMS/CVTUTF. Here is the copyright +from those sources: + +/* + * Copyright 2001-2004 Unicode, Inc. + * + * Disclaimer + * + * This source code is provided as is by Unicode, Inc. No claims are + * made as to fitness for any particular purpose. No warranties of any + * kind are expressed or implied. The recipient agrees to determine + * applicability of information provided. If this file has been + * purchased on magnetic or optical media from Unicode, Inc., the + * sole remedy for any claim will be exchange of defective media + * within 90 days of receipt. + * + * Limitations on Rights to Redistribute This Code + * + * Unicode, Inc. hereby grants the right to freely use the information + * supplied in this file in the creation of products supporting the + * Unicode Standard, and to make copies of this file in any form + * for internal or external distribution as long as this notice + * remains attached. + */ + + +Some code in core/src/java/org/apache/lucene/util/ArrayUtil.java was +derived from Python 2.4.2 sources available at +http://www.python.org. Full license is here: + + http://www.python.org/download/releases/2.4.2/license/ + +Some code in core/src/java/org/apache/lucene/util/UnicodeUtil.java was +derived from Python 3.1.2 sources available at +http://www.python.org. Full license is here: + + http://www.python.org/download/releases/3.1.2/license/ + +Some code in core/src/java/org/apache/lucene/util/automaton was +derived from Brics automaton sources available at +www.brics.dk/automaton/. Here is the copyright from those sources: + +/* + * Copyright (c) 2001-2009 Anders Moeller + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions + * are met: + * 1. Redistributions of source code must retain the above copyright + * notice, this list of conditions and the following disclaimer. + * 2. Redistributions in binary form must reproduce the above copyright + * notice, this list of conditions and the following disclaimer in the + * documentation and/or other materials provided with the distribution. + * 3. The name of the author may not be used to endorse or promote products + * derived from this software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR + * IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES + * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. + * IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT, + * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT + * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF + * THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + */ + +The levenshtein automata tables in core/src/java/org/apache/lucene/util/automaton +were automatically generated with the moman/finenight FSA package. +Here is the copyright for those sources: + +# Copyright (c) 2010, Jean-Philippe Barrette-LaPierre, +# +# Permission is hereby granted, free of charge, to any person +# obtaining a copy of this software and associated documentation +# files (the "Software"), to deal in the Software without +# restriction, including without limitation the rights to use, +# copy, modify, merge, publish, distribute, sublicense, and/or sell +# copies of the Software, and to permit persons to whom the +# Software is furnished to do so, subject to the following +# conditions: +# +# The above copyright notice and this permission notice shall be +# included in all copies or substantial portions of the Software. +# +# THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +# EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES +# OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +# NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT +# HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, +# WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +# FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR +# OTHER DEALINGS IN THE SOFTWARE. + +Some code in core/src/java/org/apache/lucene/util/UnicodeUtil.java was +derived from ICU (http://www.icu-project.org) +The full license is available here: + http://source.icu-project.org/repos/icu/icu/trunk/license.html + +/* + * Copyright (C) 1999-2010, International Business Machines + * Corporation and others. All Rights Reserved. + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, and/or sell copies of the + * Software, and to permit persons to whom the Software is furnished to do so, + * provided that the above copyright notice(s) and this permission notice appear + * in all copies of the Software and that both the above copyright notice(s) and + * this permission notice appear in supporting documentation. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT OF THIRD PARTY RIGHTS. + * IN NO EVENT SHALL THE COPYRIGHT HOLDER OR HOLDERS INCLUDED IN THIS NOTICE BE + * LIABLE FOR ANY CLAIM, OR ANY SPECIAL INDIRECT OR CONSEQUENTIAL DAMAGES, OR + * ANY DAMAGES WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER + * IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT + * OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. + * + * Except as contained in this notice, the name of a copyright holder shall not + * be used in advertising or otherwise to promote the sale, use or other + * dealings in this Software without prior written authorization of the + * copyright holder. + */ + +The following license applies to the Snowball stemmers: + +Copyright (c) 2001, Dr Martin Porter +Copyright (c) 2002, Richard Boulton +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + + * Redistributions of source code must retain the above copyright notice, + * this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright + * notice, this list of conditions and the following disclaimer in the + * documentation and/or other materials provided with the distribution. + * Neither the name of the copyright holders nor the names of its contributors + * may be used to endorse or promote products derived from this software + * without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, +OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +The following license applies to the KStemmer: + +Copyright © 2003, +Center for Intelligent Information Retrieval, +University of Massachusetts, Amherst. +All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, +are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this +list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright notice, +this list of conditions and the following disclaimer in the documentation +and/or other materials provided with the distribution. + +3. The names "Center for Intelligent Information Retrieval" and +"University of Massachusetts" must not be used to endorse or promote products +derived from this software without prior written permission. To obtain +permission, contact info@ciir.cs.umass.edu. + +THIS SOFTWARE IS PROVIDED BY UNIVERSITY OF MASSACHUSETTS AND OTHER CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, +THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDERS OR CONTRIBUTORS BE +LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR +CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE +GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) +HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT +LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY +OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF +SUCH DAMAGE. + +The following license applies to the Morfologik project: + +Copyright (c) 2006 Dawid Weiss +Copyright (c) 2007-2011 Dawid Weiss, Marcin Miłkowski +All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, +are permitted provided that the following conditions are met: + + * Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + + * Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + + * Neither the name of Morfologik nor the names of its contributors + may be used to endorse or promote products derived from this software + without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON +ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +--- + +The dictionary comes from Morfologik project. Morfologik uses data from +Polish ispell/myspell dictionary hosted at http://www.sjp.pl/slownik/en/ and +is licenced on the terms of (inter alia) LGPL and Creative Commons +ShareAlike. The part-of-speech tags were added in Morfologik project and +are not found in the data from sjp.pl. The tagset is similar to IPI PAN +tagset. + +--- + +The following license applies to the Morfeusz project, +used by org.apache.lucene.analysis.morfologik. + +BSD-licensed dictionary of Polish (SGJP) +http://sgjp.pl/morfeusz/ + +Copyright © 2011 Zygmunt Saloni, Włodzimierz Gruszczyński, + Marcin Woliński, Robert Wołosz + +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + +1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the + distribution. + +THIS SOFTWARE IS PROVIDED BY COPYRIGHT HOLDERS “AS IS” AND ANY EXPRESS +OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL COPYRIGHT HOLDERS OR CONTRIBUTORS BE +LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR +CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF +SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR +BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, +WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE +OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN +IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/plugins/analysis-ukrainian/licenses/lucene-NOTICE.txt b/plugins/analysis-ukrainian/licenses/lucene-NOTICE.txt new file mode 100644 index 00000000000..ecf08201a5e --- /dev/null +++ b/plugins/analysis-ukrainian/licenses/lucene-NOTICE.txt @@ -0,0 +1,191 @@ +Apache Lucene +Copyright 2014 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +Includes software from other Apache Software Foundation projects, +including, but not limited to: + - Apache Ant + - Apache Jakarta Regexp + - Apache Commons + - Apache Xerces + +ICU4J, (under analysis/icu) is licensed under an MIT styles license +and Copyright (c) 1995-2008 International Business Machines Corporation and others + +Some data files (under analysis/icu/src/data) are derived from Unicode data such +as the Unicode Character Database. See http://unicode.org/copyright.html for more +details. + +Brics Automaton (under core/src/java/org/apache/lucene/util/automaton) is +BSD-licensed, created by Anders Møller. See http://www.brics.dk/automaton/ + +The levenshtein automata tables (under core/src/java/org/apache/lucene/util/automaton) were +automatically generated with the moman/finenight FSA library, created by +Jean-Philippe Barrette-LaPierre. This library is available under an MIT license, +see http://sites.google.com/site/rrettesite/moman and +http://bitbucket.org/jpbarrette/moman/overview/ + +The class org.apache.lucene.util.WeakIdentityMap was derived from +the Apache CXF project and is Apache License 2.0. + +The Google Code Prettify is Apache License 2.0. +See http://code.google.com/p/google-code-prettify/ + +JUnit (junit-4.10) is licensed under the Common Public License v. 1.0 +See http://junit.sourceforge.net/cpl-v10.html + +This product includes code (JaspellTernarySearchTrie) from Java Spelling Checkin +g Package (jaspell): http://jaspell.sourceforge.net/ +License: The BSD License (http://www.opensource.org/licenses/bsd-license.php) + +The snowball stemmers in + analysis/common/src/java/net/sf/snowball +were developed by Martin Porter and Richard Boulton. +The snowball stopword lists in + analysis/common/src/resources/org/apache/lucene/analysis/snowball +were developed by Martin Porter and Richard Boulton. +The full snowball package is available from + http://snowball.tartarus.org/ + +The KStem stemmer in + analysis/common/src/org/apache/lucene/analysis/en +was developed by Bob Krovetz and Sergio Guzman-Lara (CIIR-UMass Amherst) +under the BSD-license. + +The Arabic,Persian,Romanian,Bulgarian, and Hindi analyzers (common) come with a default +stopword list that is BSD-licensed created by Jacques Savoy. These files reside in: +analysis/common/src/resources/org/apache/lucene/analysis/ar/stopwords.txt, +analysis/common/src/resources/org/apache/lucene/analysis/fa/stopwords.txt, +analysis/common/src/resources/org/apache/lucene/analysis/ro/stopwords.txt, +analysis/common/src/resources/org/apache/lucene/analysis/bg/stopwords.txt, +analysis/common/src/resources/org/apache/lucene/analysis/hi/stopwords.txt +See http://members.unine.ch/jacques.savoy/clef/index.html. + +The German,Spanish,Finnish,French,Hungarian,Italian,Portuguese,Russian and Swedish light stemmers +(common) are based on BSD-licensed reference implementations created by Jacques Savoy and +Ljiljana Dolamic. These files reside in: +analysis/common/src/java/org/apache/lucene/analysis/de/GermanLightStemmer.java +analysis/common/src/java/org/apache/lucene/analysis/de/GermanMinimalStemmer.java +analysis/common/src/java/org/apache/lucene/analysis/es/SpanishLightStemmer.java +analysis/common/src/java/org/apache/lucene/analysis/fi/FinnishLightStemmer.java +analysis/common/src/java/org/apache/lucene/analysis/fr/FrenchLightStemmer.java +analysis/common/src/java/org/apache/lucene/analysis/fr/FrenchMinimalStemmer.java +analysis/common/src/java/org/apache/lucene/analysis/hu/HungarianLightStemmer.java +analysis/common/src/java/org/apache/lucene/analysis/it/ItalianLightStemmer.java +analysis/common/src/java/org/apache/lucene/analysis/pt/PortugueseLightStemmer.java +analysis/common/src/java/org/apache/lucene/analysis/ru/RussianLightStemmer.java +analysis/common/src/java/org/apache/lucene/analysis/sv/SwedishLightStemmer.java + +The Stempel analyzer (stempel) includes BSD-licensed software developed +by the Egothor project http://egothor.sf.net/, created by Leo Galambos, Martin Kvapil, +and Edmond Nolan. + +The Polish analyzer (stempel) comes with a default +stopword list that is BSD-licensed created by the Carrot2 project. The file resides +in stempel/src/resources/org/apache/lucene/analysis/pl/stopwords.txt. +See http://project.carrot2.org/license.html. + +The SmartChineseAnalyzer source code (smartcn) was +provided by Xiaoping Gao and copyright 2009 by www.imdict.net. + +WordBreakTestUnicode_*.java (under modules/analysis/common/src/test/) +is derived from Unicode data such as the Unicode Character Database. +See http://unicode.org/copyright.html for more details. + +The Morfologik analyzer (morfologik) includes BSD-licensed software +developed by Dawid Weiss and Marcin Miłkowski (http://morfologik.blogspot.com/). + +Morfologik uses data from Polish ispell/myspell dictionary +(http://www.sjp.pl/slownik/en/) licenced on the terms of (inter alia) +LGPL and Creative Commons ShareAlike. + +Morfologic includes data from BSD-licensed dictionary of Polish (SGJP) +(http://sgjp.pl/morfeusz/) + +Servlet-api.jar and javax.servlet-*.jar are under the CDDL license, the original +source code for this can be found at http://www.eclipse.org/jetty/downloads.php + +=========================================================================== +Kuromoji Japanese Morphological Analyzer - Apache Lucene Integration +=========================================================================== + +This software includes a binary and/or source version of data from + + mecab-ipadic-2.7.0-20070801 + +which can be obtained from + + http://atilika.com/releases/mecab-ipadic/mecab-ipadic-2.7.0-20070801.tar.gz + +or + + http://jaist.dl.sourceforge.net/project/mecab/mecab-ipadic/2.7.0-20070801/mecab-ipadic-2.7.0-20070801.tar.gz + +=========================================================================== +mecab-ipadic-2.7.0-20070801 Notice +=========================================================================== + +Nara Institute of Science and Technology (NAIST), +the copyright holders, disclaims all warranties with regard to this +software, including all implied warranties of merchantability and +fitness, in no event shall NAIST be liable for +any special, indirect or consequential damages or any damages +whatsoever resulting from loss of use, data or profits, whether in an +action of contract, negligence or other tortuous action, arising out +of or in connection with the use or performance of this software. + +A large portion of the dictionary entries +originate from ICOT Free Software. The following conditions for ICOT +Free Software applies to the current dictionary as well. + +Each User may also freely distribute the Program, whether in its +original form or modified, to any third party or parties, PROVIDED +that the provisions of Section 3 ("NO WARRANTY") will ALWAYS appear +on, or be attached to, the Program, which is distributed substantially +in the same form as set out herein and that such intended +distribution, if actually made, will neither violate or otherwise +contravene any of the laws and regulations of the countries having +jurisdiction over the User or the intended distribution itself. + +NO WARRANTY + +The program was produced on an experimental basis in the course of the +research and development conducted during the project and is provided +to users as so produced on an experimental basis. Accordingly, the +program is provided without any warranty whatsoever, whether express, +implied, statutory or otherwise. The term "warranty" used herein +includes, but is not limited to, any warranty of the quality, +performance, merchantability and fitness for a particular purpose of +the program and the nonexistence of any infringement or violation of +any right of any third party. + +Each user of the program will agree and understand, and be deemed to +have agreed and understood, that there is no warranty whatsoever for +the program and, accordingly, the entire risk arising from or +otherwise connected with the program is assumed by the user. + +Therefore, neither ICOT, the copyright holder, or any other +organization that participated in or was otherwise related to the +development of the program and their respective officials, directors, +officers and other employees shall be held liable for any and all +damages, including, without limitation, general, special, incidental +and consequential damages, arising out of or otherwise in connection +with the use or inability to use the program or any product, material +or result produced or otherwise obtained by using the program, +regardless of whether they have been advised of, or otherwise had +knowledge of, the possibility of such damages at any time during the +project or thereafter. Each user will be deemed to have agreed to the +foregoing by his or her commencement of use of the program. The term +"use" as used herein includes, but is not limited to, the use, +modification, copying and distribution of the program and the +production of secondary products from the program. + +In the case where the program, whether in its original form or +modified, was distributed or delivered to or received by a user from +any person, organization or entity other than ICOT, unless it makes or +grants independently of ICOT any specific warranty to the user in +writing, such person, organization or entity, will also be exempted +from and not be held liable to the user for any such damages as noted +above as far as the program is concerned. diff --git a/plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-6.2.0.jar.sha1 b/plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-6.2.0.jar.sha1 new file mode 100644 index 00000000000..873661a7e73 --- /dev/null +++ b/plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-6.2.0.jar.sha1 @@ -0,0 +1 @@ +9005084a9bc71a7ce11618722f9509f6c669cce2 \ No newline at end of file diff --git a/plugins/analysis-ukrainian/licenses/morfologik-fsa-2.1.0.jar.sha1 b/plugins/analysis-ukrainian/licenses/morfologik-fsa-2.1.0.jar.sha1 new file mode 100644 index 00000000000..88f43752dba --- /dev/null +++ b/plugins/analysis-ukrainian/licenses/morfologik-fsa-2.1.0.jar.sha1 @@ -0,0 +1 @@ +88e5993f73c102f378c711f6e47221b7a9e22d25 \ No newline at end of file diff --git a/plugins/analysis-ukrainian/licenses/morfologik-stemming-2.1.0.jar.sha1 b/plugins/analysis-ukrainian/licenses/morfologik-stemming-2.1.0.jar.sha1 new file mode 100644 index 00000000000..ec449346c7b --- /dev/null +++ b/plugins/analysis-ukrainian/licenses/morfologik-stemming-2.1.0.jar.sha1 @@ -0,0 +1 @@ +94167b64752138a246cc33cbf1a3b0bfe5274b7c \ No newline at end of file diff --git a/plugins/analysis-ukrainian/src/main/java/org/elasticsearch/index/analysis/UkrainianAnalyzerProvider.java b/plugins/analysis-ukrainian/src/main/java/org/elasticsearch/index/analysis/UkrainianAnalyzerProvider.java new file mode 100644 index 00000000000..45bf27b954b --- /dev/null +++ b/plugins/analysis-ukrainian/src/main/java/org/elasticsearch/index/analysis/UkrainianAnalyzerProvider.java @@ -0,0 +1,47 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.analysis; + +import org.apache.lucene.analysis.CharArraySet; +import org.apache.lucene.analysis.uk.UkrainianMorfologikAnalyzer; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.env.Environment; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.analysis.AbstractIndexAnalyzerProvider; +import org.elasticsearch.index.analysis.Analysis; + +public class UkrainianAnalyzerProvider extends AbstractIndexAnalyzerProvider { + + private final UkrainianMorfologikAnalyzer analyzer; + + public UkrainianAnalyzerProvider(IndexSettings indexSettings, Environment env, String name, Settings settings) { + super(indexSettings, name, settings); + analyzer = new UkrainianMorfologikAnalyzer(Analysis.parseStopWords(env, settings, UkrainianMorfologikAnalyzer.getDefaultStopSet()), + Analysis.parseStemExclusion(settings, CharArraySet.EMPTY_SET)); + analyzer.setVersion(version); + } + + @Override + public UkrainianMorfologikAnalyzer get() { + return this.analyzer; + } + + +} diff --git a/plugins/analysis-ukrainian/src/main/java/org/elasticsearch/plugin/analysis/ukrainian/AnalysisUkrainianPlugin.java b/plugins/analysis-ukrainian/src/main/java/org/elasticsearch/plugin/analysis/ukrainian/AnalysisUkrainianPlugin.java new file mode 100644 index 00000000000..ff8425e201c --- /dev/null +++ b/plugins/analysis-ukrainian/src/main/java/org/elasticsearch/plugin/analysis/ukrainian/AnalysisUkrainianPlugin.java @@ -0,0 +1,39 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.plugin.analysis.ukrainian; + +import org.apache.lucene.analysis.Analyzer; +import org.elasticsearch.index.analysis.AnalyzerProvider; +import org.elasticsearch.index.analysis.UkrainianAnalyzerProvider; +import org.elasticsearch.indices.analysis.AnalysisModule.AnalysisProvider; +import org.elasticsearch.plugins.AnalysisPlugin; +import org.elasticsearch.plugins.Plugin; + +import java.util.Map; + +import static java.util.Collections.singletonMap; + +public class AnalysisUkrainianPlugin extends Plugin implements AnalysisPlugin { + + @Override + public Map>> getAnalyzers() { + return singletonMap("ukrainian", UkrainianAnalyzerProvider::new); + } +} diff --git a/plugins/analysis-ukrainian/src/test/java/org/elasticsearch/index/analysis/SimpleUkrainianAnalyzerTests.java b/plugins/analysis-ukrainian/src/test/java/org/elasticsearch/index/analysis/SimpleUkrainianAnalyzerTests.java new file mode 100644 index 00000000000..6dbc37ea4ab --- /dev/null +++ b/plugins/analysis-ukrainian/src/test/java/org/elasticsearch/index/analysis/SimpleUkrainianAnalyzerTests.java @@ -0,0 +1,56 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.analysis; + +import org.apache.lucene.analysis.Analyzer; +import org.apache.lucene.analysis.TokenStream; +import org.apache.lucene.analysis.tokenattributes.CharTermAttribute; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.Index; +import org.elasticsearch.plugin.analysis.ukrainian.AnalysisUkrainianPlugin; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; + +import static org.hamcrest.Matchers.equalTo; + +public class SimpleUkrainianAnalyzerTests extends ESTestCase { + + public void testBasicUsage() throws Exception { + testAnalyzer("чергу", "черга"); + testAnalyzer("рухається", "рухатися"); + testAnalyzer("колу", "кола", "коло", "кіл"); + testAnalyzer("Ця п'єса у свою чергу рухається по колу.", "п'єса", "черга", "рухатися", "кола", "коло", "кіл"); + } + + private static void testAnalyzer(String source, String... expected_terms) throws IOException { + TestAnalysis analysis = createTestAnalysis(new Index("test", "_na_"), Settings.EMPTY, new AnalysisUkrainianPlugin()); + Analyzer analyzer = analysis.indexAnalyzers.get("ukrainian").analyzer(); + TokenStream ts = analyzer.tokenStream("test", source); + CharTermAttribute term1 = ts.addAttribute(CharTermAttribute.class); + ts.reset(); + for (String expected : expected_terms) { + assertThat(ts.incrementToken(), equalTo(true)); + assertThat(term1.toString(), equalTo(expected)); + } + assertThat(ts.incrementToken(), equalTo(false)); + } + +} diff --git a/plugins/analysis-ukrainian/src/test/java/org/elasticsearch/index/analysis/UkrainianAnalysisTests.java b/plugins/analysis-ukrainian/src/test/java/org/elasticsearch/index/analysis/UkrainianAnalysisTests.java new file mode 100644 index 00000000000..a45549c22bd --- /dev/null +++ b/plugins/analysis-ukrainian/src/test/java/org/elasticsearch/index/analysis/UkrainianAnalysisTests.java @@ -0,0 +1,43 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.analysis; + +import org.apache.lucene.analysis.Analyzer; +import org.apache.lucene.analysis.uk.UkrainianMorfologikAnalyzer; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.Index; +import org.elasticsearch.plugin.analysis.ukrainian.AnalysisUkrainianPlugin; +import org.elasticsearch.test.ESTestCase; +import org.hamcrest.MatcherAssert; + +import java.io.IOException; + +import static org.hamcrest.Matchers.instanceOf; + +public class UkrainianAnalysisTests extends ESTestCase { + + public void testDefaultsUkranianAnalysis() throws IOException { + final TestAnalysis analysis = createTestAnalysis(new Index("test", "_na_"), Settings.EMPTY, + new AnalysisUkrainianPlugin()); + + Analyzer analyzer = analysis.indexAnalyzers.get("ukrainian").analyzer(); + MatcherAssert.assertThat(analyzer, instanceOf(UkrainianMorfologikAnalyzer.class)); + } +} diff --git a/plugins/analysis-ukrainian/src/test/java/org/elasticsearch/index/analysis/UkrainianClientYamlTestSuiteIT.java b/plugins/analysis-ukrainian/src/test/java/org/elasticsearch/index/analysis/UkrainianClientYamlTestSuiteIT.java new file mode 100644 index 00000000000..590d3614b97 --- /dev/null +++ b/plugins/analysis-ukrainian/src/test/java/org/elasticsearch/index/analysis/UkrainianClientYamlTestSuiteIT.java @@ -0,0 +1,42 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.analysis; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.test.rest.yaml.ClientYamlTestCandidate; +import org.elasticsearch.test.rest.yaml.ESClientYamlSuiteTestCase; +import org.elasticsearch.test.rest.yaml.parser.ClientYamlTestParseException; + +import java.io.IOException; + +public class UkrainianClientYamlTestSuiteIT extends ESClientYamlSuiteTestCase { + + public UkrainianClientYamlTestSuiteIT(@Name("yaml") ClientYamlTestCandidate testCandidate) { + super(testCandidate); + } + + @ParametersFactory + public static Iterable parameters() throws IOException, ClientYamlTestParseException { + return ESClientYamlSuiteTestCase.createParameters(0, 1); + } +} + diff --git a/plugins/analysis-ukrainian/src/test/resources/rest-api-spec/test/analysis_ukrainian/10_basic.yaml b/plugins/analysis-ukrainian/src/test/resources/rest-api-spec/test/analysis_ukrainian/10_basic.yaml new file mode 100644 index 00000000000..48d513c140c --- /dev/null +++ b/plugins/analysis-ukrainian/src/test/resources/rest-api-spec/test/analysis_ukrainian/10_basic.yaml @@ -0,0 +1,18 @@ +# Integration tests for Ukrainian analyzer +--- +"Analyzer": + - do: + indices.analyze: + body: + text: колу + analyzer: ukrainian + - length: { tokens: 3 } + - match: { tokens.0.token: кола } + - match: { tokens.0.start_offset: 0 } + - match: { tokens.0.end_offset: 4 } + - match: { tokens.1.token: коло } + - match: { tokens.1.start_offset: 0 } + - match: { tokens.1.end_offset: 4 } + - match: { tokens.2.token: кіл } + - match: { tokens.2.start_offset: 0 } + - match: { tokens.2.end_offset: 4 } diff --git a/plugins/analysis-ukrainian/src/test/resources/rest-api-spec/test/analysis_ukrainian/20_search.yaml b/plugins/analysis-ukrainian/src/test/resources/rest-api-spec/test/analysis_ukrainian/20_search.yaml new file mode 100644 index 00000000000..34d8fd2fde7 --- /dev/null +++ b/plugins/analysis-ukrainian/src/test/resources/rest-api-spec/test/analysis_ukrainian/20_search.yaml @@ -0,0 +1,32 @@ +# Integration tests for Stempel analysis component +# +--- +"Index Stempel content": + - do: + indices.create: + index: test + body: + mappings: + type: + properties: + text: + type: text + analyzer: ukrainian + + - do: + index: + index: test + type: type + id: 1 + body: { "text": "Ця п'єса у свою чергу рухається по колу." } + - do: + indices.refresh: {} + + - do: + search: + index: test + body: + query: + match: + text: кола + - match: { hits.total: 1 } diff --git a/settings.gradle b/settings.gradle index 81513fd372a..4c662ac448f 100644 --- a/settings.gradle +++ b/settings.gradle @@ -36,6 +36,7 @@ List projects = [ 'plugins:analysis-phonetic', 'plugins:analysis-smartcn', 'plugins:analysis-stempel', + 'plugins:analysis-ukrainian', 'plugins:discovery-azure-classic', 'plugins:discovery-ec2', 'plugins:discovery-file', From 1f5adaa82472d21ba2811a9d62e46eaf168016fc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Fri, 28 Oct 2016 17:19:33 +0200 Subject: [PATCH 057/132] Docs: Adding Ukrainian analyzer --- docs/plugins/analysis-ukrainian.asciidoc | 42 +++++++++++++++++++ docs/plugins/analysis.asciidoc | 5 +++ docs/reference/cat/plugins.asciidoc | 1 + ...lucene-analyzers-morfologik-6.2.0.jar.sha1 | 1 - ...morfologik-6.3.0-snapshot-ed102d6.jar.sha1 | 1 + 5 files changed, 49 insertions(+), 1 deletion(-) create mode 100644 docs/plugins/analysis-ukrainian.asciidoc delete mode 100644 plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-6.2.0.jar.sha1 create mode 100644 plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-6.3.0-snapshot-ed102d6.jar.sha1 diff --git a/docs/plugins/analysis-ukrainian.asciidoc b/docs/plugins/analysis-ukrainian.asciidoc new file mode 100644 index 00000000000..78f8232f1c1 --- /dev/null +++ b/docs/plugins/analysis-ukrainian.asciidoc @@ -0,0 +1,42 @@ +[[analysis-ukrainian]] +=== Ukrainian Analysis Plugin + +The Ukrainian Analysis plugin integrates Lucene's UkrainianMorfologikAnalyzer into elasticsearch. + +It provides stemming for Ukrainian using the http://github.com/morfologik/morfologik-stemming[Morfologik project]. + +[[analysis-ukrainian-install]] +[float] +==== Installation + +This plugin can be installed using the plugin manager: + +[source,sh] +---------------------------------------------------------------- +sudo bin/elasticsearch-plugin install analysis-ukrainian +---------------------------------------------------------------- + +The plugin must be installed on every node in the cluster, and each node must +be restarted after installation. + +This plugin can be downloaded for <> from +{plugin_url}/analysis-ukrainian/analysis-ukrainian-{version}.zip. + +[[analysis-ukrainian-remove]] +[float] +==== Removal + +The plugin can be removed with the following command: + +[source,sh] +---------------------------------------------------------------- +sudo bin/elasticsearch-plugin remove analysis-ukrainian +---------------------------------------------------------------- + +The node must be stopped before removing the plugin. + +[[analysis-ukrainian-analyzer]] +[float] +==== `ukrainian` analyzer + +The plugin provides the `ukrainian` analyzer. diff --git a/docs/plugins/analysis.asciidoc b/docs/plugins/analysis.asciidoc index 884dc2aebae..df68be57ab9 100644 --- a/docs/plugins/analysis.asciidoc +++ b/docs/plugins/analysis.asciidoc @@ -36,6 +36,10 @@ segmented into words. Provides high quality stemming for Polish. +<>:: + +Provides stemming for Ukrainian. + [float] ==== Community contributed analysis plugins @@ -62,5 +66,6 @@ include::analysis-smartcn.asciidoc[] include::analysis-stempel.asciidoc[] +include::analysis-ukrainian.asciidoc[] diff --git a/docs/reference/cat/plugins.asciidoc b/docs/reference/cat/plugins.asciidoc index b2a193dfa8c..b4aa02a7af7 100644 --- a/docs/reference/cat/plugins.asciidoc +++ b/docs/reference/cat/plugins.asciidoc @@ -19,6 +19,7 @@ U7321H6 analysis-kuromoji {version} The Japanese (kuromoji) Analysis plugi U7321H6 analysis-phonetic {version} The Phonetic Analysis plugin integrates phonetic token filter analysis with elasticsearch. U7321H6 analysis-smartcn {version} Smart Chinese Analysis plugin integrates Lucene Smart Chinese analysis module into elasticsearch. U7321H6 analysis-stempel {version} The Stempel (Polish) Analysis plugin integrates Lucene stempel (polish) analysis module into elasticsearch. +U7321H6 analysis-ukrainian {version} The Ukrainian Analysis plugin integrates the Lucene UkrainianMorfologikAnalyzer into elasticsearch. U7321H6 discovery-azure-classic {version} The Azure Classic Discovery plugin allows to use Azure Classic API for the unicast discovery mechanism U7321H6 discovery-ec2 {version} The EC2 discovery plugin allows to use AWS API for the unicast discovery mechanism. U7321H6 discovery-file {version} Discovery file plugin enables unicast discovery from hosts stored in a file. diff --git a/plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-6.2.0.jar.sha1 b/plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-6.2.0.jar.sha1 deleted file mode 100644 index 873661a7e73..00000000000 --- a/plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-6.2.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -9005084a9bc71a7ce11618722f9509f6c669cce2 \ No newline at end of file diff --git a/plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-6.3.0-snapshot-ed102d6.jar.sha1 b/plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-6.3.0-snapshot-ed102d6.jar.sha1 new file mode 100644 index 00000000000..af4d754834c --- /dev/null +++ b/plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-6.3.0-snapshot-ed102d6.jar.sha1 @@ -0,0 +1 @@ +a08e5ddd375ea17a878ba077cc33ce59e68758c2 \ No newline at end of file From c10a6ddec112737fa3b4fa265480eef8e3eaa1a0 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Mon, 31 Oct 2016 20:04:33 +0100 Subject: [PATCH 058/132] IndexService#maybeRefresh should catch `IndexShardClosedException` (#21205) We throw this exception in some cases that the shard is closed, so we have to be consistent here. Otherwise we get logs like: ``` 1> [2016-10-30T21:06:22,529][WARN ][o.e.i.IndexService ] [node_s_0] [test] failed to run task refresh - suppressing re-occurring exceptions unless the exception changes 1> org.elasticsearch.index.shard.IndexShardClosedException: CurrentState[CLOSED] operation only allowed when not closed 1> at org.elasticsearch.index.shard.IndexShard.verifyNotClosed(IndexShard.java:1147) ~[main/:?] 1> at org.elasticsearch.index.shard.IndexShard.verifyNotClosed(IndexShard.java:1141) ~[main/:?] ``` --- core/src/main/java/org/elasticsearch/index/IndexService.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/index/IndexService.java b/core/src/main/java/org/elasticsearch/index/IndexService.java index ffcc7648293..0114d68b4e2 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexService.java +++ b/core/src/main/java/org/elasticsearch/index/IndexService.java @@ -52,6 +52,7 @@ import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexSearcherWrapper; import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.IndexShardClosedException; import org.elasticsearch.index.shard.IndexingOperationListener; import org.elasticsearch.index.shard.SearchOperationListener; import org.elasticsearch.index.shard.ShadowIndexShard; @@ -692,7 +693,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust if (shard.isRefreshNeeded()) { shard.refresh("schedule"); } - } catch (EngineClosedException | AlreadyClosedException ex) { + } catch (IndexShardClosedException | AlreadyClosedException ex) { // fine - continue; } continue; From 1d8d8ea55f7c4845c98e083a6572d256cd2e2e96 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 31 Oct 2016 16:34:21 -0400 Subject: [PATCH 059/132] Clarify production mode for bootstrap checks This commit clarifies that production mode for the bootstrap checks is only tripped if transport is bound to an external interface. Relates #21220 --- .../reference/setup/bootstrap-checks.asciidoc | 20 ++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/docs/reference/setup/bootstrap-checks.asciidoc b/docs/reference/setup/bootstrap-checks.asciidoc index 8c1bab474c8..cb9b8f106c7 100644 --- a/docs/reference/setup/bootstrap-checks.asciidoc +++ b/docs/reference/setup/bootstrap-checks.asciidoc @@ -23,14 +23,20 @@ documented individually. [float] === Development vs. production mode -By default, Elasticsearch binds and publishes to `localhost`. This is +By default, Elasticsearch binds to `localhost` for <> +and <> communication. This is fine for downloading and playing with Elasticsearch, and everyday -development but it's useless for production systems. For a production -installation to be reachable, it must either bind or publish to an -external interface. Thus, we consider Elasticsearch to be in development -mode if it does not bind nor publish to an external interface (the -default), and is otherwise in production mode if it does bind or publish -to an external interface. +development but it's useless for production systems. To form a cluster, +Elasticsearch instances must be reachable via transport communication so +they must bind transport to an external interface. Thus, we consider an +Elaticsearch instance to be in development mode if it does not bind +transport to an external interface (the default), and is otherwise in +production mode if it does bind transport to an external interface. Note +that HTTP can be configured independently of transport via +<> and +<>; this can be useful for +configuring a single instance to be reachable via HTTP for testing +purposes without triggering production mode. === Heap size check From 344a8028f80baf85c7e118b07230c9ba3e334e03 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 31 Oct 2016 16:38:33 -0400 Subject: [PATCH 060/132] Add analysis-ukrainian plugin to packaging tests This commit adds the analysis-ukrainian plugin to the packaging tests. Relates #21219 --- .../scripts/module_and_plugin_test_cases.bash | 8 ++++++++ .../resources/packaging/scripts/plugins.bash | 20 +++++++++++++------ 2 files changed, 22 insertions(+), 6 deletions(-) diff --git a/qa/vagrant/src/test/resources/packaging/scripts/module_and_plugin_test_cases.bash b/qa/vagrant/src/test/resources/packaging/scripts/module_and_plugin_test_cases.bash index d5dc095e70f..fbc0f4a887a 100644 --- a/qa/vagrant/src/test/resources/packaging/scripts/module_and_plugin_test_cases.bash +++ b/qa/vagrant/src/test/resources/packaging/scripts/module_and_plugin_test_cases.bash @@ -205,6 +205,10 @@ fi install_and_check_plugin analysis stempel } +@test "[$GROUP] install ukrainian plugin" { + install_and_check_plugin analysis ukrainian morfologik-fsa-*.jar morfologik-stemming-*.jar +} + @test "[$GROUP] install gce plugin" { install_and_check_plugin discovery gce google-api-client-*.jar } @@ -341,6 +345,10 @@ fi remove_plugin analysis-stempel } +@test "[$GROUP] remove ukrainian plugin" { + remove_plugin analysis-ukrainian +} + @test "[$GROUP] remove gce plugin" { remove_plugin discovery-gce } diff --git a/qa/vagrant/src/test/resources/packaging/scripts/plugins.bash b/qa/vagrant/src/test/resources/packaging/scripts/plugins.bash index 02cae2aeecb..55e7fdfc484 100644 --- a/qa/vagrant/src/test/resources/packaging/scripts/plugins.bash +++ b/qa/vagrant/src/test/resources/packaging/scripts/plugins.bash @@ -129,20 +129,28 @@ install_and_check_plugin() { shift if [ "$prefix" == "-" ]; then - local fullName="$name" + local full_name="$name" else - local fullName="$prefix-$name" + local full_name="$prefix-$name" fi - install_jvm_plugin $fullName "$(readlink -m $fullName-*.zip)" + install_jvm_plugin $full_name "$(readlink -m $full_name-*.zip)" - assert_module_or_plugin_directory "$ESPLUGINS/$fullName" + assert_module_or_plugin_directory "$ESPLUGINS/$full_name" + # analysis plugins have a corresponding analyzers jar if [ $prefix == 'analysis' ]; then - assert_module_or_plugin_file "$ESPLUGINS/$fullName/lucene-analyzers-$name-*.jar" + local analyzer_jar_suffix=$name + # the name of the analyzer jar for the ukrainian plugin does + # not match the name of the plugin, so we have to make an + # exception + if [ $name == 'ukrainian' ]; then + analyzer_jar_suffix='morfologik' + fi + assert_module_or_plugin_file "$ESPLUGINS/$full_name/lucene-analyzers-$analyzer_jar_suffix-*.jar" fi for file in "$@"; do - assert_module_or_plugin_file "$ESPLUGINS/$fullName/$file" + assert_module_or_plugin_file "$ESPLUGINS/$full_name/$file" done } From 185dff73461ae384f44fda5c7883fa6ba5e8c466 Mon Sep 17 00:00:00 2001 From: Jack Conradson Date: Mon, 31 Oct 2016 13:48:51 -0700 Subject: [PATCH 061/132] Cleanup ScriptType (#21179) Refactored ScriptType to clean up some of the variable and method names. Added more documentation. Deprecated the 'in' ParseField in favor of 'stored' to match the indexed scripts being replaced by stored scripts. --- .../java/org/elasticsearch/script/Script.java | 2 +- .../org/elasticsearch/script/ScriptModes.java | 2 +- .../elasticsearch/script/ScriptSettings.java | 4 +- .../org/elasticsearch/script/ScriptType.java | 128 ++++++++++++------ .../script/NativeScriptTests.java | 2 +- .../script/ScriptModesTests.java | 2 +- .../script/ScriptServiceTests.java | 4 +- docs/plugins/lang-javascript.asciidoc | 2 +- docs/plugins/lang-python.asciidoc | 2 +- .../modules/scripting/using.asciidoc | 8 +- .../query-dsl/template-query.asciidoc | 2 +- .../mustache/SearchTemplateRequest.java | 2 +- .../test/lang_mustache/40_template_query.yaml | 4 +- 13 files changed, 107 insertions(+), 57 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/script/Script.java b/core/src/main/java/org/elasticsearch/script/Script.java index cfdbf966b05..edcf6666817 100644 --- a/core/src/main/java/org/elasticsearch/script/Script.java +++ b/core/src/main/java/org/elasticsearch/script/Script.java @@ -109,7 +109,7 @@ public final class Script implements ToXContent, Writeable { boolean hasType = type != null; out.writeBoolean(hasType); if (hasType) { - ScriptType.writeTo(type, out); + type.writeTo(out); } out.writeOptionalString(lang); out.writeMap(params); diff --git a/core/src/main/java/org/elasticsearch/script/ScriptModes.java b/core/src/main/java/org/elasticsearch/script/ScriptModes.java index 4f9651b290a..15393948d66 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptModes.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptModes.java @@ -72,7 +72,7 @@ public class ScriptModes { } static String sourceKey(ScriptType scriptType) { - return SCRIPT_SETTINGS_PREFIX + "." + scriptType.getScriptType(); + return SCRIPT_SETTINGS_PREFIX + "." + scriptType.getName(); } static String getGlobalKey(String lang, ScriptType scriptType) { diff --git a/core/src/main/java/org/elasticsearch/script/ScriptSettings.java b/core/src/main/java/org/elasticsearch/script/ScriptSettings.java index 07e0deb5b71..27a6ad04a70 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptSettings.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptSettings.java @@ -50,7 +50,7 @@ public class ScriptSettings { for (ScriptType scriptType : ScriptType.values()) { scriptTypeSettingMap.put(scriptType, Setting.boolSetting( ScriptModes.sourceKey(scriptType), - scriptType.getDefaultScriptEnabled(), + scriptType.isDefaultEnabled(), Property.NodeScope)); } SCRIPT_TYPE_SETTING_MAP = Collections.unmodifiableMap(scriptTypeSettingMap); @@ -102,7 +102,7 @@ public class ScriptSettings { boolean defaultLangAndType = defaultNonFileScriptMode; // Files are treated differently because they are never default-deny if (ScriptType.FILE == scriptType) { - defaultLangAndType = ScriptType.FILE.getDefaultScriptEnabled(); + defaultLangAndType = ScriptType.FILE.isDefaultEnabled(); } final boolean defaultIfNothingSet = defaultLangAndType; diff --git a/core/src/main/java/org/elasticsearch/script/ScriptType.java b/core/src/main/java/org/elasticsearch/script/ScriptType.java index 77865daa372..01592b57aad 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptType.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptType.java @@ -22,68 +22,118 @@ package org.elasticsearch.script; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; import java.io.IOException; -import java.util.Locale; /** - * The type of a script, more specifically where it gets loaded from: - * - provided dynamically at request time - * - loaded from an index - * - loaded from file + * ScriptType represents the way a script is stored and retrieved from the {@link ScriptService}. + * It's also used to by {@link ScriptSettings} and {@link ScriptModes} to determine whether or not + * a {@link Script} is allowed to be executed based on both default and user-defined settings. */ -public enum ScriptType { +public enum ScriptType implements Writeable { - INLINE(0, "inline", "inline", false), - STORED(1, "id", "stored", false), - FILE(2, "file", "file", true); + /** + * INLINE scripts are specified in numerous queries and compiled on-the-fly. + * They will be cached based on the lang and code of the script. + * They are turned off by default because most languages are insecure + * (Groovy and others), but can be overriden by the specific {@link ScriptEngineService} + * if the language is naturally secure (Painless, Mustache, and Expressions). + */ + INLINE ( 0 , new ParseField("inline") , false ), - private final int val; - private final ParseField parseField; - private final String scriptType; - private final boolean defaultScriptEnabled; + /** + * STORED scripts are saved as part of the {@link org.elasticsearch.cluster.ClusterState} + * based on user requests. They will be cached when they are first used in a query. + * They are turned off by default because most languages are insecure + * (Groovy and others), but can be overriden by the specific {@link ScriptEngineService} + * if the language is naturally secure (Painless, Mustache, and Expressions). + */ + STORED ( 1 , new ParseField("stored", "id") , false ), + /** + * FILE scripts are loaded from disk either on start-up or on-the-fly depending on + * user-defined settings. They will be compiled and cached as soon as they are loaded + * from disk. They are turned on by default as they should always be safe to execute. + */ + FILE ( 2 , new ParseField("file") , true ); + + /** + * Reads an int from the input stream and converts it to a {@link ScriptType}. + * @return The ScriptType read from the stream. Throws an {@link IllegalStateException} + * if no ScriptType is found based on the id. + */ public static ScriptType readFrom(StreamInput in) throws IOException { - int scriptTypeVal = in.readVInt(); - for (ScriptType type : values()) { - if (type.val == scriptTypeVal) { - return type; - } - } - throw new IllegalArgumentException("Unexpected value read for ScriptType got [" + scriptTypeVal + "] expected one of [" - + INLINE.val + "," + FILE.val + "," + STORED.val + "]"); - } + int id = in.readVInt(); - public static void writeTo(ScriptType scriptType, StreamOutput out) throws IOException{ - if (scriptType != null) { - out.writeVInt(scriptType.val); + if (FILE.id == id) { + return FILE; + } else if (STORED.id == id) { + return STORED; + } else if (INLINE.id == id) { + return INLINE; } else { - out.writeVInt(INLINE.val); //Default to inline + throw new IllegalStateException("Error reading ScriptType id [" + id + "] from stream, expected one of [" + + FILE.id + " [" + FILE.parseField.getPreferredName() + "], " + + STORED.id + " [" + STORED.parseField.getPreferredName() + "], " + + INLINE.id + " [" + INLINE.parseField.getPreferredName() + "]]"); } } - ScriptType(int val, String name, String scriptType, boolean defaultScriptEnabled) { - this.val = val; - this.parseField = new ParseField(name); - this.scriptType = scriptType; - this.defaultScriptEnabled = defaultScriptEnabled; + private final int id; + private final ParseField parseField; + private final boolean defaultEnabled; + + /** + * Standard constructor. + * @param id A unique identifier for a type that can be read/written to a stream. + * @param parseField Specifies the name used to parse input from queries. + * @param defaultEnabled Whether or not a {@link ScriptType} can be run by default. + */ + ScriptType(int id, ParseField parseField, boolean defaultEnabled) { + this.id = id; + this.parseField = parseField; + this.defaultEnabled = defaultEnabled; } + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(id); + } + + /** + * @return The unique id for this {@link ScriptType}. + */ + public int getId() { + return id; + } + + /** + * @return The unique name for this {@link ScriptType} based on the {@link ParseField}. + */ + public String getName() { + return parseField.getPreferredName(); + } + + /** + * @return Specifies the name used to parse input from queries. + */ public ParseField getParseField() { return parseField; } - public boolean getDefaultScriptEnabled() { - return defaultScriptEnabled; - } - - public String getScriptType() { - return scriptType; + /** + * @return Whether or not a {@link ScriptType} can be run by default. Note + * this can be potentially overriden by any {@link ScriptEngineService}. + */ + public boolean isDefaultEnabled() { + return defaultEnabled; } + /** + * @return The same as calling {@link #getName()}. + */ @Override public String toString() { - return name().toLowerCase(Locale.ROOT); + return getName(); } - } diff --git a/core/src/test/java/org/elasticsearch/script/NativeScriptTests.java b/core/src/test/java/org/elasticsearch/script/NativeScriptTests.java index eeafbde8850..efff2b0834e 100644 --- a/core/src/test/java/org/elasticsearch/script/NativeScriptTests.java +++ b/core/src/test/java/org/elasticsearch/script/NativeScriptTests.java @@ -62,7 +62,7 @@ public class NativeScriptTests extends ESTestCase { Settings.Builder builder = Settings.builder(); if (randomBoolean()) { ScriptType scriptType = randomFrom(ScriptType.values()); - builder.put("script" + "." + scriptType.getScriptType(), randomBoolean()); + builder.put("script" + "." + scriptType.getName(), randomBoolean()); } else { ScriptContext scriptContext = randomFrom(ScriptContext.Standard.values()); builder.put("script" + "." + scriptContext.getKey(), randomBoolean()); diff --git a/core/src/test/java/org/elasticsearch/script/ScriptModesTests.java b/core/src/test/java/org/elasticsearch/script/ScriptModesTests.java index a56d056cd6b..f6a02ae9206 100644 --- a/core/src/test/java/org/elasticsearch/script/ScriptModesTests.java +++ b/core/src/test/java/org/elasticsearch/script/ScriptModesTests.java @@ -125,7 +125,7 @@ public class ScriptModesTests extends ESTestCase { ScriptType[] randomScriptTypes = randomScriptTypesSet.toArray(new ScriptType[randomScriptTypesSet.size()]); Settings.Builder builder = Settings.builder(); for (int i = 0; i < randomInt; i++) { - builder.put("script" + "." + randomScriptTypes[i].getScriptType(), randomScriptModes[i]); + builder.put("script" + "." + randomScriptTypes[i].getName(), randomScriptModes[i]); } this.scriptModes = new ScriptModes(scriptSettings, builder.build()); diff --git a/core/src/test/java/org/elasticsearch/script/ScriptServiceTests.java b/core/src/test/java/org/elasticsearch/script/ScriptServiceTests.java index 8ed5f4c957d..2b14ba3f4d0 100644 --- a/core/src/test/java/org/elasticsearch/script/ScriptServiceTests.java +++ b/core/src/test/java/org/elasticsearch/script/ScriptServiceTests.java @@ -266,9 +266,9 @@ public class ScriptServiceTests extends ESTestCase { Settings.Builder builder = Settings.builder(); for (Map.Entry entry : scriptSourceSettings.entrySet()) { if (entry.getValue()) { - builder.put("script" + "." + entry.getKey().getScriptType(), "true"); + builder.put("script" + "." + entry.getKey().getName(), "true"); } else { - builder.put("script" + "." + entry.getKey().getScriptType(), "false"); + builder.put("script" + "." + entry.getKey().getName(), "false"); } } for (Map.Entry entry : scriptContextSettings.entrySet()) { diff --git a/docs/plugins/lang-javascript.asciidoc b/docs/plugins/lang-javascript.asciidoc index 64be9dbb033..001b7f22231 100644 --- a/docs/plugins/lang-javascript.asciidoc +++ b/docs/plugins/lang-javascript.asciidoc @@ -120,7 +120,7 @@ GET test/_search "function_score": { "script_score": { "script": { - "id": "my_script", <2> + "stored": "my_script", <2> "lang": "javascript", "params": { "factor": 2 diff --git a/docs/plugins/lang-python.asciidoc b/docs/plugins/lang-python.asciidoc index 0730f8b54b7..0e328d79f60 100644 --- a/docs/plugins/lang-python.asciidoc +++ b/docs/plugins/lang-python.asciidoc @@ -119,7 +119,7 @@ GET test/_search "function_score": { "script_score": { "script": { - "id": "my_script", <2> + "stored": "my_script", <2> "lang": "python", "params": { "factor": 2 diff --git a/docs/reference/modules/scripting/using.asciidoc b/docs/reference/modules/scripting/using.asciidoc index c3af5861879..b09a54e5c46 100644 --- a/docs/reference/modules/scripting/using.asciidoc +++ b/docs/reference/modules/scripting/using.asciidoc @@ -8,12 +8,12 @@ the same pattern: ------------------------------------- "script": { "lang": "...", <1> - "inline" | "id" | "file": "...", <2> + "inline" | "stored" | "file": "...", <2> "params": { ... } <3> } ------------------------------------- <1> The language the script is written in, which defaults to `painless`. -<2> The script itself which may be specified as `inline`, `id`, or `file`. +<2> The script itself which may be specified as `inline`, `stored`, or `file`. <3> Any named parameters that should be passed into the script. For example, the following script is used in a search request to return a @@ -211,7 +211,7 @@ GET _scripts/groovy/calculate-score // CONSOLE // TEST[continued] -Stored scripts can be used by specifying the `lang` and `id` parameters as follows: +Stored scripts can be used by specifying the `lang` and `stored` parameters as follows: [source,js] -------------------------------------------------- @@ -221,7 +221,7 @@ GET _search "script": { "script": { "lang": "groovy", - "id": "calculate-score", + "stored": "calculate-score", "params": { "my_modifier": 2 } diff --git a/docs/reference/query-dsl/template-query.asciidoc b/docs/reference/query-dsl/template-query.asciidoc index b4b00e5babd..2d3b5724d49 100644 --- a/docs/reference/query-dsl/template-query.asciidoc +++ b/docs/reference/query-dsl/template-query.asciidoc @@ -108,7 +108,7 @@ GET /_search { "query": { "template": { - "id": "my_template", <1> + "stored": "my_template", <1> "params" : { "query_string" : "all about search" } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/SearchTemplateRequest.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/SearchTemplateRequest.java index 249b0080775..d7ac37f8313 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/SearchTemplateRequest.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/SearchTemplateRequest.java @@ -157,7 +157,7 @@ public class SearchTemplateRequest extends ActionRequest out.writeBoolean(simulate); out.writeBoolean(explain); out.writeBoolean(profile); - ScriptType.writeTo(scriptType, out); + scriptType.writeTo(out); out.writeOptionalString(script); boolean hasParams = scriptParams != null; out.writeBoolean(hasParams); diff --git a/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/40_template_query.yaml b/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/40_template_query.yaml index 2360dfc37f0..cfa97b8bc9f 100644 --- a/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/40_template_query.yaml +++ b/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/40_template_query.yaml @@ -44,7 +44,7 @@ warnings: - '[template] query is deprecated, use search template api instead' search: - body: { "query": { "template": { "id": "1", "params": { "my_value": "value1" } } } } + body: { "query": { "template": { "stored": "1", "params": { "my_value": "value1" } } } } - match: { hits.total: 1 } @@ -52,7 +52,7 @@ warnings: - '[template] query is deprecated, use search template api instead' search: - body: { "query": { "template": { "id": "/mustache/1", "params": { "my_value": "value1" } } } } + body: { "query": { "template": { "stored": "/mustache/1", "params": { "my_value": "value1" } } } } - match: { hits.total: 1 } From 38663351dcf76955e8ef6f4bc959d777825ec425 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 31 Oct 2016 17:23:21 -0400 Subject: [PATCH 062/132] Fix logger names for Netty Previously Elasticsearch would only use the package name for logging levels, truncating the package prefix and the class name. This meant that logger names for Netty were just prefixed by netty3 and netty. We changed this for Elasticsearch so that it's the fully-qualified class name now, but never corrected this for Netty. This commit fixes the logger names for the Netty modules so that their levels are controlled by the fully-qualified class name. Relates #21223 --- .../java/org/elasticsearch/transport/netty3/Netty3Utils.java | 1 - .../java/org/elasticsearch/transport/netty4/Netty4Utils.java | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/modules/transport-netty3/src/main/java/org/elasticsearch/transport/netty3/Netty3Utils.java b/modules/transport-netty3/src/main/java/org/elasticsearch/transport/netty3/Netty3Utils.java index aea5b31f457..f32bd5dc19b 100644 --- a/modules/transport-netty3/src/main/java/org/elasticsearch/transport/netty3/Netty3Utils.java +++ b/modules/transport-netty3/src/main/java/org/elasticsearch/transport/netty3/Netty3Utils.java @@ -99,7 +99,6 @@ public class Netty3Utils { InternalLoggerFactory.setDefaultFactory(new InternalLoggerFactory() { @Override public InternalLogger newInstance(String name) { - name = name.replace("org.jboss.netty.", "netty3.").replace("org.jboss.netty.", "netty3."); return new Netty3InternalESLogger(Loggers.getLogger(name)); } }); diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Utils.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Utils.java index 53cf1b329aa..877d50e1674 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Utils.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Utils.java @@ -41,7 +41,7 @@ public class Netty4Utils { @Override public InternalLogger newInstance(final String name) { - return new Netty4InternalESLogger(name.replace("io.netty.", "netty.")); + return new Netty4InternalESLogger(name); } }); From 5fab17c7bc03dd0d336c19a23bc7655aaa12fdc2 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 31 Oct 2016 17:33:15 -0400 Subject: [PATCH 063/132] Fix markup for bootstrap checks docs This commit adds a missing backtick to the bootstrap check docs. --- docs/reference/setup/bootstrap-checks.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/setup/bootstrap-checks.asciidoc b/docs/reference/setup/bootstrap-checks.asciidoc index cb9b8f106c7..adc58840e6d 100644 --- a/docs/reference/setup/bootstrap-checks.asciidoc +++ b/docs/reference/setup/bootstrap-checks.asciidoc @@ -33,7 +33,7 @@ Elaticsearch instance to be in development mode if it does not bind transport to an external interface (the default), and is otherwise in production mode if it does bind transport to an external interface. Note that HTTP can be configured independently of transport via -<> and +<> and <>; this can be useful for configuring a single instance to be reachable via HTTP for testing purposes without triggering production mode. From f81d28d0a05a013c3a1e0dda6c72773b4310fa2d Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 31 Oct 2016 20:48:55 -0400 Subject: [PATCH 064/132] Fix transport host setting This commit fixes a reference to the transport.host setting which was mistakenly identified as transport.tcp.host. --- docs/reference/setup/bootstrap-checks.asciidoc | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/docs/reference/setup/bootstrap-checks.asciidoc b/docs/reference/setup/bootstrap-checks.asciidoc index adc58840e6d..9c2276bc213 100644 --- a/docs/reference/setup/bootstrap-checks.asciidoc +++ b/docs/reference/setup/bootstrap-checks.asciidoc @@ -33,10 +33,9 @@ Elaticsearch instance to be in development mode if it does not bind transport to an external interface (the default), and is otherwise in production mode if it does bind transport to an external interface. Note that HTTP can be configured independently of transport via -<> and -<>; this can be useful for -configuring a single instance to be reachable via HTTP for testing -purposes without triggering production mode. +<> and <>; +this can be useful for configuring a single instance to be reachable via +HTTP for testing purposes without triggering production mode. === Heap size check From 02ecff13e4e192378bf06305e3fd961515aab981 Mon Sep 17 00:00:00 2001 From: Areek Zillur Date: Mon, 31 Oct 2016 20:44:11 -0400 Subject: [PATCH 065/132] incorporate feedback --- .../action/bulk/TransportShardBulkAction.java | 4 ++- .../action/index/TransportIndexAction.java | 2 +- .../TransportReplicationAction.java | 4 ++- .../replication/TransportWriteAction.java | 3 +++ .../elasticsearch/index/engine/Engine.java | 10 +------ .../shard/IndexingOperationListener.java | 18 ++++++++++--- .../indices/IndexingMemoryController.java | 10 ++----- .../TransportWriteActionTests.java | 26 ++++++++++--------- 8 files changed, 41 insertions(+), 36 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 6324c684a6f..18fe1c15d78 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -134,7 +134,7 @@ public class TransportShardBulkAction extends TransportWriteActionfinalResponseIfSuccessful or finalFailure to be not-null */ public PrimaryResult(ReplicaRequest replicaRequest, Response finalResponseIfSuccessful, Exception finalFailure) { - assert finalFailure != null ^ finalResponseIfSuccessful != null : "either a response or a failure has to be not null"; + assert finalFailure != null ^ finalResponseIfSuccessful != null + : "either a response or a failure has to be not null, " + + "found [" + finalFailure + "] failure and ["+ finalResponseIfSuccessful + "] response"; this.replicaRequest = replicaRequest; this.finalResponseIfSuccessful = finalResponseIfSuccessful; this.finalFailure = finalFailure; diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java b/core/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java index acc512f1a07..111edf5606b 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java @@ -89,6 +89,9 @@ public abstract class TransportWriteAction< @Nullable Location location, @Nullable Exception operationFailure, IndexShard primary) { super(request, finalResponse, operationFailure); + if (location != null) { + assert operationFailure == null : "expected no failures when translog location is not null"; + } if (operationFailure != null) { this.finishedAsyncActions = true; } else { diff --git a/core/src/main/java/org/elasticsearch/index/engine/Engine.java b/core/src/main/java/org/elasticsearch/index/engine/Engine.java index 0cd1a3022b1..0df026ad617 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -334,17 +334,9 @@ public abstract class Engine implements Closeable { return operationType; } - /** get size of the translog operation if translog location has been set */ - public int getSizeInBytes() { - if (translogLocation != null) { - return translogLocation.size; - } else { - throw new IllegalStateException("result has null location, use Operation#estimatedSizeInBytes instead"); - } - } - void setTranslogLocation(Translog.Location translogLocation) { if (freeze == false) { + assert failure == null : "failure has to be null to set translog location"; this.translogLocation = translogLocation; } else { throw new IllegalStateException("result is already frozen"); diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java b/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java index 0e605954248..e0114a918ff 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java @@ -38,12 +38,17 @@ public interface IndexingOperationListener { } /** - * Called after the indexing operation occurred. + * Called after the indexing operation occurred. Implementations should + * check {@link Engine.IndexResult#hasFailure()} for operation failures + * and delegate to {@link #postIndex(Engine.Index, Exception)} with + * {@link Engine.IndexResult#getFailure()} if appropriate */ default void postIndex(Engine.Index index, Engine.IndexResult result) {} /** - * Called after the indexing operation occurred with exception. + * Called after the indexing operation occurred with exception that + * is not specific to the {@link Engine.Index} i.e. persistent engine + * failures etc. */ default void postIndex(Engine.Index index, Exception ex) {} @@ -56,12 +61,17 @@ public interface IndexingOperationListener { /** - * Called after the delete operation occurred. + * Called after the delete operation occurred. Implementations should + * check {@link Engine.DeleteResult#hasFailure()} for operation failures + * and delegate to {@link #postDelete(Engine.Delete, Exception)} with + * {@link Engine.DeleteResult#getFailure()} if appropriate */ default void postDelete(Engine.Delete delete, Engine.DeleteResult result) {} /** - * Called after the delete operation occurred with exception. + * Called after the delete operation occurred with exception that + * is not specific to the {@link Engine.Delete} i.e. persistent engine + * failures etc. */ default void postDelete(Engine.Delete delete, Exception ex) {} diff --git a/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java b/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java index dd950806b6f..11dbfb36f4f 100644 --- a/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java +++ b/core/src/main/java/org/elasticsearch/indices/IndexingMemoryController.java @@ -209,16 +209,10 @@ public class IndexingMemoryController extends AbstractComponent implements Index recordOperationBytes(delete, result); } - /** called by IndexShard to record that this many bytes were written to translog */ + /** called by IndexShard to record estimated bytes written to translog for the operation */ private void recordOperationBytes(Engine.Operation operation, Engine.Result result) { if (result.hasFailure() == false) { - final int sizeInBytes; - if (result.getTranslogLocation() != null) { - sizeInBytes = result.getSizeInBytes(); - } else { - sizeInBytes = operation.estimatedSizeInBytes(); - } - statusChecker.bytesWritten(sizeInBytes); + statusChecker.bytesWritten(operation.estimatedSizeInBytes()); } } diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java b/core/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java index 712d1ebe993..571bbfa72e0 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java @@ -30,6 +30,7 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportService; import org.junit.Before; import org.mockito.ArgumentCaptor; @@ -131,22 +132,23 @@ public class TransportWriteActionTests extends ESTestCase { } public void testDocumentFailureInShardOperationOnPrimary() throws Exception { - handleDocumentFailure(new TestAction(true, true), TestAction::shardOperationOnPrimary, TestAction.WritePrimaryResult::respond); + TestRequest request = new TestRequest(); + TestAction testAction = new TestAction(true, true); + TransportWriteAction.WritePrimaryResult writePrimaryResult = + testAction.shardOperationOnPrimary(request, indexShard); + CapturingActionListener listener = new CapturingActionListener<>(); + writePrimaryResult.respond(listener); + assertNull(listener.response); + assertNotNull(listener.failure); } public void testDocumentFailureInShardOperationOnReplica() throws Exception { - handleDocumentFailure(new TestAction(randomBoolean(), true), TestAction::shardOperationOnReplica, - TestAction.WriteReplicaResult::respond); - } - - private void handleDocumentFailure(TestAction testAction, - ThrowingTriFunction action, - BiConsumer> responder) - throws Exception { TestRequest request = new TestRequest(); - Result result = action.apply(testAction, request, indexShard); - CapturingActionListener listener = new CapturingActionListener<>(); - responder.accept(result, listener); + TestAction testAction = new TestAction(randomBoolean(), true); + TransportWriteAction.WriteReplicaResult writeReplicaResult = + testAction.shardOperationOnReplica(request, indexShard); + CapturingActionListener listener = new CapturingActionListener<>(); + writeReplicaResult.respond(listener); assertNull(listener.response); assertNotNull(listener.failure); } From ef192ff2cffbf8dee6b3d2c3a12122ea75561800 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Tue, 1 Nov 2016 12:54:20 +0100 Subject: [PATCH 066/132] ESIntegTestCase.jav: use ClusterState.prettyPrint for pending ClusterState assertions --- .../java/org/elasticsearch/test/ESIntegTestCase.java | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index bde3ebaa85c..1e98754a798 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -164,6 +164,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.BooleanSupplier; import java.util.function.Function; +import java.util.stream.Collectors; import static org.elasticsearch.client.Requests.syncedFlushRequest; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS; @@ -535,13 +536,10 @@ public abstract class ESIntegTestCase extends ESTestCase { for (Discovery discovery : internalCluster().getInstances(Discovery.class)) { if (discovery instanceof ZenDiscovery) { final ZenDiscovery zenDiscovery = (ZenDiscovery) discovery; - assertBusy(new Runnable() { - @Override - public void run() { - assertThat("still having pending states: " + Strings.arrayToDelimitedString(zenDiscovery.pendingClusterStates(), "\n"), - zenDiscovery.pendingClusterStates(), emptyArray()); - } - }); + assertBusy(() -> assertThat(zenDiscovery.localNode().getName() + " still having pending states: \n" + + Arrays.stream(zenDiscovery.pendingClusterStates()).map(ClusterState::prettyPrint) + .collect(Collectors.joining("\n")), + zenDiscovery.pendingClusterStates(), emptyArray())); } } } From 6e7e89159bff9dc60b619c0ee752dd88c6d4446a Mon Sep 17 00:00:00 2001 From: Jay Modi Date: Tue, 1 Nov 2016 09:02:05 -0400 Subject: [PATCH 067/132] ensure the XContentBuilder is always closed in RestBuilderListener There may be cases where the XContentBuilder is not used and therefore it never gets closed, which can cause a leak of bytes. This change moves the creation of the builder into a try with resources block and adds an assertion to verify that we always consume the bytes in our code; the try-with resources provides protections against memory leaks caused by plugins, which do not test this. --- .../common/xcontent/XContentGenerator.java | 5 ++ .../xcontent/json/JsonXContentGenerator.java | 4 + .../rest/action/RestBuilderListener.java | 15 +++- .../rest/action/RestBuilderListenerTests.java | 90 +++++++++++++++++++ 4 files changed, 112 insertions(+), 2 deletions(-) create mode 100644 core/src/test/java/org/elasticsearch/rest/action/RestBuilderListenerTests.java diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/XContentGenerator.java b/core/src/main/java/org/elasticsearch/common/xcontent/XContentGenerator.java index 8d1b8efef51..478f3a8a08f 100644 --- a/core/src/main/java/org/elasticsearch/common/xcontent/XContentGenerator.java +++ b/core/src/main/java/org/elasticsearch/common/xcontent/XContentGenerator.java @@ -94,4 +94,9 @@ public interface XContentGenerator extends Closeable, Flushable { void copyCurrentStructure(XContentParser parser) throws IOException; + /** + * Returns {@code true} if this XContentGenerator has been closed. A closed generator can not do any more output. + */ + boolean isClosed(); + } diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/json/JsonXContentGenerator.java b/core/src/main/java/org/elasticsearch/common/xcontent/json/JsonXContentGenerator.java index 74e1cb5e58f..763fac4c6a3 100644 --- a/core/src/main/java/org/elasticsearch/common/xcontent/json/JsonXContentGenerator.java +++ b/core/src/main/java/org/elasticsearch/common/xcontent/json/JsonXContentGenerator.java @@ -419,4 +419,8 @@ public class JsonXContentGenerator implements XContentGenerator { generator.close(); } + @Override + public boolean isClosed() { + return generator.isClosed(); + } } diff --git a/core/src/main/java/org/elasticsearch/rest/action/RestBuilderListener.java b/core/src/main/java/org/elasticsearch/rest/action/RestBuilderListener.java index cc93e72d80d..c460331afaa 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/RestBuilderListener.java +++ b/core/src/main/java/org/elasticsearch/rest/action/RestBuilderListener.java @@ -34,11 +34,22 @@ public abstract class RestBuilderListener extends RestResponseListener @Override public final RestResponse buildResponse(Response response) throws Exception { - return buildResponse(response, channel.newBuilder()); + try (XContentBuilder builder = channel.newBuilder()) { + final RestResponse restResponse = buildResponse(response, builder); + assert assertBuilderClosed(builder); + return restResponse; + } } /** - * Builds a response to send back over the channel. + * Builds a response to send back over the channel. Implementors should ensure that they close the provided {@link XContentBuilder} + * using the {@link XContentBuilder#close()} method. */ public abstract RestResponse buildResponse(Response response, XContentBuilder builder) throws Exception; + + // pkg private method that we can override for testing + boolean assertBuilderClosed(XContentBuilder xContentBuilder) { + assert xContentBuilder.generator().isClosed() : "callers should ensure the XContentBuilder is closed themselves"; + return true; + } } diff --git a/core/src/test/java/org/elasticsearch/rest/action/RestBuilderListenerTests.java b/core/src/test/java/org/elasticsearch/rest/action/RestBuilderListenerTests.java new file mode 100644 index 00000000000..2bc0d0bdc81 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/rest/action/RestBuilderListenerTests.java @@ -0,0 +1,90 @@ +/* + * 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.rest.action; + +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.rest.BytesRestResponse; +import org.elasticsearch.rest.RestResponse; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.rest.FakeRestChannel; +import org.elasticsearch.test.rest.FakeRestRequest; +import org.elasticsearch.transport.TransportResponse; +import org.elasticsearch.transport.TransportResponse.Empty; + +import java.util.concurrent.atomic.AtomicReference; + +public class RestBuilderListenerTests extends ESTestCase { + + public void testXContentBuilderClosedInBuildResponse() throws Exception { + AtomicReference builderAtomicReference = new AtomicReference<>(); + RestBuilderListener builderListener = + new RestBuilderListener(new FakeRestChannel(new FakeRestRequest(), randomBoolean(), 1)) { + @Override + public RestResponse buildResponse(Empty empty, XContentBuilder builder) throws Exception { + builderAtomicReference.set(builder); + builder.close(); + return new BytesRestResponse(RestStatus.OK, BytesRestResponse.TEXT_CONTENT_TYPE, BytesArray.EMPTY); + } + }; + + builderListener.buildResponse(Empty.INSTANCE); + assertNotNull(builderAtomicReference.get()); + assertTrue(builderAtomicReference.get().generator().isClosed()); + } + + public void testXContentBuilderNotClosedInBuildResponseAssertionsDisabled() throws Exception { + AtomicReference builderAtomicReference = new AtomicReference<>(); + RestBuilderListener builderListener = + new RestBuilderListener(new FakeRestChannel(new FakeRestRequest(), randomBoolean(), 1)) { + @Override + public RestResponse buildResponse(Empty empty, XContentBuilder builder) throws Exception { + builderAtomicReference.set(builder); + return new BytesRestResponse(RestStatus.OK, BytesRestResponse.TEXT_CONTENT_TYPE, BytesArray.EMPTY); + } + + @Override + boolean assertBuilderClosed(XContentBuilder xContentBuilder) { + // don't check the actual builder being closed so we can test auto close + return true; + } + }; + + builderListener.buildResponse(Empty.INSTANCE); + assertNotNull(builderAtomicReference.get()); + assertTrue(builderAtomicReference.get().generator().isClosed()); + } + + public void testXContentBuilderNotClosedInBuildResponseAssertionsEnabled() throws Exception { + assumeTrue("tests are not being run with assertions", RestBuilderListener.class.desiredAssertionStatus()); + + RestBuilderListener builderListener = + new RestBuilderListener(new FakeRestChannel(new FakeRestRequest(), randomBoolean(), 1)) { + @Override + public RestResponse buildResponse(Empty empty, XContentBuilder builder) throws Exception { + return new BytesRestResponse(RestStatus.OK, BytesRestResponse.TEXT_CONTENT_TYPE, BytesArray.EMPTY); + } + }; + + AssertionError error = expectThrows(AssertionError.class, () -> builderListener.buildResponse(Empty.INSTANCE)); + assertEquals("callers should ensure the XContentBuilder is closed themselves", error.getMessage()); + } +} From 523f7ea71ea71d26474c76609ff7e20faddcb34b Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Tue, 1 Nov 2016 13:55:55 +0100 Subject: [PATCH 068/132] Fix a racing condition in MockTransportService#addUnresponsiveRule where a request can be delayed even if the rule was removed. Relates to #21129 Also properly reset DiscoveryWithServiceDisruptionsIT#disableBeforeIndexDeletion --- .../DiscoveryWithServiceDisruptionsIT.java | 12 +++++++--- .../test/transport/MockTransportService.java | 22 ++++++++++++++----- 2 files changed, 25 insertions(+), 9 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java b/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java index 5af2e1c6624..f0c8258864a 100644 --- a/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java +++ b/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java @@ -50,13 +50,13 @@ import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.discovery.zen.ElectMasterService; -import org.elasticsearch.discovery.zen.ZenDiscovery; import org.elasticsearch.discovery.zen.FaultDetection; import org.elasticsearch.discovery.zen.MembershipAction; +import org.elasticsearch.discovery.zen.PublishClusterStateAction; +import org.elasticsearch.discovery.zen.UnicastZenPing; +import org.elasticsearch.discovery.zen.ZenDiscovery; import org.elasticsearch.discovery.zen.ZenPing; import org.elasticsearch.discovery.zen.ZenPingService; -import org.elasticsearch.discovery.zen.UnicastZenPing; -import org.elasticsearch.discovery.zen.PublishClusterStateAction; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.indices.store.IndicesStoreIntegrationIT; import org.elasticsearch.monitor.jvm.HotThreads; @@ -155,6 +155,12 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase { private boolean disableBeforeIndexDeletion; + @Before + public void setUp() throws Exception { + super.setUp(); + disableBeforeIndexDeletion = false; + } + @Override public void setDisruptionScheme(ServiceDisruptionScheme scheme) { if (scheme instanceof NetworkDisruption && diff --git a/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java b/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java index c63f968011e..dfa30874221 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java +++ b/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java @@ -60,9 +60,9 @@ import java.util.List; import java.util.Map; import java.util.Queue; import java.util.Set; -import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.atomic.AtomicBoolean; /** @@ -299,7 +299,8 @@ public final class MockTransportService extends TransportService { final long startTime = System.currentTimeMillis(); addDelegate(transportAddress, new ClearableTransport(original) { - private final Queue requestsToSendWhenCleared = new ConcurrentLinkedQueue<>(); + private final Queue requestsToSendWhenCleared = new LinkedBlockingDeque(); + private boolean cleared = false; TimeValue getDelay() { return new TimeValue(duration.millis() - (System.currentTimeMillis() - startTime)); @@ -386,15 +387,24 @@ public final class MockTransportService extends TransportService { }; // store the request to send it once the rule is cleared. - requestsToSendWhenCleared.add(runnable); - - threadPool.schedule(delay, ThreadPool.Names.GENERIC, runnable); + synchronized (this) { + if (cleared) { + runnable.run(); + } else { + requestsToSendWhenCleared.add(runnable); + threadPool.schedule(delay, ThreadPool.Names.GENERIC, runnable); + } + } } @Override public void clearRule() { - requestsToSendWhenCleared.forEach(Runnable::run); + synchronized (this) { + assert cleared == false; + cleared = true; + requestsToSendWhenCleared.forEach(Runnable::run); + } } }); } From 5af6deb5b5d9d876f8975e587d121808ac402193 Mon Sep 17 00:00:00 2001 From: LakumiNarayanan Date: Tue, 1 Nov 2016 19:43:53 +0530 Subject: [PATCH 069/132] Fix typo in keyword.asciidoc (#21237) --- docs/reference/mapping/types/keyword.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/mapping/types/keyword.asciidoc b/docs/reference/mapping/types/keyword.asciidoc index fa260bbeff6..7c09ef46e55 100644 --- a/docs/reference/mapping/types/keyword.asciidoc +++ b/docs/reference/mapping/types/keyword.asciidoc @@ -6,7 +6,7 @@ codes, zip codes or tags. They are typically used for filtering (_Find me all blog posts where ++status++ is ++published++_), for sorting, and for aggregations. Keyword -fields are ony searchable by their exact value. +fields are only searchable by their exact value. If you need to index full text content such as email bodies or product descriptions, it is likely that you should rather use a <> field. From a612e5988e60cae9325a69f2378f5ce1706ed3fb Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 31 Oct 2016 17:03:06 -0400 Subject: [PATCH 070/132] Bump reindex-from-remote's buffer to 200mb It was 10mb and that was causing trouble when folks reindex-from-remoted with large documents. We also improve the error reporting so it tells folks to use a smaller batch size if they hit a buffer size exception. Finally, adds some docs to reindex-from-remote mentioning the buffer and giving an example of lowering the size. Closes #21185 --- .../HeapBufferedAsyncResponseConsumer.java | 17 +++++--- docs/reference/docs/reindex.asciidoc | 36 ++++++++++++++++ .../remote/RemoteScrollableHitSource.java | 14 +++++- .../RemoteScrollableHitSourceTests.java | 43 ++++++++++++++++++- 4 files changed, 103 insertions(+), 7 deletions(-) diff --git a/client/rest/src/main/java/org/elasticsearch/client/HeapBufferedAsyncResponseConsumer.java b/client/rest/src/main/java/org/elasticsearch/client/HeapBufferedAsyncResponseConsumer.java index da7f5c79721..56b89db1694 100644 --- a/client/rest/src/main/java/org/elasticsearch/client/HeapBufferedAsyncResponseConsumer.java +++ b/client/rest/src/main/java/org/elasticsearch/client/HeapBufferedAsyncResponseConsumer.java @@ -46,7 +46,7 @@ public class HeapBufferedAsyncResponseConsumer extends AbstractAsyncResponseCons //default buffer limit is 10MB public static final int DEFAULT_BUFFER_LIMIT = 10 * 1024 * 1024; - private final int bufferLimit; + private final int bufferLimitBytes; private volatile HttpResponse response; private volatile SimpleInputBuffer buf; @@ -54,7 +54,7 @@ public class HeapBufferedAsyncResponseConsumer extends AbstractAsyncResponseCons * Creates a new instance of this consumer with a buffer limit of {@link #DEFAULT_BUFFER_LIMIT} */ public HeapBufferedAsyncResponseConsumer() { - this.bufferLimit = DEFAULT_BUFFER_LIMIT; + this.bufferLimitBytes = DEFAULT_BUFFER_LIMIT; } /** @@ -64,7 +64,14 @@ public class HeapBufferedAsyncResponseConsumer extends AbstractAsyncResponseCons if (bufferLimit <= 0) { throw new IllegalArgumentException("bufferLimit must be greater than 0"); } - this.bufferLimit = bufferLimit; + this.bufferLimitBytes = bufferLimit; + } + + /** + * Get the limit of the buffer. + */ + public int getBufferLimit() { + return bufferLimitBytes; } @Override @@ -75,9 +82,9 @@ public class HeapBufferedAsyncResponseConsumer extends AbstractAsyncResponseCons @Override protected void onEntityEnclosed(HttpEntity entity, ContentType contentType) throws IOException { long len = entity.getContentLength(); - if (len > bufferLimit) { + if (len > bufferLimitBytes) { throw new ContentTooLongException("entity content is too long [" + len + - "] for the configured buffer limit [" + bufferLimit + "]"); + "] for the configured buffer limit [" + bufferLimitBytes + "]"); } if (len < 0) { len = 4096; diff --git a/docs/reference/docs/reindex.asciidoc b/docs/reference/docs/reindex.asciidoc index 7d1d26d44c5..27acf29ba9b 100644 --- a/docs/reference/docs/reindex.asciidoc +++ b/docs/reference/docs/reindex.asciidoc @@ -421,6 +421,42 @@ version. To enable queries sent to older versions of Elasticsearch the `query` parameter is sent directly to the remote host without validation or modification. +Reindexing from a remote server uses an on-heap buffer that defaults to a +maximum size of 200mb. If the remote index includes very large documents you'll +need to use a smaller batch size. The example below sets the batch size `10` +which is very, very small. + +[source,js] +-------------------------------------------------- +POST _reindex +{ + "source": { + "remote": { + "host": "http://otherhost:9200", + "username": "user", + "password": "pass" + }, + "index": "source", + "size": 10, + "query": { + "match": { + "test": "data" + } + } + }, + "dest": { + "index": "dest" + } +} +-------------------------------------------------- +// CONSOLE +// TEST[setup:host] +// TEST[s/^/PUT source\n/] +// TEST[s/otherhost:9200",/\${host}"/] +// TEST[s/"username": "user",//] +// TEST[s/"password": "pass"//] + + [float] === URL Parameters diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/remote/RemoteScrollableHitSource.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/remote/RemoteScrollableHitSource.java index 207948c9215..7a45e2b15cb 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/remote/RemoteScrollableHitSource.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/remote/RemoteScrollableHitSource.java @@ -19,6 +19,7 @@ package org.elasticsearch.index.reindex.remote; +import org.apache.http.ContentTooLongException; import org.apache.http.HttpEntity; import org.apache.http.util.EntityUtils; import org.apache.logging.log4j.Logger; @@ -29,6 +30,7 @@ import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.Version; import org.elasticsearch.action.bulk.BackoffPolicy; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.client.HeapBufferedAsyncResponseConsumer; import org.elasticsearch.client.ResponseException; import org.elasticsearch.client.ResponseListener; import org.elasticsearch.client.RestClient; @@ -37,6 +39,8 @@ import org.elasticsearch.common.ParseFieldMatcher; import org.elasticsearch.common.ParseFieldMatcherSupplier; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.ThreadContext; @@ -67,6 +71,10 @@ import static org.elasticsearch.index.reindex.remote.RemoteResponseParsers.MAIN_ import static org.elasticsearch.index.reindex.remote.RemoteResponseParsers.RESPONSE_PARSER; public class RemoteScrollableHitSource extends ScrollableHitSource { + /** + * The maximum size of the remote response to buffer. 200mb because bulks beyond 40mb tend to be slow anyway but 200mb is simply huge. + */ + private static final ByteSizeValue BUFFER_LIMIT = new ByteSizeValue(200, ByteSizeUnit.MB); private final RestClient client; private final BytesReference query; private final SearchRequest searchRequest; @@ -142,7 +150,8 @@ public class RemoteScrollableHitSource extends ScrollableHitSource { @Override protected void doRun() throws Exception { - client.performRequestAsync(method, uri, params, entity, new ResponseListener() { + HeapBufferedAsyncResponseConsumer consumer = new HeapBufferedAsyncResponseConsumer(BUFFER_LIMIT.bytesAsInt()); + client.performRequestAsync(method, uri, params, entity, consumer, new ResponseListener() { @Override public void onSuccess(org.elasticsearch.client.Response response) { // Restore the thread context to get the precious headers @@ -184,6 +193,9 @@ public class RemoteScrollableHitSource extends ScrollableHitSource { } e = wrapExceptionToPreserveStatus(re.getResponse().getStatusLine().getStatusCode(), re.getResponse().getEntity(), re); + } else if (e instanceof ContentTooLongException) { + e = new IllegalArgumentException( + "Remote responded with a chunk that was too large. Use a smaller batch size.", e); } fail.accept(e); } diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteScrollableHitSourceTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteScrollableHitSourceTests.java index 3a6a6dc2f68..3cc8c3c5e6f 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteScrollableHitSourceTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteScrollableHitSourceTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.index.reindex.remote; +import org.apache.http.ContentTooLongException; import org.apache.http.HttpEntity; import org.apache.http.HttpEntityEnclosingRequest; import org.apache.http.HttpHost; @@ -39,10 +40,13 @@ import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.Version; import org.elasticsearch.action.bulk.BackoffPolicy; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.client.HeapBufferedAsyncResponseConsumer; import org.elasticsearch.client.RestClient; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.io.Streams; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import org.elasticsearch.index.reindex.ScrollableHitSource.Response; @@ -76,7 +80,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; public class RemoteScrollableHitSourceTests extends ESTestCase { - private final String FAKE_SCROLL_ID = "DnF1ZXJ5VGhlbkZldGNoBQAAAfakescroll"; + private static final String FAKE_SCROLL_ID = "DnF1ZXJ5VGhlbkZldGNoBQAAAfakescroll"; private int retries; private ThreadPool threadPool; private SearchRequest searchRequest; @@ -429,6 +433,39 @@ public class RemoteScrollableHitSourceTests extends ESTestCase { assertEquals(badEntityException, wrapped.getSuppressed()[0]); } + @SuppressWarnings({ "unchecked", "rawtypes" }) + public void testTooLargeResponse() throws Exception { + ContentTooLongException tooLong = new ContentTooLongException("too long!"); + CloseableHttpAsyncClient httpClient = mock(CloseableHttpAsyncClient.class); + when(httpClient.execute(any(HttpAsyncRequestProducer.class), any(HttpAsyncResponseConsumer.class), + any(FutureCallback.class))).then(new Answer>() { + @Override + public Future answer(InvocationOnMock invocationOnMock) throws Throwable { + HeapBufferedAsyncResponseConsumer consumer = (HeapBufferedAsyncResponseConsumer) invocationOnMock.getArguments()[1]; + FutureCallback callback = (FutureCallback) invocationOnMock.getArguments()[2]; + + assertEquals(new ByteSizeValue(200, ByteSizeUnit.MB).bytesAsInt(), consumer.getBufferLimit()); + callback.failed(tooLong); + return null; + } + }); + RemoteScrollableHitSource source = sourceWithMockedClient(true, httpClient); + + AtomicBoolean called = new AtomicBoolean(); + Consumer checkResponse = r -> called.set(true); + Throwable e = expectThrows(RuntimeException.class, + () -> source.doStartNextScroll(FAKE_SCROLL_ID, timeValueMillis(0), checkResponse)); + // Unwrap the some artifacts from the test + while (e.getMessage().equals("failed")) { + e = e.getCause(); + } + // This next exception is what the user sees + assertEquals("Remote responded with a chunk that was too large. Use a smaller batch size.", e.getMessage()); + // And that exception is reported as being caused by the underlying exception returned by the client + assertSame(tooLong, e.getCause()); + assertFalse(called.get()); + } + private RemoteScrollableHitSource sourceWithMockedRemoteCall(String... paths) throws Exception { return sourceWithMockedRemoteCall(true, paths); } @@ -482,7 +519,11 @@ public class RemoteScrollableHitSourceTests extends ESTestCase { return null; } }); + return sourceWithMockedClient(mockRemoteVersion, httpClient); + } + private RemoteScrollableHitSource sourceWithMockedClient(boolean mockRemoteVersion, CloseableHttpAsyncClient httpClient) + throws Exception { HttpAsyncClientBuilder clientBuilder = mock(HttpAsyncClientBuilder.class); when(clientBuilder.build()).thenReturn(httpClient); From 7751049c14ed9960e745b0ad35ceb437d2ce4260 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Tue, 1 Nov 2016 13:51:53 -0400 Subject: [PATCH 071/132] Add version for 5.0.0 This commit adds the version constant for 5.0.0. Relates #21244 --- .../main/java/org/elasticsearch/Version.java | 4 ++ .../java/org/elasticsearch/VersionTests.java | 1 - .../action/ExplainRequestTests.java | 5 +- .../ShardValidateQueryRequestTests.java | 5 +- .../OldIndexBackwardsCompatibilityIT.java | 28 +++++++++-- .../bwcompat/RepositoryUpgradabilityIT.java | 8 +++- .../common/util/IndexFolderUpgraderTests.java | 4 +- .../index/engine/InternalEngineTests.java | 1 + .../ShardSearchTransportRequestTests.java | 4 +- .../resources/indices/bwc/index-5.0.0.zip | Bin 0 -> 265095 bytes .../test/resources/indices/bwc/repo-5.0.0.zip | Bin 0 -> 125399 bytes dev-tools/create_bwc_index.py | 15 +++++- .../org/elasticsearch/test/OldIndexUtils.java | 45 ++++++++++++++---- 13 files changed, 94 insertions(+), 26 deletions(-) create mode 100644 core/src/test/resources/indices/bwc/index-5.0.0.zip create mode 100644 core/src/test/resources/indices/bwc/repo-5.0.0.zip diff --git a/core/src/main/java/org/elasticsearch/Version.java b/core/src/main/java/org/elasticsearch/Version.java index 219eaa920d5..e9e950ce80a 100644 --- a/core/src/main/java/org/elasticsearch/Version.java +++ b/core/src/main/java/org/elasticsearch/Version.java @@ -89,6 +89,8 @@ public class Version { public static final Version V_5_0_0_beta1 = new Version(V_5_0_0_beta1_ID, org.apache.lucene.util.Version.LUCENE_6_2_0); public static final int V_5_0_0_rc1_ID = 5000051; public static final Version V_5_0_0_rc1 = new Version(V_5_0_0_rc1_ID, org.apache.lucene.util.Version.LUCENE_6_2_0); + public static final int V_5_0_0_ID = 5000099; + public static final Version V_5_0_0 = new Version(V_5_0_0_ID, org.apache.lucene.util.Version.LUCENE_6_2_0); public static final int V_6_0_0_alpha1_ID = 6000001; public static final Version V_6_0_0_alpha1 = new Version(V_6_0_0_alpha1_ID, org.apache.lucene.util.Version.LUCENE_6_3_0); public static final Version CURRENT = V_6_0_0_alpha1; @@ -115,6 +117,8 @@ public class Version { switch (id) { case V_6_0_0_alpha1_ID: return V_6_0_0_alpha1; + case V_5_0_0_ID: + return V_5_0_0; case V_5_0_0_rc1_ID: return V_5_0_0_rc1; case V_5_0_0_beta1_ID: diff --git a/core/src/test/java/org/elasticsearch/VersionTests.java b/core/src/test/java/org/elasticsearch/VersionTests.java index 9364383b9b8..42ea115bf43 100644 --- a/core/src/test/java/org/elasticsearch/VersionTests.java +++ b/core/src/test/java/org/elasticsearch/VersionTests.java @@ -276,7 +276,6 @@ public class VersionTests extends ESTestCase { assertUnknownVersion(OsStats.V_5_1_0); // once we released 5.1.0 and it's added to Version.java we need to remove this constant assertUnknownVersion(SimpleQueryStringBuilder.V_5_1_0_UNRELEASED); // once we released 5.0.0 and it's added to Version.java we need to remove this constant - assertUnknownVersion(ShardValidateQueryRequestTests.V_5_0_0); } public static void assertUnknownVersion(Version version) { diff --git a/core/src/test/java/org/elasticsearch/action/ExplainRequestTests.java b/core/src/test/java/org/elasticsearch/action/ExplainRequestTests.java index ad2cabefdbc..1bc895095ba 100644 --- a/core/src/test/java/org/elasticsearch/action/ExplainRequestTests.java +++ b/core/src/test/java/org/elasticsearch/action/ExplainRequestTests.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.action; +import org.elasticsearch.Version; import org.elasticsearch.action.explain.ExplainRequest; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.io.stream.BytesStreamOutput; @@ -92,7 +93,7 @@ public class ExplainRequestTests extends ESTestCase { .decode("AAABBWluZGV4BHR5cGUCaWQBDHNvbWVfcm91dGluZwEOdGhlX3ByZWZlcmVuY2UEdGVybT" + "+AAAAABWZpZWxkFQV2YWx1ZQIGYWxpYXMwBmFsaWFzMQECBmZpZWxkMQZmaWVsZDIBAQEIZmllbGQxLioBCGZpZWxkMi4qAA")); try (StreamInput in = new NamedWriteableAwareStreamInput(requestBytes.streamInput(), namedWriteableRegistry)) { - in.setVersion(ShardValidateQueryRequestTests.V_5_0_0); + in.setVersion(Version.V_5_0_0); ExplainRequest readRequest = new ExplainRequest(); readRequest.readFrom(in); assertEquals(0, in.available()); @@ -104,7 +105,7 @@ public class ExplainRequestTests extends ESTestCase { assertEquals(request.routing(), readRequest.routing()); assertEquals(request.fetchSourceContext(), readRequest.fetchSourceContext()); BytesStreamOutput output = new BytesStreamOutput(); - output.setVersion(ShardValidateQueryRequestTests.V_5_0_0); + output.setVersion(Version.V_5_0_0); readRequest.writeTo(output); assertEquals(output.bytes().toBytesRef(), requestBytes.toBytesRef()); } diff --git a/core/src/test/java/org/elasticsearch/action/ShardValidateQueryRequestTests.java b/core/src/test/java/org/elasticsearch/action/ShardValidateQueryRequestTests.java index 34c6999f4e8..c1d18146a08 100644 --- a/core/src/test/java/org/elasticsearch/action/ShardValidateQueryRequestTests.java +++ b/core/src/test/java/org/elasticsearch/action/ShardValidateQueryRequestTests.java @@ -42,7 +42,6 @@ import java.util.Collections; import java.util.List; public class ShardValidateQueryRequestTests extends ESTestCase { - public static final Version V_5_0_0 = Version.fromId(5000099); protected NamedWriteableRegistry namedWriteableRegistry; protected SearchRequestParsers searchRequestParsers; @@ -94,7 +93,7 @@ public class ShardValidateQueryRequestTests extends ESTestCase { // this is a base64 encoded request generated with the same input .decode("AAVpbmRleAZmb29iYXIBAQdpbmRpY2VzBAR0ZXJtP4AAAAAFZmllbGQVBXZhbHVlAgV0eXBlMQV0eXBlMgIGYWxpYXMwBmFsaWFzMQABAA")); try (StreamInput in = new NamedWriteableAwareStreamInput(requestBytes.streamInput(), namedWriteableRegistry)) { - in.setVersion(V_5_0_0); + in.setVersion(Version.V_5_0_0); ShardValidateQueryRequest readRequest = new ShardValidateQueryRequest(); readRequest.readFrom(in); assertEquals(0, in.available()); @@ -106,7 +105,7 @@ public class ShardValidateQueryRequestTests extends ESTestCase { assertEquals(request.rewrite(), readRequest.rewrite()); assertEquals(request.shardId(), readRequest.shardId()); BytesStreamOutput output = new BytesStreamOutput(); - output.setVersion(V_5_0_0); + output.setVersion(Version.V_5_0_0); readRequest.writeTo(output); assertEquals(output.bytes().toBytesRef(), requestBytes.toBytesRef()); } diff --git a/core/src/test/java/org/elasticsearch/bwcompat/OldIndexBackwardsCompatibilityIT.java b/core/src/test/java/org/elasticsearch/bwcompat/OldIndexBackwardsCompatibilityIT.java index 7bcc00da5ce..3009f7d5c3b 100644 --- a/core/src/test/java/org/elasticsearch/bwcompat/OldIndexBackwardsCompatibilityIT.java +++ b/core/src/test/java/org/elasticsearch/bwcompat/OldIndexBackwardsCompatibilityIT.java @@ -82,6 +82,7 @@ import java.util.SortedSet; import java.util.TreeSet; import static org.elasticsearch.test.OldIndexUtils.assertUpgradeWorks; +import static org.elasticsearch.test.OldIndexUtils.getIndexDir; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.greaterThanOrEqualTo; @@ -445,8 +446,15 @@ public class OldIndexBackwardsCompatibilityIT extends ESIntegTestCase { throw new IllegalStateException("Backwards index must contain exactly one cluster"); } - // the bwc scripts packs the indices under this path - return list[0].resolve("nodes/0/"); + int zipIndex = indexFile.indexOf(".zip"); + final Version version = Version.fromString(indexFile.substring("index-".length(), zipIndex)); + if (version.before(Version.V_5_0_0_alpha1)) { + // the bwc scripts packs the indices under this path + return list[0].resolve("nodes/0/"); + } else { + // after 5.0.0, data folders do not include the cluster name + return list[0].resolve("0"); + } } public void testOldClusterStates() throws Exception { @@ -481,9 +489,19 @@ public class OldIndexBackwardsCompatibilityIT extends ESIntegTestCase { String indexName = indexFile.replace(".zip", "").toLowerCase(Locale.ROOT).replace("unsupported-", "index-"); Path nodeDir = getNodeDir(indexFile); logger.info("Parsing cluster state files from index [{}]", indexName); - assertNotNull(globalFormat.loadLatestState(logger, nodeDir)); // no exception - Path indexDir = nodeDir.resolve("indices").resolve(indexName); - assertNotNull(indexFormat.loadLatestState(logger, indexDir)); // no exception + final MetaData metaData = globalFormat.loadLatestState(logger, nodeDir); + assertNotNull(metaData); + + final Version version = Version.fromString(indexName.substring("index-".length())); + final Path dataDir; + if (version.before(Version.V_5_0_0_alpha1)) { + dataDir = nodeDir.getParent().getParent(); + } else { + dataDir = nodeDir.getParent(); + } + final Path indexDir = getIndexDir(logger, indexName, indexFile, dataDir); + assertNotNull(indexFormat.loadLatestState(logger, indexDir)); } } + } diff --git a/core/src/test/java/org/elasticsearch/bwcompat/RepositoryUpgradabilityIT.java b/core/src/test/java/org/elasticsearch/bwcompat/RepositoryUpgradabilityIT.java index d7ed0d8db5e..ec8c12cb525 100644 --- a/core/src/test/java/org/elasticsearch/bwcompat/RepositoryUpgradabilityIT.java +++ b/core/src/test/java/org/elasticsearch/bwcompat/RepositoryUpgradabilityIT.java @@ -19,6 +19,7 @@ package org.elasticsearch.bwcompat; +import org.elasticsearch.Version; import org.elasticsearch.common.io.FileTestUtils; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.set.Sets; @@ -70,7 +71,12 @@ public class RepositoryUpgradabilityIT extends AbstractSnapshotIntegTestCase { final Set snapshotInfos = Sets.newHashSet(getSnapshots(repoName)); assertThat(snapshotInfos.size(), equalTo(1)); SnapshotInfo originalSnapshot = snapshotInfos.iterator().next(); - assertThat(originalSnapshot.snapshotId(), equalTo(new SnapshotId("test_1", "test_1"))); + if (Version.fromString(version).before(Version.V_5_0_0_alpha1)) { + assertThat(originalSnapshot.snapshotId(), equalTo(new SnapshotId("test_1", "test_1"))); + } else { + assertThat(originalSnapshot.snapshotId().getName(), equalTo("test_1")); + assertNotNull(originalSnapshot.snapshotId().getUUID()); // it's a random UUID now + } assertThat(Sets.newHashSet(originalSnapshot.indices()), equalTo(indices)); logger.info("--> restore the original snapshot"); diff --git a/core/src/test/java/org/elasticsearch/common/util/IndexFolderUpgraderTests.java b/core/src/test/java/org/elasticsearch/common/util/IndexFolderUpgraderTests.java index 94fc0d88752..916926e36a4 100644 --- a/core/src/test/java/org/elasticsearch/common/util/IndexFolderUpgraderTests.java +++ b/core/src/test/java/org/elasticsearch/common/util/IndexFolderUpgraderTests.java @@ -211,11 +211,11 @@ public class IndexFolderUpgraderTests extends ESTestCase { 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 src = OldIndexUtils.getIndexDir(logger, indexName, path.getFileName().toString(), list[0]); assertTrue("[" + path + "] missing index dir: " + src.toString(), Files.exists(src)); final Path indicesPath = randomFrom(nodeEnvironment.nodePaths()).indicesPath; logger.info("--> injecting index [{}] into [{}]", indexName, indicesPath); - OldIndexUtils.copyIndex(logger, src, indexName, indicesPath); + OldIndexUtils.copyIndex(logger, src, src.getFileName().toString(), indicesPath); IndexFolderUpgrader.upgradeIndicesIfNeeded(Settings.EMPTY, nodeEnvironment); // ensure old index folder is deleted diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index a77c402c74f..f3a235217ed 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -1856,6 +1856,7 @@ public class InternalEngineTests extends ESTestCase { return new Mapping(Version.CURRENT, root, new MetadataFieldMapper[0], emptyMap()); } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/21147") public void testUpgradeOldIndex() throws IOException { List indexes = new ArrayList<>(); try (DirectoryStream stream = Files.newDirectoryStream(getBwcIndicesPath(), "index-*.zip")) { diff --git a/core/src/test/java/org/elasticsearch/search/internal/ShardSearchTransportRequestTests.java b/core/src/test/java/org/elasticsearch/search/internal/ShardSearchTransportRequestTests.java index 8c501d71e0a..819f93fcc0f 100644 --- a/core/src/test/java/org/elasticsearch/search/internal/ShardSearchTransportRequestTests.java +++ b/core/src/test/java/org/elasticsearch/search/internal/ShardSearchTransportRequestTests.java @@ -190,7 +190,7 @@ public class ShardSearchTransportRequestTests extends AbstractSearchTestCase { "ZXJtP4AAAAANbUtDSnpHU3lidm5KUBUMaVpqeG9vcm5QSFlvAAEBLGdtcWxuRWpWTXdvTlhMSHh0RWlFdHBnbEF1cUNmVmhoUVlwRFZxVllnWWV1A2ZvbwEA" + "AQhwYWlubGVzc/8AALk4AAAAAAABAAAAAAAAAwpKU09PU0ZmWnhFClVqTGxMa2p3V2gKdUJwZ3R3dXFER5Hg97uT7MOmPgEADw")); try (StreamInput in = new NamedWriteableAwareStreamInput(requestBytes.streamInput(), namedWriteableRegistry)) { - in.setVersion(ShardValidateQueryRequestTests.V_5_0_0); + in.setVersion(Version.V_5_0_0); ShardSearchTransportRequest readRequest = new ShardSearchTransportRequest(); readRequest.readFrom(in); assertEquals(0, in.available()); @@ -214,7 +214,7 @@ public class ShardSearchTransportRequestTests extends AbstractSearchTestCase { .should(QueryBuilders.termQuery("foo", "bar2")) ); BytesStreamOutput output = new BytesStreamOutput(); - output.setVersion(ShardValidateQueryRequestTests.V_5_0_0); + output.setVersion(Version.V_5_0_0); readRequest.writeTo(output); assertEquals(output.bytes().toBytesRef(), requestBytes.toBytesRef()); } diff --git a/core/src/test/resources/indices/bwc/index-5.0.0.zip b/core/src/test/resources/indices/bwc/index-5.0.0.zip new file mode 100644 index 0000000000000000000000000000000000000000..422b25877600a46330372ff80dd777cabf169c48 GIT binary patch literal 265095 zcmb@tbyQqok~WS*;|{^4aVJP1Ap~foad!z89D=)BH%{Xo+}$m>27(25f%I|sWY>; zlo_v`sOnRKT(KzBMZ;g+`KA#GXz*x*uib9wq+)FXP&V~JSBA#sLR(u)b8BmBO9N{+ zT_EUR#K5JJ=XX};lBLK(X(P}9Y!tPrLK*=fQgHzU}Vc*u4?i1D6JVxE!cPAb9TIyC8dterMFMtXa)+zlBVX zn(^$hS*(@V*{bTlD0BDy(y-Q6W8ah!CG6!T?bS1{J)%y|48gFgwpjPhQ_;f(cIKS1 zUUm~N-?R^OE5W@e8lKGgM)ciV@2dIP&sR@9DGnUU%XsGRT~$N(k8|qLi-tLlO3~Ul zTupxWtuURottt&Sy%=`vPq4pa9+!(2-ACxX@kF})b*c9FNVA`MuVgG_?uh~!5M9Q1 z<|T+!i}ZgwVPPH}Q(!NP;sSGway&ckGjZSkjAXlj{UmU9`kl~&r}tRivmtGnn2~R+ ztOGIsSmj*Tsuf_YD&q-t+?ZJJ@~56meGGy}ucC-+$AJ5fit_ha%=u4p^#8S5jQ?HH z5&Wm^xHuWxJKH)~{4Mzc|NHXvx8(c&$6)%=VC)J24o*M_4o>D@H~88EE{w?F|+r_#6^PW)gJe5N^)#7YzOJ{xMq^gQaJlk`r?rY~OEWt+i z2|LTy)~oC6!D}Lw>U({ObECOEDg8wMiRoV0-OUeyW=4LGz3x1|tu*|0SU@ULXOwG!%F7SU$q2Lkr!Z2f{PNlhcmo zoAh72@2z(oD-u6PB#(I4)?bhNyTGScT<|?>PVt9YsSl@2ACz96MpNn&C3n=4W$|Ry z1jShxWU=VTBq6&i+7G(KbZYNGwVS49;>}?nR5A$RHSoV&xK?M5h!dvZs1TlgfRg7b z2h)xAeWg}_?25Eo7!l_c_&{JF*>sZpI64M2OHBBsPScGEos7brW%iK^+R?Xf05WNs zkUhkZVppm4SQNEnHBUxygt&9ahn!8iwh&y(g@@~mWSJx;^bl56P8FmA=0HilWL%E- zg{+a(3)*xhKZIpNS$PBXB)`v`Gn4r_y{2{pe29t5_c&nG>ybk z?X@h3KZY#Z(U$x|8g zjh8gx3E)AOZT7kGd!Ev1LE~6`=^S{bjRd(h7i+ZmGG^!uyo{h(Ca(t$Jo+W6)JDU( z$y&`mQihO{7r6<2rP_kHycq%I7*&KH%iaw`ngTx?Os%Rs*#wYT$*HZ2@e0V_ zo~EXoD>xtF(MkpCNu!%#DaHSBl$Z_5% zi$THp8=lo=_}VThb3s_1NHC4YJI~u?0jsDfP1Wg+Q52^TyI(_qsUM~)u?!|rP50{p z`RJs-a8F`H6(DW?pMPC$p$a5H){>4TQ<-y8e(%~ncN2l&((D=JXWrjsi3rF>S;~2; z+1oLdEPk<^$jj?|OuJ}mX?=q5J*!o}Oi=_r&3%Zg(&e*`!NRYfCXS^)S)?9cBDLPO zK%1JFsFNeLwbrdFg(6B#Bwh_5Vamx za^)$1TY7~w_ZJ%bu@r*K^<{+|2d-&Dl1m0rduLb^J=(q-0-w9{k*Mu=(s9L*)?La+ z`p*HFABo24L0pZZlQ~y@{kOJ`N2eJw0U_@B?zjb`P#c^D8lLxh0CKw*Duh`)`DGvR z%Y>p$|2oi$tR6b-?IVH9Nx1b75G1?h`G>aNqT}5t@kxD$9vUIt$^!;q@_NM8VHuYv zh!Q=iQol!PP&9;$U`vfQ#@8XD zD+YP*y*kpWb3v~0^^U-5tjONjW~1JbvJxocu&RE68re_!OIsX$o042$sob)9wduxV z;0u&*Gcnlk_-D8>vlXPqpJ?mKFyDm`il>0$pPigmg~@@toV-N#jqi2@ zHcBY_#P1Pk4`2GV4(;MvI-7uSgI6zP+Z-V>M`+WBu`VL^@uXdpG8YPxCM==K2`3CP z6CC!fGju!-&rvRMQSVd`+Bql+ADC5VtKcL1ncZm&Wm+tjacMRi#}SbZI?_x|WlW@6 z5f9i)SDjVin)KjZTF3#@t%t>arsOdyCm-H8ke`yhO2XG~a$p3VR++<-OP6e%p-B6V3voe+^C}F*!}IkA=FRRWGTOtVG`pE(?rIr2;|(t<07o+xdx13C z)b3XXB?~LLH!~b*;L}$CVbl%xVLgEAOsUFssY0(^b0JAa6=N=w9(>}6T}Vl>-XfD8 z;NrJZoAu3H(|$Au=9C-4<$e@(RfCSHi(aIh>O@k&Yj;eJ?w?3jhi}Qvq_OIN&KKpiz`>r5$auWNFARXMlP9^^E4bHLuh7fVBRY_8c0@3eTT~ z6>+LkO@&7<3*HO7+YB2CA(p+#_iOAnEcI*2O(S2EkerdOm);||SPrEV2-sw{3(~ zGH3MkEt6r(uTP53Ng{;mg_&7elo516-shYQnX)}dp*P`3#`CMYLZH3(`?i+f6qTw4 zgkF}rX3wH%AO#u%Km0U&R?on~tEUGb@lUTMAuY*#$jiaY8)Rdpz7zb*w)sDHmq$QN zZ{VLskNb~E{O@;{oTfIP|Ixc+#w?pR~jAr&F<2uYGlOgiwo=~B0}?BMq&+C z_fNZp1)BXS$)Acsine!18TC+@kcmAX%##W;lJ~vH3y`mYyW87LOv!MK0~SS()<~>T zPR@LotQfVJW3GwHEaA2ySVq+tCpcEZ5g~nTkE)k%hOu#m7XkD z`jWxNGL3zVeUXv&^1Qe@aXzRk>H?q_poqg`V1HvKk*#um*$7z2t}@DTI9)dq7Ieq^ zR+=#0uj2K{VbS{Io(iM5P>xURqd4hqo*P}yz9lzDo*U- z^3^D<>lJhlh!ZN;pj}0{F0zfHZ-&)b(hnA{BTm_J0xSPxf~IDAa$cE1cVE>6z&%)}J<#)6x^R(o&L* z#^8|Zv%kHvyC_+zt1;iYCi?Dbhq-cmK5$nkIZ{fq_@ex$4v`tD>cou>_YWfwn8stt za3GKq#~JDW87&zeCmSVZXy>J%6uqIjP9q!cNo?I_3TN0&EHcru2>RBtHQmrKhAy?T z+fB$_)1UIGh?|F}H)|YJKmgDg@5VDq)>Y3;%NZK`Gs+Uo($2wi6Y?jM(XD=050v0y z=qgAJ4NSYw{ss*p9GyiBKb8eE zN>5BccN0amTbby^&q+<_8xcpGzvvGpF!nJtvA5LJ6#tK; ztG)6*_)DIf{E`1nDD2(-=TIE}Efm(rDMpEU#~He?)IT%5;r}Wc?3l{Reb^R`z1l=C zfdMansZiYUL<+RXe}}##>cm3uugL$m=*|8|BJKYt^#2`lR6fS8j{yjwTcrnohgCG$ zTv7TDR@wihr_HSXVSW26={Nt4^tf?3XE zFdqgok$-5-&Q<3$&YxgKxLiy}Hzd-~(yZpnV4yGr(IA-M21NvfKtdrwkkpSVtP$`K z2nPh&Bw%%oe^1EI@zi}t1R?u@+PkFheD`J5yHaT0C)=oAP>PUs#rQGqLMltE>r=`U zNfeno_-M*h8k=n1da^~VWqw*oZqBdbSAq0N$EUS>Z#~7Vo`x3g$FNu7o>bN3Yc3s) z9?R$0#!5ZJ{ojW;dmMt+vAbUPdLH>Bp!AO+=He3OE{SO`(o3g!3P*z+KDIfUr<$^fI0;c%g zxMbR2(q2@HAGtsCCI($>+Ivvc!#NW!Q?}ziAU$&+G2#87BbBJhuHoHe?#+Q#(F%*d z%e-ZUxfQs_*)wLK;PC~0`uz5|Uq4hQ%9FGnz8X!SU$c)ZOqAjU<!W935sWp7tbvmXXAM74~1tdpp&1K&1mi$b)%R8n%W;o{V5qdtldDZKU zqDSTZ{#y0e0I3*vO{OK}%r`F`v`1hd<$`fu+%FAYqw5T~#H%wgOZv#%5u6sbw|1Rz zzSf%^bNVxuu|#Si=QHd1w|-QJD6NhXaol)_-Ugz4fDolK&aPYsMF---r`v(^RV@$L z7XdL*akL9OO}dUiOR6a~P&L6X`uCyRzOZ$$d%`Tkk&+wykKl41Q=W33oK2!`8^}D$ zsfDfGF>jD~h!(yglfFzQ-VNiGOnKZKgKxkwUkL`m`MZuF53HK28quwkAqT0+%*oEz zcYtubKRsXS8Bd-H=mz&mQ9o?qsfY`<&0r-zow#ia<0!b3Nr`(eJx3VUcabWVnsZjR z$q7I>c~rQ~IA8A;4S0F=jDzHkPeI2ncFVWv4Pu#k1saD+;aAcu$?69$ ztTu;(spy2ny@(GG>QKy1zP8~e8w-ET?&}C}M=_M{h&;3JZAWe=eBpb1ji!s+M7tt$ z%c8=QXANS5Izuc79O&KL)iLT6Yf$ElF^lLvv8-`-fITKmO^*d0zX8JVDrhTYTawNc zdMD8QfuoX%;Ux@H%6S;jQ~Kv@Gq@37;6!`}SU0VKZVvRo;J$-UBb& z+h6MAkSWQ}>Op^$7mi0Iv}@cergU+AhzIInTS~z#V)EtP**;@3tL6Q z*mNDUN?lEzI4{~*4R?){cYjPl%ck>zPIB;uLm3HHB`=BNz309%a*C~Fdmw?eVA5c{ z1Z-eukPE?a-*Hbz^gNmaLlgQn-|^=P@1hlIyvLy1!gJ}3_2{mIkHUd^LGNkVrEifO z*|yyJ?J3D6wur-(zz&SQ@s}EP=w{=?g9)o}M?4JN20ZJge*J6GpQ!@9Wlm}Cp>Js2 z%1yMkjv!hIKWJV_)xr8ZXS?F&=fO-*7GCU^(8AoUr<$nVXn&U85dc6gT{-n;XS4ZTzVejroauaf6 z)SvVH4pE|%{*=!*NCRktcqk#rXb<>?lIqDNZ>L&8Ik1+-8?x!U0wdTgla0(=ge~7Y zIB9MXZfBkgZq&23jjL|^p|7z|$o{y`g|}K;?UJ|jk2z?D>ZRm_neyB$iaSTB?5PXetIbJx}}DO7FgA!SyWxKnhP^iW@pC@{>BA zDo~gk^)>Em$_w5#s^gIbS#5VFpYU~(7nR52Gk1b*dP{H(WWy>55h;hD&BzmbN@`2B zp9s2|evJjTo5nV?ujB*>nlf%McxapxZ$5=ZpRKB$@r&UFQ3N$1trIjc9e;QeaOSh+ z)h`GB&XgwQgfT*3<~h7lt%m8$`iVGOZza1wX3#KjRdOSCU4SMtY)D6R$CMXm}i*T^p+%XWY_0FW&BL z-3+-4Ch4_*r*o1UmEWoaz6PHFE0u(UFD2^YhjmZq1{ploO|jgFulbLQZ!I=&dW;}l z7%!ATtVjv~2PT*Dafy%qVg z2HQ~`*!(zsvAzx|zA{OkPL}7e1t@ONN4xQzaLu;&?B z5nmD6VC4e&x!aa+uw6`6kT2>3*0a}RA5trtEbb4Q&+Qk~pkn+++WVx1B<7Ta_loy~ z2kmC|@R>N#xqwXM6Gcv_6{Hon9sVc=&s0HpNcDgw0^MGz z5DR=mrG~J5f~^aPR$LTQYxK6&v|_;%*aEggcO)x-@moxc8M}Q0&L`3;MhHig>-il* z^Sy}FesABAKOfxsSQY0LaH~6?PP?h z_G}&aCgCHAF;u~We zBkq{?C_n$&*XCVB6X-n@hFeW96ufORO;o^@FiUfU#qovcF=&K4*HRS0~J6}QEH(gCO zNZ1e%m+-)!qRHEXtO4)iTZqp+w&b8p1naUL$!5$W3_v-V6Pml=21*_By2wrS7L6~Fq9dKO)H&v4CfG~aw*VlwItX{;Bb=+3pJ;g2l zej`{*p$EGa|Fl?v_XvKSwK9BJ(9!qzgL3D*6?zg?LtFu;n)Mw>Rk*8C+Hoax@$Ijb z=-x*!@V_~Ue~dBX8A&Mclvzdh5%vj%a_?}famN}rZDvKg(?4Sd{YE+&?}$B<%Y%V_ zL9!X_6*@Bd8!=Pjsp!7RX(o(u#A0nK%BzhZWf^#z4OIce{+7oO(aVObc>#`=A(OiiTMn701MvkTHS zU0XRJ>x3_?kMn?cLH9_~XzR3xU>AldUOKJGY*`=9KPQRBGS>;%G?Y@p5w`3tqRLpG zUpAYX@={;(kQi&4==XW^X6r@Um{^!N^gf5IXKv^Q9Rc)TZfSGN2YvHwpzmo;BvbkyB${vr}`z<32Fx^XPqU#Gw7*P2%S_o8=%$2pO&`jT6xoL;pO1sKg^VSOwMt zOiAbv^Wr^FtSd7+X>SoYHW(xK$T2k_x*_YaM!1=XMSF|imeh&~r9B6)&U6M>pzGMR zpnvK{xg~T)HMHn(^zc6a+U?w_q;)K!tz|{s?-1?J-=lzZ!sruTgR&_JqJ+v3w11l6 zAfpE}#>+FNiTx%&&{w^|eo+)au@?8<4^APk<~J05r}fJ}NEG!&8sSDig`op_MxXZp zm6lyjte|zp7=9{v7)jEhgL_ZSP!rQ(*#(d1ZT|SG&zhr0$*)ZtyiK36us`GqH&apsnaN;}Mt`_Gd)yjRue{Jk(B@7!b7ueB|lr3|R#X;VLk-Wi9Y4 z2;#peI`M+RFezZA3`+bMosHsLnk`M5H>O_i8xMRnqM!B$tR7ispG=i{i3i#w+v)rm z9@lPTaepw7f;I#itI;9})+AcQzQ4(rgA(JmVx5p|Ekn8((aNbQ|{^+hCQ?qCP?A+?CwEJX#-DxOS*in^Q3vQsp+#hdV;9 z(3l9SXgI;|QWuiG8pV46J!Q}nqUaZ8z0gZ-8Ybx9Qm;^ZalgY0Sf0x}3 zUa_vhd!cv)-0Gom(tMssXH&#<^xq2X7fkSx-c867*UYS-hzk40gM7BM+4`Q1%ME|d z_k_K7DMS=EiQZajdo;rxtebGj=;)oWJiaCsX9h-WSfZ zfUGfoO}=G<1r*Rg+0+aW7hYkT9GZri`t`LVoND^aJ3WhQht^Q5xaJMKOsOS~YgM`2 zArOsfv1^vf?9ge*YH;XOnqBeo%Vz3+{dAx^V4YH(&}^UP6?&BNrcx#tA1hLjb*05J zLA*G{TDMN8_4TL#*J7UKH}T5RY@IsSlL6Hqt##M(TUs(zw3M=5f(N?O;WGY+^N`l% zE5r_2kLFDg$j$q;a1Yd4_$ER(PB)=zv18QR;w|^?oY2$5!Bey${56Gr-S2lGF1&Wt z4&ISaAWj@gK@|ok`e~0lsXO9v|F>W*f>XDS=32_yu1#_uEzxxu<DC@gWyRbHM9$ULw8$?V-$pW=0-YFg|pYjwBqxt6bz$~bbMbq3Vb8Q zjt~zLunx=tOg~LmV3B~$lqGR!|6>uiTRL^kt2%(BEV|Su}hvYhyb#p))C%<=12lVF4zHcF}?+?tiC1s+W(^)bAWxQ zOxF^RnzvOIPFJ%^AzHZK8CSsD_Vqc?a(o9m&oDD` zsGf#+o#Z6OYt(wQi4B98y{7rW1;qvD+VHcZO{1vOCXQyH0PTuIL+AqiiR8QFTQZmm zusMN(j#9!4paZZ(wO3({x+9)=ilg3aZ#r=gdPK5Mb6xiJ?4=4h8G48=K~iDKW9^rK zhU0P3yUqSqgw25466Bdgk{TGTOsC7ig0MEaUux6A1vv=@SoUl=*yWpeV;{$;J!lQp zI$n8H={ZJE+}&0M+y$JWO=W1 zLJP3LIItryHG^;HnB9kj;VL98|E4f5?na&}gfwU~{ON03-(!h|@LGqT?Yu=)J#0u- zKzs4+>(HhCRY(r76g-{dLQ&d2?#D=uvSTc`OW$!(J6rp+#(gWc-~C%fyTd6E9YTyJ zqImoq6WN-dHvVden7p4VXqrD4iz_sF-7AS~~|0=z^HnO{M zJ%==d`txvmh{rzE_RkWc(kx9e*wk|}sC16;Q>%|UK=jo*QK1$dF(bR|Lh5J6O#@IU z#0&RHY6WtOy>%nQi$5QuL1PEHCtjpr#(e5Yy5Z_mIzuf&{v^7K+L5mv$q`}~m=){` zSnuNs?!sLElcg%0r%pL={V$*Gi>1&T{qgk zkvi}-;kcn*8$41xE+JJlWJ{j`d+I|(0H}C|3N6{6Nv4oNIC!1n+b7goIW2y3ux+|m z;!Z3hf(2Eyx)Qr&V|qNr8~lS0a2dvOrVf4YBm8-Dl{`>3S$p>YqBL!(!%5B-xDyO( z60o=yCtu)!$#kLFui>z`XgcTenNJ<=wSp!&{cV252!13!&GeUaEg2hrK}vua{3f7_ z(x^c{8Rq|#|UiDx)yLrJUu_Mvh|3$OVn}b;d*}mlVe~x`iAA2_ju

v= z(bNT$n*pHmBMhev&W_{o+I~`Oe%okVI3?iDD9Oa)MIv*n;B%l7PDHabe4`JffZ-bG zlTPGEu7K289lwILsE`?c4}BHm-ES`K76VjL35tMeO^?AyN$@EB_9zs7&Nmy$7}!XO zF34{Z#%mBl&;wLM91x@3h8~TxbLJ$Q2Xz`EbI$J1QtX|hMcjfXLmgY>uQ<2ILCeG< zxUFD_&il;(8Jz=G8Dxlj`6Xse<=1K?z1RcaBnZs?X6ce(%LSxuTyQ}kj#+;XQe;uE zElJ@=X3!-Q_ByoUN)|N{iv>9BJy-?sR;*FP0WalmL3yoP-Zy)V+s0Hkc0!wP$bSZ7 zEH5s)xt^{z2;U=)Z=7jI*1+kR&8Usy#(5ULYu40828|g+4#aD@i=44rTLsxKLtQrC zGM9~@Wd_rKoI3h)1sh9}_`A5$_c zUG$plsEK-zle@YD+{)ZAMAk`nf&vwUuf$f>D%@iX>Q^X?Xp;u-Aun9%4?$g4xuBjiX@@AKJR&w0OL~ z%nr_pmA5R&bq-?r{BlWXYLuxyeNj%m?R&!CvM4u%WE#}?^ub^kXmNBjBb<9)fKYW< zyl)N62#eB+g_bb`WNx~h#qRt73ghngLnYs$d0X2vL^4>Wv~!cI^k71hkt3x{X1*lkk5t(kO9_mA1G^lQV$ zfm#k1RyF!n#LdSa4VZk{UBBQH?yVWOzMq{}MCRnpZhMo&C!ZygLY7L269k_)?Z87F z&-=0lynFU|C`Yk(+R?5XSXFnPm|hbT2}tXJgg{Bu)b@Xfh4`!OYdVazmWbk#shDW> zYyQE?&CNPqmNLIwi?y)PjIxw2l*5Xu#*Pj+Z7LCA$20e4&m(Xd{2poxViI2QICe4= zcCuZpbix12!!du#%?&5Jt~=LB$_-Yev9--)R!iPsK;s}hh5e`BYiC&AYb~N_u7!jF zh)qzGs&q(~ahT*#%ak>G>aW(;sA}T4%{Naup7(k$=V4xUPQT9lq42 zyit(CVZl`VVDoIf7majo6Q4lsvz)Nf&7>!myw?y-t_Zm0kg;)hY4in79CGyisPaJY z9X_TL8`t##>91clWi-g^GM7HZzkgkn4bB6DHIB_{h}=rb7tS$5P0do5fTTBXasR#n zlbqWAhqncJr(A!!a-4!3emT1FXS4g_&ijP%>Qaly<2+efL1q^7qqC@Ebp# zE>Uc~zQpU>YLLct7@pN&@q;_Rp2X9V`B_kfHBYn6K*j`v_f29Q{|)~?pGl{7Gj&9NT}(uMdAI)$Tujh^T}(Kd z{eMjZ{trIx|C{-g5TBKfE1!^x6{r4XDGf}HQ}6jQ@8&vdNuObjE~SCiAhp)n(Oeol z_zwaOf{iZx{NOnn&UeMZ(Qfiz-gErbOA%+;khdyCXos(FFfb7?#o{Tr$_#plP8-2O z34!-7c*a3ll3ghn%PMIMTdceXpu@>qi>DU(J7kO%agpW{YwN1D7-M&L*1Iuk}v zN7Ob)jDH+n@EkBD82R0dUICHBW7+A=E9pUTbvZaoSLxt!?V3>8)vl)F!;JTwoYRPF z6TUd4hh1fSQNH$>>kxqNqW~t%{h#C}w{|f^+39Ls|2#-t1jX&?#2nXie+G{IB<*vw z10?4VzbZ2wzW$z!dezhw`MuHKYQd9Sz!g#Vmn}JvS`pjg$ePvxBlw`tgO-W?Xi%S4 zf!*ZOi58jN#EmVvhFT#R5!9fJ0Q9UtV0`p+#PZpa!V-hwCJd7)w|RvwmC1)qPrsuk z3p)DzbwpR~w%Ct#5R>{$ZL*f$7ewLVf4hozH`*HG{1+k+hNnsIPm*unT=f12%i-S| zD$ggo#NaPE6?9qGH#)#oORD{WLO~rK>0wNk5YrYbGz=r8h|3iX`SJTZv*7SI)2jA1 zR{Kf7QM$u_{Pi`7G#vOit0-wSDEaqnFU?hLwJ}|#drZ2$-(-Tbfh>hPZ)Q(IPvZCwSEN-eLD_gABSDy|0gy-S@E+?dqf zf$n1{qdt5WL{MALAXgKzt|nQfC=0_bii+8&2b?sN^IWY>B;7X_e!6}&w)hZM;3wM?_|E8KOLqj(oa9h|VtYEKF*DOa> z;ij7jyDc3kP`A@=XqNm3kZkb}0P0_W5>$x=7w^jU;WocHkx{m1kAt;#DvXXw&serZ zx@)MZzd<)JU9unYnLcvW?@&HPo@&*eK`$(sIe0p|RtHqIJn`36e_JDbsL!^P%^76@ z$8=A#5?4|xt|e#he{oMy{zT zX2eSm6U|8gnIuN)Vo@eyEy{6wYVtXW`>sifUtx&?mK(JKFm*HShI(15iLxX#vjVq; zS^Wy;Ds}ZQ60VYjM8j<9v(di*n*RWlP5%LiS1_BL(CBT`@IA8V9Tbf&{n#;iYIuoo zbe8fi_m15fzTFZZ>dN8nn|fmGQqb3y%REr{!u2lOEZB^$(%t5a`@~m`P3fSrb`eus zy*%Wp=)9$_|TpBptm;emW29Rms|0iZ2zM-|yt_8EYqT38a(ztQkI3+SJf=dW4vn zWz)%Dgo~Y*akU9mbPEJ$`{CxNLgFtmR?S^(ys;ROhbc+MGg&U*_ zt}1oy(!?XBv9bD_c5}BDW$lJe{(k@}<_7?YD(Ka!u&s6?n2vZ0LFU^aQzDZq75h); zNSj$zRaXhOs4S|jT@}WwCeOUEiKc+W=)|sFSiJ^ToqZMgBdtu-3ojwn)7#Mw-cCge z7bP~IU+=H?S2sD;io|#4rLz+8%I37W>J|aXSMh0Uw*{Wn^ZT$_FPor{|d|hD=hi1@X|kF%KETU zjzo2#NqHmKXGJRL`q6fCqa0SJBO}Rk#%djb)DU)5bEtL5tMT*<_d!^G? zO~I$7D!D7wL}FeKa%f679Y>-!&-xPc=c|Xe5nGMY=Y+uwrOsyQ8FR1Y>_6INc70RW z3ze^d>QWz1!L8~QQ)f#I=I;s{!jWfeZ<4RYdVS!|89`Y-^VyY>7W4bi_)Ar5*-=XO z&%L}$Hu6)S(fB&WbJb!7b$oqOJSC|wSy*gl#6tmH#|pPTpI6vtSX}_gXQnNdajY-h zzVhq*yfcf+=3C9LT$Ez48?9nAloGH3bG}e@*;2HG60ievzCd-^5|q&$eraU;m>8x2 z9Q^(LU%}u2x?wp!X-M(dA2o*6{Z+(O0hRj|f))AYKgtcy>PEBPOFh~GYbpA&!=+U+ ztT{HE+>h_(PYUKVilU`;aO1r>-krdYBW3^vOd=+glUy0k9MF!`Xs6V(YUOo;CMA=M z83!DDj)P~C(>~3Ta$zK!Br7EIBvT}#Bm*SfByA*(BsGahp;Tg&kQ5_ip`>CYki;W| zp#);T3KacB7)5r47uE9Wl!uHmCfze&Ik25rPtF%t^6FGfvL?OXH;na9 z^^bQC^!IkR^*49d^jCHl_2+kI#9LJBnrks?U2B%B*{f5ko+?!yu(k7iVa5*@goKLuB-O6tTi%R&I1SX}5oFL+DCqq zLKzYq$xfBW_B>W5)1%o9d=CC-;51)SjCvMR8(WnsVgBXx5s=r+Nv{kD4tS@^Q_Z=S zN(X_X!but}>f6Ri+>90QIhaE1ae52``bFLJ9#?^L7r!1s{}yn67;tPD1`Kohaoy&g zpLwf4cQA#qA{a5O=!bL*d#L@zz~f;HF(g^`pXPhA^Z3a8C-vdLwP0H@9odfcrsrZ@ zkBz3+vZMKQ{Jp{7VZkwAn3t`phg>NG|5C5PG-99As~T6Y>jw6S`zwKS!a8Hxhp5?9 z_vq&JkoqrxTfi)0l{2N7&~L@`O@8+e1P_Nn!=`3XHLqUL4ewF*?*N~gTiEaH0rd|C zkA}g+=3+24|6*vis6_@;lk9QU)0>&ij^x*gQN=HkT1l>^)-##e+$gjH)7hGJZD+RQ zdW`(3z*S%?F*Vut^S%4F3wvn%*T7u_Prx7Z<0Uy#9GS}^5k8{4z+YkEFmRaJ4Q-dV zLwe-=4Zw$BJBJ|sHV#Pg7P9T@7)ZW;QH{Ep`%^_POe z&}W=COc}9_M zj=;nNNy8=4(x@5K%xfio=*o|S#fOt7OIoL_GXwPP=l4@a?BbD0Nz$2D%xfnM+eYBx zGf3Mcozu=4Ry4cLPiWx*q?H-c9#rdOvm~+RgH& zKfgqXDf|$PhFV>{zSh)gX0Y_Dl9R5R>KjZ=Y6kSn23W(z(Bf$dRV5ozjIVWo)(n06 zg#*;#D`*bXhw9^Xs%AB#yJY2!ss=TS`WXXX0#UR!nnexE>Sa?UY=ZW2((nm1P8w&m z^O_W!n%~;C)M#qdjZbvcHLB({$?F1V&MvCMKqV{Zy_FNk2H9yT=lMob_=_y15V-C zXlyjvdr8IJyHWMcQW9y(tQA(1s_6~vCf5>^8k*J3rgkH{wF9U#iBckI(ySR5=EMOP zn2C-xC+2e#g>6#sX_>4YmhNkJH76FgOLuiAHcZ{d*%QCgdRPT4gH~bcFs*Mvg_k4h zF>RRmv_-8*S0o#O`b-PPsajS6f7v9PfO<@G#z_6DNIj835zU4cV+XC(9=&=2lVDmnE!ugdN5ueXm}ApK354oD;@5eZO9DAAN8VTm_CgXT6>2+4N)yD}S+s zM=%^5E-nXPy|Kl>Vd*e)!Yd7e75{enF!rfy0y(V^ZUQHjv)=Z2Z+dA5`2}$2g0W3M ztry#;A4~!#ftv)NcF7bcm^TRQ6AxB`W5O}!7`IQkpdZq!?Bfdd(d+E{9vld_hcn12 zX`6CPKcZLLhZ6ibsTgbqH;4PciDS#YU%jdq*{2@t1eaB_spsD(94rL~@%6TrcFU*L z6MAh5U*9OEJOWM}XO46G<%{Yey{bMQzFwouG0$Y)>h6QgKCfU1ICxw<0GG4t?(}>C zaEy0$y&zCRB3YTXLTk0T-pTA}cC)}=<14oSTrG|kN9U8>*@c3%F{I=I+A+=6wl`#M zP50J2`;&{=p@J$2-sBHjP)+dGHzt2g@HQAb%u~j3qvo-n$+5I~n!;^gx%KYDNUotB z(;94vZ~<;J&KSjw875QEUNI~frH(l!_rZzaL~-CauwUBt?&SB`01j+dcOv@Kf*s(F zaB0~!9h>)US9ZKA*)^S-4{bMgg8HO_HFo*oTyQRx$q3FzpL{WCnY4^s#+`wCwbMqi zWBSQ$gx`|Y*InGUYbT7_#^91OXxlWM+s^G)&Tcp#TFzYn+xF83F~1M(`wxo87?SrE zM1JDi_a5YrfhC92CTo77?c9!Uh7U?5yeeBXU0QZ(dyNXlD3cdxEi@Nf%N?E*?rE_# zSz9gbSI%n(jLOEOk~L_XvZM*#KwUJYThg5Yd-jV5>0@p)twXZWdU&uHmkhlI`9tK< zGk6AEgYFTx7*`BEdykTLDfr(Pnsa+{UiscPR(g9Ch4>uH*M9M_)Zv{tWR) z3*l*SH@Yo|{%m9N9&nDv#pC35cEfpaJ-_KMC=p?bKE$(`?#(L@p^9F_v*KEHufH|D zn(4{Q8%lG$IX<5m$SV_Ji@wC;=5lwxyE(r2=Nl!2AKi)f&h_E}^B3coVc4L0$U2%E zugkbJeQaQzy z^~^KorTkQWR~Cq3zW}eSdQi7&(mwSgE0ibRlk`>c8hFYyVG`2om=(zb?TPk6eWP|> z*KblX#hAsyBY?X2Qhuv;Ro7G0B!i#T$@A{{@(S}ei2`bNP@G0UR^T=W5C~nFqi;73Zt8&CJ@QjDe)B0uorsZOeIa}@btuGn~ z4X7}AJyi3a!_raalvfr654;!NGuMNgycUX_%=B{FjEr9VlyMdfkA_#{v*qK;PTibI z@|5+Cx?c2@c9s{Biu34v=TUKE0`mE z3i>orw#n}*-%Sv&?=RqY(3#?5jj3T^QmVY8jWFSC_B5*H;K69PD=>35Wz}5||~_GEOy# zu3LAQZz4#UpE+@;7p4}9GDJE2GD05-4^S9_xpYfNW?r?d`3V{PZc;JF4 z+$&Qm6$~6E5+k#r>hRwfL;_@f(jZAFAT@J%Rgfl33yM8p-+Z_==AHnXpEbx5%B-Xs zU??%X7-L1SCg2=!4zq$<%V{X1O0yqBCP+17H5?n0M=-=66+8|N#IOBjs5eY#1{pX& zAiEPq|js`UJ!8zIz}9$%m8vF1eZdr5jRjDP9%LjW+b?FAkYOG$0%ljtF1ItfA6(>E|Jk(Pk!87AX%^#jc^%_v#M{;YRcz zY8+|meF(utbRaww9uH5!ulc2~*Pk2mKBGVpdZGUVQGzf@m^xe)uZB?HsDCg-3NZ}~ zA&>@9gXmI(E2J60j^V2_(?JLoq6!#opatR*k+tA@L^G-#=dM_PUx)}I#1F|pf5c$I zuh}e=9m|e-*R?-1Bp$IS_p7}9G0#w>MbbmrvF&JgJ^M34+7KTJ@kE#-)e7lZjWx%g zQXm=P4GR02!@s5h!a}2QdpDafGu%1SLMzF(lm`Y}eU3rLFd&`^w%E^00fV3cm}pEy zCPrh$@#7Q*h75z&{&K}*tT~!O56S41Ook49_aVthAQ}^cabjA4ySQC^DGr zPmctUW#riM9P*D3q)0F%8&vjdMz)|ja2f~QMfQ%Ts4>(T9QKb#0@0axjrqoVQ-thT z;`(zULU$1vB=nQSA0>Q-%40SW9t@O+%0q#~!GS4G3}*(c{gx3cD0Kj1iSfl0D?+52 z$rM}$P6OwG^QaZ%IyPgS@sAXX1ra0#l*({oq#&{gQ4HWTS&j{VSHK4(C{p2Qngjo1 z03IYavOCG0G)4+Nxt?urOn@0-X}|*{E)oZ+qxf<9G)ay{?|TY_Oc8M*6<35C$}Rh$ znB-i5G2{fYI&r5-I9A*?1_5Jd`RGNzK0NTJ1+Stf0Z@8Z>S z?oI!4Hoi3^g1SXUBc_&AO{gZ-v+8XPxc}-@*f=(b>P&ntxtv(dsVCDL7!VF=f;>r* zE>jvw+&G#8*s<-64akEWLV6%Qm-uSf_iHEnTy00O_auM`atK+PI9<{-W0v&5xfhpD zaL2lr8}WB20FFl00m9vPRIY3eMiQ73;} zf-%K}f_i?vq*-Dulab2kWdb(^^uA$YEsK%H=tBZ7g@?jJ!MV&y%q)XZ4u4U?B!!lI zYoVjWN#-n3fr)rk!lv0*j$5gV6^iT-M2NhB)Vx-n zB%o4IDJU0|ORL4#vKocg0`!#>IeXj50AOWgc6E5`9&&888tHAd`r=llv)a-x;rW1 zwupz7ud0TH!e!(zb{suUx1&Fh9od1}g$#CFUE z$^5qwXqYri>PGdGrtveJc{0LV5#E?>s4pfk!_ArYL*zjW zZYek92Q{vI!b=g>Uyj7?hF3gKpR}Q$0{D~%;(<4x)BlULb9}BWdfRl&6Wg}!bdnR> z)`{(O)ajsOb!^+VZQHihaXN3$)Xax>rt0_Ut=jt!*tPakweD5dx-Uu}EN_d)aqY}H zN>llN!O0o#fP;Eeg~K$bQ_W0%IyIh+%T#xmacm=s4-b(Y+u{=EkNMJIdqs;x}F@H{9*kE+D?g8F5`^}M7scnh2~_c8$%nOd!+Rr$9fiZ%$pF?CBh zHLQzWRLZ!!z1R=p`^|$H5VWwW$3S{}A%`KdMoB;EOW5Bk0Lpl!?9%47@vRez7J829pmIlL-ca$$7PdRI=S`A<`LuS zNe$$XNoVvN>K5gV>efZ`x<%2ldP(2uPbfLm`Agl@&kE;tiXvsTk{sx_G`veQXHLuJ z<%(Ly8I#uNUDc1v3Dx-uU9_ttn!U6Qcq88kG@Dm8Tq>NUOjO|!+n82&kkL zq>&=Q=Uq4z9ut~3UC-M-H(he-Gg)=98vcwB1dD>(0}t1}k*_naUwW(@mMtU1zq^Rv z1)qEfZ+TBi-wE$|kDWIUf<|P1s}Y|FA^Q;C@t!x~%K)C+{?Fc+229HNwdrZURA&iHh? zbL7uzoKm~tw7+m73I%10vE;SPN}V!1VqC*N7h%bmbZo4@Y@%HwXa(E$yYGcKsjbpn z;vn-X3&REj`$2nQPU^hWggAojRilC)_{1g=$`n`N7nN~)9Gza*W z;HG|l(C0f{xS=pXP;@^vsG(M3mEI1c8Bri8E2dAlBWSCi7*tcx4eHFx$?F06gL2;= zybgl)N*zyaPAyIythfQ}0B(SD%#88?b#F3CZhd3lV#4uz#L^z~7L?vV{ge`@4lC_evlD(3uPGe4MPD4(Mx3rh0j|;_c+DiKm0- zzhBZ2-w>~uRxN2IgeRmWgjrUrDC2&UXe!B~D3o0xt$I?~g1#ZqF=_3sTPEK}7oBC3 zP50F>tzc5)nC>A7dR)vru3kHUYFyPktzKUzuTou;t|EzRT$ies|0vzXpe=7x-GQzn zNod^AJZGPKZ5;e4*G0Omf>1MtRw~JK+}WPWTbjd63;8HhS&_aaNno7rsL@4|uOd@z znt@7RnVCK%abg_nsJz~AE!R$+n~rF$(oUtBek1W}9NE02UQWkE$3`z0cfJQlx^}XD zv%bH6etuzoVt!+OWPW9SCcd!8+}}LcJl4F{Jj}e*Jk`9_JgA6>f~kP9fXS56mg$1= zg2|L=M|*hfSphz62k|5GE*U;KJXt1rESWC3EZI8QXWX|)2qiIeoPx)%HGEFNcbubW zNC7a8ctl}HU$Z%w*Qv5iZ@_54w8gl^G?k3}SS#N$Sv%P{S^vW}&Hi=zzRNtuUP5d& z3yAH?RKZxmWXnjv_`vwUgvf}<6v?>8*t}Vr*^t?yU#s7^*=W~jS8vyRSUXg|R6kX} zRXQ9n}O@&~TZGRM5247<=L_Lx*C@DVW2@Xrt{`UZ7IP-qnU zR}=;c57$gH)>^a}HVy4#$!f-xa%DXoqz0+0x8c#*P*z=#`Pph-+u;8E#;@nsw6+$) z@io6bJn%KUUTomEvaU$*bza1&(Xh{wUY_-uRIEDiRa*Aj=SN;yd#K;6`e)mYZ8_-K zk#nhaA1B==U8SZ>aiIL@pu=3%qg?yEWuy=!Q_ZRPDxDtklQ!#3+Zpsq`gxQ>$St6G zGCH&V@#^SDpNxcx9bZYE9y*h$XC|{i3}5 z%X=#WQQf6))1k0mDifAu&&LBV$eQ@#F%Jk?B+gpRdcLk(Sgy7G)7bpq67dvZskSg& z@(>ou6gzTLd)?lrgGfmawQ@ zC&Z$IHD047#yTUM6e<}XGtPJTYroaIeT!@*#A?GP+uQbxvxP=>s{WPeben*znru7d zusLJKz4!5BCI4FZGFR&Khl7RjZM2AWA-k@YIx$l;Cl!5vw1oLyd!Wg;G`*yqMCl;` zstS`Kou#_B{Rrv)nwaDvd~~FtrPn`0Tyup+#16C{XVM0{{t`rk-;qV?e(efmB#jv~ zH^|H%IP+`^19yTtet6_K)h_If{v3&%VPQgO+(c2wD%|Iqi{IS*PPG@Z|JG0bm9X!r zDZ!|)TCc6fuAz)q6GP)daW#=&7WgJ^K)$y!LV&*Xio{GQv4-`kahDUghz85O&kYBX zT5NsKxgizjSVQi37nC)Sv*b#wLxcS`6|yO>WG#7rIivz%yr1h%e49m1?*2T2A)g2sJw#NH6c@jF?_{9D-omeo|K@J3u%{$? z&RN+B;6o&eikDx`ucmoY9PzrYUWF!095d&{hzR?2X4#2Oe=6vuWo(X9!rKGz;^oRJmq z{baB(TY4*`83}ToC+k+pZ|!12r5nnIJvejY=jqg6w-;&0#4jM|zymrA65R3XW*#~e z@=Ue=W*mOv+%)}c;qhJ>Lw-Xsi@%J9X9vj)JF;e@TA3{|Juj5ML-}^s=(p zTl}!U$774Y4I~4I> zJ!zREQg8cmAlNnw-GiQgGm*DeCu{BI~qg?;Ty z5z7*Hz!|``E8v6k>ls>=pBtGX-L?+Rc0kP5P_Ex1yK6 zHhkyTl=XMjrYyhH_hxH)6|)uOYB1L?`DyC1`ITuiCIW@5dQNE6cPk60J+GLC^724jjt?QjsoL0ZVdqm2?C>2$h=TfyK z6J+(lWle+=sqRDH>`bGPd_eb^q2k|$=Ymfk>Ta!yJ?|HAHePRg3*1|?i!1fdjC1S9 zBQHlJj*NE`Dxl7;u#jyNqm(oNc#x!b5Dd`T;6%h}-Bq48qfgL5aO;i-d@C)&mJ;a} zJkcm8(n7Jr3!0@j^3KM6$s9Zj9%Z1Lf}fM|EtIac&$FQ#jn{A}Oq_y&%YHFB~B-)xVvnP=hs86!_AgvVH*zBb=YOuR}zfv={JD?nkWtaf-LcG&kB~H<}Zb`m3O-I6~nFG(f<{3NEVvL zE5WuJX0G?j{P;uEVvt|Dxsk^Z=QERg1gcLJmnKmCN5l3CSCB~TkJ>#(E8lyL`tO|X zinwjn3^aA`(UM6jmP+c`dz@~|oxdr>_=2~}qv{C`=96?gz{MQdQfdc@n&v)9N`%Cr ziZtTCgW+O^+sTUA(&Yxb+UK13>~BM(8vAhv>xqkEbLF#1=YYflcq@*vdqie*(geSU z5Ke--#{^d-7NRGDJOa|c*S7|$bFLO45AH1oi%9IDBOUzQLZ6ZBNpRt+5_rymElNi~ z;Q+tJmzfi7ZrNXmJ_;Y@KNrz5i!L3E@Lcas67UX~T=In@%f`jVrB^iTom8l+*A8*6 zkD-9Z;ZcK!4!wG#u0$FSo^DQ>44N*i#H?lzOJ_@yQ~5U*^(w^4tE}7N!i{KS1G@6! zLBv@88<0%?!NIxVt}(DLGTTcp#<2T;ui$;C6oeKx+)t7yG3QrAzB4w*<-%jD5QG^n zx7A*KgA-XRWo$XbF0)Vjh2{xJ>*~knOeQ}QnpD+aYU9GI`v;+)HIC#Ji|g9jcRBx% z&h+h%H-+O}1{NhY7@pM{J8`ynRXRn=k=jg5ZT2k%#U4)PVwUa4!r_K`(i(wE3-rd0 z10pOkFgeRiMlL%0Z<#a9G*dAM@29`(FdD4sVL%+)HSC_}A8tU*5CoNvs4Ozn1-odh zg~Lei&=Wko*9NgYq4zyJDb@T*%0)}BDBgCl{PN5`tiT&E~MSJO-x!8){tFq+aGtZkJWRwfFy0{&YeYUd`B`jd9W~_)B({p?nqdAx=mOZjrnFafhBqLtBPB^Cm1+mc@LLOMy$JO@FE%& zG56rl#m%Ev=KBCj&=s_~Ju%!o-%KpNd4bj^pbFT}DP9YP_fK#LI8~y;{7#gJpzHB0 zAmLe&SG8t$gzU-Uw`mscxPETN%`NGr{$r=kVYpT8+e6VN7TZ8tfb{&tIn(Y>(xxnQ z1WS#nF7;-5%_b-s!H~^T1S?jn7;%R*rQN{2RfY~}oH@%lIuiA0V}f3hyQGY7t8)o% zc~UGD0rSgE5TX;JT@(lVR;=KkHi%%EabrTe6d-@aX7s+QLN; z2@@sn@zq!HNE(vT#8hiRacJL<@5JLj(0nm0hwqKyS=t(4wql(Ma4&#kzzyGZP_92O z_qoKf%kW~GXs5bstVOAv!XN1iX-&nqCG*SOeB}_FjZL|Y)0M`<+#in*jF=G+(E#!N zIY?4vv6=2~m*Pa=d%0PxS9@u27<4#V7Gh_1Z3PH_om~eGeA4_1-Eym|R4ww@`LU z^M`9bGH*c&c^I5$nP7}cFRqGOUAd_NcLwST(3u$!OIwmhgUYgsv3+A1iXVDjE|k+^<26~O6orj>?uVy_^qY@T#Bd*G2S?y zkypY@hEl=u?AY)Ea7eG|FMQ1C$*e+uMl=Djx8<8R2KD=`xXnzv{P~>!R9v*nF75Z6 zY;sX@ba{xDeR}FItt^A^&s{wSQosJdcZj_=f?0Yef<4vWEm{X$xEx{8W%M2gxI~-& z{#=s0xP~uzlE+BQnRS{rmo=HKa1iSO|NQp#Po*pE3Cj6{c|@gx5_~x?R_jF|j)?26 zfiB~f!|SlgUGVwv-a_-itSplmF0f2hodFBKw05gb8g{}?fy|nvN7XGo9U}1h@1RuO zs4xZ#U2k(bDttP3!sY=VRGc>!B&2%vmk@>jpm&F`R-(M_p2_FExR28$iA&N>p{G! z?#wG!8v+7A(kbqqN9r}0(Rm+L96X>}h<3zA1HB&)HXd8PQ)PX&K3#=1rNiu%t43?g zPW1B<-@jP%7?U{4kDF5TF|MCT!=)K;9@cp)08uHJ7xtSr7T1S|50&3nZwv^AT4cy& z9+WqcQaOb|5o7W}w3dNQ+zDcMEDlCn8@R(P>0!%s#b*cIf>m;@D$dtASI9#^7}G)L z3bKX5Bu{CJOn;{4+nvfSNbKE-PIIH>d*OU{kwjb_ui^rf@w$6vP!;oi{Oa)@UsE?% zA&xw;h<=S1&rr>CgbK50w{<=xOQV_dRd&6Ld5faNq)hIfPJQT7U%u=R;}sR^*TSd} zGc9maY0?t#t*kR215+sIJ?p}=PEt+}PAw*ol}V5cJ`EoO`vHG6cr`kp+9sOP@}$$0 z?k`KT``ZiIofY)3Gu>|jHHTf3$7B_I1Pe|YozV~FgLJNd-`-|~T86^P&f&x-0WHq= z5`k&r$zTntJb^#RqWK09qz_LPY%`QlgSxgmAb8#{LmI%^TaLx+V%fop{&O>}@pASc za&gQ2B1|Apa?&3^NiIu%W1Kp@z{YnCVo1kcy`6RPGG&hrs69u^j!mKQOF64BTElXW zvB8thMa4ddB5}(n_~x%24jz4RgIHi_=q6SL`dEHTabN=JJeAhy{01bSy>)&`w8OLtLR6SnXp|s&cB{ z&nbAJ%SkSqSOQ?Ycju=M5I?K|Q>sZe%*wfb{37MJt*k2=ES6taokc~= zBk$7FhP(dAe}29`?`CfP-t8P*82NsA_`RU~_lUlv^Xa0c0tV)5`#+X%%^XetQ$)Y& z@>4d|%O*L#dEOlG^=S0CF}z;boEz1Bee%VRuGDC>iPqG_lqi}@Xwr>lv~6S`QVp`g zMKHz2z{U`x4P>)mi=>MfF&1OZ!%ZFvLX8+=%%hD!l}FpW&UXquSH@gg>NB`PK09yh zSkJ1iRy^_XS@tWY*LbD)l7F_vU!3nc`7!hS{z{hh;5#B~USL0}9QTIyjz2(mhZ}-JVWVMlpqo|KFz9333ANY~_FS;WG;T~Ms zV8DiL?4f{_3!BVjVU&U=7$dP7xw%Y^zcnX`zb~GrJ|EN^L8Q%3;Zk}v)GV=*#YUKE zb5_BDT#XH^JD8R|fVaagfTW+^CTY@50feuqL#zCS5LhuuD{nd?B`qBAgvJ<39I&x> zl3|Sb@i}G=@Mz~O>l+cea*$eYhB-nWvxK2Bh7%K}q8JDDc*2MtgZgS|NVMG5*#82Y zuq6Xd6l!o~IMD~Q(6AT}P0P!hNfb zD#9bS8oR5tyYt#34t20^6buC7Y|@&4F}!wl8UCKA(Ouv$Z0Kp2WI?b<>z(fu9j5yj z4G;hRpZJ;{R(P!JB~20dbFUnB@HDf0Ka#_EZV1U$hC8NH53kNO_!lUf7hJavy}MS- zYUi>g8tZ#rE0dJJfYjA73(`h_j1b1%cgc?=CtfQUVKKpb;tsEZdYG(eN75I}?S=9q zmOFg@=34~n%0l4U6Wn?Th>*vh2FI$(59e22eY{mNMmk!=E8~?>Q{~;0+HwXSb+je!CF&;+!TFLJ z*0J|T{)B9gkjl}mZO~icP9-JoxBy4`{>Dna>GL5aUKHm#TY}9zMuv)QMB3!5^xmd{Cc73cL9;bustIg8iq6ya9t@sF%26H)NB*zhubpRqJY+8t+-@ zzJ$aoebc=NTAV&30;A>|jd)?2e$Wf< zpZ>mK*KHhM=0M>`b;B1#@tLK3hv)VnM`a#152e|p;IW>wmsKlbu`4HHR+(CFxl10e z@q`YR!|wj&y+Q9IZQBfn{EDtyz9!Vy9GXXw?3R=gEa`<=@4SQ1X#a~PaUanWMv-2} z6`8)6=Pp$Nc<87c)%MP#ydNiVn?eE~F?V&?70%}7C1%*x3f1hSeLSyMN&SKMKt;kE zymIR=ggL$~cZ&S8U#TBZ*ohJxLj;PG;MDRd?ijjulZc1ECQc1j9bG$%DfSY~U+bK9 z2W&%dTwDTU0nI69ex~X~uuBzg8*OWq75O{T=kn38jpjcfDoCGQeyim`H;rUq(F8n4 zFfHs_-8cz~9$_~dW85DxjK9^IIw5)Q)8Ojnju7v$zZliK_kX(~9L^rtt!14i1n#Db z8%(s3i11MG z%wtP>8x+Etk=p10<39-gq_)E{=g{y1R6XiaPA+xDu`a~u&Y%aC*s>q$T2qQmIUcXs zO}o+P#m6pFU^Ct;f)xad%ed_W9TKiAXsMIMbX~_a?Nm5{D=^2#=q_8+q9xuV+_#Bb z-Niq)B??M(JtmT8f9fCDHCP(;?!r#$i@Z>9TBUUohd0mo6QU$jIW8KmP!CZYv7MeN zdytv!{Gz6!Olks2CsXOo8P4gxrPYGA~y=4|Y zN^5a}cp1$KUhxrYFaQ^|Prg9Ci}1E^j_JKE`fgl3^@KUCk`64NvzBJ3+D$0bc7Vp| zBZ|xh7Cb|jatlMeRxU1gBBS*fF$^a5$&{+W0+CE(ret~W3z>++A zFK$ysO~f%V-Ypa6SCE>?tzf-!TXeJpB7~dmw2-?<$yo-r0JhFMt44RG-2kwAs zejmf_62yomJs`PzHPsBwjk;8IPkWZR!F27}@R2O5bNz(*^TkFV04h!XC+D2pI3Y8+ z>5$0SatOMgv@T#&zR>^ffP(!{KVvTq?nTs}(d$hPJWWmnx)`1mrPr|0 zIgcJhZM`eV+;RhKkF-psY>3D7f}doFxT2$OM@v)#K5ik&zn-G7YDNn=#!F@&s z4P%I}?lVZAaL7i^aKKNOeiF`KqdMyjEg1_!0Be4qI?HBRTFDCe!L1(99m%R#Xwmc~ za-LCz;fp5a(BQA+mTrl(=Gj7ff^JdwBdVCqc#f*I#GPZfYPU=t>Xo}OvHL7g7O5cr zbnA?EqP4Rqr!2uc*_M^-8xB)Zmg0rx?!)nQK3U+@83W@+}+*@GlT&9Gv?WYZ2vCeCTo8OSj~axp5d){AhywqAIF zN}W7*j$kCckFbnN)OAc)F?fMO`-!01{|prtD1j(|SR679CxKBcg{hI{H zr~OkJ#IdZh0sZSpo1aR8AkgNEtr&)LZFpoC%ci=|TJm;Kyd6|}`jq4Efqk*AUN*Bw zd}!E$EkrKU44Ey#8!}EDEW`;j(GTA}i~*5t&TCR#YnpHaDiT^J&I+cn)7{ObdcV*K zCAOU_K%%1I8H6mAM-Mu}&SlxN1{gq02t@J?5OYr93Ox}6mrNv(t;a+pGhvwck@<+n zeIf@?BibHKZV9qGzC0l9Z*&)&c28Y zT$?NU6vijoBksl&%O~SQn6{bf?+E1V!w4gy1Vs9(Gx|GtgGs4C4dp zySP`>*_el#IaLDQ2vYMuxQPBL?gjrqj940&-W`wYDN>tLUlJV`=8*31J zpx_*L9oH+IEC(0I?BX`}^mnpn>^1DdKb7pp1f)8!h?f-qG}Kd`Q*Rla`GTo3jU5t7 ztxP7rJ%$Fl_&&ifT$Q!HAxW~b0V|Ae=`KIG$DES+{b?{wjmo+w3=@mts0XF3FgpKf zeEqSeJDK$`Qr>^}CUOW&>eDdwLGSs$emTsGx{IoXy=JQN1t@Ex@Jmat=@w6mo78+to6)>#m#`VjQa*vQJ#JsYc7qhiW#v;z_VdF>_4`$yl zQ)&7_z9AJUuRuGeT0{h&a<(@6JCT2r(75?-&NZ|(BIw?}82|8>S(4)CoC40hc>#RY z^!3#p{da+90ISgHcgpY^4u3{!@)4$?6Qmn~Mi{TG03%@63>#~@iH75ttK7&j zrbLKO&ewrJ&6EA>87h#@!%{0=u$BJRbxN4pt94{=Nk7=Ec?1MAg-sIh zbDBMUL|zj(c^%gJJ)A6mpEJT+1=GAMH{`h2IWl?+()ILtWR z+iXqzyR4jegJah2S2*b+o4;5Dfvmx~aT_53%;C_W&GFV2HkPekmTAV}2}WkRx^ZNC_^dFoZ}#K<}>X}~dJj4By6?+%LwtnH- zOhtn-Tf~ugYwASnri`DY%L%%f_odAcb2&I4zPitd$R1Lr9J0iCY)AdjqD2O+Lcd%X z*9+GfTpCv-XXp-qFy)<`9NZ14?Z3CIl;iX{hU+U9 z@dMTt-Xu79Qc1@Q;cpAV=$8Ircx>X^GN`N;vgMHS_wuZ^CVnui@dKC@SDYiPepLOX zF;=FMEK+gFiT&CG*EVP<_ew-veZmp^Zn`)Sql;FsfEtZXN zgjF5XAj&?(+_ot!44 zr;r~J!9>8}E<&LNS&1Py3Je1yokGIJ>=ts{Dxp5ju#0UxekU-+jsTob{9=|!;WB>P zUN7(zob8Gl)4Naw;Kny0ui3AK*^fkM>e8swRtJ^WDXIUeT)Iw3nc9eQ>NBP&gNb** zt2Bf~*rzsMMWqF#KW`7>E?Qw~u@hnHIu5yvsqP zIhC+!CE#HkLwOXj&KTD5)c)2a)G{Vo>D#hrVT?nfWVlj4kSQsS0kU6`v+XULb?p6? zyz=ERwkyy(1E8vu6;U^h8|XimxJz|YT)3xI+FWR9iK?X!_w~Uacfc{PZ~mn7iut(| zrc3Kf?O3Y;LFa8*2buV}GYvzs3_f`JJDqtt>ieL++ z#QDb?V=)~zVrRt7goGc{HysAFqTkHy6K)-^_WMfIVC~HLtT~?H z%sJ%p0jjCXI&^1=*d!pt9j?iTeJ&*ku@B2>x{o3?f2}6AuXC6Nv+b{3CVC~$u%hP~ z{Cox$yg^akBb#pOe^u-0uB5VFK<{cxd zj^vnk;((s5ZrE#WswvMS2hukR1B+lwmb{ha^66iDNb>!4vzXKvKgR!Q zlZ|Wq0XnQVnUV|GQ$efzGll`eu-&29bc}G76MGt8LWx?tfI~P(1R10VAztvA-s8tV z_{?jLrh^pg?D418QO$`%u`04@W!2(lxl+{52EVedBArMwTKp?0izZ%8t-@PY*+sM8 zIR@dL&3Aw+?4{k^Xg?$rd7Y7H_OZ>cOuy4nc8*kT<9##V7t9n%FYhifqE{51K!viS zG?whWNJ%0?m!wmx?+LNVz;tI2)NBDq&Ut`${sK5z(k}VI*ZYDa{45Kv=c6YAirhm# zt0bZX*^tVcwT&bU!(@b1%NUO0HW&-+HMEL&(+fqS1cmmBdHIRj6T#@j|Abxw_YKFa z%HIi4FdXy;)}Rv0MGKint6wl;6-$$ebDSI~H;`U`x8QGuX0-Fk@Cz2hKg z5yL^ik*qPNH}R7MTrqb>k*Wv$aPRsSK4>#$pHN+AEz`e}j-k2o*h@5E4&_WvRpm#4 z6LNmVeLCqEms5l2u|(5rnB8x0Sh-jO@ONWV*TNa$pqyHbtJ01#9a76b_)n`yk|Y6_ z6WWeM=R$KZWmM;`D5(k2H>$VX1=stA_6B+*+$&r!`o6RTKTHc!9y0fbEi#!Si_U+mJ}axGSvCLnYJE)asx- z9q~N%E>wgF#`D7R_rh$(^qD=%JM|V-P?7kwt;-^gPrrdXXV3*4qdkC6KfK` zkI`M*yjU|9M_6O7Cuv8o#XYfr)>$rzCFAf@W4b>jVTO-biVBHR2S;{)0Fri1`d~c3M<8F)P+BBBq8GcC6iNW*r1FBpQLaV{Q?|x|Rnl z%j(ezpf*V1e5M@U=5a;c;gwojr+gcTrwBqE^GXyA!aIPNCk4yOZ zzBBUbXLau0AmAQCGX%S5l0(Qq=kyvMbl%G1;?M>S9Wwv`i0_>I&d8nct~jzlrx3`2 zSO-lr3f$;o*$p#y@4_FY`QIR~cfFUTTgB_B`qw)sR8_9o&No#*$!;+kOEUD2c|SBd z`_0@42bfxnZ&2JQw?#*U4>#j#f4_unaMfI(UO;y2$}8r#C=jRmN3b*@Af0fz1{UZz z!YLG>tysQLV&ALvLKOR;tIE@1rk=VQiWzNY)H?#}sql!vTjmQ(q7|zJ)YvB`$Q%iR4?O8E(1IK7(v9OyT&S7O8fm@=$1N?^}er=`vOMO@+ z>ECv>LBA^ZnW6G6xeWS{>VFJC&39`BeT;*5_cLgA2Yv7oW4NB2D=F{)p+H5X){SzW zy*$L3e`1Plx)&}rs*j)AK5|{vltxs{Q?h+R4nUu~Vn7|2&Zcvs>w1cpV_F3=&m&%x zQtIh%I}M=v3IN-EqZ=Q1Z8xf3MdEY|+d?^1cLHytHm4xE{k0F;cvmw|6Vt>~?5E0@ zH6Kq}uW>`38aJwvsai(%5q;261O#KeUjt@arU#yV3w#mibboQFRWGZ z(%{Y3?p?z?y1<82u1-@2L=fZViD{j8@L-DXA{>qNCh%(m$OvK1;v;AeGlE<qy(VwX9Jb(jl>*`Orbg4kpFzI$(a0L zrr)$nT9l(L_BCr&XaUY$8v69gcLvCoWg-Mcg=M?S#3z#@sh!>-124GZph&bSRk*?T z=F+haBb^x3Url9++$B6^@YlC2%JGZt!E6<)M5~L0Qs%b)s<2YXW2Lc2{WLs;3f}V< zT5%I=)}LB$ki?ECe;Tk)QLE`k4K8y5{kBSUm(6a8umxBWwvN5hkfEu31zjZ?5y3B z7o^~w)+`&c0{HKq+4wU%(rj8Na~hr4Ky)~P&6U)aXUV`!vo1z=Xzir!3R>dl)}UzdKESOcVxYH}KfKM7EeUM_val@U1ez zO7@aCpJpAWXj{j5_$S|?*(H!T>IQ^>P=|y{Ya8Ra>2-3_=z?Kh0j-yL$ZH8#xRA>& zV3}U6^krw;(6NE`*&2!aNbrt}EbXg&lGy{;#h3oT@Okc?qGppH$ndfpe)lhFVrr{0b|S;^)YDO-4&F zVYTGNMb#$y5t8f0vvna+OrlGF;4|a%y)T1{Lm0#;x~hs88=f?p$CZKwJY^G!je`5M z`m>zmmX8pAAkOjvkf-dwL1j2%5m?X~sM_d8gwTu$*!wRI=^vq~8cc#C>~W6e-dsHsq(*pW(*V zQFLYqq^D7dMu;&i^cU{4{GGup(qULpeSm>`c$bJZ&iGN1Edm`Obna;CSsiLH6SKw| zJ!&8?s*~qMXi8#M>N7N;^GS@d%{z9oZ~#zMj{l!|eaIkoxwja~02!z{PhM0Tjmpa> zzpY}!j+vA2?z;w4jcdQ98i6-uNOjqAF{PpFcCC^T>FDL0QW4{dP7#?p7L(_ixcp4U zc>}4Ij!>_s9`g$CBX?ScwSJ!Y8S|D%_7wU0NVbU9q3IO)CE7MmH`LHbgqbXZge$VS0=w@Sv@)}HO10)-Pk?TsKyja& z`B3NiXj^h`+#Z_NE(A)iZBK9e2ekAs=Zmg^z5NxvV|GEJ{c4EaXmNQa$uEJy%QXEC z$yzJ^kEi6uFMzI*E~HNs3+wu90xk3;QQuC!x?X0VR%>X%q5HbE*g1OfO%So4$C?gi z$EPAJRYjjShK-`F{U7So4|L`===DhlF2XgaBeX=jx?V&l{=eYHjtN4e$D?T$8eOk2 z>T}i}lI)ab0$3>pr83){!%O0VFPXSzx~D!8q;8|hpm-|7dpG?LFp#E>=#cY}sc+q+ zfRzKs8ZT~h<|d{-&>dctIJ7FD!es6pN_y3$XRc^8VvAeM9ba}D@@yr*Kzl%|vgtkZ z*>#1_EF*_pFF>9#UicSasC8pO+I&|C%fXnaUS5F~`3wcT_Sz|)!5rR zYbhqx5+@&-?j26tobc_9Dkq^#w>KYdDT6Hn`j@y(q5xS}T{xn!Y_SX1aZsK36G~PjWuKWV$tXB%)5)Jq=^gbne*=SczH{7- z#H+W(ct!9olfy%1pRG^TaShX(B}_svVU(J6+?!n3L;235+mzq~9@+(MC9mCo8dz$G z<}dMEXH@c>2Y+`8^1p?L1+4}GYjE_sWsuQ%VoihkY0KUjlhvUd9Y?BFPTawxnEm(^X&*Z+ z-SrarlfIFF7c)DT7|%SwT@3yU+>C~;2f(BtC;tbBf_Uf?_43-HAh&5FYoHOCO=i7_ zGE&nQ=ma2J@PLpg`4is0U(PZ+xuy6Am0eeDu?woYbWabOPx;lB_6QR_w%RbG5b4)F zCaT04W}!)~X@plc@C|aWv^9EclKL7IX$PHBp%)B<(P(ftk4T0}=9{4P@mZ{9^Zj@N zIO_qkRsKuPn;+}ZQ!LK#I3leY+i2XD!ItUCOmVW*q+B32K7&g_#$GIe*owSovBDr= zqPTtOc=-6cJ^Vzp@LZ1DOCVKEF*2xiDXd7_hVUd9_!Qqb0f!B-HLb@>ib2fNo|)h| zmG5JhdSTl5=(S~GdQpmWUn&ec=H%05*NSATR!ZFgmq&Z_P8d-NokVa;r9RaSnVDb# z#e{X=*&hRfljLCg-39@9=PjY0Fjw6Eu_HORvuXUnRk_o@$QHL#={dcPI3PyHE5Yju z4?)E`&P69B-@#?Xg%= z{;Nudjy6*Nrsxp;58^ISXXUm?uEZJKf<~J7#8Q4R7CNiyV8(1h8bKF!aj*BleC(o&H#pHW^=m3LbuXxuP->Tv-o+Su6ASx3W?sMXFR@retkuF_c5VyJ zx<#RIj>&Z?_8=F&0i#{y2>_-ZIzeHv-HmZ7R!m-_=rzyy^U$)USm7XULlw}qhHP#B z+ACQ&7bgQL?uQ+AwWzJ~B9ZcBly}XnH zbA;knqU{0d!<|gc?b%~9;M$PlW}$d77$0#Sxv#83tvtm$m3Q?zf)c9+C>&B-y3EJY zD~!~B7#6-1_-)esTJ;C_+zdMR)?QRZwN+jEb@I&tx5hnQQ0HFQobf{%0{oE%t-&so zuqA_>CFdKA=u0fj(%zoXe3)F+`? zN$Jf|`3f|~5?s-{+uB&9rLJ=xb)xif%#4Fabb6E(XdVHZEIIxX9+I9TF|Qhpr7nBQ z-`MA5PFuCrDUuhqS6l0s2$oY*@Buq22bAE|HO#Hy481Jjd=A!bu5ogT=aAHAiC~6= zyjn8dO1U9x*WZ!K5q<4SL08>31XiZxWlaq|15GZ?HBN&WCDeTlo-v42BSg?l|75w> zJe!4>*Ih{!(xy21Oj&s=9eZv+pA1xC5iD!5Lt3v2N zIEM>pPP0I%k_&Bt&I-+i?#_};_4TTWh;fUtG2q9T@Dt4F{A=`5AYCrmmiKnO1)HJG zfS*f&L#T`On!LwQH)&>C6-UeUVh%>mtP zO=`J_*QI1+O%WxvPM(7yHrU{C48-+3pW+d17BtPd1zEetmXFvSv5vSa{0KVyo^xn@ z0>RGs6rREgNKeS$Pg6cg!vs2HKVLxdjP-vemvvCbplFF+-ME6xEjaW_!DW8|0awr+ z-2lC^2HZUYiL|a5|2<&l5%_q<<+GRLEg8x1iVd9^9`0xgaktavj-_Mv;wUH`PNo~I zQdf{%?o^#e$@J&!p@e7;1=EEh@AfHKUaz++XXu)$c>$kTNnRC{hPU{E$AVYby~2iB ze3No<3~g>HUM5KtX^`WEqP7m- z&~<3~HMqOx1k!ach1b0IMA$+jxJZU!j`BT7%K>WN8}RT|%CsT6$_nIMO1I=;16#y* zdUOsdS}{3G+B+O|I_ALWZ;Gg1DQILBDa?jYcLd$C%2>M2?#;I6&4ysQ5+(pOoH2h` z6(3S2KZoQssmWk6Bz9N)6TM*R#3bMwkJ)yhhT`Dijo@G*j?r~^W+sk5Sl7!C!6eN-K zOnMJOOc%)(J;nYlfnUMAU~MtlK)ba~WE*^)2j9Bmo@L8aI4238IS1_1LSd7ON_q%V4JXiX$)XEiJcpt@je$2pf^@(sKEdR7 z$gXSL^Mv(@u;I(}{stEM3#o`wG0li3CT*(UzXoMamF8FmJxSb9AO1ELNhjK! zvLQvMv78V14W3yh3*+%A4UWJSRqQs&hIroPZ$HAIv)SuyY+eS{(>a8lJ-hrP&EOHq z%^gZ^m**5ev8PB}c!53rE4U-HNa;}sH7P79dsNM>+8k0WD{G0e9$=co-kOo!{1NbF z(9E3l9GYohX4S8yAA`duHzdzdwdMM>Y)}not#yvZQE0K*WUop*q6Kq&68gPWe|pi*>I z+H+#b=1lEliG+u@;EH5KXTU3aGxc`=pgnIKkfv%5jX+po>N?b|W5GGfdc-R<=0Vwm zt_@sZnOS!D1-a2HI3k$pcMq7%d`1Q<>sG?U;L@8bPUWwop4bINjJiZw{QwOOF1yhl za>kh3$sA8>-npO011pL6IZ@BfB9WOFPLoPlk6l4uf!m$fY95NbG+H ztZ($Cq-vXg-rDfsF>H>=AR&uK5eKBOGe8zhQ3srZoeK~T`qCcNzYP6pzZ%vbLnJxi z>dlW{UTB&N!@M7 zU{k}kkW@bba#R{aG3QEYHq{&CeOWEd2gvYsH&&8t8e;BckTyZhUx2X;QCW30+G35K znCu}N+Q!eoF^@s5$?n(yIqX_uBRDA#+-aS=Y!4n|&m8j_sRuX#OC*(UK+UP^=zN8d zGr<>)+VU-R_Gn$Kz0t4*Ba#yLwAlQSyef@QP~4Tqgx!daqCaVSLOkhg{Vs-6yxr@z zl~%E?wb8@!tl%QGLM+h~kbA_qGKul9UoME5Wn?zrB_!%d6^RAYQFeZ0)d{#Fjg~v( zL#!Ch_$ZAwA^u!g#SLgZ<_q9-I9nX*JkTXPjI&0K7;XZC;i7Y3h>Ci^>()r?dn>Hy zo`&>2)SZ!N{SoQfl|!ox2i_?&{5?erbdd{0=*6iLn}RFd7QW>ebSTramO+8i{? z?UI8#q%Ejw3JZC*z(}`gJ4A3KSWJK18wqJ*SaUSFJj(<{SK#$lZFdlK`z0x!$n#J% zLnTiW-6!Ddl@w_k(|ummBYF409>~fW&BvWDDV?9OGZ@B3l8?!c;BTl>A}m9bt7;wf zN?%jPM`vohHn+Ns%NW0gLX&F^;L|UD{a=6n_ou%A|L=eQKm7XQljzp}?iarn98~`< z;Qz<}emTHj4)B)){N(_DIlx~I@RtMp|95~w`Cj`9WFz_I>qUZEg78y;}9hwSUigSXEQC_qAFrr2bfAKmA$mt8aeit8d2s`B&qA z>zlEk{+GGJzpDTKpZqve`#V4VX3FzTF7xG&zy79_(f;D!{IeftzF!s8kU_UH?%g-l zd*3rOL#;J_OkeKbuQk3@J=QSj%C$zQHl=DQ>nF9@G*rXXYHNS+b?tXdL8~?G_MgO5 zO}+0`za0+Mep|h#hQ6ZVs%l8H#-@I0hVH3q7v1THhWhu+Qmt`6R7)@L{@Ca$3Exxi z(RDRLQ|Upv^!I9O)lzlX{*zkkfggrKKm6)T^`3Qe)1=tkQ&UE&)*7OEU#%?}_XLWQ zs{h!ieyM%=C0*fh z!Tv-2FMc(BmiylC;_H9g=6m$5ufO@7-}~m@_8qdrWA7j5zW%p=l=Cb;u`0iL?TKk(nANyPX(?2Zy zcTY3l&i@DB{HSv8o8PJ(qxATC=8t}n-6F36WC93?9OygXjIxv}iGD!(kA5-w2gcti z|C)X0Bcz@{=>LG!Jf{9nNc{urZ7%vg^nXFM5!41&u9|0_srVbP`K8v)(ZFUQ39yh6OUT@n*QAgnW_79 z72{qwYX4qsGp&|ES4l_jQ|eJp*HUWLraW3yt^J;HulC-TLhb$3mxlUR&8ndt*M4bg zw1l-~<344Jb-y2|YAH;Y_`Kw`pDV*i-KRl@YOR%~9|a04-7@^8alfj6`K6i;tM`6V zODRpc`8B2S=j%{z(M_!U38`P+8~PsoU?p4X>$0J;?|+l~={IMgKh6E>8o$@LWU}Xrt<^J#QSEb z*8U}3?7m^$=*xR*gz&)p1C7d+*8INVp^$t#RQnyfX2r>Z3cBNO|LJn5_M5SLzx&gF zJf8c5Ip23We*5Qt^!9$KU_v?5m&GfBo~ZfA!5za^LwzFn*$% z+I?&KUmBXGT7x&fqU#$eD=gnPKSzh+UZR|%f&4^Gl~nV-YOl2$R3-Ne!BE3i=rpzV zyI-n5rti$3Qf*mvo}poVX<^&9HIv5rllw-P0{vBqVr>74zSTUm!eN18SNmJ470-KV z!;1Y^rgb$+5^Htn=h`kxl;5ho|3fP3p|9w1wYFM&pYD1;WT-y~vEQl9{q~>czWztK z(6|5SH@-Rhji3E<0!7a^QyE9@xBnzp`m*Z#vizS=!Tr4Kx%Wqu@|4nyF%W?LBJn>b zxb=n6f7Ukuf*>+(JqYZJ&riZX{{sxIznS%P>t_Ms7cXJfqujcL^$(x_3HU$wH7b=) zkXpmkzk$>!rd|V8NNNm2uORhzKR@^{A+#twgZH@1KfyR;QlGvUVtf?ETU>v(m+!0F*pNWTR!{V>QXb{+hpnR%D2XF3-@ppGx=+H4%~de ztWGlJVXVFu(ky#`%%z9g9Pu%@#7g7m<}X8V0sZ~^(D=FWOK`_(`!ZEC30#{8X&-t& z028}T;3?=6xD74A$`&Z^;S)LWx=+ma095^w5AL(O&)pqL(LG@sAA-HveMS!zi~15w z;y!G#_HEX=K_Sr2#S=aY2Y3gG4K~9{+aUUO#Pl&nE}+-_7X2;j>=nO&9ij3B8Uo=N z=p2Hk+$a|t5N6)|^BB8XpP!@QF3wpnNI>+jpVQISGk#HUWLfNcyqR z_nK|sE;!8Q-lRSb8#UK#4tm9R@CuHZZ!M5)UVLICMhz`Imn!`o*~E| zLT~wvFpEm8#cEqowXcJ@jaIvgwjtaRq=)Ph_5_o4O=5=>8y6;E9lbBWImVi|kyd1g zZPJaB>5puWzW)TjXeFV_E3gDFVFhQcEmq6IBB=Xpk=ne6Z1Z;OWL>Z;A5T4nk8n`qK7mn?mzl&jz4pD0W%Bh?e*&Jfe9#hjA=D6?fnSUSqZW9s0*;ItRibWy>kNW05%+ zWpww-7+b+B@Et(rxII#K!_lC5Xf4yR4lbw>IB5klki<2Jqr&j`0pfkv$}pp$HpF41@dnPe$2 zjVlm!O<{hWO%lNJYj{DWGK<={a0W3d-vz6?#Wq`khxi!EL&7-?Z(0};f`=H5=bqvx zs4Gu^bi=Ikly=7oYhHvK))#z+QgM$b&)5u3VQvlrZRsm<8OxdEwvc=Y_6h5#qxAvx z8nHr~E4xjJF$MAH5uW2mN{!@Yc_>?- z*mrG!W}O>chPKdK`gs})kvBEdLSK1Zu#JE|PGw^Oo6HpEVdbRWYruDztSPwj!$I}5FMv@E)r$eu#n_kpflw*ZguDJ4!Q($%|2H`t;o zykV*4Si_BlK|6X{3;u&FhxFI3Q=jhSj!YkAt2|c?s>1*tG1Gy1+2U+m}6ra{iP~dlQ z9b@Hi+Y!9PW#+R#mE2E-9>+0NRP+Ix6y8!v9YN1FS{>jfiPg@)xL{f~vI4u-!J=c# zHKpntPoc{?&D)e-gZfnJlogZ4EV1GP7Ok6Rw>TquW2V^qp2flqqTRe9RJA_;nUT&Y zljoqQtkMdP^9S@bJIC*^653@2DV_HQ90x+%hoZ59ZE{; zJY@3(65l+hz`mj1TWrfZ<6}ER>yW@OI0_9`XSqqLq8l-D&Bj1dCm4H%Q>-$HrM!Ed zuyUHUToOD)9FvTn(t$M1jaOWB#+VNv-aZj976mZ+a(@-NNQ!1uwxYJ4zYg$Ue_#P+FlsL@EM%r2y2L* zi`IVeap;~$t4r8BwhjsL1=Veygv${a{j~PhhHf*}e-8e)U~S{JIxn#PPe?^1?$UZy z^mLPL0w=M~8T4f*;2NYmP)6aHQsg0a^*iR^F^SVt+yXWVmrQCapA+G%)l|!MuD)0N z3N}d{eyBNPu0n(3DaNc#&hNn5;yli#v8|u)2=Ae31{oG$6Dls=tossEUs#|2PN6%!$8YHs#1hjGTq42LHUT-!YW%SV z4qY3?nrAHTxMd&t4EDyJlJ?pM)4v3*#S>=RVfAx-nZ;<5 z1ilCKdg&SCU9Z4i8HeL99zncpJg|~t@Mvw9M^FF)6A>{cG<#ok>39b=B(1y>Or8Mju*eLwY+|MPIi@b*1=SkWYhehkpz9^2?g;C&c9*i)7_G5?5A-#1uCEw>19h9= z-NN_;*0qeWm?JWYTVGs2{WT@=7@?KFv;1m+~q}>nM5NLE5&r>M@4FmuZesLWtBTyw{>`E^p&`L=wzZXBF)cFdyBwzOGoZ=N80&GZHe zr$7?LoT2sa{_}l06z8|3nF4#PVSz|&8nf0;oE1WfNXrGum`L*u0jPVDnWZyyoTK)U zJ;57FkJcS#)4jvAxI^eC2gHcWSa&US&wvr| zPT-(D|B8i1Anh5&>>AF%KuJ$*(sGYOejBb)?iqt_qc}~05LaN4kFpWy>`nQs?IbTy z=@^2Z=VU^6plgCGM`Z9Jgoj~?RLUrBU~BvdUg0>7v299_XDqyli$Zu6)3ew@a5Ib6 zkBAju{>e#g=4gip`Bwk(dzZX&ilH`MO2eD{s z(0wiLS+a7MaI9m6Pr*9(-C*k&_{DdW9B(N-MxjiOZ-zB~Zg%{hk{pe7aGsEE`P{X5 znBqA@I5`Ayzx0vM~RQ>oZFG+%qf_tgI1b^?CZw$X$-H)Fi$jJX|J6@&qccAp2fe6P%kg zKeO5D4SR`Z;B#Bp+i&&nQo`6p6?%p!S^*oG;PCd6Dh~37V0L+v|Lld_6+Po=pif%wqvwfkG|km%h4NLA+dJA zqMn{9RGz_{FoGYb)}%G&-xB6(yqDxBUqIHi!=GX0HTDfOk+~sXsC^(?rR<4z`vQ3M zF$_`mJ4U+Y)l2%CGN1k?HYL-g29M$ z1d=V|!c&NTBp?}346$npCn#e=PBAEs^AF&DfRPztkBu;OiIQf9XIA;buk;@iujv!Q zAyGd$F1LS3unzg1LTl-WEOWyJe&Qpnb(Rumf`=wq;D!K>_xD&`4a3DTLFshFHZiF$ zP%v5^V^iemzw1l54++;KP6}^OT_H%EusV_wzOzlH_Qd!nc!z`6u*ZjK1-BqUw7yFa z{JB?PXTQ}aa2X##_8j$C!%IjN(&=<-l)q-VS$Ij8SbK|Y=iU6uZgHJ*XAUa` ztA!syzOCH!0ULH9b3moMj`~H-IH^xQ!#02&@;PjNgA>esz+7#lJ$JEX1wWEDHO@(U z=i}=Tn_?rlNBg3QLIBO&#g`!8p?eP56zSc{43yW1zvm(9 z8`KN$ScqkJ;GDfgQ(FR}!ZQe^tAyyJxr^@i(D{sAp?jPqNqvl9@)3m3u`M*p$Wjw@ zis?mc5hX|F8mz`|srUR4ny#QTe@(97hJb4kvLyIjgNeQ+P#4j2O4>VjM1CihU$x$7 zyP+X{5FByqt??~55k%VxndDOYD&0)=KViPW7D+VkVDYiI57H!&%mKI03+4*!5u~la zfv^VYEua)^j-(5@)FVtkp%7?qNG?ajVJ2QvykClOaTXg~jgM$eP9YJBcXmce|2~0W zb)9Vr-dv_}0`!2~qP{|pB0WU)EsMPrr-?hqAwI{NFRf-^lXOr2Dp*axK7#t5Uf!2n z5sxVg-;*dlWD7N|E3Cd+^%bYdsx5-6Zl46wTYSML1U{?7{TgF?Sf7soc$n28aj9U$wE`?LN=~J9#XK?yOBH*^ahnD^) zn24;_3-W9foUhfFBnEEzLb!`z+x#odw;fkO%9#OTr522%5 ze1}$VpanA8dpIh#?=kW|<_nfx=bMn+q8P`B!3EO^yg&XP9E^fOGlHpzOQ z;3SvV;g|~H2^5|&+b*OnD7q2aJ?()FA@z|obEmDdv)dn;B~S1PQ%_)?TateP?GHY8 zZ+U^PMC(W9mG}mq!VdQZTd66vA42#9*P&yXGVlXf+w3PvFIinxj%fWp!VbEuGp=0+ zWJ=O`A~CCfiuIQ09M}EhAQoGD8l*jv&e9Q~`vTkvL&BBtoY{u>x#ee}(y#9c0rK9S zZK&`EXjn&O(N7w1o-+3djI+ulsjy3|>-dPyNyY>HosmL*gGgwDo;c2TY3)RhdcW+l9q@IBM0ZMnQ@&OD|$6KuV4hD~5MtB1+Yiol@bDEMw9n!5e#v?de6JtRWtnr`r}D(w8xOExtjb`w8GySzHsnWQM<3V~bSr z9rKhV?@6{Q6V|@xa;mX13=-kCxrPsMnd#@yrVpck22%^ndCi<@V+k6}Wx8XxI8R`* z0-HDmtsUd+8lR%|c1ED@0{w-LtZ;&X2Rz)hM_@9>q(xzyJ%vEgUAIE+H+n*=Zj7`4)rQYylE$z#VmGWI*l+vKuzp66N(6W)p81Rl>k0azwYx zv5r>vfJSTA{We9|JcX`R;2Y4#I<2$+BYX|bN1!`as3OJQ$ME^MfAulle-!M&V=B>j za+s`iMJd#svfkVoc0b29eF!$0t$r4~516hJrDo5BVQBT+Z>%QI)@k`=UO$5$`7Ca+ zZfDF(JaquBW4?%F*;v#%`>sA?1>Z|}EVMLgM4=0 z4fMPMXR$BiFjpbb{t#OJ{5<(4snlg!y(KD{5n+rjpnBC0DyZggy!%(E9A*Mi{5sOlwK&sCB=#&v1whD zA}@v37ch8;%Ulz0&~3lPq|ss~9iAIpta-%D4OqbN6MRNsOnCkI^zbAW&-pcp#>x`W z*eI1!>4v=qWr0X2Ar0go;64VVIm@~9xf)y#A=EO06*2V?HRU5rv8U*YZBZ<@F*rt$ zHVS=9$gGAw{23WSG;9Qot_9%{!R|-YQZqHjU>&r5McIDuR}KwhW119pVX>z2nqAU8-D}V?FhP=H9j%7eD-?zk=q<~p>5)<`-NLQB2k;65blM{Anqh@tA2*gTJr686e# zR4q@r{s~h1*8f`I8rq^Qb!XxmqH%}S=X~UI*IL`D)sdc^2@U!Y^RClmPDy2ZrdWcV z5&C-F3uKQ16)LVvs+6Yo%o!=2VsqhzU<^ym-lq_&dY@t99l7d;A@-KS`U)?wX_jcK z;GZJGh%d1_spz6%aDUw}0h4}!dz`zx+g62#&0xn9(BjL4 z?_0zWPg(srmB|gBVc?QRC08gD`^!`aX=m~lw)h(s2)(9?cR$0xM}lKzo=LOBF=teo z_N3!6>v#yR?$R2?dO7rYnv> z!TOYvx4&b#$HKWV$+m_1C#=|~*7s$#MRa!$v-cp4)wIsQh?r^$+>!N3ji7y;`Gfhm zr_(=K^9s+1dwF~2F3{tf{`}XU+_e4Z{{~P?0|XQR000O8&VpJ=c}#Wn1K0on zi#h=SI{+L2WMOn+FK%ySWpghuFKKRMX=7z`FJx|PF=IJXVo`ESPHJ{%HFPaYFlJFN zFfVCtWMz0SUuR`5W@L0#R0#kBgAfr|gAfr|b$AN^0R-p+000E&0{{Soy?4AFRns>B!SRtK&9Em1|bO@ z0-;E6Dkx&%_pCiP%KQ1f@7F&d$;_Fy_u6Yc&uV+*$s-zLc3E=pp+_B>bU(lN=;IDO zWV;229&yN`?G9YDc$)(kAE+n_``&Bg!h3c&dDS)Ne&Yw%Z+-gt%9&>}uAIpgZm#+M zb{DliIBeZY`eWXjrE^Ywdeywu%eOoAZg)`||H1`y`SR^>ZvOi* z+y9gahtVs}IAOu!`7;hVdhwzOGlwRG(S%KhCLDD1(MRB4haGstqC@e|BaS|5-T{k_ zK5j9Ng&7VkT72AsqvmxLFY79Y9CGxMgN`_KqVkCnCo{UI)r~`rK6ql5v3=(&+)w5y zRr@eyD2zLbbC}Xd4^#a75k-?HhNFDGG9yjrD^ud=5yda&E80*P=88y}-Ad*wUU<3U zJgQ92@w-le16n8P&Q~hKG5+pHIa3CboXss#w%=8Z^Q#s0QDyexZQ{ceWl9hp zrU;!I;!0K>F7SeWFkfLz=XGb#)gA*f#OA1ac3#>k|(&j02i2C zvybKiZ#ztH?**%wL*G`=KaAh-Yq=9Pp(syW<=kJvtE1+oiT@l3`gjl zn?cU}_!zG7+cY}kJi)yn=T64#)wr0xG%OS`PchrYQ(Uaqn2issX}XubqSH4X!&xiL z7=MJx;mx*CJ9+XX#vrY$xXfy2ro3!Q9LN#TJ4JC6ciU0MydvQk(}7}%*2%N0n8tI3 zzW#BpwDVI8euV1GQwH*F^iH17Q*`rTMO?vHJ6fUoc|}t^T*q!VJ9)7Vum7?x9#Jd@ zKUk(Y4=V#<%sNH)Fvv${Yn9gn0|Z%Yv)gD>=5QWqD)STOzjjg?kD=JYCC;G6AL)w zhQ_OgA1pESW`2uF*I=T=dV>u`>1svJ))6EqE4a9dne8}b>`>;U%zDn~1bB+ZAiIYt zrn^-2!Yeeqo)$|}QQuuK(?Dr`y;&sXL` zs`feC;mu;PA!e(J`-sB+rC@fMt?`s~y7^%k*v)lusG>L+p@6%GD&|7o=@v6pP5fS| zwjm|_V0wG|aPD`X(ljsFoU2E0p>fuZ7a4y3jG*9@z4%GFFxDunfW$+T7b+Osw$lmnSxjqYD^)tu&igGdF;!;;^U~uu-`EUpp>xA8Bx#kzo z@~Ug|^32qrJB#_sZaS1eG?RJJLYkdM z{wnU5ufbIq`RepMSfH@a8hB7Mt#Glus{6(EYONh&dcJBfb0+IV-JhD8h-%g8A(V;o zzKIoV<~Sbw!q)8LxVV}rt^82Mi+3@gajo+VKk8O=`;e;GRc8)n@A@8?9l@XCeiY!u zJvynpc5x*WV-03~+wr3wP3PBp;s(`nuc>wI28P3%j9%sF4_{lR|Z)PuIZ7OwPbH#dw%VXk={_1cX*xb)IXpH^JR`n(3W7t`%&Cg5_vcL?@1HsF7c_aMW($*W%57#{bgd1yt(g zj&9F)@g~Y3RuHpPXN_X*to!jKOXn}s>s!N70!a>MLXyMNdr4`#Rh^$v-#8>GscpzV z=JsXHk0)BX7Z*~0yN@bdkwY2XYj&an4`Z9zw6bngxwDJj3A(3gN~74r?8M#0c*j0k zSJq(u8C_KH^$*#8{MjCzcWf!;!8(Nx=+10rjx#!O2pO{X>Ju*XA8f0i;v3YpE?>-j zoYMWI!Yl4QI$y6sUaYV7%A;adML4f&{B)fb%f18jH9ydG{%TEK#>7mez|l3wD-Bs^K%TG!z;rk^Iti% zmSYHHQ6H!CulL}7rh9CazipC&vbZoIBcC$}dsrB^9qQs57JmK`?y?McD2z|m_(?h`x$q?%KYffY zerQ(R3oKg8=E<6$Uc+?zMS4@_{-d9t-9y6nvpzrlYOlez=ISY$=+zYGiK_U9!I^SnVVxaF;cwNUDZ-lQK zetK(N=i^NpaFejk6I`6(G4scECr$f3-FnCcs#Dx*n_oPcEYCQ|(9Lz-0@{060gUUy zW?#vm^iqJsAHYQZx*xXNaw-~MC*SG0wx7R-clx100+B&{zV3+cRPc05h`&uT+0v?% zT&Tn03C!Bl79USC?2G%QZb>&+*`v(KSu;BtGShEp;t5`xo@O>@m=>(=Ygi}i(&)d_ zD~^N2>}K+YNsIJ*Q}}MCpS=c%l5M9$5is}n6UBEM%-R68ddNhmsp(JxX642~Hj_hr znOgzr?$uemjF_OQAoVi@L8YIfheYaQah_!ct1%rvhHm{OdAC8K}oWCe% zIeydVWD#sWAMPU!K~LY;7FSd-P^WsGj4Y-&bxf^YtYCccATF`qHvDWZWN|&Owep#a zUn}Uz#|u9>)CI)YM1`F=LsQxr-b$Z@b9}=vzeSa&?R~p)jpbd;^m*f$bvh(!RS&-W z<9<5fzN!g6ky(FtI%y82E}rGJ3Px*o8|q~sK8|<7@F$qNf7vieru+IPe)Iz!SmJ#H zCywR17u_Q<t12c;@vM-`?_zBqFmF-1u;$L*Z zx}~6UF&#Xu{DWBAoHk8Gm=4-5oh_9K%*TA~ieh@D& zg{^}85eu0SL$^&$iH}HT7LY2PFA3J8KrujLLs8UGVAcje;Q&k|SCPLVozueAGl*hM z4Muqr=!MN_2@ggKlyN{&jd+v*HDg$CB1l;QEt}=s%~>NP{&OslTlT0@b+2K5P%vd` zh-EPnvwe=`9J;hJ97T^RGlC)mA_cchz{iHNZcd8C8qPrz*`E64t561h z{1nXdi5daseUIkcmS6^U_-e)uV=mOo3%K|8y1~Fuk0IrzwBs6h#J?0mPRDRMse9QL ze3Y9F7b(W$DZ&tf;A_ToWrVAe#^AQ#x|?gnYk`$JVK9$|eLyF5(*~)dwQT*t5ZmIG z;dN4#f@mjBb|f*37SB zFfPpfp^1y|7LcUw`@oNNVtD3o9GVi7VfqFQzxxvy^yDO1>Tft+ahKuDWEQmDIfels zUv=iFUVJ1m>LykLjzUUKP$jzf3dWwac%!&Z^}7=QGxOu6Lh#EJtuRGS!-ARfKm|v-Dknb!^f$d-2T~ZaU~sksM@@Is#qFRh#b5ANb{Aj8+IUL^HPlh!hr5$~xJPa&Jxt%&bd81S8+jqlf+1&HDGs?r7*0%L*^Y82xE z^?qjSVs@?OezuPvL#xJORX+pGA(976YJ)0WE+rrU(#qb#XZ+nJs|+nUTtM%NzdNjf z)dYaJ2tYW$S*jNruk?P?3r^Bqpo5!$3g5=1Tl0+|JOC5*lFLaoSq>MQfTi@RqNk>X1cYHT~fNORDKZ2RwI4|(ziAlC%k`25)_F$6z7ahQS$bONUa_ToyK4Z3@r?6(nGGq)na*#J>vOmW zp5lCMjpVn->m)?*#EQY1yQ0Q#&`95a<-P3b;#pB`r)w0ROTWSWrl0HtC7Gly#tf_D zCA`N;8Z&|8;|G8n9(L%{04SW3G`^~rKB2%X#X!;gqdq!!ELSRT_lgg|II=_BPF� z&7$P808W^wTgI|RdJeek!+<*&$2DYL86eF9m<_11Mg&fuC*+8Du$L^a`-1S(U)zfH zy65M60EavaRhVa`bI{&J7iL@MCk{&D32h7ur}XO1aoiff?~elsK1HhzLgZK>#}I$> zvz>Ll8E#J;nzf?U3g6EmyK@e}Txemap5X#Q2Ythu;e}SInSatt!n%fpWouRJZ!-%j z7Za#A)ANk-695FoD;Tb2tP^8Yp)OC64B36Q?H60Aiq&!ad@N+^XXS@M$A8_DKhGgz z8lNiRyjy8|@l+dU!DId6A?WhNVKm)0sfzmwOv|GcysdH2z}zBW2eig%!r@i-)VM(0NjB4Lc2~H@{4Dyx;WRXxgQRR zTU2I!5qjd-KW9*9D&v@>{5EHIW7LLkky?;^Q|~X+idA5HeUb(RG-4>KGufXYib( znpGD{`&@&Wj|}+HD`O1)qLe$bav9V!ScYOAGSG&9*<=WU1_Xw{Yal| z5N88|apMNhtN04@>8d<#tu6`d#9paY!>3?JGazZ}^Crne8-Mo;jEi~HRX~rr8!|Zy(Kh~D2AKt0!{21wG7ub!&!H3In8`JF zcT*hQWaiGg`1N>0Oth-*FKqtfA;;ze!C`pGE}m)sa4b3X+R=*|xi9eg%0!%bva0hB zOcI{qBEnGJ1VLV-bb{_N)hX_;Y(LtvhG`y;Y3|nycABcLQ^YLr)ZKf?|1wL41o|l? zP$Kr5RW=x%j=`%CbjxnW(B3#j=Y3N~?Ls@+RcFVkE+$|RFv7Ewb^a`qJUy%Mr>D?s z-H2;W)^z)U>Eyh1F0{mE4jTxrs*FTo^3`$LP?G(!7pB5;wo<{a6?;pSVhj8wIj!zd zod3S!q^vLBQ3u_My2+WmjE*he^-AJIAW<_HzrN-s{A=-RGFJBI*Cp$UiC}?ly|#vU z)?$#aJwdSESiO`YV6U$(so3h}l3?K{SA0xA!CjWjTeoyOx_k}A%80@d>sko9wcr0r zx03%B9ZS2Pj*bOg<n3mJ5>Tfp!2q(fo2sRX*n>~6Gh zZQ$q`5i~0-CP0QgO4ZunaK!3>d_daB_2A_NQ9`RtFpONM7EXW^>g;8e9s*cV%h{AT z1|XEKC?C+U+Kz5gz2Z0(kgpLtymUuNKRGbUahxw84g!9<7EhBL7$Qq9DFE8mQ zpQg2q+Q}n%+0ANoxX562BVfg;QMOh}KO`I0auH|iB<@NVb3gAVBljE^f6x@kE`59t zvljD_lpWw3(0JXo6k(}wFPhDrwaU~yhZV|}YCC+Mdx4yiW`s5OCs@E`ide#} z;WVF3x`Uf<;0v2T5xbFZ+jgqlOIqY%F}G9w0w+8Qf?{CWu=}ix6cn3U<*E3&GRhH? z>08e5(%Y$Xi;(SmJO?u0REBb%hx*K(=e#PF*X!hhl@QQoWHEdxps1xE5xxAa%@1;5Car0fAAjGGd>xWgEiIa z#KiJ>r9x7Fy~_Lq?4SuQDe!aq9fKHmJ8T=|br|)-7stXx|JoY;2zQ+;HsA{qAooN- z)8(WA$-m$itBKIP37~ftNLu)`j0nRbwLxCJY(#U8<;{Qs2Pc9Akz@8W;U_@qjbM2N zh!u|!mpiozm@bw_vzW=ib9La$YGtj<0AdS}s21*iIX-NG14Dzwbhbf)uUIfhJ$RWf zdt_jS)2Icbr;(##zUwCx%Q>Gsf0lcS#%EOt4nGSicOVauU5ZlhdD9POR`4=+0}#-I z_qA)SWF}*m(ZX-wMM2a~)--;Kgjs}2CRqF|>D`7v?FbuuK<_9dX@fe02QV{be)>fi zz<4KZlp?rq8sy$+#0R^6M$kKcQI!&q8<1U{eB58O>5Redl-H2oSg(5V-i~xPV0h`W zW-`elmgD}V*NaLH5s?Oh7-(KrFT*rN{uGdzWMGZ-TMnS$AY?WAcccM#@AiWj7MMH2 z5Jy%vN-F7LFT0{jUmgIg&}o|uNCD{dmqEuy_BE@#?x9}EGyma;{ri~pP+upF?*xzA zPSE=U?@SN6Wh}*7t8~%?%dnX`GKAVr20ihK!>kj5_-8S4;8fj5U2z8y=H(diY?ZEw zL8IybYJ3QQa(|DXo(c_+Jz0?g5dT5rYnii%TTfKMh_7K9e}`PcRs|CA4wmS}mY+UV z1)%oE|nqpp8!?Lb|bcH+1+yZcqlQl2> zj*InaoTTB(=qhOR;w_!#JAs5JSlBRT%(zp>**F8@e#?OPz>a-F;$uv-a2(RBVX;&z zr+VZ|83-WF@s5KAAE;xREmE>|;|Y$3pEE`mo7ZZN#l28^$<0k1+#tUik4>JIc@kKt z?&d1HirgILb~-L1K=oC?r+sYx!XSYWIvU`poslPv4X42|obTBWaWJ<~{a{S5E*4`M zs#QLxO?SDdAHUxY6S{p{U0RyA05JM5kdY8LGw?J+yl%({vQ{H!c;;zzRJYEQLhVd< z_)>bGCYitFMJ#ih*W4>A;zf-OL`h{N4$>`E%{rN&wea&-4QAop>R6vYu?QoRd`<4B zm$hEhI)pwoey~&0OpNk2bunad3($#{^xyh6;#^Jdz>Oyuc2pt1EXb@lqRNTohD(HQ|IC+A|DLhpy4di z;UOHx+T%-`U;t)i+EBgticRbPMa>T@xX-)2(u12OG;ej{KIVgcxM+Fp-`*s5r?C`C zz?oa>BXJntplW=}jZV*Y5?1(oK26Y`KF-n*^0xHq(OZ|pU_8Y~q9P*>)K@Q^63jiz z@Y9`i#hTyri!D{1N3O(1LFmXq9SDDBcOp6?+{sncr7&o^CIalxZ+{ z&7%NBBR?PaI!)6F#xE};vuhOY_kqt2#5b!Wah$s_RGB-Pr{T_U#B#>X?dqa$449T* zb=@}G#h_e)g9iMV4t|K$@_ujWM6~v^{e)Z|Q%p>v-2$-VmrkUq0HI3;Ji|&W(wb!; z|0}^Ol8PVQ1+_S~R~(JCcopMsna(4MMJBm>G)zlymquwC?sYZ(Xp4~8kyY{Y@rM1~ z1_`Zu*I0gvq{l_W^W&GWidL%REp?NMj2#}ZX(x{N1K-+$6W@B2RCaJQ=dboiN74zJ zpX}bJ+bc=e%SZ$JM8@|s)=6Rull0~|C>!SPG2Kf|a-PZHBDvu2Gsu0~>O${*riYo| zo-h(8>7$BneWs_xv^nFKHKlv=bEasIGx(AzWb**q9I#wv25ST~S}g#Q8D%6s#Ay9x zfTCR)hJ`fw0;SfDF0yD+Eg0Kq$GXn1X_BwI6<0XVsszin9>iqNb9KH=o2Kn16(A|4 zXz~Qe%K?fpJLc&dSp(aRRwgU`6tWIHwEhXQICxMog~x`H^mp|k4^;jiu2@5b(DHh z#Oe)-O3eg6TA9lD)lxe6`nsDaBT>?__To1v3fB7Iwbdn8lSE8L$Rt;jp|OqOuQC;} zjEOCt4|aBC@d1%E`q%r>|3STi;=fR@V_zEa?Y8I7gcUlv&L6j5hfHj0ck7P{#P@# z9+oTzvk29YLd>V+IT(4-1?rrdk3x?QrR~(FhJ>R7nfL>$Ut zXhpRR#Na{yxmb(91O<^9FX>aUyutp$S zO1PFHBrqBM8m*hD8ORh^&A#h_zLJ3wiV^e=FQWtv3uKK_3n)keP5X+*hT|y50t4Vz z2D)GkseGGYYhLQxjJW6;Nu?#n-dt>}R$Bpv=u*W! zOfgTx?Z7Z}F;}lb1Bpi98=Xw%X(nIRrMn~~U9gSH2SRX|j3&Vg%OsRjHHtP-O$wu^3=Tzts}bY|i(g5@JBKa4r6m1d3)*+Rv` z?$XJVH5h;1WTo#Pp90Ffg*v5`QEsNxbr#ek$dN8i(5i!YGJ8susur@CMf9~1%rO9@ zOO{%)XzOi^5t(DL(5*SQW9}%H6TPHS%8%|xH>eQAopt_Knyfyi#0%WPm!eC z{kkPy*4WgbyBma^BP=C-m3FYYV$);GRSkt58c}j0ntm`Tyu(}uB z#!D)4x+YW?SG4Y}WR`_Iy`-p;Y&BfQ31#hhIkNuqc;)0xX;mSXXm z#v3{*gXefBN)EL(3se#CqEe}{uTiF~{F*a`E+VHgoW##?FP82l8wm5;y*j_rRv}Cv zW0ea$Hbe|=AUhvlDwF<(%~3g*;tXA=n7_A2&y`7_WQhX`lO+>#$cN zz{1Wn$-M&UIUJla1UbTA#Y&Z%?_1(`1BRCXtl=9o@_JlgdEsh`5)WimjAcpz&=*_( z<}7&cwg(meM+@2-c#S>DM$!CWuvqY`DRZe2;7u}!p4h#o>b^5Z{0SWR7%mUldyK_; zV8cbQmCC;}Aboc;8PxSmuurvGYll;M`3E+swy5G~K$7;kV<SQ$vC|-HMCTfQ_gSuc>1DRMn8M4-^pIZnWpLxS#*8Fg8M}JGC_SezN`TUr zK`*3_RsG;uZdlt&caytjfYL=A5Q2*61i@jh#&6>Pb}?YQy|hnKAxvHHmSM1Sr8KwG zJ9U0~4`EqA@8Bt}vwbZZ8b~w%Hvf}Nk^SHdUI#G+vs8SL2AtXZs7eTsKtLDVQ}(Jga6cGCS2*VOZK^iffBDNTT4`pGk(oO|{` z2aPWUmpf-0Byy6ALPGrfX z8jY+Mdg4w>n>O2Hu%hPn=oToI&6UbWL-i~#vmwCKl7RHhQN%H1a`@x)+QSZ+qN$ zR2hkje7>vMp(8UeawdLkDyBk_?hAVfOx3821g{BUnQHk(#td<{F8v1L)iKPxLG2VF zRF&9mTCJU|(#6x5%;Y_z#IgLs9(7Aq+^B-wyo|}Lldobew-NvLoO)1*sU|&n%gtB_` zH?b;js#5X>X1<+)Po6#&x>_Y1nVOj3+#LLC9$hdqdSl)4^vb$$YRQ>hSU zBx3tgG%QZaAGGp$;C9^^ii`D(zmLRhe;8Al<2_IL4AH`zQN(Hn#fqk;#H3(ewV|LW z(};jgh}U`HXo#OItOUeQk5;%)jfaoRl*zf|Mgeg*N@K22Y}?BgFn3LvLl0CoqUdK|Z%={7$ zWE2mi=y(?zqZ?mM5kQO56Jfi;lQmfjH; zBe>qkx6x%CReCz`iGl0V$GDF(8pT4WFrnA*!b6yIB{LsU zI^9C&`sf+=CRoQu6x>axNhBL)f8@w?f)%RhGuS}!Bp6Kg4H)CSEY*}=Kp5S!a;A}0 zv`$0j(=UaYs8nnob1U_wWlQLJX5tE!f4fzUB(?_$JTI6~tPBkEV zF!ZMyy7j!w(5HM={@pU{0h1m7#?`&zSW>E%vx6=xN+?boc$&u7RYpk+5TO?zr&6@Z z!!^Hlbvvj?Z)%V;-dFOAEma`QWi{O%V@U-UJq6;pJ>{1g#|oTUW$L!4$xzye>c^)W zfLhi_8YjD&lo^k;_ceznoE(f#04WHHg6>A?s+PM!qs&BQAimG;q~Q->FZXk6?eq=9 zD~?fR`PCLG0TM`}M(x%!Z5bNBLn|>!cSl`psbUSCZFE8kP&cDziTknXt6mx&S5DGFHO zeo_%{jaG9-5>1lDr!-i+)iPRi1KIrQNE*p}`dieR`*&v)UIn{=jbCKxd`*QuW*ds! zoNG`vA=&y4-9Z!MdTZ`_&bRiYj$2GdU+1q%8w7SfyGJ~Y6u;xMw) zCf(LG_E6(jdhXUSi|^0Q(l7S99y9TndKz7dzs>7jaeA#1ZV7%HHnp>dlt&s zq*U@TfNaSirI=dw)0|&LsI`HY^3$c2DZ>c`%>i1>4k#3**!cx>tFFbl%qD6?Th#m% zNL);@YHr)`vKJ{?!MwE5Nz*tirj>2{vA2_TVIS<#fD1qy&`l%k(Igq;p3H1^u@pu?p~7=1u@J15 zmEdbQjqtjICgsUy^JLw&#t-bLPoXp`DkR`f)#$ybs#$shPcCtNfT4Y2P|}KTIpXJi z27g<|gKkt6_h)_LB#oQ*nVl>>ctGPjjFAX|(&5hbnDqz@VTR{i&(*`q)G%`6deLVs zyoNb(Xe7<@53EtlZ&j{l^40UV4RPWalZ1rygkI$**NF(|RX@M0E+wITs9RPOa81&NV1TLb+9M1OEmTujw6KlN`iNOtKXF z69;&C9RZRGquRgWI9S2N=@7MMAjpve2~ol@u>OOlU;Ne5o9TciBk_?}S#m>ws`(rU$;r;5&Ffwg7Yz10T0r1V3Of~yY;u4ul+74dQ_y>KoWVW2rDF8wsXeARXa)4_95OdgqEPolg>gDoYRFuS}v+bSX zk*pg)9ooOKrJ~&5_ww1K5zVpWLOvfC9oLWwe5)z^nJIMQV++?rYr(L7pjX|^D?)Rb z*@bk)xpa;Hwzu?kS2_QvPZr9Z0V{bt*F|@z=DtwnCrF3w>uo=I*w*b84e`1Lh>Ah` z2Xtie2CUmZpn?`pCO0a*tDh|4iv16ZoTG;L8h}muHN#KaLa{(KuiVVAH?t_sZ#c;C z@7WWWx!W{Y*!y&yH&k^=8D;)l9rnv{fetE7pU#Pss=%>VQC*~y|S9Gz`qQee+nt?=JaRib&YU7oNf$iRNP*h=Ncs+57bvVt?OP1TvF zj`l}zud6VFatQVJy^${e&6`(p7#Lv%O81*%IB{!wKhQr(d(P8=-ekF45o)&;%% zzQgp-)%PC;l{4elR)dn!!SBzJ(KdOP1?z~)Et27~c1+yw%GRpa|M-6q zv?BO_6SN}#6hW(hnxB2b!sOosEsHa@Kh!f}{auJ&Eue~5@GHu(o0|;ZPs^;` z1=R6lfl_P6U@pZ>5DH1pnKC`jiP=H>%w!aYn45IjzF2reMOI!(5fs4EdLhfSr=$hN znSdZQFS%M7RTQo6JUkd$JPW!+lE`;F6eKexvy%~Gzy~jv)XsdAfH&1>0k&3$iD$~h z0)CS#)RiKGVkU+KMlvN!pXQw~zd_Z!{A87MJP1}R0cn{*1x}5G zDuXDuEqH=$tVD~iU|#w!8HWmA1U2dEFlt{j%WN;!4xZ%vWQlB^;GKvP0`0R6s_!62 zcO#>w1v0#r-Dl85Kr;=7M{@Bw6NVKh1B$uwW7)<;usd|B(o+WVqcLQ~<|@_fRGjso z$5kC%tC3&D0L~Ain88)@^ME&<$2j{wr?_B@@6^D}gNK!>dj(_Pq1SHW(*+&PkJTz- z9j}w1`;kW6n(E-v$&Q~c(iE!)Tr%k~JkUhe&Q_>i@CTEeH@ldA_Y^~K#L-<)yP1c9P?hiKmftZKdkslS;LnpKI~?vI;60wB%v?9E+47m4i?bVF+pw!s>~7y~?ZD+s<#m7V5rNQEb#cVxWn7(r1KiCsyI01rpQ zE|gdKpbhyvn$DQ*U^Ujl9l9(_OCKzA38g=8isJvsne%(a`kf*cQFI4LSLLl_A@}mGOdz?b2iEds5IpFm zhoweyHHl7|SW=cu6x%DA*jiW2rzwQJ8a(QK2SATdEcr+9!D7%dET^o0 zze(p`tjI{oO!GgtS4k*m(WuGmUv(i2i44>;BKuQG+j6>F^JgYoO)VyD)MVV!z= z%lOByT)i1yCz078F%3wZd9vYW=K|72Hka9PFZX)MJO;{P?ri!QMPJVC1HYAa`(&B7 zIiuodgMf{z3^}5c9LZygQvr378$pzECNqA*eO*7IXgFj_{(dW(XcHXgSk50HPiP}t zWy(`5tIBHAIGLyLd5nhhIM2^0!!4~!Y+Bd&eX>_Vb`~B}T+JX*Ud_F+$IBHNhlYUW zcv-v(tnY3nm%S$jlKtFH9#}w4M~iAZ`DaCJ)x*p~t2p>lufe9+RC@}p&?5I5zJ4G+ z2{1F5L=s)26E%8W|4aAdx zwl;VMKe|SGm;iIj;5+rZGnsLsHn=>9mh*Px7fhf5dY z0%agvrQ_@?m@{*5vuz)QKK-^5O|W){NHEa1tG+@EbJu5ykwu?H6wvy1kT= zw4nzkT2vuv{oV14YsVVwXo7NPyD&JD{i7_k3aUfN+bQLG$5%4`HJPA1L5CXHe7db$$CEpX+4#I&B+A`Z@8sRLCm71M z8a0BT00Dcb0wilJhNyL~uIYaIIIp;O_VEX)y3R=0NZySN(C?>&z*oCpa(ODrgaWzk zQcsERx)ie@%$z;12?kM1Pxf$*N@_i?=HTF|U@ZGi#<5?!I)A2C4#|Ka6#O{PYdA4S zg}VG}S^0aCCr-x&Z}#H&iyr3ph0*ltn#M1yIu>taz$w8Nst(<}rCM>e*ZCnbTlPao z9u|LURNdYBs5+PQL$JZq@qX38EUQ6dD`pdS4*mchbWN2C?43Ut;ukJ!1{5vvbUx0M zukns0E^ILK`nsQA0OEC{a|!4yV~P!NOQvls z`F#;f_8P|bZ4v%1HjvEPOk^>=I38(8lp+w>$0p^ znu!xMDW3G>MdSFkRF`Ugb4n-AE^(m$OLy{c7(?AH?jdW=z>G+Y2s)7n1CJ(Lt#4yNLp2uezqtUVS zqM6a_;8|DBT=Y7zr_rLZqU^O%d_dJ=S3FbJc@-ZVCi4-a*YU@NfBRs8jCn=WHnB`n zBz`upjE5~)y>erVkJH+V-T?W!0xa+UjE+Uge~FHTpN@_N>8GP(Y5H&Sl~el)Z79uU zf!;>?6~+scdRfagHR#T#{Jni1Rd}5seF@83!J*1f5zU~6i&MkoFw7}1zo0gAc~*Af zP^QMC#b%EvoL?gw_@$JIoD#Q-*OY04lr?lHp{y9~K}dx>#ZkbY8SwyWPe|qfX9k1vjhpm~l=|n5@G=Qvo+203 z6TG}8qXn|LD*I1_U@TyE1n{2Ze)J^Fc(9Rvk_#DaX^=A)^f_b1MgbK_|ar_;Nn%Vj^b5AJkBXMlTgJD=lO43TXRFA2$u;)nLNxHI?$&oN#)CGyCS@Idc`d|rk z^!et-&TLUllOIZ%HZ>rm8uiYSNcdQ))qRTddn}n{2wi}ahP9n8eNE5fvPWf&PIxPs z9Wd)=ehrhfJEop%Q2dSPMzxcWC&3z>#(;{%5}ayMd0&BH6E{M?#M4TR3O$uRLlb`r zrQDnCjO1qr9dEu`0UH4Au*X}JKHz@c5kD}QHH{qcfCBaz)K`PIva-0nQT)Ixvsh)t zq5CI`Lm55G8|g1>Km8c+V6F6Dw3Amfzo4qH{3f-NCggwu^LAwZ-f*7GWYk2ck$*-n zb6F|Vuj=B>gS~Bx*8_M+wlx(e?Ms~n>5fz`a;i>r09@ZM?$!;ed6UIL{2C#X)2QhG*)(%r9z0EbpEyp0BrctWsXiY7Q<;5 zfc`yDxBYB+&`z1`EcyheE-^A;D!R|qz4UBdI#%Z^1Z}g@)*n*^iW8%m+0;AsNQvNI z^)Qsd92K8g%uy@tV9?-KQ)yqbxI?F)JfC2aMx3LvR$_B6yG|oaI-=sTXj2fp<9*0Qmxk$2J)|$Y{BIQZ)FBZ{Lr;dgU$)rT4}L`|zf;@CEZlLA&e!AFWPI9}7d ze6mfS(9TCx&;pF^(bNP?RtJ~fJWLW6*_p<_Gi)J1;qBA~%D0djLp9@8R zDKF@CkCG6FGb%bgyvod|foJ4jur=$u6(StY4*IBL!@5x=@TrNB!CE^5ihaoC#s$_$ z5y>S~Y^BPwFz89~oL~)Vp!GA`uy)dAVgQBm-|r*bwj20VYn;K3W%zc>xk2S;QkpQq zd$OJMs>IA;jFPe(IN4dJvKRY(ONQr)4)5&fk#n{qPm9 zQ$%jtU@!=P30uNq#=s;`N?H8~gA%`O^pvd>31N~>%%SmP%3>AQK#D+R{P3F&GjFVy z9iw=UN_TQNr@je4w-xIHD&HlZd*@i0C7nWt95RuagX2aD8Eg^f8nxjfT*3IxhEw6@ z@ohYJG1TM`#bB7bYY$s4`@U?WiZjL-_JgKaj1~Jt6&i@zp&VI--4ePuz^=MUg}>56 zzdgY2B;mgtjX!9pi)1#ad#Wb(vla8SF%VrM6!x!dO3rM{>_*JEx%m``@2_l~zgjN~ zhakSkaPn~Ftbr4LU{nqEidyMc)*mlb7{p3f9c4IC){Y$ z2?jrUQkf^r+&_#Fn@wQmxdu?hG1TYsE^V}$z&RKCAM4MOF&CX*WJ*kx4;o@bW#)Q; z$$APie~(u-#$fJwy%dtCUVTx&C(F|CURdekD#olAG~l`GF;P!eYl9F53eX?J&E0)` zOp3A4W0FAf}U*k#3wdv6bSrjWt-&3OEpnx%@ZYBhH`qaHhu`K@Yx zdKwVewrZtCi7O8dk>1+O5eMK+P8uXjPx~VYWBhEZn(PTm6m|FMGWL23SaGlnu;%V*(uM<+-y>UybsRri z?doEGyV?qmBT&uE)e}dGETRlb%%ponlTu3XkxMIn)^T*6yY!X4nlhXM_)}C{pS%gK zkU5u2D^T@Vkk(|TzpR0F639Lr(~{DJ~k3l*P|?K?rZkoZHfP^m|lW<`t6O znQh8Vmo{uDA)%GoZnBiA(#G@oILf~8s2>Rh52+>bAE``&+%Qz+On$>YO&e{x&w9(2 zd0-~|2b3ByEyI;`L*W}$tYa_PnyNdfc6brB677?zQbPu7h0k*FQfovxWvbqRDKe0RP5+3dgY;hk)&~AGr2D2Bjq3rYg=7E`Dw@ulSaw z%`17i-6$b_jw_2ASL|z9CxCE)ea@71ehXzATfx1kt~%GUX7rfS3GfWfezEL6P&{P% z;klaPMf;oT9Oegun%>M`;^I1{w6ZewRIx6j6!;GfjW1#9EZOa%@|MQhx;qp6!tEp^ z%e?F(QxFxBV)Th@qn_-b@~ta=(uP92ePeq$cM137Yp8klWGbN`{HcqUR&6KSRQ!O< zr{i`q>Nm-QrCG`>ar|OcO}9_eB!!JJ(^dtV+|C@IQVhh6r1U_YwgQ@1 zE)!HS&o^?cIA{eRIQ}&gKlf+|UIyi(9&S@*AMift^sOph{+bTr<7J>66|A*in!5NZ zEXF;$n8ldAkJ$+`%Jm7hqm(qOoxQ;Q{D(Tk@LHyB&irt<8iZ%Ot}ax>36!9K^U-Wj z;^YFu4+SjvK0-p)I2W8MCcW$p4O8EVx^IHIZ%Qq>%)#_TPLE@aD#TvA;Oe|Wmc?}M zRK4gi=A6Kr@hpnO)?lJ86t#A8E%TDy4ZL|P#gkujA?j0gyrDTlWajM3}Z%jk%s8%3`buSJqda*G8 z$HRtr3`VTh=f~GV1|insg+9Ges6Duk;vQ~$2|)X7=H-vs8o32F;FZ;$+HjOz$2cCa zP-$i-LcVY_@dRMp6`r`tV$FCj+G+rF>;$*w?5GO8hw%|dosAWf?xboqu<5l_tcTaI zY-U0G1+ok{Z)+822h|TYxX@yl!r&4^x8E*Xqg~;ZIW)~+J%yuX7Wrgd9V8Z^49DqW z&cA6=80hzPKlEIwUX3Wu0G3n=U4CQ_##g5jGC$nVr?ZC) ziFLkj@e8RraWisp;vy-3Ksr zb}PA6^^+=>>P$Rj>h}Jos1m=zIu`qxN;Cf~u;BYuogeOzCZh_w%2@t`D(280JK)Ek z({)>RL{i-Ed*XJL@g-w5$nr~&xoinJH8G^9rIzp`LB zn*r@G@HyRJGfS-yv_XF{^wymfI<8yLmD{UzF|X&w7??gS4+hpbv+)_*H5wxYnsu7WGgL{CuSu4{Cn)-hgiZq)`^0Zqw@p zlC|+&`!$%*VSyoCu8^~bdgMN+vnyqF91QJy0|q~)t&XPRM}IZ=7bxqrnLk+blP^JO zZKu{+(b<|f(q{IRmORFXiQz)^6!@yMtK154dR3#-z!=7V1BUE~5V!xuKv@mN1``O*#fDcvJUh>xf4TPtnS^X^ri_{d=PgSwxe3)47z z0%dx-p(S99`S@vADe{E%AlV7t3is40L2(vL_@21r50IA|dq{TAqqga9Yy8J@zmVd0 zrk~fSuk)Z*br-h82Lp`Q53H|P+p4yUX@*!01%3PEPM(|q+WO)kfactnU#gL3p0F#VM1R{ZI|&s-p92Tk1&!DY$bz z6L0o0>oVcz*I2rEWs)G*^bFJ4oS8qZmr^R@uw-QQS3VoCi@iLBQ^&O zwI=Yd>CR({HQSO$b^8wzs<<0$@s}!VWsl)=@cAPx71z&I758#OT<SPyRX?nEu&)AGcKDgY_ z`KA4|lwlK`omKIU%bXvM=@eMTIy*sbeR3UDGRNE$hbxRX<#pZ1An4I$x}3{DYvLZ2 z0ciOFt^HR#$z=cNiX#XXepM5XV^zI2SOqhH`ulFctnd1GNgUpqpi9T4lR> zYSfV|d>s5~@5OI|TU{x+)t(?+_}8wo{A)Rw7ByAe^_DMw08X~&mF-1uJWlbk#cw=* z6OplbbWZjF!ChA2x;?312c4UVKUVJgK>O~6@?89X2gc(6l9d?5|AV0=pH55UkxWap zh@#=!a<=~h>XR7C^gqe87Ls3Wp)!;e(&0QUiwGNI0*3MozbZ8>NXr7;ngVFF3lRQM z)*{xUa!s!tbxRLrX;+qmQbsy?v}t0gX#oAA31~l(uzi8kM2%O5i+n*@Q?^!VQum0U zishcHXR;BQY|$9W@{MT;0e}3b*YhQ1f_F-U5F}6`>GrYsbauQvGZC$U6ls zKOeuSO2D7o4fR?8vv*J8S~JG7*jZ(*bn|itqm^u08dI7B1lEsNvWkln#dm2;8pYk} zNDxb&P^{u&C9?*@$LQOX1_%O91)DQ%G%3CfR7E{3u{frLxH-;B*W5Uj#u6-{9e)Pk z=sF4*z07eI!mRP-(#`rO0Gj+ynqQnmU^)nz=PeBv-UKiQv&1-613mtFh4bV=DpFkL~|lQDB2Dvn-7 zNg;TX8g-T`DNl=%>DQXnOr+Wd==Bpq)9F~+RSp`FFE@b+$ws>Z+JBV_q&*g|JeGRv zX}A$Z;A-3pWFLxVHk-+FUb=yeqMLU99#Eh!`!ie$wV%{%ogXH<8_Wh$JCS?&99#B< z#GEDox%l-9qum^VTSF3q;94extv4YPxCoYI1>>7@nL0kk#VrS^Ir}$tG5{1rT+YOD z$`oI1mgo4Caw7oh1sZf;;EDx26IM6|<4##b){Y<4z4%TS=+WL@m8a|=1hnSG<6M3cbO+%cBH=br}+)QGbjO%XT z3V=2rCZ^s@W6O`GfL15vu6Uh-YHTTW1P9f(Xv-vs0?gtp?t-(>Mgi-&0l0@B4|tKR z$ZRI(Xu$Z(A+ArB6e(c$=Lcq1i0*9k0vjZFc0`S5LMSV<8XFGmC=f|t`n9u8ebUdCD zS<*$^g$n<5jWxs2nj2+opDkhHKBrgO*%H~D6knd*iz}AkZnw(%oWbB;+82tE2LeZl z_e);)JojRSVjD7sAB2`}&8Lo_Sdh;{6mEl9(_iR>A3V>iE@=7{HWqm^-(8o7n}p|= z$Uw>Q_~F|caG3?Rhqom~h^cR9g0m+aJj7lPkoS-Y$WWn(jNG2E6#~s&Y#2lhCBZ0| zp92^ZIm9O$>^_Z_#0eC>xI<-sH6e?w2ykw#VzA{>>pH5ZPw+o*Te=XG)vT>~VB$;&^l8{z@I>fWw; zDNX{=V>tc%8^RE8XqD-4{A;t^2*Wx{>!i`TOLuI@5b~o!`?}2LH-Fw2BQ%l`g=?UEYvBuk;$mH&oe!9|rje6}dd28?uh2bFX)lod5-j#A6cjrJ(+72QCd? z2xBzKfOP&%_}O_?to9AO;@+#Xzy{1}7NmL^c?=_z8W;h9_a z%8ClUz2x63s4F2>z%Qsl0n~4_qY7;;w|#P6;QR*z&hgx8PZ|lMU=Llhi#D#{t@H#g z{w|nxOD|3k+OU>d(&ZQ9#Q7RG4>55fA|^k-sY;l^+*K33@Oz3ghgs*3#e)_>#7eWf zk_kQ{>q3i5MR`W-U8}io_VHI#7ut4W6?X$Z(8b+q)jfWKm^#23(M0Or@rg0H9n>Mo z6RV0h=R52W#(foiN_s(b-=s zWhG%iEn*fFV!f4pwcdyF8ia5%(yTzaTnxtu`O#> zcOvX;-7;|`j441UT)Wzqu58N7AUr}twwQ}|HRf(VQEX~(Q{LjAzAmK(zt?#>7aooKB_K3AkAW&-)~$e9&-dv3b(sjzN(#m=m9d-dG(6@aD6}o6*V@!H zxNj1(PV6ZU^37W$Zy1=$_nIKUiuobtBa}t-{ zi0~H2V0&mFcDD02EWYlMHs4}8t^m^1u^6OcNnsb4k7o{~@D@ksKkq9=PH6| zHT_yY)Knw;hcOZq1XS3LQm?shxcqTa-HrTZ^5QwR!D@_Nff!hVTas&i!H{2^084!n zZPO>j1%2E)x=x1-0B1m$zw6I^QpL<_@(;(siuEV|GC;k8B0bX{4UX*zUIhxZr({wq z_q_SzRg!bfTt)x7W#%d(VjxuVkD2iTd2TUn3U&+q1oW!CWI_4M*Xb^xSC5w=wlz0x zM||w@<#JzWoI4E{{4rwt{;$+4|1a$mi|o?{VaccV341U55AO;J85s&XG>Dplq0^Ov~eSWfR1qEC3HFLHTeT%w|J* zNzbNbMcLp3{6`{hb1(-$cue+JJdxwhDWUSwqOxll$0L29b)^x=v&6$d4XdMef+!cU z&Bv5!0hklS4LEEA8)YjqWw?(WNdneNPU3zsPSdQHBvl_wR;l%vOEem;Re+KO^=PcH zQItF!!!mgB6Qyv1tAt}U0!QV_RNBDHUT_lm1RZdckYW!rM^BQkh`?1Vxl5&@G-@8V z@v>+Gd6qe#P(}X5u*yqd_xV?7GtmZk57vmBfpjm8#*i!rT^Hk%taU^y?ozd8@t!tX zuXY0WQppmh7G1CyjES^p6w-i7fcgY%_;T5<@e_lKF&gcsq*9}9sr5+l&Z2&S)Yv}+ zEgL1v_D-sw)u>JUoty*#lOTyU-&uwZLF_vjiGjFFtqq3(1zJ`T=5FTa14zK^Y0c;h zItT?DCL<^|F<6TvZX8t}-K|0~l`H&34(#zW9Rg^<1ntNrzRh=1`DG@JM=(|!2{RBq zjo-;PGPzldUr$Bb){bh2iv0*z)mji!niD!c2CtA&MCE)9iupv&eF_B;4xMM}_IWPp zm39FJnG?;723f+4|^47J3lgpY(yun)X8xFb}v}VPtOinxGh!C24bu#W_gwJy z*=!{0(ihn0P!x(yk4Z*-P5Ohxa-rL^E$M4Xp@_oGDKtwOX%j=^m-G^LBMldJp)~fY zDE47yC!q`iE_Y)CYxsx3R%rl<&Ej3nPsn*XO)nb=KqT;K;rii2R3`^w(e(o_1{_Ps6&?V#}$b@u5K!7lJ;z6C+ zw(X}aSGUau`J7#%pWDM$w_HEH9V#;~L)K(Te_!*X77)i$f#JH9N);C<75Ab(Ki^SR ztf|%V==h*}mBijVsmwjtWOF!{Rs$>qrw=v_b}mKS2l#Ru?_@N3K_S{XfFyu(9Yk?G3s(X{ie@Zmb+QReGul4D?$C1$* zY7Xq>CyE#wpFRu}?vYu>oqVQ^JTVYBh_HW;;^a5F68K5rB(xL?u46XRsa zW)+s)p(csOa30Js#QtNLy|e1apMYSmf{wqiPi)a-5D_}0+h6OGDeBSf4)5=EmNOfR zGp5D)92?!n+xD5r9Eke zX+Jg!(3tD(P86*NKe~<6&N%KKI^Uhb!q$?hBT!TuoK0Ts=7V5mu8vr3B2gXvdu|) zmJ78@*ZIk_cD1r^zj(_97J1n8gBKx7fMGS-k(BuupqjON1)p{n^w;dBG|t)I3{f58 zRtaS^`>}!?K>p z?Gy0p*-faMs$4!dcx7j2yMp6YFqNP6$u74CPgcj%?iKbyraUf&jjB6ifFDMENbI#T z=%D+>4k}goKw&ElhsWf2^|>u$g$+WD=z2(GHmQWJ+x~ApD1W~&YHQ=K^%R>eOI#WaWf=*S+GV@CZbxm>`lS0T`vMfqa4Nd-PBw0miCf`l!qvj z6LD}AHK^3=hAQ&F~k{=48c(<*nld?LT&9v;w}XVnA}>#a<)^wqD){1wW0lODC00#FF9U2 zd=3`B4Ed#5F}k<9pKhyGXd4eM&rAzaiBomT8KsKU@pmW7aKX@Fw~ntn!!QhAm801bGF z-Sui4s+_;5LYKCPv=L~qFE*qx$#T%u@QKoDU&lI0ezhfOW0??9z1o8OKsmk-W1AG$ zGxAyw8*ZCCbzjt>0_HFuC!rq3sRXH=uW0-gTI|ih+U-J}Dibi;^cC3-=}>bdO5-o;G7>2}5<>)@b)#eI-^hgJ8GqPlmIyFImy0H=+I z?%-gotF$NIj%sOptDMqQy?h1%*;W8}X4wX8d&R|vsC8yt=g}@uvt)=^>iK`bphNvn z38RB*I7I_UMi54SdXb$-{yC@ss4cMFzWspe(Q@$k zx=P-r@J@9krc`as{<}d4b~vXHevXlD6q7ME5*FO<$f4i6pMt*xS9k5jncKi#!V_TY zvFyb8C8v=`#cH!QSZwboD{R6kwmo|Id`&nIx{IrlMw0VG<4*~7M@llFR_A~lGGZM9 z`9&ZJO^OwhBO(8%A*NQrV+jnW-vX@s=0*rB_1o|0z7}n|K8xx0(h|bn>WBktARlYx z0DD21Z|4%{!#ypf(TSrc>pK5URicE<;@kERNC0_=Uld3h^#e?6a;ZEkW_DCE@>suOyT&eT*4|>JT4nQg$ zhhm)UE@=1#;qV7^KPT~cm9{mq+7W2mZWX-y9=b?CA;wqKGFtAwsZo*7GU%$7)*)J~ z@!uGu@t)yG20(9D@vbevuotKSh+(Ov7t%el&Uesc7*tTr*zV9i&$H2#kalsbE$G7Z z7~2mcJaUpgn$SEw%3%9&&e|7J9A>1|KBJI; z?8M=NknYb`$feSva!46rq67gy!WlkPDXlI=2*g&7+_iPT8n9Ej_o_A|9q~K1u@oMl zDz;Wz&erWJKdnN!k;Uz5r%M)Uv{TgaP+LdY6av9II}gwA_CkQ~Y?I*vs;h7w(C!TO z-wZz-HgsUTT06SY;X;*G10ND@>FE61{p34p#~V1ms7ANC4#Gt#$$6vnkwNg*f<8(l zU=KoG--UJ9f%f3+Ud#Aa)Xc3JLt?_Y6{(RZ)iVa56Fsbz&mytbn*}8GMb0YMd*V=2 zv3@#6rk{d>_qoz3nXWb|FAZvba3oHWe;Ar2y<-ZHA^#1@yxpO_2gYienl*@R(*h3D zI_I024xrK_rk~V%bbbbv6cLFLN7P|9ag!h6txDEPQuRTV|Am?u*t=E4C1VYIr1Ttz z-!xb*vnXF^`7ut2yVR@SJxn>V%z^Gb&W8EgQ}@$nsKe1rD&a8Q z6MMwlPz@K49f{NEd1#;qbu5u39A7M_9BNstZd8ZkY&$GBDDh1O#My%e-)#aJ&gE1D zl2A?&=GB7E#VPx8Ax8($PvjP}mNz?b3WX@9jjgtWJ#`McwxddJBvO<7a6tk_`PCrX zONS(J4+Q0b==|FHsKawOiD8qLP{$c-r|Bbcp1%ZSE_0nR4KH!w*v!7W*U#T@4Yoph z-bn-^+vdL3q#EepA!={VvV_Ak~$8SySv#_{DSJ zhHp4hmY=iv7Cn?|um~pmsHwV_e!nJBT~L-%+mu`n@!~hzv|LHTL89KH(4YNNU@#Mk z=Nn)l11`(tY~Abtgn_{F$do+$0x$dd+r z6qdPrljDcrvd!XXd3~T|NcK z;yR{_{cFN~ksR|1g$%?r-2ZR7w5X@{l%2j0BBVJ@mtOA3u-SFo*}$90&-%rVt91rFL3$Hkd$83em&*NbMZdaEqsl0{uq@I|#<_+#mIS5oF6eu95d zOsl1W3V9K<(cwENQff09A#p4t&B1Cs8hb;GE*@MRbeNNy zJLzs_kfR7#${bF*+pAuBHwC7wB^-~633-%TTG4(4v=KA@`g0wplD4LK>FzQGstpDR zhUTRwka=yyZAwLKYlfI^PIUTI#G1LNVM?M-VXcs%FLvgmV>_hGcILQzGcYY%7d>*9&?4-&wcH5QF z#67_;7E*r*>s=~a1S)MN?`jg>tgt9%F|`2j^n=wV@CTsyY+(E8z=9NOft~P!ad-;E zMiL@S@_8;_Nm+x9=-Wmop|d`|tlBP?Su$U=71udM>BK7P__4D}?$l^b_WG1tX6<^2{pf**)A_Lgs!qEK}>E7&c7%(i8;$1Ny4sa zYb-V?Lrr$9^P{6-p!2EJ$NhFyBcO znN`bwZ%D7@EshtjkkQCga6bmX_5;DI;5-x#rdHRufZ}u9vZSUZ*!@$TqNrlAt=s=F zK&7!*f>ZFhk12``8+#nUcq*eFd+Mwu^Ne2xQ2!=C6jZiIMQt!xBk@l&IS=TU@=HOj z$NCBQbt5{k)=9!exX0&R@&>k(ZyGcYCvr+0rdk%wzPL(E!v*)cKx6i5Dp32Qrn4%O zY9#%WMl`1xWmcKgDcj1B7&LqMa>@g1<=0AYtj-=WN{=Ya%6qWr4^%o)SGL3aqF!yM zFLJ8VApZ$ft5M5kzO^kOAP$uAUCXk4ys~aTYZEvbq+QnD!JY1}_|X`0DI9B8r$=3o zjCC@326Wkvw*fM}zOIgjzp3|ZH~>XM{&0di$5|V(DRd_#B> zVSdFKF0+REa7PLv3>#G0;k-zG0VY3*0zx2KvuIvcfS1pt;k3VHbh79@O%ppfwRSk& z5TETW55_pi?&PECWWxa=Q{)g6UO1`+N1%DGB*b zs`#1CC~BDhy{hwVY>CongA>4mCdR9i6UpX_C9T?kH5nHm5QhkmLHh&<`lHYw@e(=O zm_u<$4a#V}?dOF6 zHh?*^t9B;`_Sc%xcMMth6(z7ca{wUEb>;7yu%8qqayB)@`&a;r>z$n3lj0xMn)}o^ z@reP<_Ss$;B8P^qwxeBCevjtVc~hF$2&DLUCQjn!a1_ZnKBEA!U-)>JXn-6TZRlz7 z8H(t==XQ#a(AN*Z89v(g0}i*ZF7MF{M>ARiTu91a zaOs*%M)U54l&3cFoBd?{W`Ku912lP~&Q*7DT_`4NCYw~n;p2?al#zAPs{{i2Rb4I# z0vBJ#lK7~1bZXrnjpsj??Pa1>SW7D`Oq0yhwOX|KOne=|YTK>*@%gxHx-Km?Y{NBz z*|S)gT`zt;rslrq(&Ozf>VEQNN;6n5t6gfGw_v*bVsq;EG}iT#>nz=#S1XgSey;QH zD(-d6+8cH{>TBsrXH1#>F85wkW|aFN+~IT*f<$~=d$c^>-Am^SWroCXwUfxCzDLcH zIAZldn6U=PdPvXR$y0^%MIFM=>~T0I%bh+Pl;X3cPwK?Fd=hSgnYW*75GHbOZF<3x z%-vkp`A`hd+G(`*J;1tOU&p7sfK^E**0#inocvS#FJoXzq~ANBcg@P!=kNP@t4%NZ zMZJ?sXVad&QWH-d6hFs`I+1tMXh~he>(p9ZOzL+GZjShV3R!6;Z}rHUr};|le9hi$ zlAktvr2iAv&^^?YCLKz|Pi1}gp{DGP{$l_a)v5AkdRWa*bC}O98-&7k1i@)*d>h)b z*`f55J5)`)rARccphq&bZIsn(Ho~*{9 z?OypE!^G?^8(XFc*{{3On3LoDW3$XTr_ERHQgsUymt3Bwx73$a@)$IFFca%-0|1qZ zzuc*Pe6T{7oHB$H2i0`3g{4j(7Crl1Z4l(b9MVEo#i3Tr;NUN~9Sb-;urw~8oZrmSG8KwK^WKm3E##RoPSsF)YRwOf9O{HD3v&NNW z71x#RB@5Rq-Sf)5Wx1FPY%SgOw7q2t;+1=U@Y>_l*?tLrx&M`W-}OeBx=1OAx9puP zpuEJBSAm&PJnc>DP*2&7l(Sfti|xH?!TSN*=YOvZ`!B6K{!Pm^Q-<<%#;150Pm2E# zvotPu1RKgIg8A>JiG$I9B4W@3RJuh8h(&ia0g>_sWfw!*LX!`Vk*@%%?)F=-%Q561R8h+liltEz?eGiB)eRClVrXjg|sTe)QhS^ zo7YKG9j27lm4s#b3jqv5BgCn4sxBhD@5dC;rL<`;d7Mc*eggGrC%87My5p2Go2Y98 z^gR{4=?(HY3nBu^Xh-~Y_=M|d<60T_61*}jqjSJ&;F?&hatHfKZO|4k+DVfh{Lx*g zyW6U!I!8K z-mH};^ivo!$O-Q@|L zKWV9Q^GKL0NCu?00&Ts4`^gl;u*wu5XDbN#m_6A0!&#q}pJW7P3j%y-`jzCZWiC&^;2$=rMRD|#? zL$fcc5^{3ymf3|iB~MEsPaZKVZ2(X|Jk*pA2q1Qv7gyyI{NNr{w2F=8#J4ySuVk-q zFT>=oWcIGA-z7{`=d~8d4drIjB)PWus0tdpT!tzk*zTX8(RU@D(8`C%3kiyvo1;9o zMXkZs0OiC!BqINgp%E!4kxYv=8(K#pWDArvTgJ-$apqB=qRL4)E(ZMBk!Z0*PQ=kZsu7IS5~R2YA}fy`3!hbxpJST?xOk3?}d< z2V@lMtOcqlcW>O7gH9n4hl(#31CqY2)rJ!&OzL7$^^?1Mb^eBm6>E|gfwErg<66Ln z#7|X0sB$}%xyzdTXiqBQA64Wv))FrF30OzIXDr}e;KzJ-V2xVl zldyj8QVn*kq8_dg)oK@5tFO7t-r57QS&6k$>?j?zb66dCfZ2zdos`=4h+Djxdp_9Q zQ#6P6MA&!GevR*6CEO+*E6>|ZCX`zOTk#xFix<-ZSi9~Y8#cj2QyeuWlLm?Gvmk%k)m#fKc|MJD=dwFTPJZoUsK%A zsPuRWmfelzcV$gtqi~I!X~n+A?-T{)+}`R)Zj%1Xw4w2i4SI0X$sn+Uoem1_(;6EFbT{6-x z*6Br6W;sK+BBph^w_bDSnZ$-Ht%2Kveu|ycYUR&^qQJVD(1&9WPSn{8($ZavoByrq zr5%%kT5C*kN=vuDJ0Qb0JuRva#`pabgjgr+z04wvPicbZy3Co=r1hqAoyN^3=MFh> zlJ_;)E`lE_BkrzI7hlvB>wU^wzs1wV9gc9oGS2CRTDE#xI7-P3-`2|_?GSqZ7dmSs z&(%k=EV)~yI-i>RM~%PJOR|vet&4~I4PIj;oUd?Yu$U#-LfO|g`vlrDn{Vo3mQ@{& zAXscI#olasc0LQL@2Lh+R}tpc|8yMD^-BOAf* zp@^dOvx$E8yMDuqzEx3WO9WZac&Wjx^WBjw50AGs?zcx9D!J#m!feQL@hoq zfIepX7^O#|lgHF1tfk8E3<%J-C=~}JV=vth=V(>8x5aC6N5A4HuqK%>N6k5im&OSfnPE3HP*^y`AId&{+2Fy(E^-TblTNjWx2b z4#FnEV2#pmPY%>s6B1xI;wFOsiphT4o=nmU{E|*Vb*=q zasBSSIy;Bpgyp6dKbI21tzV772};m^dZ5g8#l^c|?%o<6$%_2TFb)?_r=CGhZt+Jg zh=~37s`P2_gQ_$gCsEHLU*mg`m%%!6+(;IXIkOklg|C5{?XC0oiD%h=0_;4i1rpRd zng(6@g|7U4Q?gsQWAKiv^9$SM-rLC3aA_|Va<8pHv)>-gY zOK%4I5$YhPq&StfmENP`2c$awF;R9SCPVuYXZjKw{3#2vGpl814N4OvuCwPxt}`=! zb=`6bcY#0ce@h$OifXr($N<;OyIz-NTD05pcGQvMNg2|5a~%<`#brcnk=%)S@5SXG zRO~h97HWtndw2Zby${3UzqFkHAH5HY=+m7wl=xGjtt_BkszY)2(|t8s)H5T?%HR`q zXrQdTwTk*5=ieac!njZt&v!!FGZ{}D&E3jpvY{~TP-npg&dd!1wm&%dP#%=tr?QdZ zEJi^FVrT`?8Mz5!utF(afrR7BjfJNrNtqo8#J;h-i{d|tEdP?(UZ7M|oiUrq=6GJp z|HGQ7?+t8prCqFGR9!<_X9pEDjOvf@zsmF+@W0Gnn-&$s1ZGhd;S#3GHbTIvP{wzv zY#Jp@$}FRxD$xsN;Q&$%1&PcXYT^C;Vm9@6cwY8*aOE2*_PMl6)@1lX%h zO@rA)!DJ!JY*xX-2GO9FKT==dIevn-bI;Z2Ph}c4Qlh61MKM&u3MNyy<))O=NI~tT zcOr6y$-USp9qK%djn=i|V-7Z?4lk0&6fXmGD<{j98g+(}iO4Is4FN#Ya( zu*Eowhg63X5GH;&eUdF+;FD?F6hAERVdsFUQU&!)xv|6XRN7zVEVU_ea*j4Pj@aH0 zn_O(KR-D*ko6+6`_GIc^`-Y~oGgR4kI;QsAHtl?NtLn#pF<{Cj8`PW6vE&vwicOm* z{sr`>+%e91hPEOCpnJ&d#My4TCcbA?Wz)Z7RLTKL54ZjJ14E~-dgQ?ZA-xF@CR*uq z;^H(-vwx(EXE<~3)%n&k^9N`LJdLKjz_c!rO#|Aa?j_;{Khir1MJv2~JH9b9D1095 z1tm<=9w%g?jyG_ztNE=o9P$zGP~m>PCm#c zb#qB$%MhCwzhJ#7QuU1Km<>A1t>p zf{YsMQwD5u9Uhb5%8r2*QHI1p<91Q2?&~h!qHI>uG5qv5HQg`1qJbD}&Zt?NE5oyX zhU0c9C<-tf(etx=RC;RFJ&AjTq@MPrw9oCPhK7Y)on8Qs);2Y)l~yP3 zz7NB;16OxeWjL0)Jn)}W;hS}rI+DwBuVzrX4yIEj8mTx?r33PA$o83Ssj|wW!fP{G zBmJq4E23o>lDAa3E$EO|^}(}NiDy}1lk7GQGSf_fL_duU=MD{0W7refdjP{9_2Vp|7G zSiw7#+m2_`41DeHYO=)$IlNL!{Is8u70A1PgGNcuP&alm4$jj2jJ7TWfxCrv=6bf& zj8hC+UJh?&lMQO3iTS->m+5F}?s6}ZJEYpCEzhxK#1yfz?yY#~H{_Uo^CJSrUNHW+?Z&&>Sf3YrxE~D;EP<_?PZWZ=D^0~F3 zWBu&=LbIV@ACtLfn=|!y!aZD8h{ibTp+qw8W` zgIN!b>Ey|DXyKb3%CM$9Da!21lbgUxCXXR71elG(b?c#ix`D~=mORpo#o0VoiUK$OEQ__?LnNX0Lwma zOB|3>q-dfnk$Irf34qu%e$ZI;1WHI?gAp^uU8-VTi{%+y*r)Ta_HK-cRc&#+%^L9( z+E4C@x*N7L4z?5Q#bLTBxb*~U`U?JIrR zHlL{b6+WYkj3423p0?y>7cjJc1GPD_jvr5{>h_U!8LYeA0dX#8E(Dw4->b>Q-m^76 zwMGp~iwAX|PS}gNTrKhKYMGxRbAo<2(GXiZHTUTuzTP45{KJWzJfa@HTiLRTXjan; zH`T3HTX+3Mp%-EUq zL#X>5RX@DS(g3nZv$pfkR`_=sWY@EX>qnan=;F(LvPfeGmBB!kWmG2)SKYG*#qY;4 zie*9!4fZ_QF<8uydKEuZ!}Z7WP6T48TL+o)*4Z}=Dwrm){%9Gd{MZmD40CcNUp~d~l8Zeq@Hs(RtbyUpK&xs^aM(gZ{u<4yF(2 zVk<|nkFh&xmrAMrT$RzRAL|r6zLwd)H^uta)Zr{21N*JZM1V1x-@Ox-?#u}_=e%+r z9vI^XAph2hT-JRjt5kkpm9$$7fg_Ie=L%Um*(Si_{7EuE&U3b)%=mw8CSK+gy%l%0 zxwUV>16;9y7%H77A6vp~3tZhB^IlyDIb zdSl&9+fiMXY^!niLfJDTF8gUjZHsl^ZOV!K3LhZ0aF;sBB+2 z7?-7{vi&$U`typ7b^dj#pirg;F}dt1>|<|E`buf%Yh`13g_2o1LAk~8Y*uSgZ@>h4 z{W@l?RmvWTg;EA(WJaDBuO#-+BE6VU$`~~PIFWO#883R70PM5`BxxjN`@Y1fb4!hi z1VT^`Sw<;SeAX(}K|&0ITrAU4>J%aSa=)uphXdLdDx#)hOSq&$MY2p$D_o{ThqAXf zu;^6UEE$B9B0Rx)#Ql_Z@h*&BhiZGL$?Qo|ZO@N(@FQvT@^!LE2`I7?QWCBg+_tgt zgKTba0>{e)qtPLmWVul8Uyc#UcGjpD2zB-M-fXt@_6!=eGqDCJxWk6xZ{Lu#LKBG59hY;`a>iI+i3pvF!}OmuwEy^BkaBGpy<8TGpe`hBsxSU z@|{G~YqTv15m9<$a?0XGs1Ri+$>x@gvMDq@(~|zz$4E9_=D-RBYJJ&^DJd)RPNJ@d z-Id@Pb;^0ZK%GK*1eS1sgtIhRA-RHC4lwgU!dc8|M@iY^uDC@qX6p9_xO{pAHHzGb9TFUfh&6&1h($!#TCV0sv3XUk)&f{m8&$Wq`V3ONq%LtSZp{+pl2d-{Ku!_ zK}E2ENcIu3GlqljJqyVDg6aWz5wZh*V{#jiU;_t_qDHRJ3W~jz;}<`y=w5WKOuu%= zSN#Yp6>m=dRQ{1fOM?Y2I!UJBP$VrX{xM!F#H9DypmDPmF=PP2Mvfm zQK$&3@bJ3SS?53^_ArRSj<0r-_*~it3#)ggOoMcLT=A~~*2t)qHc19CY8O;wZ;o>` zF}}(yte+zJ&VWv}q!lXj|0U()S@%@%^Q^6b^-~-C+f+YaPrY2fQpNF{T4$a~eAIH^ z@1s&GW`1no1XPW#{8jp*r&Ad}JJ`a_poX8aJ4H&_D_5Bm`PxAKGT@MRb)q~VrA7Lb zO_x!vr5`?Jk7iFD5AJqbOXAn3Y}r&3YBHfpHfqxp94{RhcS2JXm&q=5_^w5GQv5F9 z?Y9isnlFq5=YwNpHRE|8T{4S0{D!RNtB4+Uw^WA-rRnYhF!P#IcEtfn_H*LC_40(h zNx3`0`+ef4W0>ucr)UoN4bV2VD@*=qsF3(M%=Y{ICr6k#)nH${~3rtaki`TTV;N%>-UKx>#P|QVk#~* zbn_Rw>_4-}=7;&@Yums(vMgHM*%sujR3vpZKcc7|^}2KHs_V4G8!orznSS@X!eHOj z#WG6DcvoZR>$1h_IRJr+z{gH^X~hqxT`-Mr8j3aB8!4i^scFPZ$TyQUWvTwOv3~TI zszmUl=p#x;Yg0Fe18SIy43e5RDILS@pFl;=)?K|gOubJVt^|VHtNco0-;F|E@Fpg& zY=h)pSmocMd{aX0RJM7RCc7VHFZJ-JWJ9@a9Y6jY^w#MUYVN}gUdRHUC+Z_boV-}J z zr^u+P>!WJL{d!e=ZX&ZrJioZ5ZHV8DtI4P}1zM?j7k+0bSD3Vt$>u##7avYy(kb)p z0o__wBIWc*+fQC~nYCNR&(6^dv0J4ETSIN(o0{|M8jJ}=pWdO;z1xFwUmC)Z*8YQL+!HiRW4}=7lf0j`6bW?Y6dG;4ptc z)&{pVYH=*IIx9Jr&L5j1l_vQAt%Fqlp7*(Q8~*c2G`>5kw43cr#{WQ%w(f_qj;R8U z_B49hmbppR$oP>WPaYbl@#+1NfKx_?LC6qDj7E$cVRY*)schIL@zUVw>^IS!Db1XH<=U zS=PJw8VNO}eXONIxY12TUtshxl%cuLrtD`@r+)=@{DMB6@7Ob%CgHv`EPgea6ip{1 zWoDn)Q%d-oE;Tu`?+pB6a8S4JQKft1z6$vsu^8{2NKx+=`RIwrPfv>DIrtcK#;_PO z2xU2qHerGQEa@`Cp+HwrL}!YUchSzrF?BAa4us%Vr%_?n%3Z1FAce1Hj%Emswk%vc zf6dbUsf6qCo92LGEubdWDT)i$FjGPW-@w#duPflb+_upY{`7SpJlOL7hGQ+b4Im0{{0I3v%Y0(QcitpOTbMoVY#B89nD$}KmU&@lA4zdlZ%Yi{| zb=N3fAj8};Nq+?lZ(44+Jd=FR4g7O^O?J1Z_J75Wsx~zxNPh>`cZAzfrty5>S}L;d z5^z|ijHUn&$F!$uGK_Oww%^Pt`bW-0>U>RwqEG^Cfh?n0 zQ|{J9os7v_+e!ysfHCO#&0 zYjS|H?aNd4T3d#uqF>8y*sX}XPTN7rzphnUxVx9llsm6wXK0({r-O>%X(xJU| zm~}R_@{nL`rmQ$|?$f>S^9B_XM9l7_@e(fk@wzQdCe7SR`zKe3kI$2pi|%EXEKY0z z5C9w)Y&1tSpq?8xB_ahM8h+XYCg@l)+v80< zG`cyH`Dg4B32q6bQ>!R^zYk;cnqGFEB z1=O3xxN@_ou%Iosh%S?s@ zUvPu$MVVQC^hb>d`53t?gZ--8iDXnP^tCFbS9qE1rY1WeMzW#8@g9}ywQ&>vJ#sbB zo(p1z00)^5n z3xBHXR9C&d%G^QB$NM^E;N_Hc+k{PfT^2q&d+JoG+zcu6^q&HhwE!krT^DBqcx|i7 zWBiPAv#MeM=I}VKSZ`=V6O7RgG49?qDm1v7Stq-5!-O61Y9F_6=p|JNW_FVbLd4}( zHngksrUD$`q?7{bbFu$i?ZmS0UCCVCM_Z+*>VSrf5y7b3S$0*2V&=C}Vqhy+ zDOI*cnXkiHwvv@?6`IMpnjf>e?t|4;T*FqdvgYBPw#3jF%_fJ-MEiv6`}Cxlwn*&WiCb(c!LRF*To2SI~oJeKkjNFZj_OK%qCI)Bt4 zxO@WdL^7>!Z@t!ze5O5#^SbEySC! z4xNO`(d{vPv@9!^8sf`+%%VLwWSZdDDx-Nl!x?o<1k|*pm&4TTFZaoO9~`|(Fng}* z$8&pW8y%|bA6I-;YXrMO>32Qo$BV1v%cA{B#8&|b`v*JZ2sBvDD#@4PE;RWB-8q?C z`}O18_RwoDSh7*hpEPE^#Bpq>4}wU$rY&RqWwL%PDuI~=GV|J?+`Ay2P^CKIg|hov zvcl4>t|0fL16y>1M~%(Zk)WI21;{tltB%&MO#+Yjo=~hA1C-EW>h^v$xiv{TMx*9! z%$!^uJyB+ixDQkaa9ZCLe=p^Ovzen^85M-PaiPO z4-Ob-TC**ApY8zvK6Un>tYEqTZ&P(ouqg>pG1t{$c?U~J=q%|dq87c>$D&2vkoX%_ z*=LIu+GBMu+n~}a4xgv2Z|XKWxgrBSZ@R24o1kKqi5+`sfB13I74pjy%b!?=y3ThG{X;98UTxpL#d9eV)=&T@5% z%E)lW;3OF&rW9%5lQ6bqqmjz34ts#I$)pFkCO?A)|TbbFE9djtRlNDun$U_S%`gz&}g>V)BNZ%-vA?>2|V@$_eF78MLxd)FeGd%#dDd_gqJ{?<<&d%`3?TQsbb5ytPSzTiV@viYutyYD+$_cC`B zGAV<>JDPH=Lrs-#Q*m}Z?f%l#C~tx9CAXcpX*{`%bTO-0ZHEBL)5b|KM_}n=sM>=J z;ncNG)ctfc>{A8C!tWaSysI#h3zZqv4``a~ z$R#((%IO3(`}UK^z?;a3(Pqup6K^V;aI9n$Z}C&WqiM9?GWDXLuYhS(7-!CwBx>8rgMnhb_4^SGl7UwSkxY5{vu@+99D8Jtmvc zYt8(zGLT195-S5LrbcjtNM*^)s25`xf?P#1>`WWP0h7$6eW$y3s#RCEa+7>6evb-^ zH^^k9PTZwAICVN>^E?67$?!D(zqg*{5Hb z8l|7tK_xAtpzKdnrE6 z9@IJ+F4p8$Vlp{L0&xh#65f%UoKxh5$d>aQ_o7P`XBlhJ?nJa%xP5m;CMWEnlk=<@ z;f1qbn>w#M(hvQs8ZYr)xH~lnd>+CQ$o+<=XTemXY*Rzqi^=fO3Rc6L3Ht@=Nl&ya zf6O9nO_iR}r zsPl_;lKO?E$Cke%a~aZ;!GBWPweclKs`)H^Okt0eE(PGwD68?w(_;#^kXL}#@LK5O z6IfZ3uDCM_jCsCxJx)JGRhkx3NVL?Jo>l{FgEkAXl2_=`z)6V1>O5A zVpo;fD=a^G%hl~uZv9FZU!j`nc?$o<0G41QD4HWdUcBv2YKM=S zOzw_Bp|VFC2EGjWBZXgyvWt!v%Ca`8puE>r4XUx|crSN@ROj z1G-K2EuiWN5s4ZB68iC4#Xhixqrg%>)8#57vcoKiw7+A>Gk$zfta;o0xf;-rc zhgYcq+4V$&96GqFSNF)st}C}#Oa5xG$+UYqHNW>0XEY^ZehGxWH9(2?FmJc1;(nbR z9X0Bt__;AM`h0h<>^dS_@F;*`ezhNDw%T)o)?pp!%gV*rmL}e22HlJn2Q$mu6vG)hiW<>?3LcBJ*h=Hv-a)pgz@2ZT|8a~ zvbmaxkH<1=$LW4Jf2wH#Ol*sH{`**})>_Q%-79{)8YDYkW>wwKPNev;wbgjs00-^8 z`$1?G_t+ttNP9cgk7;+nmW-4QC%}sQh$fMea6M)zPIrQ}ZDj9bT#gkiR|Go`Y6 zocMySSYO-}2V`FJnd8OdiZwmxp2le(w#OBxhl2}Jl2FzCg3hk46FvGJR8dsO4p4Qu zm5AFFYa6iaukf;?mi=u*oWkqWu@l0)z781H)a`4VlKO8tmfa`2MJ$he+;bTh-1Yis z7Yx~^a8-Hv-3$KD7+3gTx?#lsW3R64f20;>|Ec8)!z7ipT%)OlvICmzr#r2n8m^3L zxPSz4)ODzi2U@CT~^U848GLWHVOqLKWV^gWK zL?@-nuc@+Uv&(5?u(e9PTSnAmD-D_kaSf{s7oaL$Hox2>*5Py0knQhqV4ac)PY?go zup^e4cCs9-%EWq3d$lf5W)R03E%>5*j%Y>M+%+v^H!e1f=wVs#M2K>97pve!K7&9Y zZ9O?RP1!yx8K)db(rQrK!!ghzrKSKh$gPFxg=?jSk~On$0q~RsB{g1YCzP-#+cAMP z`NdtTMivt&och(wl2upaCC<8X|J$-(QaeAP1a>>AvOGVdl=1(su=5826Uv-DyC;xiOGp5Xf`+D4Nk{+@NN5%Up?MXhh$2W!Xd)Ow!iQdI zUhY-7kc1S#tAvE!m0|%z_|~)bMDY84Uj2tS`^=tQ-m;#xp7lKGB(vN~C^DvTiJ3Sa z{#A53gumO&rH?7X*2yR~psy^u4+RIih4zuI>{X*PUgr}sYJu1kWB||8!2sBV)}wP+ zwY`<4n@Y8$S3}-8LU75NVBI$CP7HzV=h*u)rN|8;jB@X8y1G}^Vs0fvzC2rRQ7l)8 zevdhf6|spaDsK+=x1T*#B4E6YC2c?bGn zw>lcPs?&}Pm#d;VCA-dU=i!0qXxhv#j0<23m?voe!f5AQXe!ftNf$Ag9YUu;%m&dO zJ8@Gq04(PqTeDfRHaT!*Eu?X{Ow=h*XosV;q&E`RQOn?s*wA6oX#LeJ4#<-Puh1E7 z8}FDb?K0L8hguFH<59wq@YoISHENv$O^-1Di4J{jg=>IsunD2ZrfuS%c7u?~D8vtv z<$qV;%14KFn4r+pJyni{qcOoiKbhgO5F~s^gfq!hf2pAURD~(?W=Gh7WwYT!HH}?~ zK!vYJ)DT@5CK=91rRa@_mVzX2%IZk5nb32DAsU%@SB<0(|Vx6F+m(9P*QgGB5;8oXM75bpA} z8K}OdOYZBOl*LyoR{ZzlMNXO9$>pAmf;t3GVk{1|j0FrGAUruohmr+~oxn$}3OAC) zpDnXZU2;DNqp0P%JymZ-jjg^nQe4|3$2utc6L#ymriJMqLQd_$G>JBK+svX<9U6V- z>)LTlKh18k$2xhkm#fQ#;EvftIlV-#q}4muQ~Z@fP3#61G-;wmjOREv!%P!|G+w8D z^6maY%B`N9uhS*OPt)FiPhWL7pg`($DXkmCSQy3-wF^FPpw`ifX-dmWs}#Ojr^|L2 zehOKPZxn*r+4eMhj^_#dpJn*$fDLoLOJ|~qPwS%WN4J|p(QQCuGxxc%@T&wkZ*8bW zbz%RZmrnScV>sO+$}h{TaR=@rUUn2B9vG(;BrM6rd8FhidZdFusOlqj)tffx=g&F9 zx}2tG)I-R}uTzC&@Z(UaC2I3tG7`8rz`;^oovhXN^$mP&bL zgPU}J>iUsJf#lP6*R|WbqXu4kJ^PA7C*z^G-%$5ltq81#5M-rj{8$)|GJw8;X+*6Y z))YnZ&RMOsgZNoTcOJDEJix&5PVOVz)%1fKZ8n()VKJaNB-$2tobF7Aj^EA@O-)S| z_E1b%_ke!HO=||)Ax%zNLGEEC*@vNgLVK`!m&S2^JN;IEy{*eBEa69s*l_T$#YNeB zn`2>z?e43PFtfeAH2J48OsQu0?XYtxoiZ5KIA4ChAqz*v^s;^muP|8V6_lzmA;<6@ zo#U#0=PUH+ms=Prz1Ty8UbM>7{Cf&cXY(yxI^#Jbms1EJY8tv=;14@YgMW*I3s+wp z*E2*Lf%@h|LvB?pw$L)w`MtD|(fZSY7VYkedz+3PP*2dZx7lfTvPFbnUxEFj@P0|5 z`k=6P?WcPK1yx*VXoX#dw-M>n_bv8^JFu@E#@7Npj5 zhNh0R1n9|ry1&6XMN@?s7ra{mXU(3f=qx*%ru{1Dj-}ybjipw&^sb%XXzsa~}QiYH4rP z8}jI?S*??nLc{saM6Q2D&q&dW7ItSGpyh{vO!DP`A_weNxaV@2laM%orkNf5E@kDD2{rfPS0>2fv^*LKsYC?CV#>hsUo?wtIM-A z79K8oiZEu8E`k6ajr&t2W0_eKxXX2_4RdKj-R4%j$;0w`hjTS{Pdf_w*h)y{Z#lT2D- zX-7xj;#7<;IW#(r*_>!}NbT;?=Nd&N%&x-xP@d|7b&Zn2 z=vE4A?j2>G`fRP}tQ<;;cte+#-MuNVwwsKx+1ZcoLSIT0D>KiR#@7PbDGu7lQCRTfA3@Kw5;eJx%wiG#;tvj9ud1oIDo3JyBR^NpL zduO>-?_;~l%?-CE?cK4L>`KG>Q@bISDyQQ`4N;@noJ1Hd3OY-AyCt!iP|aA+EW(0X zc*lZAmVfGNQ@aB41$g8Cup%Kd8dnweUGyhV+W<51kXu>7gL#UC3;7GMWL~_0OJb{8 z*5jZ%j{c`5u_XLgQuE=5OJZ^U4<)fq=K~ZMp=(H|MXV@^!OnU@R_tDrgM>Z2K2KXW z>%@1Z<~Pzn%QOD2-l0A@7E=26n%@nfB6WmOIHfcmtR8FV^LWrnUMc0Wo zT`Xkbo?xz53(i|bkEm0iM0VmVH?w7gu+R!|8?yBaK$%+19bS-fk_#Z?HWCJlD0EEO z0eN{L?Ve2)wTwxH^>`WuNNKoSKvcBiuF#KDHjfx==3ahU=P(Lx)?_JXt9KABR{4_!i6=7I?~DXF&#YE zEwU?hTJGSJ2VIj|ROXs}D~gut+=v2uAU}JOzLB9Qm8{Yvd*e*-Fq)>P9BJZf8$6(CuPFd3`4L3%%uSLEs8a=6QTKLd6}r?^95_&d$cYQ zt~O-RbHj}Im6||Rr4!PoUklIVIv(u-Xnn~SvP0;D7Jch%9o85OwDa4vF0-_rmT|^s zJOf_97{!Ze34hFyCu=-Jmg-{>D{(fkW20F-zS%`Spq|{O!P1AO-}b@iDaiR+t(2l| zKYbTUt*`2I+M_IkwuDM61r~h+Io~N}@iE+n(QYhH5*41RQ;{CA# z>IjU3C>myW^sPnXL%PgHM#SE+Lcb4l=lmQEIWp+c(qD5gBz&%)q-m5RnM=ct#uu#v z^XxKR8$6N?(>`Bk$$<1A^pFT{6xdyucGrP8?3^lPvrOZ}I*3C199@j^unL{ZY6j!E zX`>$OV$u<)0XpaM*lM8{rxyJmXs_7cGmBy!lgkdV#}RHljRGZhrArH~gB!(!PX1|~yYK$k-wv}+)C{|jyc`OjqG<1Rg2>6Mo}Yr5 zl`XK!-Y05)MA5GU6>CCqjp#LYN|@52dqYNRny4o}jhnb+8S*o|u>0nML6vOnnBemVZ>P&VP0a6MG0qaYz6h7CnVQ(=az;2H^^eF)QnnO5!*bv7CVsMvYV{m zXgE2`5PC~JIUklyP&9`Hg5>4QY9*cPXcM4Ss*%wJU2B`Q@RfpmK);BLJ1v1*hbSn4 zTiSn{U4QfzhKqb_fL|+xGZd5=a+u2(T^UsR*W^mczJ>yeXhqGC_YCvYdsX*oK`7a0 zdGz^KX^O(crMed$a6y&8a>f?2ILy1K<=y!hX5^Nv#qt}nqN>^~NuX|UYy0b}Gr9ABC z9YuqMkVkwxA=5_{mlc*yWHYRnJ$3#t6h12Duk$p5i*o+bL{lu|=!dXHGp4b74gKvW zTt5f5;3QG#n0?VgixUWu>I67=U2PRER7-XF(hlk*Y5VyJj%n|y6?V*PEtwY3!lP&N znKF#Ng!iI{|6nT19z(r6vC@i1*I8mJ3KGufh!3imI;2`{Wf{dpe_{M_Y>%H`>%&-! zbtyuuz2BmpCU~m19pxv{TvAe;v+Wlb{W$qnF);SM48FR_9u+Wpe_$st{SiKRlC z@m`wRIzcqji_Lb5Ex8u0?%n1=mJ+0cfhM#Q;WKb!4~Fv;v*evSEcc#>9)%aZcADM2 zgKm$oLT4+W5KsQ7mqXR-kdZg!pUWKW0gBzjmInE!%M@~52W{^)tnT4G?@5-TbNf1q+X@by#~oQ?ht%{^Q_Yxc z%Anfqs`nnY3Qn~1Y%lvY!DnTgN^pLjUUo2ObdJ3nb3Qg%>g|4M-Qm*3v zBfE;9-sGLtuMDX_I|P81B}@8>A{$(jV?t3gFMz6iQC4G(#OS#~MiI&#w@zL4A_U|K zVF7kESHA#_`DURpe-IT(EKZ^saE?1?WmcKiR5<1knEfMYc5k z7XYmjdCs0-FTnHYZ3}ZGJ)unfHy1&o*FW zE-3Yc-P#z6ZYgwRyZna@=p`fg6$XXZs6HVA`ongJY7&!-SfD+JaA5?6nUr=(3YkDS zBXGrttyt0E&k%PYYjGJkjSloM0zj2c!7podIMcpGmbh!H-GL_=exc0LyQuwvNQh1} zy&VK)>9sMsP$Gav5KP2K6`^ zA$n-?4+cs(=b~wXaI~K<6J-J>%jk-k#@&dQi*^h*_8;ICqx$!&*3ty@cVO$*ycSK;naYA@i4X zG+fj=Hv;6^e)^op1z)W_E@y*we2OPA^pSg;l%vYK#yVEe(YYR6U< zdALE*h(T}sCBfTFl*=#Z{wTKxE_B*ab{oB-{45QbNE!k|IYq8@!#znl4$XWTjk;XR zdg(4?9w-8y4{gWq+wClWTpHqXYg^l8L_ZoMRT0h9y0YTeyVYhu*gc(OVeZ8Qvboc` z7Hnamnr7!TPadqgqMB-EpV8?csqPsW~wR;_HP>klQ|i%oTrS*k-OG8x^cY&drt+@^K2 zTmAWfb)`)wgbj-Nj#>5KAHLQG{*3s~|1xOEy@ojh7t#w%X0rfxZ1wudcH>Q!B=2TK!00Bmdl^$gR>}{f$7?U;J1$Mr|+=b=t?>tca>mu4U{~GTHwe(3{R6NBe7)|7r{=IJQ{Ct)qq!MA)?Rp2(w^=5$Ju} z6wJ?gPwDDqy*w0BP?;Cqj&D}l3H+#?D{Pvk1njO6a(%kof`S2$|ta4$iULN*-(m`zv4-Y)~gsYFY-lgn*{sGz3O-e zn*klUP8jI(EqgtlI1Ca(_3T?_@wxnKb(>n=O06(ya^9n#ncg(Sek>d4{YO=@Dxr0( z(~i@BClobBRUqxVg}cx56@7mCT7~J6$dV@cN=@~8q8={Q$KtdD&ix+;3Wdu)xuC|5 z2B%P{4b7Ua0#QH#^5<1S&?pX^#feDT4ZW}-xy{{?E$$NrTboHN(%oB=zr?qkbH@AW zeT12QYk5a7oN?|kn6=-Cy8{R1{QG^twd56@56Rz`k++(uiQ)7tUF}^iLEn^K zth0&<0Wi|OZ#&`4o8V`(C#%q|v=a1l--mXodVG=TC%+l$<9_%E{ylF1=2`}ful5}< zgk8g0R_5Mg+911nS9nL+4~K&<7`Erp>ExwLc@t;thEHQlFv^voVA69<{i60^YuF6JUZkj zn|q<=%k;`TqA+s~i`3I|N+y}m%&$Q1YNhVcBgdLb!Ixn_?V&Ese<0rHxm2B{`;}M7EwlT66Ky&|GAA1sCTR*5fRT7ZxCG;cMYPaa@i& z{{m$NAC^hX!hdjF{=nFaZY?SrwUVyka*Tc>L@A&Eau5F*!g&_p^q<+urpQVIbP&OwbS)?;t5p8)>-} zkHTxSh(a@3bK=j@mmoz?lkje%1j|oW`fJ3u6Vga&giEfcZMA z+Y!XFoW83oNxPmqy_{H!s29Ie|FShGOM4JlsQcS~gE29CTPWLE^*ib4mfNXdT z&ek*D_Q7Z&ZMe4vg-io7G_VybKkK14(~1d{J?0pKw%(%)erT{&!tD!-z;TYBBZi5r zZWQWb0Y6)ZP+~i$TcVf09!ZZ!om$5D6%!WF+E7=*>y-kJgJ59etS+~6!bUoJ;iiHL z7}Hb>U_~|AB-l{%bonv|I2GDdBO6#0Ll5On=&;N$Fqgd2p$OXkh)b{m{dP4E1x-|S z)VUQ_<^hpOt?Q{kwhK{`y{v||vYH`3g{qcSKAvAajI^(^>Bu{#X}_ZLuRiDb6hfEk z!JIN3U>EG&i%y;)9Z?TdTT&SB)gr?#ae@goR7_DJW!cFoY)U zpW>7kng(sLg_-IeqG$*kN;WaM7v-c)Txao4aM*&K77(~t&MBMpx`yQ!*as*+!lDyF z3BI$EWqr`LK8{=b?=#vdxI4A1ST?*r=wM>eV3|$OIyhD8>bFkSd)I_+iy8A}7M z+r7yl6ous8Q|Qa|=qVkDbFH%|AnV^o^9}+E$>!NyY~Dk(6N=#t=iUJ>WTtVEZEnQx znST0Uodp3Q&lCL|4Ur}1tp0Ac7`kuB79T%U%Lutm6VCBNem130&Otv}fZqJseH1s* zx|SiQf+E;7CaO&!>KHZbmeG&dm70&b;A$@LuTUJs_bprIXcK*6bE9($V zLHcuf=Wv(b?ftpp=W8sQiSJNgD|?JuUUa^C3wzX0w36G<)w%tK+SMP2SirtjYmbJv zYPxu%Tr_0T(!aXjCFpSt?V!O&U=QvTApo?S7K30osQdJ9L%zu+_D>s<;(zfX8zH9~ zkt>CK(yfIUY>0eey12Jwzwfux&L1mg=>#Rt1g)QuA z?^xQ2j;!m*k|8pOr=&BHm2l>Su{d<~I4N^BDWqUvIPk}-+xsM7M)+-7HI5vFX6E&@ zW(Br-rN1(khA9}Fg>(S;e@X(YJt?_3lM<#eU^?JX%+1y%&SgsdxJlT4u+qwrI1XQ- zh0mKvCqk}Re!0PVvfnY)Q@vQ`A#J3qxdyko?ldBtQjfBSOu3o2c(kD2=&JK8!hUX; zP7&f1Ls~b*;QeAiO?R<6w!gUYO)Q?`(xyXxly-KgV3`q`W zLr*)3-~skmlQY-p=vJu?u8MlHPuWl91k-t6@s2cGla(Dng22Ldea#TlouXDM+Mo|t z?Jw5pCuld2Cp5OL#bN?zm+0qRW2sXnn$B4}a@ajKMLp|rA1VXpb1Ph1ji54msiIB# zT>xFQuV;;7n*@RLeTbAeN7dUMmREjZxC*PWhb@WZQ72x})WJMdvQ)lcJo7KsT7CyJ zXODt0k>&K~E1EO}4jAxbB!`yv5nQyu$9=vG3E@+E+bL?SVlvZ>C~|Ud*J|<4U~vN# zlr;8t<*~sgN-#d0%Ee-*2{cU`6u}M#=A%ARNE>LiJ9kI=*6o@}TM5IQV<*V9SO@8; z`tke96yGeYy&cmv8urz8VWxDZr>vH+zdJ%3Guj01Gb(Ul#6G^eum^S? zOCgv=gUR03KX!N(!u*g8O(Ujhd|@}HxW>s-Qmv$wF*;59(GB$We%B}#?ou;brO9Ou zRI<(n9olfNwTqPoH5NpiG|JVWXQPnFT1k zd&=q~fi$KILSAW-E*U5Yi0Bu)W5FXoKI8%}SFtiZ(ADV#vRLA90l~3b4_bUQSGyK1 z0OV>edT|}tg;9a7Rxe!tW0bNWhIs2ii;B)N`hNvuk$H%V_|yljUii-mh}pkDUQzP@ zYH}R)L4hm#0D7fKCcc0IR`>zIZ~94zdLd5jFJ|5`X{BMgpDGKP;}6WptF?opNPp&=U{_n_#$I|vTg{6>!|p^aXshm zF*@rW%p`9{r&Bn49Mce@J;a=JR2=%01pWA!Ql*(cLYKjnq-Uj{d=e(bN}!Z<6M9^G zrMQ`867yOnSwHzq&Eop{Zb(o(av(n83qR#?E66|P<)U5?de~%ZB`^Z z?Nm^`$0|iE~cL!KqxSuXL7g{ z&eEWbKq1M_6cnekH42!P>g+Fnij?nL$gR+wTx-J!rb_6pmVKY~JwAZeXXzfTlFR8A z2{FKGXe=9re>HkM3*6;SGTBEO?C6Zc-0X=0$pjIk`?y7DFxeA6NlXwObg_RyOBYac zF(Tw|J!h z%3QQEipdaQ@RR!t-P!E0Z${@ddMSG|*X=BizQ}TKu2Y4e89HZSw>r9=Zi?|t&)Zaz zv{lHHUHXE{Zn{PzJYO%2ui*3ju355a`9wClmKLfn@I*UTpOB|_=qP>o;a@3uh^?;6 zUK3;g7Y1!Cv?Oko2GR%RW_S*QX#uC+jQ-vf1&y?suG0Jt;Z7S4bo9A>uFfIZ6A)q{ zG+pmp48Fr=FhA?swqO6MS4)>O=`tZq!Y3sBQ-M{@EiMSkH* z{Kgr}qTo!Op(y<7|5QO5uaR90G9^7J_nQ?~v!(Djb)d}?A%8#%@lMU`UI=Y)CqsmM zdZ?Hw5r;Id<|B9SxK1I+*NC4)rAUCw!hTqwAVm})>y7kbWIrjKfNafWMZIo~#V4d2 zt03Os49^fEH+*=aQ)#6gy6sQ6!v0f*E+50zk3MIN7li>whLwpx8N4}vuUmx#G&q{@ zEd#+-eXCM+FEZ7GmSk=pG|hgm<6c@Yb5>VT4y}kLmSIFvz}W3)YoYKoRplJEq4S35n4g#mkPPbfAT+_C-OIEoh+*SY~~D@uoSFrm7>+(8Ue=X480Qg^zmI<^=J zD@gnp&Y>0;aqcqY%LWwsY1X89O;>+5E8Zu|YG);!8(sc^33+yyi6Eu?ok78Z_Kh&0 z-=?$3|ETbCyQfy`oKjdU(FPa?Aa?G{P@LYd*X&o51_gH`7n+GTI^B(LIGEMPyBy@* zuCxpO*U>$-jsgeC?KVrZ#n47>lLiaA#%_W)3bjfD(yX-NV4N=4y~TOGQrKUs|EgA- z@lVU@-9b^0Zzgn@i5^4#bpVkb+83d+fetzP>R6hwM`CoLQKVr|LG7`>?9i8vv<$&6 z2R4&S4yp?C?g9E7@+tcb6v`kpn@8YRQ10y>*na%B)k|I{0UrVyl4*(%6gt5V)#bh5 zrP*70i%;RUbsdW*w!)nL@sK6#@2Rmg&CqdlV5Q3^o03vU3g^3xh0{ZtV#D`yaNS%- z?N1BkLv^~DcpfePo!jL$o9&F&kV%8ma&SY6mcJ}5h=84m73I)MI(ZO|zsQ8m?A_KB z6f;vzz^w%hUwR|@_cBu|cdW=p+3k__@HJZhlbHk^SHmmLh6CAw(AV|+wv0tpmJ3mSwI2p(SPPS^Lg3)A z?TU7@XuOU9iUKuyR(sO#-`?A8dxS@YI+o@xVvUrri#(f|DPDU-J`Rgu`+C4?DBrDM zlS%JPPu>PC0SkLWdT7-UpY~YeI=IDEmLr@~d-+7_30a{fUf$S?JR2O9r%k4XZmKAI z7nyP;3db$9RQoae?=0mV5o7-tieOxHp#=oMS$pW4;|m$1V73ND$9i-LetLYd4{*~f z0g4F&6vI0=33X)EfD?{K2i`|w6rvtEIK@+8##2Xk76JYF)>xLoj9tB^SG`H&ReMA< za$ol|I2Iqyj_a@KCGVFzi8nY?kUcH=m|8fgQf{&b44vEVQLlO?xar_JpGhJQ%tglkO#1wbx(DC_rjRL$c_Zaj7tF3^AkZ65k zr;Fe}4MSaAsc;XCdVg18AFzv`PwY44d{bjvo07M5=b~BKrX7+G6$EN8_Su<&Ui(Xg zr|1`38Em(fHH1A2`u4Cj`=W`#wT104X$)URGqW?*E)J^DuP~94f)v`2+Jk<+L%#_N znerZ@C%da3RlVCaKOY%yqN#s(!8Rrkp}BqtUC{XnJ!5$%m~7HJp2Yc=>oHf3veenb z!hYAJgB^BJT5eZ|MwOJOL9_r?w&6-w7e&t`=6T#Xx+Krt1?NmGoHIhS7gV>|HvQL5 z>eY@ge_;?-%kp%|V3&)?7Ry{0TV1q=u}0!?Q&+!ONF^><2QDHzjNS6iq8HFsHuWTQ zg6;mZH=bK}#q?u;_T~%guSmW&w0i!!tEYeUo$!TqS0~3c!{^ssG4reczUaC2%lX6g z%V&Nq|Kt3BF8C4r?}8se=fi>@qyHdkpL|I0Bg-@9B2I|X58F4lp!Hmfe^9iH(Z7AL zh9+fEcb`KQi~t}*u8}B(vi-I|o4HW=9%8pR1{VpU*T~7IT~~`Fp%^FO9`FL?-5@m3 zSoyUez-L<7G%*>yzXi?t1Yi&Kzv;)3o@tNrz2w>;lnjHI3GaWunj7cw~y+V-{N2CViD4M|RdY~)l{ z`5awvj=F)q#{_BTXgLGbW{0A!fDKe8#W0=0L1%x1$;}xWaRPp(bry`GgJ!9yfzbn* z0-J2iK5Mqd#|)TE;K*>cKO`IobHONLHpDQV=rZ<8Q=n9XFuR^j6*PQzRW{l6-bA`&mGnd7B{EDveG}`#HUfq8SQ~yxiRB{DQ*9RT_&`749eA zDvj`msE2!LV>z2sx0B>aStelGLqLH$ot~l}3kw$3B}O4)OL+KxinF>5$xS7H^mN** zpJtTZgzqW#lm`8yJqnotg45GQnPBF0!R7;?8h%P^7pe2RQU5 z>0?kU)t*`fMN1l2L4Y8!o_(%N$GCa<6-b3gSg5`Sp};K;je$MHj?C&;pDUx@+eSwu zI_lAggcvG5n;!3}thIjwvIGF&kW5uVLC%*pLBH&5+TG6aI2s2h{WPfOqaEp@ ziUGo3VT=yHHADqN`FIqCOuO4kLa#ek!d~hfnun1%_;Ut(K$KmIrRyHy>`b)tFcYRl z!*yM$RG#6g!!4Q|7oex=Bx#5jHSHR~Dw^>HCbx8sbVm0p(+I(EY$Pu0{=r`AFD47G zsu?Z~@*~Y|1mzuGt;HLJdz)5|<{I=h<}fbNy5#pMVY;U#cpN*CU5G$+UG^Lo8SxuM z$-b~Mmb17-E!nRSNt;}ei$bNc z9#g=OM?b02CGO4I9^R)k`7qOb?$p$bnpx`<+C42a3D&Z{qHAyp?5?ucbX~0BrlWl{ zH7r>l?GOnDXC4tr+UuNCyf zHbZ=DkRbZL1``dl#VmtqJxXCnxPxibLIPa`^bfa$H|F}C=XHY9k*&(oP<6f1j9=8H z>3UNIwEC97=daPV%V;IE(n1H+QN0B#c!^#aj>}R^W5ciX=X5RedvvkdD+=3m9O&#Hhkzn*esXp0b zs*QA~`P!1$&vhdq-s=-wVzE2l7%f`j6p~rB!Az$ouV^NEbH6_b_ z5g2$0+Gt2STq)%DK_{pMvmG1_HNs38+t?&p$=hX|I;0}3we&#d(M^5B!%ej!x|DB|W@sEnEp~daic)a!XxdmErJPm+Rx*4SBRi2Wt73Org;i-y-zuIYljhcdIyPO*7h^cmo9_d8R21 zyp+x{6nRgi$2pjIgm0$Ngk3)bVmp$%2|bg1%qm&`W2*gqvlcwgWIpex?WNS*b)~_~ z(f!UDRYTs&eq_heSffAP&_}OsL!UIHvgr|ANf7v>CsBNH(m=^e>EtGqJCw0pTi{x=4Qs=_SvDF8;>xq_gJE$T_C;bVucjUK3XOGWKI zBCJzfKQz&Qwj;e$3e!$BX~4NZ6xMONAD!82Dns|Y_8?x%ypm~~AQK(>%COWLN?TJ> zF=_}Qjr2W`7P5W}-XTT4-j{W?my#xwx>tCHP5*Z)Z9^Y-)H78fzt$@@fZTVyuFC`Z zRSMS>n-JkGPD%Dp#8gROg|NiKtD=3=d%j0bHQlR(wn|~Ru2zP8vT&h~OsV#+h{3FC z`pHcMtFK{JNO_q?s}COH)`|4w)iRp4`?#1ck5J6Fwj}o-p=UT4oB-~|l{A-{{HVOU z22{lqa`rf}j$ryq&u|7@`ms{8vq{P`6h=~dB?0x$I_!zEIwi?GT@ARUIus3HpX7J; zbRb1emw7c!q;SA(rw|iY-|^KD?5PG_VLwbE9X{J@7v-*WZ)Vz$>B+WG@d$svNKQ%&!kXwmJrWpV6UV^oWd zc9WrfYZ8uP3uC_)@*&QoJDZeM`o}$^qCCFGtTS37d2!;OK9sYo=}7?deW2cd3%xkGy`Z$pL*psXvpKFC>Jm?%8;PGm;q29Z3jKF zy|qAkM$pw1=mh7aYr$heMgXqZ`lBLjiEI%RIpBiWEW|qmphF2s{C+M zrd?JT?wIlPJ#8zPrE~mRZT!3dZwKBcY3Fv?|o4-!HULc>%13z~qfc&L|%t-C5 zz@KFk@jSq+Z#{{-iMR6=%W}ovX`xf>roJ4MX5NMBmfnn_v%%4Y7DXnL|I4~)tw)Xn z&*Y&KM?L$IdA{{`PqD1ei|6*HQqj?Dcx2*&TCA^0=@QyLIG%=mLqE`EC4fOt0000W z04Dv^#U!Qg+HF67v;TZJ0B`^d^c?i)tgHiV?IM7JY8_E6i1BNbiI>v^yMg}fQijV-n{s@SwzaK{zXaGQvLtp@afBdNL&LXBg zQAo>3OExn@(2UbbNZU?IO^cU5{dBNxrzXd#_xAMkd_8H+m|=}Bqk+~Sv(nkpT>8;N z(mUNV43SWG9J6nLVC1AL`iR&=HrBgRQ&Xd4)w$FgE>msi=~~!xc>_HL7&$_Rk(1Ln zLWmCVU*B?;+?FAm`HqBFBEpZnf!z#6 z(1X`Jzt_Bhxbas`yb=dFhM0cS660KB1ZxwM0eHm8z!4kF)MLOv$9Nd2C%c{~zmb}> zBg`YrngjF`QZ!QI8nXSOl++L(NA$$~M1%c4)ayQ2@YFu#3?21^jM20t1@(+x(0N!T zQnNhcGUF<91Kon;BnyKI6+JyPYok6$#*!8ihGw6o>4CMi`RQd0QA~CE`k~_DL(*bY zfg?p$1g(q_a(GB~1^>;s*SvX@=JENA1pVgMytM(O&b8MBMcOY*n=y}@04gfzLBLq? zI=sM9TeD(dYJXWE;YEFiwa>3$}F*<3<+i5ED z37=CL{~xOa<3a7_YDfWtkON*=sdhUnPnnY;Nsilx56|>Zq)Xq|L6zrLbfU-;(?UE`tev^avUTq;QE9tpNwn@pVamnFsdZbH#&IAVRZ1j z8;2ODlALLkl1wLXX0mRdV|^FyWJ}P$6!b}xo#qN006^gLU;WPtYX9#Fx}a)qp(%pO z&E$OE)v;a~|BP!KiJb9G=zhKP0%>fZsY0+Jrh&E)qHX*_?w13-bJV-Fh;W z$#`-+gUR@Zf_K^dy6c5qSBoO^98+U_^kPynRDG)v+@L%lRSQGgYf!h=) zBcURx2R?bvVCt|}dP)+E2HYHgbD~~5n79CxR9C8;0 z^~IW05U;0-uFo`@KV<&m>hydA_ID4uFAB+X5^P_vQi3esB8l0&`4Es3CPyRpLdgq= zA2G2(`><=dXW|ZTFDh?*ML#$h-1dmAHY42_P&Z6J+NlIvo$8dW?LV!KSzApo2}_bL ze~chsl-zaEerqC!T?Hf~Ti>cgLL6QJX^3M!Cjq!aN+{ka3$6v=06UH>HV$*Z0#I(}e z&Vc~KxDP)PcwKj**BNynZM=o>QZouf}o?$3PtTvGam~nm2W7iiGCmmgb;wBj3^P%DiRJ(qma3U=qf3iT!HL7+9`=Es!F>8 zhv7iwzHcBVO(>ObS>!iqPN6SY+}!31C23Wm5dEY>J&6>z49zTAD8mh0xehf$96Xk4 z9LVP(>)O$&Vn&ZY8NA}J{28KbE@4QBWyXARsC6Z{8L9#tfTRS~$SF2m?{L^m60ql) zbnd(5d*r+4d*-|2OVuOop5q#G)ibB>u5KX{Zb@nt#=z*{>!&I%G%g3BLhM(M+`kYa zP>e-tZ;&Odv>~i}qaXSSPo)F~g0CWW4M}-EC z1cS*H1mb9BAUGZZJ&eCe@vVjf0M7P%^Y!~ramn;6w&8ReXD&s>BXV{i1 z>RyoQIP98Mbo0(V=S-4&A+GbLztdW;%8KoSHmc@J*XvR7Nc3>IVAQpn&9EoD*FA{! z5>2qa?&ynTRh#A-J&;V)* zB+qXYwG#T(1AL(`r0X;@F^8fVi7uMk{YW>GXhw>{iaxMl6OV8HJS-waF!4je{m~@f z*m1Nkhx^TuKF+opP7T{+@|ew^V+LG9H)0=wLz79glODhMSYN)SWV5N-c3acjVY-O# zgYPfapaeFQvBKW>O8LLxR7-&^dA{%IaCU0J>`L30TIv&&oE8slcm`3O+?3HMDyx~X z?q59P z?sh-lF|f9^V4fKgqN%>ERfgVPcctC?<*5N*?oliMyQ>PDzEk;ggIdIPTsf4rB)Het zz-a4Mf2*gB3XmegwL2R-j#o`Y)KDSRP&PQ1)&6ou8Q|Q@uv76~ciA|HSZzkp`}p=y zJ{(4gsQ6&9tPtdn+b^zhg$ATZgmtk*3>%{-0GL1Bi>-MjZ$pd%UK{XHlMwV~%}0R% ziWJ5RR5)VJk-bxR7##H%2LeJ{k>*H@L?xrJVf1!cEId)E3ILO>zD=`y_of0l8k$T( ze~13BTrJ>%Z%OyZT>bs;<7%3JakcV)6jv_~tc}k`q;hd{snM}lVu%;l`5tad3cHF9 zr(%I5@1hqLwHOxd7ofE7q3@9;9q#QH7ud14f&~wNgSWJ`(c?^~yaHj_z$Et@jRE`x z1o5lCc>Vfgvi|-zlQj*D$Lp2s9@!O`8^!m9XQc}Dx(3~h2x<8Hq7}D8@m4lcXbpMA_gPPv5cz;o5qVdH|yF zD)_{Y2%_w`Y#79SS-(+h;FSb-FDoIp6mED?a?NSJw_VY;A@ z8~!uB6p*@xPe|YKGh^h*VR!;n1QaND48{LTMMHA7nj=900Oa8R1g(Ep^xvZO1iFR= ziqdTchjZK1EG>!zO`$nr!a#!OxRtXUUXr{7z|VjmDhZB1Gk;lWIt@=&_(3g#JTd)9 zc`xBf@kx6mW=PZJ2k{O7o5zRY+06Dp%->QtZoOHSUj3CMnC8O9bg*te#bvMkkWs(t zUGCs~R(Y^cmKJh44~IWz8x(JifstLd4<4;s4$fZtfyi* z?j$A+XxtJ20RWSl4ZsMK=?!!p>T7yr@-=CWrj|y7)W>$C zXpGuLS+|Xx*j;bjB&g05G*m+yBi(tjiRX2ivpPQVPrcR~e zX6DXpz5CgPds1}O%*Hx2R47BWmkUW*k8*F09IJOGsSKV+a19rz?JOgrV%mWZy)c?h z80=%r&U)3D09Ai)afDMNmQJ6gs9Zj})35bF(YUdN@G(Bw1Ny#jjM>91$>trD;XGcTCRQAJ9rfcPk||FYVPaDT~2kLk_3u3OMWE( zc`r6rr)8TQ@eJpf-XHv--1H z|AJR+62Co?&e-!d_w!+?7F}q5c@UNQd*fAFSyZ`$lXGJZUZJIFEQ*zK#0>C`AQ`au ztP~mLIA_HLKa<5vVbLO<^J3hG!^M07b&nD7rILTXm;-P)y8iG$e{cD`H2kk^VpbJUeJ z4*JBznMv@AK|@f2dAw4y*({6IIQh-0Vq|C&sB*WO!c6901AkHO z8P!_Mv(&S;+$39&0e@I%rRn5pmad4G^8IA-t8PP!lgSfCHoT_*%QDSBdKMf7GDRjK zMhj6VAPYC?>oH{l3;3pazeOI!L?%`o3;H~A+4rN!cckfEt=%z82#TEK%v6Xq7RYlK z*A9k#z@b-jWyentB$dDzZ-dfF;rKX?^qT2u6yE^dLuMrD&VmNaTqET+w62KH_?NLUW4U+Pt*R&8Vx;b~c9x%j~mDbaNOjEiddL zR^9rEboSZ0Id(A1mcF1))^SGnI;nl~_I4iZ6Ntmz620TrPH1jsi^dHZo)~y;? z=jBUkX1AXV*H@gh-zS%Fa(|Hb%6_nGB$Gg5Yd{D5s^?vPgI3?UVIx{Xdl8xL5OCTH zFb$w`2?y>qMU-RCjJw>W=C#+iR)fBh_OfFwEbynIDbr8iG`jjD;J2MTwon!L)3~1r7W)s z&qQGN>q?qq_G%BQ#)dJ|FAB-UU7xgv?-E@vABlqB`Dg)CKbX z3$e>3C9Wi?JVBzaHG|3watlRf3m9Rc&pYH{lr%37SBJL=&*5scA7hXH--kUD|H7XCEIIKv_VoGu#vU;`b{bT1 z^!oo6_L!FG`~>{3U>aGkR`KW5qyINk&&0pi6RSy>(0}WRwFbR=Qa;xdhO(~8jt~=% zfI{BlUW|enS~!&WP`?RfYuI;M(uh0_>N=coTQfW-?dvp-5&+_XHmKv8F|!Dt=ji7j z$__O1(kB&6NyeqT_*4{uZYZoF=Hqb{$pE!NI}A!NJDF;aWf2 zZo1JYY1?H&>Z}x|`+Hlf7)k7lpg~ZX!yV0qm zi|oj{`R$fC<}6wy%4$y{n3FG>rL0k3uS~N7#=O5DyQP^bAmAzb9n<3lzX1NSeMx}wzSGgXr zX48&FR{(g{_Y}nlTeo;Iqm*&cQem@Mo0EI=MI1FOYbik9uf61P3M8A~U|GJt+JoRRp;FAZ>?2Zuz&z zb?|vpA+iS&{Tbf~&_JK`GkMA|DEz=@2ndPzJyE|>$OV8#_6^#36nw~(>|ZB_WtFR- zn=uK|EPZv&E7?aI4b(l?82mJAXR1Uil_ys5J8r2KiW)Qqv5w(VQ@;;&YgvsCsZ6nwNN)F$%tJS1cW({&~$W_4%0p9+S6PxJ3s2egqFO(E+MwrnjAi0#NlDhtZ zddp7sFp?07#-3_Jw(6M1ZD9|3@$giB>KPH&eMtw0$$}dbMGk`j1OqTjAp#~Au$d_o z|C-|B-X*b4iIdpjkn;;%MbPqv>jnG()Gw3Drv)64g*IL^gq5HFZkp9eqPCtP5agTm z*FM@II=%5bP*6}{o$$LZt5TFJKsMr)zl(u!#xA`g5-%$!9HX7=@;@(?{hf# zhazc4b>3q4trxmq=%bB7NsGtX*i3f$dYJ>r9iG;r;!<00r-G^Gll_Q6b z;Kj?v(-Un8Lk5%JB-ykFaFDFgZRI23#az||h>%d1)YVHUj7(1#P1DNYd(q~dL6$j? zorPV5LV|utUfW78BITspB85yQO>kOk44 zkLh*`2=l8fRZ4$dDU-t6s<-`crF(z*QU88pHHPT$oN-Kx=@;o0WvSHYg9j7XPBf$H zA)x>Lp=jctTDeA0kVu|wA@)wm(TofK#VmiQRUy?%xsc=O(h>A!^G7lwE+9Nw0?F6u z+ko}*r^qyq?YGXg5{*SlKm)?sHXv2?Jd>KhIR2Yt4zJ5AbMJ01tK-`ti4XBDwk$mJ z@PqI0kx`iY%l9)3Sx_ePB{V3e^H4Vdb-4U3vEPAM3&?UdjUp;9Aus1&GhrERbgMu{ zO&E`S7{OnRZW7%v@kB|-=*=|5Ad+*JsQW`2VX(9}No?&N!C^mT$5f;>YxXhv)ZYBD z9%0;mutJb4vQ3}406AiblXRQz_zeG83uJOm-42jm@Yy4>h~l{hPV+?F*&kohdW0ij zgr6tWL=9%Cy>M?455AS;H@y)JY!nJLF@UG&Zy~K27gY5JBo-WXp^PYC1Z{w-196dk zM^0jZHtTbNf<(p{<{sHL1mpU_L9bsV3q4u;$N>QS(|)vM52xP-%6%NEQk*4q)MAqC zF(+2j=!!k(EOv#spc1Pt<{lI{72Sr!9Zha(6XK|fEW_TTzt_t=iu4o`!nkH=k{nrC zSdTKxc(g*+qK9*Afv<6-NGL=2)_o#Xp|{@o3B|NwviQ*+bc(>E$2_7rwwu=PyHzR3 z(!NTCBAh@+Jk$LUL7 zJiZ-=!-m?ao8gVyLKWlPGT|m980uaUImt@R)mUcDuWE;Zt|iD-=UU=>;V-9~mcA3* zjP}QGLQkFt@(o)&)jSx{gI zwuXqQnkeA|kR{`x5asbJC&yHUkJLvHAAF;#pK4lT%V{X!9%5xez zi=QUUQDo{M%A`|LY8T#F&jcuLEjfE!-3_F_nXg1IY2cm>0Zr*%29Hl24kma_Uhted z-dUk)l%M*{xs{_|yM>n@2SjtX{6t}sl~IZvzn`3PiV~mR)6#xC71!F|mcbtsUi9<1u3=g`m|Xnb-3A)r<7pnp&_$d(_x6BIbP z7oH5HInQ=7J~3bS+S^$go$^55-Y=bbcnf*hT{E^eff+qX@uGt{?Op)Ch*~GQNWWug7DW%f|~|I*(nMrS@AV8bz=~Xm%=Ab z3C7Bd$OMf}`HVV$YKrq7w4f!cchj#hi3eByL3#bZO({7>3?N08-o@tKZhz2XVHVLeZ(0v`l z*5MIZWb#;P9)rP2p`uKUR#8&ZPF#SsEKNwHx>Pf18k@k4g<@bKbhScFVn;v=Cq>wJJYS{+NBsJ_H~MbJ>@u1#O_H-=;4(kW)z`_YU? z*$eaj73S9cGZ66tj7LAk6U}H>3$$aIq2RP^s4Ma*ZgJMaN`~NNXuoSdjp|>A`{@Z8_D^gw>FTvuP;$dv3I0A=zc zbtG!3lQBDRg-{pyww29tY!xqeUAE<0RN!H!YxKRgt0A< z@}*D~vKdTxA~X+YCHZ~)K;s~rj%xIas7dpWsWC?Qt z%!1tJnQw3!EtwpNHom=RNWASG2~sNHAWqvnW)s< zt2(ujTr0+q5cRnAvs=-%QbZxlX@{f#*lofVC*J7+LvS0|kCem-SiAE{xXKCOyulcU z#ggVfPyK-jGyd4avz%{u{C*R{Wp*;5(d_kOu_qKIJY?P-4!P0EeaPi_GxM~;Uq850 zjG?e`b#f`w$DO7TAe-Lzz-Z!VO<|j!0zhLWtT&!In!3ieC!1Ip*5}|FMNc z@x5=6r)joM*WHrL$6rL8{?@_hKj#7VzmEqZ|HT7d|8YDp5Rpp5{Y$?3Ydo$XX>lR{ zk^cYh(?z8zbVk6xv@l!p601M=?XmyOJP`R$ovrmBJP@yXJA19`hfIJn=J46wB})j( zzXcdMfI}Mbf3F7bL-ny%U5{C&1KPXnVeaKG?Zz{-YmCX<2y0Ooo>#f&qNB{ZZsM#p zJL?lpB2C~GhLwmUTR<|Oq>iu+ANwd*MgYhU!ADAdu%7xeVyAHq zKN95oh1Y}adFz4WR7N}eYtu@z*T<`*ZB@$HX(=v;c6v5dgbZ7;Z+7*YE&WS;G^giQ zdU4m|ob|=iZv7>Ig7szTMD*hE5EeXY`dr|`FzDkHQ>`);VuN(YQ1ttA)4>L$a9Xxk zlgjz@&Y7<9{Ulm>pr?JIdOpod)Usket4=n>=+B2&a012!k7E}^NFjQ%CP|c&u%CjJ zGZ~pwZ1;7ia}zUhel9s5y1cB~>8=iK-*hZwj4OXsG3ZcEWc4gOfq)Pc?HfZ2E+eI{ z7l$;jVr?tPpq)hiT*(0Qp*g=$sb7_-O5VLa95PN%ocVO4A>?>oiiZ$4srV`mlz8J)Pb-<2WO*YkFB0#>e|=p7oCZA=BmkK77XxVe3JzQM z3s4>)6CiZiuNzxGVQ~lJd_e0K0&hUT1tWSB22FM}=gi9{BNPIC+x9jYM0NU(*I;<` zjBAnKQcOIgC5|jlR9_rODzs0`)tBSa{u|sSPEX;aCBVzV2Z&5*?Hm9dU}6IK9BVgSv{kAGkp#ZYwjVUeEarj|MY`z}}lKRgm#N1}JM9fx zv*6@HD?#%73+b>vfH-QT!X7ot{)uD4F0{cFO#yff+m7kOw;-}JN_bUDV#n}4y&zi% z!&Y^r;SScov|eJ|r*M$`T5+6(5p(1^1FZ9$-Ll>EMzgtrA5+T(Q@mxyGk!Kmc-k0;l#7zc)EMy_o6!_esHh#J z{?Gs*l|OzsR2-5A?swv||9 z!(57%)-4P7Sav=f-RQ5RFTKGO)kNM}H;{@O*5!jV$Mp8Cl}pz~RohZf*`pgn#9ix@ z2P&?8PN_Vp?~2Q`Y@RiXk%YU!_U8aWw zxbg0<^Y)|irCSWSRqxliZ5 zd#a1%NywOJ{nBI!%>J07HO1JXB6JVXnk^2jd>$P@Pf&7CQbF0NXOe31kH+!3g5#Ox1-i5XenO>dWv$!MXsi&>A!Xm8NgH)=P(q`O}P(fi-E{EeS-W(@#P@ zGPYeVMrEG%eE_9#(@s^vn4roigbD{J%tmF0qE+5Ub3h<1Eg65iPwl28q;!|ai1C`e zphL_R{PWy=t7dlBV1yy?`>0VCPL|EcM2vZiIb<>_dR0SC`j;ibB~)6fu1MX&;xStMj-$|CBevgc^1RYa z-okRMwzpRG1yPko(hC+H^=_oJIP?QTVOEa*V_;s%JIvx8dlK8sY z#^pht8HQQ06GZ#b^sh52RoL^o`;O-l?DQz&v9VU8_B_1DS?NOU)<%z5gq?n+ZAc}6 zOW6e@1!xEOSv+&A$L(e&ksPgZ!U-z(MSwblE~>0IaW&|pNvFtLuXsDM*Qi5@IVMK{ zvX`-Nejh=?Ab*KvyqnVLbebg)&#CIz zCXVxWqqBxnG0$He$|8>^`2-Coz9)czM04+Qe5 z?9ZwF>)%K1u>YcV)BiYXACCD=?fid2?V;7__1hi{+HA7ryV_` zPbx3m{UR{PM2Hek66PUhM61{V$zT8WY5tQa^|ASCo|7gZ%Z$2i5<2bk|iHCb@Uu6d_IgmN~W%TMOCBN`+pEd--`#5HoRKA*bQzm5J~R z^W>G(AyFa4#SSt$+qF5I{M3o=^4s0H9>&L0Ihc%%CNoZOt)-o+yS$7Vn3e3HRaD^C z6jD7O~TCcmAa>pZSZWqV$_PkkI7TaVY_4p<7V-6vDdftOL$;eCwf^3)&R zv;un{BP+OL!?)GSS4v4nDp_Y{l_VB9D+wnf$6DCHqt{q-S#6q0hJT4GZE|Q|*Q*$< zI6pl;KbqKItk6ovpS{gksoYHjY-9sZejr0S3iy_woy|91$mlAd5-~-I4Fg8y^)B;# zAqG)mvl&7O90&#gMHov^(g|!Ptd>1&tPZ?WOuI?~^nGD+Wc2WzYp!rMD-NP8bOuW< z5n3=HB~gPrqaY+4WS-ik8-$!Zuoo!B1P!HLR`v*sI!Ek#a(Y$EUF#7vh8+`$v_!3m zf!dlK$n7-LG*m083F`{k3fv0q3f@ZYry@k6vn3mJTN(Xu_N$4e7e$XtE29v??%524 z3501Nwa7h3!lNB>NvI}1r>8JvT@2xf@zAU|71OUAL>igrika5P;*G{9A3_2ro}&GSKSnM~3w8H^YiZd6#9(zZ!`MdVsB2fXhfU=1CZ&H%cNpkA3C7Nt>rni_aodxAdd&SC%h+7z}lDc1fLi_ zU+});o`Igho*{xBf&f`UFc?c1To^4pH~ro37?-l+Q2hYvk`}h0^S^3*wj^4}itzg^ zBfxBlLLDvR5&4S-2k3+!I%nmeY-1eLV2I`oIuHahip*vKMd652$l_x;eL)KSL&RL< z&OA{cFNb#vST!rhC8Lw(w^lqK5*{zd1sS+s6K5S9^B4y&sual-21mz##gpo%SC#IC z&KfXKe871*=CM}Mt-+yDHJa(yF|v@UeP38%-m}!G+KcCgpdlvBMWM=%=~qGhkP}x; ztzZa$hn)rVV;(b{LYaiALl-*NG^Qzl+AJ;0 zT)}lvgDm2x|2#YGL|&=ZY!L}C<5$ANPWA1i0X5J}|21zy?pC^DpW%K3?5_5G!|??H zE-0~;ruB=lJzl}=7W!cObd!oOA==^Y+_1;6p3sGTu1CUO2eQ;o7$%*MS4Ip4F0lU3 z+UToGZ8vYHhZH49OZ>A%a5Jkq$vZ z&3`jS{y@`lQ64}XsURNMWZ3K7F9^A#fs;U~l+Y|d)!U2L*lku_pLg@E<5jJY^1RLV zR8PJDUv!#AID(OE%(CM*mTT2 zCqE0$=77APd-X0w+hE1#2t9QRSAPWQq(R&w7}8w=hR!(yslAJnQH`q?jJkRp=EF9L zx;`?B>&NvX=Z;%yAvjR;__vOs!QIFrnR?3>^DI3P(LU%L!!zUx-oX-r;%Dj9JzQuP z?hNB2D*k8aU^Tl4OBgPs<1GEPOZLNJF{&vt?T_!e*qt%woOySxXozEyV_-MqH9Yzw zu1zD<83P?M^JD(H5l4mrnl5RM%eMozi<%7oiFm_x|AQ}uO7>ru{Ma{5!aR|6}GxD&`85Aw1qXA%C zlAv9)ljd!3L5D^-c5Vfy4xYb;!-MR9?eFJ&!a8C4^_}3b$qxs(D1ymT+EDF;g}K7M z4}Qh;S|+t)^G880Fx<~fco_v(&azF;x!Cf<_{55_93<&`=XTcJ&iuYY8Ooyy7W}dH zVoXyz$w8)FUUV0DibDKCE+kj(8W#28D~vb^|G9{V6vLwKsOkFv;n6y)6O2PF6Mq6qwc+xCpEBG3nHc~)Kjpqz-xa-}VjO+?!UbTwY6jhh3Ia|=iEeJaeH1+l~ zU*fR#v95OzJCybs(&_fv8%LI8t*kTVlA9vr4q%gna_4t zBlKn)rS8Rx8P5)8+k6YqMyht#^eORi{u7|;6n@|{n&`q1l%Wxc< zU@eeVzN=fb7`<{a6!kZ4T73uf*wF$4vLGV%QrV1DOLI0>?H(=b(}fb0aj{0Z%z08~ zD}*#-;)C(C`m~jjp&+O~ViD{0E2F-@#nK`1kLS_n4;SVXJ`!4Y8^~tJhsZmulX;^g z360!^yvm`-1IV9p5gW_a$YszlL`uwQ6Y(3&>Y$TI-zfN%G|aTpHQ8kCm%BDZckA1V^#?$RII!_S-ZDWsP;b9e2O26OpZZsFd zHqDazdMm4ax08jvX=RQey&xN`v6J||a>{en|<%E7I=V7OH&QheFGrT1i%V(-2vR+al zUD;kaK5sB_54-bjs~I$&egVq}))*3b==}C>{r}p6CqC0Iq@$w(OFr!j0iWh=p8sYJ zguS7$rJXI`k2Xl+v|~!R?AhJ<4n?N?4_ScOdwf`*2_Q2zp(SmpI435DGZqq4HjyQ5kq#%a z^|-u&j_r)s+sE12agS#(f@r?hU5P_&-agWw_J?I-+>w8IT@lQ zyR0#~paByD@V^J}C_pK*VG`ke#;^hQppmF@uhVm{yFOz!=%vPG8NkdHrJe?5UCm28 zn8UTjjmQ*juoS0p4Up!E$DMQi#CX{b_(0W8b8ml#U=R$kl#CjZI~DB;aP_DGCr$~( z()k`8c!0td>X=-EBG{gPk$uaCH<9!LydjWr%g6ppDN`9gZm2%}HwQ3(vTOfc%75YlVQd+>KXbCuq8ku7qqLM691(wZBTUEW06;5*&Nl!INEuq1qiK6Si|^G4Iv9Ygh&YSgMPW9AO#?`gC~{@gleb( zFn|ESIlQt61VpQiZ$}7Xyqx_i;ND-KXEJs7-gvxQ2LxEqK<*RFe=5=~mF997PS|Py z25|veu>ulDusIp$TuXQTV3X`ta@I*-q*)r7lGzWpfBN#}rkG|-w?572Yl0N)7-#;s zq%+2Ki%W|e4U3*>P&^G03MppRAKD2QI*EW@$mw>1?S6#u<4o; zinofeT_EnV2`#NkuTn5tW=>94r~aCu9)l&vvQccumLL0&d7@Mn0^+_Q6`X)0tKnY9 z$?+DyFpmtnm;_VC`I>C(F1W0uov{hrNoPWHi8lpOzJ7{ zbHy=fu9;c-{K%;Ys?h3ebG`&e=o?dQYkj(e*)x>##noBg_RW}uUoM3hLIudjv{w`g zrdU-yy2t}9P{vqyQUjvU)76H|G8b)(;m7fbt?%Z@&=m@zf~nXfi~+uz3%&E^=5%g< zG)!~^x0`{5$98Q=eoqntPA7XcSf(AHv=sbm0Execc!xi`Y(L)pLU7e7EvcbLV!yVCm!^ zy?l)b8HFjyy&&p(vhYu}RQdX3cY6m8h!Q-BCbWP_6qUbLlHwA6AVC27R#S!l^}WU}4WiNw*zTFvuC3axjqjpq`d#=jc3?U_4a{tGADTd1yH~%1|5sWGc?!S%b0wYi?`y^^|FO+@xzF!+ zGnV|5F|&t^tnB|)`2EdG1pDzaOMl#rS^ws4%<@}Fr$|lxhyUf5+U@nIU2cXvdFH@B zbE~Qex-lrdaK%OH4-5DBSQxPQeh<;_7$)!xrO;H4jHy#T%dk0Qt0~{^x(d>(b zOB^ zmN~hIH^9{+=Nd4_CjfHN7fSsf6K11EH9wWwF#ghxJAqG_DE_z||E}e~Z^tVCw8Q>x z$1Ak9=}9D(IDYoz;0bBVweca8g^@(^>L?Vn86kUaONM$okxom@zCIo$M1*jABAP9z zp&=X)1$J8pf{5U7AsQ{9Exc4#fE44WZmh`0Y?HaMDF%(!>m9D%H|uOCM>^wO@hWL7 zow?Bmv}zJ#%JsPuioYoHcJeoZessCq-%VYtxm}FO&~-Kj6_q-4(`k)QXbAT^8qv6y z9&0ZUJ8dpE%)UN{!h4CF%`COq`xuQQ9nf(Tq}Vxi4)47TTE^21nt|^tC5X zy^mAFT-NN<46$IbWuRO3FIawqu8-TCOMN#@lRo~Dplexqh$7(B` zU^(9UCda64TT!Gn_w!iVoq$ATZNU(i9$}I=)%DkXg01meN7Q0xa$hF)wmy^cSzy`E z`Tjj9x#J9@POA#u20U6H(Lz%b41rBn?t<`NlEj7AX;vIT3o!_Oyf1phC+Ds*nKaSs z%xlRaEMtxkad=(`yKRQ4j9^b6?tW#aELPLc^}jG^T7$DqWc8!W6_&O|VmAwnSgO<8 z4!Z4_-;w1@(~88SdR`OYWkZb(6iHX zkzr_r0K57&0M@Xu@=2b8XdTXpdRk6RCDQYf#R|Qrfv3WYl-wIFPTbUrr+L25YqnA1 z0t{O$rP?q*C_41spnWFkD%vs!OMq%pZijXN4Sz_`W%0;ZGJPBg(53la^p-=xt#K>5 z>tbvsExqV_j;`x{ak4E&_6tzH)gS}yJJb=}5|AgICFI_o9OFhlmBE22ojJC2`#gzIeV0UW_P2-`~M!a5JS~x=A)TSv;zTtDXE>O|XZR z(&78nE}y1FuE2olOJb>yrVOs44Zl<7EYPZBV5{-90nytO>RLvB zcon|uCE^TwHCrec!T6OmOo0u+y!Jli61dN`q{iNk*3( zrX{Om2ebxmL#bVg{)-NKaxKnaW_%&xY0rVS+lhwhy}T)n4{c0iZsRI|%J%M`?bQE+y0?moJKEMg69^F8A-EMD971rH!rd)U z5Zv7=!6jJX9^8Tk_u!g9L9pNu+_iv$YWBXT$GH3Sz323}yI=aDp6X@(m(8`lwZ7jB zo;!W#JNwAM-?B5`SboZM&pdfWwRXuzcfm>6k~i6DO0av{Q#6l2{?}wo9Cz;M`79f# zBKR~y}ld@D`+(O(fx;E_)9y$r=W^=^IR@sgp> zkfUFkDIwN?lwSwAA<+xihUI+u}_I zNk+5jS`)j6IhmAj4Mp>=P=xhj{NH8@YcL2|)|PMd$+9f|^D z6eQ#&JA@flh`X58aTQ{MNjT0F1%R9Y9+;s!#Hl^G= zJbk&7vc-hxhLgSc=4nPcIhpyx&vMmk!CZrUd~YEzxuQ|6=y!k&7h_LxM)>E<`#cNa z*sot7tSv!QojfC%$@%FxDquNA$6kQK<0qSK3wBOcB;mMRl9v;?5c8~*WDIW!G^gcX zeFXU#$^E0!$n%#2p@bg-%`Bbmc}O|jr~e1Fb^X>-O6)&v2;h3KHKpgxmF4sA-{}fx z@Bh9l4*mlQdX#RSVtkZs1j&FHKt}$bNKjp@w^$k){Wu@p`V5E>pFbf!4Q-QhC*Aru zKcfAo@cC-`odo|G`oA0ge=9U{?)^*n|GUutA^cf~T-nflKF;C~F+&F2>XU$(Z2qyDlG6P5Kp-%KMD zbuMsnV*aOg2E`3|{rkrxf&XS@{%7d_?=;Kb|MxNJ|8Eeo*@b`7I97C?k4dT6vZNoS z+XlZ}Bqs6+yyhI`Qc`srSD6H|dXICBslJ^apHv6_b$7?ciN?io_44}g{vXd{_}Aae z&E}Q==c&+^n32BpJhc8jysyCj$H%1K|C4R7;EmYbR)I~%Kb{<=lt>Pa7aS0*U!Y3f zg$yW_!@bHoxkjd*&o@96wl{r+>1ce>=xn1F*aQ>d{{WLQy=;1xB{Suo>A(MBF?b*^ zANxc5)S{K*P#?IyYdi!c<<$JZGVRm6i+U=afTisNWuMB*Z*`7R zT&6(8S*xc<(~i$=M!>EvbAG_Dn3~JohF)FT1Gz?2ZB!4G9lr^#1^0!n`bdl64{JYy zWdnuJ@!9ypZ z#|dII0Vv)A=UoHO-h6f8WKaD4j$&iQo@d-EpBssfpkGj+IUEoE8qN>*gk!`06d*N5 zta1~S_L-hxYbsrFmNMKZ9X$+?os&TD0hd5(`Z!<$Q_v?zaKo$;D4xPaiq?KjYD z#e6`m;%Z7Kig@XpSH@8{0CHJR)M=lLn+&^!(gnGJNc?=MO~rWTw$F`R`?k;`#WmUh ziu5h}4e~9VAA)_irR8dFNI9>wqp0cb@n^{SNWSBJf7Kyz<9oym*aGI$1?61@Epsos z2ZG^_aCDeCt6GHp)Ej?knB#M=^K5aKS*Qb66MRw6=^@{N{kR{o_Nj_*6N43PM*FyK zMNC|lJxyS0qi^KDdO@%FqLKZnUre&Z`#Gamt&je3KLf!5M7e-F?K$dfS z0HdNeU4;xZrC0zc2;lR`@C&qbdh;N3#NgG!+coP z3SQS%1*{VmDsz``Tl0MUz&W@a>xcUtr6Ust+WfBglg?CHFR=-bE$fZ7*wSj(sz0Kf z9H~DmG=&}Y2b>RnWhiK59+m?!HI?c67u@gu{y{`SMXr3Bz9sTcqh zgOC&pj5^Z?APMGm^7;u}$ZgMW)DF}Z?J4tovBVnO#|sVRJrmdDC`{)0&Ev$QZqfXq zU(@@hhzp3A=Pj$%jOGiQ=+3NbQ;k6KOOyMg{D-~6hpoxkM zm-_GZ%vSvr%>Xy;&F+D%C|u_51Q-Q`4e%=&u14Pd#4yUc+}0X9!k#E0`}DS(@yROY zg6KDkjaFFl4;lT!3|ro5UKRa`-~}f@OLExlK56SoD_yJWJI?;Dh=(N6PQVE*f@F<& z4H--yOegp8jkHcex#0AC22O@6wH||bB+MG(4hT|oW>^XLq+YB65s%_97{?zfc5~f* z-rnF|Mp+B7G-tu?-ymqs+tVm4iZUA({qkHa>-2Bk`T6BQPm(8Nrffrm zC*HmR^iVOMQ6%I@se!_6g8~=r)C1g^Rwx*~I>bL{65kfnO6kV1 zNjRYA=pdUch;LwMpu+&g%m<; zX5p+o=fVdbU7ih5EVSvt0SZNGTECQSo(4@=7t76OoVpigjP2np%jU%O0`&~_3LknH zPox8>qUlC~mC0Y23{@^(Rxmp$Eu~$_H|V>eII}sE_*q^2+#cTQ$NkQ#@;M=T&MLxU z$3ndKrVoedDY2Mvv;Z)p$Vq#ww2A=ZgttjgNRIn;f-+ z-xap_$!h_vUL9Yb(NHlfiCXerY*7jnnb@UxgHglk{Qiz{yCu0Nlq(@WL?V6jMJs_g zeS>TVJ~plJOKk!^n#&hEXcI0ALkk51UeUO;-FYzi&Fm-_2K2JI@K$Im>8K9_F5-~@ zOz?yfV}mDtSuMN4eH3#X85G@$>ZZJj8C}a5oDT?W!dA3N&H>&F*}Vrn(N-zk(4N>| zT(aDRon27B@WBgj*{+c-it1zMqx8JycH-`_>|uI}ff+*yie;VwdQ@sROopxaKYOcx zbg2g;-3@N9B_iU%)ID*9;&QE9ed-r)UvnieaR25mv-;V)9C3%!L2>T7sZsKc)-Q~| zQ*8`F6^q$3lM_gD@i?MR2gY?dXJqMPFCz_e-*hS@myIN(lDR@J?bZ9?J(L$06w>}kV)wcE&@HR;huhLki)D-r z%;QCFZ8AZyi!l||>99|e&eY&Z--Mj;IRnQ#64xSWV)DZgSXwOT)*GS=Dvgjm>k0QG z(Ie~yw7LX~d8A|*jDBt(t6jo*y6!BPD#QkW40L9?i@8FF&MOKg>oNtr(#M&{BDK%vaUZ)M zg)yApYNizLy-wZ`DGZIO`GM6tF_066e(PoJXnwS{fEy)tZZ%L~_6RAa2RcDqqa%@+ zOjwM8w}4|A^2*6G>wL*(vUzO57cDP*kWR?l)y31{0)d8s>~Lim4QqSfs_Sdfms|u* zjKUeVY}5O)vE0Z!q1-K2${(055{7=#iKoIaA?<*TGSk#64d{Mxfp3*8$y0TG>lvYg z>S|gi)t%!-?MMp*@kRYy;4EOX-$f|Jm65x{5@#45bsU0So9PDFU~VS3rMNTP#@*^E zEmw4V8+Oq|5w^5$Qb;Y^AEjp{dSh`Uws5`BQlC$d&RC~Ob1Mlg8||YwV_(+Z<{Q~fj`aU3{6rgbLiET|sxcArdX6BKVM(%) zcm{cf)Z_Jzz){D5M6!tzLCh)}$ZI4E;>CoS{xiKbf8!FyEi z@Jw#Bnj!vStU;{)2)=Ev&H8~2NeVf45<5Gn8|b^MlN*C8b8i$RM95!2CyPOm*w^Cnkr|U2j=r+|kda?^T#tr-D!m-A4I?q(yrZ{pD&HR{?#XJP~=#*+bTIInDq=Z$dZOh1*l z;vnpi8m54VBSI7FWzprhZHhCYZuBP`I28=-D=Pozvj+^9UWIkB!f4rD{1xRZj_F?+ zNg2C!!yn8Cx)qjlVU7@u^)9swDmy-?MjS36fUJf2#OR`aI}$<@fs6g{#EoahR4J>I zUrsjdCYz4GBG(wdhh6A8ej0(!)p_;~*Jn(2b6ybxQVWbsHdAc>a`myn2E?_l)+Z(v z47iiN_&zFQyo3%$2^IoPq4be1XR(<0CD&I)d2F@;Sb@rrM_^hq4qedaE2jn{(Br81 zQ0kBHrG4y+t`X)Wm!b6jgnq7G3GC!`HeY0M0Go1S7)xid#Nd~{x!&N=$M9DWJpenf zn|3Ar8<8P)!w-gU$%e4d-@`?Fh%)fvsX==u6S$=*ha25LY$w@YrCzq$d85hk9-O=aB^C)RyKFVs-C#r|J zLjHntpA9N9TA4(ps0w&S4Q5!VAp{ zm_YkPxS~<0Cl^#12&T(cTTKY04KFHJ;mO|g^(8T*Rf;kq2>RTH^&Qoi;tvi5dA{|+ zX_Ny&3%z*0FX3oFF^U~B)CzdY*g@p;p3^gn2C}C9^yY4Bn`=E_V{M>EeVU?Dnq?~? zGtSdi;W!U&(c+4pTUn(B?1LQ|HN`hl?Ic zn)_k~)3t}}(G{-o2-%44_)ZLFJkj$w#8AgHFjr@mzIHb3H6_!yOqC&-+qbb)E-1bukkvuQ1*=rKDLM%cqg8K0G26OX# zU|cbe!AW*0_R70qDlbwU_EP+ZxeETz(J3MmNl)_W$&CFBLY@usb}|aK{+fY;pNSU`1yi*ZbXOsrp*ZxQ8qbEUEiCKQ}E*t*6t9C%}*(V*r=LKqc~nax@?f zW9$eHAY00tXmJ{JIhr?=+Iu}n6>Y4t8q-N}_XGpNwpa;bpapwj?Rd`6{AeS*Yi5kU zhhDPk_+8Q_F)%U6nepqmewX4RMRh<>&XvP5(Xvx(>NeHLLo$tu_~$_EU_m$?(S|&q zE9){-s?K(_JV6{?`^)bn#+U&TZc?2gzG9;_0H5TDTYVPI`wi#5m5_%wYvJ^i5gXsB zdL@6KG&ucZm zgj4hI5dvi04PT2?3d%B6qYGZrkb`+0x~b(qJEg>D&ynj56+g z$dNoW)brgwHpm09adi2`NiUW+1AqKT@z1n_;M3!U<)^h-jsoC0}_-Tu$8HM&{0@ucora=Mj& zJ4o_o#s>R_zr7L{T)!C(Ppf{_4d~_P@iiQd!jv99bxx}^(XH5(?o z85y0=fR47c58C*%OgvBHgMEU(?6HlT2##N)K;Q)n41!*pg5IYvS=mFNPE;~OK;<(H{7Ke%a8biZ=@Cgrk213Q$F#-nPNf;apaU^ z^oX1a#6>ZF^(V;UPdjCZXShb^MEkv@_)vzoCP$!@E;cHifAQ9t^5pe$$M$fdMAYf0 z9#k5oF%9F}$VI%RgA3WF=m>&v8#WV}$$B4))4|{`k2$S`Zi1VWfNP*~QW_&RxHZQU zd4C*)yJ54b#z3PU%n?j4&M#6RmZ=Vk%EAj5t|NGNrJ+D5AxU7S8R4l9{SN%aXsSM! z_Liu>z;+Q+@Qu$K&XjOvH{2~V2qDi}YT(N|-R+GVexePn*QvXAvd2;J%kD`>01uZ zjm4HbN*~xDfrV~^pj@MZa=5Dc& z9waa;%umjdcW1B*fIbx$fcKTWUo3io99)Mtx*1%gpD4QNf^p%xuqN0?79;J(f(+cp zbw(IQFzr%=-RCxZW9)!eH`tqW1Lnz;iUh)I{Xl7RZw^DJhAAFX`5U6U@a_4;$o@L= z?w8$E!J@(9!RX;7uh)XhY5G!y!}g2{&AC}i$##$?ZWmA&-YqZ={Em(!P^T4-fZ1#g zN6UoXyxo5VjV>-r>Q-z|brfCHZ8dAvb~8^rl@k8~M~ADxD~PTwkd|K+oYg4K=9c45 z3uQ26Xqb2n;Pl21=Yyg-81VYmlGJ*ghJq;S?M9>+)MnnQ(GEU5>~lua^Q>CX8#w}8^JjM@)}~!#v0_2$mB?OOR8?ZVA?0pgg+C->ig>=tt1%U4KM)F zAbE24$cAJMXK8^MM;yzBt%9Qt5L`OMOR&U~4BY0m#DQrYcD)O}$mZ0_1fV+ykb{^{ z43xRxvr~q~7sr;c`WU;dZ3Z*qc4;vNJ`0cI>OWix%H?zjw9)xUJ&<;oCTtwN9F-mS znBEcOOH-bdx(eh!pMow$fG3RD=4Z+vp+ywX25tdTCl<}Yfgp=~znLq++enG18z$jI z+vgmRoi9hEbgLZgJdo1=<%h};WgPb3nI8ii3g_%VE66QS`ud4|QL$lWq=vSGS|5$n zjH4$Rmyt%bI|c3@s2cblC_Q+=DxTB0rqx*V<}0I2tEF1RX>>dOB)}hU2Io7j4b_d{ zW~5?PG8S#7oIBB|KQyJ0Vt=E!pji-vJsOc1#~^~PkL*M1%*p9_dzwfiNv*Ve`V}o~ zrwUuM73}llY9Npyy=;Z51LXnvoB<%CC=YR@6Z^~$|Av|v6hje?S?%BxlR;1U`tQ1F zp|C7fwDj;jUXso7=CvB_6U$p)+LGTCXXY9SfJ7<> zzL(REQJcbF4Xcf2!@~v$hwoVxYCyw+`L0e2q|6e!CAt122hq5sL~vFq@VMnOO^WP+0<{nfi)S;PBkEr(+i&VC-b21H<|_H)7`_Mdn)sQ>Ef=*uztFir-Buo1 z8%pRik3@tOBFTBg*dZ!EcRV~gT$L|;x34wos=BS1YH{ybdSu5b7q1)os)Uw8W^&j| zE^~lY&G&8SObJbwT+`V7jIMNF|@-NCd*{Bp7VG) z^K=Ow=Hl#W6-D^NFA+k*+&%3hUh}QxbG(On>Ii}$|Xi-*pOf~`%E)` z$!2H)%(1$akm|x|+W4N{(C&Vkg+=Tg)HB7lV8VA9GWTZg2rajSZgX~ z>*@acOA*~lbk=S2=}@O{jd>R+T=U-5r37eOUJHITv!NS*U8r?A_hn6Y^2uFLqSpsxjuJ7-0%F8&ewBaqq_bTF`B(`Z(xZ}8h_)3?zByn+6FSoD>ihf0DT96LK3CP6yf>Oc473;ZS zDbMC^nVPsHGjFI2;=(@}+M~BaIpSH$<^PS+7rm59%^2`(GfbE%*(2uFNLpFv5)NC{ z-~A|`wKSC;7BatFnO#_F6lGc}cf!|>R>kBXPp1?y4$yecZG$`VhW#w0n z4;}vXKQ~wMVHDWv=Cw?!EDKXJj8DeTaul-JpM{GJ>Z0!H9s5%OapuT=1FDOuAw1PY zom8Ma4#41(`$XhBX{K1K)IUb0p_ioBqIrc!Lk5|qqn!wL)4is-{UxCWHFSk(iTQ+% zz0RJF3lnzGUuu6Op^K_R|6}1y;P{6BR~*!CyfcLx-*k4Sn_?4Nq>iBSKP$SzE*Ph`9Gyzz#pcF8>7|K2rbyd!`Dn7l7vC#~VI)QW z{HR4fLg$k>kpn`NkY>Zl6rM;6Z~OA~W-7=6Sn)orY_D*BAIsEPqMA9#gM)0&p53)7 z#66B!biXoV;iRDkYdG+oW@Jc76Nk9e$1@3*Ul%!8ZkH~eP(-||2XFep+pik$m8`uk zyF!pxUfDuYkC=`V*5uX&HYY8O7WzpiGT%t@)g9ck8J zJKtzvfQY$DgpVt+8aB%(VJ(1d!=eVJkimuV9hF^IujAK&ts0Uop%r8+KH|8}-=^BAM<{YkoTjR&arR@h2bmVV zm!z`JeFt1T?3wZ!wsQ}_bh@w6@!)k5QyGRLdxdoCp!G_ym*exWghWP!o|zx%#_AnbcUL- zGa{brVrng=31ie|3tju|mlee`Q#|dP9kTD}pcsMW4`GCj7LM$_xf(XO2k+1Ds$oAN z^)(a0Mh;|~*0`)3791s?*w;&B$N!Qf&*) zj$XapI0dC4Dr>mp4^B2w3_C~O`-R;!sKI1^@ZrvyPT4$va7AdL3=!d*%0F%WUX_}< z8M6DBV5sh>Q-de5Xc+`4VfXOcp8Y=ct8Ctexx|=&44vhDR8z60w0onuQ{z;Yqw?`g zI$^mEwf(+yPlM~w%@S45JhzG~cHXt>GKz#`PkJ-+`m8&ozoX0d_$y|5n!VmRUkM8a zNypqjdCE2HS2BDl=+TDzl)Y+v#dH3) z7(!Hy5c0&0i+X+|8IzuR)S(_b_tc(wXHx3fP?1p#QYALssU^4vl0ObN4*KazRR($5 z!@gOFGytor_DtwBQYGy+bGm$!Q$%SN@sHJxwnU$si~ z*3&!NNsf*^;mxXobmP|F*g=!Bb{Apa+BDMTay{Q(C1Oz*H<*=RmHuXHO?{rNXVh%i zB`<=mt!X$r)?PM*^Tmd3MHwG}%D+s>K)=1ATM4`0uP_b?4?)8S@+e+k7<@G-$Ihba z{mJo}Y{YFOXk(Rf{%A%)Ww#;3s=eu)CcPkaD#~O4qB^4`xep9O# z$fD2V8?e$JM@^2Zj+bcnO|~K$#t4xLi#93ZK*N8j(`y8AKhB3#9_0)(5R=&6b#MLp zt$tk0$BtWL(SK$Z6>eEXXkPSalM-*0ehfkM3jDv%e`^D>46==doC#JF7u0? z|BpHj;)IFhIXuBmpCrm2xx`6Nb9M(sF8%)g46K3m_}0LC0#-Xb@~RLk9+Te@H?{EH z-J#sG<*OEP^T&?-)042>w4VJoFJ_6B$CY1AlZ)sHzLEJ21b773tOA^bi-u7rvu6Bg z_}ad}08ZGhuFq z>O>e-dyf5961>*cXl_&@XN)*@q5SkO*u3_$1drN+?^ZU*Zi-MpUOykKnq)NMF{glK4OY+$K<3s1G}DXZFC(lPjE8~Ps8yCkMj z03`A@W!h7ByIafJ5jT8MuM*oWcEq-!Le#uB!!qIkw?C!1vgBc{KlSKdQZD0+kFt9G z-1dA2((xN3P^Hmi!!=&lsxhI4saAK8m0-_w?<<+oFmZd=F~fX~V@s|6m|K=22#NWN zj=>&X!3+wsK~>GnOoJflU~6HPvuVnipCZu(+}Vu`G;b*LGCR65H;s5i8|!PDr7PwK zUJb)JflZ~3TOCX%wvg->m)bfpZ8B~}xJ1@I_9eL`DzNX;4dFiD_R!;q>=ul>25ZI~ zZ*_?fY{XW$`sZhqpR2-Ih2?|lwTVp@%O$6CkvL`8r9OD8I-8kYQ|QFNmCJenv2T5uu8g*Wc&J5O-{x)=u5ETTiV)dsf|OT(Ibg| zJc^;Er@7;i({Qg|$*5`6s7i+s!LI3A#;X>Wiryw~*X{$VXWzV&E(np+TPb&YdCF=d zYN8L%CTg%};{F0QTGrk0u4J#GsaOq8ybls&bMcEKH=YC~b2>k;j|I2j>HMN%uk!ft zI}O909y*L6u+@z2szq7VH(s0ZQ(7m>N2lAiVB>}MZ$=?qWVEzsViwSo!)8JhrO#*g zEmK{XX?3LyrsfhO#Ri!GeRer57y@J6Y!fYU)G94r1e6d;=um8`Eq=$e6E&eLjKhE; zI9Kti23kQ~Y4pQRIl~wJz*&Xj36T|PO9aDAh({kubYE9t z&TOK_kZ~Z*JF}Ub*0Rh3A>^7m(^a))_5y+x6JBkb&ow<8L{zNd)oiLgj2FsA+IACF zfC@=|Xp)kg8XlMAr-eU6JLW7=U%^`6fFCwUDhPPhQWLRb(8?^o*yQT6+kPeKWsWIN z)dyTK5ik6S*cSk^Vu||>5!wbAjrUi znTISrnmxC>Ac?d@AYoIDPVW0{m?~mL5gBf6NuAwlp*|XA%tE6@@vDLX~_*bSRBIx14>_13%XkJ-4ZRm#m^j9c{ zY>WITfj*|B^{4@me7eU-GYh$PqZHXcS<95PuFX=W&Jz>mm(2my|8_X*lGSxwL{<@X zSJ&<=J@mQ64!Gs_nFbN-Rx&PiFH8nt7(0mTT3>qs-RmUEv$=IpcDu>H)te#H>tWb( z{Pc1OZ}288Rw(~|L%)IX+9$^HR|h-Gf+mYdbhwbDC!gl{=@cj$NZ&}(72eJouK{7! zWWPkUo=C&oVXJdC1UmDq05pG0Jm0g2(9Tr=+vVi?K9KXAS0s87yuQq!RT-RS)ADNy zwqZH-&l-siS*)ESN_PVv5LAU}k&x~L>RPU8u=s!TJs;IjWrNABl=Qy}Q%hIci*XI& zNPY4*0R={r2zA_Rwd|4+GdG9(;3qKfCY*-l9o(DRQMo*@?WiRp&2q!G)L~9yt)X zIWHPVJoH@V(`-2$E%cxzKSl8Q9C+suE2KQIg9GN)&QgC?G<4AjxrnUxV?OnndTx5> znlroXst12~Jbq|JoVXdqQ9WXPJUaV`C!=orWDeL@5#3h7GQhVvu+x|&8g;{)pN~ml zzqd7(8x;iFq!}>Iys9I~^iD+;yn+yMjF$QtPeykA)(DaseRyYSN!m5=lBC}w{2*71 zpR}}zoZM<)j1axKXl&t8_Lzi5gcR+4m$?k^Mn~JB^Zqj57OqW!jqsPA#OCU3Eyl`W z&mnoOy@oog2m`3e;aGW z^npg-4lvB3%6-IQC`2Ft>pnB{iC@xCITPZ#|(xc&o)X6~Sph&ia|PAZ13wdywnwvd#LD904SRhmt$#ajP+lYu zb%u_w<12~sMO%QeF1Oeq=H3kbp+@QSqwT_!#MOM3Wp^~5fgI#5bW1n*!R!msdHPN? zoGV)5)XsK`KN{sKUs;a`Q$Tf{IIba!jx(ls{r4=a`jNk76D2O#w!n}%R){TYn=Dz#=UfH1z! zoc-(hUz>KjEmdd&n%E0B?PiaXxi=z!>R<0jklETKXXJOHUvkEb?~>ntuiAP2EinoR zlnyjW9@n~_a3M^WX`t4N`AaXP5KR>4_S*5RAk)*^Jv=`?I~+gQVc%dj}fTI6o7s0{6JDMw}54fIerEiQ?#( zpR$P*`tg#w=~Wm1zFjqjG$w!5xg&9p%j^^l14g-Nd8Ut`;@7a-=Um| zZ;*fwONABQp@ZxsXudSppkN{ia{}VB3d?!<=vW%r`q;dg#-edKg_JLO%z(k)!Qb2h z%!-W7pvz$iy31nPR!15R zvfd?>0BS~@(i(|0kDd~H#oc&u@e?!&UMSn2&14d=v+f}Fvp(G=74HL6wneg-_5qDN zxhxl{<<3^J_Sa2vhnt|dFOFQJUu=?x0zH$s#nIo0=|IRKXaY0`ia-k@m+ESRqC&?w zEg2Ve&3i_k@8PJuVzvGYTG%sWmHUd<0v488VO>iO{HF~?fY$>-{QZTHsD{z&Ea9sf zRE$~ejh=AS!nh=z_oInfoWRnWgi=UuxsdInEEm@j(2_`9`R4Psj|pv?=8ovJ0%H^C zYiX}Zm7S84fWp3e+^UES!8stYDl6+(ekG-@y4Y?eq?pi@ZeI~HI3>BoYW~}^IF=v3 zwQImhaqhrIHK|pW5R^AfTNx=wK(hGYHz{6Ki{oS*H$_$AN1y5MFmJc;HxQkSNIf7= z_9t*b6|DN9LzfV|Q%UHXSN@$KcS_aW$WJf%S}iSu;xdeU-lekf=JO79Fnw^5EXlhx zO1x1}o0(N;TBX=iMk#tidJ1E`Ylk9H6d-Da!M%1{gP){8GsH~-dd5_=kL$$mbf
=wV zo14P=Dg?pr*HSe!Dw2vcue=Y=g>~)tO~#RzKHV@~_ueMI<&Fd~yeFN%t0sdColBwPDZAu)@%Qbm7BBc#4dOgMeqE8t7v%;QxD>%uS zc4Q$KwBhZoK$7jZcg^`gA-rfU!hPB5E&}G%cNzX=?H70a(DJn9wt{fQXxeni z-Cj1hJ$*0{MBw&>8vEE{G`pjj;kO`4fD^fY&V^eaxYU|YoM7{aIPh&NSj7Am*LinX z44z&#Ue)q2VcHS;@$daHoSNn_n;A%f*!gv9espbceBDIRe|^K{5Ohp-k?)FY4&8#= zWRrh$c6~~dkhl-Mdz82b|ErnNX*N5f5aq=SzkjP#{Xcv+|Fxx_)Yz8tpQY*>YKya- z$p0*qE|zK$PQA|#x|OO!pPp@eMGi@rodVdsX(nA`;?U9um_~aC%$#Irzr=1kcXPfYOYFRJe#2os z+exnbTpH8EpEaLKiNempYQLrW(OmR#bMTgQt@UbjeVp0i(+8LOdR%gUsYx6Mdro6Y zHB#(DlA6}0x=*h6}FEvi1OZx_!J#q4&q#~!C>Le++L|tg{;O9Z+@#APl z$giJB=n(`WQd!|E@&(0GE~4M0xQW0>o#a;lOLDBRvpUpbU0?_p)ubzW0qd3J=_eml z;f01OFey}W6FHIp>{R+lb>Jf%o*!xF_bI|SFs|u`csvr(H%M0z(qCkFItI9QdYT;s=c_fxm4e&|s#$8m*% zYj@9u)Oz~8XEV(BGbUq_Q6@Dot@gL#d|6lrfp6=s?wr)Veb1X$xG?nUF8zbf%W?$5 zHKA9_Dyn_FCoUtavyH&r3%Bx0t%V*hr3&J{clv0yQcJ$?*mXGD{Q#-zyjZBZ3vh&0 zh5hl_<@@=eYm~0^wXVX&jDJ0oUw&-($BIPFyL~Z-7bC1&u$-7z71%|&R`ybO-yeA< zK;?%=G|Nq3F=MeqVdS7 z8O0r|YKeyst|{_y3RI< z^cE=x={;`U7r-DpWjK{Vu~|iBWaQkjZEY2~`mum6gDT;PdevSdrFc(`=1Ns=FO?VW zMg**zV}ri$7tsVGcp)O_?jYiQ+lO3mER(zSd&YL(yPFj4LZuQHoqEdO-O-kWiAuWo z42IrOzp@_ZL@g%?=S1!1yI*D&9=%m3nI)Okvq-!f>k8TUI(jzBx8v0VEj^og^G*V5yAX581XZ<2aul)y^LYCLbx@H#7Y`&;wLbc8M5OBtKmSz07TSBgco zzf9?Wwyp(QzFoGU&b2t%f^D}-<0A!K_LrAXMux!m8^Zct4!%ynD~bRQ|DCq;kjnU~ zP6{Y5sz@sN>v0#Sn(7ML`j{F_*j`l>270(6i;}b(fs)3mY;v76Jw|kd9sg?Cl);zJ zmgoeyGrkZGLY}Pkud+FH@`9PmKd$WI8(IB(Nn!!7mB~A`F5#bh&1vZ8QYIm{j^##{ul~efi1j?K_QV;_eyzYG5Nr1R=*^@8 zCB-O1{Sbw9Ls+-WUj*e1J&5c6SZ={g7DWAOPlUtYs-XGk#|KJgS%@0LxtDtapcS|E zbMk5om9a#hAJTHh1n)9!H|L4K2HVM3&^PisK$dOqQy`s}-_i(&olN#y9@H<+d~FAd z9N9LFv(}&ey5!Q5s)lIjlSKl1qy@!`)Ncb#XCyZrwl2GRT@ zzqyK+m9?|A0KYjmC+NTSSewqMvxK@)4xg|6x}1UxNnrzVj6~imYR}pS1XFrvxF=Gk zRQrv6C{O`Hp%B328UPFBw71G>9v;83o-7jD0sxeT@mh}Je)bbd%TtyG)XN; zDi^YN=l2J-$TVpoN@&@6h#@G+S2l2d$wAdh_`P>~YL3Johuk;p%`;utTFDKPO6Nek zEF!(8K*1xk_CQur4k;yS2Wdn$WVlU-Q>T2q4^i~@@6p#CPN?-X;_J>6w(#2=vPe#< z!gZIaOkkTzlKxYe^{3Mpai3Otq74J<`9~27BKGTjwLNTDg>u=R6xYn(`HY);40 zFtVuZNvQA-Z54)FS<`h!vkxS-=cq2{oVSolX^GP^{zJoUPV{qw=0F6>zYc3Hu6^tE2&ZMyi&~L;%A|U9hdN{KC?C&-gY)z%-vBo7xg&nU4 zF;4STt-o4(7e1Q~zD-43YSWN*!;s^lx2tSHMYRG?Gv)IH~~NG>BDIHzS0CdM0OyantJX_GA?P> zEVFvf+g3)8p9D|ygLcmxz;~f*4+}ns>FfNb!~XNhd6Bps=I2`-74UsO_{Iov?Gt>g zyjYuT?svKl4%&Y>g~Jk_=+AQzSOhe8cm4Vs=7^!)o|~UA!QfM_<|cK-kjtYau6W>+ zY3y2K_sY>@EY0H?%}W>fF@?l+^)L8Fi}=OwGcatGCh$og@wWppy!N<`c3_SUzczLe z?zubo$Y2f^`g1E`qMBNoAM~&h{HQ+DbJ=xg9(;zJ58L+1&ZfDa3P2R?2hsQmRP_W~ zzC-NYDvxpn`d_Mp;eQZk;Vs>d6-z!6*Lev+XS?7V3h-sE^Du}+!)A4 zd7W$>I%!w6+Oan8G*=u~7K39>A>RUP0D23#ad?M6~H7})vA(22jS#c5ipp-TL$ zBz~uL6__xUPix=2P|z6a^PR3BgOP4c)Wr37;*~XT2&yocwtZWg^;)%G!qb|@P;e=s z*$HI-7i(`76-V^7c?Sp(pdrB>f?MO-SQ6ab8rR0%Eg?aJYeP5g!QI`VarXrGKyV9W z_`l!G#mxHF%*}hzbZJp`Ro8vUBKtN4?998TvDf$btnX13w<7~E*@yFqTTVyCGF{!bBA5`-* zSOIGmKaGG$G5Dn7y`zX&-PM@b@42NCndPT$U1|j=bhRFHybnRYj|m~JoCq}saBI_` zGc}yEWG&!$yd4)hr4o#vGKGD0Q|T>13pg$u^hb~S+M8KvxhEfq3K(5C}M$xvV~O*r8Ln=%9^%-F4*NANLTO%qjm=Dw!8Hl6jQ z%=TmcyC#7z6a4$&w#vudaD8LC)a)09dV7nmYpgb}wIX>d=S)ngi_fjSI65<*ZQ~j`wp-BH=N=#j6(QCz6tw`zwf;=SVxsRR$%Pax#I|?=;PB&cREI_U6FK@SYHbQc*&_ zY{O|R%+_mNO~5>!WqV;EET>SrY1$fj3q`QxR-r``sFGUeFO9vnWO(FKTsXz!UY7n| zl-g*uK4xi!2SD>%R<5JY#^TP`Lk7hnq*9PK*N5qtGHyUa!BmA?n+Ge0XnrMtzd!bP zFLwF?^AvH1SQ&B3^)hQ*9phGqUk7PYQQmx4A6}pg@))zlYnwCsDs=NxL4@d>py{h* z-*P2yBdXB{UpB>yl6Tw`x(i4v>-yw%+}02TlY=IAfxq7hw;A>2B(Lmu>^OTC19xYH z?Ui@l=j!i8@bkH2$E-W?Z1bFbBwZ->C3x_Cins-zHD!!+7v7P-HpkwYP9~hjo*D1N zcRh>d@ciUSWwHW4nZJy?vhlY->*z%(-|0kd+1GE6I`-!-5LQLz4l4=;e!+REqD>9& zjlfInC<#dNjUAIpArd z7v<D!Sh{#nH3pYQ#bZh zuEp@#@Q0$s>P}12>Gh2C& zmwa>A~1zQLoW{}{iicgcW&G7TimcRG!ckBO1%@`;h`NPQ>gFTk8VK-(R1|rKs?E$RDJnF zIRh^N9+;LXtpH|**de5(7B{l~^>1EB*|bz)tl>P|?NyLmoyBh?he>6bvj-BIJJ&dd zhTHV{8N3IWHuk|@ygQL=-1)aNE5qyF&ULAY{FSlmwckoEWhG7mUIJIVS}|g)g~4Ou z^P#IMxmRz!yK4AiW?fZ{@zqIo<9$VOc0%Wg3z|LGlF0M2#%L49#a>gwEA$Y4Y_D+2 zzTCW@Y7U&nI*Bp=(k#V4=ipv5i$1Xp4 zNh=>xRWGTLA5)etmT-JJaKv5nQJ}Zl(|DV&ELh{htEevyKMQ?SsffO*Pj^1hxQD8& za+)#%6mkBpWUVG3KR&yD`OM~-Ie)RCMr9&3q`pEOIdJ;kF`0Jx+%KMex6V@6i*K)v zLi~#%39v@ucPRVm&pYFQ<4MmL?+Kgldw6k`-UVx@-M7f-)*}x)0`g|{jU)pS_E>|h zg_0CCzcxtlade<;27IK{Avlvtts12#eHLkr)A*HJhl~eQt+1>k%-#oiR>JK4&3n4~ zXn(Gg%{u#{A&}m{>#$oGEozy`V*iE~W5I<;bnw6%!;opwJijcP6y}Yz0$1oqcKSc< zKVdzCsEcc6lBt&~Mrl<_nQLYO^^^XKp-)=VL-Xvxm)$^!rf0Qx-+bZRM{WqY@28B4 z$NV=Mn%Q%1A?xV=IbT%<7buvnF45YoJ!;QDt>_b)D(<*FuIS>SVR4!&Q9n)w7h-o0{T!%hDCrSZJ*`r5il)-jOVyU2Y zDx3e?N24I;g{uq_PT<%SJ`&MlLYw#CS#%1q|G>qJewsn}zDKTSscJ|r9dA8Zku1IQ zUX%gka#n`=+NAlY!`EEl>-LQ3r2Nz@VaqZlug7tJ6CZ!cFi|@zwx`@RtXyA$wk#Bi(DO>8dAle{u?w9t~aRq@iQ>luSoDW1iVh0O8V=@@k+$wkV5JTl49 zZs+z}IpG`?3yUgopWCW*t(H2d7Xo((>n_%LI*FV9DVK>pK&ObB*@CC?HnXVlQiVpY9Py{VTGZi!a_R+$W?9>-s`e8 z`$C-KV60&W%(8`*BFSsxppOl6b0jF6Q2d5?Z$~&WjdU@C~G{PK$a5+4RQ*t zka;BNK<^w4Q``84id(r0$C2W@JZrY?a=AkUz-R!MX#>BVj#(8eU|(ZbK>8G$ z@*p3d58^ZYa9tkdEjjML>_;ZZs}Qk2igq<~VBFeHbu+dhIdVXlYwl7SjfFwgKJFoB zU4Y?JQlob{&W11?rasHj{EZAJTouE+N^#$>F+VbAY;-=Y@tVyGv0)5xZMxR;54;~a zZ-l$2U>nf3rf9uFAZhk~*+*fmx+V%!oOMZ3#5mnj8Ma&ChhaH&d=05>^?A7!Z!&J`G%#EA- z>%*JPO_Y4wd|{gDi3EVE^{G;l+mb4>{#k4OHp+%j7E&nY0C?2FtKnMq7QVC#4LAMg z@y5cT;E)#;K-iCl8}Qvn5AV7+Wj-OCR!oYw-E!5W-fvS4mhB%yWj>rXb$yi zTiq@f2kf^fi!}!06s^@KPO6tBYG9C2g7oB%36oLaWpehO-+jGEj&%~W?yc$dgUz0Z zu^L;H)Yqh7aYJRXYv5ljD_N&FL^aV{mGJlgZA^Fjp~#&mc=q{kH_iH->urBX_R2gZ{?2t@TD|=dkH}h8#XpHv2tXL!58DF?xLV5Kq5197QH)|3 z=`@o{jBUj}dV7Um%k^^c{C-Fm(IR*FK6Cx zXid7z2Tgm|@jQ*7a@QFzYwkhvbfYs%Ap&LRR8?(lZmtumWk|99mri%B`GjVguf@0$ z$F6LqL>R>!-xe1e@XV){!&Io_o^0zcTx%6Tru>*ox0u8Xg|4o7{nWa zH4+4$O=sIS=@)2eQ*zqf^atlXjda>E`pv+Y4F)|yQe+Yyd;Hb=vm<-02GoDX&TP1} zs?u6Hhu8t!Y>wo^6elH`I%M}EueqEJ6RFm}vN$^*YNczzYGlrV{kNiLt?U-Vl|Rsr zh7nrnlE+`g;F}hchaSJ$PxGb<*a?j4<{_vsEuAl~=YshpN&;v2^SqP<%v7AnmWfWN zOtKh*IYF^sg^lRM$wdwIn^#;xSl7Eg?c4%L0+mtZKkU^clO@10Z6YlovhF!^*R1y$ zvumy&N$e*vw5$V>@90g+;$;0ymeZzwmaNLieoDx|Ou>Ai(=H3z8WYmGx0Gh9L|ymo&FRY6N9{|Lp(-0^R~ z7&AutlynQe4b0It)Vx$qucwGQnY5P6KNzN`4EfTC0xd?#WL5&-`DRLm%;D+4{C6`^ zcYGYZvSuV(N0){^2@iYB`-&J#0gLu5lsKrdu-UnKvhw1{@hak3O~mMLFGI?{qLi7b zuU-Y})dWF_k8<0l_1KfqUjXTthl=5(t@mpkB;U2KV*b3RYP%m)v-x2*bU%WX_=Vwm z#ryeQ`TTsIv-96Sgs{Pvd z5I2$!fA@L9-JKlvd4k<_-s$@z>ZO-@|Kn4mzRzivSKuA?rm?j%vCbC-e35j8P4$LGUX82AHJS z0+b4h?z=!bcq6p`;Lf2bWc|;T178T81UOi$WF!NKt_TB^6R)PF%qJ~hENd)HS?sLP zLdGJtAYC>Sf7e64zsxs>Xsxqqu+?}z<_d{*b-6WYwD%R7((?ZV(H`4M`~M$k%MBc2 zvS<2NucXcX8_$FPE9G)7-8!DZ`lTT~mabHI3uE|7|LzS2+B%v?NvUfz*c9G zqNATeS=KOfbfTA^ve2%l@ywgG7kG{FbC2{h0&8v)WUSkTFhJB@{N<@C#qHW0{i{*= zPvP`Q*Z1E|8)t`>cY1GA*AR6jOF(||+a~6KeTJV#d1?>%WD{yX9%#-gb%*~2`7IJ6 zjCX^}FgZKNCWqKWFOjK@l0{64egZ=ysWonY^fq-4t>P2@T+3&*YH^UUV$cT*%+_jD z0Ml*3FdZF#v8og+UUs%cL@f-eZDX~pG|Jjg>riB+{a<-@|CJY`!^o?B(c+-hDK%=X zep#q59tG!|y#UVz7>_P~Hm=;Q`*DxSo-IADYj+Q{>s0tp()@pts-`dSuE(j_I4|s{ z4+D2mKbHn1nMBEa(Ci$pN-n;W?HoQvr1~iTT-?R@%(n8Rb|1NR$sj%l!&GtY8mIHSo=X! z?RH^VyH@(XMb^H?Y?D_&&P9a%S=khVS8SB6;X-Nhg0(_)wys_OKe4K5L>*9jr?!n% zq5j2`Pr5`WLAru@mbLZ=p>G9kl)ieUwxv27YklO(BU{6N5>oz?@MGaW3HbHqnP5J`V^?#VvPX6=Pb^!jUDR}BYJE9yK;`@+5F z_ZiAeh=e13gR6!gt$kIuu66f5Z*RmIKiE{sf%_iW2(yT%brtz_&(lXVZmZOsV{A-% z{1xrqQvWL5-6ihZ0ji?p?yvfa<4!C7gWcXP96jF9lRo2hrCR=HD*1B-z+V!t0&W>T zgJg1d`~%jc3Xk}ICZt($UXI9zR((koPT#51^|1~Gd}$C^1VdTfVH_z8`c_5ORwX*H zLom2V8{((4*MLx(M7UM(%+}RPl#N}Lm(Q`<4{6(`z_L>2|0_902WC~mnrc+RI$c`( zL%UF4o2a2wsSJ^)v)fP#w9{_UNs+E7o%&BK<-cMIWu=rR71~znWp?$YK*bNZply%k7+#v!kwa;p>W*c(D+FM#rypdHZt*B_9MqC z>C%ZveF+|x(AF=`r5BmHKF*RAR!m!oiL3I;sZw6CGPZ^zrO6}K|N2|MtTIM3U)vg7Ix}sI?M(EFsux%ddU_8<9;Bp9qz~VP456Eg^PHNx=1)4p zldr6Hs&r%2jS-^9_oCF81m<$t6gsv2bHuop$g@TI%tukX^X-GHiIHbd5x<(W>#nhid&n~Rx$3C>HAA2r4`{+d-uOiQ07PhBi zrxG}oHXuHT{%Dw%8Q0D-%(wCUX}?tcW5PP-`ObFz!Q{u~%Mz%GN!jHP+Vz$2m(qWS z*4Si$c^$ugs7sa^=YQz@Ro%IEbGUDEm-FcvP5I%>i zk?@A5=XLAsFRY^>Ov1$fvj2$*p42TquzuN2;B~tuo7Wa0wts{(S3M;u>9VLR?x>eS zlS>$U8ww2BOVkuu=Mz~Q3iR1a)D^5;Y05Cba>Ha2(6_a2@zTNN#Fi!R%U%~Mk=0z8 z+IG$O79J{erRcC3>5!FQEmX07xh(z}QxwT`)7BM%B259@Qo{|864L0Ori~?ZZI8g0 zuI&~A_m|MB5W99nln9S|>L1k4*%r>~rw+)HXEqA|_E@5e*S#5xEhm z5it?L5x&g)Tcqa1TLk8KTW`&=w=m7$Y@wQ?{CZwx`B#-)6lrrIkncHoptJzXVVI*X@V-VL-^6-=9pp%^ut|lzj^}Kh-gUGGxm^T5tQq$ zv7bC)o5_nD(&MsSvy+31f{jKB5IC@z8p)~qL>yd+CJ6{5`aiDnOe^Y z|D+lWMXLZjf3y!4Ky$=iA*~lYFrAvr>FRkEjEnYVE2xJxcpl9jus~MMtLI$ZW18Q? znEz4e?qjk~*oV4h)2JTf;6OA3>_$=t!6mcmS<}=pgUy^CVDK`UJ7AA&j5p1>y4y6j zhgMzMtfmJqxDag=03|Qy(zC7pW!lt3mH%Gu2b=rvLarIpz++aSs3;L-!h+>FDQ}cP1<8*SJH>aF<}17 z)M2bpUP-&+Idq&EDySNv3GPFa1w5n6V#ktF2nw0G%xon0*aq`%`JwFto-;4ehyfgA zhCIy1^AcwoSMf}%747pV=;ux0hgyQEyk zKBchqXf7<|&(Kx%lUVZ(6`Sg5Vn}ncm!-3iKtaoM%@#&vES&O zG%Q&YzQdiI(J<>8K#B?`3J2mCP}B(7xsGi$4`7s#g(eXd162$%YJ}`vC$`!LP@|%V zQVWU*I1CJHnyp+{w_>@hTqn0W2T-EILnWoi%~tbjl=0Jx1RNb!R_g0*t!7vDN?meY z#XeDo2{DZvTbDhW4$Wp~ zi&B-HNo{5qi?YWRLZ1at9MDz(wWJQb{g5ByTiYLglTV+ZbdZu+#kOYFzasKo` zfk#K2C5n0wy|0dtgM0tpk8$qwV}Vo$j#a~&7Blw+NMtW4{ZfF~fn&v?X4$T89MU$9 zlb-3-^Nw4)wQJldT};5=5xNXabx`PZ&{-~rCaURK)Ga_V$Nkc=1;`x*mK_>FPVN#f zUO7E`$>X-^yvWItBH=eUloZ0k9&S{Mv=C+wuY;|r`P>>w{O}^2aSFIdy^GCZwL)%+ zq*kyh5O=O)_7oo%I4Q`$#hwPd8K^t|haGKDBT&oIXk!p=bhbL;cozH*n6>idEs8?dlb? zm_Ge*Pn=^)0U-wuy&u&hW_5i;;T1Tm6pbP$E=z~iqh?K-Tv9;orS0lPv+Tal;V3xh zg;_?z%^n`0o%g<=6PNzYAAQ{6+c-=VD#96VLcd)G&02cF;qEy5lpI2a9?d&0%bSsX z+Tr$bOt|?3l_G2J@{F8>*W8+azktaq`^3T@H8wT=;Jl@z0!6!%oN@J-<@M2rkKkw) z^b!^cE4inga`l-N^#Q||aoj2PFmaG0dIQ2yHP~>XD2POWu6nHBq6%=>Fuz9xxnjl` z!JGL!ZTDu@3S$%vQoT6XUcXK~e*p&Qv;jOwcX*bpB1Y9x%{cmuWSW)KNDg@rR`sJK zsWlwimdS?RumheEb`n}fRwXak7SF0x?5JU?FUO_PLUTFcg%N}m=mFlTT`H_oBu=g5 zST}5KyK*@^`9?jHC z4q`)&NFB^nLyk5>*Onu<@!i%@?4F@r=%}N*#%AoOQK~=3tr1nTsE6;)%u@b$S3qz!xq&;@8WW)wnZ0O%DtC2YNxG|-se;= zTSblPq&jk}8hJIJcr5SMu3E*t%Bxwp6oX5TEi3 zz2Drk7SJKu&}Axj(LKMh%e1qaZi`&QTfLJSDPybS-!)?vEkk~B{8aXZ6QTk>4p&QB za#B*jgm1S?TQ&0*=|f&|lvE;OUcM(cOWUr}m_IGDhd#%lP@#+A_)^>mZMn|n*Gy`M zGKwn)!teQRZ8v9Uic?gXvM0oi{(hs`%5ZoU_|jgyX#VXwv|TwQ7WYW?R+P#wBekMQ zmO6Vu+%aI~q5jhLY__L3U)4Js3sWq>@8Rm=@VEQecJmNMToP5WsLF>dDML-6ZP$hE z4CW2jp6$FLhPcs0|F|-@QkU`V)*tm7w5 zvuVU71HVXh!TG>5d3W zK*~p!L662DA$L(TGwi+#;zzPafxZS}>k(Cpno4&ga!ezdE4gS)&ASVNMvyd-i_ny? z(x|G%O;eOUnt66d2f>i)kRj+}7-{5OoTi_DB_&}~4v*^yXx)IV;=Gtvx6QG=_{GTsQ0y0C6|bi;y*XvQVr;Ti1ov{}|ky)8kP zNW{n-Xoi^06fPngD&6Zr{)yJE3-KTtcgej_;G09z8MXp%x$O8(8Uuod(5ttk?|9S% z7-@nf!$JVqq(nTTKoZ(U%vPFhNyZQ*0E}RUr$AfoI9`CpQL>Sx9?O<`hNi$mt~mZb z{v)kn9$1G|wMJ_+mTtqz@g+2~5;7st08&C8VN0buJ-ix8Gl${7@rN|L5|FSl%ogg4 z_dB@+Uc)NO#Ih%VWx`q!OV#0@@qsjtk~kq008oJobAjuNNkkEDxMXYy1tF0{YdhS|L4M3D|E1B}w6u3nHP`9Ch#Z?4;a8X<=3Pe&zvQw4E9IIkP`EX?-i8`STGJMMZzd}86b9J^=XGM2Mifp(?}!a6@jRXt)=;L zUs@3L17pBMvqF1~5=m)A+~el(q5240(plCbPo?y8eWV7r={$oa9mj0{gL&8~+B2}edX`i{DEYSJ^(JjSX%n+Bc__SQ^KJ&1l=m=6$ zURH>iA(zyet{#Cq_nyYsZWy{tW!?Q&7zaaU5$+yXGc07Z`MiT&AdFfRrWI zBO23wbBmaJ4I0L^kh!z($tbmj++Zkc3DbKh9VAbIaup>OL!2<4@X_e6eH39Jq=7h;Psi_gfQq>`3c&2Hx3TN|>Q7lDWjNx>+?dtrnszE}>tH#6(a z3mF)(ke5!P74}QCdo?W-)W`g@#llo+an9^l5|@ z*F&we5kbjcS%NjtilgO7VLYr>-n~>Tlqnq;k9aps6mmjoF3g)Qy$K=t7(O$9^#L2i%&%_N zd&2wYibwIQvU$PKuR2CU3g0pLBtSt0jZd&7NDatJAnvDex%FEDNv0XjHc``l= z(*T;5N@0cyIJs8HN**zjY{fFFsZ&r+kfa7qsTH&0-pYr;5^4a$yOi|lJk+}xPM2rWVzApLJ$?nt1RCATb85oZk)TH5y8pf z$N}8rk+#N4$L1>pyb5=$Mbg~tQMTGj#pc2Uvo&e5EHP2G8e=xgDhj$9q{T#XgLP+e z;uz7oupAzBOe3pIgdYpiAwH|3HnfBpn5y%mR>P_mL2#H}JFf++Zv z=2`8v7D`{9g-vqhgjDZe#ma*y}nfvlDpCfH~&yQ#i)Dva1GB;2E{K^r&wRm-o9SEg=4HXNM~O!>Ct39L)XP% zh%oGf%k7CwZGJRR{XNFJ*B|jt=-+q^j5PNvZS~vUC^D*?^e<>q@!0ZyM*I*gQ+rwT zYm@RUz2#8vATEnK8i&>fU8l)YiJq%@?^BshE{8we{{7d|>?8Ej2~^*y(W+5WuhPrc zE3;86*DTj{P+QaBEX28Jscg=*(V)<#(4^4KP|MKD(8$msaFKD z7l~|(^pAXuL?4$$FBH`h)ezMYRTuSSP(4rCf^D_ryI?P1 zBVZ?BDtU`?wNc__xv|8hUH204^x~P7R4duMt9+vyYTxU*8T1DQ2QxfGX0zHwN{E#L zKNf306sz}b3$r0!=Q|E8!I1P`jLiZU1tIov8FXiN86ma|`~{5TQ0!hAv!rV967AoJ zie76z-e&BCqWAWtYN9uY;!$C#h7|Xv zR!Y2?4B3~Em1OE`^APah_Tcy6^$@(`y5hUyxe^F)x+4t)L}4oe6tPDE*I~F}31RAC z(_z`XgIo(-6I?&5XSjN}rnyGAmbnJ_=qLkHC)4cGAbL)EW@%1oW@)x*R%woD7I*0G z*q4AyY%Bm4b`&5G`w8%b?HTrkr<-evYnW?^tDkF*Ym94cYI&yc{%Y=l+=zW0xXvyH zLen#$481dR=B*5H2e&gDwj$6Mty-v55i1*r^m4z(8Oiy9o&g zd1YFv271?i=_Y)$bu(DUGU>YDlL z@#^*JzUtZP(dyOeq3Xrz$v@nW*xmqd>|4Ms_FKSP>^MLiwkkXcNB>+lOE|&65A0{~ z0Pf@qu=0WUI33vB<7WaD+3VQdf$r>=z)|26yC)P0s!C!*YDwxqYEEiL3L$kOH6yhp zoe%K%!;;T#0gM7hu`2>C*tdb(?D#-@cFHtMm;=ll<_tCk+mtjZH7M07HT{hwg~Fg< zC>yj4Y7aez?m`KnDK{@yTou>~>j*rO3wIU%t0=(--cfy>V{oHd-@S~Eh zlMzKug(?IGrqoPnE#@pM?`7a_574=>iK~MV536DFJH)bq9UB^o6|lL3_r*-c7a|Rj z&k?O#B0eX&LrDIjD%p0~@^IP}pk}fwt*gLqaZ1YZXXe((I8oUO9(KXXKW+i13bgW$&J6l`uTmF>llG3t#+MTqlt9^7F`kghnpA}#tEyUD7o3YOcAiOVgi$vMn$@{2cC-2wJ zMOY}Sf%TVg`LJrYm;haY#}KT@jze&7cR|YnEAf4pmsvfORKJ($^JuEt&iyB9ulwjHjr<+` zvroiIb45u>0zH4t`SHgsv+3O3J{ft|FF#6emaF{VG5Kye<~W8!MV$T?zxsZ>O@-YO z%yGGOq}h17+=cYNtMC9nQFA0;e43SCe*aA>JQp+H|ECJ?@PCQx{-=#Sg>f=CQJ)}Q zZ{+{SfZdR5{v7T96OesN>u zZ)cuP9Q;fXyWI2iUiY5_-L4(^J|=y6zG?gQcz^cgX>VKb!@~h_*W*=P6ZOBnS(DEf zH+6#9qJfK9u^;~JKYsYTIw}6|SkUD2^=;SRtEkB?-k(opaZVLahrz30_KT%`5OGMo5tnT&r_Ls|aUo!TJRro1Z$FF*^&sRd5 zzo;9(1U8?kR=sfh4iWZhetK{_e#x z@?Wt7?WjCNRFA{lE<_p~`eRQ+7GmLfCV9aUFLg7kMaLQyg%h^o;w#~Z+H}+pcY|{S zZE@*J8jb}t#k<`;ztc|T`Cj*F%f8oRnp5IqKKxT!w17Qnf^_7Xu9-N46iGl;PMW}} z_Tr6Da_lGHA@hFe4QD~oFDFB0<6B0IRB2VwtoUXRYRBodw>cO&i1(qduK&vX?#9}k z3Xn^T)WF!Ff7ql#Pm&=DepjR{S1^YpI614=fcAqi&0TtVE^|owC+}Q>l$w;?*Fo%( z$`PrzB%AG|BzoXzPkI$g)ad*yy|8}JH8n*)Ue1*27BHG>fR_wi3QuG?AeTROtd$FT z__=A8Ebf}(Muzzjf|}Q^;G&=#iu}pJou>#lhu2tIs~Uq!OPh9M?lm$Z+CG5+$V~cZ zD=tTa=69lv$pfnE`<*|@FF)O6pavR&)%trX* zoq#h2xn<(F3D#>JgK*?J3cmL1Jc7RX2O9DAZS=jhsQupC^$GXP7?&@Ku(3R2@yYNzVMHE0+C%j=Y1m#Zm^ zDR^f_6G4q{uB{B$i{tgh~o}_@NPv% zlDEt~pMltUSy>ui`Z~H;AJSV=lGP7MG3&yoDAPP%bZu-S?<@@ejC|~83h#eSe7eU$ ztnWZ~rRcYKx`Z;VqY!v7&!Xwfx~EV0<0DlDOkOkjTd2ALsFUj+H6ejWBCn9{yk-O^ zWp=JRgT()vRL3$c)dUvslU>-9oKL#f`N{EJcV8LT2-|2srAu^0hW}|)q(HQ6-@X_{ z!YIp`40o+-#{2G-3Hk&N zs+~6V7tRSsCskmLOgkWryMP1|&8pTeYTbI5`t%0atVzn|1ZlGA>_*X>)ttG9;h0K~ zMW~5hWC}R%#%7+cnylZ~sF9^M2$*Bc1(FrZu|v(H7d7Sm-#V<%JyS@0^Zl7)xU{HK zq!izdab1d<>|ft#DHhqDa*5DsF)?4<)7uH!PVa2A-Tj02I+zMIX#cfU_n`1pfq!iH zjr@AUeNSwE&96yYH8mX_>SapsmYSR)%2a|)pRA_F%NHqr%S*NH_;|fv%aT+|-b;(4 zzluY3Pp@8%mX_Old(6mjD4GTE4qQrc32FXmsZh2iz%W$Y*P@vg2tXSOE7@_o@y9~r zZF7&>PH?xyTXQQ1HD{RGK`-Z{hUe(V68$()lz>i?mDIdyI1XzQU z!B#3h{it)OyScqT;TxRk>DWNS=4C6YOeYW`ik1|B6Du&N6T;9DMFU^{-~K3H0&K22$egs$wW%P(S5~q z<=|-Mp+U_81Wrn7yTKwY&aJLzrpl5n*#>1IM`sbq7g0;*tL8u0+L4e?97&J0KrQbe zEtl!TMF$P?DL%N5we;KE(*x`_E^PA+?D8pE(J>kOf80;Vr7>y9S1idFE3O=yY=B>- zkdfd1TYD2^T{PeD?i<#CB4NC~=&U3gt=*^V0Rl?D_7f%S>L#-If3S8A z!I?PG7XQaKCbn(c=1gqcwrx9K%!zH=w#_eg@@D%sZ}qCWs&`#o)pZxAfA^k~roz7x zan7Q4-+YfCLl-$!Y9w7*>{dBOjf5zon1*|4`{${<_dgs8Gji_N(xt`?iq`Zq?(7r? z^R!t1FK}scfwAPmUmJKFXw2(-OClpw+YloOK5MvKSf(^>F9Pz{Rs(;z)Fi-Xj`gM1 z52UNs&FYmWst!KlqQ<=xzlJF@HpC9NLeWShgbF13Z8|Vj43ec?Xgh)x{vgm-6Xt1G zB%4|FF^SZp4v^RU2RCWX$ViGNnJ=emQBKm27)TDa@-RWREcTB}I@ZhQvEWg@I7xCFaYR=??KY94p|d2zcP^Tkt0HC)P|TlY%k<47RR*3B*n>HI#U!Qd(CvZtd4> z+>B+&m?$5Z!O4VaDvHuOJ#PZTf%%z)IFm*#*m_-hL1du24G8T*=<#o>qN{G6GstM( zv?_*XUL;74Ag({@>7(x`cvTUSz}`XQie9ofvPVF#`Fv~(TcxA9#`TtAwO-O;dgNk`@KgfFo=vGgos@j^xw zRo}{BQyK4J&#Kji6IXkNL)hgp8cae!K;S{+2EpuqFC3t4w!a#$x#fP>mH zp{lt2T=pL}L6sRmiecjZro?sql#l?2#G<0ZE;VVTki$iIVhriiV-ff+a&9bWZ2foJ zZ{U?|4qh>*I-+z`8e}1=AZj#AL?yT}R&Ti|BKVMPp|F{3!$PzOU8C9g*|<#?ZkmHq zm7MAcmzh6<*vfrjqh4*y*y}s~0h=tO!t;?8Im44a+7`VnkIF#AlsxgzTDeaO5u4(8 zEu5FRmjOKU^hP9^-t$CfJD)dQf{s_lfAVH`U6#R@EO)Rm@=nKj_o;+vAfJlaMAyp~ zD$+=cr*sOa!FnMuU{F`!z}d-P@c&$s&U284>kcY<44s8ijAwkH3-3b>Q_)pslFe}4 z=gZ~soO*NulQ23lUUJ~n^+*YnW)gra$p89OYO}ZoHjZV}PD#um zMn}SOW6iP^G}gC&Ve5&xH73B3!xX2SJ3>#<~#GWi6^bt()a9AEP*VR*}dXNafXlAsF zMPMbe#a@h7Jyv z%t)GqOLk2{3LP?dxgNRnjm#wqN(bBlBM9vMwhn?@`yAr9Nn$yE*N6YrV8untPal%pSr-$4DZ4ETig z&lgB}(r3aVWwL0^1h70|H!I}%nl$t;eC#aE3~^ou_R0Vxz759jek1*FJuX08 zAc3d+-+!}K|8HW}#NPP-(&N`%{&J-DS%evk@cv8^pF<+Wuamk&J4%LMUgTx}N@Wgl`bd#uv#6ze9j?j*D z;an}tBE?9qTp{mT#mkn7YE{ct&F@;dJ!M{IC5vAe=il2mrEjL{-M0?{v!~%-J&Fh8 zyNh_ei;F5BAI-qpZ#(lBL7R_{zuT$*w(w@bPeb!b{UX7i`{7!j2YbIu!k^0$!K{rh z^@^VxLjOxf!tKwh*Hu?Gt{#O~na{NAU&Z<<{?Zldy)FLQA9082xj&(o>RSw?zL3|U zmOwu6Q}C3e*su)ex6hzVHrGvDA8Vc7js6fS0}hV{u^=sJ9b2_ z5frx4M-+Vr$%tcBe@(u$kg3D7SO6(6ynkq0612Fd4>Bu3oKfy!+mWv*zO{fyS7>Md z0MT8<6lrwA{H!Bt{Rw>oL@d>GgP)9=%{VO0^k-^_j+8FH-Uls~cqMNOU4ktYDG5^& z04UDcc(*Wh!z~(E*YQFHU;kwYzcAQFI0b}lftXK69G?bq!tb)5eKN@f@()EdsLCmd zDB#Kw1ow`xRGv9DhG3n^k9Zj;?awMYePF?WV*+TqL8j5_bkvtcgL?)b5D*dPQ6k0e z1C~o-is+DtA!t*PALnf%$G0{g0*HS6@_?EZg-(7%HskE?xQ|2A3OPWroeq5 z$z9;Hw&}ffSZeQ$suX{hI4XrQTyrxOBu@vl!^6=O?v?Igb5tkkAA zNBZji@uVcR?TJ=LLbzyTST552V*2FCl+uD)@4PVVCYW|9=VOSdpnK>6;?B+tRn6$m z&=T`AH7iF1JtOs5kQ!xMy!x^xWd2Isc?pAGzv|yWPl?)Nku2A>&}}p8wMsgPqPn`b zh)dYYH16J}BJ@&mhjpllLS3)_L+3ZM5q9TWI=94!+|Gt)WjBs#gxqJH5tTdprASZy!h#VUIPDxwak9ygxW=(utWhlg|E5+b13%oat7 z3wY0_rvtis3n*MRbH0*=2In{oQ<%e#R2chD;-ExAGmd~EUd3kY4f5yIFQM?q*lVqK zX)-kq5O1=W9l?;BG~Qy9&Ho{uqgpLb5twg^K17h~kb6%sx=0r#M82s3Y;k?elJxsn z9aIW%oRv7b?kH_=26`n?PPTU?6OkB$dLJ{ApX?}iaS27!M;~QE$?HNzP#Jp$t$coC zafNaM6Zj8gc4W^G{YGZD;K{n<+v4zNu%OhvJ37tWI|d@TZV56%3(v1yBvpLqsyDi3 zT;X>>S!te$rNB3%lX|HOV}cJ-JFmOu3#o(j6)of6b+0P<#0qsJ9-Z7JV{=@&EUY9eLFG%oUPAfVt=qj0& z)yfyLjF|_#EpK0RG!d^O%{?X7QkrlU=qyLz zDu>CxE}y$itiNr~ea8>r;lfu8kN29LIBFqLndtf=SLnu?gwdXsBpkYz49q9E1s!Oy zewKA5y;-|M##ts06c!xgb{WbA7M1kU+65F<9i@m<{bVMcxt9*39KaOz?t;f{@$NFM zc2ijk!5Zh5byh}0AZaNh(IX7b@)cdb7T1CRmbsPyQ5VZD+DH__)H5n2yYF9UgZ-#S!Mb8OR z^-0_rhJrOLUaHWC@mQ8Z{n!=$+;5Y8jh#~@<$QkNwCi<6is82>;; z#}x!^k6Bx3Ks4;tpO!rCmoBFYHao;3Q<;zj!u}xEabiRs5d`rb?m=}El(@qft9U>+ zh9nM4jj7x48C@rjbuLxDLPDacuN`X!vu~Z=S60UMLs8&~o*;iBdW(|DTI{TuyUVtZ z$S9mwac60?EU#kIV$K3Mf72g}R2B}+0QPSS@vy?Iq&fIPb9@N~o_n;L)~`DJ-zD^l6~xS;iWE-H^5g!!JMO=JRCqtc61bh_>U~{7b9p9(+RfzoAO)tzu8kdHPkO2HUlZqQWa<=m!E&dEVYU z7IDy*aG}}V{{ELw$QP1fQxy}qn$V$#R>r;3=N3&rsdJvB4W7-fBn^(^)0p5l>8cN) zphnwMOa4>Q6eGKaimGOa7t}E5;&DySkSC=E$0!=}i`Uz>ONVU$ZVK%QA+T=<#w({L zHEMLv_9%Oeg^dF;=XyV+1LoO+Y8l%WZqr3_qBUecz#0dfdOtpML4eGOG1SiJ#nWTp z?Fb=BJh=`Md?5WdX!p4X^$H(ocnBu*E=09sh}+P-jQSb&7BcZppfW7Bh{_^XExVf; z=yea81LcHql^V^kPlU}v{O})B7VXX z9{D6W-#@?x|M+Oig!C%@8UZy#dI2krpaxhuWaYSgMeN)YI`Tc2X~jDBRLq`G4CuEZ zDC+|S)qjpjWV^I|?s&*2p2`oxdf|sSFi?y~L6*oUOz{lgIaw}7=rA*{6qk+{cEEv` zlq~F>KE7j*4l+ie+|T-*XAh2Kz{Hkt>4#+B&Yz^k3q;j(=A{#w{6;CPUt@lJgpiG= za(m~g>;yE@$&wFC87rL&jvNKhtiLjQ#8z56WTA*L2i}-tZhfNxXN*eZQ9kf)gm*60 zNF~Vz-Gq3D?+L_r@PbWmz<2jx3};#JnAOMX$8QbW`4L(8TXkEI?t>k#R+7D-~o zDQn@Nl=8^nCmDqFw-!zGKqtB3G`9)ZVyM2Zz_OM_G+CC#i0I{laj?Vb>`%Hv4Vj7< z{?2s<<@QF1ESyG^>KX0BSCVpN@Dh;|iz^WKK--43WE zauRId_TtD8((oXB;c(vS4r2yV898c%SMY(uy8CEzh5M^YPwVjVulRXuBn{3TEWgT3q@y2nur7~}LV3Ts(Py|e zgaZ@-SE@d5fbt<XSjC4UvPeehG}_LS*#Og6R__DB9?>Q*}Yvz;>Q2n^oiFjAUENN z?N>p|gWGHii{CAPKfB4)KEW#ZLvhaMf0Sx zfw0l6uB>xgI~+4^#2<+3n8i&od&1>zLvYtt+FL6)@pmHz3dvN5 zNWv0uW#1m7A@b%X%v>@<6Uk)SO)X5(PwtRx{0IF%rOa&-<#3F9n7^bd#PaX{%2vN) zHsMYb)W9v}Zigkr%LQA*L6bSD{(iJmmq^|Aol>12(pvd&XF`33v+GD;M2oD#=xiOO z-MsEH53m$&B_%L4Jt3yIXppNvne)f6jm3~m4c&OB5L)<6G5~%%Oa3ds6t_3pZW+DU zMKa60=eonP%P8-8)P|@%qSitvpJ^L+3K+X+l(L7c5l)%~?Y3rvL_FysDc2V0(?267 zm{=ig@|tnB&yQGpF+s_4Em3~^A6feu)8Pz9yBA*X0Y4|{q~<2*_njPSnjnwO%>^2TIsmf;SO;h9)80_Fw<;KO;bPva&KmB64{&F111h z3Ya^~GlX|aAsD#W8_P3HDdA=uTEIwDT0C7X>F)Is!~yNjGP>q}!5ZrrRyh5B+$hFT z1PL1E9}I#>gIkbfwFOKf6x}~~<2mwmwp18}v}D6l@=SxRLe1(@YplA7VpbSk>kiTR zwKF1`j41~NA{^IFl%JOKxv|OT^|Yp)0PtcdrE`3+O#36xdE#hGC-s){1tYW}X%C2b ziC!9lKo1D7LRWaaLc3}a2`3JOxL`@^7+U6KvN)}PJ}g95QUQ;?zja&Km#8s$+#>bV zb33RUmLFtzdd`Kx7#s%hzZ;(f-Zye?7}P9h*Wf#X!e=>Vjb|>O;sb}+#5F;(dzXLE zU1)|QxMT>L&Y?Y~ksooccO*~1a$hRgxIq}0Nz4UO&A@vFREryam!T$n?mKt z+vFIyx&`#aSH4nvpZgY9OWO!c`>b&0(N2SB9L06siIso~dyVRz?Ji3qa<9q23Z`S$d;Fa<^lO zMJ5v6Fh=*jtzCZS7#nZ(k&ql;-h!!|H=ybl!9R#1ff`s-sIC`e4n)HwG4~i{(9GhN zy7u<_l`RuEv9}{FOn$kK5!%%JoInEK zsG(mN^^Kc(3i>K?&vEz#o`9xR#29*xet*l30r@WVL~bfObfP}y*M0)ME^VsKO!!<0 z??WoYf}GLXc7}07m#lhAY;aCAJXyyYKClWIx>dRm89xk?dT+M#D%CSB|41bFT}Rg| zCi`&f_><3jdS^gDYhOFiRR_wwmA}`d;`axx15JI1>You}6sN>AhpHsq7fF)m9_+c+ zd3txqh|(0JMwMgTt`kMeI33B6OBRjgmD=hc8qWCOV9JzRl4=X*kKrNicMM6HIOrCm z->w9K*Z5SV=(Wl>tk_|$ZUYsVsBq7mQZo(^E%noSs8g z+7{VCK|kC_q|=0sX`4r7$~z&)9qMxO0wK@3(CKQgo;l>Y^F(8@HFEAnsz5QB$)5TZ&3urdw}v!5NcasL@`H6%4BF}4W54Gs#9sg^PF^}$f6rt* za!P&k=2|$9ft?%jm=-!vjoZm_IYtf;InF?=QmX%Gk63H(s3B^9P|LV3Q|r-sJYw_L zNyLuYvR4OC#x==c4hSer+>-H5O5zwOOvu!|1uAI5O*K{S<5g@NInF)aD*A$rmT&TOM18zhB3=Q?iR z?he`eO)SUAj+kPfERyCVgNxzf#H*-qC@(V9b=S4^9_U?zNlYND^o7a|mQ{TgJ^oSS z4G5-@P_*xv_DhWA0;}NBNo+iN};pUg&#E9)RfwMxDt_RaXob{n? zh&2v1yzSN_5g-^S{hi8q_giy3BOr!&yd~zsQ#x@T@w_*L z?n7cfCq?BAq5m6qFIFOwk!m%j?HVzSs1fwJzckGu-xpfYxN1SUy+WHP`3drQ4MBn+ z_()b?l($w+xLCPb+~)>FwEEcR4bI`j$Jn3|B7;Irl?<~h*6yHCj%$Os7KLg2&Q0-y zv2c@d7;il2%mR;``vNcgk(_M1!h!B&#YRE56-s6M+hJ;a^XmqjkvwOB4t|5|2S?0q z{GWl_K1|l_{2QJg3JRXFE6y9KlHMMkUx1Qi3_8v#cXXgB6W%y+1lK9E3bE)C`b7)& zqLT9U0L(}zlEq7gL5lR)@?=$1JK5M7IDkh>kZYN~y)?Tq!WENd7CB6T& zt8PrL5&u$R)ORlO=LtgmmM`p?qDpu*{L1o9HF!W$x_S<8YLNby3#`MO4+L%K0RvX; zTS+t~22GlhFwm#TLUVM^Us}SP>@-kJSEaIUKu2t-79Qr!aY<*J@8yck!qU{BwHYp_ zdw?C#!Y&cKV0CN`X^{MY;=MWaS)jpIuFh5&85d<-yEI4nAdxSlJVYH9*JLGSJ<(Wk zSpVXylpQlW?YB_N>*MNJEFx+@EcnBwic>EixZ@}r9>6vZ7(TfUID z4N6DUp$*~e4qF*Z)D6NI zPpXi0`}7pN*!4bzKR1Vk0%$Gon^__ZqM{~ioYM2ADBJZXd(*t=RcrC zNl;2&Ky^1S9cD2>_W>p@VxXX~cjdZi z%%J@N1_j9>R^@h3x8NL#KN9{&f@`>_fD!plUcLcBpsoW2#r!4(f}GHLx=!fGLkIWF zd?9<*V)$rg zDmrw5WyGo78+{9um~J#zko`Ia+epewok##MC*=^GOckO*l%8FGEf0EJZnIddO1G$Y zBf~?AXNf>%4GoAVT&E*N?krp@pZd^1k6tnQLUjRAKPBSU^PYIGt?s6z?wR^lh+j>q z=TtLD{mVVUcq>%ssJe!>O?bvWLulhAD?neMZ`EU1COrXn)cD}W#xQhG>Cpj3Poz3oWPd%oo8x-^lZ*VTpsWUa{?3v4_LZC$TOfZCE{S`sZO4qYfU19XirXKQybj!9COa8>pfX z7VKK%G^NQ*DjxsLeiTb?ijj={m+4X_r490X3&fEc#f70XIX0SAJpxwE%FGgg@*?XVougr z=v+c$83Wp!BH!8$OJt=kfQnT-}yLZR_fQZupgw3TchZ` z3C~Et6a9a)d|f0o6*~~f!M1sS(2ZjqZ|M+-K0#ZUjs^FEw0$-ylUQdh$c(Fb!p*r9 zc2jnZWFCPPNu37|lV0)sna&#)Mg#cg-ulIKV$#@8V(-TqhtLv*e%|D}S&le0($MON z1AOP>}H$@)rg8^4bY}=j<$0@9o+IpcqTe>bv zXvF5T-X-O-7Bu0vCuSBc#`6`KbCieN2Y&=(Qeg`UJkwr4m^IAz4qgQCbZpDpxtk#* zPmoaI+$vnq$%CnVDr6oXRqv|8MED5h7w4Zf@OpB1=uBaS?F(m1Ry89TWtK*#( zOJg$UVvC+q(La(}tV2<7o|3l`5k64T0kUZUCz~`gev+apzux8bovTpk)? zNQ-}qmS~CCOu|J1SY-rPPU$!?oXO+nF|Ob_sFzh#(3*OXZ<)r@`fH2$vt}m1-?Ck1 zl|@QQr$C^$2gU9aNh>S0TSoIl)L&J%Nv=@NIRA;CQP?A;NeE*^^1oS%^Z`Eo9c(3Z zp9GaCJIn}Sw4_Jqk}J*NacqQc5DdWox}M>xj~IlO%KBF?tgNloU(>i{wnl8nGC0Kd z-;5oyr}Ac6s=(t$VAiauf3TZ+8b+SL^^J<_DfQs7cyoFw)U9XsthWZ08~Co-uuWl9 z-F1SwVZU#Z8i`y!auFvqbDu_1t2Z_-Vp-2IbQ<8Au(xHXx?t{&sWu7&)Aw9HS^IX( zyc4Dqm8Yq+CK0HD7H{ComSN6PbmM|(yqsxjhm7w9x`;!;wOF~e22X9cXs9Kh3$rd2~+8;%_%PGADF9m1Svu+}zJ2{f$Rta*Fh33gQgCO>PErnuZr zrPw64^eTc*&J(AP(;-{rS*fX4I6s;;T&|Advi9191Bq!P`9%xJs13SW- z82Ucr5Cc6iWU{^uy}J%k^fNk^(C;$&{4bnI-abOj#Qze8F>1n;Bo0Bi^IfBBB;NLrS|}uFNR?aY9j1 z?s(xg5k{Ec+>55I^o8=sLnkYjUVOme4I{>qmgFCSg&)E(5tbIqqHXVu59%#Z{@Hx} zcYGwId8qSqAnr+2KlMNssRz{g1~G9M2f_>w3-?)h;5JzVG4!TLu!l1>6u-M zgWEzk)5d<`{mG+_(Lzw1lrme@pgXD}T86qg0MWeBWeOB+Z%#Lk)B)3knhEaa-`H3Y z8(MR3IMV>S&BbmXh00Je8;-U2II>I$+YhR{h8I97+CHJWp*!3#O)%^L(sK`*u%X>3 z7hL>~*-Se;O;5LJcRDT1A+{G|ykE3G2cQs4L+H ze&yA6k~(Q59g(-V!TnOOR^FldcgB|ujMOHyu}Q}a%f5j03pM5&+ij$#%~N1;`3aFX z!xoZ6A-~W(y#LxFNzM?TFoC^lm4Uw^LmKF*Cxm!7 zeQsPbKPB2~;$6Pz8hmZ!L)V* zOKDOo$`h6V_?`31ARkiRMJ&{6hwy-Why24PD1X>CYY-bnCsBlbncg>e^Y@^+ftRSE zZ={LN9VjIbV$-0?)tgCHg64=VhW`f5*H(K($PkE1P_Q<($*MA@js@I|G9t!{F|kw1 z4#DKEXcjA`Rl<&bXjs`^85wAnIpwn&Iy=lDfQsLH9HX5RsiumI>x~O1JC$0Z^-`^$ z%zNZ~;?7)Dvvq~tkDuG96O+ox35s7+76`yr(;d9RnVCyInE7RJeTzfc=z$1hn?_|C z5$qJfPAxu2pN_-ocFix1*T2ee%0BEXN@NjbVaHYs+9G}SBs|Z)Qpe#ewDsoinQI4~ib^K`)!rMC4%n{8>LrT?d<2`G= za1=NA632VoI7%C|8K&$VQ+`UOaJstDtHcEgX+7;RK{$r8awhJ*e3F_=UwNuID+UmK zwN9KEkZB9~^7Xn^@-p&2m|`U))BVV}HjiA8Mp|+C2ZI-M<(1BGEN=1izgl<;@tM`n z5|ZklsKI%SRXt#7i3G^=1re%<5Ye_0AbepWSyz5Sl~;tZgM&&u*uP*G3bmHu|K_-_ z>x=YZxJYsc(bj_8{zC>b$xlDnOkY?&J{!l4=|;^><=&wqGi-zx795_fizjiVh0g<2 zWB3ZC=hZsn(G2bl2Z@Vcnw{9cuFRNH8G+jZ+=HAJ*Y_PjRDs6Arit;`TOcVRf}Y_E zEUsC-{KPPZLv%fAB*38vee&0mD>Q_0*bm|wlZOv;!)JB>myy!!m{9~m!3lH+d1(Fb;hX8E)0h#1O1yt#V!65} zq!S0f651%fu;elhC?(#0Ah$lL;~X}~fz6gX9uraOdrw~J1q(NAfTt&`)45D1D#%V%pG$f;YyxfZ%^<)#?pRjkbK6OpTSWt%d71)b$7q4BB_m8OjByYP2O}(* zfeOJ3!R7A3D{aPUwXX!OMJq>?O>phQp0<0Jd1elH{R|n1(-aU|rJWj(a>f#~d|V+&`2> z-v$FqH0u=;SQH*_?my-96jneGZny*IVbC<$ull8wP|6HUi$*)mFQjLM+$D}~5EffT ziREnTh>m7b$xZxTQr0fQs5I|5oE+^M=SJjRyh1kyWG zmM+}t>El#H>L5il6c1nEV_`md#@}UAJCZk<}FHcGh~wX)19>pmhh4B}daN-urX>QK)^7rqw*Oa5{5q zw2+n3>!HR$%K@@0iF4cF1!=?i9fZ0Z(>Vn4kLGBXbIKF6diiW{7nCk;Ti-(C6_BGgtYdjNyP3B(&kYuX~2L2!FQh=blG`(duF7vO5bWp-_qKSh z{$j@P^0Py{_wmbK)A%*KXei2kW(i&ijY82=&t0K#Ry+u8pDvgJ<6j^2SSdzLmN#Jf?SKfcOpWBRO%v&9pNX-xn9vcP#foRiMWTMm=Iggk zf0--2(->4q#eVuwNcI9wF^&dnY|^z_V2Mh#%exdJx++32$dDS+GGqG;$qPEciuExu zs4oJ97^Yu5=PiuPtJ^@Ym?%?1==hzzb#l_IQdK=89KFQA;}PBl@#l|OsX5%^@9Yjy zF^Qxh;AoejDuzg9(fY9_#cO?esa0n%V3RSb-Xo5a)H28^>qH<>x0Wh;GvnfchG4!E z>RE_l=`nEOg_}3n(1s*3;}!gSgogf`g{`x4C9r0R-PFURbjI45cC#&}-g=C)bMImH z`?DW}9^hW_zaRhakE+D4<*{!o6My!GUYvFP*sEHJ z=lq}ekHEqUVqN8nOa3Q)|Dy2e3;q(lw~^mxl2NC)gGxQ^>i&wg>W{UTveX~(T9013 zz3IPQwrl)njcMpUE{**k_gZfSU!w)z&h>qM6lZ&0h`fI&HRIotuUZ-TpUivT4Zi(< z8V_zi$K!8*j;5R$-GBPy*I&L(uD%I>UORgo?_R!-MteU_RRw*N0-E&qfkpVOF+6I`5JPkm;;jHMrM)t?s4Pkv(=<|l7m8Cfaj z+vQ6uUlq1iGJ78>Mthu^psJB%}~Gm7M*8r?m5jf`E@5ygU9*I`wC#lX4LJnvmK zSN6=MOMh-n`%oG6sbb9^SKqt5nzFOXVf(75m)Q4r`?QPORp%$-RKFUki`B5o-6HQ* ze|Z$ObI-7M{=;mmFQgmlmzzbtoAbxXLWSOU3d=cuS9bJ5+(!Q=z&_Hq$0yQvwa-ug z64Fol&$Iov#LCBB|Lvzq$Mt2Hk?+lse$3aXzxMlL?dZ*q$>_tazx`i<(VmxI#xFmg zt+zjf!@UoCHrJf~AH9AK>~F3gg4viqd%_$ZKNs=eT-o2T1NQt_-cL`jU+t5(Uzdc= zFBa0S$r1{}%F~qxO%(n=dcOZ|xLc$__3rCE--NoK{l0|ME1UnLq}=m`Am8;ZS&5;T z_WdsDU-aGjGngOyb&@ZiTM_@W^7FdctZ)CZ=>0SP`P4}G9qo@@lmBz$&baWDuDY9P z|NcPe|5N6x#^5|g3c3I_3$h&fgjD_WcUI%&!CrXp?MM6D{fhSoK6%w(Cx|xZABb?(T5OFZg*cR4koaF^9mq>|F}_R{|5C&Wxh7N9DX{1W4>CPS@X z`T-VoE7;VKOQ7_-?fuZOPI;*W0aX%YeZJpwuDpXt>Xm&=A;OR6VyCar&Oo1~v^5n9#ZB}Xl> z+F>55tuBArz!qU7WiIH$^!x?of5+?peR#`n(fN~Hktj%;DLD2W?63Uw^7GB;V^zPe zXM8REuk6PC-y{X!=8*`GZhkb=KC<$pA6Wb~3afiLA9Xtrr>Ep-WT>R26*Ql%^;IPn z&*Ww7hd)t#zeu;M)kR*YWdvV+pE=V~1wDZFw?B^0tNuz|KSVErpKLE*`QP2C?>`Uj z1XBEsH>2qRd3O$9Bd4bk$|Cvcg*sKvs$_ zUHMM9!`dr5CZeQV>Li`jTf4L?dqG{RqR5VRP`=Q!wCgKAZ*lT;!TC7$@$^y-j`8(^ zaoAI+@@IbNqSxG~LbdXrkt_c4r>_=eC~J@Mm3XY5;$;O{wA$;E7q9NEq#BVZCeIwB z|C@D3L?Ny}pgas?_mSTk|GaMx?{lmFE8vl{_wBlmQ!{GT{!8;YTyQ#T_xtXJ zFvs_YGDOcf4@%Wi^11b!P5ZM0R;>_>Wkgf|%d~+sIREG1C-p6cc`x9L?rHxNNiu9D z`0uWBPTwnIyt)3@r>4HZ>%RUw(iQJ5QG1*;P3o$_pJ6DbA8v|#;d_2ELDBmk&_z?9 zqr5>VofE-9uHxkUUL2!OD%jRG=tSxk$Rp_Rf=^KVd;>H_N04eX#6g3nBI@meT4628 z#le|=FHgIa2f>jVUOF3={V0sGen*(U+CY$UvGE#z13_y|Ouuc#KuY(U%xj_w1|CHA0wWlX5_&YuLRnn3+N|0DAoi zvwE5|U~V)J)JiWX0@Y82x59Vg`$B5aJRmo~?L_Ih)|w&ip_u*)O-ypt0oJE3t7Pjg+ z7Yb^knJc=S`aI$>d=dL|dd_I4CbOr=@{QI0HHj)gu}$O~hXYbK81J0^jkr3|m^f#& zy11S6Jv?LjO(|ODA&zs2$R)}aNqzsY{3kmNC`*X6J&fwT>gG_(Ol@$}eEJ4nc*Qv; z*{@k9VV>#`5<7GrpijMD(nQ zyZ=%4{#Q#)endN!E4mCnH8+Ht_wemngS8zdYM0a;5>lq&=ux&GrQW$)KkCY+_qhNK z@J{6omDojgp6*A~ISf;>%Lny^NpwhVSS(yyrxXkxpsMETe~4OnRRhpQ%Tjb%o07_ zWZ`}>#t1*4yKV15d<2YB`yg%5+JP_d0x?79NN|Hoc{Re7Zuvf(zTdF_?jZIIdop;z zkSf*hghga3dgNX2q1zl9*Kwp!WckpR!z%>4%6SG|dckVD+Mf0uVb7@h#GCtEL5NAL zQEH5YD|teHJ=6tc!cF9}8yo{^%CcKV;j|0cE<{9S%e$b7krLlVUJIo zu^F7g>>LEzQg6Rn#!@=5%_UxeeZo3iXuTJ_Mx@Z@%51|L-&_Me^q z^E`#m!gE^rDK7{$Ok0k3NwO4cjzT^pcU2lbYr&3s4SPh&p(wr-m1E3I?3 z5zxk|FzkieTdL9CLp*1azx4^e{AvZ*V?ulDeZhUWp$Uff;V%!#wa>EyO3g@nBtnDz zOeuW`UA2amIh^Oo6HpEVdY2(H;KIhd}wC>40PEz!ms>qu2<8zjH6&W%y1UJ)wn^5LF+ z3ds^TjD;=M6kFn-LtyU9si$RZ5l3(vZx;yQqgL%Hc8BO2Koh4GvO83wa)MyW41Y2aXf=A>*#EAY7H8a$x~KH=(AuPbCJq4 zyTuv78#RR94=fsL6zs+gp{n(k%Z%t=qJk*MtF*e~{6qRbGsoXy+1z8tT;>1{zPu;% z3HoZ`!XkyL)78nJ)QJ<&UEL$-@~Flm>^vqYaL+Jvt!@ciA0Qfa?!psT=RQH^nluLQ zIfF}X7(R0J%LS&S5Ahi~+K<3G`jPGTCujK$R9?_BpM7Zwb3}EePr@b(iBI5|uyzru z&JDEQ-?QR12Yq6u_JLSk%*_+$I5T>!wnSqb`f?2U22-oJ$P!05%3Ttm4)MIb^%Mf< zP_!E6z2mI|5IrrO;p9FU<>9MRnsn?yMV#2F-RRjGF4oF9mceUuO8BZ?XwF= zy`)v|e~-R($Y0Z{DVM1A9CC9QjKBNJvxe4Tfjv}Pc3>FI^bJ~F99$2XybP%c(k(yR z^FKneXcn?g#~ms+^l+j7Ax0k|!N3~9(nGLLAht{(G0qDL?_2u2#T?sc?aDtxYe)Vt zIP#5F3%5zCq#Gf9O)MfR6O28_DOR4uVs4MB@&Kn%#Wmz48UV4B4%OqplUQab12BS3m?2m{C1`3$|*_VtcazUhwJg1&KrZDpG< z<`J`(YqM;LYyM0_7M|1MJjWeKFF@usX(3v=0B`D+FlCoW35L$#97kAV%{FT*Sj8FiWhUSn#5<5j;TQtC zd52y7jyZTjqV*KFfQ`Z>6WdDX#5!wr)l$XP_nNhIw#<$aof^_1 zDdAnxK)VoWz2Jx030GTT?zdld!M!##KO)8ShG5kjl#ftR?R&t#XVYu}_jp0+*?~P8 z?vU zo!`8yy8+RzrwdoyWA>cuxPUR}c#CFun_x-vozs+AWeG@VKPE^!;=<3kXL!O~*1M(I zE-UVlsCCRb5U~kudgz8?5wiBkE#fG=Vg8oF9AwsrER*D%&g$NRb(qkjKl!qwEq@3( zeu#ud*?$SH5yDaH=3n-s?ZD5rqb))AN1~F_28mwR8RuSL;TfoHV!7o7CNJS7)mr2+ zY0J;dhf6xp34Oy9Bts5;vauUgB`BB z<1@K+feO7Xm#U0|Q_754EZUYbs_oW(nITN~2J)vsI>nfw_3!@q4jBsbJDlJGd#rJR zsB9WD)*hG@)Sr?adkNu|9l}rdBr}R<=r~996MKp`glDZg%%*vVX=#VhPY&2`lCQr4 zew}>KJU+qKs7`>FYVQO0kVtVEE7x524Cn#x1PC8qUB#QHyWVQjbG! zoBW&9GX~vyVVdF~tiU2a%0{5GH|ewXaI8Vc5cIqtFS-L=6J$EVgO4CI3`?X_MsWjM zV^8r4$8n5pQ)Zm8MJ}|8sadQM$jqX3cygM^z&*^7iLX$ScZqLJjF`&!t8w9>mv!|7Pzr(m7;-C*k&_=WeB74Il7 zMxjJ@Z-zB}X(arC5*u~PX%4}cuE4`oe=~%QLlEnw%#rT6Bc--sHTYe@ zcU<3w)vI^0Zk>hw>>*X@M=(ga;RXMJ2Gp8S<#hj!_u>)@ulw z3((TvW*sn(9I{JR7-Gp{E^~u>+-stz=jbjry?}wfZgCY_^)nC`AW+pF5r%KlFk6S| zzTw6V)_lfa(D&moIK}pXxn?2X_NC|RBz)=lI)X~;b4vB>b1Yp!*BTL2pQrzfOyw9$ zPU1@{lVvg@PoZ4#k74Emx5ha)X?8}7)q?dBjlh@gtGD0k-^E2PZd?-R47zX8yn)OR z*plnykk^RUiqnvp2d~v|)*OY{A-4+Bp4APeXNX{b2Kzp;6Lw6LHVvK&2wt-lVY6-v z&pB7e97HRwPn07osHTO=GmH(96j^8VVE-r#eyAIwN$WH&^GqBn$ZxGOitdf~;fT zKr?w7a)RndvQqM%V7D)TM;k+vvL@Fbm}5OV=*|r;qH}|S9KQx>0h+QGR3_dvs?aa} zD_@Y2=~_nb0Gk9KqZy3Hz>T zw3=`kBovbYc8NQzZ@W&~hF~!49Dzh_oO=e5PlO%=vfdHyn!*W67}F^Pg>n8PxF2G8 zhTCH!Oj)9&nc>r`{I1l0Ok}1_aEHY9WeLxA; z*(OtZYMuONc#nhEu*VP6DsDlX$b6T8_e*EK&VH*a-!eXi%sFb&##fNcr$%5P!oOkJ zS$IX)Sb)z3+C=VI#+onj-nO5*7y)MnvJ%ej}}YiV7^R8wuA zh6%( z0G-d-6}rb+f>g%{CLTlR9NWxM_JsVwa}-jGSQD1Pk-i42(^~Qae+bQ2(3!htYotV9 zQnF=8>AMEweM_J$qUV$}clL<94!JPwx}hO`$5-D&0%*KV`na z7AZ6DV8J~EkA-~@Cy8MWc>6qOtiT@O+6o+SYmnLk%EgxO4dk=Q$C!FbVNluwcXaT9Fyhe{KK0@Uki@p-3 zi8#k0_K5(<*XEq0oBCJ5YP2=9;`at+n0uP{w6Z4q3ReS)ZW z_>xUg%AmvR!6=^{ebDCY{Y=VC}(c|G=KJYay;~G1n?Q z{DmU|u?{B5&n##;X45>gj)$F5F5YlfnvN=h|I$NUr54l_8>0NS~ zF{>NWC3i}9eTK8_3{JnYjj>w)Q;dgK>jnZZq4*kjt%Ii7;WApT3C>+pI7Z_QZE_Kz zF*=Ov6r(Slg_5ta@T{)5!au^~a9wqRQeh9Q7C_T@2-{zoGo%NcfiL}jdRN#Udc7l1 zGqi1%?AjvCa?JzV!bWGf>4<99eFz=h!h5uO@hp(j-osI$eUFjjF$DcK+d4yqC=Oq!vBC(+RHPc!tK5THaPV3H}K;Ny^Jf>Z2RNlb}`W%tL8=_*z z0he5dZEk~P{vDp6)%oQPokew)X=j*lZIbmq!ATmk-D;MyNPw1q&TPAovS8_kYxlGV zHn`*`b`G7k&dzRsc$Q4TXG}hYoiD0?0qqaJv`cx3u6XMw<`wx3ehNF>7i^{O3PUHf z2p!9mh#yJjm~4_XlhwuK2x|SGU4W_y~hQ}e0#Q`%s)ikI+BWhQiAi`JWXeul_yD# zU1Fu<6FMguALxHh1+YR|bOguwT}psdXZ(g=C2I6Pi^(%N-0J!wmNpUe(J+-7wX>ol7E`42Rw=h#$wOc*@GWyn8sxvoYzp8brS6LrrqOJ0`s#5)Qr zu)&Q|bNGFNe_Ru#ulmWJLV}zK9F{mC#*fkrDRh%-n<6J8a~2h zrkz8ZHjMrmOfE3zHFKu)C1^C3856sOc>v3o?-V3Y}1Bdli3<( z!TXSD3UO=Zj2nhlzuj`iYJQn-n88o{EN-!GXVgnvbpWnoei6yAv50lNTsbBq`-H1C zsl=iqL{v1)G5$tqb?uY8nZRC~qKHn+w7p4sb(t1# zi3(6czSCq=5n2*homRKV_9dCPy!obeA%)&gFR!}<5Q?=dqAFc zfn>!fd_?QrhZ}C3sxze~tfS2R`aE>(;t>W?r(9FxDtNU+aP`fBx1)Lo!Y8i03Il7z z*=M{qhp~<=g62yUI-j9^47=u;bvjtKsa=#KuejEiFnEZ|yeizF+kT4)y=EjFo*P`O zd(1TFETKo}DLyAmCfxpVHg^&W=lnHE$np}A*(eoK@rJztxg_smTpY+f#C;5ibC!SU zb2Yjifms{DvXFd)s{9G2DCu0$EsEzh2FD1`vTu>CQ%dsB$q=GWMe1`caE}RfKcSkO zsXGSi9P4Y!{R7YoxwWt46DVF0udYGkomF2fsyru|G>WbXA@qdnK7@U?!Dg+l;k;~~ z5@1?~LHi_HL5RBE(nGtMd4l~DIM5V&20MJqER1lgY?<$%OPj;klp?vEU#u&?VV87M z_ZqF_1gVO3w3s}zE6ev@1UD5-a+fxg~<)=_+$-m_E zVX?*g45C%!D{V4jaHu{)}<6;KCKU9NE)VuQE%ln?I{ z2Hrx4v)9lLU`$xTL(b}`WgTuiBW`*`+F+DS5>>uDis`u1?3xFD4KjR#bw=$PU-14r zT9t;>I$I{ddB%-l=Ns%Yv?qAYlrb_;tE8Ye@R(HgJ9@x@b$|igF5Y^#sC}IEAA67@D6B3;>JedpM6+QkrU6g-eok)7Z9bs&|&fE5m$;+6L)JYPLtB?*p1_GM# zx+jz!6U>?2#~0A_8H!K26Jq835&DNgStrT9PdZ~Cg83*b7~I&guR=0(sh;04M;0|{8`MX!=|wq}7mP{cVz;B(9T`_SV_$*qMeVkEMk{5poO z@CqA7Ao-Dmt8Sa7%Z)(ZdSjArf6uZ{xO1}C+g!s_R_Ie2`ZDSwy1R!-lZs3nb!(XjA zLSt~oc}GbnS%968fBoL{fB0)Z|C1DK-~WFAP)h>@6aWAK2msB3 zT1nT`#i&--006@}0RTGy8~|itbYU-UZ)9b2FEB4@Ze(d=WpgiNZfr4QIa6X$a!XEX zc4swoElV(FQ7Md?crRaPG%jXjbX8Od00VmXY|HaHtF zNhl$*2?E;~Y@8z~A+kUOlam8BV6fjl)gy~3Uu9|uLhkrVD^|BP_UR}Lx z(q13lJ8SK-^zskyeP#8snP-1Af7aS-@dvKEc9#>Hf1bT|+04tIp8p#C@ewY#7Qg)S zY~1?tr&i5bd)qFTe-yvIe&s&bPm5oFFJhlNYr?Vgic=TPnLm5#7fzZ#Z^E>}33)VO z>%j>}opjRi_}BbX7aW6s9e>h^vyPZ|(y8-tsEBZ2-uzSNoH(nk45nFI`N9`YT5#0y z$84cI9~DzO=X4ogIO*suqRuQu?+S87ou%w`?2*UMo2?8-#cZW3X)DewWl|W-Qbq!H z_@fFR48vRzvlMGMO=l@Sex%f+*@_>oRx~d%nV5}JSv*_u;*R31QTpQ5iXY)*J5N_D zmEjChSyoGnC=a1YKqsW$2TxPpN02K7Skt~ z!)g8~eyMYu!e5G%X^p_qM9SE3fOiv*apt8rurA%zMiO?|GLa47k(;lK2pJ^)DpgDhIz8D@oh*02Hnd$BvKj`p zFPx>d(+(!lx=t7CFcndN2~MsgNj5PWsa4VWbgn*1FV3788t69+LfaS3<$k&hpH!+h zXEqlVyjoW4Di2p|3;>C@c-&$2{2I*89Io**d+0MWmAQ*eKU>QcbB67wOLW}}YqknC zcpXz((69V@&AE&@U%)d z&6&~;0$UTta=jgQNdGTpILiERfhBl%7r)8&gC{i2dW>TPxHD513oK~29Wb{E)4k{f zMgoUnb63?#H(ba1A_uDZ6s&O}*Eoa`L(0eX<5z|cb*S@lb-T?zajVVx!WCF`PineX z%v0%t1lqF5j^kn_3=Skdd#-|!>#NcWr;x@phMIu-!o;rQ?JQjf87Y>K#=n7yy*$Rx z>t^e?pFEGXx0{P`QAphQ63x-M`3i*eZULb`OD=yWS9!JS9;+B5x}V^SM2}Nzk#6)G z%)F%4&N35UPYWN*;~B#Xt2PaP`l!;*aII#a;mEJ=sfk5htR64d#j}QPr%b3^anI`# zuXi*4R*TK$?G!pl6L0j@hVulvIlhKF8S5*iseY<+L)^fs?vKavy}Gfc`5#(rjnYoC zHIVZ!*46b~EN0A|Z}5|JoHp;a+Xd{Y#y`^~O@yQIv2{A};W)o|r(5SYDl}RqSNbzY zV<+oUnjhzWkzvxZnjGhhk^11zm@ssPiE|GKov$9G3o&L+e?^?c9u;iAC&Wbo~7GEy*Q&(Is^0v|J_)XKuxcq@mF-Q z2CEXE71S-oUQItYSRsY0vV2SW3LE#RK@?xn>26P94qrC-qs&>%%x`x4`9@fWF-EPG znOtntSYO=k^>Y9ndtVzSRB1&525f+tXB&S0evJfIO-b4nSNHkxs+%%nCtaQr0}imr zau}W(Ge5yyuIh!5y`*EjEG!CA#eg|)IDWBJw=S+7s19ezDH`vyF*9uL7VJ*5T?Du2 z8sF6*y}Aaf9E&F4IBkB?@QWujgPkJREQehEwM%@;W#&l}{o-+a-F?=m0vOI=zQkTO zwA;rGj=zqp=%UxEx^Gv+9L_ifZHDd#3m~C>g)X24z0m*)(8trgf77Ge6KfDfU;u!& z-!TZ5^(^c~)Tc9i{PhIJ6c=kV>SO$jVN#hlFujpi3<7zrXoU`p$CdT+Zu9Y&7l$!N zid!`B2jg4T_-v)!$!^|Kv8MhF3~j+Li%wx zq|49LF)+DLNM*W#vnfI6IYZ3FOKsOu=i>fj` z1KNrqrDybW_5+vQfq562Zr?XWtW*{EjsY*7EF6cMcbR^^hpBt7XGoqm(Uut7HZpq%4XYH~Bu5aOwmAhQuy!SO61W$Vv|GHw@{8{Tk-ah;j7#;oV z)sno;rgN?NV0*h<_jF8b?}{e<1m|#<#k+tP5)-_lMZetVoN3W(V0(xDpV{8Okv(E* z|8KSzcK(g+bwzFEn3C*GP~Ih~XA;#5qJSL+bT$|ylJ0E_)_APa%#Tq9<8&%PjG-(9 z+Y8y}U_b}+te{_!8YSGvJ|XBn6bI00Bxqi2vQY&*T!e%R|4u(>;M_>e4qu~ygxK;I zYe)}Ab8y*OMQLEv{RERTl*a`VI5HfANe(6fO!8V~sK8M_#d!~p$7!oSp2qM$OxvUk z*S3?+DWDulxKyFJ$MsIm2D8poqLo825z_#s$d68eXoXKhCLOK}r5S$vG}q~-Do`%r zPaLtiV&NwV&NPY=WDLiF&8Ox?!R$5i5Ts_^Ejci@vUGrS+}a%oK#JJj#0H#T2*@j)6k&sm!#rcixG?P zZ$F&IltzfZ$PtPD`OU*A5%2U=(AWGWtsQ4_LJBfWri-D6x-8hFC||<%jS6RtQQL~N#E7=MoeoEGrcp|utH|HksfIJ=~ANR=V~-4t>7pdYn;`?rC?%rsvxs2 zuMDSYA8_2cno`f8LZS?h63^hhi=~nTVk+nHc9id=>Q)`J5tE$ZO~t-CYb3X;e!j*9 zc{|up=Mk>Q2T7S#%#$%?#Q~;fKgG$Gw6c>iiYEgjZ>P5N0wCQ_9#JZl(>0cGn1y<} zhu$ta2G_h0QU{i5x_63bb% znm`|Xs*B-k6cEJIp}#LQ=(SovTKpFdDYzIyRm3Obfx#w~0ET-?mA^#<+_JbZt2+OT zK`Yq8dn7pWfGMBTE>c|62o^JmV;Y7RJg-A)EHXVgxW|)(;S~2y%;OVSA%C{~>}TMc zSRs|l0*zkWx>E<<#BX_*3R$<@v?9K5VhC2des-s>i%l9pZXf6?WAvJS_5$$BCl0i5 zcC+f`vzcS@dUA{l>rjB!{=uqNHVJk*K}P#kqn!s(_Fni5!)#f@c?=0{8O%CU`1y;t z(Nc!Bjqy1{hZU$}g~Rsk0M_<9hs0|~7=g(PNQpIv&{$sh2*Xp% zah9Lo-lbc|lDmNo2OYf5^%y|VWokfH#hI_?&P>aKCaa{$Yb+*^epi8tyai?(->bNXtA6o{Ny}{)g?x?l`Umn@un&9!zYdp5e1_g)YYX7`a&7b zJ8J;IjH+UtiwT6*^0Sw`iu(#QVq1R6-<);|i=jaQ)hefQQr3O(sk|MA51TswyNwSw zha)IuKl-i?P5T}&N_Yw^Q{;D(>VFz*HMD45kGTaAZ<1WpSM6>9D(hj$uS z_r?j;kud7K*Q4>rD|8m<4T6o6Gz!5}+>b8BQn95QZ%Ra_bO)8`vm5O&GOCcu8;OW3 z?u!~9=hJIAE!+GtsH7Lf7Gro*zj=G>4 zzv5$oS*^TFV^@;*z|1SSBF!K)Pml|WNnyHr0yAG8z$Gn1=W87dvh8|OF_~v*eo!!j zbQn~1<&iB)3t*1H&;+~dIzOlfE8fE55jQlMbvl0jqh8(4nbfTbK4p=C<@-F2@2VTt zA-3dDSJuU47Bim{xXG@#$@dI$hYc6$cB*(17U5+Y_)jM2Ktq*Qas%TtG{@leV5RDZ zdv*h1_vs|q8Aj=4kL26fr3+W=OpcbDq_P=~$P08Gr&SHxE|&Hid|p2-(N=K+_tL8f zT-DQeEI%>;N}|QA;;ygqoeVl0{@xJ&pwTGKF~oDwbHSau_y7Rkf(66-YT`$BtyTQm z74PCBemdR{2L}zWIKPXe8Vkg|MdeTQ6F&DNg5QHCv17MufKE0DvLEH@Vm_}9hrzR) zu!4D4*GL$22niH>N+fWb${gHndX3o&g&!PZ>s*s&4!?$Eu&WiEHka2hfDO-3CYWTH zFyzp3IU{9?F%UyIdqkhkf3M09&vWL7e}K+9$oA#&odg{pjw9%HTmr@XlIC|FY3TgC z$SOhw-|#yR z+q(Flt!nPq9Ij2m;v;Y9WtI-}R&!>me*6ck;&(1IbTOm9>i!RgR=Q}f#=!}o{(TN6sbu7JD z1HFFOH_qUSk=>en=R9?@& z>GK!chZ2)sBJo8% zcK~s|XBzART`dpU8RJiOk)-E~U?yL~BECnHezt#X$;}A2(!X8@4-3xnwe2M^oloI~Q*C6SuDa0{GH4giSuYRHCUVwF@UMM9 z>WGz{(~@+Io&X+p_DAt-@UX-Fk31|5|L;+^H2rrT7EIk7beo|3gB967saOb&29hN< zN8B=Ct8pEoSNO|F&JG8-8B7wu$kH5wyw0XDf}}_(aVbzCtxc>75CZ=l40CCGCy`4F z_Iet{h=7`w0Ax};i!%b$Lxr>mz{g$j5(3-Sa02G*hs%IKGncnBA|dvva*b0Qr?^@^ zkyg5&62~B9D`f?wFDGc*E?HqraW_bXm;aOVwNgwxa2zhDXMl+7fLoghOi-*fEU$XuY3+napBFr2(03Q5wYx>bciAxVF^x&`qDpry$d|rW z;W?M8+yj5uL)V9r;#O7iju?;3OFa(`>c}nE@1_+~1P>%PgG>dacu<&m$euh|TJ(|Yac|yoL z=vB!Cw2EKaUg}B;o8&~5_BTsvl`kt-@?HQy0|vO~PMq%e3~+8zNFU;AfY1R$oXT}^ zmIio;i7LtuzmfUbo@Du!Nb$zIH>1@G|D5Aua4!U?3oWSar@6ow()`xLvw$cevI zs_uL)z5;`YU*`pp?;%q<9DpC#kd3pTyFNxh_Yy~n1@7&>3>to$Mt2empJko)jfRWaZ8 zMxsvVDX#HfS}G(8z|Xzd^W$IJN?$TSCxc0rMEau=?rx{D)y#odzHi}dNMV6ntL18g zFv8f2j>G!muUPG<^E5s;*g@(9G8i|j;vxXtjydZU^DKcG9itg+ zK2z5d!c;2zbpz)UKM`Y~7}G)`qv?$GN9Ss6Rz)tb+f_jBs~i}yG3uxpYDN3#>{Qhy z>;aAz&BE|J!z2{RDqgyj6qX5$8J>x|MwgpqFp4>sTVflEINeYgi9!mNc}W?^O7X=t z0oJbA+pDlzI)71>w>y#Z-&LFxaAHzDD?(D)6Bfq$9G-!4zr?-zbtJ`O3(p+vehHMwc=;*$YDDUi`YCgoa>Yi_i!5&tR`$yeD{OB=K(!Mds=P)Ak zYh0Lm3JN^XU7HexuswSjd8b&nY@e(p2# zOng;-DFEn;O_0qouOtiSMfpB0azzn?K~sD=`$4?*?3}ark`h47#QtNMxzm>ID256Z^Q$%InJW8<J7< zIW(7(EkC~2A;Ly0)`T4Y+!v=bm^EazDKLj+@3 zz1ZIbDmq7K{8v^PzE#}yuGpl(Boelx=vZDxOJNxQIF4Blb^FQrn9QlY1W}-nDwBHH zh~}Ka&11IqqX#D&dC_^rfYtvhnU&8lPUppxDnq z8vqyE4~`|k<^HvY&#AjuSoi7WBM-A{%>-5r$jTfe{PdoF-OHenq4mKB+zA#x-z4J$ zW%jbr`4K&%YONLHTldhJnKC~;7!3Y*n?vmG8wNlz7UR*bsE39Zk6PyZV4Hh7$gO@b z1~nL7mW2}RQ1lGUEd-x{Nh@g(B~DV5OB>Ac(gab^ zf|yP!0XV1x*jYipXhfq#h(ZyAL437@?|A{JA#=2VVM$Df1KfXVhW`u&@fyXaR0?8Z zmEi*aol10ZD9dMWW@o@sOG-}`=!qm}Ih&M*vz2|*m>qEnS4VvYwu~y5+}>KJw9{lB z#k8O}$jO=%)0tO*Pp$`JjMkGOmCAxpY2@;$L1!6k_A<6FVB}%b8KlwW702OTB5{8` zy-jVW0Z7^aI@+at7+)#;r6&PB4}G$DHnGn1y;`hHUkg3<4Im|r#O5uAB{Zu7!wWhQX7sF z$jb{FV8@_^H%qryHS0_bc!N2I*j`=+5fZ#B`Eeh%)Eh@B26PJE-g-d;sHTX~(GE%A zDV+6#{u0**n*lcy-DL7^-RvM}r4QT0HcYMBjzc?8o!~9H58_J3w+cL}+Q53F_4*H2%aG;Gl-pF)uZKX=wf~K5!>Fb6cf{^=| zs>*Pl;@feVVi7E>2t5C@2By}mTxPK@^0wE5AKQMotX4uC(uUzJD#a!!GIzSx4&z%? zjc+f(`^gl6E3hI@E{Qf+Zb;I_TRMF&@L4M@2gP`jk3@ycrt0k)rA>fBe!7PaasMM& zQ?b3O^T?I_JlH|H4*J5UczME0cHoJrACTU7FXm`{t5IO>pNBRQo^-_2Tvmu#<5YTgnox+tLRi6w+kSU+V?e8FYr zn}#2OACYycw6Y$E)S)&^EdCM!`P=msxne%I+^G)GKR0*jf_dLt*SvhEf67lVPvXHf zV2yiIbiEhC{k~PDn@MGDea`lihfFZ29o4FPy70okyQAnd>GKS75kZzup>WK_25B8L zsrb>o0PbUZ#0^aFH%;eJ#Uk+1If)!V_iC_|mU~mp%TmJg)(KTVhM3vku1iHz0xgxTMMCc_UqMcTt%pz zl$gw>w=#eq46v7OQVFii!lI8bU8zaZ*1!rDyXpqpo{xr|n;raU?`|fcSCWD-ibL&M z<^FEUMxc28XUukzjS}EBrBmxJEY6z*-7WyRY!WoGr3i+na{iY-f*&txk^uMNUv~d z5>~t1?WYNHpalQnfR2@}k7kYmTGUMx4iA$9epDZQWednuoZMp8_qzS~r2*Yu-?)2R{d4b%1H@R?XQPU;Us3j8+d$+P*Nyhh8^(jg${NmAY<+8FdvDo}W<>Ssqn;rX64Ka_RPe?u(VI#2l7yooxw z8X>!t^M;f&`{}sJ#pi9EJwaKE7&E|>V=1rOx)-#1I|$I9$I{x(C}pI`7;h8Bu47@h z^Uikaq$V+3a{oa?U`^{{A-KstUE&CvSz8abvq)wSJ!p_pfxULI1XgUx&1n@sn_o37 zmrJkD2V42+FQKm^EaY@4)A>0y;-9T#Po4kOCjCnxf;4zP5+@X^;1f+5q`%C=AI!R`;-~8{lRq8{W}5D*3e{ze zaJQOx9?R%;3fo)#!JSQE_Az5Q8%e_COCCVYu{|^l&tn}`8{#07na6JJCtn;e_#b4} zY=u+alQ>^MmwwH{Rd`{1%dk~#CFBJMYS>k60AY#=P++Hw84Y_i$*;FaW9A&dfR{A8?Q zVXc#69WWPmA5tN|V3E+-GnG=^E5iv8s`yOAY)J1ndvt!iO)}~1pz;abfaJZ|ce}vP zk|W{|8KId)Q7$kPKay1BNr~hggAc~Zw-nH0FhiV-snDTJ z`Ubvy6ub^kr^G`)Sqyd%j!5XXn6i`SY>xIdK%6M`RVVJVn30v5Lv?qAJF;nDA;mOS zX(dl`pc-JaI!K)lKB!sJZu2HUyB9n`P9hsZ4ros4k(E~XB$pX0gnJzs-7)}WAg&w> zh;OICNEmo9MZIA00E4U+eL(?kGnB^?U;=KO!ZxGVXtw0I?rO!%Sv&1mT=E%0#YBaQ zoRdfUWKtjC4PJVWHtOhYg=gWPcah_+p4e1i7WG6D%HUG~3jNNvRAI9r9C~ ztaQ_8rxaWRbd_Gg3xH1N$gIaWKb7yBChX8R4esTRtSiAlvPI@7RHAH>MtHN#+#r9? zN|o1Bss_x6qHV`15x!MiW_jUiCaw4c%TGOGAU69aJHhvkj$iTztDRE(2-=&31_~D3 z`bNfYBotT=E-?HI{G~zh9J-WJv;GLveKMEF{d^s9T_AbCm}p@-rm=PwPNRatd5m&0 zhQl1v5ZJ0SjnyOkECWJpq+hmR@F~0lH8q=TMLk}^{LV8#E(L>9g;;snRf?n=7a8(| zm!8T-ab+vqQx|as_WdHG-6=q#SQ)?sL~sTSag_ryiTef5aKoC%NAq-x(x*^>{gYch zIM&tpZG6s_i^Fo`Wuc=TS__uz7?9V8bSFkok@<)UNFcrjj$Jl*jROrHWm+$$YY z;bFEgs;oijjvqS^_M6p83lcBp3w+Rn_;Hs;sZ8pqDo-(6Ifv116e}?SAUymLNBY=* zW|t}K^?VO-1YFgXcgyEP{M;^$05|6TX{v!dg=;e54lBpGK;}G>PdxN!a-ZT=s=9i_ z1x(xoQb^FKk(a3x!+A!t{(=U)yMqeCNOtlpmF$KLNcBJ`d|PoGh$om;vTwI;Jl_01f`Zj#qOCOEUD1p`H>@aCFK#Emb; zt*7I}1Kob&a@~)2Qmd_e9{}w23`XDDzUFs!1_U|WlG2z4)1@5O!&fxFGYv3dmK?BF zynu^0JxCNCzoG}gs7zPh$d$^LJ%ZA6gPl~rbD*hvfiLZ~TjI=Ts*J*)?M^z6YxdE7 z5|AHc`Vp?NJ{d#l;#fXA(?VCK1d+|zQZAEJ*2+|Joj$DEDrEXFCuw~THhCJWH7VLY zPeRXAtA2-$7NGo4=;AGp8*7x2xX5R_njJdQF`854jS?{x7ku*a9+~^o451>ndvvu6>2G;a!wOC)*ZcqzCV;-IVz4u7qpR2bye%H;W;Qqp zg=AR2KAQN=j&PjCj4GoGDvAK(n+#$dO2Sj@eol zXVSWEw;eoHp#)DaTgNb0lti4rfpvOYbyQunvL%e4JC?xjm1F$;UP`6BMXgZjNK>X? z##`W$Z#lX+w72FQX7fMwN^RS3`1zZjZhzuPC&yj7_>8IaCEqtkl03#N*q4qIp2miw z@MNg}d&mP|eGzy1Dfn@)Jtr5=a1{MBDWtToR3s90Yc(nkS=_*X`|`8!n)#gkQBC7IFhB^ZW1=IWMYmG9HR&A)8v_9Y(Z z8VmpohIq$g4Oo4F!`g$&wQIwjuN_xW)S9QU2h7qzdu721F2p zWx{q8f9cU}yDE(yCR?0bWrkVx)AK#UKE#)Hp?KTy!u=>4lj=sqPkdcGFKSK-s@6@; zQE1Fg8do&_KiyQ(MU|!b22L|>7F+u1D+4C}jx~2ZK+bO~v~>5F42ZEt(d{G0%lCx* zAI}wYMH7cNLS}asKy{|`tj1p+OXZQtPCCvUSJj8|e0@VyH0F*O5WnqXRtWAsb%Jin zI#0#juU9OoG4oMHw8j~DlbU-Q?80MQv3}*YGph2lKPV%*%T>_}5_`{NKl_@a^8+o_ zl>RMur+!Hh=3=!)By#rSyv!q5poy{LnAx5H>U@qt5r0Z)wYxORxx--Y4dNR;1vzV1Mr;*HbH*R^#8UO2)D)l;YAFrm?q7u3*k?y4Y?KwOpX2AQEw=NYMTal!pU3qILx9TKkt4PJ>P(MZPJA2G4t_> zBWakOu4&?PZmpHRZt#7m#H*f?s~@mcd|a=(t4WwxU$&(KoOrpT+jmbUm2YpS@-3-U z!h&6im~=X33B z4}2?)l0|ySx4@!g?bj}MOp9mXUzdLb!n6`ZYyPq8Zv)X2Q_QQmY8C~_n)l9LD258l3{Uj26%Z{ z&lO~IfFflkV-Qd#V-)^dCSwBF+m6CzgqO++C8Fe%?J-2LFEF*kmN7nj3_k*EyLSPr+3(vK~yP8dKsfsMQbl;e`301 ze8`AKF6VP)EM&F<+e=j!Nx!ZxE63v=-3ukRtpiR&&8^0}hc`0V2fPD84)bnkKT5ar zil)p^hjwXEd_^V=O^U+XR6hm0_R_OuLZI6+ysS(XEP&g)kof$+f`(wo@zp`ID5_U!?>u%$3E;bhbC~oF$|MEf{md zywD)nKb&SaGOswF;DLP$kTV5XZk7DQ0+M_RlvdV_nd7Z?)`8ixN8=~ktHd7xS-UmK zxri-Lxzz5^l6UKj;%Qk(Kx&k-5-V6tHSg3D#gA~OMz&IwuaIE0e4si7ic^ZVSCqjM znpvuq2#B=g-*EhFq6HKCulUY`rZ_|6<`2w$vy>e=ky!B#GM}`GSM_4d(-LNl(olXg zrwVrRKTzzKlp5Y%ueP)N&(@TF?1B}ZvN;+Sy1ZRW!95(2Q0~ssze0jfY4+BV<{zgCSa#BW!Um)$38 zA%X+Uc9Go<$*?Mvc;xJ)64RShB=mMve&hwGGIf-$jgdJ>7RUwwA7r`AdmRp_z%T&) zFu`;4nDQU@ksb>g-5x>Zfc3CPYv=TSlt?v8^3vg$I_AXWpn-h*pm{pmo|oNkU~BWL za|*Z|I3Jzyg4s?9wr^9pREXy1ZFz=&)s)u+FKYbH67`07jiOkJXB}zD7c6pMZh$Rw zm_+?jLvI84*XPetNqz~ z275);cz#4z$!y#3yL#1~Q4t!*YtpB+V@f9SagSOP+KGSAi>F9(avD{n>`J|BKBe=8 zWjZU7mq%>fUS1ckX{<^1@?#74c^%aGV@vmoW#DXhSwC6875ihWod?Sqp7|xC+DfN0 zFFlUDADEXD`*eF7;+{~08BXM!$y5pG$oL03R8w75FOb^M{Y)2(ycF#>g*++FsDkg^ zNlCyUE>8leb?{Djp{m;w;@UCHe74cfqU#);ZDmqP4 zJE;5=l@fnfRz|WYX$XyZGTe5*&fJ?S%%C(v{ns{5rZXXavaZJ(4#USeKd6V?Uj{VC ziCk|G?IQY5`3^j!s3YYYWD(V>^XaZk`Wm+UbkI}Gott~Bg$sJfsij?Mcb)g8%V zC=X41VAY1R&Z%5n>oD`vU4C}AZrE?hxNS41f&dw>Hzz_yWcroe+k(yL@}fct)tya0 z`>SP&1y$-AArs(OU;L(lcl{GM+jCX&Sy5&)f3=I&e&KRId8Gx8*IjLa@Ua)E5XfS# zpL_`vXXhF%T`U^@K6TmviEL-dhCT@USVOYEgT{(=E;B*7J1=Rv{Y6>z*~)SDY0k{P z)<_m~o(9prWI)P)k)6WDb5+(C?OJPRfihm_V|DSeW>#?gXEsO$YX>rAdEMaV*!yV> zS;U&H(q-m^##d?563(kq62d$#E5YvrMSisf08@bX zf)ZL(5JPv`krQyCa@Dlr2P^B$hTZ-UR%4#@Gp@rMr<4P8AGyKy0^8WFsq@RaN*1ZO zY9BfNfYGVS{ZUY!*IKgY#bQPw##z9o*F#dn{HxFmS39+4k}#?&swXGm_vahB{i|MC zgcM)L_={AIV}8&BVf&e_^9d%=EgkA4c!KjYrCtDbmr?u+3|TmlQ`QJj#9fmNZdS^S zRRzozvVFME>hYtl9~|!M{1i5t+3en}Q~thUo=^qoJE5wvGQoj5xNOwK$}NG&Z(!}P zb1cN_d)-7FoX5D>k<4k#sEv-!2fC>vn0p=dMuX}B8SCP%Aagqve&>+0T($Nx{mxm?p7Du9(YOn(N*1-aLvMF7 zqTlmsG)Fk)Jg*TY0wpOxbvk!a<7 zCr3=&%HS7v%c5z@_jzX=vmj*Y52-KQ)T%ql;ZOHdW>|p_xi*oKRffTP*MlyET zEh=>;Z~@6?DZG3zgr%Ur5*yba7kW+qM zd8+f~L=KFZecnZeM(n zaYP=*?9i7qRm|g+W@imB<64=GKoP<1RnStI6Rq?YAWz?tln$^}#?`#+OjULY1fTcQ zKUuIgJF4w4n={&znwKahIY$8)JXNw6^Cb;O$>ksIsgoTajzR}y zX)a~8^o3x+Sx#kN`8fo5l_?~T%Do0D-yGy5FZKk_IGGXKK9(Zm(Xk5EnfS#Vi&xAag&N^T4Yp)6WKUNll6}2 z<6vAMxElYPP1^S+l}T@jpVtit6+Y)1j&5(#WZ3d3*UL5$eQG3F6UBLkW}Qa4s?2^= z@seYym#DcjM4&_cNUX=X?0H4F^AQ?%XyLqb2|#$dfmd4jiJbqCG`x{8ToL06`$-|e z&*xn(bJ_Yrg^1*LO`U(+RZo)rL0Z{!nqUA^=|T{p&Ik;c=hYk>JR7zoz8{wLSFX;V zu9d^nX9%Ur8dN$kQ-#d_)s``fGdyvw!5Y+ascp6+*t)AO zQ7$brz!@LgRPlkAA zLDL|DLA`;(rU8EWreyu6a-#HJb`mAk^%c|>BZv8?o*5PQ)4g6WLR`h{^2+00aF9g> z6XJZM%fjg84ZQ9zC9>&HWUoU^!zC7GNIGoNGP9H_+77b$cAg6`@PO$@ms%QS6H(D{ z9G=X{ePmM0yv0;3HfU8^DfLi=UbX%Q6yxZsZcna@$2m1N^O8>#CjmM>%x&lCxmYRt zn6e|T?A3N2*d`u$M8oi6i&0w<*{|?Bxa~}!O5g@SYS0_@U1PKY*uA#KZ_=EC^(D_k zL||=o(EzMoI?lxEqVRqrdXksPYhcx0_yloF^HIoR zb~;q@SDor`O5wcSbY?!>hofJ0z(jY`EAB!~?AuE{oMc%t-(djLVYy$c^DAZkTFt|c z$va=>2Q={`ewpMLUvscPWz(=7)oR5ZVAJFR$e$VT^P?IDfKm<1jxQo!-k08sHI@f7 z>G2YUqH||!3Z+D{Q)#(VXi)Z1qn0k)!>mwiD4A{{% zohA$~x~L54?cK|tRAss#<`hD&z4CEQ-HSz6A1)oH!=2)eUCF|#l zYgptdtGJ&sz2F3Srx?gAKdH<_p{}G6mUF#_TaefZ)b5~$byLV7yzK5=+*s4a!FJVo zLXZepxNGzCyVjokn4tLv8`bkB@$I4K^ z-llkEP+riEGDwRxLlw(a?qD^O$Fv^bjtP8%Rs(QLUY02ekK(dewh#4<`bvUi$wU#}nk z836vv^=bu*{sgXA6YGBbd&96!kQoDx!TDn{M>jbEVE?!Qy^%ey$xaXBEb+G{vu>!h zlVE|S@k>l~gS^JQZ#+M@SJsc!fFY>Iq_a7X#T_#!4#X#%3-yRI2Jl7|8ho_!-F$l7 zNS?^`({79=0=KXZsd)p&crhMDWwO?2DS1%bri#xuFi?vOKiS#W#jVupbh0B=$CJo94!+{V#t6Z@NOo+SNi#eZjYx36Ke(;<10PNTc7Os>FTO3nmsjf$%HQZ^&USW zw=q=*{?X!CViX|yneC?^VVKtu$Wh#zyZquJs?_@_<(3jg1e*r_dYW1njR)Ii)^=TG zdZ}=S$BMWKa`=$xr#nv3dCOPJ+c;AkDvVyj%x8v2@`5rf#IGCGDM9BFgD;^xXzTqk zl)tNsKTN0&Q*$vc%iFC3Ouu-B>-O!e{G`7<=IyUN*Lp5{8>4P&v2y)g30gcA$>E+ zY8Uy_T;r?jvKKi-j!EomKVEGY)bNl^Q%Y|=n?k8p)Us)=Vy>7xQgr0co5xG9RbRv!e(~^lgA(}hu}$DhK6N7b$SPc(;7&xelTy{i z?=`M{+6V6{I{6}VG%7ap@~35{;W^X;aWR;djCGwbZGK6(=D#kxES7B&i$8Dfow$10 zKIcpW@1k&6JZnRlVkiS(@oQ^W?sLb%@6X*ZT2Dq5WDDj}65}E9asO*nEb080X8Of{ zWFh`%R4n~h8e-8Y%BYxZp7^h*Sb|kE7>@?F2o3KWbl71S;>{&ct-K^viatvLaL;A& zRM}hHri?K~cAFWZE)^pMJ9M?e*9+}Q3DVvvCO!`3a0lF&de>V4ejKpXqKD{M?$V6(v1Ca@4 zsu~98N$KO63BVf22#~^LDx&NN0De@Gj3#yB$WD+h)KfSvj&7n%z+$u@O(}bvq*qZl zdwn%VApstM!a7Z>HYwCbB}~*Cc!7pfWgT3Ny3F0k%H9`NO=~Aq28S`JQ6xilFPWo^ z!C9GFkxuGG;3*r7%X*9+In&}#z5S5~#h_iD1)>YTFS!;Y#itF7{~ z6QwIVNKm&$3X@Q7=|Q%viF3{X&TUg}DoRMkh&r+x z+YYJ>lEF&&u|*p%fYSWLHo(=&H1ivo7cffUtQS=0ln{?An~pi#sk}_z1iF{#AS*W& zphooNWYzI=APPu~4NO4oHz_Ap3P3&G6PV)a3aG-(DzSUYpWv^{7+Bb)wWHzzQ^pZT z>!ac|QaY$kX9v9MTZXIhhKLXjdM)1$Gc^jEfg|!N}&3+ zexQJPlvZ~ejA6PC7ybpTIFOZ0Kr-SYqc&WG%NgI(aF(-r_$>G1`S`-{T(!*pEau2O z^x0I{?k6~9|Iifk!2utyLPJwr;e;y8y3m0=t8P+}m7DnW5q3KPTh{mw4Ry4=S#i56 zK4;^*Ue?+Hu?YJYw(K~UkPDf@R@Cny_=Tuj zsKD$j*(puuSD4bFnD-lE1OvHIvPLykNv84rB z{MaGKl)*-u{{#>jx#a>zgv#y`>cqzzU86;fEpSsI1ZLy`kmXHfnZ&0}v2Qm(@B3Ce z0escj4-DB0U^?S(nC^6De%5TKv`EBkw+6MnoKce*8l$g|@zZCyVXaf-lt0kz7Zu7a z+^NizE(>@lTVQ_^M|Y_C={Z1gJFAsuKt;BGt`irUWv4aga|z9&fe(_8`_QD)d% zM$NrLC9p$Du)rkg+h8-4+WibKvlA&=e`7`VL!%}-v>S6KXZszkLu-R-A6)X>i?8ws7tDNbgcIUsvKPY-6sydt}R0BWRCfIvN%`d1)3*a*0;Z}CL%E!uGW1yb8tWzeA z1f7mOQMZq@=~1ns7&Y)-@@+?xNr}2x=T_ZsJA52D+`RmVUJVcX0Mqzgjo#ZS>$vfX z9eIUQ)me{P_THgD{C6o+z{{SlZob_ogH@90HwQQs$r9H6eT|sFPMDMrE%9TM@n72- z4qQg=)-h!+%ykZbVoPUR#+`;XSMjq?>6(3>M&(VF=d0{XCV(}sYd9r_$LZ`)gL>8M zqAJb;E`Ev|?o!jJqJn~`bs5`rf8Gs+L5;&!09$nCV!|4&+HlZ$f%8uf<8EFw%T^zw z#Cx2xU*O^~GN->Z{qQ0UCl5F2S^boCzRamKGjIlD0yRpx%JP%#>)oQtmCBnMTPG{g zMkcfqSwI1Lgf}8THFd$_r_@RB9NsOyH_704Z7!#E@skJoIh$zkzi&mS#B96f?o{zH z`-f(z~jrZ%+M#w(YY=_x;TyW|jm9e$5=eeKPz+vXQ>I^8_Pt|n3WQvOMzli|(^IU}E z@Hv+mekdSc%bn_!IKNvJuVJFH*JV?ld)fxy+64B3lETOF<|>98)j;fI+U1=fiaG zQF(Wpl9E5&g!K7gjcZ~{Lpb==6H~Z_KW2|L#MJ}U;Ua<{D&xrGq1b-(R4;TF0SME@ zTs`S?<*8nck}an7XkwsI8_r=p#KjG!PV4!qCA-6SqI8i}hQn^#<&J5x+v4>^sBSyp zdfC$cfa?!gFh{a1if8R}#UZP3Zl60SS@E_5?wGdVjSa-I7R=c|@iS`BvCAEY+&5># zZ8*5ArnvboA|9L)tU>qrtjxxhb0a&{P&ZC4pJqY!+P3TYCV(>9qHickJ0Kcu3aD5GUR zS5byCtk!niS;4e+9IaPe8SR%SVZAaG%QQ#XS1(_th;2#dEoV}E8ko^`kS+&$j-HcK zPT+koP9)~Hra=$L*6yR_YR+?fY5*?O2rAT~401eJ1k=jq1I1D%p)hD$un}ja4^#Gs zj8hko2S}cJX@yYSTehNUOm=LLUWj%W;7(+Q2>L+FMLXt;KqKo>SB2OiVbOs3-z zn2=_OnMtcuVquCIFq8o7l)Y@XBRNOU(jw#)W*e6 zKBF6KS{X;p8ycTr!rEGeJRw_AJG-i0q!2G_#CNOhB-&Ng*`1t}#I6(siWhLcoI^y1 z!mz^qWIxcrbX!(+D;&T`UUnDoAoPfr>EhA(;@t2H&y3N`MJBPzy&uk zFa9=(rj4hi#d?|&;mSV$KBQM^L}ze66E1A@{$$Va+T?t{h0`S`kemM_L$}tGC^P6R z@dBO`wW#Yi_13I8@=KFt0I^p}d5rt1f8(2)#`o7s8NbQ#g0dxL_z)#B$^9OdDP?gl zbAL#?r;!<=(!{$ec#k{zX!czz`l}>Vn9920Q)XGrB+%C)mus-jc`s;L8RC zpE7qZsB-H9%}*u?f+7|713^~3kpdGXM^JyVXBh=VhrWjvqdt zYkvB9v+82mVi&mFn-KCP`>!W@NNND9nS4=eIdKF(bbo(J)J|L-b zexya5Gk@G}2k~hD-}y>gR%zehi4P2B{}52(B8&uFv&ZmZ z7>JRkibsA?Q{1Z!akB?|0--#U8_HdBgEfFOf{+-!ZNV=#VIn$b0xKy^|DztABr;0SX~ zaU9vP@2cW42yk?XN=TGm^-r3(ACj~!Yj*+yH!qj5lRvoPc#}1A{AvT%*Q@!Ru+;z+2)BKSj`;1o`r>HvC|!0Rec)_OnaC@Ve|W$POvHPjWU{|vU*fj2{4}n!;R@xmlU`WP_$+~evbHjC z(Yv6Rw|5cdg3fZk-N&~WC;OAUX!`}O6MHyS_wJULJtpmz*wfJgj%%%CH&slt@H;@b z;(7e8+N`>(9I=oy`v6^BsOk3jUKk5N&z43jt)9HCjdY3-!pN7`)oMA-^Si_+SRymV z`^8e)K=L%1@$5pXU%-vpeSR_4)y2UBwN~E6`C(P(3~uwfUwjj@Hg*fZc(JF-n&}yw zA0msN(Pf5~A4+we?!jEAUOaX*rx)*8Ujc(|{ zA49)RF`(_BWSiM0PRZb~8mYj~-{{u;XiBx}^lM)D6GI}0H@f}s{wccsw_b6#s#Lyi zh|_@-CmVi*qkQt#WE^1>`7p6+YXhDd?da(ImLZzT%BNe@iLw#F?$TlPO%=0ey2IP> z>D*nf7+dy(t#0H7`x0@$K5@|m%+-aO`1&>me}bC3n8P{F^BpK6W*$C(UsFBgtqyer zY84eOuCrJpd(!p;-PWyJJn;r3@8*W+9%k0r-S{RPhHR+QO6N*si3$D8fnB0TBFhyL zMa+_W%v64w3jG1fF3niM1fBRvzvO7oaWDQ1sV>V8LW?@9$sOf@Q+HLx#S@ru=~kcGA_#(b zkuBNPhTCR>`@BJI>!+=vo`}~^yKgp#)a_tG8)T=$*EfK29kS{T*=6HC`pFfoRj)s> z?0`ECp>xE_=7Kumy1DZ5Rc~xqe1N1?;Ar%(l?Q;My@9*XAOCBzVensDZ~RAV#QZ;! z4VAp}FM1UO5`{xaP@dr&r3gA8L{kGC7|MWi_Kjk8=wk{73gGE6vOUUh1{6LnrF?{0 z%1BafIaUN^cMQ3w6y>os!Zpxup{x*to^h%0o!rL|N-D_1@gcb%3AFVvm=Ke@R#U5v z07L*B&0x@O(`EiH*SrqQ4Z%nkV++>@hyv^ zmX%uoh2Pg@Ad4U(C`{Id$;|JNj1F|g-L_;TP_Q`>x4Xc9cH^>w2xLWEVQ}jWPzY!< zFz-#g;$kh{#PJr*0{VCh!(eTbK=~$`F&x31z*zVJ?SeqArR9uG$%2K1F!6E^0on(a z4oy-epq}Bu(?JN{B$7hKOTk+z4IR;_bOm1QxN_X9V5xpxPEe z*hcnT+3bRnM2{yQTO(V>+Ifz1q!eTtxzZ1MLVXMx`Cp9E=g+<)xeb+)UkC&~AV)W* zrkr5;8lNw5#c&Wk&-wFI9oPi=CGBP{r+4h)lTFE^YM8+lOy}2{B)cof{1QWrACrS# z><`KaMkjux*IJ?AVt<`kFg6LTk^Q7UYFj41R~j6UMtNqMjNFPL!!r%rr(_SltDLmu zm?d9zhKfPLUYN9F>HW+0O@$)HHtG6i)$tm^Pu)iWdbD6M`Wv& z;!1(nz!`B!4X|q^<1ec&2*65Ew;4YfhmcKv&!$mIWiY;MD(Qw%xD52AGZQQQ|wmRRgI#^PB zlod@M8zW&%kwLZ-r+k)!bZ|u4#gejB2$K<=oG-ysizH-ggWC)6{^VOKPGdAu%Sk8z zscAcJX9lwl(91TWUREj90Z1(tBNXtbD}MOU7=te<_oAaU^r*?~I>u#Fw{GQ3whqCq z_-siCQ>>}rCQs+WlPoUB`gvVr)}5*!JlC!B*JSztZ4JRomq2(b`MsW{B_wor9e0v0_6Ee!H&y`@REFwiTmrWnO~1-BHGTfIAkqaZl`p(z(& zKUB>fj=}cV0IzK4E&TI0HGJtX$;@LA*h#t+*c5U}MDsBPFiYG4X!O%wLLDoCI?n3$ zLRGFmh%_La(WUW)rDqoc_s3pl{ifCq!=H`O#SK<&!&jcPNP ztMD(4A%2)a2!et<%4~N|{2J`deck07q*cpbBDbGo8?46Y4TzT|nMHuO9)6jMZWaRg z+(!FP39sm)ni2c{9v?>y_Qi_qaxuNZ-ybLW!QX+u3kyrBN*gJDYCznhG8=I0n|->t zw72Sxuk!7?DU9@ZSG&kbEI-sK!ZjZ>?X`YC1%qoA5XZr)E;^#>R+@hH7H0QubtLR` zh-{AQkpO`VZly{e|pEt|Z3Zk!ixE!9ZAn~Jl$sk(Te0zLo)dDs}> z!XN2A9po3N(vO~pZ$X%M(+FX=eLM*IOydvAot;l7{@4%|o13RYC3VPGz!x-1<%&t< zeLX~|f&olaXD>YH_`%u~f-zz*s+k(6;hJJnRbvB^O3+GA(oNo5AVj*Yd$?n zvhH_W@lHQCF2(J_v6=)TQYv9v!6ts^6Byml<#*QC4e_2$g~_-qd~pUAw8ed0|u3hD`z{kXGmXZgHo_?Bi&I<87%hw@MGd z8#%L0u}6!6iu(CQuI}S4D=ucW-(^-TM#7vbq-`7^fDx`zNn9Gmp0sJfT=Glb%kjyU zyIAo1Tv(t+@dw=Kpo-4FH6W)7s*FjnXMe31NRW(GoP%BlXL6B|Lyf}9FdqUq3Fds$ zAZx_SEk95CbpE#r<&-p|RVqK!Ak>rGAcTrrvHfrX$mPdX@xq|N-*RN{ z85rxerefXUwZmeprt?RQ(IRPRASf~mvs?tk$@gi><&+j!HNV)JS<~AiVIk-DT~~|) zar^mzpX~_+2jWlBmWeHBQgKLy+1vW!t!~49iHkF_?!VH@zoH z4bv!#cU@+D*!wBk={Z>Bl(dZYaV82ynSvl<2S{4BY@a(yLUnx$WQ(ZRDw$z;NHSY; zwN-QER?qj%T}>CfKX?6NToB9V^tjoE+YTo>x0bltg1PHK;HKf$xaT4B|M!md*?+46 zi?e^HWl{R?w2Wu*zi3(13EAi7$|A67#Dq(7Hk``+!s52#mQZ~pqEtk+t0U2#Ifc4* zC{tzA!~!I2B$U~9gK<fDM%aGfihV(f~US9l(jvbvEe|r!0RZK@S7}v z`j*IfFwdT6qZk2a2E$ykzXmiEz%%8X#WnJ;E8yrp7@;ymTSQ5E;pL?ppTa*$0@c){ z#z+hKNYsfbzL(6Qa-Y#wCDYlY2xJoH5-D1>`HYJ=>O%AO3Dt-@JEHmAt4eZFEPqdPcZ8p^|tCt;ky7 zAe_#q*~eUJX1Of`p4i0o1}(z%^cAGm3aJ>HvX4F~hV(SrV13x|V+jHlk{@XgJTqOc zs6Z<-sHaBo7^!(?QLC2Ial`l>jko~C@hI_YTY#r`sP=zbrdtDAxuNP@(sVd+EbXg7 zS}T6aBCXg8_Ei1+CLq#J48JoLwL;dxK zFOdyVtXZZ!rSnA&i9i>pKFWKme(@6JU4hvAumY3*5)}D3O?(4@9&?mks%hkH9`!#` z4%wG$uvi39aJO$5x}96))<^iyOi6U&AM1EI*f))T)*W3i(DmXFoz&stLE;n^Zlx{I5Jqj zD7z4olHOpdFu1z|lfTl<&m(`ajim=opj-E-(hd33F*5EsTWMzCkFt{G1R*u5uq^-9g!uYxZ;-_^b3{%#q3%?7Yszh;sKszEJo57u^)B|Wi4zD}y$QiDd(hgFT2 z7r7sSw?c*37gJJYjodIbs;>q%`ZlQut4;fXZQIcKZ>kh9c5m^R{LV&}SbkLH6hy7G z!U|W%(;iUvQD%7*)ZU%i=VxCxq&x*U<`-C~KxwCOq10J3jxBLH0P$Il4AW8(gZMNM z!)~e%#YoNz`BgF+|5!QWRqg|Wf~=Rd4qr3KI*3P1)(9sXfLXh#uvc%EonDd%Yk%Jg zblLk=KZgYJR~#A0Sz__iRj}kn`uz%E)?_+h<_r0md#2e_@)aYAvA zuZwpyg*py_&Nb4V913T+4-d%nAzA~|mmx@ z{Ni4`$9>)ELM~h7Y~AO_P?gp@{YON!(G> zOWX&t_R|t$Y2yA?%{`~v3&!iteX2Fm@T1Rz+KYy`nZ`5nuxf8+ioKfyeYi^1#Py=) z%rV(-Y5zs*V9dv_YPvm*AeB;?-e6u?kM@btF62g7_cV$3QM@eOQ|03bKsO0v9WtQv zXZ_9LxmhM3t4rhj6{ZI2qk43AWi&R&&{2lq==FmrD8O~6vM zZQcI*SVA1`t}0(fZbVz{^0N#34ZLX;B$uy~ZZ7Fj+^9 z8;P={p=rWx)>?4`TK_Y;V)sq(vo|%YwO`9@zY8=n4SZ3Bhzq?EOdtQgTbg_39BBd@ z(Q4SW)y`mFHT$tHxgYc0Es93jm-MzXvaa@*4Y@B*mY>1-2Q?^W`;)$QlHX`)0CFOfvMW$Hl=q#Q#!F{C1+zh(N0zqY{uAjDK2o@d>_e z&>3>3k!mfY`Ia;2M>xtqGl9?o0V%VKLN$2h54QBv>xIq_mdnpIy7&ZI*w!&v*z*Ss zYi!wyT@I3WY z+owl^AWPk8+x&+#_Du>9|3ugA0ogRuxmXj=HJNoj<~G4$@pq_{=Vi`1CYGwu|8D5) zyDq?n47m{FO-(;})7AM6ayL?nB#4XqnGMtR%CM=- z92$k$?#YAV@geA~19dRaDopzUnp9jZmp{-;+%5vU&VygqH2d%|(j^Y*{pB{&Vcq8^ zAHcqxH$mp=1MsO`%(}_*kHgNn-D1#0Z{V|nh;B;SJrm3YlI4Az2jRQc(R zGU!U~K(XU^HYVO59LJ&Ey{x3&T&! z9l@q0_+#cBl6&n!|5^)PcBV{Cyh3(WJmC66WWM6{hs=A82pOd`(!UaF%JF~BLJa>~ zoa{fg5c9JC=*f})i;l%XAd?WibaPK;@UWl^+)NHBmyoFPV4Rmenc-Z*(QC@wL`rCj zW-v{r8WNG)mbK#bpnqY>SkW6jPcY0rUcj1bXclM=1rA^-$QBt9YF?89gR$W zh!)ZoN3xGZnH<|6KhL|exkS$z**&xk56BWPUzp;$WzzUikSu5IIFQ+p1Ps^89oiHa zG+D8u()mE}(`V=c@!iNt3;yvFFL1)(H30+hK=L;IEC*WT0mL9QQ&= zk{dx2)A>9hqIJ}=AC_b{ne7KS-3~zpHTE1U_l!u(oEc(Jei(tYY^j%d&B|~@HGAP* zDo(U7x{Z4%2Ad zDFWz^en@@D7P2y*Qi81=9A;BY^fcZzxF$X(1wWRtp``P$?ss}zT-2iKH;A8CTugAn{*a{Vt zyneVn?V+=YST2s zREYRK-ml8a5T4@xxyt_{iwHhd@sq#7PUVYDxdy|Z;O{TW9idW+E_NV5;$F4dih#8L z1_N9?Q1S7Oy8Vbr?A{q5&Iu6)%7Oss>?idH&ouw}GQ1Z6__(Q{lAKh$fj04(Vk zQ_emfOPsqh9A~?z{7W@jQu!B5FnAeXt;*2!GIBm7r1ehY%z_n5DXq`GTF4L<#z^EA zYo?FMl7>bImU$0`KiI_iS_`^KtK&X;)jK7Z2+E zWU-NbZpculx3kTYG8QI*LC#i(S=>1?X# zgnoUEI(L87Ag5N%!Roz<6M2A+@ncE@#0x@)yDPlS+1W54e*9=%=Rc!HhE^+io7BBx zU4$3D2p>H-Ah(WK%J?CDG|*tV^0?=0HfQ;EPT3%{LC((P&sAMKq*sQEt)I&&0wkW<4L#C`9>Xn(i2UABXG&JrpYE=TQ*5l!XKY|tP+fixAI(8* z#VeWgdrhV!S77*Vt`l=R4)gSx3fLzY`X4k>becW2kCrY_``Nr^A6fH@o0-lpQ`Hex zvXGgKKfzp&TQ}pDcY*hQe+%lkFMd9T*?;P77x}jg%}f5SOEO13%=9itp^qzE2>uCO zRjdZy+e77U$%Xyxq=UnJXh23TZ}z<4Cb{k2d>d!Bh8lxf1IcUUhw6Mht(JPcH8k=- zzhRGfWy~Eg{p;gsKePq-vd%UJIZue_4JRE?!$WDwTMM|K^!JLW=9vZC1=x!S1q2U?|2ILX^p{IFR50$Rr<cfvkz&EWGvx98D!fydphs}A*A#=FH(nn9*bsL(ik(}248FrH*@xH7E%87P zk$f@V)@}UeMaqN<&fr$xJqckmtG9Lub)8-56tBvMgxA0FWM zP>!_qy&e#=Ee)}kI+pr

;0*R7RSHscT_ZGL`c5WQJf$HTBsvrL63SOyfVZDXZD04P`6TT%yF0{jl+D zTUz;(1DF6GKuhMM&ZW+s;4vvk8`yy|SvG>Fz95vfJ)N=PK(@f^D3tJy5WIU!BgbqL>WxCOz=t)uFzwegKifmCQ6UHbpEL{E zbn4_+Vd`?(473I0E4Mw{mXRO+ZdKpBcT$zSs4Z=DN40uJxH+MccnYn^THqj@&ZybP zTxw>yEd!p|#PtR(!u9kOq}B?l7@D$=J}HLuG}>T&*zjWs0v3`VX%9RzU9PA=D>JC4 zM(`M^d1g_omeX;=_#KV70LAer@oQUvr+29Ke_N(o16sME>Ri%vIB_iPt3g^Te##=P z*b4Sk{rn~%(oYP(GZy2KoWU`WFYWcDwx;oJN9OfVjfR&mEYtKR0~@C)abgMtF5s*`y<4Ra5^M}b z>I;mspX^4h0Ou$)ghZGyf(2<$4z|B-N%x?ty!5+gNp#DGbW;w6eT-RpkAoLL?1?=>2I7Ah2B2&Owty-s&4N~5E(eS#=YoAvb#xoz6(twX4s@+n9M$v~=jh7d>AAz?* zh1eHUQe}L1~&ROsR*l0`+;rS(D`qw6fkyg@tFM1MwVE9ROJ*zt+c`lSI5&H zQ1($~c@)&%o!aMTUpJ&Y1vusxSg1g0r*WawSu>6;aXA3-S&j_TQW1mrG!VmXst?6T z&I|cfG8+F_IpbCC1A~IBm$eRGGsrrKM@-fTCmVoSyQ#2OZ%@S+Ezj58X-2?ioL{Y7vV}Q(%@hn#IsA>2eS6l z5@TuN{#MOBr`rq0>&|_uHPY~-&x6{FhPau=GxD%%Z)S?Un*)8gO4Y>mqUOvo*>7q8 zMeAV9$FFL-J&hohQkmXhURjU!iP0|PMp*YWiT6>wEZtM(;|M@E31b~Hpz~+_&EdIO zCLgOyM2UoEebXp+Ye~u;R^;UPY^5D{eo)cb?Y2zEq(*mrE-Z6};YRFTeJm5vAZOI0v)ZuL#TQnhW}{`y!# z9PX|vUq)_3TkZ0*3;PYcX%!@wuaj?+%%>W)+a292_9dlblDC0BKwB_bM~xeavZSGD z!fn=CaRXZaGrD5;P4KffHLSH?%WS_3G%^i*QH6*Ly%J0x|GrzAd*&Qz0vpk4*tONp zU|%)+u`anE^W80qM%kD2wllJ>_LmL0FHe@A!TAR@C}#VUzIKw|XldjFD-#B)H(Ctr zG}F0Q6VEl7bw1`c!C~=tsFder&N?QRs?h&#=<47P*oUi-ucqZ}i0?m)XzRe!8Ne!@5*m(4sEY zg-~LBLilxxWl%asE55uc9@d%tezhHEfIRlw{bH=fo5{mEt%k=G2RwNqG}zwVWz<)3 ze^nDd#uXn_{TL@2p-vk#2%ZeN5aUfvKY7#D`3-V6Qi>#qi~E@k)Ah=*smvT2h1u@O zgW~Za=&b{FFwiPY`vIC%TrHPB&`aDd0=v$GU)MDI@G;UQ4(a{nHqv3;=O-V)zMMBf z=IR6Rsa?#v$@Al@w=wuK;>ni#vB7`UN66;f3AE!8c~YR<@q{Twb~0jWXy;?m)5Qcs3^P zb3w@BptHyw%|bk&jFTNe)m^J^JK%~#=2JEy^-A1OM$P0O6br*o${oR`CHQ0J9g=(P zLjPI|UUsHTO}s*ORXpJOLu9_<^@q%RjR+Z~G}6BkYRd6{&O!|TTb%4awh;5O|LDn) z|BH^rK_HV5y>xR=X7I3}4BSi(DVLC_@?e~oKAGWM!qIEW+(b%fi)JuQrWz8F+m^NB z^`L)Y$>WOAPK!Q}X$@D5!#^g+F?Hc0rW8|}AOZ9Oisfnra%PLN3z5u8lpDFMU|oq^ z_f1$Ea_MMZAd?cU;UWM8NhCfY;OxU_0?1V*(x7;IP7lIwXmjcnY~Ltlhrt#MMp3m) zYRt-PNIce(`*P}DScXXlG}ZzE@KZqLW |%f~K|cb-l`1{@>g(TEn(7Duv=M424h zA3x8#vbjXh8reOx4G+i?FJGAAx@FS%P>?KV?KqIxkOU0Z${pGi7&KY2qthm$Yo$IW z48iY}DOV}6J$C4GT=w;)y5U*ipLc6ArH?xKH?)bZ?6dkPT5>5HlOmvbqjY9Rup8`3gAY)U|mCh0|Nq)*#^_(NT9kX@g_WDe74+$jR+kA6sf$QH6P zpHhOY9UNv;O!PF~HM&2=8tEv2_<)Od9->0FTbv3JJW9gB7~Z9}I}ruE=E)xQfa&gu zHU&SHv7w~%uNkj=S6obR!v2t=FJN)NaJ7nIvlVjWAiRFKJ?){hiC8X_ z^d_Jje2+=OE2s&>wye_1p0vGurbGb!5RZ4j9d@HIKxYNxJGgQsyhC>;Wla9dJyfL? z{|Ku87)#CreS3`>wv<4V+?(W(^OIvtoSkCGpo|xehBDKS4Vf)W;E`fnnl-0PPWX|| ze@$skc3ZTQ6h?A<(Ak~K>>3JJ@oh|)bqiKK$?kAKsc&M$Z^ZFkj8usDKHjg&$`GF7 z{<+HkB8vzO*!A|9iO}PfcpWyE=${nFniY|5_K;mAt+KPa*{{{nGJW%oRj=KGb zN$lPkAkGO92Fii}=;Y#|;)+%}Y_MgtPXuK`_|bD*6F<~ztpF_P7*ozZ9!s3NG8|{S zsQgPcT2lEJOfYyEU#-f}^fGcjB&79D;ct+zaM8lH3P^_Ejj|Jfqo7Nqmz`QO4=)Mc&SHSTsz# zEM2E^aNevCW4o`%&)=a)-*RP?Vz(kdkDt?qMK+Ll@eWAGJrvC|D`0!40)9KTU;MhJ z^9Nn2m!PuPnN*u@hr%zO1>}3(ks#?=n{Ow<(>xo{Jhiaa7Rm$y(mX_nZ7Q37htk?? z(n%N>B2w>CSkV44hSUTA{u)RKAn_$6{nqVHJ4?WAc_hI?E5|xXpRytFnxFN7^ifjn z+_K^3Nw$}rCSU8XRiIrwC8^8Ty3O%JO5!QLQ1`R__0gy=C>8AjncAeZdtD&WOM0(=`8G$q+lzVJ!4Z=H(JHKT$T64Fb)ggNWAHwcNP_p%=p%Mh$onF`n^82TSHQgoUz<%jBgJgt^`yfrlPK)+#+cxB8TF#YS} zXg{e>;T#Jkv7DpJ(uy8G={HgMf;j=}D- zNnj|(7($fEusD$50mvjDlB#a~(9}5l^On+Qg2Vi1V>E?>N`uH57VM$QGj_`*O6eS+ z^K0r9b9{$Ne*v=ioJlgcv)m$IcD(>uNP^X-ZtvjHecUOA*b^K(f06sar+O%`M(l-x zanZ#t@16n`T#vqq7Y@eZDd%>}PbrSBQRG^&zSLWuunv|9A+FaXXJaGB7Y12hw`TvK zM|v-z<=)X0>lF-GkOH^dO&$-c8T4yBvtjyems4&H-+pj&+nZ+wg=I7Ie7Bv1pV~qf zU#-H-P$}j{kicI|97)39E>(UV6k_F+L5h`4WA^?!(4&m6-65^9`|~kgxLp1f$dO{f zzcb|V{VKdum!L;*vDXxXpEq6}|JV?BIEtNHehj|DE!l_H%`Ndj50QK^-_~vX=0(bc z3eMtumX8s#U*zJ{E}f?`DW^rdi+AuH-yF-VuN!^{l_XM8a~~ey_fU?s^}QYtvn>s= zmpYdEeNc`H))5`zpLJ#BQ1W$yuat7W-DG@gx~HnyU#?1D)9GAXrZIapm8yME)%XJw zq$}>LRrVXwMvW-5+9B1P^Hcp2Z8Epf&6n9EdSBTR-IbWaA}{_?*zO3eNnCky_Kll^;q ztoRopOQJwldkw{%e+I`WUvDrd%0NGHt6yj}W!-qU0< z9u=6ZQrj89LCH=fcz%I<*@y)9$su|>1BBCTs7AplsmjYX(1=Mmyr-)30ae=hRce{> zw#%H0o9PLrOZM$92X!F`7Be` ztgJCX0~V0A=#zags=N{0qxv~iF588VE-zxQUL;cu>~9)=zQ#1z!0 z_U^cFEVSr+o7{54NkVpAWp>qW=V=$c+qa|#ka(qZKSwK>7)w16=awoNdR4p!txrRf z=MrQuHOXful`_Qc9|jvr@hd-vQWn!FN#sn<@7J6&d4tLmi_WEZ@l|{j@(xEJNu;u7gHf*%%7C42G3zj0 z-l_o*DC+|#v6n2L;7z&L5dpeorBe-G%|Bkxh4Bw~j?-^HP}F+TZ1PPCOaM!4H)CoyHM78hyq5eMpgt|bFWFwZi@ zZ;e@|_+fQI!m%`70Ctf}CH=ZiX%DRkHWy~d9tRUiSx$aFlVM$UR>}+`dr!wtWNyt0 zOCk(%5_!Q1RBd8|&{CAp+Cy$DJxuL}*p;%uXYf1+WyGs-+mf!-hs$oxUlRTUsg)_K{0N{;m0p`4fxzY%@>A+!caVjaCVf_O zNKN_q-Kr+;bO08@`CPVIvMhVwv|vYk)!-o9Z>d<6U{KHk_o%E9VlijWc7eT@MFiyh zzSE&5&tM3l%xeM{=WmpKk>WmA{LW;SQO6*CrK;OU)uj8obO+T_ieAGm=yud63z=$M z965%W%V?`akUCyrWtKY3M}|4tmQx%n+f@Z7enZo&Bh9jJ`Yc6ks56IX<%6`R!-M2x zw}vy?V>ei+N+S=JRi;$ECIh7u)8-#hF9L8v+NtJVRcFf;jK1Zb+f-jt$+b}L#++=l z4FGeo0jOZ8m-km>V%bI;XO61rez*a$J-JE?Sg~H>e)MC2ppyhKh_fhjVVowjYJn(k z5V+(euJbD@l13&6>Ei1KQ1KZY&o2O2M)$y^TwigX%dBhr+G+eF@*zn>V7fo3h>J94 z9^ZxIGQLx$p%(Y3w1=xpyE?aF-zh6yo<4~^Ab^D(2rRQOW$1fd_9u3OfAkA*iAs`8BM4q9udPY^G zxUWaIQ%i>0){Nt?Q3aTNyW>ZyY{Gor z?xfFgP3@Ay5AA|w+w5o7E3`M<-DQgSZeY5y=2~-{>8wfWd{?~xj|RkZU^)xPC&KE% zVf28zV9kHmN2`Ns@l)ErqDq2pZ38h2PQ=>Tp*bQE3yvs*B~l zKoYjQdPdS9q%zm+Zg;d-7SV|(ZD#dNpk3N%*(hByJK`mmzoWa4$#75nlB=_U3fV@y z7)(9?W}RaA)GYd;F}ig_xi!{b$5J^)Bmb)*o@yF=aA34$BGtQ#M)9)iC#P|P?dg{B zdBuIQ>E~ZIlt#RtPN#JKgtfUKVnH`m3PaCcJE@%}lMKTuv7@ttL)f7a|CRD{G0yh( zj_e}_#w30N4RmflZFfas-@AIMREP7nCa!NV>&_OAla=B}8DwEy(^%Pq?%bDo-o+Y~d z?6YHZ{>KVAeVkWa3TduW?7ta)1|j2f#AYU+RJd4OXZCksJ6>~j{`o$dYLc0aoFC7m z8}tIGpJPF@zat0K{Sj@{W+~R8Tes8v+d}6zQeC4toM*eK7S+#8$Et=#3hr{`1n?21F~J8lED?U+wdGzysguaUC+zp=9LKPt+C z;6FlT|JJV~O{2~II!5;mD`W@7vdR2p3bu^I>EHHgK31@94EJ~p(KvSsPZf?v+#)9Wb0*mBbf|XTt8bu>5b)v z(9}+q{86CW6bq4UXZ&zEt2j9|DRS48F5D%w$%-J-4b&*=G?Zze!u1QlBqBUBW7ITj=y^+JLuF&<3wmBPKW6 znopIp%Vfq6UK;3xavC*RYP}qH0W6!ELlHN!H>pP=ajtwLCCUNsP>$lZ6qi{<@dc`Q zozANLur%a2S;2~2AQvJ#9d=#I%zzbhD%B-nWg68OEJTCM&+2R@au5lU^&w)Bt$^B}z8NSS!+}Xww zRDN|j^#fQgcY>aUp*Ta8YFLI|bgy_$CU{k8gG9)V40Vv{rAPq4l3Ru%<0DN9ssRqX z3w~EraK}8dC`3V-TSH+6>RFHiZGevu@0$`rIpOVqj~hV=b}@XWRpwoEiJcvM&L-ZL ziOycuQf2Tq38>=a=ejbvDDbz)qn)%^ZJ5#}sdjq?jM+*islRx}E;|Djd#O-P?`eK< zl>x+c6Ht5Jq4}X%4d2t+MK;IO_&0T8cJ~94?5z^3pDuGX#eIN7VNG_GY=&z`{`txw`bM~iA;Y}AEmj&1*E`>@G64W!?7B!y>~RlwzPD^l&&kTsKTfVj>~1hZa}L| zq0%p^c&TSs8Mq3RR_vjIMqJ{0fu~Yp*?^1Rr}qLh?#k&sSHQA-SmkXBKALN>esd~! zl-?H^NI8v{)!5bnx}WP^7?Pz{xp5NK!bL7%j^0Jb&9%;WLfkAX76rCINvgWvmIw8Y!Aq9mQe-8H}$gceos~WOa~{uPlXAgbg>z% z=KevWUEatglO6)XG!90gCTp=?rt;?dR6bY8Ed%~h#ocC?C zoT4XQ^l{l;7=nH!7G*?52;revd%>Z4r2=}pi7V!zwBf=mSggMrG{jSibh&KEHl5jY zBWm@7HCJ&##ePzi4aRp>#eupZF4U_1RO@}9uGEXW96!s1Y0WLWG%vBZEYoRbF88wq z7~VJC8hMqZLxk$tav7+G2t7}cP!k>#!Fo0=cLrC|_1z#tYl zBi_<#?wc;V%ca;AKsX(w1#p4JT$okJk2&A=vqTtPe411mGQBFeN9DU~0Db1hF5Kxk zTen?X>NKz>^4>PmdsH#cHu%!bzCdt7wY!D4g6NAB)bow*G8q;+Rs68RnpEJBT846D zwQNAZ+|;6UXKLaxmR9swP2*4Xm5I0Hai-oh9~&Tk(ph~p`F(6BxA+>1Fw@^81Jn6Y zvc}85d~%6cDdzt`N5k_-3t`ylj-8*wDY3z z((Mvb#Na-H?N!nbD`?{heAtUnIty$+u>$2SLr@9~P*P}ziYE8-( z$)2Y`ZZji?!Zb@0i@K}sGhO^6nPeu7A|#SJc|e%{TJ^(k8H)YoYC8-l9BtHV?r+9; z`SXsfLRxE5%Nbq#!;p;`{%&x4%U(Zxlj5Uqsw802@A@341haM2SxEK67DQ$RHB4fa z?R*jfN>QexqN!De3-X7=`2o6uMF7z-u>LCU33ajB#5%g$X@?mV#7wG?a7~w`z2sJ~ zt{DqyP+EGlY zGrA?4lFGI{Y_z0Q<1@>-FQC%IHw;FrFZ$R3SCQCu4^?qCasCa;K)0W<{TQd%lNMp6 z#CDC(2Tton;IB3$bXaMZ+{jCNU?X_9Gn!_l+ZV$gGK6?5b9r6K%E) zwY{dEQ*r$9SdCAu(MHGAl9V4Io3I}<4oLnlhHn3OfVu)Y7U$g+y8n5&|6dG)eMv9( z8)webvgK%_nCbZO-}-cMjYZnZPtK)veH>o>&oH|ApN&c@exY03hIJTqEI&SaoNj+# z7q_crpl(M^*7fdLX(ydCT%alU zzNjB4Zv$@j9a(kdp5yYfRcC|Zj4*NRINghv*J*})O>yUTvAwB^*xt7y8p9^rQ?{~o z|Io{?r)^E_b3jc=t8}(jWpg5K3qCb2zG~RFI^~&qtK28(3@(03UUO)aG`0*@)@o86 z2bDw!N-PNPj&7A?775t;IKpO{dE(S(snsoMLkM+}arp`Ya7_D}#{B9!qYmMsqnAuJ< zS~1pCN~{rW+H3vNUQjF|qGxtfsfGKR%eRxJIaf_T{dG;Jx{Jzilu%d&ERLIghNE^U zBvU(R3vo5%|4^Nfsm_iy)Y3)no&f{%B3Fzn4Y0B-3m4T!o7*37PKye|sMXwcTCSFm7)2^SkV7APHq;?+%+#&mX>{9mm z*yWsQC8=x0bKi^DVgJ9nC>EW6>!R2dP2J4NCMf^l!M}C+$U7Yw3-dF%Wmr6$y6?&S zw>h*MGx0SlC)=Bd-#Ue272^OfDV=yEWQWdG)UI?I;S`g0k|f1FIVE}%EhSIdu}TGm z%LgM;Cg-w+V8LWJJ8E;A7L0PTD|}Lx;cgif@9OoUv}MB?kkDWfN?%?*JdFUROk^X^ z6P>3t2``=@D`AQ%oyLCw;ZG;7U^Z$0nCG}M6q55g*j3dB)7+ICH??GS8@_N-1{yJv zcc=%CRmP#4#hNmRmou0noJ$*20&vIGT(~W#DU( z%u%{ivZK%F)QOO~N-UJgfyu6_JU`k{W*W6km_uDIr<37s5^>9^xGwuA%%&79CUe&( z1?hZzXiB?hd(pY2lR8<-yikT6>p1{iK%>8XPE7q%8lh}H3m8Q$A!TV)y;#O@mf{e8 z*r!nz*D_YNQn&6@G3T#oy0uiM=(SRq%3>E7wl9;nl;}idGoEVYqv@;bl_reGNXCvh z!sOHoiYA2;+uA|r3?gC!R9?1$f>wh3&%n|zaxd79yBid9PZh61f%|{fVQqlUevG8< zW$q_a44J7V&jhPYaVeBJI3qOh-l`^B5bv#)=~aN#jo8DPkitxVleIG~ilyejYgQNT zHK^&0==TzBwj#=oy-ZUkCGj6=zX4(_UNTNCq5cf!#SphMI5oYZd_2rTwd~W8=lg&g z|3W4}&$Dr5YTFuZ206P-07_Pyez1^JWwxw#u@BK@W$ZSFm(O=(-ytSXK**}@&ve;h za*k|2A`dt6-OI}1Y%0L3Ob#B3!iD8YkMeC-P?H(Zpfc4jrt2y|vd`INrf#wT=IC@f zXFpVF2ifi_mRVWBMF|r8uarG(t*~gLI_laHUPz@&rdzeeyBfC+(#sv&skU223d0zK zJF&{_gUZHJylR!j7ywTT?ONH43JkRBKCH8{B#VkLK@)X9T4PozZDwkb_q&Zu*z)jb zLEh<{z|SaFjti?D#s}(}IE`1J2&nNW#A;31JTUm7M!wbs6wM0h``VOMZnswr!(t^_$oR+pO?Ym zyVNrMNA|dhI{V{|c67j8c2)UCN2ZND$Eh$$=R44Vo2~GA%L_M^mrx@|@?%cY%Ql!# z+pw}ttsZxE`}yx`20vLAV!2OraY=Qgr}F{|zE&iSBmVN53oBcNwfw3&3*0|nwp3<4|6*&}uyccPs7!Z?em(j)+)gawK{kH0^A8k{1 zdZ2txfM7reS+V7%-z#fj!24*N-RHS2)29tqHn4>{UcluZ*p2Ko-j2W)bqlXS_s_m= zP&p^&GdjHHM-S_ixFl->Rxm$5yoTd{uD0V&Y>^kL(kcMh6xodHnPxktc0PQ*+?kX- z_a7O4xW5kd0SS-C_v#JGp#w>C_pMN~GCU+SZ9hpY+55EY{^s7~kqp>p1MnO=2IG2{ zOyq3kPuVj6@{iSaQUD9%FZ9CwlncFsqu3|geu^8}f2zvhz>jnaTLaPj&J-J)L#+&) zZa{J-=rX@UfvI>8Sz<%gDHKMh7Bjg3s@Kx_pDOAZW&4)M zo4DM0{p;YFon?%4VU-fT4$_(Rmnx2JS=IU5E!jT$jBY=? zS`fkWi}%1)Z!kx@w6=HoO-(m9<1opdg3u+w}mCP zF%;{vUL2og8tmZSQK!B8ErU#?A~RAB9b<@}kEywbjqwwgwq&;O^Y1vavje`fa_$7` zdAo;Nj}FvFiXfq))-mHp+XuR5cJrf!oCuKXAU$+I_6fCz4a#UHn@LG@BKK=8sxK@o zEHPNCAL+VyN2^jc*AX>lFW{o&N|cM%3LiCn00z(weCNSq3_qayMDl`&qfJ(izT?Ql zMBE!B=iC-|8m0CDnHhcJ+KFY{}pJ7Te>b>{N ziL|8)A($+)N(4iS`N&Dmy=p!9D&{BLZwsCO-YdHwDNWjj^|WTyFAl?N?Ajdl%28bW zc-cn!UfO|kI@8p7{$__9RX&S$UlmUxtRx<;)ZBw~ew^Gg?ddKYCYy_StF`j`RsPel zWE6rvzc>Pqm@+`V&y|c-PN=h62v^xx8Ukm$q{JfB`{r9?`7x7ZLd9hov^5sbnPm&4 zBgPPctp{(p?IQY1uP!drM6(0ZafU+=e~F9NyAA$8x0Jr;2E|reGV9f0{FY3^fn~Yb zP*q%MaO2~#ezACxK}vy!fb%AF1ix||CHz%rdUdgrS9QTb$4YP7PM)7$q8V&a+1KLK zAwGv<#`WwYsQ>gw6%5Eg z@W61{LWa_ggV`p~6NWp_=w>Uds(V2<8R8kTn<}i4tI0Qd+hxZ|ul6G~+$TC>48RE` zOxv{mz!yx|yBC^%MvG^7fIe+MV8ggWTO=%y7@^(Q&Zq~R_{gZaxMX>oS=(%bTgepq zoyL=PcQFw^9bnBABSBTsZ@ab1C9*xH)emU;!wPjLt(Ix=%cVBEdK*6@%XOro%DxEz zv7k7MVGtxvs!!0x!#ZW`a({rP>IMuCdE&%Swd%e$%paiy8tb>U(WCdZs_yrDC~M!& zeLugrUl$KnYB1jXC@L*AuNC;=;adaZ+`-ifm%4iwRm|`ddqzXL_{7oUYE$BTBWGWu z&i!K|r4EXf=jkFb8u$V^N+|FJHZ@yPrB!9%YyN9%!I2J;6<>F>R#DT9D_ZxJDTnvc zPwqJ6-Zx6#HD~p*U8s5D+On}C?f#7GWIqoOwUsgtQD!CH_rI5j#s9UHV)So;udaVJ zswa*WM*qgKa@nX}?MkMVHDVzke-V@o>Se$ATxJudvPR5qQZ%6~Afx_>vS)@DjTUqj z+fwr}MX}Z?YJ|(JCMFm8G`YL8%q$qrz(a_(U=?qsQeNazYz-K5SJev_QhJ@aP8lhZ z_#$wy(UOn1HQHWgy-X{^3SUp>sr;k?z#}8+7izTk70K~ED$Y^2crPpaDm>3AS5hVk zO)5fA#v-9M{Ebw)k`(7Vs+X@LTrxQ&s+v<7^dfch4v7J|50^2D{9v5$e_eT9COEdD z3a6btX;T1dBGN8W%6ds^Y;@>a5s<|@I##aP(ng9jpq{UD6sl*HjYi5WtWb6~*^AcUkv({$&vY* zJVkYd)Uz1~DuGgqfj7K(IR$}$G;?Zq_7kEvcGu{+&VflgV5~_Js11H%f=~X~0vLSNS(gr$$33sOQ`H zn(VEd)wox*2&Y;*JMB28?&{gmX0_7PSeaA)I4r;1QGxn|tSHk*gO3cprzWTFCf4r6 z16X6lJ$$suv(vAY9SXsm+9`#W`4yGT>96_zy12k)_Qa~69$C}*XGT+BV3hVyWosSC zSaG24rym;96--}stq>0ZJnhqi1wj2+z4-3UnWw|lgjI~yn02(-F0#KFy7-`4aleAw zA8jhu%^nWI)Opz#jIz_D;>H%gphBcGxmWdZg`r$m?)Ch5mlxgIP4Q#e)Fawe)hxOD z1fz((pENZ{|IH2+N*QdPED?1_1Is@$NEiL81`|wytGBhLskWcvM)rl; zXme_zb+M9337H5N@#O_f)@C)`&wpY~uX)70QNrP;cik#YUi{V1h^Y zMr_8**hs<-{d*thQ^u678<#P%#MUhiun)=m4>a=TFQKZ|pVaV7;8`Uf)(7Qq)H03lTcyk!33tr{$PSs&krhPtld7(6*rTk z?^Ii6Wk*|qnaFIV+igi6vYCuusyj28{VhXA+I6<0ErC~@&tzWlbJ9o-)^|)T%`a6) zJ>M8C%DVy3x!&XpN=^*rHf4^?uuga{uA5{2H4WK9z5wWl-#aR|?tWpW@kR}+1D`4-u z>{})O&sQ+oS(8M7N(g^b)5R6lYAfBu%Z+C+(=Xsgk5h*j)9vo2G}pC?a}i|>=+K!p z+DArwtB=8gwTq&uY3!yh+I+)}eX*v_3~HF!>8Cv_$77L)cdGnvrkgY48ESyoxz^W3 zwY$=v&Q{p=6qM;pGDatsdpf_3)1G|&@zb0(kfjZ-SRFo{9 zX)-gh=wvL>r!=p_MX-YL?`Y0rxb4oe>Ej}-q+c2|ox{{w-U}a>+cSWNEdta#VX#UK z4WFfM5#|qcx}Z)=d3U)F$`1g5!w#jn*5r6Clr3H~`}5;un)FHF=jUsRxkHPtiyP37 zU6tX8_}&9LMt&Kz?jNfLTPiD9hhvIEeX)jh(H8_3F2PBiKh#^6Z{DLa*~Zws#o2u7 z&QXN&0BAY^U~+ljxEJj#BPq^YTU=!^^XDc;A)u@rs$!O1z-cc`jUPOTR0c3EF0$$x zwJse_BbR&OG)h)54zAIY`ZS%nw0nx>UfbX=Q5?&B-la329-~_&x^U-l@i~*3+qLLX z`OZBW?;jw{LI#QAsc*)6pVTX7d{5 za@#W>H3P$D48h@@enwUBqbS~1M&0V0$cdC9i!_oOK`pQHShb))9 z$t^o!iY+JA+})uP%6&YD^B-K(*>H`zKv5|URbSap*kwr#F0J81sZD~p#U#8u4(j~t z3WtoNJIRom*{rovAhSaaS!w^N zN;{HKwYrPn6V&9=D=qhfm@0HW-=cJ-m&o@_2@XHg72`$zsV%}z+*#CBM#_ksMQ_Tg zvioKcW20Qd?@(`#*UL!RYjVS|Rd1B%DE>9tWwG2$Om<-0T$@H+7Rw887wV7SS|uBW zP;TRG6fm3nUiiO6%JTouyFgD>2GgYdAN}zE)#(EZvzV>uU15o6zA3#%FqC;!}` zIhaS_g&I@i5=t|a#cMG;bP3~S{ycyrIf`Z}0CKYB{t>ej?j=u2b7*r$$-)$?byQ~_ zZ|3v3pH5?1SL(<_g&kF;KY=ljyMi?U!`gB943)zy9nG6$lxaU0m$+gmgd|fe|8U4+ zBbXvNj5+SUZ(F0*A}Xc*=UEDG3{u$4cvIL%^x)35JNOfXHJO-BUJG zNNCqR*-YUvxd)p@X~e@>=5aqd6VJGaVoIx2jbc6|e9lCJ5~qj4_;l)mL0v}TC30^7 zYS|7G=!J8s2gc-7W(-m%mEvBt(uyAf`Poa8eTVkf{bCv0T&ezoY@q}n z@eJDel$saHPUnqcceR~R3Rfe2QL6aLm4=@ksw)=lk0KKhlLNKQuEmU6JYsFy)I(;j ziRX>gWEK#(TyQNV4A}ds?Kto#D*8uNhT!t&Y^mq>QQJYsgJr-8nrM%)8t06lqt-ux3z1Ly#ALwo(sXxyhswz|{|B}qo z45`njrBhj8wG~%56>QJlTp*~kDbzmBr0qbwd@tD!9Rq+nQC^@Iy(M3a+U{*x9UbmT zNvN23gE^*7K1rI5W;&ozYdL2ArCOfZ%$6Ri5aygr6@mhX{i3Cf_WeCTcKC}RdjVzC zWV6kRyUrZlYeClg(eXyHtyLCM67wu8N3%W3bJDZE(3YG1S)gxpj6B7s@iOSTmYQXN zS64@ZApTlaW3uDA=|0xwWt&t=^bM(l3^ico6JGiUUiD;G`Qz~~lY`ENp0Ba;Xr3-@?Ta{M6qa%hWy@MLo zM1q&)oK)XeoB`D{!X^JZnHt{9+z-86HpOH^{0M}Cx>|>)QeD`YR9gn1O`*Ck_Ts1y z8~%|_X7C_g^7>fT-2GS=Hwwec$I3dQ;UwRddBF&6?AMI3P>59riQJKx>f~{^W{622 zhq(=>GfMrC<$%mu)ZGrE)Y{w}Xe`y+KsoXYdZ-IF$a5#7M5nTi4t0`Ctg;1hJ-)#j z>14_7KnG2Av=4oy)5pr)EuwPk23y8(mC8;I`ybhvAem+2s%*8XQLi!QVaQFL)#G&OYd=U4V|rdqn(4?tArmuV}m1NPDbvSDAg3+0JDPjFX6zQq8nzP#LV;sgA#;%5uAVTl~BIL}6us zfNa55eKq$zZM5G&NUd{MQ(5tgGDCPTRdK#y(C%PlaL4t_llD(^nRYsR9KTswf@lQ; zg{GG5;RBVbd#uT2K&GBPV?%_0tn1VrfcWW=HHky77k*S&I-gLXyfyb{W4v^4DFD;` zb`XDJ==L5h+L_G#h9Ul-K`qJduU_$!&DNxq{!!zP$O4`}_WE&af^N?;$rIGfF4e@s zL1wLn=8b7S7Rbc|`&!w~8;#>0Ui0I3HEQKkb!7&eq;fHB<+iguT43e=wJMhStRAeQ zZO{H;=zg?crQ+f{>gV4OSlH{W7SF2T#c;?R}{Q8+=Z;yqAqTPzz((M*1!XtMVXy zBuF}6smjLdiKIuaPb|7bYqC$Pmxso1u!6^ZO01TC~e_L+MJ&*c~ zLDy2ly;BCE*}tRe*7@Y+<<5h;ILc&}7%mU{+2?B&_p<}y*R@g3K`kOD8s!etoNb^= zY-@DC96|emzT4|1vb&6Ry;UB)R#e^VEq+}WdFUY6sbTB(v<5ywac}m-gXC+Wtr>n< z)5+1ZN!sz@6#9!Q60sT*s=Do^ouFy;_|X3~>IF)Nz9d6a3%J}}mH$eNw$aL79@e?) zZ|-=Lv@{V~ign-^Ke(vZpkhRFp@S5kKS1>|E+;3ZX5UwlQT_d?Ar4+@k|8d^GrD9O zA@>(GE^(1{T$dlv@j&*pa{sU;|GXtTXsYyc${xlG3ED6%52===-e04P@4YLmQACy> zMOfE|)vE4Z6~5gBsxQz$bvAY5i7IU}cp8-|7Z+%}-6fmXUO=5RDgFM-B}1AumA@_H z{OOslAHCSE^Bv3N%dx%UW=EmqoOVRsB2nWtH}?5C)cVu=!#;r1VnQOhzG_mVC7p_6EjmjPJIj50rt^GBNs0tgDy&&K&7SXhu| zi@Y+JM%;=TaFks+R?51u4c8KXdTss6!|#waW&2!zrrek8x~Jz;Z0wwAM5^|=vQs7@)e{1*!4N?{09vSvwzXB zfHwBk28&#xoxZ^=E1PZv)bze!|4(6G9xq2(rQPpaOV`rf)qQjC?d2w^n~>z@2Bf?8 zsw`aRQ$k1(6eVJCh=@Q4%!fb%1V&{S9YB@=SCnjk$`%O242z;TZn$IvM0(0o(TOAHiz`JlI_&wXG)ZHG<)bT1{*PNb_UDsj~9?r`vlAV#!ED0-Nsmo zJ~Sv)!6xDTN@v*w=oad&f;ECD8NN_Q@n;9-`)GrGT=H(mZ-KPEl5Nkn(D~>P8X=YO ztOw|L2A2eJ=@{;fTN*vbR);n-On#N-q?FrnS(7$N`asm=nFqO})XcZQR%U0(Xrvb? z$&eT6OKPkDAJMan<-n}ENv?i;r70KKF8n$dV%*|FUH7A@deuEk3a;dq_n@{Ku!?0- zViB((qQpP?y(wC>PaoC`#fq;Pl5$5xFh>~%H;Y*&3M}FlNAnz6otFpOSa~6*ZIIo( zPb?WY;9|IrQjfAjNUDe(sd9N3%O&ro45&Qmz@6YwsyYFeo1&9FufvOgHn7_?3i}9W zHHbe(yf9f<9&v6Z>lWXtl{)!7l}1>>fF1ZS&=@?#Rqmbl(#}Lz2X_Z_6k5d|1!kbg zpWX@`qh|2k^2)H6h?Tc!n9!_n*h347cIrk<8Rto~G6;z0(OGl?1&8Sx-(x_6vV^WS zJpu+M5-4#(acdE!E-&a%w3>9%L0^$pvp1tkiSk7{-6}$gt&9>d3?Cm`h8_bb24Mi+ z$p5t_#pN#6K{6E9m1Z|c36jXWU5?<#mgPq)&N|u5ZF?f*iDk&VF(Anx9QH@bp^Li- z){Qc@QZ#jl2>izA|ANIRfw3QztK(_9sw5u~F2!?t62so8KBp7UuKLAq^^$c;eIg74 zg8lMZgPmR;CKNsh=00i_tS~Px;ReRt6v&Upd^Cjy*Y>&gSLG4$Lp5k8qLWGoo564B z>ql2q4MOKEsP+yF5oxruL?@NeB6HFDlVecIZ-wjVGk&de0|`361h**t{SF&6V^ICe z4er1y5gg=bHCxyx(Yex=tBVs@1;AN@F;!YjE4Fg8@l@8?z@`eF|-gYj$@0Cq<4X3;{SWgJ$bab{C zvxJ4VUf@eqKH_?}X#|IdTyza7vo_Gp-wl7L-ffz^$3zwVz6wQlbdfgkUg2IbEI(Ca zjn`Qw)LALWERn1g7oB3upK%oypqXo7&uf#i0JK*W=T%%Ka}hGQQ1vkPzifF$nIR_X zVQ+Khbc&00?>bAg;sml^pjU^AUOg}9eMG`+?%|R%Kmof23KsJ`8&X&g)h5D-#gg7u zXO47+mTi-g;%B5%O7mW(r0N_AS<;1~9i0O)cEjG>@}8B#-EW$>7u7OrHTc1*E{ zbzwiL)AIU+jxn%S2%`P=5=-qPqC~V?EM!Jvbp8mfZZL*sswLlvj15pSR=(ZxN{Kr^!s8Ot9Y`(7g;<#AYWsH>F7YMo6;)N`zpJ` z=z~44%lXWNd&Be7U$}-m&B7zx)P+mCI2-eAZU>1EpaEuj=D` z{AeH7`!lm&Y1G`KXG>=WmsG!HC>1d8?M{u7n45(6=$L3{2MEumY)t-I%tkR)oU9U7r%7-P_u+J42r3Bjp-NVx?#s27A)upd5LpTiWQ1*7glQ9g~fjOS;X^4;KZmMeRw zi7otOwp=>LRfP24GVE>9pv`oEfSC!cy0hS8lze@vEN0jVZPd? zb(6%DS=ja%{e}_)Q1UPxK^v-=$%BS&k5{>H0naAXdCY&@kDyqXr{@mzEuxoSHvcY$ zTPwD%et6^ZTRL~0xP9ZH_peZH&?~mC`U&RA?jkI;{3D&dk_XEyUdMur%Ws)~*D9VT zi_spttWSiTzxBi)-`E#xWDLfAv9+sT{hJoie^1aOf3wVD{jaRZO2J=vjr84+P>mK* zU)vs3#?i3X2U-xf&nLC-`$lC1#xepuU62h+V|e>vMJnD;x7#P2RAuJl{z&5IhA=71 zcr*=)fT-?3?2RK5d%!Ia##j+3M%;Knm|(O}(_&+352ntPr_y~Ijlf>s45TN=Sv({- zo50r%Vj_wt&C)3eKdDq$A;c#?d$?3pdi~)k!kG|rl5_&^%UCOdpxy)$ zE&5YA$d0rJ2p~J1l^g#VBf#kV{F7isNP-;9WLczk@!d*ldAaO9}8Dn&na1Kf*Ia*-eBRLg+9b zuS97lgq(m(YpZzBpakh`I^V`Je)a*3JOu-=O%&EslrX~HyOlH~?{WoUdHD5g@10yX|Y5^#392*GUOO=4&12V67EALHT&m(P8d|VGRs6 zEy3jWmn=V}vqK_pk~^5v#w4F)k-WSnQP`oRd#x($Kru@msrtCfjcyby)d<&9nCax} ztwziex%T$5AJMki@79V=0()o_Re}W;A7QgQr(#PUyJ*FA+EZdsCN3DLxov_8j(~n} zasxPE^uX0orFceS6kunG-p5Derz+}cD`<1N5&9%{U9w>0#RJk>C+W$7E)NW3+WD(G zdRp{j)$AWr1Zh<;E#G!2J^fKx_ik4a|V%?(5bO&w!+sYoeu@IJh zs77mm@N=+E8s6h!+UGsIl(%R&YG=7wnI%PaITsPOsOhiX;#2U8o#b_)XIcAXAjFO}!>*p>US zwjX|+mJDnK@$%dXuK8&s#}sGxqhx=HLwBEW?h;C})9_GD?ru?%Wi6y{8N3CW2+zn$ zC7UK14tE##txm*22u{(J6(GB3m{YV7H#2hhH`z228C zsq8G5{uQ3qHetdKH%+SOvdA8TWH~ zGy9S?5oIYs&PFusU_+g43itP>ykZFb=-(44$UZyEfUgI#mv%y{`3%Zk4#Fp#hg#21 zEv{}p)fTLt*vgKn`8Z-3Xq6q3h_OZV~+FknTlR#r^cKSZ+GGu$LQt@q2<5Cwokk&4pYxj{E)XA!P?7 zPv&9acK#l?vvR{Fj6mp|#^Htnpc7kiB^zTsRHpY_s2g4fvxC0Z9j5p+*N+R|l;?4G z5^c7h*zybw-SP```tu8DLrpKBzhBz%PHu{3db>6eL*!rAuHO$h&DHDP;u)eDzl1+O zu55^7*&E2?e*HSy`dBEhfLyNB+Z0$zc5SpBvI1CPy2bZD1*mc39GMx9u1IIAq8VRFy@^ zG@fi|2f5|F&oJzZb!-Uke@^v|nkuYo>A9c_HK4)|Yn*1!>kZ5DBhCKH3oI_Gy{h}^ zQ}ncaxiJx?IdsJzw>gEGJ`;pV^DQQUvnVajoigc525;x>kXd9A`&;I(`NhMm$nnC% zESI>7*@|m^$>?g$b5Kp>KH2i?R7>oZ&e~NEGllV&+o5xR7k9=w&{tgig-2O}5rQ6c zvX9W8{LMaD{&)IhZ^C9>h#tPVPbQ1M2z!K4uGBk5gX}NOu+^pUFhOsskI~}3KUp^; zlmTuv09?`gvpX4Fq<58My;rbEAcj&$laRO4u{2u+?L&&?m}|pxgseL*AnBNLPi{^~ z(XtCIT7ylSH$&}W^`BdDkP7TO8q>mqRaI~nA+UIpn1~a$J)>NclARDkmUM(;5(c_3 z3kS0BS~)%6B<5xbM$7g{HkhgM06-&H4N-z7g)ll7ghj+AR`=qGnO&5{F2E=kWkeIX zIEuDyxrcL-CwWE;gBN2_0p$`*YIJm_69z6lrVH3iCfo+0!?kkuI7F5}T4!5m2yu+?dl?O^|Giq;SgfQNAg4`T={0{tk_bolaB1aB5?ox6W| zNAJSWNN+_;G)kaLoe&ymkOKsXCGNrf1&eocJUYtrT{4ojCUb@S;UQq$`(hc`M_`^T}g42 zp?mjNM*QMWV2nLTkmlDqPoPky2G3#-I%`K6pc)$9;gu>XEk2=V*gJQN9-QFg4a_@) zSyw{qkCI|*y#T3_i4cEORc{NhyGv<}i#`0ubA z%!y3M@QZg<2E4_ZaG!HMS_;s^mUYT8K4`KMSJKkOd^8hUNt6Ege#7u~)@zhl9;hj8 z6Z__fT$dx@~Z~e}O2EgtXvp)3~bb7w74w^=?DS?3^LJ z$86fWH;aj+*PCPN-YbJO_Wy~VbJa2i&$Gid73U5Of>kxyjJ{}f6F6de7g7RnN8e$n z8P74*%kubPC7rUP)QBBtQsoDZ98mTc(cv!gq-+FRSupIOvd`0QAh;qQ3h!iAVGHAc zS#l?#r~hjy1$IJl(E|qkd{>#>EdA@5x=wa%*?Zp*R)+`MgBv+T-%58c`%u}>hh1UO za?A1`5}tx5Gek(kPp%jC3SD`AapJXLNHgG30}TlQc96^RmGP&Dk|3 z+h@q2)=wtgZDmE2rsSpzM?OPTCv`A8#m9Ad1XsJQI{oe+r&MW?U|)$iB1E5UFPE(6 zTsrd60n^)Fu5@C`J|1g;Blo=;+?&9{dfw$?@Cw88bLr(1|>BkwgSFz@Yy{pvaDY{m$%wbLLUFH5J zY|YT=vbvPL9wPf1q9f_3E>@y>YaLnfb6mn9nv$J+hFc$W|1Z($sta`Yj~VQ%Q;nyv z5_Vud+zg(U6G@P+E*sWoO%}g7sqF2bA7T6SZW3U8^hX#@sms&w8#E*MO$6Vp7`EBu zry<*_dGme2mDWe=hCI#FHmkWrS{^l9l&khc60$?>`I?eJ67E#Jk?(|cCpg~WLIP*u zwI(Z5gc0GkyAZ;g=~;Lu6x-!*r%Woi*mY#p)}d3nN%%m8ha=Hw*>9RRG(fvJe^`te zicq;DKyTpZ@C>uo5Tv6E6h8y!-*VOqZ;nYD^h337n9?pXn&rbYRE|&2S?EgnYzHvy z-hJqk-cvDAVnSd)sVUfU)!ryDjBi-Gr4nSK3%s(*oyO>VfC9J1winQ4+{HF#(HPyj z*`VrK;|Zcp+oh<3Nrm3CZCyUfY}k1LMQhKsB|UcDj_IcTY#UD_gd_wkr7VZOVI0zy z(aPzUZ_sT#p{7h>c0%O}inlCS=iDgd#TB+vrpGA8Iv;Tv|HgTpg+|LZPL(`Xhixj< z?68AUhD4{v7hmsnllV&(1;tnO3M8f25TFp=Dz(mm{YyQb4^K`WOQUn9Jf z>tQZ^!;_V4@*?<{7c+q*-ewRvD{QKb{Asi3#>IuYE^7`KwJ+4c5MZinuQ9HR`}IHD zEZweo|KkZuv;E+>rXg<`P%aWvy}Us;`LmWEyv3U}p2sc=SGSUHm;LzLglcojFrOG7 z2zq0VE3KOYS|`68H0;OPY#xD8jW?{4X=9Kq35#F8YL;AFqx2hr^FU!dQH`Um z+6icO*Z)FcEdCCK@vop+5dIZ33*IzImgL3%7n&i#8%>lh$Qb^i=BP)rUcXrbGmA#X ziR4_^1F>v5{-?@)D4w@kU1}4+nFzV7HWo&GIiW<&lo9Ge7YAb-VvnKJAVRTYymRVEg+WfUpr+Z!?8C_6pnb;gxvv|G3lR+Y@?8jaGE zxPA_r2uxf>TcZyEQHk=RKsf+pfTOeybw8&MbKs>(i9|EHLz_r@pu>;GxniBshF004 zDQ^F80>$uNKqwO?q*b&7(*6AX1)VXbGxn!JSZjQNkl!?+$x*EC1JS(ApbFnHy$agcrDk|LE|x83?l;sFX`rOF zDV|0YBH=OPeafu4^F<@w$zn-gi!!yiEPT0y)KitB=nGr`{M24F5Kb7XjzrAwJH0K84G2-MJu1Dkp)yKCIJzCYr@_l=h{pRmE6C2<@MQ74u z@O6U;m(YO-D1&~#r>H9G!1+v$TcVZbWeKI^p2mZQ7vXf=o<2fX=q|zc-=jSkiont8Z)m0_AZyh7@$!>`w1`J48A^?N21c>d+ zhr2l?o;rLcJXe<74>At2zha4L>=?`9CV+L)E0JjIKw>UbLLk@5i^8cVJ;;a*iWZb8*(;#%xr`~o_y4j2MoD{#R12Q_)IAetFqe@Ma41Gr02B?2(U$W=_XKwOA`nluBF9F5A%B{ z^?IS*MqUrYMEF>J`Nk||0(F#37P5NP0rfDEn{Ft)n$0Y|u zIN@2x(UTBvqiFulA?V0-n38pwS+dr-6KNVvuj=0AX3d#xVrIXf&@t?%E6avFP-+J% z%$B@nBBVEq%?>JfZTem=?(1`U$ywm|MS){I=ciHIsuY^fEnh za@r6w^O5{_rnpbBdeRf$DqYZi}`VS5Gl)*mrrLQ`i(*^nT8qQbh9&&X3=@NZr@`l zNJkruxZu%Af;O-R}^%{TnQfH>O+c zo#hHU@A%0t2I0e}B&2GnGzx18y0#ekJD(MlQ4ZZvlJ6P@qxTAe{J{_Ex;;F|pqUa~ z%K5GHbU*%sXLwJ~s&(R;>jy7ZOBk5?tOX}W{$k1Z%CdslUW;~!_(00A&mN?Fx8z>x zd!O<|EBkJ_8z;9{bb00!23b)PG2q3aQ7=#n(Dl zN1KQs>GNpcWAH$?4RLvm^~dtX8bup)T`~4V2kdn&Zvg4zQvQJlc+(|>L`JHH_YYI5 zZhclG+*{{fEd5YLhyNbVWo!49ySPmXG5w9g6lXb5eWo3=a?(ui-A2jYHPer$4I6*9 z-AV2>yxXP?!T6Q0s%4NUerK)@cB{PRIGiIRn%9EwhJ(9kSu(!_kqqz_xe%-5d zvW||eYG=uQWscfN&YP-+r{K$UgyEe{*c?EI>t)`$SvB_=cSg`-e_d;t=B*_V>~z!! zqNqIU3Kj{leprUPpdpUX`-;zQGy{fYv@z_{rJWL+cwFu$iq5fRZ&h1ZuMYdgCx+?D zwu6@bh%L)gI0uB6)Z^0hjxZ{nh<4n&5ZUalm2Ls$DK#7iMe#kd5$xsNc& z3zO+4{0v{ImaNZPs`0sAc`riJ`#PI;BkNt$CsIi6>2||Kf3uAV*^8^vro{t7kN4wx z)%~i21i{+S=w`j&3>)%Lmw;cna*9n4bZIF1qD43?>jmcyDI5k%X9(I8f=W(g8sq7G z(T)?ieU?Cw5p6I9zyQat>m(=gC>a4T$TMzz9%A?N?>cEa!#JgRPv^W1r)D8X0%x$E z?!?dkO&k^^|2qzg)4v)eOaB^&1%JU|S&vW>a9E~>$;z;@7)^RC>M=KiUl)o_qcN~P zhztQ`p`*ESc?JPtA`wS#2gxw}3yxbLSR!IH>-X58GOkoE62(jEz~iw$Z3bHZBEl^y zWt&$}9xoOlZ=V|@?^lWyH;9pL#&%vwUk%a^;eMOLII-ZFCv@eZe^64c(whZOY{B+= ze<&aT{3sOANS8<@(95n-T!Y+6U^B$v6H83Q7jbUyK80CSd;$2f`ON5vRf}X1$aTRY z(MXi;e7Hz-Q-X&^BmR{d(Fey9c}tU$HQFwUU+Fs1{z?Y|%Ge7j(V9rPm<**j?#j{C zfmcwPlMwc#g8`JFl+j+h2W@dw!Be0|6p+wy+{99L^VodrYz;CW1py_#REUSbm2)*E zmTWkj6AZrA8}+EM1WRvdPIMIq;4Z6XU&f>sd6f+Sr^ zksv^atXOWqBL)bp(n&`8x(*ScujQ0%kLcA7VVXv?U9hhvB~N(yAPqCSNc2{f#v;Os z-J-W!a;Jv&a;p>4+0!dc3OH?)y=72kP0s*`yAJL$I2V_}VQ_bMySTf%+`--5-FvoTR$?BVL)#^XQc%r%bL>6z| zo3kyoRPzD}x4;{G5yRwwfm;}FIkk`+^soBpr7I3BT@s76EZS@9e9nr97l3B<-aIIq zUz%eHU)Fcx=S{M$p9^ClX+^RYODO_}IB-(BV@_@yqHMo&$ttX|Frzd~Z!eTr1jl3v z4wKZMHbjxVwb`WQIjc7Zlj49YE)mDjX^BNSY|Te%Wv}mWA!A*Ce*=_oU>&Mc7D=Rj z!hg2cdlDQqQK8ffSV)PjsjS4LV`J%$G5Oj7mDU}`S2Q_r&#zQ3vMx8DJel82(*z<3 ztgB_a04GB@VAMs1px$b3*R7}6bNp!$J}|MF!)9ij5TKAdfVCk0d}K=Mvmk{U`ny4p zeoHAcwjwUm0GkggBk^ZHR!H5|q|XHN@^_Vr&E|p>cNq=N=Iit`mpxf@R3xCt(l_RX z9%$mr0&@4JOu7Fz8&pJ3hR0}*Wd<&t z^CuR#EupF??1Tfz$!1EqVLMX%HQ`}|pl4HZ8$Q%mi1|ux!3&nNu+V|8NTX|OCf&w& z-y+$b=Pg9J_B9G)D2+YaY5`2osYA5X$vByGV)!@BvVJF0K;e?|DZ>){54Sr}<6+7E z$PMr{|1yL=Ss<%4>o07ajwPB~^I;F%xX;?(A5GX_4IuXR_UN&6lDZvK^SHOPD)DE7 z&d)5{`1`k8im`B2eiw&wc`M|eLHp8M_lR-82C`5GC%Eh~6QttKQq54XBa5&u+MpHr2q8N=wnR>9&tx}e zKfS*E7ud5BF*TP)UxcHU5$#yu%XdlsTPN!4JU>?_{B3HBG>bABaR z$2Bgx)*`_1A>V7eRVgei%(%%FZVp!DeQYt`5^7?TW``(i<+-b!8i7?CrLnpLo?A=6 z1%__x4mhjjowYXFG?Kz_LEW(zEmaZ)w{&VQesB7%Rra%g^*tK7`M_Q7s#yGjL%AN8 za<<+LL0f*nB&*#9q(c$c69aG$ZL-CG@>jE#`_OTxnrJ9?+#xNhROhA_8A%YK84jNz z>WG&$tT5Rxj`Um2@@WPHr&C4GhL2>!kmyLOh;Fd5#?QGe9I-JB&#_lCc8)@eQdx?y-NOg`U7b8u)^`%s!hbHk?ysFi0ABPl3hX<% z$n9LGSnOI}(XYNVl`6~y+7IzYb4S|)`*@Pt8uz_OTcD9yZh%X}?jNUWs?LY%YL^zC9;+@tml>4yi|>yDPtFbg>QV+q=B4FK{pf3ro}r z4P2(~9Xw%phPQIJ=;vdC{@}M3vCbx$frOdJ z!rfMbvIRrK-vPWLzH>EI`HuN$i0On8366{8B$9Kl`B`0fQ(M`Vy&FgcI9@q8-q@r~C(ks!GPvnNbE{Q%T{yH9LHF}pE0lj- zz-sh#!OH@dFd}&JVT$YtQldbc%2`R9No1m2Oic-ZnVFEMefpD~AOGum{p6DcXe{6{%b5?YI&) zatgF$F2N?);7Xwk^rjMQRNvIhB9f@aU&09yF)0hfwGxbtB(Q7qsmY564AYVia;PWM z)Y3m=(O2sfaKd{ULCe!Bx}EN_(&j}Q49pf9i)9*yvol{A(eumap!-+A775fBG$V6U z5z46vMABj5*-9A2MLTVus|xB;(U$W3@iL;w$9>|#7A|0#!P2vG7NGb_Qs30>52<&G zby~iySw!{h1g(h`VjYXiZlqq-qhryEB|t+TxK6IU&jrG@?e=xV_#>!|0oW*NPMd?h zuk&j9+M1QRIBJ$~lUiUHkWiKqKj$crX!|X|rBdckS(HzjNj<=|`dCP;*H}auPmkWm zystP~8X+3n_G_A7oKW~)jmj)~QjWg(BSFBu0J!U(Y~ZfTG{=NsI6H-U18SZUV_hLc zFvXj=?(^~B4;r4bEf7C=?IgUnp{RR40E_BRh7r>Ok2AX5!OM=QUT#*TaH zcR7{sR@;=4+@X~)wsXPDZ2e)VL%ymjwdk8IzGQmbHBV6uL6E`K2obTt^S^mA(xL{z z%IpRFJhaG^awN7ewqA;Oi1dFgxn=G{puhu7DAtFPDnGHc`|t)ulG7LsNBR{%ZbDP8 zAz;SQ3_N@TOnyD^Hfr51%ZLo8=Mi=s)yj4wCg?H>!arG?YikX-TKFRSW_}*~3$2X* zCZ#_)i6mrmL@h|NC5=NfJ>{1m?1*6(%N-pmpvD2ak#9{RDoA(#UGZxaHsN#mqKuZuFKjmHATCxNNcRn522DAOm%j`hae;c z{io?3d603)H-F$QDnQap+^~Y`a_=eO2J~67qA5sm6gBovKLuIts2XBf>!ZoJA|AOf zRmb`bxUFH9aBop$GNVn<`?EOn-imyKV=X5_%&ow5YQ{x@qcG_`Ly|@e;Q(xo`ey>+ zN-C1E!b9h+%gyOu0>tK$2Ybe-F7X|OZje^%7I+Q!BwxLCMLsgmU^->I##T^xkUQ^b z6nidl#0*U`1+0(|T#Ea*aDjESDYUo5Xtz6zKC!Af5-CLT<;$9Xy7Gt8?lTwysolX{mKJAv|*nswYG+k82aJ1ZGz{i6}&?#>6ZkPNQ`i&oSW$~_sm$TT(n^X!M4^T3#MLqtXTf+7F<23DW1pZ z{lAdh5T7Wpq8*e1b96kineE~3ex&?{DD63m?2+*b+g~OSz*``Bi1vq5;Q6DD`^_g{ zmt5D^(<*??lCac-dq^${-NiKeZYCMpNfBxlQA;bJD9na0oy&PeQPbpXOaqngxriR{Ey;^qYSEl+XuyxP7GQ35h3ze=XVax5`9`5s+7xvHKHDMD`C!%{gafmvuE{dAC7DZHW6Odgi6)70RnzLgKTmc~w3FBIB1c z2XORa4Kr}biG}r|M&#Lo=k8xnI!WyVUe>Zax9r5~@CQ=Ff`f545b`X}(#&jI9MTuk1r;&} zU8u(?dDdt^njcm4LXF3tLKx%OZxFiU9c-OvvBkD7nmpIFHKyZrwKUJxAvzaW;#jmK zL)u%r<53YVYE*m(T0dP&{8EP8jpB2wa>%_nY3EYkPgKjILM5ZXKRb@JiJ%(xE8d&F z;#edMzC{erBBr_PS-jx#gzpvwKEe9jy2NA}9I~&wRUh|UGNf+L$`^N=LgEE)X<0-a z2*B*dx3C49>T;ohJ6gZd)*$P5`QP;1t>HEA3o!ST$z`}%w1Zqs(%P+B0#g~_`Sy8` z&9fh`r2_NvK5iRsqg&kNbiMv8WJYH4ijQ61BAa5?NNKpM%~gAOh^8e8d$HbdJ2#5&D1sz# zTa2p{RS02plLX31A}M#1IQ(HOeed`$@i2~7jrUb*U+EO)aiCP@XZhWdDYmS~9l$h2 z@|S^X%84H{fD0K&ZHS+-8q_CYpUwqnF=ozkAsUAROYrt)^a~T(YB{rp^s2T5Z8#C|Ge{3Rwf8~OwPp?_@c&4u&qVy1iq&MEc=UUqGMGf1Y{IQW!* zrzND_b~|I`wdRIge8S)?COo5AyM`bGXu@aGNOYgIo$x{!VeAnuC7PqhCA`Tbktsxt z6j-Qs>lzZ`|5YBuR&p6#`7O*2fsO08EUkCk%Cs)%Ye9t@bA|!Db?OZ^89v5aFf=+H zCm%_592E{?P-`(~%C_yVDv;iOaHrOL)6#pNtMFXz>pO(995f6z1Ox;E#7u~;48MiK zPh3a{h;$eT2zm%&2p6D}Ba5|zxwRS4mBs9f#1{`QZ+BHOpp%`h;ulvJ4ohoG7FHG` zBTEf+Lg#$YjU?R-oR-atZW-Fks$Dlr!7zLJda z;LPVdV;S)5tO?N6oTQs=JTo~z9Zfz^g4(X1O6oPp;?S6pnI4KSHWwFiDmTd}D;wyb!QP|kSHb^9_`jNec4a@< zVEQrs8^T-s{}TTE|0KN4Wx7#{-etCKQbv+?()j-y?Exst%0pV~a?86S^FS89&%rfn`hW7}d6 zs%mnRi0qAwbr!Z}tKXMP!!AF1Jf^R*q_(2(*-2xa>dfjDlY{TYM@N?h$IJ`B`S*h3 zH&i}A4nZU{NztC;T&7H?e4^G{7%=D>A~tBd@0*TaBj-2~P&*oRoJSHq!Wn^JH)9mL zj`E%B;TOVe;uF5-pP*T***nzTn;})Uz|zl~WwHtGl`b{?Hs(E5XoA}Dm!Y?&9M7yl z_V~jXO^#Jq-y7kF)*@f=#U6A>rCK8{I)bx1 zV);9wbG~nkK?{gqh4Kxk$$x$nH`t>#G{LREfpffJeoU$o6}~R#ywEvhM!eB(^LvN= zpW1+c87AX8&nrz=UbVK@p6xLKzC>1Q@QJrD8npSLQERm_>*IR9=oNS0T1t-_6^p#|M(p z3vQ05v(b(3&-MNwJmc!hQj97#@vlGqd>2k$e@AIwd2tRR@R)`qRhcR><30?h%Ij^L zZZBTHxZ56jkvs9=u1`%W#$K4a3ba8=x5;ntsxlZtgu$A^z?=ikknT-Xvxs=)#OEqk(7cDAe}rn=~_ka7Z<$0H+s7U3OS4!znN zB+K#q9vL3rGj=VCnrZ{bcxow}T3Y^nbn;K8 zqR8KZKtW+aK`{vswR5uL(rq%t+Awd>X%rN$qv#?DpFCe7*b#nMvc4Q^Y~N`dC-BnQ4deBb@3ARG94eDFp1`4q43<-WO{?EQGrDB$*?`Afcl z|7u>G@cYS&@ZZf@q4$d~MghMc`~KcX&-QWbyjI6M)V`jDZl(wR-MeWQ?rGT0dlNG3 z*$&*od-o04e2p&ei+>j|w7CPvKQ3zho<0b?%l0N@s@=@GLhjjZ5e&HJ`LRpZ7UlJ!rLu<7!Ta`mrP6Ya*0bi=aa zlD>?yIpM1I=uW+knF)S0Mk@*xwfZoB%zCy+i5*boRw4L6P1&%*e@2%^zQ!43(_(+48E}pL9}`D&ij1 z^AA$^WJ%mmj4}o3k|oG5vy0lTa1}Het`h4@xnmML97~Df%HlS`qo|d26XNLjKe`F< zwRK|LsT9p&V~X>%BZhf@lMxSpDwtE+2gHz$a1g?af8t;FE98os>S9eh3us>?jQ>si zAW8RP0bA59>m;ic4)xW}mAwqJfWuHiy#bL_U4wFO2?h!rW1C0~pd)y(7Lulb^SRVO zV&`oh`dK{7B{Scyr)*J5Sy*_pN@XpNTIf-<(t)*bW`1q`t)Pm2oDru2gWD02$UG@z zhT*rSUL;gAu@C4A+KA7PbAngyXH1#zuHceln`){P@}{C(q;||KPu7`2I{gCj^k3#J zofaZO1-7^dH}D!RnFW|%u2JY4$Bv-kp&{GO(|eVkfAfxmGmCmx&D=yphkcj806vZ zeyFcT^qMt3_?jxVT=RF?Wy(nH#QkNjM;A721Fn5Sgay(r>&FQVM3Jebj_pJj26mD> z*O{-j`1+tpc@2~?Z{?2B%MJ9e6@*~llIE?xVgOe(?3+rI>9(OOyU$%o8x;G9mO3ZHFP3KoH$8;Ey>p8 zoI~EfF+-K;1{Dj9QW>2s(godL5E)R=8+C?u2Fq5YNqyBiJnPprvv8Nw5f6E^djZ~g z{rXYY&^(b^7c@muhM!@#ycLphM-(mJi8z`;a-omRk;aLoZfdzdxHiRr+%#|O%U2Ky zPaw+>m2pZCe{127yQtK(s{ZI%pr+^*keeXYqA>(gLxb+Te5xr7irL_Dl(cEfs% zg!D1`1r76=Kpu*x|5~b3SjUdJZ1om5^;r2(r&@+T3fx#k<)O=iFESj*ch~5HL|5xi zTP`)c9XZN0>l}27QDp519i79ULl7BV>xbzxry$quus}4LHDkGQD3&@YIzUCQVb~?u zqr$bT11@A4WC7lIl8XnZa zjqM<>d=$#x>Dc9bZLA>HnXgyPe|`;0xs6^k-88LW?1qH8v?sVw=k3I>=)B7w|0qx| zlj?=}(nNK@Hb25-<;1w$$|RGj9uu2$^2_y-NCJt1Xv3Unwf5e=-3kya4h4n&zJ(BE zS+?AY5rRA-hm)WyuqehzY4i2>2sVjN3c7@PLPE3SoXO=A8;Z0H%K=7~T?uRQSj9^$ zI@-D{L?2?%a7J(x>IfvNskGf7gu|%A`oUhcY$4jz4s^u*LsWU1s^ALfj9v4#^%hkY z@NKHhRHmxLqiUQU9Zggzjp)K&<9AOV1R;$DHSeFYrREKq_ViQU>=Y-*wF;PH6k8-AVbHakit)!!CD?oNtl@hnd)_mMip@N z{!3icgpbne2yMoO_&!fK7Nw*}q2z#F7p|&tvWy#JSE!-}DpL(vo?d0Lg-t)JXyd0r z>N*JINlO-%&sb9V@@iHUlmlqN)bJ~JV5)VA(Jb?Fr)I+WQ*c!19CL)iC`iuS{3}=NC>1{OA zNGG*qJ?!CM`>y_lPqG@Z79yE6vkV5&q%9gq_*6| z{bVZqBcV3aqUT6Ym{SIbB3hk_+ylr-sZdp$$s*q+Vw?to4B2VFPv^bEqIdUs~!# zRz^ag{cD2kl0eb+p=Sl=YufQrHpq3|*`#bqU-jZxAh9N~>@;KXTvqLqk&W@D3?i(Y z>iGokkENih(q z9Rd#VDF6+HdPVYiM;A}%a)~lZS!y}d8|)HFQX!|)bjAnXWj{$WW9MsY*fX29)RJK2 z+XvK67R+6PAw#{3EUR_|a4Sx^_dk`Z)r0^P75Sca;Tn$34;o}h2|u-g)s&VW${`3y zt1SwonZdVLrS9t|WTb?YR+U{2sYxqE+-_gNidv~Qti+13wOd^u%sLE^r@3j>sOf$au?c2S z*my2%G-{5U`Mky75m1Fydf#(kWOy@0+hcd+(HV;ZsS`EUD*RH&Ih7{rk$o(Ej8WL8 zH@;CBK7hY>3itvN^nEfQ6fADKtwS%^Z;)aXT#oW?Q^~L(|0-pZ|6D#-l|f%T0VsY7 zH4H<9hQEaQH9LuiG8Wl;mV-XhuwU70;wqA2Hsgm-bQ^Aximf`6Y(eBXU!g$k(yJew z#J%W2^*Qg5S@buT01iS?r$QjK^ojL1mVGp5DZ+;CgBhLVZT>;8O#+riA}bZ9vdWFW z#sl2)m1D&SG7pc!=j$tWNA#oipmzLslU#;kWmFjB0f@kRuHI~RqvqKf21)08Pcez7 zbtZ>S0120HY<*U~$tgM9X@g&F1OR1IWSl1K0+FA-dgHAi*0bnRs}g5>QZ4@2LGoD6BvGf@NQ4xDWIDF^?hO+w)g8YP*WovQl`9qHC1XEW-klQDn_-YZ5Q}y-EdN zI*SKSo`R3uh1%uU3L1AFg{c{Wx+P7MSb5U0lg^mLq?9cAg4yRLlhdB4STySw+rULp znC@RBbdiTE80+7Wza2~C*dZ{W^=64+kK#&1BJokVub#LxLV^m$GNawBf~zpB_F{Cu z-Hvk*eO}hvLq#Wi5ef)p4N~XUZjGXB`N)Q=aH6N!ePE^{8gJx8-;nL?F(er=|8~{i z7VJUPx+nkxii#9qCv5FT&Fr=2bvcr=+jOQnNN`-kq6zVxb!~DXDnnz(doaMt|kCCm8cIz__HK${(-WHTH zCWMs#IOe~C!X5@i{kiJl6E|aaa-wyslkgg~=WwhVWDE+8y+V2fDD{CVM4~+UfpZOW zg2u5Yw@2N+0;aS7&zKz-IU6^*g=r9kq$G3MUsKKesqO`& ztD4~^@t&ro?v%fj6z@gDhdJu7Cx44V{uT`Ctg{i3@KhT;wo3d`DLoEAd>$UnoK+l*gRBe3<_g7}`U_BYdNRfRHfx?>NHypE$CV zZJ9uAnOc@Vl`UWTm`NQ>g@J$whoRv<30FLafXX%$0++!0Nz|s~Ds8D)5;Z9(XgkyX zy617n_y2zN(At`sz1aa?EI&JF?fTwX?EF<+e$2b;K6xoR=v{S8PxaG*@Jj8w%xhEA-I{(=&0_R#6lY?ZmpuY#>8jqs zdRu78wk~Wafm;@uE;CkT@75dj8JT9|aCEIDJZhF?8Jo>gHFZ--s#_P%FSUzPSGu)T z-#GG){eL6w91{e9k(Yk(8fx`{^}x-yf7$D+F#CQ?{#C7em%Vz`@_v8#&eEgqnLXIv zH%3-R?^m3H-cILtLcL>9r2LFUIHO$gOmSVOHU1vtvx);YJPNHwV(Fcl9b*)@gCa9X z;WsMV0fa}9sXkT=w)c*0V&Uv;6*B-k?IeNJx;os8O{%qkY-^oNO-*0H`cxnX6XPnl zDKkyO3b-ym$d4vpJRJM~`iNlTTxQeC0wESAMvoh3iR$<+YE zx})ZbCnTnPiD@mHCxA__>_14${~*=R-+uZ%O~%M_Yde4De*_y)<(F&}Eh((lJ6@ks z{zs~J`~sZrrLePlgc!iM@v8h3wRJ}=G$T0DH*tekbeAnV7@uP4WiTV0aMP!rSJ2nA zbn$0Nt`E2K?FY#@35LJX@gfltmvG?>^ZX;K(AHk8)$!trFEwiS4NGrv$Ddm0tKITL7&)+(cfq(!X1s1zJ=Jq@Sr97NuJHq@~*&W#%0Lb-l|hy$u;>?*S~U zVB72JIWULdBxCEX{Om15t-xY)x9&f%`gw3OK;y9KJA-UX6u?HJQZrGamTr-usRGBl zmNHsbrB1_Kg^{5pYU72m^&f=Pe-J8G{z2f=txy6|Mv zTL^Pr?mhGIapnBwfr{YmQ(wCIKdj^`hizHv6`8{N``cUT88*ir6~IQL zQ!`bfwrcJlSn9tp+3G41qgoB1O0`W(6+liHGd!@QtaazkHLobx{77q1GOhB9_|JA1 zmS2_TM|fb@!KFKw*XpFNH`KrBpZ?nVkss_T_YJ=Y(37V9qL~HzE6Js8^ts%yu=F(d zt_1HGl>TbHR`yh=W$I}!N=um4lY2U;WF!=NWIDI6m#CVF(iLT=5AO)cs0&w-oxW@$w?HrbARZAU5Hb_s9))e8(jN z|4Pz&idCP|ODa9rFB5bBrkQ)DV^F1G>7$u$y_}i1>|M*Z*xaRGJ#$}EH&?|WSk2ga zEbh99Vhu~3s)>neUA3ZRa}`_J)+~UHLZ)W2MD2tvm(Eg_OD#>bu3F(g zFou6&ssF$#S8SRpxVUr`(n?n)w!efIKT`8M%5d*cB^}%?2=>lM$x;#cN;cha3Tvu5 zchgeIDcvHsYHL52xe*Z15iEq9M@CPiJpf_B^0ZS}yHx(@7QDTcSUPnVt7bIQ+CtFqok zrU8OVf-0}!*SD53Q^MUU7w+qhUfR(Yn^22)rQPYM>DUfctzcn+iq?6_X^lMnVk`F@ z+qH&@8OzwWKi1pNMiqBYMgc}f)puiXzwh}1tKQEnQ3?HvdUh*R#HtO8g?o1!dbb`v z+I0UZ`1%_0dun49?|p0UF24KS=f$)~GC$CQ)9!vEvbFtf+cH0pVKR)C5AQwyUGU3g z^XfB0;6WmX%WuME4Sqb^mn^#a*JODudKI}tm8!IqN^0-c5`CshWjTiBB!<=!U8YJE zS)emTHKLC6IH717dXr0nL}(43c_n!@OsPC!*y^nYC z5m@HvQV#kqvBkI_fQcUEr9E6sNqE6IZ(VhVJMA`l9Av3c>CCEG+~%c87qxr^OLPvuuKi#Wbo7R^xQ2JpSQ!=3s~iR6Ce3w4h;MOoIV zZdSDHnSslVcnSG9&i``6k`72n3eAOBUDmx@J)6-E;N$ z$XxVJ;u=hxG*5D7ymOKH@Z6ct^jDB0LgPD%Ts2Oz671%N3n~W;a(($8-APY5joW8P zb9?w++#W87=XfhzrB51WFb#0Wq}=1qh*xz{Qn!nnnJz4sRLXE9AJZ_&>t@AMBfaNp($p$!WZbi#*)NQh)K5@^4#7#G zyunMM#t{&I;W2Sq*h?9(4&~VQfjdEY%e{lcLt!S=XYa6c8riEF;9`>tC4ze&Pzcq7 zYe&68prUzIxFX!*=y3d1kf8jB{XIDnE*^z2o0HAjc44NZM+QDQj;2|jka3DT(~N!9 zym8`F@(m4^JWZB7dnTY5yDS5RmB5O9+`M4|Be{@fNIoJvh7}nq{F!~V3`tG_Hw(a? zY01889z3C*Y)!MMU?Hu;IK@d|%?>tinZQqOq&bkU%hKoQWR|FR8as@bP)J^+v6P?5 z%HiO3bQwN8HG>zMM2Wyf8k3L9&SB-Xb^#xTOh_cl(WEQXArK&am-ooNWLvjwnl4nS&`cgO~C=r}YDnwL&6Brnt0%CAH@)75b3I5rKK7f;Y6Pt)khw`JLJ ztUKT$V30)-8WmO#q9x}-)s8?!2jhAwTxD*3@ixAlE6E=G{9~}_n}k}l1@1JHu69ey zspXkWIjS>j+|&Mv5$^o>A9hvG&8 z(DaBKd2F1g_B%!pD+$AsamxXUdO3|ew$3yA-6OEk(YWa)W!TJm`i&hx=gs{%R-p6j ze(wl$bY!@gIFZR_QKJG@b{V(5-Nr^si#2d@&y?qXRAXkiFzu zOpA3nE7+oC8b7;{`@n8=Q@$bdkYm##ZdyNE z2^{Vvn{)+`>!ct;u8OI(%F) z&6a(^oo>gxso&UX;<{oHHOQNN$Bk#lykXb4Zqqz%@qHRCJJ(|XgH5BWZ`vVSklWWj zZ5@zqC);bMxn7f&q^xb$yke0%?URklO=Qo#Zr8@^;41o&Vl;4^GHspB0hJ=gANhbr zLd?hK=0Yk*X+h`aak@XZT-Ydv6|eXFh0%ysoR zZ`wam@kj8qShf^*F|j0njQi>F+;(B4m{O!SQV+2Y|Go3+1?y;GzF1%+BpNgc951Z+ z@M=aq>zWB@NIB93ZHPpYCkC5MU$?8pMWe&X{YTW0Mx<>#EoL!x9sd?ak%0r> zmP^M^r?DS(LxPbnsy|eJp`nwI^2WI0U$YLF6b(^DPN1ok4C0jW$-8D;u@0G(4FMw8 z(Oij-k*Iy3(pWjYbPr}m z7?|`Mq-uT5?8QcEHLzd+xak}=t(7**;icCxZ|iq`zjr!2W0&imDQ%Y{ zOYdcV)qnnu;-s_Rv;@q2(f-7yXVN*To^H*&XkYSRW5&~Gj?zR**Ju8&@7#Ir zGJVuFi8?TLls0LvqWU9l(jeWJ`O$#1L%_}ZaAB?Zo0M97Fw(R@mT${1tFw7bkBp;~0Bo($$#D4Q4uW+<9red{G|p z%UB1^O2+8o=Sj^3Mt$yoou1847PoH|&y5w=O9{unk^U6e@OArjdOAN*-2C|Qx1xp7 zQd;p2q}zg;UN!gl*K7l3MPpQ$;yJ@+QDf@y)})Jq7G5>C+CSMA&C&tCxkznGX9T#t?C#gprNzYoi6M_``;E(H*<&8@ zB&7U;9^RJ^YX{B}NIPcvV*&Bdr0{}h-oy_)`_4pG;aVpeURtuCK;(;Jk8>L$~KVa8NWx9Y2}m8(-~G zVNz+^{BJgJa>{bNSQw(Bq} z{AgaVOr5pK=Mu8@UwNu|LGJ1=%=|{5W9|s9IR;Kz2F3Gbgp&Qs{>cB%Jm%Q8jF^?r*9jLJCTL{O|0%TTSNF2@ ztCQaC&(=p%1=rP+nnA&Q5+QznkEhFr^@HYB%lz4Zd}tSy9|f}%CPI4+vzYnCgW_yCIz7&wVK{5_u0&gv^ERf^kDd0)fI_ zk+z|HVDHgRu~>PGrTc?|KF(vHy5Na1j8IMq8kmi}`awa7ki}4v@G*$^L{7>k`u%@` zxFBtz(_jFo+PDn@#uEMD0%K4|9V>;gX@5!31f&{N8Js+F25EzkajLwCiFHLIV3O(%S@OyLu40dI18j4V{h%gjX0$g?h06t|K zQWwR67)_WwN+$LKdx?hhMFKa4y;vK43$iuY0!4|Lba}!Ig@I^Sgd6f1X_LVgg}KXk zN(X9;=~?P6D9 zTd19QO|0fB+9KitW0>J16mp_z5ddTz(k4oC;CNNSGKHL1)eB#gg#QIK94wY`Z1{s*P-tC?*t->FRcbOu?Y$f2qZej}Y} zBf8TKU6%>n^P01}~hl!}sCTSPFM%pOWO=YSz zd<7E3^!-dK0V^hMnTm{w_nBEjKd$4GGuOWE@NbYm=1VDxI+{3f3=fqgo-@_H)o>#S z9uxiZmAIQmCy}%M+BY5oXNG;Z;Y?67<_YD@hql^M)+T5cu#lMA&!A>2HX9v|i6y|} z5IQca;Wncgm5<6K=AgE)nQF;6`&h8W=$2{*4hO%5;?!1#H`|>V?l2%$2X~R@xJ@)U zql|6ZG;6E{WKFQhQ06Y5ozdnDiFqS&H7-k0<}V+a5yzHm$~Wfr?Tm1dsmyoEK!<+# zKIRN(k-tnyem5h4?bQr!7$v5&oR!YPX(}P3kZs5`Vk`zkK)}IZVWQ6}zNMv&?aFqn zI(3x6$TnuCIGhv1i{r|79Bh|khBiVR$NR~Z^H_T-GozX9#B^p%K)nN_nbE>?Dleml z?ZxzAj2NVY)65qD_~up8Kx1J))s^v^jmVUH)GlhBxS86bI;92NmhqE~-_&DNKfaUD zmFZYg{(IO1qJlafm7Dx&${cT@oakzp1L7+d9M!#yRn@|mLZd;20V#-You9+0OR-#y z%vq=?d&N0YBiZ83cMze0F1IJNoN`uqbCiXpQ(~nWq8DM@i1t`rwD;1dnRASVo`d9> z?clvI3Pdri2x=6ij9gY)6Uo7;Fddl>V^RcYH`oz&gNXHsh9*WDKa-Wg#BC5ivl+Y; zW{tRrMMEvGn32@LWa2y66n0bu0!M|VB36GorkRq<$ZsGwF&Qih8<{YZ5tmD)MwG+y zpsrEWPHx~fF&KOfLq+8Oq+59@zn0NJZ{j+b7FL^dol;0Yq#9A)k_OZO2c-nlhpNJr z$4C=ovNHfZ2TAEkQhN|zK0VM9OY>wn(fx25ObjbV983BzpXoQMh?RH}o!Eb<51xf_ zAX;M0Q0FLhB+MxunTkfH#L?%f@|C;EoMkOAmHNuYr#vvks%4dMDp;p22$V|5PNry5 zYRG1{7=um=7Pw1u|BlcXt1(mSD|RF~asAL8{2eyXB!W#Q*OBVP|3h(bJIojH5sQ>c z02@!)o5~|$fw9z6Hb148{#DhZU{@)t zWTywZfKyT`=@(VY%BQ3=Guat|ZJ+{nTA%~CJLQQUSrxZTA3OZR0i{)N{HooQ(Smn=hNP=#pzOIQ&nr1{9fNH z(cThXrE0;6DaBCIY(Jwr^ISe3S%7O(Ht9Ww_su60Tv(0*z@aITP=0LFQPzg?7p5E7 z9I}5H$|TsBqOCRL%T4*P7aKFA=wqX;Ri~`h6=k(l3CeM$N1Lyug%D%3G7H#MkPLvi zU>|zOGb{!wYmGpgB@fW2U-_2+7Ust(|--bM7NBx;mCb#br?tdr`>a01Myf2ennIcfsq zbriX19A$ys8)Z|o8QRqys<#q1l*zt4R==9A?K&%gxuw1n|8#AcnrD!@Ahvhe((*1) z9TsErw|3^uI$SGi($Jh@*wpYC7yT+>`G0WsmQis<+16+v5Zv9ZaJS$Ryl^LY(BSTx z;1;BC5AF^L4#73SEm#%q4h0nO(ckSi-njkl=sWuTK7ZC(>zrkK&biZ}Wn+}K4=j(` zx8nQu|rn$!o_~l55#!73s(C3K?t@CtEngySBkYHW5v%1L84-gm?E-irnKl zgxYua(~A7#DTSVs@`U_$_H&EW;=6?&cJ}ivB;&sdz1#5#|0NW$N1S9~8LuHEx0jJ< zp%7mn6tROE5qieUJzPwl?5;e@%|F~sKJH#U(!evEL%#9JDc)CTckg4Og=4&qknY~c z<;bTy*h^chkKpSjP-DA6 zyFpo_?qku%s#hqjR<)jk`l=>p3Fc*6ZEJyzCbjzg2SKvi>Vd8iUYfN^`-x!LR+!*7SwHUFOt{B=|=37=wCQKGgW>SJk0{K;D zd)*`J;?OkwCaEUPCWWStO)^c|P0G9mymGv{T780(K9dNOGLwKw%Srr6?Mc2#50w;> ztjsL^tmQ1BERmRw82^~Z7^F#gq!LkWQB6@@Q4LW~dbNvGNG7DMuxNMKW5#2_W7=ch zW7cEwdg^-adggio7S@*Kn{}Usm=&JoH|aLHmnELn{ZKOZOYNZSpsr#TeF}XJeFl92 zeHwipeHI1+XW?WeWocxwW>sa;Wff&vWNl>;i|~o?ckp%a`}6tpKk_m1G4hx3S>-HL zPgl?L*(}&D*uS@VZ};Bzy#OOcr<9N(pCSKWzQ6pR`9AaS@$IdhAxv7SAU^UV@gea? z@_ozd&qB+J%~FaNkfK-PQ{y+a6GqJqbLtD6d^;&UNw?{1YWp1Pa4+qRpv z+rAsITUwfBn8h&ZSY|nCJSmngWmcok$wstp5nivk&vA|IhuRgk-nS=3^(v*9A`C@* z3VW8y<&q?3LdSiFCLLDL$5jvk?n5>!=29ePkHZ;4-3q(tLnsi#3)kthGmGca%V3-! zB1A?D2lm}%vMJ+~qRxgL^r4%Taw%o7vw>d2rstbG*oSLYQLk)+U5l(2Zr|q#{&M~n z6E!kS4qWddBgjII7mv0Vj@F0O#$<%sjm?235SHDC0xot@6l9;2LvnVP6J*W7TR=Gq z$LNzYORm>nqJ>4(^t}hCO#vK;ypTo0jr*L;63#LG!vp(*?h2__aZXY1a<;>f`ufwg zkXl4>Dbduz%KFl4rI7!G?McZ?Gxm3Qy!YVt;P>G55V+>L=DX&(ejnfjB?&~2#ZW?5 z!Wc)tiNJ|SiqMFdj>zX7;#%PP&9z=X!}W`6nroD6nQM@bjv^raPljEF^+%_VW*JTy zW*N2_RvC^N7EmO2j4Sjj3^a5!j9BzQj3@Lb49|#Qo?fmgt`V*!t^uw&t_iNyspXlH z`|G&}GGq2Nwl#J!HY9W;j7aov5&aQ#5k2m6E)y=RE`2UDE@Lh$E<-L0F2B!d_ArRh zi7?X1QP=|60@+Q8Iml`=(lwEK_A0?y;C65XxD=dLFzWKReyo1w9BF}gfs_>e2>l3y z44n)k8$BDt5S^-FzJ9WPt-im0wtl?+Z~bunV*Q_U?nex7bZ?A1^g9eJbS#Vn^aKpG ztYl1s3;8_ZBtt)aKf?$8ZoU93A8Q|{eVco{Ts9^4CU$o=clIl`akeLR&osC+HDVhQ zOA-eXa}qleYZ50CGZI^p`2de|<|1|rwpg}Ub|p3o_ARz8c04vbc8UzkOovSKOlSR1 z`Zg6mm0OgXlz+lvNYXOX^wU_=s?zM!j?#A02+~q-U;nr|eJg!OeG7ei+xqIJ>Yonv z4$TgYu!zYt_OzNb$F!3)inQD`@5tX%=Z)X^&}t(?rrbZjt(vLFB_Fe12wy@`_4oabEAeT*{lQ{vPJA*PWd>49?a5 z*2R2IPto;!R#vRaer74*FMH0xPyXof87Vro7OG(0aL|+b@l|7!UU&UQELvx^BK6}= zrT{>u9KiDZO1rc8o{G19qY>Mn?Az#@Gn0e81HU=E3pw5~y5$Pz zG<+!C`b3I)c46Gi1T1C7#q~R%Iw9uI@r%7u*--E{ET#H~kS$_2Nb-XpP|iwMUyzZ$ z!&rkl4JZ9t;935qM`?$hiUw)$=slIkRF{*bP=+`XTc!~b>)=<^Heap=j!D^2%v#Qd zKKZ%jm25gOmCvHG-5_-ngW8_}bvd}J-s5^)_et+_zUQNE`(LLYp`?*XyKb-LATsCll9G1#gX1H- zNux-Z&wJjrL#Qxk_W<{tQ`TWc&+(`dr()e02dd0WWh3e=3-49n&M!9SA3wRe7;w>A z{zO|_wyM4&)gH}06?Z5`Djj|*3VEt@!38JWuJS_z80N;Y?S zx^fBdFO_l?32PFvpvp=oX$K$mNwR&+(;*qXO^bZJ$Sp3W8X`Ghn4*CXd#^yTSh2A) zv!_ksCAlS!?cy!kl!>XG_m*6BN>t>-w%}w_F6Zf}_ZQkC7NNb0qWtD;)(*e*X3x5;;y6Q#?Wt5s3TXmv@);gh_fy}STZSwY4Y z+MGR3bb|XTx0tu!?LtXaJD^`D7eR@ee zG0L%F?UnPbIDA@l>B-Le_@h?*&=GpRa(37`hNseO<#ca1Ga*140hssgangbn=GG4< zH}B^9JbqH^5wcv&6G@o|A8j#i5>vEP~Ty=c<0A#)rOA!9jSo0MgMw6Q}KI_ zLizUl#7~G}lLJ@;=pT5Q5n1xlQ?b5`lfBtN!(}bF%r3a>X7lsqQkhWaerK<(azCi< zYhNC~7i*$u&7kc0y#Pdv5>uN?L~PenM)jwBEF0w@xGQQkzap}KL8f)RJ>q44CCu)j|7XZF zfq&U{ zUWyAte3^#Rgkd9p|3Qq+f*>Ay~a|wRkRT?)a`S7^XF{U?`&L+v#TYd zw8Eo@(|BUiP_Ez8n%%$R&}5#@Yjdf22KEq*6(oN$z1Z&lwL)nAeO7QDi_20F`xpz@ ze-Q##;$~+R8BaOg*OC6`d4di7yndH`n4eT2hsd#i-ccB>3A8zrx>|<-_n;nPDDj2> zU*{#$VB=Gz+Bx|IX?w*a{~=iTm5=XQU1~hLe5mSA4bWSix1!*}{!N$ZyLOx6`uV9d zexWP9FP98r&Z~P8Ub#Hizk{lAm}A}LR4v&|$?-*@#MXvbdh@+_aqOBhUCCvZlSssT zB6f)6qZ!@Nsjpftd%`;3YPvlCxF<4*G6;LS-71sk-TjMyNTn^m)6_%L8ZmoYWWJ5M zdAz;WB1ArEv)G!yF1vlN$Bpk9PqQMcR58ByQG1ZJPqC=ISR}e=pB5)=wMAl8!6qKT z9o0#3T(7>q|A!J$es%#KI=d`fci+aFRZ`L-z#?m;c40cOBzWrWpV(3z z{}zWMKmD>P35z+Sb{2{^BMyt1E_`>|9>1;#%SkS^UHKxF5RF(M(Vg%kFG`$w5^9dT zF9*_`Szph7RiR?8ifj9SBHm0}d^8UnTQ_6$axKFP3DdncWI*iOH09feY9v1020^H0kq+ZR(4CVbtgzVu6iXL zTq4K6zWiJg0QtW}OLp|MV_VX7#3c_;y zi5im1Ks4H;xM`4RpyT0}KSWur96RxmRKs*K@q+~XNy-%d#f*V4nX^%?gxede9C6+Z z5e=kNm~zevCUI+b&>hO{=k|ToG>4iKJsVmY&c>)Hf&>%_~^B&A>!*rSAQ$E$3AR~&d+m%Gq7~@EO)R4Eu6jgnR}w+ z`?F=K3bQ0_G*hPe?-!1j0|-JKlvctYK8d-iz3-P1Vtc##~*pCz6F|aAgX^{6|lC< z49cfCMc#dx#fkqEt+^G>-_XDtIEPpg;tbI*=!D;~-2>WD=&}x3F!N`&=Q_1{i(b^<%-m98e3^u|6Odgr zQ2}}JdYteBYz5>OP2@Yh7}i0B1D%i@*fn%!W&*zf^$)y2UFXbHugBTM`B;4?+bm0e zpQKr9p~v?I*=YLAd*2B;>(YOC;!6Zm=uXEcntyQxM*eh+{HrVep|kOyp!NCwH&<|g z|B2SePR!5L)BG=G63DC6#{Vd@q3ocyD#SoxCHkHF9`P-H=Mo18+P_$ezK-}^`qfI6 z|Ikw2|JBli|L3>+r-}->Ns2zm06+2L=>L~ppku@}fAQ|WKCKW?PoUDjw1SyDFITMp zKROzoe`n>S?*H4-2uHN4DB)&>Z7P%_;p;PaHsa{0<5s`ZjG99ZOYB_n8y+-kGW1$Z ze8ViBVxn>1<|19FGf##L$AYBpj88L%=E4He^g~}n)v)&JI%;XMznfBsYq`u8a9)30 z&%Rrq=yM)AFPN80&AR?*Ho78e&#xS)@6JCkR2Hm)-y?o;tCyeK1D#AQao?MQPo{JP z{$VqmPve$q@e_Ngouz`}I{d!2jf)CTVpLU)kg4+h9`3VtjQ|yf2H~C_1_GeTZO$}6q#hM}8P}KtC;t5qEFE@~rPFaj%1p>CLnD@=HS-hVp z%1ungY|Y0dVjsa>Uj^~AsaKZ4Se?-xSPYn)Vv!VP#?>aHy)mDZFA>LJW;W?&v|D-A zXzX3lRqA**qpyypVRa?+fTid4y#+Rhme!StT>W-}|qIR=TO8vGk&ZURL~cezJ8l~D-(8!C}0NNgp5z5Sa2et1$ z0YBg~!Z&bLnD1DF=7hc()N~_``oY7=%*h!Kx31Bi2TC1(CIud5&v_lDG1sG_wrtdW zeoDgfy1@4yE|8U49&;Z;YH6p ze)~{;NMB%QmpwOnEKeZ&cjjl2l1TIuZ$n#Cwvycgs_xmvNgw2D+##fZ8Hint@G=z? zk52-uZ6cDt$lYy}3D6 z(EqwciWB# zT`~WpAqV?^l1+MZ-($1`btPnW+IW!*M&LvWweRa8?pz@A0@lsSRDHPCWSu>ATf^Yg zga&=Jrs0ueVG2TS?9HP(HvagbPksB>z#eT%Anl~_TY$E%U=0)H6S?9z!Uirf+}Ao* zMTS_;sEO)5V>Co97ZvL_tpv|zy+7|o`3P`nH^)ihTmI4Ky({C^Gs;nV+kK>#=_q@V z)Gv3vo$mD~7x;nA*@$}or zQ{w~?|Hk$wIHQoW5S~sF&-;Ptjh=lQ3u1nTY_YkuLQ7Fnc?U(F);O8d&V3T%0 zy;TrW%~-&aUXY;;pZs||((d_fx&Yi*S+7xF5=QEx+;l!DA01~e=N-QU)+7u*ii-$M z`>?_bn?P5IJGCBu9JzF)$9{}P7s`zxve z)jvp6W7hS{I_<>&Qf;n&GY2zzB|uI~irlG{>OL*%DnvyERtG6BLDvIFT! zt5S`b%>)NhjqWa3oA1GB0f+$vFz0#^?Fg20beO6K|FlSkJD2IFprO;ylLRp;e?%|- zi>`s5jIk zS!Dj;21dapf5IH3{2~_NgL{!X;EWwCe+f7kwmcb;OJ0j(jU|ifK)55m2X1eG?ZKRb z5(*jwS^Xcla7ur5os&)%$*7szS+sv*_!K{26v`f*`?U+L1G6J_8++!bf&{I-j2HQ` zv1J0tk;WOgj$?`yEFdex9Svc!L8yu%#TNU9B5|A0a8v&o8w(JWxJ^8ht&k6>Q&>;w zL=-P2Mlg)J1(3>kyqosUxXrL#DqWHth{VbN08xl%Z2Q{Gd0;bBq_9ElPnN!A2PWOZ z{3XzLS6Z&>3ZH|V9Ysm=fHOz>vPNq*aGI$1m;}_u5zuq1q8z!U?@;i zX4MG0DPlhdgsFp#+6& zL_Oe4(yqj{y}cs-fqp9qnHeAho>JdOdtfd*S$d8dB%jE?&pH0oIA~cla{9}W?o2U= z_x^nVbAT~y8p_M;RaB0Ks6khlC*vON)F+eiE5oqi&TP?K0`Qzv3;>FO2#fhgooM~x z1@bz1eEBbBw->>619e6FO5Abg=z|B?p`kqI;u`FQ$=vhYj@)XWewg)ZxYGj>XL73H z^;sL=TP7qs4$k+t^n3QH3>n9DBm|JGZ~htH2nmiUM9{EMR#6U|sK{_`Xlh`z=qLLD zaMglz4{SwYF?J_F$v`ZCKglpv((Y%vQJ&Sd*4Po&LTbo$w{ChMzc8@&Tk5R|7ps9`u>&D;$bFVVf#~%izCok-wwP9OS2_sg0Jy6u9!dg= zv*ZMw0@4$W)P%!55D(5;QCjy#@h-)oE6^b*9JGn~Hk954aaBGA4}(!SYu~Bxkz0ow zEQ*daJvcz7KuPVJvd!J70qtVC{fb%t#)Q5-oN3jRpn<=Eu0h_chyGM5fFhb^6j+%Y z$Dpr#30J}BsJN1LE!U{$is;1RMCfaA`FneKs~@Y0S@~;1^nyjir`=EDy|;as49|(h zc%ub?Sp^R26GcDn1NvftA=CD7KYQp z<)ylMvx_w0yfm~_FyI-DMcthTm4jqQIn$w(Erhquv=WVa(_y_l;RfSABgR-`i(l2q zfVqxiP9g)NTi>}VLC~V>=!5bB0WBB`*2y`*2O(QQ%V+8;`CIB!yUQ!4+pzOX$~WHF z;Xk))Ws0KuSb51kZnzw|ddzzmo@1bfGq}alz_KI?)msMrR-E6xHNU#lg5d85w>J`B z;)5xBz7>kgwr=&QU9ur_CeU%sbCp^A?p=+z$Lt`xaDk|oR8#wg@pY<>fhc0pdS-J1 zs4kyI)M$dSoG<8^`dG_|!rUN^g`_f(cofptD5br+zr2R>z6FM~!nuA1WfWtKx`0Lj z!=NywHY|M~e~w=_1L2L9bh3&`T6|&WwW1h(uHH1OYJA}~G}B@kV*`uWky{W35JoYY zf*K9RS<<;GEU8+^35O$Kyd!ZVk}4)Y`~_W;3B?L5x}@9;+P9i;I~G00SVF2#_>@nF zx$*o4LO-B~Z7Qpoye$sWQy5Req!vrP7M)of4eD#@U5kIr3Shtg?2}*`(}VItD=P?; z>lYM^D`(NrDh!PN&Cs6fgMaZC@K=HTpvrO))FmIpd7ljBfmU5?Xobag5KKR3;AHf3 zd0Xt^Ez)#nK~+H3fVV&=hWnUnl$k{Zfn*&9e*`_uMRX#&Y;L!)hfyfq#hpe<@jh}g zSfnsCs`eLp@5De(7|NZem4oT=))H2f*oDPFfyoo7m=@>=a*2+FXE0(i1l|Eoq)96$ z&#m$$e~>I<2)z0E#vA^W#7#{+9mXG^AHWJzf>JTJ_pQ4ii^6f@w$KY_*sx3=$i#BJ z6c_kZ$q+u*G_;=M*{iIlbyD0rT-J^J z1ii$mUGSg#L;9VCQe5b{I?OSLVNoX`7%;$E2S|tl#9xWC8 zir*smj}}?xwh(%C3#S{MK$vtkXVY1`Xz_^zffOB1z_F_ZF6pM_V&ng z6IDw4h}?bkj>^Id7Z}{iaJgjb4TzqsZ>rxj*E1Ku_NHEs6zqIQh`k}*7-q>lx&u&4 z42!vwoGBaaBRgkZ)!ODA*-MV}`!4)U9dnBR#8j$25rVvcn@YDL*-S9|c9zH;d6)mV zV?ZL=ND((?oyF2~Bn#xpfR_F{y)}R2Z*rtN6bHr)HDtEaLN5?}=M|pGg;YDlH;g`r z-XFoc?FnfZ085g|x)Ivi&bV4Oxj4GgxiBWD(>u$phr^^h`>EovhMjE)g|RUhgh5Q& zPJ~*d?qa*_5F!QZ#9P3c+(MoM>auL>n(JP&9-~AUOe3a?Sx$po9tU&OK%GqL>jnLE zri?SrQ$%J=W;n{)>ff9^8DOMOQvxl>$bj?2fCnG6iuYTCfbf8;) zH5cjtQs3-Sy`-?^ol*aW1@I^N$#`mT*{~f6qKd%6czou9bQwR3K;i5>e~;xoCIeYiKn2wSBa=-O+T*T2H(FbAZmRZ)eFy^FOIqk-kqsQ6Iouke)vjLWVO z#w6#V^!|i?&Rz+O}m z;ia@gIZ80ClS;tb(O&>}1*wneZ7worl4RSFsu8>FPD6W0~tW(`p%M8FdY7_0shg4Mb+9)F!f;S;cff8yY<& zj64%L((XwCpai}BhxbXx+KmL3ripKeKw0p6G)MBh8F|}JE4nB3~q&o&Gpm;b;`_g^XnLaog;_wmOK8k#ryzrbS@0)&% zFQN!Mnd){ zm!~bSM)QVJdv69Qq79YTV>-$1pP@kL7BgFwJGRqIezXDh4I}E#k*ACrPM4HX z3{(tsZixJ?-?_L*K@AX?b8Wwhzv|eUx=k_im`tTC{xtw2NB~BI50>L~VP0iO)!vSl z!~I6n4%b9zh~_Wh`k^z#M{Kkf;GGvudmU=?*oyPm%N280L*PCS(pj&mStoMPalLp!UZa+P+KPZFTT z${lC8)H_eTa6d28dIdI-YWRdZ)B3^9zTrj0`+dmRz-iHMzv~Z~9snyZ$u%MYp-Jy) z#3HKs^jq~8KwS~6(NzilnVll{)R(T?YT#_Ca!ILRZ`|qJ43ms9u6)q3+)U^zIT^#! z{bgQjjzho9Z>{m>aItRYyJmlk|D26>jq43V%U$(Kvr|)+zmAhhnQTQGzoE`bhu@K+ zV`X#AOl`p97{%C$R*glfYKzF?@=t^GG2?OXsxH%(Q+mR2z`1mz`bM6-Ni<9d#sU+7 znw|%%hY^bNS%C>U7h7lr*Ni_6Hx22yceJPS$kl7mOinRG2BBiC3`nuxjhW>&|Ux20B@LpJC%Qx7XBR|Oa4YC*PApSv1WMfOg6oyZ?gR7Biv6fVzb%2Ee-3+L>V52Rj zT120-anw?4)Ot%7eA*lC`~%&W;F5Qw4iHQ!ffY>t%m-tL2`R*sRf^HYcPtPWMg7yC zAcHgQm?56w5}gz63rA*FhP@$+tC%h}DwTi9=0tvqyxOrnoG1}>_NC_?mExGX;a%i1 z_R8U?5dgsdh}$p_&DVaN)^Ov#F=k{ZM{Da=pKk#nQ7 z^`ChzE(ktQ^m?=GX=(pT!0o1UmU<@ZrU}A==|EecpP3A_nhP?po;K;Bs6o^#5w>63 za11g05pFRcGy|r|#K``7(r&nJiXhP-@gS7& z66B4ba;m;m;jn#!LQ^j0Qj%TxiMyqDOYfKH2j-(Aan-2BBcRrh;b`g5Tebs)ndsuO zq;7@wR0q*zomP`pEmzaTvk&5NFcg?FtOEb$6Z|Sd!FjF1Ty8nmv`_{^hPsjG0A_Fe za9(I&YAd{JFyvq$KQZjlNqDp$umSqbh~cf?8cqg&*p%C1fDl5~>8O;u;Y|r7zCx&( z5R15w%qa&}}|}0SThV>e8eScwIb>tNn5=D3{g7)k5JVa)*Cq{a8DAIw(2pGrY&mmm)tcb>Yu{ zor11JfT#2rrsqnQLd%GjV5|c8PIRinLjfkaeiIj5wn&MnTL$4oo7WnUoiB^8c&8NY zG?3CC_e*(*{2Rv3?5_c^`~@q}0(1wIx_M?@R%o0Zsip3q)I%aNVed)CqNh^nPJy`v zs00WCr3NpV#dDfBG@FZvztc;%nyW^fMYrQj0{pONF`KZgDQ*QIkqTMK=+v3AZup~q zGbznv2jJp@9|D#bqY;VU=tNNT-g;9zad3Ftoh4F9QYx;VeMbu0t-{b~4fg(ZJrF>b zUbaTjf%y3Lf({_9AO~`w5&OyqtA3Xl7(*70R%7oSlR-<4yt8RsC@ez}Ej9cA*u^^5 zSvp_W2WqAjvyU>&j1?m#bxV^5@mP%ZiREpsY{@~y897J%L6J%Ug0fmMs#7@YVRg|g z*cbre@O_Iy^_j3B-s`i14<-p+Je6o=ODa%qMP=cE5!!2>G(5b^J?y6^N z-H3hiBE3W|3_idxV4qxC;>b+mqZ(hxK8bRiYsN4)r`fm#)*#1FL@bX-)RjWtqG@h3 z_km;akqf0Bq8`B_;vKfH(53!%!N^Mr%-#k?DVnDcFS?J4V>ryI(+*82K8>t`@bvBBCzeZ7Ypz4801lN4|s=!ppkH z*uGSL?Rb2$zb;>Dva2)bs=BL~`svoQ^2CZ+E?z(MT@fjT#OSD(RQeFThPQg?ToFl! zRKwS@DAb-p(@cAQ+H**o!qC&GIaYWTwLqD#Ku~ne%it`+OBV%*oNyD*ECF zyLwr9`Xe#t^lTnZ#5$s2GDH*V$#p-O-vYJ_R4Oqr!2kuR+GU#fN&c8AfI8H);!#{W zP8&YZ>f1i7x99k(yt4?&381e@eIvtr)8gCl82#=Lb#vY&BIlx4O(`zYmgkah?OZ5$$C*-x<3PrEH@~c8HJQghk$y-@UujJBz$3RM znEs@jt$`{P!sM^xeDm;tSzg9|^G$f5ru5E5Y3D=yywJX}pU(T_E?VcKXRy}?UWACl zlGv6};f|96f@>X~lEmdv-Q2>K8I)^s%I5-6kU6 zOEJV+r2aK14ZR||5zQ++9{QMRJlgrfYP{dFaIhlOsEQ&#EwPx;vESL#acRVAxkKp( zFLYUTyysPa9eC-1K$x?{&!7B*ttW_dQbCAc`@&iMnBD3 z3;*4{Iw6#YEx+b(Wi-4Yke**n(T#+wr{u(1o__cDobN}+oe0tqb@pqfWd@4#gqffx zUE89Pu6O$W!)c(H}?FS==>PbpG%oZcEt@x>2HVW>&bzdvh|j?j1~PUKj= zlaOM;%oLtT3vY}2emfQT30NT*R<>Wbcz|x~BvHc{=*~{EV8`lG72@`dK=hz8W9hW9 z7JWG2y+&k6NejF9htKB{On)wO&|R;bJ!W3wUEF!n581w}3o2T9UUh}MT|=+|r5-b! zBy22p*=Q|CjD|q?j8!I*Np&mhm2P;BziSmfwQKDXSo13(t*TWB*MoP zne~6jt{7F6RURo`(T6|ZF7EsdgZjwtA&rY& zuX_ftJnGtnDJ>oa34ol&v$@$7ixs9Lue5d`%w<@``#dTvT3h%LV%T19P%6Dlml8TV zU0u)nl*-U*V!eG}(#L@O!K)kHTiTT|{Y8O&;w)8Fm7^cEJ>+Y&&lemEp*S?{5&Ori|oFI*8?D2E9wiIC0HxR4m*pP{{a@XlDQCGEjr;mk0V>9~A6lSWv!L-n99-Q(j; zD5ONiBhR(snw4jxri?5h*@M=^v?1#rerI&`0cXubSEJV}=R01(AkmoH7Z2ISgG#!% zf*#F9o~Kj_`|rq0l9^TEAT8^6I#x{-5;Ts9e2r>~M66ZgYaWYt#UT8e7a6@6KFK^np|&uNqKYCb*8XHsaYpbvX}lq&|CF)X%7V}B|U79?kUt3X0DxmXG_k&PBOvhGOW(w|Y&)^^$YND_S!--w?6G-`BQ zb+STzV6+v{IEDvbSOg*aW;wh=nO-Y^^?5O*@;GOhj)2hSzI*G>yxK`Ir>h8MuJ^9{ zFI69tw%NQY@7=4lmz%vT3Fspw{Y&ZRZ@IS)w>{8fSkEPS{Z)R^>+`6SKn|!#p8YfQ z>{+7hiBp{DEN5>}%p<=04f6{545{{Rb9{JHHZtsEK z6(#Pbd-$rQzaCwJMi(jexj-Kf(O&QQ2q1q7!H6CO0iUKz} z>OYzlN!UHXkFDqEjYLOP+Amu0WKjl}Eq0;96@aM#H`c%?U;FLhQ`0ku+OlsndA`@C^NvSIzzC8`c>5`T+et;&p%SCMhPE3 zvq*XT<6xf}wgiX2YNCV&PLFzeO@-4pq*!=5O=%>=%8r;=8aWd>xUMkUIyR3gV%jdt zBldvBOh^NRUipHC-h%Q0tNGv

yu}o>Wm_g}Jq~t7f0{$9Oz#;H~*!jHZ~@rc<$3M)PlZm}XBt~js*F1udPtBi z{4Me{qLHqW!VjCR7m3t0bnG-%^AvW!gh$yIa%B z0V{l2w-UoOcEqN!Le#W3!#v^;t3TyOWy#}4f9mmrq-@4HFL}-8h0Vn-sAC>AK)Kln z>=LhI(VS4rP^UA}hyk5#p^ zvyB2ogRO;GPR1$cz6$u4Feg_M%SC-@xa{c4+%$p_E%fhc<}PSvJnH%j{1C;CJ8d*a zmXPc>S6bRJZPKnqSol`nb|twb%FrgM#&GZIeUxwbwo8UxgSBH0cRKhlECd!#-sWz6{Stiw%yyZffaHuRS+VpyH@TBcgAcjYNQ9t#&5J^;ED@2 zSk(c0RkBu5RjdakJ_HJ|IQxDhHJr3e=5Tss9Si!2t^J3BwaVRWJ`L55c4iorf9nT| zizazh-*{cdZz=68Z|!cI0`MEHd3qt8w@7Kx1WcCC_K<`qa__HhTgEz2sK{ zGB620ZFV_zFbK*FX%j7R&@3%p29)4QXp=!ymz&URMU7|*zoE{&aI9lj4YXQzkv=_^ z7kpXyPDZe}QAxS)eb}^7=yu^G)m5s-#BU&D?tr&0C?w5&qctFa^KO?GD1|*@#yOoIPVsb-52`aQ6o$QJx%MQg)NS6e8?#W> zsJJo-cc9XOZrh}iQ|X1BafC1Zg)s}opA9?Dz#LCOq39`#Y(PBFa{<8Wxov`b#APE!&AIK!qgVOp>CkDmJI& zm!-e>yQWN0--9))fo5RB3S1u5)I^LJq%!k3>s%dHo9~3Zj4|b@dVos?f~CJPV5X0K zKBdcdWq6M%b zD6rw@F9|e-;j=47EzqR+FeXLIMEOe>Gtvkh*<%los#p_K4`n8D{Yyk{ z0jitj11+W)o__bb=Nq&=j1{F1I5??v)>gN6$jWG=N_qYtMI#P>ULKi`DvZW6b|_ure)aFo{Hm3rTwLYK)&vSw;hCn+dzZ+nM9lL5v!#SMRJQ($IEU>Yem~ zPTXq%jb9V5=jKRU z!Aa!Y?nvVy`1bB$Zs|FAiz)$5=gZ>o$f3yXMbY@nW6xDS)t3G7 zQV&w{a|EyVp;sP(e99whu>ZowdFtol z5$Xvuy;9!^T!Zl0M@xMTCnLM&|4(UW9uMW(#_>_aC|RS_OBhi^oko%^yTKU4U@Vy$ z>tO5^EmT@C#xj;Nj&%?-mXoYG({hxOED<9iTSTPPAnW_oSu&mTp7Xr#`^P<>XFi{q z>w91KbKUp-Joo2w{VuC>dX0+*@puX511|CU;L8&k+Hi@2N=ZrA2Tw$yRkSDF%Z5|p z(wY*yhnP46#4N_f=jVlwSzb#v+XYrn>o*Ii{#=u{=|?$qq>+FC%V*c)8t(RtlC}sR z_MlTVw_jJ7>gznXB)w(404KnDlIdP72CQey#smmsYFwimvc6`APfsP7A-X!GB2;9NYMy zT#0V=MwEN|EGqs^v=G*aqTD`g8@}igEB4{qX!Me5wC*&)YZxBQ^C`>RPHdwZ za$pO&oGPOdlQ+=$e7j+QK$VW=~pc`$%G$yuPLKTV6Q2@Rr9E`&xq49ECrDY*m=YBZ;vC_?Umb~Et08*Bqy_}mp@!KHrs2LR#IiQyANIp z90}pqce>R%Z=o7bWD*k;8F(CGU*bS*CVqy54XqfxtzyUUuJYhc#`w(fuxD2X2elMa z?_6rmrw;71rg5oYil$-EeQNg6MM?;He@2aq?X7l|gstlki#@qTCnFy!iSvd^v)#hQ zba5hEZWQ5J#?d5c!<(ecjtW|nkx6oFrUL9?XLxbo85f$9OMCC7c-vlW8`nuIpThjZ zS)L5+nG)D9QQNRKazDHz7_yE%^{#x|MBjn6>wuUQ-EA$< zv5h`%%2qpur(RE|K;#1~ zfkCCVX3=-pqO`}-D0!ksWyVb@58I9m@5Xh8s9WZug0MM(LuVnT5AR zGTM+qj$w95Y|GT^+dp2CWckx8W-p9Nh0BK148;$o?htBpt95aWOr>kDTrYseUrUz1 z8Bk{`76psylM5;vv4V@|pf3jMw!hy+8y6(PiJz_58?F*8`e~BqXNtJ3O{uoJ>BTqkMiK-m{CYApJ%5~#Wu6=3hHIb;WaUO{U8O5C9cSeGl#wY+ zlqkf>)Xp){+p%j!BuFd;jS4Lv*=>5!S{tfSK1bJo<#eFME=oZ;ZC#}NLGzi7U0wy9 z`zeaa?vgy^*pHG_uk%fJCJx^zqrb-^NU6T%di2b$woTa@6ybVnq=O?(?<$VI5S}s5 z91UwGOBSrm?qc7YOH$s8KGH;>jtx=;gCg^45mqxOsz{x4$qLK1n^BCgWx`&Ucx|(s zJ20gCmSt3VvSMB8qZcPu;#y_nAHM8U%9vAcR!N;arhabl0Cs+Lvni1Vyfa>QIsgeGoI&~NOK_R_T*O^oWrr$_4|JzpGg$$Gxc1>Ck95gmzkrogUEPJq&&#?C5u6D% z$$-5EAZzp-kR)sXvKBniJ>>XaA5$Z&DH>shHVAVJGP6J$;Z5~0Xcf2PA%B-J;-1L# z6V(%X6u%CNZfj4vB6jm#h=xPK{l|}u)Em#g=s z?`e)ZSb9FLH_k*qA0sq2MB%@MW;tb`?-vCo*-P^`2h%;d?+2!`luq&CJ${hG@THWX zd?)Jy!sc(UDxTKSc(ZB7rK>+oJfGJlmRITOV&Lk>X%Fha>v6^hY0ePxM%`@P1BB2O zpuv6*q5mUBj>a?1)}w_kv-E{<_xjB)D@O|nZC|ZS-YHz1+Src>YV2b+EkmJr_i`xg zzJBpp-Ygl^vA0TC$BEvCz7rjX9_~Bqc_?`Fje#%G&Q=Fo; zJ2y<yA8Y;MZsjF5+&UBKb;(vJ;kRy!gUN$+3Zps)#l@%|rn%=5fnRBg^ zTF(#U!$BY|7ab8_FMj*rfo%QlN(i#7_vJFkJ-}bp7Q+KsA1Ne^JrTMQ!o~e&Jp}Ug z<5yE+z|Z);nj*ZCb?5o|4FVDV!xT`o`-1N8rU-s+?x8Ba$3xtMfjJM(Y&ry{e+n>A z2eeJUnFYcIFdNTdb}x{22F~J%_MZ*z1OM#@8jPc(7J%WeJ_q|95BS^l>hDN@J-Pya zkiLwr53qQCB>_W1aCHyQfyEJ@r(e$n7KSjv?{QxW@_~yZ9F7PU=SFTg@vZja+QD3}1Rm!L7FU~a z{aOz&kiVuEuf?S(gU9)R#SJ^j-cSKnsb4EguEn+Q1&{Lvi_?+GG2Vd&CW-g=_s&%n zJdOYs$9n#-&<;q}u0PE6U1ScO2aodti_`Q#PKg79c@0= parse_version("2.4.0") if supports_dots_in_field_names: - mappings["doc"]['properties'].update({ + + if parse_version(version) < parse_version("5.0.0-alpha1"): + mappings["doc"]['properties'].update({ 'field.with.dots': { 'type': 'string', 'boost': 4 } }) + else: + mappings["doc"]['properties'].update({ + 'field.with.dots': { + 'type': 'text' + } + }) if parse_version(version) < parse_version("5.0.0-alpha1"): mappings['norms'] = { @@ -339,7 +347,10 @@ def generate_index(client, version, index_name): if warmers: body['warmers'] = warmers client.indices.create(index=index_name, body=body) - health = client.cluster.health(wait_for_status='green', wait_for_relocating_shards=0) + if parse_version(version) < parse_version("5.0.0-alpha1"): + health = client.cluster.health(wait_for_status='green', wait_for_relocating_shards=0) + else: + health = client.cluster.health(wait_for_status='green', wait_for_no_relocating_shards=True) assert health['timed_out'] == False, 'cluster health timed out %s' % health num_docs = random.randint(2000, 3000) diff --git a/test/framework/src/main/java/org/elasticsearch/test/OldIndexUtils.java b/test/framework/src/main/java/org/elasticsearch/test/OldIndexUtils.java index fe46251e3ee..01474e6547d 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/OldIndexUtils.java +++ b/test/framework/src/main/java/org/elasticsearch/test/OldIndexUtils.java @@ -30,6 +30,7 @@ import org.elasticsearch.action.admin.indices.segments.ShardSegments; import org.elasticsearch.action.admin.indices.upgrade.get.IndexUpgradeStatus; import org.elasticsearch.action.admin.indices.upgrade.get.UpgradeStatusResponse; import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider; import org.elasticsearch.common.io.FileSystemUtils; import org.elasticsearch.common.logging.Loggers; @@ -56,7 +57,10 @@ import static junit.framework.TestCase.assertFalse; import static junit.framework.TestCase.assertTrue; import static org.elasticsearch.test.ESTestCase.randomInt; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; public class OldIndexUtils { @@ -103,10 +107,35 @@ public class OldIndexUtils { throw new IllegalStateException("Backwards index must contain exactly one cluster"); } - // the bwc scripts packs the indices under this path - Path src = list[0].resolve("nodes/0/indices/" + indexName); - assertTrue("[" + indexFile + "] missing index dir: " + src.toString(), Files.exists(src)); - copyIndex(logger, src, indexName, paths); + final Path src = getIndexDir(logger, indexName, indexFile, list[0]); + copyIndex(logger, src, src.getFileName().toString(), paths); + } + + public static Path getIndexDir( + final Logger logger, + final String indexName, + final String indexFile, + final Path dataDir) throws IOException { + final Version version = Version.fromString(indexName.substring("index-".length())); + if (version.before(Version.V_5_0_0_alpha1)) { + // the bwc scripts packs the indices under this path + Path src = dataDir.resolve("nodes/0/indices/" + indexName); + assertTrue("[" + indexFile + "] missing index dir: " + src.toString(), Files.exists(src)); + return src; + } else { + final List indexFolders = new ArrayList<>(); + try (DirectoryStream stream = Files.newDirectoryStream(dataDir.resolve("0/indices"))) { + for (final Path path : stream) { + indexFolders.add(path); + } + } + assertThat(indexFolders.size(), equalTo(1)); + final IndexMetaData indexMetaData = IndexMetaData.FORMAT.loadLatestState(logger, indexFolders.get(0)); + assertNotNull(indexMetaData); + assertThat(indexFolders.get(0).getFileName().toString(), equalTo(indexMetaData.getIndexUUID())); + assertThat(indexMetaData.getCreationVersion(), equalTo(version)); + return indexFolders.get(0); + } } public static void assertNotUpgraded(Client client, String... index) throws Exception { @@ -128,10 +157,10 @@ public class OldIndexUtils { } // randomly distribute the files from src over dests paths - public static void copyIndex(final Logger logger, final Path src, final String indexName, final Path... dests) throws IOException { + public static void copyIndex(final Logger logger, final Path src, final String folderName, final Path... dests) throws IOException { Path destinationDataPath = dests[randomInt(dests.length - 1)]; for (Path dest : dests) { - Path indexDir = dest.resolve(indexName); + Path indexDir = dest.resolve(folderName); assertFalse(Files.exists(indexDir)); Files.createDirectories(indexDir); } @@ -140,7 +169,7 @@ public class OldIndexUtils { public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes attrs) throws IOException { Path relativeDir = src.relativize(dir); for (Path dest : dests) { - Path destDir = dest.resolve(indexName).resolve(relativeDir); + Path destDir = dest.resolve(folderName).resolve(relativeDir); Files.createDirectories(destDir); } return FileVisitResult.CONTINUE; @@ -155,7 +184,7 @@ public class OldIndexUtils { } Path relativeFile = src.relativize(file); - Path destFile = destinationDataPath.resolve(indexName).resolve(relativeFile); + Path destFile = destinationDataPath.resolve(folderName).resolve(relativeFile); logger.trace("--> Moving {} to {}", relativeFile, destFile); Files.move(file, destFile); assertFalse(Files.exists(file)); From eb4b6cd8163dcd01f8b9ec1b70ffe512846bc118 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Tue, 1 Nov 2016 12:15:56 -0600 Subject: [PATCH 072/132] Disallow VersionType.FORCE for GetRequest (#21079) This doesn't make much sense to have at all, since a user can do a `GET` request without a version of they want to get it unconditionally. Relates to #20995 --- .../src/main/java/org/elasticsearch/action/get/GetRequest.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/src/main/java/org/elasticsearch/action/get/GetRequest.java b/core/src/main/java/org/elasticsearch/action/get/GetRequest.java index 38dd10df963..93045182f4c 100644 --- a/core/src/main/java/org/elasticsearch/action/get/GetRequest.java +++ b/core/src/main/java/org/elasticsearch/action/get/GetRequest.java @@ -101,6 +101,9 @@ public class GetRequest extends SingleShardRequest implements Realti validationException = ValidateActions.addValidationError("illegal version value [" + version + "] for version type [" + versionType.name() + "]", validationException); } + if (versionType == VersionType.FORCE) { + validationException = ValidateActions.addValidationError("version type [force] may no longer be used", validationException); + } return validationException; } From 8dd91eb2d67dbe050710bd4ef1caacd636b5343d Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Tue, 1 Nov 2016 15:14:24 -0400 Subject: [PATCH 073/132] Add empty plugins dir for archive distributions Today when installing Elasticsearch from an archive distribution (tar.gz or zip), an empty plugins folder is not included. This means that if you install Elasticsearch and immediately run elasticsearch-plugin list, you will receive an error message about the plugins directory missing. While the plugins directory would be created when starting Elasticsearch for the first time, it would be better to just include an empty plugins directory in the archive distributions. This commit makes this the case. Note that the package distributions already include an empty plugins folder. Relates #21204 --- distribution/build.gradle | 11 +++++++++++ .../resources/packaging/scripts/20_tar_package.bats | 7 +++++++ .../resources/packaging/scripts/30_deb_package.bats | 5 +++++ .../resources/packaging/scripts/40_rpm_package.bats | 5 +++++ .../src/test/resources/packaging/scripts/tar.bash | 1 + 5 files changed, 29 insertions(+) diff --git a/distribution/build.gradle b/distribution/build.gradle index 42b696d9cce..21467ef727f 100644 --- a/distribution/build.gradle +++ b/distribution/build.gradle @@ -214,6 +214,17 @@ configure(subprojects.findAll { ['zip', 'tar', 'integ-test-zip'].contains(it.nam MavenFilteringHack.filter(it, expansions) } } + into('') { + // CopySpec does not make it easy to create an empty directory + // so we create the directory that we want, and then point + // CopySpec to its parent to copy to the root of the + // distribution + File plugins = new File(buildDir, 'plugins-hack/plugins') + plugins.mkdirs() + from { + plugins.getParent() + } + } with commonFiles from('../src/main/resources') { include 'bin/*.exe' diff --git a/qa/vagrant/src/test/resources/packaging/scripts/20_tar_package.bats b/qa/vagrant/src/test/resources/packaging/scripts/20_tar_package.bats index 7f9ce21e85d..83c12f960e5 100644 --- a/qa/vagrant/src/test/resources/packaging/scripts/20_tar_package.bats +++ b/qa/vagrant/src/test/resources/packaging/scripts/20_tar_package.bats @@ -73,6 +73,13 @@ setup() { verify_archive_installation } +@test "[TAR] verify elasticsearch-plugin list runs without any plugins installed" { + # previously this would fail because the archive installations did + # not create an empty plugins directory + local plugins_list=`$ESHOME/bin/elasticsearch-plugin list` + [[ -z $plugins_list ]] +} + @test "[TAR] elasticsearch fails if java executable is not found" { local JAVA=$(which java) diff --git a/qa/vagrant/src/test/resources/packaging/scripts/30_deb_package.bats b/qa/vagrant/src/test/resources/packaging/scripts/30_deb_package.bats index d0361339bf6..d435a76b9c7 100644 --- a/qa/vagrant/src/test/resources/packaging/scripts/30_deb_package.bats +++ b/qa/vagrant/src/test/resources/packaging/scripts/30_deb_package.bats @@ -74,6 +74,11 @@ setup() { verify_package_installation } +@test "[DEB] verify elasticsearch-plugin list runs without any plugins installed" { + local plugins_list=`$ESHOME/bin/elasticsearch-plugin list` + [[ -z $plugins_list ]] +} + @test "[DEB] elasticsearch isn't started by package install" { # Wait a second to give Elasticsearch a change to start if it is going to. # This isn't perfect by any means but its something. diff --git a/qa/vagrant/src/test/resources/packaging/scripts/40_rpm_package.bats b/qa/vagrant/src/test/resources/packaging/scripts/40_rpm_package.bats index 5535d1a67ce..b6ec78509d1 100644 --- a/qa/vagrant/src/test/resources/packaging/scripts/40_rpm_package.bats +++ b/qa/vagrant/src/test/resources/packaging/scripts/40_rpm_package.bats @@ -73,6 +73,11 @@ setup() { verify_package_installation } +@test "[RPM] verify elasticsearch-plugin list runs without any plugins installed" { + local plugins_list=`$ESHOME/bin/elasticsearch-plugin list` + [[ -z $plugins_list ]] +} + @test "[RPM] elasticsearch isn't started by package install" { # Wait a second to give Elasticsearch a change to start if it is going to. # This isn't perfect by any means but its something. diff --git a/qa/vagrant/src/test/resources/packaging/scripts/tar.bash b/qa/vagrant/src/test/resources/packaging/scripts/tar.bash index 3d6210a2ea3..b5edebaf41c 100644 --- a/qa/vagrant/src/test/resources/packaging/scripts/tar.bash +++ b/qa/vagrant/src/test/resources/packaging/scripts/tar.bash @@ -89,6 +89,7 @@ verify_archive_installation() { assert_file "$ESCONFIG/elasticsearch.yml" f elasticsearch elasticsearch 660 assert_file "$ESCONFIG/jvm.options" f elasticsearch elasticsearch 660 assert_file "$ESCONFIG/log4j2.properties" f elasticsearch elasticsearch 660 + assert_file "$ESPLUGINS" d elasticsearch elasticsearch 755 assert_file "$ESHOME/lib" d elasticsearch elasticsearch 755 assert_file "$ESHOME/NOTICE.txt" f elasticsearch elasticsearch 644 assert_file "$ESHOME/LICENSE.txt" f elasticsearch elasticsearch 644 From cf3e2d1aa816132c70afb0d8d6b80fedd5adad33 Mon Sep 17 00:00:00 2001 From: Areek Zillur Date: Tue, 1 Nov 2016 15:31:28 -0400 Subject: [PATCH 074/132] documentation and minor fixes for engine level index/delete operations --- .../replication/TransportWriteAction.java | 6 ++-- .../elasticsearch/index/engine/Engine.java | 28 +++++++++++++++---- .../index/engine/InternalEngine.java | 7 +++-- .../shard/IndexingOperationListener.java | 28 +++++++++---------- 4 files changed, 43 insertions(+), 26 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java b/core/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java index 111edf5606b..15f269c46f5 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java @@ -89,9 +89,9 @@ public abstract class TransportWriteAction< @Nullable Location location, @Nullable Exception operationFailure, IndexShard primary) { super(request, finalResponse, operationFailure); - if (location != null) { - assert operationFailure == null : "expected no failures when translog location is not null"; - } + assert location == null || operationFailure == null + : "expected either failure to be null or translog location to be null, " + + "but found: [" + location + "] translog location and [" + operationFailure + "] failure"; if (operationFailure != null) { this.finishedAsyncActions = true; } else { diff --git a/core/src/main/java/org/elasticsearch/index/engine/Engine.java b/core/src/main/java/org/elasticsearch/index/engine/Engine.java index 0df026ad617..0da96def3ef 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -278,9 +278,25 @@ public abstract class Engine implements Closeable { } } - public abstract IndexResult index(Index operation); + /** + * Perform document index operation on the engine + * @param index operation to perform + * @return {@link IndexResult} containing updated translog location, version and + * document specific failures + * + * Note: engine level failures (i.e. persistent engine failures) are thrown + */ + public abstract IndexResult index(final Index index); - public abstract DeleteResult delete(Delete delete); + /** + * Perform document delete operation on the engine + * @param delete operation to perform + * @return {@link DeleteResult} containing updated translog location, version and + * document specific failures + * + * Note: engine level failures (i.e. persistent engine failures) are thrown + */ + public abstract DeleteResult delete(final Delete delete); /** * Base class for index and delete operation results @@ -291,9 +307,9 @@ public abstract class Engine implements Closeable { private final Operation.TYPE operationType; private final long version; private final Exception failure; + private final SetOnce freeze = new SetOnce<>(); private Translog.Location translogLocation; private long took; - private boolean freeze; protected Result(Operation.TYPE operationType, Exception failure, long version) { this.operationType = operationType; @@ -335,7 +351,7 @@ public abstract class Engine implements Closeable { } void setTranslogLocation(Translog.Location translogLocation) { - if (freeze == false) { + if (freeze.get() == null) { assert failure == null : "failure has to be null to set translog location"; this.translogLocation = translogLocation; } else { @@ -344,7 +360,7 @@ public abstract class Engine implements Closeable { } void setTook(long took) { - if (freeze == false) { + if (freeze.get() == null) { this.took = took; } else { throw new IllegalStateException("result is already frozen"); @@ -352,7 +368,7 @@ public abstract class Engine implements Closeable { } void freeze() { - this.freeze = true; + freeze.set(true); } } diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index e993e464764..36d5f195905 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -424,8 +424,7 @@ public class InternalEngine extends Engine { // errors and returns true if that is the case. We use that to indicate a document level failure // and set the error in operation.setFailure. In case of environment related errors, the failure // is bubbled up - isDocumentFailure = !((failure instanceof IllegalStateException || failure instanceof IOException) - && maybeFailEngine(operation.operationType().getLowercase(), failure)); + isDocumentFailure = maybeFailEngine(operation.operationType().getLowercase(), failure) == false; } catch (Exception inner) { // we failed checking whether the failure can fail the engine, treat it as a persistent engine failure isDocumentFailure = false; @@ -434,13 +433,15 @@ public class InternalEngine extends Engine { if (isDocumentFailure) { return failure; } else { + // throw original exception in case the exception caused the engine to fail rethrow(failure); return null; } } + // hack to rethrow original exception in case of engine level failures during index/delete operation @SuppressWarnings("unchecked") - static void rethrow(Throwable t) throws T { + private static void rethrow(Throwable t) throws T { throw (T) t; } diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java b/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java index e0114a918ff..36f2765222a 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexingOperationListener.java @@ -38,17 +38,17 @@ public interface IndexingOperationListener { } /** - * Called after the indexing operation occurred. Implementations should - * check {@link Engine.IndexResult#hasFailure()} for operation failures - * and delegate to {@link #postIndex(Engine.Index, Exception)} with - * {@link Engine.IndexResult#getFailure()} if appropriate + * Called after the indexing operation occurred. Note that this is + * also called when indexing a document did not succeed due to document + * related failures. See {@link #postIndex(Engine.Index, Exception)} + * for engine level failures */ default void postIndex(Engine.Index index, Engine.IndexResult result) {} /** - * Called after the indexing operation occurred with exception that - * is not specific to the {@link Engine.Index} i.e. persistent engine - * failures etc. + * Called after the indexing operation occurred with engine level exception. + * See {@link #postIndex(Engine.Index, Engine.IndexResult)} for document + * related failures */ default void postIndex(Engine.Index index, Exception ex) {} @@ -61,17 +61,17 @@ public interface IndexingOperationListener { /** - * Called after the delete operation occurred. Implementations should - * check {@link Engine.DeleteResult#hasFailure()} for operation failures - * and delegate to {@link #postDelete(Engine.Delete, Exception)} with - * {@link Engine.DeleteResult#getFailure()} if appropriate + * Called after the delete operation occurred. Note that this is + * also called when deleting a document did not succeed due to document + * related failures. See {@link #postDelete(Engine.Delete, Exception)} + * for engine level failures */ default void postDelete(Engine.Delete delete, Engine.DeleteResult result) {} /** - * Called after the delete operation occurred with exception that - * is not specific to the {@link Engine.Delete} i.e. persistent engine - * failures etc. + * Called after the delete operation occurred with engine level exception. + * See {@link #postDelete(Engine.Delete, Engine.DeleteResult)} for document + * related failures */ default void postDelete(Engine.Delete delete, Exception ex) {} From 1f1daf59bcd8d3ffbc6612654211c9489a9d9dd5 Mon Sep 17 00:00:00 2001 From: Craig Squire Date: Tue, 1 Nov 2016 14:55:09 -0500 Subject: [PATCH 075/132] Documentation updates for scroll API size parameter (#21229) * Document size parameter for scroll API * Fix size parameter behavior description for scroll --- docs/java-api/search.asciidoc | 6 +----- docs/reference/search/request/scroll.asciidoc | 7 +++++-- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/docs/java-api/search.asciidoc b/docs/java-api/search.asciidoc index f5f20b30a9c..2da24e93c22 100644 --- a/docs/java-api/search.asciidoc +++ b/docs/java-api/search.asciidoc @@ -58,7 +58,7 @@ SearchResponse scrollResp = client.prepareSearch(test) .addSort(FieldSortBuilder.DOC_FIELD_NAME, SortOrder.ASC) .setScroll(new TimeValue(60000)) .setQuery(qb) - .setSize(100).execute().actionGet(); //100 hits per shard will be returned for each scroll + .setSize(100).execute().actionGet(); //max of 100 hits will be returned for each scroll //Scroll until no hits are returned do { for (SearchHit hit : scrollResp.getHits().getHits()) { @@ -68,10 +68,6 @@ do { scrollResp = client.prepareSearchScroll(scrollResp.getScrollId()).setScroll(new TimeValue(60000)).execute().actionGet(); } while(scrollResp.getHits().getHits().length != 0); // Zero hits mark the end of the scroll and the while loop. -------------------------------------------------- -[NOTE] -==== -The size-parameter is per shard, so if you run a query against multiple indices (leading to many shards being involved in the query) the result might be more documents per execution of the scroll than you would expect! -==== [[java-search-msearch]] === MultiSearch API diff --git a/docs/reference/search/request/scroll.asciidoc b/docs/reference/search/request/scroll.asciidoc index bee8e158175..82a27881720 100644 --- a/docs/reference/search/request/scroll.asciidoc +++ b/docs/reference/search/request/scroll.asciidoc @@ -40,6 +40,7 @@ should keep the ``search context'' alive (see <>), eg `?s -------------------------------------------------- POST /twitter/tweet/_search?scroll=1m { + "size": 100, "query": { "match" : { "title" : "elasticsearch" @@ -72,8 +73,10 @@ POST <1> /_search/scroll <2> for another `1m`. <4> The `scroll_id` parameter -Each call to the `scroll` API returns the next batch of results until there -are no more results left to return, ie the `hits` array is empty. +The `size` parameter allows you to configure the maximum number of hits to be +returned with each batch of results. Each call to the `scroll` API returns the +next batch of results until there are no more results left to return, ie the +`hits` array is empty. IMPORTANT: The initial search request and each subsequent scroll request returns a new `_scroll_id` -- only the most recent `_scroll_id` should be From b72a708c0d57c88d82bf1ba6a7ad19b058a56f0a Mon Sep 17 00:00:00 2001 From: Adriel Dean-Hall Date: Tue, 1 Nov 2016 16:14:20 -0400 Subject: [PATCH 076/132] Add docs with up to date instructions on updating default similarity (#21242) * Add docs with up to date instructions on updating default similarity The default similarity can no longer be set in the configuration file (you will get an error on startup). Update the docs with the method that works. * Add instructions for changing similarity on index creation --- .../index-modules/similarity.asciidoc | 36 +++++++++++++++++-- 1 file changed, 34 insertions(+), 2 deletions(-) diff --git a/docs/reference/index-modules/similarity.asciidoc b/docs/reference/index-modules/similarity.asciidoc index 1833b45a9ba..7930ed573b4 100644 --- a/docs/reference/index-modules/similarity.asciidoc +++ b/docs/reference/index-modules/similarity.asciidoc @@ -174,9 +174,41 @@ implementation used for these two methods, while not changing the `default`, it is possible to configure a similarity with the name `base`. This similarity will then be used for the two methods. -You can change the default similarity for all fields by putting the following setting into `elasticsearch.yml`: +You can change the default similarity for all fields in an index when +it is <>: [source,js] -------------------------------------------------- -index.similarity.default.type: classic +PUT /my_index +{ + "settings": { + "index": { + "similarity": { + "default": { + "type": "classic" + } + } + } + } +} +-------------------------------------------------- + +If you want to change the default similarity after creating the index +you must <> your index, send the follwing +request and <> it again afterwards: + +[source,js] +-------------------------------------------------- +PUT /my_index/_settings +{ + "settings": { + "index": { + "similarity": { + "default": { + "type": "classic" + } + } + } + } +} -------------------------------------------------- From ee0b2733d1e67cb5d77af8eaa6ddc7a825c3d9f6 Mon Sep 17 00:00:00 2001 From: Areek Zillur Date: Tue, 1 Nov 2016 16:21:43 -0400 Subject: [PATCH 077/132] add back index and delete engine failure exceptions as deprecated for bwc with 5.x --- .../elasticsearch/ElasticsearchException.java | 6 +- .../engine/DeleteFailedEngineException.java | 37 +++++++++++ .../engine/IndexFailedEngineException.java | 61 +++++++++++++++++++ .../ExceptionSerializationTests.java | 4 +- 4 files changed, 104 insertions(+), 4 deletions(-) create mode 100644 core/src/main/java/org/elasticsearch/index/engine/DeleteFailedEngineException.java create mode 100644 core/src/main/java/org/elasticsearch/index/engine/IndexFailedEngineException.java diff --git a/core/src/main/java/org/elasticsearch/ElasticsearchException.java b/core/src/main/java/org/elasticsearch/ElasticsearchException.java index ea7f6044e85..eb33dbe4b18 100644 --- a/core/src/main/java/org/elasticsearch/ElasticsearchException.java +++ b/core/src/main/java/org/elasticsearch/ElasticsearchException.java @@ -486,7 +486,8 @@ public class ElasticsearchException extends RuntimeException implements ToXConte org.elasticsearch.index.shard.TranslogRecoveryPerformer.BatchOperationException::new, 26), SNAPSHOT_CREATION_EXCEPTION(org.elasticsearch.snapshots.SnapshotCreationException.class, org.elasticsearch.snapshots.SnapshotCreationException::new, 27), - // 28 was DeleteFailedEngineException + DELETE_FAILED_ENGINE_EXCEPTION(org.elasticsearch.index.engine.DeleteFailedEngineException.class, + org.elasticsearch.index.engine.DeleteFailedEngineException::new, 28),// deprecated in 6.0, remove in 7.0 DOCUMENT_MISSING_EXCEPTION(org.elasticsearch.index.engine.DocumentMissingException.class, org.elasticsearch.index.engine.DocumentMissingException::new, 29), SNAPSHOT_EXCEPTION(org.elasticsearch.snapshots.SnapshotException.class, @@ -579,7 +580,8 @@ public class ElasticsearchException extends RuntimeException implements ToXConte org.elasticsearch.action.TimestampParsingException::new, 78), ROUTING_MISSING_EXCEPTION(org.elasticsearch.action.RoutingMissingException.class, org.elasticsearch.action.RoutingMissingException::new, 79), - // 80 used to be for IndexFailedEngineException, removed in 6.0 + INDEX_FAILED_ENGINE_EXCEPTION(org.elasticsearch.index.engine.IndexFailedEngineException.class, + org.elasticsearch.index.engine.IndexFailedEngineException::new, 80), // deprecated in 6.0, remove in 7.0 INDEX_SHARD_RESTORE_FAILED_EXCEPTION(org.elasticsearch.index.snapshots.IndexShardRestoreFailedException.class, org.elasticsearch.index.snapshots.IndexShardRestoreFailedException::new, 81), REPOSITORY_EXCEPTION(org.elasticsearch.repositories.RepositoryException.class, diff --git a/core/src/main/java/org/elasticsearch/index/engine/DeleteFailedEngineException.java b/core/src/main/java/org/elasticsearch/index/engine/DeleteFailedEngineException.java new file mode 100644 index 00000000000..8cad7823cb4 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/index/engine/DeleteFailedEngineException.java @@ -0,0 +1,37 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.engine; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.index.shard.ShardId; + +import java.io.IOException; + +/** + * Deprecated as not used in 6.0, should be removed in 7.0 + * Still exists for bwc in serializing/deserializing from + * 5.x nodes + */ +@Deprecated +public class DeleteFailedEngineException extends EngineException { + public DeleteFailedEngineException(StreamInput in) throws IOException{ + super(in); + } +} \ No newline at end of file diff --git a/core/src/main/java/org/elasticsearch/index/engine/IndexFailedEngineException.java b/core/src/main/java/org/elasticsearch/index/engine/IndexFailedEngineException.java new file mode 100644 index 00000000000..cd5e8a47406 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/index/engine/IndexFailedEngineException.java @@ -0,0 +1,61 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.engine; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.index.shard.ShardId; + +import java.io.IOException; +import java.util.Objects; + +/** + * Deprecated as not used in 6.0, should be removed in 7.0 + * Still exists for bwc in serializing/deserializing from + * 5.x nodes + */ +@Deprecated +public class IndexFailedEngineException extends EngineException { + + private final String type; + + private final String id; + + public IndexFailedEngineException(StreamInput in) throws IOException{ + super(in); + type = in.readString(); + id = in.readString(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(type); + out.writeString(id); + } + + public String type() { + return this.type; + } + + public String id() { + return this.id; + } +} \ No newline at end of file diff --git a/core/src/test/java/org/elasticsearch/ExceptionSerializationTests.java b/core/src/test/java/org/elasticsearch/ExceptionSerializationTests.java index d657c32fdf4..8a2e965a7b4 100644 --- a/core/src/test/java/org/elasticsearch/ExceptionSerializationTests.java +++ b/core/src/test/java/org/elasticsearch/ExceptionSerializationTests.java @@ -664,7 +664,7 @@ public class ExceptionSerializationTests extends ESTestCase { ids.put(25, org.elasticsearch.script.GeneralScriptException.class); ids.put(26, org.elasticsearch.index.shard.TranslogRecoveryPerformer.BatchOperationException.class); ids.put(27, org.elasticsearch.snapshots.SnapshotCreationException.class); - ids.put(28, null); // was DeleteFailedEngineException + ids.put(28, org.elasticsearch.index.engine.DeleteFailedEngineException.class); //deprecated in 6.0 ids.put(29, org.elasticsearch.index.engine.DocumentMissingException.class); ids.put(30, org.elasticsearch.snapshots.SnapshotException.class); ids.put(31, org.elasticsearch.indices.InvalidAliasNameException.class); @@ -716,7 +716,7 @@ public class ExceptionSerializationTests extends ESTestCase { ids.put(77, org.elasticsearch.common.util.concurrent.UncategorizedExecutionException.class); ids.put(78, org.elasticsearch.action.TimestampParsingException.class); ids.put(79, org.elasticsearch.action.RoutingMissingException.class); - ids.put(80, null); // was IndexFailedEngineException, removed in 6.0 + ids.put(80, org.elasticsearch.index.engine.IndexFailedEngineException.class); //deprecated in 6.0 ids.put(81, org.elasticsearch.index.snapshots.IndexShardRestoreFailedException.class); ids.put(82, org.elasticsearch.repositories.RepositoryException.class); ids.put(83, org.elasticsearch.transport.ReceiveTimeoutTransportException.class); From cf6d3f7ec1b732f415d6c2ed3313feb1b8a8168b Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 1 Nov 2016 16:27:46 -0400 Subject: [PATCH 078/132] Fix get-bwc-version for 5.0.0 (#21249) 5.0.0 is at artifacts.elastic.co. --- dev-tools/get-bwc-version.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/dev-tools/get-bwc-version.py b/dev-tools/get-bwc-version.py index 54c559d1dc8..4ef9736ea06 100644 --- a/dev-tools/get-bwc-version.py +++ b/dev-tools/get-bwc-version.py @@ -9,7 +9,7 @@ # 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 +# 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. @@ -38,7 +38,7 @@ def parse_config(): def main(): c = parse_config() - + if not os.path.exists(c.path): print('Creating %s' % c.path) os.mkdir(c.path) @@ -53,7 +53,7 @@ def main(): shutil.rmtree(version_dir) else: print('Version %s exists at %s' % (c.version, version_dir)) - return + return # before 1.4.0, the zip file contains windows scripts, and tar.gz contained *nix scripts if is_windows: @@ -67,14 +67,14 @@ def main(): elif c.version.startswith('0.') or c.version.startswith('1.'): url = 'https://download.elasticsearch.org/elasticsearch/elasticsearch/%s' % filename else: - url = 'http://download.elasticsearch.org/elasticsearch/release/org/elasticsearch/distribution/tar/elasticsearch/%s/%s' % (c.version, filename) + url = 'https://artifacts.elastic.co/downloads/elasticsearch/%s' % filename print('Downloading %s' % url) urllib.request.urlretrieve(url, filename) print('Extracting to %s' % version_dir) if is_windows: archive = zipfile.ZipFile(filename) - archive.extractall() + archive.extractall() else: # for some reason python's tarfile module has trouble with ES tgz? subprocess.check_call('tar -xzf %s' % filename, shell=True) From f86d784bbc54f8c364534a96f2a0a3dc2959dade Mon Sep 17 00:00:00 2001 From: Alexander Lin Date: Tue, 1 Nov 2016 14:52:09 -0700 Subject: [PATCH 079/132] Add extra note about Java8 requirement to sub-install docs (#21190) Closes #20005 --- docs/reference/setup/install/deb.asciidoc | 4 ++++ docs/reference/setup/install/rpm.asciidoc | 4 ++++ docs/reference/setup/install/windows.asciidoc | 4 ++++ docs/reference/setup/install/zip-targz.asciidoc | 6 +++++- 4 files changed, 17 insertions(+), 1 deletion(-) diff --git a/docs/reference/setup/install/deb.asciidoc b/docs/reference/setup/install/deb.asciidoc index f2e10176480..e3fc4afe0e1 100644 --- a/docs/reference/setup/install/deb.asciidoc +++ b/docs/reference/setup/install/deb.asciidoc @@ -9,6 +9,10 @@ The latest stable version of Elasticsearch can be found on the link:/downloads/elasticsearch[Download Elasticsearch] page. Other versions can be found on the link:/downloads/past-releases[Past Releases page]. +NOTE: Elasticsearch requires Java 8 or later. Use the +http://www.oracle.com/technetwork/java/javase/downloads/index.html[official Oracle distribution] +or an open-source distribution such as http://openjdk.java.net[OpenJDK]. + [[deb-key]] ==== Import the Elasticsearch PGP Key diff --git a/docs/reference/setup/install/rpm.asciidoc b/docs/reference/setup/install/rpm.asciidoc index d6c9352aac0..dda9256c2a8 100644 --- a/docs/reference/setup/install/rpm.asciidoc +++ b/docs/reference/setup/install/rpm.asciidoc @@ -13,6 +13,10 @@ The latest stable version of Elasticsearch can be found on the link:/downloads/elasticsearch[Download Elasticsearch] page. Other versions can be found on the link:/downloads/past-releases[Past Releases page]. +NOTE: Elasticsearch requires Java 8 or later. Use the +http://www.oracle.com/technetwork/java/javase/downloads/index.html[official Oracle distribution] +or an open-source distribution such as http://openjdk.java.net[OpenJDK]. + [[rpm-key]] ==== Import the Elasticsearch PGP Key diff --git a/docs/reference/setup/install/windows.asciidoc b/docs/reference/setup/install/windows.asciidoc index 320626a2021..691794e42b5 100644 --- a/docs/reference/setup/install/windows.asciidoc +++ b/docs/reference/setup/install/windows.asciidoc @@ -10,6 +10,10 @@ link:/downloads/elasticsearch[Download Elasticsearch] page. Other versions can be found on the link:/downloads/past-releases[Past Releases page]. +NOTE: Elasticsearch requires Java 8 or later. Use the +http://www.oracle.com/technetwork/java/javase/downloads/index.html[official Oracle distribution] +or an open-source distribution such as http://openjdk.java.net[OpenJDK]. + [[install-windows]] ==== Download and install the `.zip` package diff --git a/docs/reference/setup/install/zip-targz.asciidoc b/docs/reference/setup/install/zip-targz.asciidoc index fc47214615f..9330af43b9e 100644 --- a/docs/reference/setup/install/zip-targz.asciidoc +++ b/docs/reference/setup/install/zip-targz.asciidoc @@ -10,6 +10,10 @@ link:/downloads/elasticsearch[Download Elasticsearch] page. Other versions can be found on the link:/downloads/past-releases[Past Releases page]. +NOTE: Elasticsearch requires Java 8 or later. Use the +http://www.oracle.com/technetwork/java/javase/downloads/index.html[official Oracle distribution] +or an open-source distribution such as http://openjdk.java.net[OpenJDK]. + [[install-zip]] ==== Download and install the `.zip` package @@ -185,4 +189,4 @@ directory so that you do not delete important data later on. |======================================================================= -include::next-steps.asciidoc[] \ No newline at end of file +include::next-steps.asciidoc[] From 930316561502c93ee0ef0314b4a78c79f1522a2c Mon Sep 17 00:00:00 2001 From: Ali Beyad Date: Tue, 1 Nov 2016 21:29:37 -0400 Subject: [PATCH 080/132] Balance step in BalancedShardsAllocator for a single shard (#21103) This commit introduces a single-shard balance step for deciding on rebalancing a single shard (without taking any other shards in the cluster into account). This method will be used by the cluster allocation explain API to explain in detail the decision process for finding a more optimal location for a started shard, if one exists. --- .../allocator/BalancedShardsAllocator.java | 288 ++++++++++++++++-- .../routing/allocation/decider/Decision.java | 2 +- .../decider/EnableAllocationDecider.java | 44 ++- .../ClusterStateCreationUtils.java | 85 ++++++ .../allocation/BalancedSingleShardTests.java | 276 +++++++++++++++++ 5 files changed, 671 insertions(+), 24 deletions(-) create mode 100644 core/src/test/java/org/elasticsearch/cluster/routing/allocation/BalancedSingleShardTests.java diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java index 93b9c90e490..3a6c1c45f01 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java @@ -126,6 +126,18 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards balancer.balance(); } + /** + * Returns a decision on rebalancing a single shard to form a more optimal cluster balance. This + * method is not used in itself for cluster rebalancing because all shards from all indices are + * taken into account when making rebalancing decisions. This method is only intended to be used + * from the cluster allocation explain API to explain possible rebalancing decisions for a single + * shard. + */ + public RebalanceDecision decideRebalance(final ShardRouting shard, final RoutingAllocation allocation) { + assert allocation.debugDecision() : "debugDecision should be set in explain mode"; + return new Balancer(logger, allocation, weightFunction, threshold).decideRebalance(shard); + } + /** * Returns the currently configured delta threshold */ @@ -267,11 +279,18 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards return new NodeSorter(nodesArray(), weight, this); } + /** + * The absolute value difference between two weights. + */ private static float absDelta(float lower, float higher) { assert higher >= lower : higher + " lt " + lower +" but was expected to be gte"; return Math.abs(higher - lower); } + /** + * Returns {@code true} iff the weight delta between two nodes is under a defined threshold. + * See {@link #THRESHOLD_SETTING} for defining the threshold. + */ private static boolean lessThan(float delta, float threshold) { /* deltas close to the threshold are "rounded" to the threshold manually to prevent floating point problems if the delta is very close to the @@ -309,6 +328,110 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards balanceByWeights(); } + /** + * Makes a decision about moving a single shard to a different node to form a more + * optimally balanced cluster. This method is invoked from the cluster allocation + * explain API only. + */ + private RebalanceDecision decideRebalance(final ShardRouting shard) { + if (shard.started() == false) { + // cannot rebalance a shard that isn't started + return RebalanceDecision.NOT_TAKEN; + } + + Decision canRebalance = allocation.deciders().canRebalance(shard, allocation); + + if (allocation.hasPendingAsyncFetch()) { + return new RebalanceDecision( + canRebalance, + Type.NO, + "cannot rebalance due to in-flight shard store fetches, otherwise allocation may prematurely rebalance a shard to " + + "a node that is soon to receive another shard assignment upon completion of the shard store fetch, " + + "rendering the cluster imbalanced again" + ); + } + + sorter.reset(shard.getIndexName()); + ModelNode[] modelNodes = sorter.modelNodes; + final String currentNodeId = shard.currentNodeId(); + // find currently assigned node + ModelNode currentNode = null; + for (ModelNode node : modelNodes) { + if (node.getNodeId().equals(currentNodeId)) { + currentNode = node; + break; + } + } + assert currentNode != null : "currently assigned node could not be found"; + + // balance the shard, if a better node can be found + final float currentWeight = sorter.weight(currentNode); + final AllocationDeciders deciders = allocation.deciders(); + final String idxName = shard.getIndexName(); + Map nodeDecisions = new HashMap<>(modelNodes.length - 1); + Type rebalanceDecisionType = Type.NO; + String assignedNodeId = null; + for (ModelNode node : modelNodes) { + if (node == currentNode) { + continue; // skip over node we're currently allocated to it + } + final Decision canAllocate = deciders.canAllocate(shard, node.getRoutingNode(), allocation); + // the current weight of the node in the cluster, as computed by the weight function; + // this is a comparison of the number of shards on this node to the number of shards + // that should be on each node on average (both taking the cluster as a whole into account + // as well as shards per index) + final float nodeWeight = sorter.weight(node); + // if the node we are examining has a worse (higher) weight than the node the shard is + // assigned to, then there is no way moving the shard to the node with the worse weight + // can make the balance of the cluster better, so we check for that here + final boolean betterWeightThanCurrent = nodeWeight <= currentWeight; + boolean rebalanceConditionsMet = false; + boolean deltaAboveThreshold = false; + float weightWithShardAdded = Float.POSITIVE_INFINITY; + if (betterWeightThanCurrent) { + // get the delta between the weights of the node we are checking and the node that holds the shard + final float currentDelta = absDelta(nodeWeight, currentWeight); + // checks if the weight delta is above a certain threshold; if it is not above a certain threshold, + // then even though the node we are examining has a better weight and may make the cluster balance + // more even, it doesn't make sense to execute the heavyweight operation of relocating a shard unless + // the gains make it worth it, as defined by the threshold + deltaAboveThreshold = lessThan(currentDelta, threshold) == false; + // simulate the weight of the node if we were to relocate the shard to it + weightWithShardAdded = weight.weightShardAdded(this, node, idxName); + // calculate the delta of the weights of the two nodes if we were to add the shard to the + // node in question and move it away from the node that currently holds it. + final float proposedDelta = weightWithShardAdded - weight.weightShardRemoved(this, currentNode, idxName); + rebalanceConditionsMet = deltaAboveThreshold && proposedDelta < currentDelta; + // if the simulated weight delta with the shard moved away is better than the weight delta + // with the shard remaining on the current node, and we are allowed to allocate to the + // node in question, then allow the rebalance + if (rebalanceConditionsMet && canAllocate.type().higherThan(rebalanceDecisionType)) { + // rebalance to the node, only will get overwritten if the decision here is to + // THROTTLE and we get a decision with YES on another node + rebalanceDecisionType = canAllocate.type(); + assignedNodeId = node.getNodeId(); + } + } + nodeDecisions.put(node.getNodeId(), new NodeRebalanceDecision( + rebalanceConditionsMet ? canAllocate.type() : Type.NO, + canAllocate, + betterWeightThanCurrent, + deltaAboveThreshold, + nodeWeight, + weightWithShardAdded) + ); + } + + + if (canRebalance.type() != Type.YES) { + return new RebalanceDecision(canRebalance, canRebalance.type(), "rebalancing is not allowed", null, + nodeDecisions, currentWeight); + } else { + return RebalanceDecision.decision(canRebalance, rebalanceDecisionType, assignedNodeId, + nodeDecisions, currentWeight, threshold); + } + } + public Map weighShard(ShardRouting shard) { final ModelNode[] modelNodes = sorter.modelNodes; final float[] weights = sorter.weights; @@ -539,7 +662,7 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards public MoveDecision makeMoveDecision(final ShardRouting shardRouting) { if (shardRouting.started() == false) { // we can only move started shards - return MoveDecision.DECISION_NOT_TAKEN; + return MoveDecision.NOT_TAKEN; } final boolean explain = allocation.debugDecision(); @@ -1110,15 +1233,11 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards private final String finalExplanation; @Nullable private final String assignedNodeId; - @Nullable - private final Map nodeDecisions; - protected RelocationDecision(Type finalDecision, String finalExplanation, String assignedNodeId, - Map nodeDecisions) { + protected RelocationDecision(Type finalDecision, String finalExplanation, String assignedNodeId) { this.finalDecision = finalDecision; this.finalExplanation = finalExplanation; this.assignedNodeId = assignedNodeId; - this.nodeDecisions = nodeDecisions; } /** @@ -1153,15 +1272,6 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards public String getAssignedNodeId() { return assignedNodeId; } - - /** - * Gets the individual node-level decisions that went into making the final decision as represented by - * {@link #getFinalDecisionType()}. The map that is returned has the node id as the key and a {@link WeightedDecision}. - */ - @Nullable - public Map getNodeDecisions() { - return nodeDecisions; - } } /** @@ -1169,18 +1279,21 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards */ public static final class MoveDecision extends RelocationDecision { /** a constant representing no decision taken */ - public static final MoveDecision DECISION_NOT_TAKEN = new MoveDecision(null, null, null, null, null); + public static final MoveDecision NOT_TAKEN = new MoveDecision(null, null, null, null, null); /** cached decisions so we don't have to recreate objects for common decisions when not in explain mode. */ private static final MoveDecision CACHED_STAY_DECISION = new MoveDecision(Decision.YES, Type.NO, null, null, null); private static final MoveDecision CACHED_CANNOT_MOVE_DECISION = new MoveDecision(Decision.NO, Type.NO, null, null, null); @Nullable private final Decision canRemainDecision; + @Nullable + private final Map nodeDecisions; private MoveDecision(Decision canRemainDecision, Type finalDecision, String finalExplanation, String assignedNodeId, Map nodeDecisions) { - super(finalDecision, finalExplanation, assignedNodeId, nodeDecisions); + super(finalDecision, finalExplanation, assignedNodeId); this.canRemainDecision = canRemainDecision; + this.nodeDecisions = nodeDecisions != null ? Collections.unmodifiableMap(nodeDecisions) : null; } /** @@ -1250,6 +1363,147 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards public boolean cannotRemain() { return isDecisionTaken() && canRemainDecision.type() == Type.NO; } + + /** + * Gets the individual node-level decisions that went into making the final decision as represented by + * {@link #getFinalDecisionType()}. The map that is returned has the node id as the key and a {@link WeightedDecision}. + */ + @Nullable + public Map getNodeDecisions() { + return nodeDecisions; + } + } + + /** + * Represents a decision to move a started shard to form a more optimally balanced cluster. + */ + public static final class RebalanceDecision extends RelocationDecision { + /** a constant representing no decision taken */ + public static final RebalanceDecision NOT_TAKEN = new RebalanceDecision(null, null, null, null, null, Float.POSITIVE_INFINITY); + + @Nullable + private final Decision canRebalanceDecision; + @Nullable + private final Map nodeDecisions; + private float currentWeight; + + protected RebalanceDecision(Decision canRebalanceDecision, Type finalDecision, String finalExplanation) { + this(canRebalanceDecision, finalDecision, finalExplanation, null, null, Float.POSITIVE_INFINITY); + } + + protected RebalanceDecision(Decision canRebalanceDecision, Type finalDecision, String finalExplanation, + String assignedNodeId, Map nodeDecisions, float currentWeight) { + super(finalDecision, finalExplanation, assignedNodeId); + this.canRebalanceDecision = canRebalanceDecision; + this.nodeDecisions = nodeDecisions != null ? Collections.unmodifiableMap(nodeDecisions) : null; + this.currentWeight = currentWeight; + } + + /** + * Creates a new {@link RebalanceDecision}, computing the explanation based on the decision parameters. + */ + public static RebalanceDecision decision(Decision canRebalanceDecision, Type finalDecision, String assignedNodeId, + Map nodeDecisions, float currentWeight, float threshold) { + final String explanation = produceFinalExplanation(finalDecision, assignedNodeId, threshold); + return new RebalanceDecision(canRebalanceDecision, finalDecision, explanation, assignedNodeId, nodeDecisions, currentWeight); + } + + /** + * Returns the decision for being allowed to rebalance the shard. + */ + @Nullable + public Decision getCanRebalanceDecision() { + return canRebalanceDecision; + } + + /** + * Gets the individual node-level decisions that went into making the final decision as represented by + * {@link #getFinalDecisionType()}. The map that is returned has the node id as the key and a {@link NodeRebalanceDecision}. + */ + @Nullable + public Map getNodeDecisions() { + return nodeDecisions; + } + + private static String produceFinalExplanation(final Type finalDecisionType, final String assignedNodeId, final float threshold) { + final String finalExplanation; + if (assignedNodeId != null) { + if (finalDecisionType == Type.THROTTLE) { + finalExplanation = "throttle moving shard to node [" + assignedNodeId + "], as it is " + + "currently busy with other shard relocations"; + } else { + finalExplanation = "moving shard to node [" + assignedNodeId + "] to form a more balanced cluster"; + } + } else { + finalExplanation = "cannot rebalance shard, no other node exists that would form a more balanced " + + "cluster within the defined threshold [" + threshold + "]"; + } + return finalExplanation; + } + } + + /** + * A node-level explanation for the decision to rebalance a shard. + */ + public static final class NodeRebalanceDecision { + private final Type nodeDecisionType; + private final Decision canAllocate; + private final boolean betterWeightThanCurrent; + private final boolean deltaAboveThreshold; + private final float currentWeight; + private final float weightWithShardAdded; + + NodeRebalanceDecision(Type nodeDecisionType, Decision canAllocate, boolean betterWeightThanCurrent, + boolean deltaAboveThreshold, float currentWeight, float weightWithShardAdded) { + this.nodeDecisionType = Objects.requireNonNull(nodeDecisionType); + this.canAllocate = Objects.requireNonNull(canAllocate); + this.betterWeightThanCurrent = betterWeightThanCurrent; + this.deltaAboveThreshold = deltaAboveThreshold; + this.currentWeight = currentWeight; + this.weightWithShardAdded = weightWithShardAdded; + } + + /** + * Returns the decision to rebalance to the node. + */ + public Type getNodeDecisionType() { + return nodeDecisionType; + } + + /** + * Returns whether the shard is allowed to be allocated to the node. + */ + public Decision getCanAllocateDecision() { + return canAllocate; + } + + /** + * Returns whether the weight of the node is better than the weight of the node where the shard currently resides. + */ + public boolean isBetterWeightThanCurrent() { + return betterWeightThanCurrent; + } + + /** + * Returns if the weight delta by assigning to this node was above the threshold to warrant a rebalance. + */ + public boolean isDeltaAboveThreshold() { + return deltaAboveThreshold; + } + + /** + * Returns the current weight of the node if the shard is not added to the node. + */ + public float getCurrentWeight() { + return currentWeight; + } + + /** + * Returns the weight of the node if the shard is added to the node. + */ + public float getWeightWithShardAdded() { + return weightWithShardAdded; + } } } diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/Decision.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/Decision.java index e6a3eba7437..11db0980f47 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/Decision.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/Decision.java @@ -222,7 +222,7 @@ public abstract class Decision implements ToXContent { } /** - * Returns the explanation string, fully formatted. Only formats the string once + * Returns the explanation string, fully formatted. Only formats the string once. */ @Nullable public String getExplanation() { diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/EnableAllocationDecider.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/EnableAllocationDecider.java index 64bf5942142..1a38e3742fc 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/EnableAllocationDecider.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/EnableAllocationDecider.java @@ -103,28 +103,33 @@ public class EnableAllocationDecider extends AllocationDecider { final IndexMetaData indexMetaData = allocation.metaData().getIndexSafe(shardRouting.index()); final Allocation enable; + final boolean usedIndexSetting; if (INDEX_ROUTING_ALLOCATION_ENABLE_SETTING.exists(indexMetaData.getSettings())) { enable = INDEX_ROUTING_ALLOCATION_ENABLE_SETTING.get(indexMetaData.getSettings()); + usedIndexSetting = true; } else { enable = this.enableAllocation; + usedIndexSetting = false; } switch (enable) { case ALL: return allocation.decision(Decision.YES, NAME, "all allocations are allowed"); case NONE: - return allocation.decision(Decision.NO, NAME, "no allocations are allowed"); + return allocation.decision(Decision.NO, NAME, "no allocations are allowed due to {}", setting(enable, usedIndexSetting)); case NEW_PRIMARIES: if (shardRouting.primary() && shardRouting.active() == false && shardRouting.recoverySource().getType() != RecoverySource.Type.EXISTING_STORE) { return allocation.decision(Decision.YES, NAME, "new primary allocations are allowed"); } else { - return allocation.decision(Decision.NO, NAME, "non-new primary allocations are forbidden"); + return allocation.decision(Decision.NO, NAME, "non-new primary allocations are forbidden due to {}", + setting(enable, usedIndexSetting)); } case PRIMARIES: if (shardRouting.primary()) { return allocation.decision(Decision.YES, NAME, "primary allocations are allowed"); } else { - return allocation.decision(Decision.NO, NAME, "replica allocations are forbidden"); + return allocation.decision(Decision.NO, NAME, "replica allocations are forbidden due to {}", + setting(enable, usedIndexSetting)); } default: throw new IllegalStateException("Unknown allocation option"); @@ -139,33 +144,60 @@ public class EnableAllocationDecider extends AllocationDecider { Settings indexSettings = allocation.metaData().getIndexSafe(shardRouting.index()).getSettings(); final Rebalance enable; + final boolean usedIndexSetting; if (INDEX_ROUTING_REBALANCE_ENABLE_SETTING.exists(indexSettings)) { enable = INDEX_ROUTING_REBALANCE_ENABLE_SETTING.get(indexSettings); + usedIndexSetting = true; } else { enable = this.enableRebalance; + usedIndexSetting = false; } switch (enable) { case ALL: return allocation.decision(Decision.YES, NAME, "all rebalancing is allowed"); case NONE: - return allocation.decision(Decision.NO, NAME, "no rebalancing is allowed"); + return allocation.decision(Decision.NO, NAME, "no rebalancing is allowed due to {}", setting(enable, usedIndexSetting)); case PRIMARIES: if (shardRouting.primary()) { return allocation.decision(Decision.YES, NAME, "primary rebalancing is allowed"); } else { - return allocation.decision(Decision.NO, NAME, "replica rebalancing is forbidden"); + return allocation.decision(Decision.NO, NAME, "replica rebalancing is forbidden due to {}", + setting(enable, usedIndexSetting)); } case REPLICAS: if (shardRouting.primary() == false) { return allocation.decision(Decision.YES, NAME, "replica rebalancing is allowed"); } else { - return allocation.decision(Decision.NO, NAME, "primary rebalancing is forbidden"); + return allocation.decision(Decision.NO, NAME, "primary rebalancing is forbidden due to {}", + setting(enable, usedIndexSetting)); } default: throw new IllegalStateException("Unknown rebalance option"); } } + private static String setting(Allocation allocation, boolean usedIndexSetting) { + StringBuilder buf = new StringBuilder("["); + if (usedIndexSetting) { + buf.append(INDEX_ROUTING_ALLOCATION_ENABLE_SETTING.getKey()); + } else { + buf.append(CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.getKey()); + } + buf.append("=").append(allocation.toString().toLowerCase(Locale.ROOT)).append("]"); + return buf.toString(); + } + + private static String setting(Rebalance rebalance, boolean usedIndexSetting) { + StringBuilder buf = new StringBuilder("["); + if (usedIndexSetting) { + buf.append(INDEX_ROUTING_REBALANCE_ENABLE_SETTING.getKey()); + } else { + buf.append(CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey()); + } + buf.append("=").append(rebalance.toString().toLowerCase(Locale.ROOT)).append("]"); + return buf.toString(); + } + /** * Allocation values or rather their string representation to be used used with * {@link EnableAllocationDecider#CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING} / diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/ClusterStateCreationUtils.java b/core/src/test/java/org/elasticsearch/action/support/replication/ClusterStateCreationUtils.java index ab419a3c698..55485b590cf 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/ClusterStateCreationUtils.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/ClusterStateCreationUtils.java @@ -144,6 +144,91 @@ public class ClusterStateCreationUtils { return state.build(); } + /** + * Creates cluster state with an index that has #(numberOfPrimaries) primary shards in the started state and no replicas. + * The cluster state contains #(numberOfNodes) nodes and assigns primaries to those nodes. + */ + public static ClusterState state(String index, final int numberOfNodes, final int numberOfPrimaries) { + DiscoveryNodes.Builder discoBuilder = DiscoveryNodes.builder(); + Set nodes = new HashSet<>(); + for (int i = 0; i < numberOfNodes; i++) { + final DiscoveryNode node = newNode(i); + discoBuilder = discoBuilder.add(node); + nodes.add(node.getId()); + } + discoBuilder.localNodeId(newNode(0).getId()); + discoBuilder.masterNodeId(randomFrom(nodes)); + IndexMetaData indexMetaData = IndexMetaData.builder(index).settings(Settings.builder() + .put(SETTING_VERSION_CREATED, Version.CURRENT) + .put(SETTING_NUMBER_OF_SHARDS, numberOfPrimaries).put(SETTING_NUMBER_OF_REPLICAS, 0) + .put(SETTING_CREATION_DATE, System.currentTimeMillis())).build(); + + RoutingTable.Builder routing = new RoutingTable.Builder(); + routing.addAsNew(indexMetaData); + + IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder(indexMetaData.getIndex()); + for (int i = 0; i < numberOfPrimaries; i++) { + ShardId shardId = new ShardId(indexMetaData.getIndex(), i); + IndexShardRoutingTable.Builder indexShardRoutingBuilder = new IndexShardRoutingTable.Builder(shardId); + indexShardRoutingBuilder.addShard( + TestShardRouting.newShardRouting(shardId, randomFrom(nodes), true, ShardRoutingState.STARTED)); + indexRoutingTable.addIndexShard(indexShardRoutingBuilder.build()); + } + + ClusterState.Builder state = ClusterState.builder(new ClusterName("test")); + state.nodes(discoBuilder); + state.metaData(MetaData.builder().put(indexMetaData, false).generateClusterUuidIfNeeded()); + state.routingTable(RoutingTable.builder().add(indexRoutingTable).build()); + return state.build(); + } + + + + /** + * Creates cluster state with the given indices, each index containing #(numberOfPrimaries) + * started primary shards and no replicas. The cluster state contains #(numberOfNodes) nodes + * and assigns primaries to those nodes. + */ + public static ClusterState state(final int numberOfNodes, final String[] indices, final int numberOfPrimaries) { + DiscoveryNodes.Builder discoBuilder = DiscoveryNodes.builder(); + Set nodes = new HashSet<>(); + for (int i = 0; i < numberOfNodes; i++) { + final DiscoveryNode node = newNode(i); + discoBuilder = discoBuilder.add(node); + nodes.add(node.getId()); + } + discoBuilder.localNodeId(newNode(0).getId()); + discoBuilder.masterNodeId(newNode(0).getId()); + MetaData.Builder metaData = MetaData.builder(); + RoutingTable.Builder routingTable = RoutingTable.builder(); + for (String index : indices) { + IndexMetaData indexMetaData = IndexMetaData.builder(index).settings(Settings.builder() + .put(SETTING_VERSION_CREATED, Version.CURRENT) + .put(SETTING_NUMBER_OF_SHARDS, numberOfPrimaries).put(SETTING_NUMBER_OF_REPLICAS, 0) + .put(SETTING_CREATION_DATE, System.currentTimeMillis())).build(); + + RoutingTable.Builder routing = new RoutingTable.Builder(); + routing.addAsNew(indexMetaData); + + IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder(indexMetaData.getIndex()); + for (int i = 0; i < numberOfPrimaries; i++) { + ShardId shardId = new ShardId(indexMetaData.getIndex(), i); + IndexShardRoutingTable.Builder indexShardRoutingBuilder = new IndexShardRoutingTable.Builder(shardId); + indexShardRoutingBuilder.addShard( + TestShardRouting.newShardRouting(shardId, randomFrom(nodes), true, ShardRoutingState.STARTED)); + indexRoutingTable.addIndexShard(indexShardRoutingBuilder.build()); + } + + metaData.put(indexMetaData, false); + routingTable.add(indexRoutingTable); + } + ClusterState.Builder state = ClusterState.builder(new ClusterName("test")); + state.nodes(discoBuilder); + state.metaData(metaData.generateClusterUuidIfNeeded().build()); + state.routingTable(routingTable.build()); + return state.build(); + } + /** * Creates cluster state with several shards and one replica and all shards STARTED. */ diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/BalancedSingleShardTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/BalancedSingleShardTests.java new file mode 100644 index 00000000000..806e136bba3 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/BalancedSingleShardTests.java @@ -0,0 +1,276 @@ +/* + * 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.cluster.routing.allocation; + +import org.elasticsearch.action.support.replication.ClusterStateCreationUtils; +import org.elasticsearch.cluster.ClusterInfo; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ESAllocationTestCase; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.cluster.routing.RoutingNode; +import org.elasticsearch.cluster.routing.RoutingNodes; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.ShardRoutingState; +import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator; +import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator.Balancer; +import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator.NodeRebalanceDecision; +import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator.RebalanceDecision; +import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider; +import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; +import org.elasticsearch.cluster.routing.allocation.decider.Decision; +import org.elasticsearch.cluster.routing.allocation.decider.Decision.Type; +import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.settings.Settings; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import static org.hamcrest.Matchers.startsWith; + +/** + * Tests for balancing a single shard, see {@link Balancer#decideRebalance(ShardRouting)}. + */ +public class BalancedSingleShardTests extends ESAllocationTestCase { + + public void testRebalanceNonStartedShardNotAllowed() { + BalancedShardsAllocator allocator = new BalancedShardsAllocator(Settings.EMPTY); + ClusterState clusterState = ClusterStateCreationUtils.state("idx", randomBoolean(), + randomFrom(ShardRoutingState.INITIALIZING, ShardRoutingState.UNASSIGNED, ShardRoutingState.RELOCATING)); + ShardRouting shard = clusterState.routingTable().index("idx").shard(0).primaryShard(); + RebalanceDecision rebalanceDecision = allocator.decideRebalance(shard, newRoutingAllocation( + new AllocationDeciders(Settings.EMPTY, Collections.emptyList()), clusterState)); + assertSame(RebalanceDecision.NOT_TAKEN, rebalanceDecision); + } + + public void testRebalanceNotAllowedDuringPendingAsyncFetch() { + BalancedShardsAllocator allocator = new BalancedShardsAllocator(Settings.EMPTY); + ClusterState clusterState = ClusterStateCreationUtils.state("idx", randomBoolean(), ShardRoutingState.STARTED); + ShardRouting shard = clusterState.routingTable().index("idx").shard(0).primaryShard(); + RoutingAllocation routingAllocation = newRoutingAllocation( + new AllocationDeciders(Settings.EMPTY, Collections.emptyList()), clusterState); + routingAllocation.setHasPendingAsyncFetch(); + RebalanceDecision rebalanceDecision = allocator.decideRebalance(shard, routingAllocation); + assertNotNull(rebalanceDecision.getCanRebalanceDecision()); + assertEquals(Type.NO, rebalanceDecision.getFinalDecisionType()); + assertThat(rebalanceDecision.getFinalExplanation(), startsWith("cannot rebalance due to in-flight shard store fetches")); + assertNull(rebalanceDecision.getNodeDecisions()); + assertNull(rebalanceDecision.getAssignedNodeId()); + + assertAssignedNodeRemainsSame(allocator, routingAllocation, shard); + } + + public void testRebalancingNotAllowedDueToCanRebalance() { + final Decision canRebalanceDecision = randomFrom(Decision.NO, Decision.THROTTLE); + AllocationDecider noRebalanceDecider = new AllocationDecider(Settings.EMPTY) { + @Override + public Decision canRebalance(ShardRouting shardRouting, RoutingAllocation allocation) { + return allocation.decision(canRebalanceDecision, "TEST", "foobar"); + } + }; + BalancedShardsAllocator allocator = new BalancedShardsAllocator(Settings.EMPTY); + ClusterState clusterState = ClusterStateCreationUtils.state("idx", randomBoolean(), ShardRoutingState.STARTED); + ShardRouting shard = clusterState.routingTable().index("idx").shard(0).primaryShard(); + RoutingAllocation routingAllocation = newRoutingAllocation( + new AllocationDeciders(Settings.EMPTY, Collections.singleton(noRebalanceDecider)), clusterState); + RebalanceDecision rebalanceDecision = allocator.decideRebalance(shard, routingAllocation); + assertEquals(canRebalanceDecision.type(), rebalanceDecision.getCanRebalanceDecision().type()); + assertEquals(canRebalanceDecision.type(), rebalanceDecision.getFinalDecisionType()); + assertEquals("rebalancing is not allowed", rebalanceDecision.getFinalExplanation()); + assertNotNull(rebalanceDecision.getNodeDecisions()); + assertNull(rebalanceDecision.getAssignedNodeId()); + assertEquals(1, rebalanceDecision.getCanRebalanceDecision().getDecisions().size()); + for (Decision subDecision : rebalanceDecision.getCanRebalanceDecision().getDecisions()) { + assertEquals("foobar", ((Decision.Single) subDecision).getExplanation()); + } + + assertAssignedNodeRemainsSame(allocator, routingAllocation, shard); + } + + public void testRebalancePossible() { + AllocationDecider canAllocateDecider = new AllocationDecider(Settings.EMPTY) { + @Override + public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) { + return Decision.YES; + } + }; + Tuple rebalance = setupStateAndRebalance(canAllocateDecider, Settings.EMPTY, true); + ClusterState clusterState = rebalance.v1(); + RebalanceDecision rebalanceDecision = rebalance.v2(); + assertEquals(Type.YES, rebalanceDecision.getCanRebalanceDecision().type()); + assertEquals(Type.YES, rebalanceDecision.getFinalDecisionType()); + assertNotNull(rebalanceDecision.getFinalExplanation()); + assertEquals(clusterState.nodes().getSize() - 1, rebalanceDecision.getNodeDecisions().size()); + assertNotNull(rebalanceDecision.getAssignedNodeId()); + } + + public void testRebalancingNotAllowedDueToCanAllocate() { + AllocationDecider canAllocateDecider = new AllocationDecider(Settings.EMPTY) { + @Override + public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) { + return Decision.NO; + } + }; + Tuple rebalance = setupStateAndRebalance(canAllocateDecider, Settings.EMPTY, false); + ClusterState clusterState = rebalance.v1(); + RebalanceDecision rebalanceDecision = rebalance.v2(); + assertEquals(Type.YES, rebalanceDecision.getCanRebalanceDecision().type()); + assertEquals(Type.NO, rebalanceDecision.getFinalDecisionType()); + assertThat(rebalanceDecision.getFinalExplanation(), + startsWith("cannot rebalance shard, no other node exists that would form a more balanced")); + assertEquals(clusterState.nodes().getSize() - 1, rebalanceDecision.getNodeDecisions().size()); + assertNull(rebalanceDecision.getAssignedNodeId()); + } + + public void testDontBalanceShardWhenThresholdNotMet() { + AllocationDecider canAllocateDecider = new AllocationDecider(Settings.EMPTY) { + @Override + public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) { + return Decision.YES; + } + }; + // ridiculously high threshold setting so we won't rebalance + Settings balancerSettings = Settings.builder().put(BalancedShardsAllocator.THRESHOLD_SETTING.getKey(), 1000f).build(); + Tuple rebalance = setupStateAndRebalance(canAllocateDecider, balancerSettings, false); + ClusterState clusterState = rebalance.v1(); + RebalanceDecision rebalanceDecision = rebalance.v2(); + assertEquals(Type.YES, rebalanceDecision.getCanRebalanceDecision().type()); + assertEquals(Type.NO, rebalanceDecision.getFinalDecisionType()); + assertNotNull(rebalanceDecision.getFinalExplanation()); + assertEquals(clusterState.nodes().getSize() - 1, rebalanceDecision.getNodeDecisions().size()); + assertNull(rebalanceDecision.getAssignedNodeId()); + } + + public void testSingleShardBalanceProducesSameResultsAsBalanceStep() { + final String[] indices = { "idx1", "idx2" }; + // Create a cluster state with 2 indices, each with 1 started primary shard, and only + // one node initially so that all primary shards get allocated to the same node. We are only + // using 2 indices (i.e. 2 total primary shards) because if we have any more than 2 started shards + // in the routing table, then we have no guarantees about the order in which the 3 or more shards + // are selected to be rebalanced to the new node, and hence the node to which they are rebalanced + // is not deterministic. Using only two shards guarantees that only one of those two shards will + // be rebalanced, and so we pick the one that was chosen to be rebalanced and execute the single-shard + // rebalance step on it to make sure it gets assigned to the same node. + ClusterState clusterState = ClusterStateCreationUtils.state(1, indices, 1); + // add new nodes so one of the primaries can be rebalanced + DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(clusterState.nodes()); + int numAddedNodes = randomIntBetween(1, 5); + // randomly select a subset of the newly added nodes to set filter allocation on (but not all) + int excludeNodesSize = randomIntBetween(0, numAddedNodes - 1); + final Set excludeNodes = new HashSet<>(); + for (int i = 0; i < numAddedNodes; i++) { + DiscoveryNode discoveryNode = newNode(randomAsciiOfLength(7)); + nodesBuilder.add(discoveryNode); + if (i < excludeNodesSize) { + excludeNodes.add(discoveryNode.getId()); + } + } + clusterState = ClusterState.builder(clusterState).nodes(nodesBuilder).build(); + + AllocationDecider allocationDecider = new AllocationDecider(Settings.EMPTY) { + @Override + public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) { + if (excludeNodes.contains(node.nodeId())) { + return Decision.NO; + } + return Decision.YES; + } + }; + AllocationDecider rebalanceDecider = new AllocationDecider(Settings.EMPTY) { + @Override + public Decision canRebalance(ShardRouting shardRouting, RoutingAllocation allocation) { + return Decision.YES; + } + }; + List allocationDeciders = Arrays.asList(rebalanceDecider, allocationDecider); + RoutingAllocation routingAllocation = newRoutingAllocation( + new AllocationDeciders(Settings.EMPTY, allocationDeciders), clusterState); + // allocate and get the node that is now relocating + BalancedShardsAllocator allocator = new BalancedShardsAllocator(Settings.EMPTY); + allocator.allocate(routingAllocation); + ShardRouting shardToRebalance = null; + for (RoutingNode routingNode : routingAllocation.routingNodes()) { + List relocatingShards = routingNode.shardsWithState(ShardRoutingState.RELOCATING); + if (relocatingShards.size() > 0) { + shardToRebalance = randomFrom(relocatingShards); + break; + } + } + + routingAllocation = newRoutingAllocation(new AllocationDeciders(Settings.EMPTY, allocationDeciders), clusterState); + routingAllocation.debugDecision(true); + ShardRouting shard = clusterState.getRoutingNodes().activePrimary(shardToRebalance.shardId()); + RebalanceDecision rebalanceDecision = allocator.decideRebalance(shard, routingAllocation); + assertEquals(shardToRebalance.relocatingNodeId(), rebalanceDecision.getAssignedNodeId()); + // make sure all excluded nodes returned a NO decision + for (String exludedNode : excludeNodes) { + NodeRebalanceDecision nodeRebalanceDecision = rebalanceDecision.getNodeDecisions().get(exludedNode); + assertEquals(Type.NO, nodeRebalanceDecision.getCanAllocateDecision().type()); + } + } + + private Tuple setupStateAndRebalance(AllocationDecider allocationDecider, + Settings balancerSettings, + boolean rebalanceExpected) { + AllocationDecider rebalanceDecider = new AllocationDecider(Settings.EMPTY) { + @Override + public Decision canRebalance(ShardRouting shardRouting, RoutingAllocation allocation) { + return Decision.YES; + } + }; + List allocationDeciders = Arrays.asList(rebalanceDecider, allocationDecider); + final int numShards = randomIntBetween(8, 13); + BalancedShardsAllocator allocator = new BalancedShardsAllocator(balancerSettings); + ClusterState clusterState = ClusterStateCreationUtils.state("idx", 2, numShards); + // add a new node so shards can be rebalanced there + DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(clusterState.nodes()); + nodesBuilder.add(newNode(randomAsciiOfLength(7))); + clusterState = ClusterState.builder(clusterState).nodes(nodesBuilder).build(); + ShardRouting shard = clusterState.routingTable().index("idx").shard(0).primaryShard(); + RoutingAllocation routingAllocation = newRoutingAllocation( + new AllocationDeciders(Settings.EMPTY, allocationDeciders), clusterState); + RebalanceDecision rebalanceDecision = allocator.decideRebalance(shard, routingAllocation); + + if (rebalanceExpected == false) { + assertAssignedNodeRemainsSame(allocator, routingAllocation, shard); + } + + return Tuple.tuple(clusterState, rebalanceDecision); + } + + private RoutingAllocation newRoutingAllocation(AllocationDeciders deciders, ClusterState state) { + RoutingAllocation allocation = new RoutingAllocation( + deciders, new RoutingNodes(state, false), state, ClusterInfo.EMPTY, System.nanoTime(), false + ); + allocation.debugDecision(true); + return allocation; + } + + private void assertAssignedNodeRemainsSame(BalancedShardsAllocator allocator, RoutingAllocation routingAllocation, + ShardRouting originalRouting) { + allocator.allocate(routingAllocation); + RoutingNodes routingNodes = routingAllocation.routingNodes(); + // make sure the previous node id is the same as the current one after rerouting + assertEquals(originalRouting.currentNodeId(), routingNodes.activePrimary(originalRouting.shardId()).currentNodeId()); + } +} From 79090431af1a6d9a1945a316968bf21cc66a4386 Mon Sep 17 00:00:00 2001 From: Yuhao Bi Date: Wed, 2 Nov 2016 00:25:20 +0800 Subject: [PATCH 081/132] Provide error message when request path is null --- .../java/org/elasticsearch/client/RestClient.java | 1 + .../elasticsearch/client/RestClientIntegTests.java | 11 +++++++++++ 2 files changed, 12 insertions(+) diff --git a/client/rest/src/main/java/org/elasticsearch/client/RestClient.java b/client/rest/src/main/java/org/elasticsearch/client/RestClient.java index d2301e1e8e7..b8eb98b4aee 100644 --- a/client/rest/src/main/java/org/elasticsearch/client/RestClient.java +++ b/client/rest/src/main/java/org/elasticsearch/client/RestClient.java @@ -510,6 +510,7 @@ public class RestClient implements Closeable { private static URI buildUri(String pathPrefix, String path, Map params) { Objects.requireNonNull(params, "params must not be null"); + Objects.requireNonNull(path, "path must not be null"); try { String fullPath; if (pathPrefix != null) { diff --git a/client/rest/src/test/java/org/elasticsearch/client/RestClientIntegTests.java b/client/rest/src/test/java/org/elasticsearch/client/RestClientIntegTests.java index 9c5c50946d8..941af2246f8 100644 --- a/client/rest/src/test/java/org/elasticsearch/client/RestClientIntegTests.java +++ b/client/rest/src/test/java/org/elasticsearch/client/RestClientIntegTests.java @@ -229,6 +229,17 @@ public class RestClientIntegTests extends RestClientTestCase { } } + public void testPath() throws IOException { + for (String method : getHttpMethods()) { + try { + restClient.performRequest(method, null); + fail("path set to null should fail!"); + } catch (NullPointerException e) { + assertEquals("path must not be null", e.getMessage()); + } + } + } + private void bodyTest(String method) throws IOException { String requestBody = "{ \"field\": \"value\" }"; StringEntity entity = new StringEntity(requestBody); From 4db1ac931f4fc974aae42e8fe713850f97cc52bc Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 1 Nov 2016 22:25:38 +0100 Subject: [PATCH 082/132] Fix InternalEngineTests#testUpgradeOldIndex for 5.0.0 BWC indices Relates to #21147 --- .../index/engine/InternalEngineTests.java | 27 ++++++++++--------- .../org/elasticsearch/test/OldIndexUtils.java | 5 ++-- 2 files changed, 17 insertions(+), 15 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index ef381bbcf2f..bb030d82ed7 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -107,6 +107,7 @@ import org.elasticsearch.indices.mapper.MapperRegistry; import org.elasticsearch.test.DummyShardLock; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.IndexSettingsModule; +import org.elasticsearch.test.OldIndexUtils; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.hamcrest.MatcherAssert; @@ -123,6 +124,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Base64; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Locale; import java.util.Map; @@ -1790,7 +1792,6 @@ public class InternalEngineTests extends ESTestCase { return new Mapping(Version.CURRENT, root, new MetadataFieldMapper[0], emptyMap()); } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/21147") public void testUpgradeOldIndex() throws IOException { List indexes = new ArrayList<>(); try (DirectoryStream stream = Files.newDirectoryStream(getBwcIndicesPath(), "index-*.zip")) { @@ -1812,18 +1813,19 @@ public class InternalEngineTests extends ESTestCase { Path[] list = filterExtraFSFiles(FileSystemUtils.files(unzipDataDir)); if (list.length != 1) { - throw new IllegalStateException("Backwards index must contain exactly one cluster but was " + list.length + " " + Arrays.toString(list)); + throw new IllegalStateException("Backwards index must contain exactly one cluster but was " + list.length + + " " + Arrays.toString(list)); } + // 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)); + Path src = OldIndexUtils.getIndexDir(logger, indexName, indexFile.toString(), list[0]); + Path translog = src.resolve("0").resolve("translog"); assertTrue("[" + indexFile + "] missing translog dir: " + translog.toString(), Files.exists(translog)); Path[] tlogFiles = filterExtraFSFiles(FileSystemUtils.files(translog)); assertEquals(Arrays.toString(tlogFiles), tlogFiles.length, 2); // ckp & tlog Path tlogFile = tlogFiles[0].getFileName().toString().endsWith("tlog") ? tlogFiles[0] : tlogFiles[1]; - final long size = Files.size(tlogFiles[0]); - logger.debug("upgrading index {} file: {} size: {}", indexName, tlogFiles[0].getFileName(), size); + final long size = Files.size(tlogFile); + logger.info("upgrading index {} file: {} size: {}", indexName, tlogFiles[0].getFileName(), size); Directory directory = newFSDirectory(src.resolve("0").resolve("index")); Store store = createStore(directory); final int iters = randomIntBetween(0, 2); @@ -1967,7 +1969,7 @@ public class InternalEngineTests extends ESTestCase { public static class TranslogHandler extends TranslogRecoveryPerformer { - private final DocumentMapper docMapper; + private final MapperService mapperService; public Mapping mappingUpdate = null; public final AtomicInteger recoveredOps = new AtomicInteger(0); @@ -1975,7 +1977,6 @@ public class InternalEngineTests extends ESTestCase { public TranslogHandler(String indexName, Logger logger) { super(new ShardId("test", "_na_", 0), null, logger); Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); - RootObjectMapper.Builder rootBuilder = new RootObjectMapper.Builder("test"); Index index = new Index(indexName, "_na_"); IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(index, settings); IndexAnalyzers indexAnalyzers = null; @@ -1983,14 +1984,14 @@ public class InternalEngineTests extends ESTestCase { indexAnalyzers = new IndexAnalyzers(indexSettings, defaultAnalyzer, defaultAnalyzer, defaultAnalyzer, Collections.emptyMap()); SimilarityService similarityService = new SimilarityService(indexSettings, Collections.emptyMap()); MapperRegistry mapperRegistry = new IndicesModule(Collections.emptyList()).getMapperRegistry(); - MapperService mapperService = new MapperService(indexSettings, indexAnalyzers, similarityService, mapperRegistry, () -> null); - DocumentMapper.Builder b = new DocumentMapper.Builder(rootBuilder, mapperService); - this.docMapper = b.build(mapperService); + mapperService = new MapperService(indexSettings, indexAnalyzers, similarityService, mapperRegistry, () -> null); } @Override protected DocumentMapperForType docMapper(String type) { - return new DocumentMapperForType(docMapper, mappingUpdate); + RootObjectMapper.Builder rootBuilder = new RootObjectMapper.Builder(type); + DocumentMapper.Builder b = new DocumentMapper.Builder(rootBuilder, mapperService); + return new DocumentMapperForType(b.build(mapperService), mappingUpdate); } @Override diff --git a/test/framework/src/main/java/org/elasticsearch/test/OldIndexUtils.java b/test/framework/src/main/java/org/elasticsearch/test/OldIndexUtils.java index 01474e6547d..fd7d7a7f03f 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/OldIndexUtils.java +++ b/test/framework/src/main/java/org/elasticsearch/test/OldIndexUtils.java @@ -124,12 +124,13 @@ public class OldIndexUtils { return src; } else { final List indexFolders = new ArrayList<>(); - try (DirectoryStream stream = Files.newDirectoryStream(dataDir.resolve("0/indices"))) { + try (DirectoryStream stream = Files.newDirectoryStream(dataDir.resolve("0/indices"), + (p) -> p.getFileName().startsWith("extra") == false)) { // extra FS can break this... for (final Path path : stream) { indexFolders.add(path); } } - assertThat(indexFolders.size(), equalTo(1)); + assertThat(indexFolders.toString(), indexFolders.size(), equalTo(1)); final IndexMetaData indexMetaData = IndexMetaData.FORMAT.loadLatestState(logger, indexFolders.get(0)); assertNotNull(indexMetaData); assertThat(indexFolders.get(0).getFileName().toString(), equalTo(indexMetaData.getIndexUUID())); From 2ba4dadea0c16fdb701caf79db81b966df58c52f Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 2 Nov 2016 09:30:49 +0100 Subject: [PATCH 083/132] [TEST] fix extrasFS file filtering in OldIndexUtils --- .../org/elasticsearch/index/engine/InternalEngineTests.java | 2 +- .../src/main/java/org/elasticsearch/test/OldIndexUtils.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index bb030d82ed7..d2f21cb397c 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -1825,7 +1825,7 @@ public class InternalEngineTests extends ESTestCase { assertEquals(Arrays.toString(tlogFiles), tlogFiles.length, 2); // ckp & tlog Path tlogFile = tlogFiles[0].getFileName().toString().endsWith("tlog") ? tlogFiles[0] : tlogFiles[1]; final long size = Files.size(tlogFile); - logger.info("upgrading index {} file: {} size: {}", indexName, tlogFiles[0].getFileName(), size); + logger.debug("upgrading index {} file: {} size: {}", indexName, tlogFiles[0].getFileName(), size); Directory directory = newFSDirectory(src.resolve("0").resolve("index")); Store store = createStore(directory); final int iters = randomIntBetween(0, 2); diff --git a/test/framework/src/main/java/org/elasticsearch/test/OldIndexUtils.java b/test/framework/src/main/java/org/elasticsearch/test/OldIndexUtils.java index fd7d7a7f03f..7f43c9de61b 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/OldIndexUtils.java +++ b/test/framework/src/main/java/org/elasticsearch/test/OldIndexUtils.java @@ -125,7 +125,7 @@ public class OldIndexUtils { } else { final List indexFolders = new ArrayList<>(); try (DirectoryStream stream = Files.newDirectoryStream(dataDir.resolve("0/indices"), - (p) -> p.getFileName().startsWith("extra") == false)) { // extra FS can break this... + (p) -> p.getFileName().toString().startsWith("extra") == false)) { // extra FS can break this... for (final Path path : stream) { indexFolders.add(path); } From 51717c882f23a5a5d3fae1a23477b29b4079ca4d Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 2 Nov 2016 09:33:50 +0100 Subject: [PATCH 084/132] [TEST] add back AwaitsFix until we get a new Lucene 6.3.0 snapshot --- .../org/elasticsearch/index/engine/InternalEngineTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index d2f21cb397c..f8621263f3f 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -124,7 +124,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Base64; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Locale; import java.util.Map; @@ -1792,6 +1791,7 @@ public class InternalEngineTests extends ESTestCase { return new Mapping(Version.CURRENT, root, new MetadataFieldMapper[0], emptyMap()); } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/21147") public void testUpgradeOldIndex() throws IOException { List indexes = new ArrayList<>(); try (DirectoryStream stream = Files.newDirectoryStream(getBwcIndicesPath(), "index-*.zip")) { From aa6cd93e0f860faf869e46386beea3836f635816 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Wed, 2 Nov 2016 09:48:49 +0100 Subject: [PATCH 085/132] Require arguments for QueryShardContext creation. (#21196) The `IndexService#newQueryShardContext()` method creates a QueryShardContext on shard `0`, with a `null` reader and that uses `System.currentTimeMillis()` to resolve `now`. This may hide bugs, since the shard id is sometimes used for query parsing (it is used to salt random score generation in `function_score`), passing a `null` reader disables query rewriting and for some use-cases, it is simply not ok to rely on the current timestamp (eg. percolation). So this pull request removes this method and instead requires that all call sites provide these parameters explicitly. --- .../metadata/MetaDataCreateIndexService.java | 4 ++- .../metadata/MetaDataIndexAliasesService.java | 4 ++- .../org/elasticsearch/index/IndexService.java | 19 ++++++-------- .../index/query/QueryRewriteContext.java | 4 ++- .../index/mapper/CustomBoostMappingTests.java | 4 +-- .../index/mapper/DoubleIndexingDocTests.java | 2 +- .../mapper/ExternalFieldMapperTests.java | 17 +++++++++--- .../mapper/FieldNamesFieldMapperTests.java | 9 +++++-- .../query/plugin/CustomQueryParserIT.java | 3 ++- .../index/search/MultiMatchQueryTests.java | 26 ++++++++++++------- .../PercolatorFieldMapperTests.java | 7 +++-- .../murmur3/Murmur3FieldMapperTests.java | 12 +++++++-- .../elasticsearch/test/TestSearchContext.java | 3 +-- 13 files changed, 75 insertions(+), 39 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java index e7c032a120b..f9034f6a29f 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java @@ -352,7 +352,9 @@ public class MetaDataCreateIndexService extends AbstractComponent { throw mpe; } - final QueryShardContext queryShardContext = indexService.newQueryShardContext(); + // the context is only used for validation so it's fine to pass fake values for the shard id and the current + // timestamp + final QueryShardContext queryShardContext = indexService.newQueryShardContext(0, null, () -> 0L); for (Alias alias : request.aliases()) { if (Strings.hasLength(alias.filter())) { aliasValidator.validateAliasFilter(alias.name(), alias.filter(), queryShardContext); diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java index 5b0c5a84061..b97b77de64f 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java @@ -149,7 +149,9 @@ public class MetaDataIndexAliasesService extends AbstractComponent { } indices.put(action.getIndex(), indexService); } - aliasValidator.validateAliasFilter(alias, filter, indexService.newQueryShardContext()); + // the context is only used for validation so it's fine to pass fake values for the shard id and the current + // timestamp + aliasValidator.validateAliasFilter(alias, filter, indexService.newQueryShardContext(0, null, () -> 0L)); } }; changed |= action.apply(newAliasValidator, metadata, index); diff --git a/core/src/main/java/org/elasticsearch/index/IndexService.java b/core/src/main/java/org/elasticsearch/index/IndexService.java index 0114d68b4e2..5fc0d1d27cd 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexService.java +++ b/core/src/main/java/org/elasticsearch/index/IndexService.java @@ -145,7 +145,10 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust this.indexAnalyzers = registry.build(indexSettings); this.similarityService = similarityService; this.mapperService = new MapperService(indexSettings, indexAnalyzers, similarityService, mapperRegistry, - IndexService.this::newQueryShardContext); + // we parse all percolator queries as they would be parsed on shard 0 + () -> newQueryShardContext(0, null, () -> { + throw new IllegalArgumentException("Percolator queries are not allowed to use the curent timestamp"); + })); this.indexFieldData = new IndexFieldDataService(indexSettings, indicesFieldDataCache, circuitBreakerService, mapperService); this.shardStoreDeleter = shardStoreDeleter; this.bigArrays = bigArrays; @@ -453,7 +456,10 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust /** * Creates a new QueryShardContext. The context has not types set yet, if types are required set them via - * {@link QueryShardContext#setTypes(String...)} + * {@link QueryShardContext#setTypes(String...)}. + * + * Passing a {@code null} {@link IndexReader} will return a valid context, however it won't be able to make + * {@link IndexReader}-specific optimizations, such as rewriting containing range queries. */ public QueryShardContext newQueryShardContext(int shardId, IndexReader indexReader, LongSupplier nowInMillis) { return new QueryShardContext( @@ -464,15 +470,6 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust nowInMillis); } - /** - * Creates a new QueryShardContext. The context has not types set yet, if types are required set them via - * {@link QueryShardContext#setTypes(String...)}. This context may be used for query parsing but cannot be - * used for rewriting since it does not know about the current {@link IndexReader}. - */ - public QueryShardContext newQueryShardContext() { - return newQueryShardContext(0, null, System::currentTimeMillis); - } - /** * The {@link ThreadPool} to use for this index. */ diff --git a/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java b/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java index ccca0af652b..c8b5cf174f5 100644 --- a/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java +++ b/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java @@ -86,7 +86,9 @@ public class QueryRewriteContext implements ParseFieldMatcherSupplier { return mapperService; } - /** Return the current {@link IndexReader}, or {@code null} if we are on the coordinating node. */ + /** Return the current {@link IndexReader}, or {@code null} if no index reader is available, for + * instance if we are on the coordinating node or if this rewrite context is used to index + * queries (percolation). */ public IndexReader getIndexReader() { return reader; } diff --git a/core/src/test/java/org/elasticsearch/index/mapper/CustomBoostMappingTests.java b/core/src/test/java/org/elasticsearch/index/mapper/CustomBoostMappingTests.java index 9bc87e874f9..391f987e714 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/CustomBoostMappingTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/CustomBoostMappingTests.java @@ -97,7 +97,7 @@ public class CustomBoostMappingTests extends ESSingleNodeTestCase { .startObject("date_field").field("type", "date").field("boost", 9.0f).endObject() .endObject().endObject().endObject().string(); IndexService indexService = createIndex("test", BW_SETTINGS); - QueryShardContext context = indexService.newQueryShardContext(); + QueryShardContext context = indexService.newQueryShardContext(0, null, () -> 0L); DocumentMapper mapper = indexService.mapperService().documentMapperParser().parse("type", new CompressedXContent(mapping)); DocumentFieldMappers fieldMappers = mapper.mappers(); assertThat(fieldMappers.getMapper("s_field").fieldType().termQuery("0", context), instanceOf(TermQuery.class)); @@ -150,7 +150,7 @@ public class CustomBoostMappingTests extends ESSingleNodeTestCase { .startObject("date_field").field("type", "date").field("boost", 9.0f).endObject() .endObject().endObject().endObject().string(); IndexService indexService = createIndex("text"); - QueryShardContext context = indexService.newQueryShardContext(); + QueryShardContext context = indexService.newQueryShardContext(0, null, () -> 0L); DocumentMapper mapper = indexService.mapperService().documentMapperParser().parse("type", new CompressedXContent(mapping)); DocumentFieldMappers fieldMappers = mapper.mappers(); assertThat(fieldMappers.getMapper("s_field").fieldType().termQuery("0", context), instanceOf(BoostQuery.class)); diff --git a/core/src/test/java/org/elasticsearch/index/mapper/DoubleIndexingDocTests.java b/core/src/test/java/org/elasticsearch/index/mapper/DoubleIndexingDocTests.java index b434b41242f..af217030f2c 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/DoubleIndexingDocTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/DoubleIndexingDocTests.java @@ -45,7 +45,7 @@ public class DoubleIndexingDocTests extends ESSingleNodeTestCase { IndexService index = createIndex("test"); client().admin().indices().preparePutMapping("test").setType("type").setSource(mapping).get(); DocumentMapper mapper = index.mapperService().documentMapper("type"); - QueryShardContext context = index.newQueryShardContext(); + QueryShardContext context = index.newQueryShardContext(0, null, () -> 0L); ParsedDocument doc = mapper.parse("test", "type", "1", XContentFactory.jsonBuilder() .startObject() diff --git a/core/src/test/java/org/elasticsearch/index/mapper/ExternalFieldMapperTests.java b/core/src/test/java/org/elasticsearch/index/mapper/ExternalFieldMapperTests.java index 44c973748d0..72f9d09808f 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/ExternalFieldMapperTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/ExternalFieldMapperTests.java @@ -29,6 +29,7 @@ import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.indices.mapper.MapperRegistry; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESSingleNodeTestCase; @@ -39,6 +40,7 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.function.Supplier; import static org.hamcrest.Matchers.closeTo; import static org.hamcrest.Matchers.is; @@ -59,8 +61,11 @@ public class ExternalFieldMapperTests extends ESSingleNodeTestCase { Collections.singletonMap(ExternalMapperPlugin.EXTERNAL, new ExternalMapper.TypeParser(ExternalMapperPlugin.EXTERNAL, "foo")), Collections.singletonMap(ExternalMetadataMapper.CONTENT_TYPE, new ExternalMetadataMapper.TypeParser())); + Supplier queryShardContext = () -> { + return indexService.newQueryShardContext(0, null, () -> { throw new UnsupportedOperationException(); }); + }; DocumentMapperParser parser = new DocumentMapperParser(indexService.getIndexSettings(), indexService.mapperService(), - indexService.getIndexAnalyzers(), indexService.similarityService(), mapperRegistry, indexService::newQueryShardContext); + indexService.getIndexAnalyzers(), indexService.similarityService(), mapperRegistry, queryShardContext); DocumentMapper documentMapper = parser.parse("type", new CompressedXContent( XContentFactory.jsonBuilder().startObject().startObject("type") .startObject(ExternalMetadataMapper.CONTENT_TYPE) @@ -108,8 +113,11 @@ public class ExternalFieldMapperTests extends ESSingleNodeTestCase { mapperParsers.put(KeywordFieldMapper.CONTENT_TYPE, new KeywordFieldMapper.TypeParser()); MapperRegistry mapperRegistry = new MapperRegistry(mapperParsers, Collections.emptyMap()); + Supplier queryShardContext = () -> { + return indexService.newQueryShardContext(0, null, () -> { throw new UnsupportedOperationException(); }); + }; DocumentMapperParser parser = new DocumentMapperParser(indexService.getIndexSettings(), indexService.mapperService(), - indexService.getIndexAnalyzers(), indexService.similarityService(), mapperRegistry, indexService::newQueryShardContext); + indexService.getIndexAnalyzers(), indexService.similarityService(), mapperRegistry, queryShardContext); DocumentMapper documentMapper = parser.parse("type", new CompressedXContent( XContentFactory.jsonBuilder().startObject().startObject("type").startObject("properties") @@ -178,8 +186,11 @@ public class ExternalFieldMapperTests extends ESSingleNodeTestCase { mapperParsers.put(TextFieldMapper.CONTENT_TYPE, new TextFieldMapper.TypeParser()); MapperRegistry mapperRegistry = new MapperRegistry(mapperParsers, Collections.emptyMap()); + Supplier queryShardContext = () -> { + return indexService.newQueryShardContext(0, null, () -> { throw new UnsupportedOperationException(); }); + }; DocumentMapperParser parser = new DocumentMapperParser(indexService.getIndexSettings(), indexService.mapperService(), - indexService.getIndexAnalyzers(), indexService.similarityService(), mapperRegistry, indexService::newQueryShardContext); + indexService.getIndexAnalyzers(), indexService.similarityService(), mapperRegistry, queryShardContext); DocumentMapper documentMapper = parser.parse("type", new CompressedXContent( XContentFactory.jsonBuilder().startObject().startObject("type").startObject("properties") diff --git a/core/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldMapperTests.java b/core/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldMapperTests.java index 544764a9b53..5bc70350b13 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldMapperTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldMapperTests.java @@ -27,6 +27,7 @@ import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.indices.IndicesModule; import org.elasticsearch.indices.mapper.MapperRegistry; import org.elasticsearch.test.ESSingleNodeTestCase; @@ -38,6 +39,7 @@ import java.util.List; import java.util.Map; import java.util.SortedSet; import java.util.TreeSet; +import java.util.function.Supplier; public class FieldNamesFieldMapperTests extends ESSingleNodeTestCase { @@ -231,9 +233,12 @@ public class FieldNamesFieldMapperTests extends ESSingleNodeTestCase { Collections.singletonMap("_dummy", new DummyMetadataFieldMapper.TypeParser()) ); final MapperRegistry mapperRegistry = indicesModule.getMapperRegistry(); - MapperService mapperService = new MapperService(indexService.getIndexSettings(), indexService.getIndexAnalyzers(), indexService.similarityService(), mapperRegistry, indexService::newQueryShardContext); + Supplier queryShardContext = () -> { + return indexService.newQueryShardContext(0, null, () -> { throw new UnsupportedOperationException(); }); + }; + MapperService mapperService = new MapperService(indexService.getIndexSettings(), indexService.getIndexAnalyzers(), indexService.similarityService(), mapperRegistry, queryShardContext); DocumentMapperParser parser = new DocumentMapperParser(indexService.getIndexSettings(), mapperService, - indexService.getIndexAnalyzers(), indexService.similarityService(), mapperRegistry, indexService::newQueryShardContext); + indexService.getIndexAnalyzers(), indexService.similarityService(), mapperRegistry, queryShardContext); String mapping = XContentFactory.jsonBuilder().startObject().startObject("type").endObject().endObject().string(); DocumentMapper mapper = parser.parse("type", new CompressedXContent(mapping)); ParsedDocument parsedDocument = mapper.parse("index", "type", "id", new BytesArray("{}")); diff --git a/core/src/test/java/org/elasticsearch/index/query/plugin/CustomQueryParserIT.java b/core/src/test/java/org/elasticsearch/index/query/plugin/CustomQueryParserIT.java index 67619a3bbe5..c2c39be93be 100644 --- a/core/src/test/java/org/elasticsearch/index/query/plugin/CustomQueryParserIT.java +++ b/core/src/test/java/org/elasticsearch/index/query/plugin/CustomQueryParserIT.java @@ -74,7 +74,8 @@ public class CustomQueryParserIT extends ESIntegTestCase { private static QueryShardContext queryShardContext() { IndicesService indicesService = internalCluster().getDataNodeInstance(IndicesService.class); - return indicesService.indexServiceSafe(resolveIndex("index")).newQueryShardContext(); + return indicesService.indexServiceSafe(resolveIndex("index")).newQueryShardContext( + randomInt(20), null, () -> { throw new UnsupportedOperationException(); }); } //see #11120 diff --git a/core/src/test/java/org/elasticsearch/index/search/MultiMatchQueryTests.java b/core/src/test/java/org/elasticsearch/index/search/MultiMatchQueryTests.java index 2454150be56..7d7b7a4cd6e 100644 --- a/core/src/test/java/org/elasticsearch/index/search/MultiMatchQueryTests.java +++ b/core/src/test/java/org/elasticsearch/index/search/MultiMatchQueryTests.java @@ -78,7 +78,8 @@ public class MultiMatchQueryTests extends ESSingleNodeTestCase { } public void testCrossFieldMultiMatchQuery() throws IOException { - QueryShardContext queryShardContext = indexService.newQueryShardContext(); + QueryShardContext queryShardContext = indexService.newQueryShardContext( + randomInt(20), null, () -> { throw new UnsupportedOperationException(); }); queryShardContext.setAllowUnmappedFields(true); Query parsedQuery = multiMatchQuery("banon").field("name.first", 2).field("name.last", 3).field("foobar").type(MultiMatchQueryBuilder.Type.CROSS_FIELDS).toQuery(queryShardContext); try (Engine.Searcher searcher = indexService.getShard(0).acquireSearcher("test")) { @@ -101,8 +102,9 @@ public class MultiMatchQueryTests extends ESSingleNodeTestCase { Term[] terms = new Term[] { new Term("foo", "baz"), new Term("bar", "baz") }; float[] boosts = new float[] {2, 3}; Query expected = BlendedTermQuery.booleanBlendedQuery(terms, boosts, false); - Query actual = MultiMatchQuery.blendTerm(indexService.newQueryShardContext(), new BytesRef("baz"), null, 1f, - new FieldAndFieldType(ft1, 2), new FieldAndFieldType(ft2, 3)); + Query actual = MultiMatchQuery.blendTerm( + indexService.newQueryShardContext(randomInt(20), null, () -> { throw new UnsupportedOperationException(); }), + new BytesRef("baz"), null, 1f, new FieldAndFieldType(ft1, 2), new FieldAndFieldType(ft2, 3)); assertEquals(expected, actual); } @@ -116,8 +118,9 @@ public class MultiMatchQueryTests extends ESSingleNodeTestCase { Term[] terms = new Term[] { new Term("foo", "baz"), new Term("bar", "baz") }; float[] boosts = new float[] {200, 30}; Query expected = BlendedTermQuery.booleanBlendedQuery(terms, boosts, false); - Query actual = MultiMatchQuery.blendTerm(indexService.newQueryShardContext(), new BytesRef("baz"), null, 1f, - new FieldAndFieldType(ft1, 2), new FieldAndFieldType(ft2, 3)); + Query actual = MultiMatchQuery.blendTerm( + indexService.newQueryShardContext(randomInt(20), null, () -> { throw new UnsupportedOperationException(); }), + new BytesRef("baz"), null, 1f, new FieldAndFieldType(ft1, 2), new FieldAndFieldType(ft2, 3)); assertEquals(expected, actual); } @@ -134,8 +137,9 @@ public class MultiMatchQueryTests extends ESSingleNodeTestCase { Term[] terms = new Term[] { new Term("foo", "baz") }; float[] boosts = new float[] {2}; Query expected = BlendedTermQuery.booleanBlendedQuery(terms, boosts, false); - Query actual = MultiMatchQuery.blendTerm(indexService.newQueryShardContext(), new BytesRef("baz"), null, 1f, - new FieldAndFieldType(ft1, 2), new FieldAndFieldType(ft2, 3)); + Query actual = MultiMatchQuery.blendTerm( + indexService.newQueryShardContext(randomInt(20), null, () -> { throw new UnsupportedOperationException(); }), + new BytesRef("baz"), null, 1f, new FieldAndFieldType(ft1, 2), new FieldAndFieldType(ft2, 3)); assertEquals(expected, actual); } @@ -157,13 +161,15 @@ public class MultiMatchQueryTests extends ESSingleNodeTestCase { .add(expectedClause1, Occur.SHOULD) .add(expectedClause2, Occur.SHOULD) .build(); - Query actual = MultiMatchQuery.blendTerm(indexService.newQueryShardContext(), new BytesRef("baz"), null, 1f, - new FieldAndFieldType(ft1, 2), new FieldAndFieldType(ft2, 3)); + Query actual = MultiMatchQuery.blendTerm( + indexService.newQueryShardContext(randomInt(20), null, () -> { throw new UnsupportedOperationException(); }), + new BytesRef("baz"), null, 1f, new FieldAndFieldType(ft1, 2), new FieldAndFieldType(ft2, 3)); assertEquals(expected, actual); } public void testMultiMatchPrefixWithAllField() throws IOException { - QueryShardContext queryShardContext = indexService.newQueryShardContext(); + QueryShardContext queryShardContext = indexService.newQueryShardContext( + randomInt(20), null, () -> { throw new UnsupportedOperationException(); }); queryShardContext.setAllowUnmappedFields(true); Query parsedQuery = multiMatchQuery("foo").field("_all").type(MultiMatchQueryBuilder.Type.PHRASE_PREFIX).toQuery(queryShardContext); diff --git a/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorFieldMapperTests.java b/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorFieldMapperTests.java index 621cb07d3cd..d485f9fa00e 100644 --- a/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorFieldMapperTests.java +++ b/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorFieldMapperTests.java @@ -292,7 +292,8 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { .field(fieldName, queryBuilder) .endObject().bytes()); BytesRef qbSource = doc.rootDoc().getFields(fieldType.queryBuilderField.name())[0].binaryValue(); - assertQueryBuilder(qbSource, queryBuilder.rewrite(indexService.newQueryShardContext())); + assertQueryBuilder(qbSource, queryBuilder.rewrite(indexService.newQueryShardContext( + randomInt(20), null, () -> { throw new UnsupportedOperationException(); }))); } @@ -476,7 +477,9 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { private void assertQueryBuilder(BytesRef actual, QueryBuilder expected) throws IOException { XContentParser sourceParser = PercolatorFieldMapper.QUERY_BUILDER_CONTENT_TYPE.xContent() .createParser(actual.bytes, actual.offset, actual.length); - QueryParseContext qsc = indexService.newQueryShardContext().newParseContext(sourceParser); + QueryParseContext qsc = indexService.newQueryShardContext( + randomInt(20), null, () -> { throw new UnsupportedOperationException(); }) + .newParseContext(sourceParser); assertThat(qsc.parseInnerQueryBuilder().get(), equalTo(expected)); } diff --git a/plugins/mapper-murmur3/src/test/java/org/elasticsearch/index/mapper/murmur3/Murmur3FieldMapperTests.java b/plugins/mapper-murmur3/src/test/java/org/elasticsearch/index/mapper/murmur3/Murmur3FieldMapperTests.java index e10fdb72ff7..72930344bbf 100644 --- a/plugins/mapper-murmur3/src/test/java/org/elasticsearch/index/mapper/murmur3/Murmur3FieldMapperTests.java +++ b/plugins/mapper-murmur3/src/test/java/org/elasticsearch/index/mapper/murmur3/Murmur3FieldMapperTests.java @@ -32,6 +32,7 @@ import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.DocumentMapperParser; import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.ParsedDocument; +import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.indices.mapper.MapperRegistry; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESSingleNodeTestCase; @@ -42,6 +43,7 @@ import org.junit.Before; import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.function.Supplier; import static com.carrotsearch.randomizedtesting.RandomizedTest.getRandom; import static org.hamcrest.Matchers.containsString; @@ -58,8 +60,11 @@ public class Murmur3FieldMapperTests extends ESSingleNodeTestCase { mapperRegistry = new MapperRegistry( Collections.singletonMap(Murmur3FieldMapper.CONTENT_TYPE, new Murmur3FieldMapper.TypeParser()), Collections.emptyMap()); + Supplier queryShardContext = () -> { + return indexService.newQueryShardContext(0, null, () -> { throw new UnsupportedOperationException(); }); + }; parser = new DocumentMapperParser(indexService.getIndexSettings(), indexService.mapperService(), - indexService.getIndexAnalyzers(), indexService.similarityService(), mapperRegistry, indexService::newQueryShardContext); + indexService.getIndexAnalyzers(), indexService.similarityService(), mapperRegistry, queryShardContext); } @Override @@ -152,8 +157,11 @@ public class Murmur3FieldMapperTests extends ESSingleNodeTestCase { Settings oldIndexSettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, oldVersion).build(); IndexService indexService2x = createIndex("test_old", oldIndexSettings); + Supplier queryShardContext = () -> { + return indexService2x.newQueryShardContext(0, null, () -> { throw new UnsupportedOperationException(); }); + }; DocumentMapperParser parser = new DocumentMapperParser(indexService2x.getIndexSettings(), indexService2x.mapperService(), indexService2x.getIndexAnalyzers(), - indexService2x.similarityService(), mapperRegistry, indexService2x::newQueryShardContext); + indexService2x.similarityService(), mapperRegistry, queryShardContext); DocumentMapper defaultMapper = parser.parse("type", new CompressedXContent(mapping)); assertEquals(mapping, defaultMapper.mappingSource().string()); diff --git a/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java b/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java index e4c0e1d5abc..1e86f940a11 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java +++ b/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java @@ -53,7 +53,6 @@ import org.elasticsearch.search.internal.ContextIndexSearcher; import org.elasticsearch.search.internal.ScrollContext; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.ShardSearchRequest; -import org.elasticsearch.search.lookup.SearchLookup; import org.elasticsearch.search.profile.Profilers; import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.search.rescore.RescoreSearchContext; @@ -99,7 +98,7 @@ public class TestSearchContext extends SearchContext { this.fixedBitSetFilterCache = indexService.cache().bitsetFilterCache(); this.threadPool = threadPool; this.indexShard = indexService.getShardOrNull(0); - queryShardContext = indexService.newQueryShardContext(); + queryShardContext = indexService.newQueryShardContext(0, null, () -> 0L); } public TestSearchContext(QueryShardContext queryShardContext) { From 9d6fac809c2f9daf94d9c20a8a206f03e6cbac58 Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Wed, 2 Nov 2016 10:00:40 +0100 Subject: [PATCH 086/132] Expose splitOnWhitespace in `Query String Query` (#20965) This change adds an option called `split_on_whitespace` which prevents the query parser to split free text part on whitespace prior to analysis. Instead the queryparser would parse around only real 'operators'. Default to true. For instance the query `"foo bar"` would let the analyzer of the targeted field decide how the tokens should be splitted. Some options are missing in this change but I'd like to add them in a follow up PR in order to be able to simplify the backport in 5.x. The missing options (changes) are: * A `type` option which similarly to the `multi_match` query defines how the free text should be parsed when multi fields are defined. * Simple range query with additional tokens like ">100 50" are broken when `split_on_whitespace` is set to false. It should be possible to preserve this syntax and make the parser aware of this special syntax even when `split_on_whitespace` is set to false. * Since all this options would make the `query_string_query` very similar to a match (multi_match) query we should be able to share the code that produce the final Lucene query. --- .../classic/MapperQueryParser.java | 1 + .../classic/QueryParserSettings.java | 10 ++ .../index/query/QueryStringQueryBuilder.java | 39 +++++- .../java/org/elasticsearch/VersionTests.java | 2 + .../query/QueryStringQueryBuilderTests.java | 125 ++++++++++++++++++ .../query-dsl/query-string-query.asciidoc | 5 + .../query-dsl/query-string-syntax.asciidoc | 4 +- 7 files changed, 182 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/apache/lucene/queryparser/classic/MapperQueryParser.java b/core/src/main/java/org/apache/lucene/queryparser/classic/MapperQueryParser.java index b6f0020fb5b..bbb27aa11db 100644 --- a/core/src/main/java/org/apache/lucene/queryparser/classic/MapperQueryParser.java +++ b/core/src/main/java/org/apache/lucene/queryparser/classic/MapperQueryParser.java @@ -104,6 +104,7 @@ public class MapperQueryParser extends QueryParser { setDefaultOperator(settings.defaultOperator()); setFuzzyPrefixLength(settings.fuzzyPrefixLength()); setLocale(settings.locale()); + setSplitOnWhitespace(settings.splitOnWhitespace()); } /** diff --git a/core/src/main/java/org/apache/lucene/queryparser/classic/QueryParserSettings.java b/core/src/main/java/org/apache/lucene/queryparser/classic/QueryParserSettings.java index c1fc2ae556e..a03c84d8980 100644 --- a/core/src/main/java/org/apache/lucene/queryparser/classic/QueryParserSettings.java +++ b/core/src/main/java/org/apache/lucene/queryparser/classic/QueryParserSettings.java @@ -79,6 +79,8 @@ public class QueryParserSettings { /** To limit effort spent determinizing regexp queries. */ private int maxDeterminizedStates; + private boolean splitOnWhitespace; + public QueryParserSettings(String queryString) { this.queryString = queryString; } @@ -290,4 +292,12 @@ public class QueryParserSettings { public Fuzziness fuzziness() { return fuzziness; } + + public void splitOnWhitespace(boolean value) { + this.splitOnWhitespace = value; + } + + public boolean splitOnWhitespace() { + return splitOnWhitespace; + } } diff --git a/core/src/main/java/org/elasticsearch/index/query/QueryStringQueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/QueryStringQueryBuilder.java index 807343237d2..d88484c98a6 100644 --- a/core/src/main/java/org/elasticsearch/index/query/QueryStringQueryBuilder.java +++ b/core/src/main/java/org/elasticsearch/index/query/QueryStringQueryBuilder.java @@ -26,6 +26,7 @@ import org.apache.lucene.search.BoostQuery; import org.apache.lucene.search.FuzzyQuery; import org.apache.lucene.search.Query; import org.apache.lucene.util.automaton.Operations; +import org.elasticsearch.Version; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.io.stream.StreamInput; @@ -59,6 +60,8 @@ import java.util.TreeMap; public class QueryStringQueryBuilder extends AbstractQueryBuilder { public static final String NAME = "query_string"; + public static final Version V_5_1_0_UNRELEASED = Version.fromId(5010099); + public static final boolean DEFAULT_AUTO_GENERATE_PHRASE_QUERIES = false; public static final int DEFAULT_MAX_DETERMINED_STATES = Operations.DEFAULT_MAX_DETERMINIZED_STATES; public static final boolean DEFAULT_LOWERCASE_EXPANDED_TERMS = true; @@ -72,6 +75,7 @@ public class QueryStringQueryBuilder extends AbstractQueryBuilder{@value #DEFAULT_SPLIT_ON_WHITESPACE}. + */ + public QueryStringQueryBuilder splitOnWhitespace(boolean value) { + this.splitOnWhitespace = value; + return this; + } + + public boolean splitOnWhitespace() { + return splitOnWhitespace; + } + @Override protected void doXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(NAME); @@ -626,6 +654,7 @@ public class QueryStringQueryBuilder extends AbstractQueryBuilder fieldsAndWeights = new HashMap<>(); while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { if (token == XContentParser.Token.FIELD_NAME) { @@ -750,6 +780,8 @@ public class QueryStringQueryBuilder extends AbstractQueryBuilder 0); + // splitOnWhitespace=false + { + QueryStringQueryBuilder queryBuilder = + new QueryStringQueryBuilder("foo bar") + .field(STRING_FIELD_NAME).field(STRING_FIELD_NAME_2) + .splitOnWhitespace(false); + Query query = queryBuilder.toQuery(createShardContext()); + BooleanQuery bq1 = + new BooleanQuery.Builder() + .add(new BooleanClause(new TermQuery(new Term(STRING_FIELD_NAME, "foo")), BooleanClause.Occur.SHOULD)) + .add(new BooleanClause(new TermQuery(new Term(STRING_FIELD_NAME, "bar")), BooleanClause.Occur.SHOULD)) + .build(); + List disjuncts = new ArrayList<>(); + disjuncts.add(bq1); + disjuncts.add(new TermQuery(new Term(STRING_FIELD_NAME_2, "foo bar"))); + DisjunctionMaxQuery expectedQuery = new DisjunctionMaxQuery(disjuncts, 0.0f); + assertThat(query, equalTo(expectedQuery)); + } + + { + QueryStringQueryBuilder queryBuilder = + new QueryStringQueryBuilder("mapped_string:other foo bar") + .field(STRING_FIELD_NAME).field(STRING_FIELD_NAME_2) + .splitOnWhitespace(false); + Query query = queryBuilder.toQuery(createShardContext()); + BooleanQuery bq1 = + new BooleanQuery.Builder() + .add(new BooleanClause(new TermQuery(new Term(STRING_FIELD_NAME, "foo")), BooleanClause.Occur.SHOULD)) + .add(new BooleanClause(new TermQuery(new Term(STRING_FIELD_NAME, "bar")), BooleanClause.Occur.SHOULD)) + .build(); + List disjuncts = new ArrayList<>(); + disjuncts.add(bq1); + disjuncts.add(new TermQuery(new Term(STRING_FIELD_NAME_2, "foo bar"))); + DisjunctionMaxQuery disjunctionMaxQuery = new DisjunctionMaxQuery(disjuncts, 0.0f); + BooleanQuery expectedQuery = + new BooleanQuery.Builder() + .add(disjunctionMaxQuery, BooleanClause.Occur.SHOULD) + .add(new TermQuery(new Term(STRING_FIELD_NAME, "other")), BooleanClause.Occur.SHOULD) + .build(); + assertThat(query, equalTo(expectedQuery)); + } + + { + QueryStringQueryBuilder queryBuilder = + new QueryStringQueryBuilder("foo OR bar") + .field(STRING_FIELD_NAME).field(STRING_FIELD_NAME_2) + .splitOnWhitespace(false); + Query query = queryBuilder.toQuery(createShardContext()); + + List disjuncts1 = new ArrayList<>(); + disjuncts1.add(new TermQuery(new Term(STRING_FIELD_NAME, "foo"))); + disjuncts1.add(new TermQuery(new Term(STRING_FIELD_NAME_2, "foo"))); + DisjunctionMaxQuery maxQuery1 = new DisjunctionMaxQuery(disjuncts1, 0.0f); + + List disjuncts2 = new ArrayList<>(); + disjuncts2.add(new TermQuery(new Term(STRING_FIELD_NAME, "bar"))); + disjuncts2.add(new TermQuery(new Term(STRING_FIELD_NAME_2, "bar"))); + DisjunctionMaxQuery maxQuery2 = new DisjunctionMaxQuery(disjuncts2, 0.0f); + + BooleanQuery expectedQuery = + new BooleanQuery.Builder() + .add(new BooleanClause(maxQuery1, BooleanClause.Occur.SHOULD)) + .add(new BooleanClause(maxQuery2, BooleanClause.Occur.SHOULD)) + .build(); + assertThat(query, equalTo(expectedQuery)); + } + + // split_on_whitespace=false breaks range query with simple syntax + { + // throws an exception when lenient is set to false + QueryStringQueryBuilder queryBuilder = + new QueryStringQueryBuilder(">10 foo") + .field(INT_FIELD_NAME) + .splitOnWhitespace(false); + IllegalArgumentException exc = + expectThrows(IllegalArgumentException.class, () -> queryBuilder.toQuery(createShardContext())); + assertThat(exc.getMessage(), equalTo("For input string: \"10 foo\"")); + } + + { + // returns an empty boolean query when lenient is set to true + QueryStringQueryBuilder queryBuilder = + new QueryStringQueryBuilder(">10 foo") + .field(INT_FIELD_NAME) + .splitOnWhitespace(false) + .lenient(true); + Query query = queryBuilder.toQuery(createShardContext()); + BooleanQuery bq = new BooleanQuery.Builder().build(); + assertThat(bq, equalTo(query)); + } + + // splitOnWhitespace=true + { + QueryStringQueryBuilder queryBuilder = + new QueryStringQueryBuilder("foo bar") + .field(STRING_FIELD_NAME).field(STRING_FIELD_NAME_2) + .splitOnWhitespace(true); + Query query = queryBuilder.toQuery(createShardContext()); + + List disjuncts1 = new ArrayList<>(); + disjuncts1.add(new TermQuery(new Term(STRING_FIELD_NAME, "foo"))); + disjuncts1.add(new TermQuery(new Term(STRING_FIELD_NAME_2, "foo"))); + DisjunctionMaxQuery maxQuery1 = new DisjunctionMaxQuery(disjuncts1, 0.0f); + + List disjuncts2 = new ArrayList<>(); + disjuncts2.add(new TermQuery(new Term(STRING_FIELD_NAME, "bar"))); + disjuncts2.add(new TermQuery(new Term(STRING_FIELD_NAME_2, "bar"))); + DisjunctionMaxQuery maxQuery2 = new DisjunctionMaxQuery(disjuncts2, 0.0f); + + BooleanQuery expectedQuery = + new BooleanQuery.Builder() + .add(new BooleanClause(maxQuery1, BooleanClause.Occur.SHOULD)) + .add(new BooleanClause(maxQuery2, BooleanClause.Occur.SHOULD)) + .build(); + assertThat(query, equalTo(expectedQuery)); + } + + + } + public void testFromJson() throws IOException { String json = "{\n" + @@ -552,6 +676,7 @@ public class QueryStringQueryBuilderTests extends AbstractQueryTestCase> for a comprehensive example. + +|`split_on_whitespace` |Whether query text should be split on whitespace prior to analysis. + Instead the queryparser would parse around only real 'operators'. + Default to `false`. + |======================================================================= When a multi term query is being generated, one can control how it gets diff --git a/docs/reference/query-dsl/query-string-syntax.asciidoc b/docs/reference/query-dsl/query-string-syntax.asciidoc index 9e847102469..49a8b54c506 100644 --- a/docs/reference/query-dsl/query-string-syntax.asciidoc +++ b/docs/reference/query-dsl/query-string-syntax.asciidoc @@ -282,8 +282,8 @@ A space may also be a reserved character. For instance, if you have a synonym list which converts `"wi fi"` to `"wifi"`, a `query_string` search for `"wi fi"` would fail. The query string parser would interpret your query as a search for `"wi OR fi"`, while the token stored in your -index is actually `"wifi"`. Escaping the space will protect it from -being touched by the query string parser: `"wi\ fi"`. +index is actually `"wifi"`. The option `split_on_whitespace=false` will protect it from +being touched by the query string parser and will let the analysis run on the entire input (`"wi fi"`). **** ===== Empty Query From acb12ecfa735dbf52de9fca1bb0efaff000be0c6 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Wed, 2 Nov 2016 10:11:14 +0100 Subject: [PATCH 087/132] Protect BytesStreamOutput against overflows of the current number of written bytes. (#21174) Closes #21159 --- .../common/io/stream/BytesStreamOutput.java | 22 +++++++++---------- .../common/io/stream/BytesStreamsTests.java | 6 +++++ 2 files changed, 16 insertions(+), 12 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/common/io/stream/BytesStreamOutput.java b/core/src/main/java/org/elasticsearch/common/io/stream/BytesStreamOutput.java index 3de5c757ae1..c427cbfac49 100644 --- a/core/src/main/java/org/elasticsearch/common/io/stream/BytesStreamOutput.java +++ b/core/src/main/java/org/elasticsearch/common/io/stream/BytesStreamOutput.java @@ -69,7 +69,7 @@ public class BytesStreamOutput extends StreamOutput implements BytesStream { @Override public void writeByte(byte b) throws IOException { - ensureCapacity(count+1); + ensureCapacity(count + 1L); bytes.set(count, b); count++; } @@ -87,7 +87,7 @@ public class BytesStreamOutput extends StreamOutput implements BytesStream { } // get enough pages for new size - ensureCapacity(count+length); + ensureCapacity(((long) count) + length); // bulk copy bytes.set(count, b, offset, length); @@ -113,18 +113,13 @@ public class BytesStreamOutput extends StreamOutput implements BytesStream { } @Override - public void seek(long position) throws IOException { - if (position > Integer.MAX_VALUE) { - throw new IllegalArgumentException("position " + position + " > Integer.MAX_VALUE"); - } - - count = (int)position; - ensureCapacity(count); + public void seek(long position) { + ensureCapacity(position); + count = (int) position; } public void skip(int length) { - count += length; - ensureCapacity(count); + seek(((long) count) + length); } @Override @@ -156,7 +151,10 @@ public class BytesStreamOutput extends StreamOutput implements BytesStream { return bytes.ramBytesUsed(); } - private void ensureCapacity(int offset) { + private void ensureCapacity(long offset) { + if (offset > Integer.MAX_VALUE) { + throw new IllegalArgumentException(getClass().getSimpleName() + " cannot hold more than 2GB of data"); + } bytes = bigArrays.grow(bytes, offset); } diff --git a/core/src/test/java/org/elasticsearch/common/io/stream/BytesStreamsTests.java b/core/src/test/java/org/elasticsearch/common/io/stream/BytesStreamsTests.java index 44763d5ccce..f51a85b2f9a 100644 --- a/core/src/test/java/org/elasticsearch/common/io/stream/BytesStreamsTests.java +++ b/core/src/test/java/org/elasticsearch/common/io/stream/BytesStreamsTests.java @@ -238,6 +238,9 @@ public class BytesStreamsTests extends ESTestCase { assertEquals(position, out.position()); assertEquals(position, BytesReference.toBytes(out.bytes()).length); + IllegalArgumentException iae = expectThrows(IllegalArgumentException.class, () -> out.seek(Integer.MAX_VALUE + 1L)); + assertEquals("BytesStreamOutput cannot hold more than 2GB of data", iae.getMessage()); + out.close(); } @@ -251,6 +254,9 @@ public class BytesStreamsTests extends ESTestCase { out.skip(forward); assertEquals(position + forward, out.position()); + IllegalArgumentException iae = expectThrows(IllegalArgumentException.class, () -> out.skip(Integer.MAX_VALUE - 50)); + assertEquals("BytesStreamOutput cannot hold more than 2GB of data", iae.getMessage()); + out.close(); } From abc3ec657452578598ecd992272f3e88dab64d8a Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Wed, 2 Nov 2016 10:38:25 +0100 Subject: [PATCH 088/132] Remove special case in case no action filters are registered (#21251) Since we have the ingest node type, there is either IngestActionFilter or IngestProxyActionFilter registered, depending on whether the node is an ingest node or not. The special case that shortcuts the execution in case there are no filters is never exercised. --- .../action/support/TransportAction.java | 13 ++----------- 1 file changed, 2 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/support/TransportAction.java b/core/src/main/java/org/elasticsearch/action/support/TransportAction.java index e7d6d3b508f..b348d2ec317 100644 --- a/core/src/main/java/org/elasticsearch/action/support/TransportAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/TransportAction.java @@ -138,17 +138,8 @@ public abstract class TransportAction, Re listener = new TaskResultStoringActionListener<>(taskManager, task, listener); } - if (filters.length == 0) { - try { - doExecute(task, request, listener); - } catch(Exception e) { - logger.trace("Error during transport action execution.", e); - listener.onFailure(e); - } - } else { - RequestFilterChain requestFilterChain = new RequestFilterChain<>(this, logger); - requestFilterChain.proceed(task, actionName, request, listener); - } + RequestFilterChain requestFilterChain = new RequestFilterChain<>(this, logger); + requestFilterChain.proceed(task, actionName, request, listener); } protected void doExecute(Task task, Request request, ActionListener listener) { From 68b0e395b28ad1d395a3843015d9c88116399c63 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Wed, 2 Nov 2016 10:50:38 +0100 Subject: [PATCH 089/132] Add recommendations about getting consistent scores despite shards and replicas. (#21167) This is a topic that has triggered many questions recently so it would be good to have these recommendations documented. --- docs/reference/how-to/recipes.asciidoc | 79 ++++++++++++++++++++++++++ 1 file changed, 79 insertions(+) diff --git a/docs/reference/how-to/recipes.asciidoc b/docs/reference/how-to/recipes.asciidoc index 7b626554fa4..0bb158f88e8 100644 --- a/docs/reference/how-to/recipes.asciidoc +++ b/docs/reference/how-to/recipes.asciidoc @@ -223,3 +223,82 @@ In that case, since `ski` was in-between quotes, it was searched on the `1` matched. This allows users to mix exact search with stemmed search as they like. +[float] +[[consistent-scoring]] +=== Getting consistent scoring + +The fact that Elasticsearch operates with shards and replicas adds challenges +when it comes to having good scoring. + +[float] +==== Scores are not reproducible + +Say the same user runs the same request twice in a row and documents do not come +back in the same order both times, this is a pretty bad experience isn't it? +Unfortunately this is something that can happen if you have replicas +(`index.number_of_replicas` is greater than 0). The reason is that Elasticsearch +selects the shards that the query should go to in a round-robin fashion, so it +is quite likely if you run the same query twice in a row that it will go to +different copies of the same shard. + +Now why is it a problem? Index statistics are an important part of the score. +And these index statistics may be different across copies of the same shard +due to deleted documents. As you may know when documents are deleted or updated, +the old document is not immediately removed from the index, it is just marked +as deleted and it will only be removed from disk on the next time that the +segment this old document belongs to is merged. However for practical reasons, +those deleted documents are taken into account for index statistics. So imagine +that the primary shard just finished a large merge that removed lots of deleted +documents, then it might have index statistics that are sufficiently different +from the replica (which still have plenty of deleted documents) so that scores +are different too. + +The recommended way to work around this issue is to use a string that identifies +the user that is logged is (a user id or session id for instance) as a +<>. This ensures that all queries of a +given user are always going to hit the same shards, so scores remain more +consistent across queries. + +This work around has another benefit: when two documents have the same score, +they will be sorted by their internal Lucene doc id (which is unrelated to the +`_id` or `_uid`) by default. However these doc ids could be different across +copies of the same shard. So by always hitting the same shard, we would get +more consistent ordering of documents that have the same scores. + +[float] +==== Relevancy looks wrong + +If you notice that two documents with the same content get different scores or +that an exact match is not ranked first, then the issue might be related to +sharding. By default, Elasticsearch makes each shard responsible for producing +its own scores. However since index statistics are an important contributor to +the scores, this only works well if shards have similar index statistics. The +assumption is that since documents are routed evenly to shards by default, then +index statistics should be very similar and scoring would work as expected. +However in the event that you either + - use routing at index time, + - query multiple _indices_, + - or have too little data in your index +then there are good chances that all shards that are involved in the search +request do not have similar index statistics and relevancy could be bad. + +If you have a small dataset, the easiest way to work around this issue is to +index everything into an index that has a single shard +(`index.number_of_shards: 1`). Then index statistics will be the same for all +documents and scores will be consistent. + +Otherwise the recommended way to work around this issue is to use the +<> search type. This will make +Elasticsearch perform an inital round trip to all involved shards, asking +them for their index statistics relatively to the query, then the coordinating +node will merge those statistics and send the merged statistics alongside the +request when asking shards to perform the `query` phase, so that shards can +use these global statistics rather than their own statistics in order to do the +scoring. + +In most cases, this additional round trip should be very cheap. However in the +event that your query contains a very large number of fields/terms or fuzzy +queries, beware that gathering statistics alone might not be cheap since all +terms have to be looked up in the terms dictionaries in order to look up +statistics. + From cf1457ed22113fe01ff2002392e2d5d4030dc07c Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 2 Nov 2016 12:24:20 +0100 Subject: [PATCH 090/132] Allow skip test by version OR feature (#21240) Today these two are considered mutual exclusive but they are not in practice. For instance a mixed version cluster might not return a given warning depending on which node we talk to but on the other hand some runners might not even support warnings at all so the test might be skipped either by version or by feature. --- .../rest/yaml/ESClientYamlSuiteTestCase.java | 19 ++------ .../rest/yaml/parser/SkipSectionParser.java | 3 -- .../test/rest/yaml/section/SkipSection.java | 28 +++++++---- .../yaml/parser/SkipSectionParserTests.java | 14 +++--- .../rest/yaml/section/SkipSectionTests.java | 47 +++++++++++++++++++ 5 files changed, 77 insertions(+), 34 deletions(-) create mode 100644 test/framework/src/test/java/org/elasticsearch/test/rest/yaml/section/SkipSectionTests.java diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ESClientYamlSuiteTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ESClientYamlSuiteTestCase.java index 2e29721f06e..f44558d7567 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ESClientYamlSuiteTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ESClientYamlSuiteTestCase.java @@ -108,7 +108,7 @@ public abstract class ESClientYamlSuiteTestCase extends ESRestTestCase { for (String entry : blacklist) { this.blacklistPathMatchers.add(new BlacklistedPathPatternMatcher(entry)); } - + } @Override @@ -267,27 +267,16 @@ public abstract class ESClientYamlSuiteTestCase extends ESRestTestCase { restTestExecutionContext.clear(); //skip test if the whole suite (yaml file) is disabled - assumeFalse(buildSkipMessage(testCandidate.getSuitePath(), testCandidate.getSetupSection().getSkipSection()), + assumeFalse(testCandidate.getSetupSection().getSkipSection().getSkipMessage(testCandidate.getSuitePath()), testCandidate.getSetupSection().getSkipSection().skip(restTestExecutionContext.esVersion())); //skip test if the whole suite (yaml file) is disabled - assumeFalse(buildSkipMessage(testCandidate.getSuitePath(), testCandidate.getTeardownSection().getSkipSection()), + assumeFalse(testCandidate.getTeardownSection().getSkipSection().getSkipMessage(testCandidate.getSuitePath()), testCandidate.getTeardownSection().getSkipSection().skip(restTestExecutionContext.esVersion())); //skip test if test section is disabled - assumeFalse(buildSkipMessage(testCandidate.getTestPath(), testCandidate.getTestSection().getSkipSection()), + assumeFalse(testCandidate.getTestSection().getSkipSection().getSkipMessage(testCandidate.getTestPath()), testCandidate.getTestSection().getSkipSection().skip(restTestExecutionContext.esVersion())); } - private static String buildSkipMessage(String description, SkipSection skipSection) { - StringBuilder messageBuilder = new StringBuilder(); - if (skipSection.isVersionCheck()) { - messageBuilder.append("[").append(description).append("] skipped, reason: [").append(skipSection.getReason()).append("] "); - } else { - messageBuilder.append("[").append(description).append("] skipped, reason: features ") - .append(skipSection.getFeatures()).append(" not supported"); - } - return messageBuilder.toString(); - } - public void test() throws IOException { //let's check that there is something to run, otherwise there might be a problem with the test section if (testCandidate.getTestSection().getExecutableSections().size() == 0) { diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/parser/SkipSectionParser.java b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/parser/SkipSectionParser.java index 31451dee247..b73edf7d2c6 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/parser/SkipSectionParser.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/parser/SkipSectionParser.java @@ -70,9 +70,6 @@ public class SkipSectionParser implements ClientYamlTestFragmentParser features; private final String reason; - + private SkipSection() { this.lowerVersion = null; this.upperVersion = null; @@ -49,7 +49,6 @@ public class SkipSection { public SkipSection(String versionRange, List features, String reason) { assert features != null; - assert versionRange != null && features.isEmpty() || versionRange == null && features.isEmpty() == false; Version[] versions = parseVersionRange(versionRange); this.lowerVersion = versions[0]; this.upperVersion = versions[1]; @@ -60,7 +59,7 @@ public class SkipSection { public Version getLowerVersion() { return lowerVersion; } - + public Version getUpperVersion() { return upperVersion; } @@ -77,11 +76,10 @@ public class SkipSection { if (isEmpty()) { return false; } - if (isVersionCheck()) { - return currentVersion.onOrAfter(lowerVersion) && currentVersion.onOrBefore(upperVersion); - } else { - return Features.areAllSupported(features) == false; - } + boolean skip = lowerVersion != null && upperVersion != null && currentVersion.onOrAfter(lowerVersion) + && currentVersion.onOrBefore(upperVersion); + skip |= Features.areAllSupported(features) == false; + return skip; } public boolean isVersionCheck() { @@ -91,7 +89,7 @@ public class SkipSection { public boolean isEmpty() { return EMPTY.equals(this); } - + private Version[] parseVersionRange(String versionRange) { if (versionRange == null) { return new Version[] { null, null }; @@ -111,4 +109,16 @@ public class SkipSection { upper.isEmpty() ? Version.CURRENT : Version.fromString(upper) }; } + + public String getSkipMessage(String description) { + StringBuilder messageBuilder = new StringBuilder(); + messageBuilder.append("[").append(description).append("] skipped,"); + if (reason != null) { + messageBuilder.append(" reason: [").append(getReason()).append("]"); + } + if (features.isEmpty() == false) { + messageBuilder.append(" unsupported features ").append(getFeatures()); + } + return messageBuilder.toString(); + } } diff --git a/test/framework/src/test/java/org/elasticsearch/test/rest/yaml/parser/SkipSectionParserTests.java b/test/framework/src/test/java/org/elasticsearch/test/rest/yaml/parser/SkipSectionParserTests.java index f5d46cdd3d6..7473e393e5c 100644 --- a/test/framework/src/test/java/org/elasticsearch/test/rest/yaml/parser/SkipSectionParserTests.java +++ b/test/framework/src/test/java/org/elasticsearch/test/rest/yaml/parser/SkipSectionParserTests.java @@ -26,6 +26,8 @@ import org.elasticsearch.test.rest.yaml.parser.ClientYamlTestSuiteParseContext; import org.elasticsearch.test.rest.yaml.parser.SkipSectionParser; import org.elasticsearch.test.rest.yaml.section.SkipSection; +import java.util.Arrays; + import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; @@ -108,13 +110,11 @@ public class SkipSectionParserTests extends AbstractParserTestCase { ); SkipSectionParser skipSectionParser = new SkipSectionParser(); - - try { - skipSectionParser.parse(new ClientYamlTestSuiteParseContext("api", "suite", parser)); - fail("Expected RestTestParseException"); - } catch (ClientYamlTestParseException e) { - assertThat(e.getMessage(), is("version or features are mutually exclusive")); - } + SkipSection parse = skipSectionParser.parse(new ClientYamlTestSuiteParseContext("api", "suite", parser)); + assertEquals(VersionUtils.getFirstVersion(), parse.getLowerVersion()); + assertEquals(Version.fromString("0.90.2"), parse.getUpperVersion()); + assertEquals(Arrays.asList("regex"), parse.getFeatures()); + assertEquals("Delete ignores the parent param", parse.getReason()); } public void testParseSkipSectionNoReason() throws Exception { diff --git a/test/framework/src/test/java/org/elasticsearch/test/rest/yaml/section/SkipSectionTests.java b/test/framework/src/test/java/org/elasticsearch/test/rest/yaml/section/SkipSectionTests.java new file mode 100644 index 00000000000..c8f7b351282 --- /dev/null +++ b/test/framework/src/test/java/org/elasticsearch/test/rest/yaml/section/SkipSectionTests.java @@ -0,0 +1,47 @@ +/* + * 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.test.rest.yaml.section; + +import org.elasticsearch.Version; +import org.elasticsearch.test.ESTestCase; + +import java.util.Arrays; +import java.util.Collections; + +public class SkipSectionTests extends ESTestCase { + + public void testSkip() { + SkipSection section = new SkipSection("2.0.0 - 2.1.0", randomBoolean() ? Collections.emptyList() : + Arrays.asList("warnings"), "foobar"); + assertFalse(section.skip(Version.CURRENT)); + assertTrue(section.skip(Version.V_2_0_0)); + section = new SkipSection(randomBoolean() ? null : "2.0.0 - 2.1.0", Arrays.asList("boom"), "foobar"); + assertTrue(section.skip(Version.CURRENT)); + } + + public void testMessage() { + SkipSection section = new SkipSection("2.0.0 - 2.1.0", Arrays.asList("warnings"), "foobar"); + assertEquals("[FOOBAR] skipped, reason: [foobar] unsupported features [warnings]", section.getSkipMessage("FOOBAR")); + section = new SkipSection(null, Arrays.asList("warnings"), "foobar"); + assertEquals("[FOOBAR] skipped, reason: [foobar] unsupported features [warnings]", section.getSkipMessage("FOOBAR")); + section = new SkipSection(null, Arrays.asList("warnings"), null); + assertEquals("[FOOBAR] skipped, unsupported features [warnings]", section.getSkipMessage("FOOBAR")); + } +} From 2187b65809e94276b17295f681dada54f7c2809f Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Wed, 2 Nov 2016 13:38:21 +0100 Subject: [PATCH 091/132] TransportShardBulkAction: add the exception to the message of an assertion about it's type --- .../elasticsearch/action/bulk/TransportShardBulkAction.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 18fe1c15d78..0f3f1f86cfa 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -21,15 +21,15 @@ package org.elasticsearch.action.bulk; import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.logging.log4j.util.Supplier; -import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.ActionFilters; -import org.elasticsearch.action.support.replication.TransportWriteAction; import org.elasticsearch.action.support.replication.ReplicationResponse.ShardInfo; +import org.elasticsearch.action.support.replication.TransportWriteAction; import org.elasticsearch.action.update.UpdateHelper; import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.action.update.UpdateResponse; @@ -346,7 +346,7 @@ public class TransportShardBulkAction extends TransportWriteAction Date: Wed, 2 Nov 2016 13:43:39 +0100 Subject: [PATCH 092/132] Change ClusterState and PendingClusterTasksResponse's toString() to their prettyPrint format (#21245) The current XContent output is much harder to read than the prettyPrint format. This commit folds prettyPrint into toString and removes it. --- .../tasks/PendingClusterTasksResponse.java | 17 +-------- .../elasticsearch/cluster/ClusterState.java | 34 ++++++++--------- .../action/shard/ShardStateAction.java | 2 +- .../cluster/block/ClusterBlocks.java | 3 +- .../cluster/node/DiscoveryNodes.java | 9 ----- .../cluster/routing/RoutingNodes.java | 3 +- .../cluster/routing/RoutingService.java | 4 +- .../cluster/routing/RoutingTable.java | 3 +- .../allocation/IndexMetaDataUpdater.java | 2 +- .../cluster/service/ClusterService.java | 16 ++++---- .../discovery/zen/NodeJoinController.java | 2 +- .../BroadcastReplicationTests.java | 15 ++++---- .../ReplicationOperationTests.java | 7 ++-- .../TransportReplicationActionTests.java | 22 +++++------ .../elasticsearch/cluster/NoMasterNodeIT.java | 2 +- .../health/ClusterStateHealthTests.java | 4 +- .../cluster/node/DiscoveryNodesTests.java | 4 +- .../cluster/routing/PrimaryAllocationIT.java | 4 +- .../cluster/routing/UnassignedInfoTests.java | 2 +- .../allocation/AddIncrementallyTests.java | 6 +-- .../NodeVersionAllocationDeciderTests.java | 6 +-- .../RandomAllocationDeciderTests.java | 2 +- .../allocation/StartedShardsRoutingTests.java | 7 ++-- .../ClusterSerializationTests.java | 6 +-- .../ClusterStateToStringTests.java | 5 ++- .../DiscoveryWithServiceDisruptionsIT.java | 12 +++--- .../discovery/ZenUnicastDiscoveryIT.java | 2 +- .../zen/NodeJoinControllerTests.java | 4 +- .../zen/PublishClusterStateActionTests.java | 6 +-- .../index/store/CorruptedFileIT.java | 11 ++++-- ...ClusterStateServiceRandomUpdatesTests.java | 6 +-- .../recovery/FullRollingRestartIT.java | 6 ++- .../cluster/ESAllocationTestCase.java | 2 +- .../elasticsearch/test/ESIntegTestCase.java | 38 +++++++++---------- .../test/ESSingleNodeTestCase.java | 3 +- 35 files changed, 127 insertions(+), 150 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/admin/cluster/tasks/PendingClusterTasksResponse.java b/core/src/main/java/org/elasticsearch/action/admin/cluster/tasks/PendingClusterTasksResponse.java index ae4a59fa83f..bb1afe5e19e 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/cluster/tasks/PendingClusterTasksResponse.java +++ b/core/src/main/java/org/elasticsearch/action/admin/cluster/tasks/PendingClusterTasksResponse.java @@ -25,7 +25,6 @@ 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.common.xcontent.XContentFactory; import java.io.IOException; import java.util.ArrayList; @@ -59,7 +58,8 @@ public class PendingClusterTasksResponse extends ActionResponse implements Itera return pendingTasks.iterator(); } - public String prettyPrint() { + @Override + public String toString() { StringBuilder sb = new StringBuilder(); sb.append("tasks: (").append(pendingTasks.size()).append("):\n"); for (PendingClusterTask pendingClusterTask : this) { @@ -68,19 +68,6 @@ public class PendingClusterTasksResponse extends ActionResponse implements Itera return sb.toString(); } - @Override - public String toString() { - try { - XContentBuilder builder = XContentFactory.jsonBuilder().prettyPrint(); - builder.startObject(); - toXContent(builder, EMPTY_PARAMS); - builder.endObject(); - return builder.string(); - } catch (IOException e) { - return "{ \"error\" : \"" + e.getMessage() + "\"}"; - } - } - @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startArray(Fields.TASKS); diff --git a/core/src/main/java/org/elasticsearch/cluster/ClusterState.java b/core/src/main/java/org/elasticsearch/cluster/ClusterState.java index 3bbd56d9f49..82fb6476264 100644 --- a/core/src/main/java/org/elasticsearch/cluster/ClusterState.java +++ b/core/src/main/java/org/elasticsearch/cluster/ClusterState.java @@ -274,15 +274,16 @@ public class ClusterState implements ToXContent, Diffable { return routingNodes; } - public String prettyPrint() { + @Override + public String toString() { StringBuilder sb = new StringBuilder(); sb.append("cluster uuid: ").append(metaData.clusterUUID()).append("\n"); sb.append("version: ").append(version).append("\n"); sb.append("state uuid: ").append(stateUUID).append("\n"); sb.append("from_diff: ").append(wasReadFromDiff).append("\n"); sb.append("meta data version: ").append(metaData.version()).append("\n"); + final String TAB = " "; for (IndexMetaData indexMetaData : metaData) { - final String TAB = " "; sb.append(TAB).append(indexMetaData.getIndex()); sb.append(": v[").append(indexMetaData.getVersion()).append("]\n"); for (int shard = 0; shard < indexMetaData.getNumberOfShards(); shard++) { @@ -291,24 +292,19 @@ public class ClusterState implements ToXContent, Diffable { sb.append("isa_ids ").append(indexMetaData.inSyncAllocationIds(shard)).append("\n"); } } - sb.append(blocks().prettyPrint()); - sb.append(nodes().prettyPrint()); - sb.append(routingTable().prettyPrint()); - sb.append(getRoutingNodes().prettyPrint()); - return sb.toString(); - } - - @Override - public String toString() { - try { - XContentBuilder builder = XContentFactory.jsonBuilder().prettyPrint(); - builder.startObject(); - toXContent(builder, EMPTY_PARAMS); - builder.endObject(); - return builder.string(); - } catch (IOException e) { - return "{ \"error\" : \"" + e.getMessage() + "\"}"; + sb.append(blocks()); + sb.append(nodes()); + sb.append(routingTable()); + sb.append(getRoutingNodes()); + if (customs.isEmpty() == false) { + sb.append("customs:\n"); + for (ObjectObjectCursor cursor : customs) { + final String type = cursor.key; + final Custom custom = cursor.value; + sb.append(TAB).append(type).append(": ").append(custom); + } } + return sb.toString(); } /** diff --git a/core/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java b/core/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java index ce6473ecb42..ee56d7a61a1 100644 --- a/core/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java +++ b/core/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java @@ -164,7 +164,7 @@ public class ShardStateAction extends AbstractComponent { @Override public void onNewClusterState(ClusterState state) { if (logger.isTraceEnabled()) { - logger.trace("new cluster state [{}] after waiting for master election to fail shard entry [{}]", state.prettyPrint(), shardEntry); + logger.trace("new cluster state [{}] after waiting for master election to fail shard entry [{}]", state, shardEntry); } sendShardAction(actionName, observer, shardEntry, listener); } diff --git a/core/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java b/core/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java index e6f04c8702c..12e6ee0f7ec 100644 --- a/core/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java +++ b/core/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java @@ -199,7 +199,8 @@ public class ClusterBlocks extends AbstractDiffable { return new ClusterBlockException(unmodifiableSet(blocks.collect(toSet()))); } - public String prettyPrint() { + @Override + public String toString() { if (global.isEmpty() && indices().isEmpty()) { return ""; } diff --git a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java index f3a8932f53c..895195d35b3 100644 --- a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java +++ b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java @@ -26,7 +26,6 @@ import org.elasticsearch.Version; import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.common.Booleans; import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -398,14 +397,6 @@ public class DiscoveryNodes extends AbstractDiffable implements @Override public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append("{"); - sb.append(Strings.collectionToDelimitedString(this, ",")); - sb.append("}"); - return sb.toString(); - } - - public String prettyPrint() { StringBuilder sb = new StringBuilder(); sb.append("nodes: \n"); for (DiscoveryNode node : this) { diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java b/core/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java index bd5113029c4..ddb7969f60a 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java @@ -391,7 +391,8 @@ public class RoutingNodes implements Iterable { return shards; } - public String prettyPrint() { + @Override + public String toString() { StringBuilder sb = new StringBuilder("routing_nodes:\n"); for (RoutingNode routingNode : this) { sb.append(routingNode.prettyPrint()); diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/RoutingService.java b/core/src/main/java/org/elasticsearch/cluster/routing/RoutingService.java index 9dd2cc72da8..1c3d629a72f 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/RoutingService.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/RoutingService.java @@ -109,7 +109,7 @@ public class RoutingService extends AbstractLifecycleComponent { rerouting.set(false); ClusterState state = clusterService.state(); if (logger.isTraceEnabled()) { - logger.error((Supplier) () -> new ParameterizedMessage("unexpected failure during [{}], current state:\n{}", source, state.prettyPrint()), e); + logger.error((Supplier) () -> new ParameterizedMessage("unexpected failure during [{}], current state:\n{}", source, state), e); } else { logger.error((Supplier) () -> new ParameterizedMessage("unexpected failure during [{}], current state version [{}]", source, state.version()), e); } @@ -118,7 +118,7 @@ public class RoutingService extends AbstractLifecycleComponent { } catch (Exception e) { rerouting.set(false); ClusterState state = clusterService.state(); - logger.warn((Supplier) () -> new ParameterizedMessage("failed to reroute routing table, current state:\n{}", state.prettyPrint()), e); + logger.warn((Supplier) () -> new ParameterizedMessage("failed to reroute routing table, current state:\n{}", state), e); } } } diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java b/core/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java index 2d960ce0450..051fd12a12b 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java @@ -613,7 +613,8 @@ public class RoutingTable implements Iterable, Diffable entry : indicesRouting) { sb.append(entry.value.prettyPrint()).append('\n'); diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/IndexMetaDataUpdater.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/IndexMetaDataUpdater.java index 52ace3744cd..fa30a102bf6 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/IndexMetaDataUpdater.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/IndexMetaDataUpdater.java @@ -233,7 +233,7 @@ public class IndexMetaDataUpdater extends RoutingChangesObserver.AbstractRouting Set oldInSyncAllocations = oldIndexMetaData.inSyncAllocationIds(shardNumber); Set idsToRemove = shardEntry.getValue().stream().map(e -> e.getAllocationId()).collect(Collectors.toSet()); assert idsToRemove.stream().allMatch(id -> oldRoutingTable.getByAllocationId(shardEntry.getKey(), id) == null) : - "removing stale ids: " + idsToRemove + ", some of which have still a routing entry: " + oldRoutingTable.prettyPrint(); + "removing stale ids: " + idsToRemove + ", some of which have still a routing entry: " + oldRoutingTable; Set remainingInSyncAllocations = Sets.difference(oldInSyncAllocations, idsToRemove); assert remainingInSyncAllocations.isEmpty() == false : "Set of in-sync ids cannot become empty for shard " + shardEntry.getKey() + " (before: " + oldInSyncAllocations + ", ids to remove: " + idsToRemove + ")"; diff --git a/core/src/main/java/org/elasticsearch/cluster/service/ClusterService.java b/core/src/main/java/org/elasticsearch/cluster/service/ClusterService.java index fe2adf1f4c6..4f637e05648 100644 --- a/core/src/main/java/org/elasticsearch/cluster/service/ClusterService.java +++ b/core/src/main/java/org/elasticsearch/cluster/service/ClusterService.java @@ -562,9 +562,9 @@ public class ClusterService extends AbstractLifecycleComponent { executionTime, previousClusterState.version(), tasksSummary, - previousClusterState.nodes().prettyPrint(), - previousClusterState.routingTable().prettyPrint(), - previousClusterState.getRoutingNodes().prettyPrint()), + previousClusterState.nodes(), + previousClusterState.routingTable(), + previousClusterState.getRoutingNodes()), e); } warnAboutSlowTaskIfNeeded(executionTime, tasksSummary); @@ -656,7 +656,7 @@ public class ClusterService extends AbstractLifecycleComponent { newClusterState.status(ClusterState.ClusterStateStatus.BEING_APPLIED); if (logger.isTraceEnabled()) { - logger.trace("cluster state updated, source [{}]\n{}", tasksSummary, newClusterState.prettyPrint()); + logger.trace("cluster state updated, source [{}]\n{}", tasksSummary, newClusterState); } else if (logger.isDebugEnabled()) { logger.debug("cluster state updated, version [{}], source [{}]", newClusterState.version(), tasksSummary); } @@ -759,7 +759,7 @@ public class ClusterService extends AbstractLifecycleComponent { TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, TimeValue.nsecToMSec(currentTimeInNanos() - startTimeNS))); final long version = newClusterState.version(); final String stateUUID = newClusterState.stateUUID(); - final String prettyPrint = newClusterState.prettyPrint(); + final String fullState = newClusterState.toString(); logger.warn( (Supplier) () -> new ParameterizedMessage( "failed to apply updated cluster state in [{}]:\nversion [{}], uuid [{}], source [{}]\n{}", @@ -767,7 +767,7 @@ public class ClusterService extends AbstractLifecycleComponent { version, stateUUID, tasksSummary, - prettyPrint), + fullState), e); // TODO: do we want to call updateTask.onFailure here? } @@ -826,9 +826,7 @@ public class ClusterService extends AbstractLifecycleComponent { (Supplier) () -> new ParameterizedMessage( "exception thrown by listener while notifying of cluster state processed from [{}], old cluster state:\n" + "{}\nnew cluster state:\n{}", - source, - oldState.prettyPrint(), - newState.prettyPrint()), + source, oldState, newState), e); } } diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java b/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java index 3c8deee7c5f..6d77e2f48fe 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java @@ -464,7 +464,7 @@ public class NodeJoinController extends AbstractComponent { } private ClusterState.Builder becomeMasterAndTrimConflictingNodes(ClusterState currentState, List joiningNodes) { - assert currentState.nodes().getMasterNodeId() == null : currentState.prettyPrint(); + assert currentState.nodes().getMasterNodeId() == null : currentState; DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(currentState.nodes()); nodesBuilder.masterNodeId(currentState.nodes().getLocalNodeId()); ClusterBlocks clusterBlocks = ClusterBlocks.builder().blocks(currentState.blocks()) diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java b/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java index cfbeb79ef7d..260f70e19ed 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java @@ -26,7 +26,6 @@ import org.elasticsearch.action.UnavailableShardsException; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.flush.FlushResponse; import org.elasticsearch.action.admin.indices.flush.TransportFlushAction; -import org.elasticsearch.action.support.ActionFilter; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.broadcast.BroadcastRequest; import org.elasticsearch.action.support.broadcast.BroadcastResponse; @@ -101,7 +100,7 @@ public class BroadcastReplicationTests extends ESTestCase { transportService.start(); transportService.acceptIncomingRequests(); broadcastReplicationAction = new TestBroadcastReplicationAction(Settings.EMPTY, threadPool, clusterService, transportService, - new ActionFilters(new HashSet()), new IndexNameExpressionResolver(Settings.EMPTY), null); + new ActionFilters(new HashSet<>()), new IndexNameExpressionResolver(Settings.EMPTY), null); } @After @@ -120,7 +119,7 @@ public class BroadcastReplicationTests extends ESTestCase { final String index = "test"; setState(clusterService, state(index, randomBoolean(), randomBoolean() ? ShardRoutingState.INITIALIZING : ShardRoutingState.UNASSIGNED, ShardRoutingState.UNASSIGNED)); - logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); + logger.debug("--> using initial state:\n{}", clusterService.state()); Future response = (broadcastReplicationAction.execute(new DummyBroadcastRequest().indices(index))); for (Tuple> shardRequests : broadcastReplicationAction.capturedShardRequests) { if (randomBoolean()) { @@ -139,11 +138,11 @@ public class BroadcastReplicationTests extends ESTestCase { final String index = "test"; setState(clusterService, state(index, randomBoolean(), ShardRoutingState.STARTED)); - logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); + logger.debug("--> using initial state:\n{}", clusterService.state()); Future response = (broadcastReplicationAction.execute(new DummyBroadcastRequest().indices(index))); for (Tuple> shardRequests : broadcastReplicationAction.capturedShardRequests) { ReplicationResponse replicationResponse = new ReplicationResponse(); - replicationResponse.setShardInfo(new ReplicationResponse.ShardInfo(1, 1, new ReplicationResponse.ShardInfo.Failure[0])); + replicationResponse.setShardInfo(new ReplicationResponse.ShardInfo(1, 1)); shardRequests.v2().onResponse(replicationResponse); } logger.info("total shards: {}, ", response.get().getTotalShards()); @@ -154,7 +153,7 @@ public class BroadcastReplicationTests extends ESTestCase { final String index = "test"; int numShards = 1 + randomInt(3); setState(clusterService, stateWithAssignedPrimariesAndOneReplica(index, numShards)); - logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); + logger.debug("--> using initial state:\n{}", clusterService.state()); Future response = (broadcastReplicationAction.execute(new DummyBroadcastRequest().indices(index))); int succeeded = 0; int failed = 0; @@ -184,7 +183,7 @@ public class BroadcastReplicationTests extends ESTestCase { public void testNoShards() throws InterruptedException, ExecutionException, IOException { setState(clusterService, stateWithNoShard()); - logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); + logger.debug("--> using initial state:\n{}", clusterService.state()); BroadcastResponse response = executeAndAssertImmediateResponse(broadcastReplicationAction, new DummyBroadcastRequest()); assertBroadcastResponse(0, 0, 0, response, null); } @@ -194,7 +193,7 @@ public class BroadcastReplicationTests extends ESTestCase { final ShardId shardId = new ShardId(index, "_na_", 0); ClusterState clusterState = state(index, randomBoolean(), randomBoolean() ? ShardRoutingState.INITIALIZING : ShardRoutingState.UNASSIGNED, ShardRoutingState.UNASSIGNED); - logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); + logger.debug("--> using initial state:\n{}", clusterService.state()); List shards = broadcastReplicationAction.shards(new DummyBroadcastRequest().indices(shardId.getIndexName()), clusterState); assertThat(shards.size(), equalTo(1)); assertThat(shards.get(0), equalTo(shardId)); diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java b/core/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java index b5edc1b53c5..a49c2ae978e 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java @@ -256,7 +256,7 @@ public class ReplicationOperationTests extends ESTestCase { final ClusterState initialState, final ClusterState changedState) throws Exception { AtomicReference state = new AtomicReference<>(initialState); - logger.debug("--> using initial state:\n{}", state.get().prettyPrint()); + logger.debug("--> using initial state:\n{}", state.get()); final long primaryTerm = initialState.getMetaData().index(shardId.getIndexName()).primaryTerm(shardId.id()); final ShardRouting primaryShard = state.get().routingTable().shardRoutingTable(shardId).primaryShard(); final TestPrimary primary = new TestPrimary(primaryShard, primaryTerm) { @@ -264,7 +264,7 @@ public class ReplicationOperationTests extends ESTestCase { public Result perform(Request request) throws Exception { Result result = super.perform(request); state.set(changedState); - logger.debug("--> state after primary operation:\n{}", state.get().prettyPrint()); + logger.debug("--> state after primary operation:\n{}", state.get()); return result; } }; @@ -303,8 +303,7 @@ public class ReplicationOperationTests extends ESTestCase { logger.debug("using active shard count of [{}], assigned shards [{}], total shards [{}]." + " expecting op to [{}]. using state: \n{}", request.waitForActiveShards(), 1 + assignedReplicas, 1 + assignedReplicas + unassignedReplicas, - passesActiveShardCheck ? "succeed" : "retry", - state.prettyPrint()); + passesActiveShardCheck ? "succeed" : "retry", state); final long primaryTerm = state.metaData().index(index).primaryTerm(shardId.id()); final IndexShardRoutingTable shardRoutingTable = state.routingTable().index(index).shard(shardId.id()); PlainActionFuture listener = new PlainActionFuture<>(); diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java b/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java index 1caac899005..709c4b830ea 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java @@ -77,7 +77,7 @@ import org.junit.Before; import org.junit.BeforeClass; import java.io.IOException; -import java.util.Arrays; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Locale; @@ -227,7 +227,7 @@ public class TransportReplicationActionTests extends ESTestCase { randomBoolean() ? ShardRoutingState.INITIALIZING : ShardRoutingState.UNASSIGNED)); ReplicationTask task = maybeTask(); - logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); + logger.debug("--> using initial state:\n{}", clusterService.state()); Request request = new Request(shardId).timeout("1ms"); PlainActionFuture listener = new PlainActionFuture<>(); @@ -246,7 +246,7 @@ public class TransportReplicationActionTests extends ESTestCase { assertTrue(request.isRetrySet.get()); setState(clusterService, state(index, true, ShardRoutingState.STARTED)); - logger.debug("--> primary assigned state:\n{}", clusterService.state().prettyPrint()); + logger.debug("--> primary assigned state:\n{}", clusterService.state()); final IndexShardRoutingTable shardRoutingTable = clusterService.state().routingTable().index(index).shard(shardId.id()); final String primaryNodeId = shardRoutingTable.primaryShard().currentNodeId(); @@ -275,7 +275,7 @@ public class TransportReplicationActionTests extends ESTestCase { String relocationTargetNode = state.getRoutingTable().shardRoutingTable(shardId).primaryShard().relocatingNodeId(); state = ClusterState.builder(state).nodes(DiscoveryNodes.builder(state.nodes()).localNodeId(relocationTargetNode)).build(); setState(clusterService, state); - logger.debug("--> relocation ongoing state:\n{}", clusterService.state().prettyPrint()); + logger.debug("--> relocation ongoing state:\n{}", clusterService.state()); Request request = new Request(shardId).timeout("1ms").routedBasedOnClusterVersion(clusterService.state().version() + 1); PlainActionFuture listener = new PlainActionFuture<>(); @@ -295,10 +295,10 @@ public class TransportReplicationActionTests extends ESTestCase { ShardRouting relocationTarget = clusterService.state().getRoutingTable().shardRoutingTable(shardId) .shardsWithState(ShardRoutingState.INITIALIZING).get(0); AllocationService allocationService = ESAllocationTestCase.createAllocationService(); - ClusterState updatedState = allocationService.applyStartedShards(state, Arrays.asList(relocationTarget)); + ClusterState updatedState = allocationService.applyStartedShards(state, Collections.singletonList(relocationTarget)); setState(clusterService, updatedState); - logger.debug("--> relocation complete state:\n{}", clusterService.state().prettyPrint()); + logger.debug("--> relocation complete state:\n{}", clusterService.state()); IndexShardRoutingTable shardRoutingTable = clusterService.state().routingTable().index(index).shard(shardId.id()); final String primaryNodeId = shardRoutingTable.primaryShard().currentNodeId(); @@ -315,7 +315,7 @@ public class TransportReplicationActionTests extends ESTestCase { // no replicas in oder to skip the replication part setState(clusterService, state(index, true, randomBoolean() ? ShardRoutingState.INITIALIZING : ShardRoutingState.UNASSIGNED)); - logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); + logger.debug("--> using initial state:\n{}", clusterService.state()); Request request = new Request(new ShardId("unknown_index", "_na_", 0)).timeout("1ms"); PlainActionFuture listener = new PlainActionFuture<>(); ReplicationTask task = maybeTask(); @@ -339,7 +339,7 @@ public class TransportReplicationActionTests extends ESTestCase { final ShardId shardId = new ShardId(index, "_na_", 0); // no replicas in order to skip the replication part setState(clusterService, stateWithActivePrimary(index, true, randomInt(3))); - logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); + logger.debug("--> using initial state:\n{}", clusterService.state()); Request request = new Request(shardId); boolean timeout = randomBoolean(); if (timeout) { @@ -396,7 +396,7 @@ public class TransportReplicationActionTests extends ESTestCase { ReplicationTask task = maybeTask(); setState(clusterService, stateWithActivePrimary(index, randomBoolean(), 3)); - logger.debug("using state: \n{}", clusterService.state().prettyPrint()); + logger.debug("using state: \n{}", clusterService.state()); final IndexShardRoutingTable shardRoutingTable = clusterService.state().routingTable().index(index).shard(shardId.id()); final String primaryNodeId = shardRoutingTable.primaryShard().currentNodeId(); @@ -545,7 +545,7 @@ public class TransportReplicationActionTests extends ESTestCase { final String index = "test"; final ShardId shardId = new ShardId(index, "_na_", 0); ClusterState state = stateWithActivePrimary(index, true, 1 + randomInt(3), randomInt(2)); - logger.info("using state: {}", state.prettyPrint()); + logger.info("using state: {}", state); setState(clusterService, state); // check that at unknown node fails @@ -651,7 +651,7 @@ public class TransportReplicationActionTests extends ESTestCase { // no replica, we only want to test on primary final ClusterState state = state(index, true, ShardRoutingState.STARTED); setState(clusterService, state); - logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); + logger.debug("--> using initial state:\n{}", clusterService.state()); final ShardRouting primaryShard = state.routingTable().shardRoutingTable(shardId).primaryShard(); Request request = new Request(shardId); PlainActionFuture listener = new PlainActionFuture<>(); diff --git a/core/src/test/java/org/elasticsearch/cluster/NoMasterNodeIT.java b/core/src/test/java/org/elasticsearch/cluster/NoMasterNodeIT.java index 86b64febb04..fac8d5f7b63 100644 --- a/core/src/test/java/org/elasticsearch/cluster/NoMasterNodeIT.java +++ b/core/src/test/java/org/elasticsearch/cluster/NoMasterNodeIT.java @@ -219,7 +219,7 @@ public class NoMasterNodeIT extends ESIntegTestCase { ensureSearchable("test1", "test2"); ClusterStateResponse clusterState = client().admin().cluster().prepareState().get(); - logger.info("Cluster state:\n{}", clusterState.getState().prettyPrint()); + logger.info("Cluster state:\n{}", clusterState.getState()); internalCluster().stopRandomDataNode(); assertTrue(awaitBusy(() -> { diff --git a/core/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java b/core/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java index 1dd1abf4ce1..eb5c88d7e83 100644 --- a/core/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java @@ -277,9 +277,9 @@ public class ClusterStateHealthTests extends ESTestCase { // if the inactive primaries are due solely to recovery (not failed allocation or previously being allocated) // then cluster health is YELLOW, otherwise RED if (primaryInactiveDueToRecovery(indexName, clusterState)) { - assertThat("clusterState is:\n" + clusterState.prettyPrint(), health.getStatus(), equalTo(ClusterHealthStatus.YELLOW)); + assertThat("clusterState is:\n" + clusterState, health.getStatus(), equalTo(ClusterHealthStatus.YELLOW)); } else { - assertThat("clusterState is:\n" + clusterState.prettyPrint(), health.getStatus(), equalTo(ClusterHealthStatus.RED)); + assertThat("clusterState is:\n" + clusterState, health.getStatus(), equalTo(ClusterHealthStatus.RED)); } } } diff --git a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java index a16520faeb1..342919fb881 100644 --- a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java @@ -135,8 +135,8 @@ public class DiscoveryNodesTests extends ESTestCase { final DiscoveryNodes discoNodesA = builderA.build(); final DiscoveryNodes discoNodesB = builderB.build(); - logger.info("nodes A: {}", discoNodesA.prettyPrint()); - logger.info("nodes B: {}", discoNodesB.prettyPrint()); + logger.info("nodes A: {}", discoNodesA); + logger.info("nodes B: {}", discoNodesB); DiscoveryNodes.Delta delta = discoNodesB.delta(discoNodesA); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java b/core/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java index 6243f138380..0d284a1e47e 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java @@ -152,7 +152,9 @@ public class PrimaryAllocationIT extends ESIntegTestCase { client().admin().cluster().prepareReroute().add(new AllocateStalePrimaryAllocationCommand("test", 0, dataNodeWithNoShardCopy, true)).get(); logger.info("--> wait until shard is failed and becomes unassigned again"); - assertBusy(() -> assertTrue(client().admin().cluster().prepareState().get().getState().prettyPrint(), client().admin().cluster().prepareState().get().getState().getRoutingTable().index("test").allPrimaryShardsUnassigned())); + assertBusy(() -> + assertTrue(client().admin().cluster().prepareState().get().getState().toString(), + client().admin().cluster().prepareState().get().getState().getRoutingTable().index("test").allPrimaryShardsUnassigned())); assertThat(client().admin().cluster().prepareState().get().getState().getRoutingTable().index("test").getShards().get(0).primaryShard().unassignedInfo().getReason(), equalTo(UnassignedInfo.Reason.ALLOCATION_FAILED)); } diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java index 60f1688ad3d..4fffcebc79b 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java @@ -306,7 +306,7 @@ public class UnassignedInfoTests extends ESAllocationTestCase { clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).remove("node2")).build(); // make sure both replicas are marked as delayed (i.e. not reallocated) clusterState = allocation.deassociateDeadNodes(clusterState, true, "reroute"); - assertThat(clusterState.prettyPrint(), UnassignedInfo.getNumberOfDelayedUnassigned(clusterState), equalTo(2)); + assertThat(clusterState.toString(), UnassignedInfo.getNumberOfDelayedUnassigned(clusterState), equalTo(2)); } public void testFindNextDelayedAllocation() { diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/AddIncrementallyTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/AddIncrementallyTests.java index ed7a944963d..e658ff03a18 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/AddIncrementallyTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/AddIncrementallyTests.java @@ -89,7 +89,7 @@ public class AddIncrementallyTests extends ESAllocationTestCase { clusterState = addNodes(clusterState, service, 1, nodeOffset++); assertNumIndexShardsPerNode(clusterState, Matchers.lessThanOrEqualTo(2)); assertThat(clusterState.getRoutingNodes().unassigned().size(), equalTo(0)); - logger.debug("ClusterState: {}", clusterState.getRoutingNodes().prettyPrint()); + logger.debug("ClusterState: {}", clusterState.getRoutingNodes()); } public void testMinimalRelocations() { @@ -150,7 +150,7 @@ public class AddIncrementallyTests extends ESAllocationTestCase { newState = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); assertThat(newState, equalTo(clusterState)); assertNumIndexShardsPerNode(clusterState, equalTo(2)); - logger.debug("ClusterState: {}", clusterState.getRoutingNodes().prettyPrint()); + logger.debug("ClusterState: {}", clusterState.getRoutingNodes()); } public void testMinimalRelocationsNoLimit() { @@ -212,7 +212,7 @@ public class AddIncrementallyTests extends ESAllocationTestCase { newState = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); assertThat(newState, equalTo(clusterState)); assertNumIndexShardsPerNode(clusterState, equalTo(2)); - logger.debug("ClusterState: {}", clusterState.getRoutingNodes().prettyPrint()); + logger.debug("ClusterState: {}", clusterState.getRoutingNodes()); } diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java index a56af9f2b39..f48c9acb356 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java @@ -366,7 +366,7 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase { } private ClusterState stabilize(ClusterState clusterState, AllocationService service) { - logger.trace("RoutingNodes: {}", clusterState.getRoutingNodes().prettyPrint()); + logger.trace("RoutingNodes: {}", clusterState.getRoutingNodes()); clusterState = service.deassociateDeadNodes(clusterState, true, "reroute"); RoutingNodes routingNodes = clusterState.getRoutingNodes(); @@ -375,7 +375,7 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase { logger.info("complete rebalancing"); boolean changed; do { - logger.trace("RoutingNodes: {}", clusterState.getRoutingNodes().prettyPrint()); + logger.trace("RoutingNodes: {}", clusterState.getRoutingNodes()); ClusterState newState = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); changed = newState.equals(clusterState) == false; clusterState = newState; @@ -386,7 +386,7 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase { } private void assertRecoveryNodeVersions(RoutingNodes routingNodes) { - logger.trace("RoutingNodes: {}", routingNodes.prettyPrint()); + logger.trace("RoutingNodes: {}", routingNodes); List mutableShardRoutings = routingNodes.shardsWithState(ShardRoutingState.RELOCATING); for (ShardRouting r : mutableShardRoutings) { diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/RandomAllocationDeciderTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/RandomAllocationDeciderTests.java index 6722e048030..23992b91541 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/RandomAllocationDeciderTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/RandomAllocationDeciderTests.java @@ -159,7 +159,7 @@ public class RandomAllocationDeciderTests extends ESAllocationTestCase { } while (clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size() != 0 || clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size() != 0 && iterations < 200); - logger.info("Done Balancing after [{}] iterations. State:\n{}", iterations, clusterState.prettyPrint()); + logger.info("Done Balancing after [{}] iterations. State:\n{}", iterations, clusterState); // we stop after 200 iterations if it didn't stabelize by then something is likely to be wrong assertThat("max num iteration exceeded", iterations, Matchers.lessThan(200)); assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(0)); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/StartedShardsRoutingTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/StartedShardsRoutingTests.java index 454e8410484..74d3dda8e36 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/StartedShardsRoutingTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/StartedShardsRoutingTests.java @@ -70,15 +70,14 @@ public class StartedShardsRoutingTests extends ESAllocationTestCase { logger.info("--> test starting of shard"); ClusterState newState = allocation.applyStartedShards(state, Arrays.asList(initShard)); - assertThat("failed to start " + initShard + "\ncurrent routing table:" + newState.routingTable().prettyPrint(), - newState, not(equalTo(state))); - assertTrue(initShard + "isn't started \ncurrent routing table:" + newState.routingTable().prettyPrint(), + assertThat("failed to start " + initShard + "\ncurrent routing table:" + newState.routingTable(), newState, not(equalTo(state))); + assertTrue(initShard + "isn't started \ncurrent routing table:" + newState.routingTable(), newState.routingTable().index("test").shard(initShard.id()).allShardsStarted()); state = newState; logger.info("--> testing starting of relocating shards"); newState = allocation.applyStartedShards(state, Arrays.asList(relocatingShard.getTargetRelocatingShard())); - assertThat("failed to start " + relocatingShard + "\ncurrent routing table:" + newState.routingTable().prettyPrint(), + assertThat("failed to start " + relocatingShard + "\ncurrent routing table:" + newState.routingTable(), newState, not(equalTo(state))); ShardRouting shardRouting = newState.routingTable().index("test").shard(relocatingShard.id()).getShards().get(0); assertThat(shardRouting.state(), equalTo(ShardRoutingState.STARTED)); diff --git a/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java b/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java index 3f1e8f032ca..00d9a8ff096 100644 --- a/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java @@ -22,6 +22,7 @@ package org.elasticsearch.cluster.serialization; import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ESAllocationTestCase; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNodes; @@ -30,7 +31,6 @@ import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.cluster.ESAllocationTestCase; import static org.hamcrest.Matchers.equalTo; @@ -56,7 +56,7 @@ public class ClusterSerializationTests extends ESAllocationTestCase { assertThat(serializedClusterState.getClusterName().value(), equalTo(clusterState.getClusterName().value())); - assertThat(serializedClusterState.routingTable().prettyPrint(), equalTo(clusterState.routingTable().prettyPrint())); + assertThat(serializedClusterState.routingTable().toString(), equalTo(clusterState.routingTable().toString())); } public void testRoutingTableSerialization() throws Exception { @@ -81,7 +81,7 @@ public class ClusterSerializationTests extends ESAllocationTestCase { StreamInput inStream = outStream.bytes().streamInput(); RoutingTable target = RoutingTable.Builder.readFrom(inStream); - assertThat(target.prettyPrint(), equalTo(source.prettyPrint())); + assertThat(target.toString(), equalTo(source.toString())); } } diff --git a/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterStateToStringTests.java b/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterStateToStringTests.java index 69d18933e6c..9ce3d1fcee8 100644 --- a/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterStateToStringTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterStateToStringTests.java @@ -21,6 +21,7 @@ package org.elasticsearch.cluster.serialization; import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ESAllocationTestCase; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexTemplateMetaData; import org.elasticsearch.cluster.metadata.MetaData; @@ -28,8 +29,8 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.allocation.AllocationService; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.cluster.ESAllocationTestCase; import static java.util.Collections.emptyMap; import static java.util.Collections.emptySet; @@ -55,7 +56,7 @@ public class ClusterStateToStringTests extends ESAllocationTestCase { AllocationService strategy = createAllocationService(); clusterState = ClusterState.builder(clusterState).routingTable(strategy.reroute(clusterState, "reroute").routingTable()).build(); - String clusterStateString = clusterState.toString(); + String clusterStateString = Strings.toString(clusterState, true); assertNotNull(clusterStateString); assertThat(clusterStateString, containsString("test_idx")); diff --git a/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java b/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java index f0c8258864a..0208664033c 100644 --- a/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java +++ b/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java @@ -381,7 +381,7 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase { } if (!success) { fail("node [" + node + "] has no master or has blocks, despite of being on the right side of the partition. State dump:\n" - + nodeState.prettyPrint()); + + nodeState); } } @@ -468,13 +468,13 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase { assertEquals("unequal node count", state.nodes().getSize(), nodeState.nodes().getSize()); assertEquals("different masters ", state.nodes().getMasterNodeId(), nodeState.nodes().getMasterNodeId()); assertEquals("different meta data version", state.metaData().version(), nodeState.metaData().version()); - if (!state.routingTable().prettyPrint().equals(nodeState.routingTable().prettyPrint())) { + if (!state.routingTable().toString().equals(nodeState.routingTable().toString())) { fail("different routing"); } } catch (AssertionError t) { fail("failed comparing cluster state: " + t.getMessage() + "\n" + - "--- cluster state of node [" + nodes.get(0) + "]: ---\n" + state.prettyPrint() + - "\n--- cluster state [" + node + "]: ---\n" + nodeState.prettyPrint()); + "--- cluster state of node [" + nodes.get(0) + "]: ---\n" + state + + "\n--- cluster state [" + node + "]: ---\n" + nodeState); } } @@ -1267,7 +1267,7 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase { final ClusterState state = client().admin().cluster().prepareState().get().getState(); if (state.metaData().hasIndex("test") == false) { - fail("index 'test' was lost. current cluster state: " + state.prettyPrint()); + fail("index 'test' was lost. current cluster state: " + state); } } @@ -1368,7 +1368,7 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase { assertBusy(() -> { for (String node : nodes) { ClusterState state = getNodeClusterState(node); - String failMsgSuffix = "cluster_state:\n" + state.prettyPrint(); + String failMsgSuffix = "cluster_state:\n" + state; assertThat("wrong node count on [" + node + "]. " + failMsgSuffix, state.nodes().getSize(), equalTo(nodes.size())); String otherMasterNodeName = state.nodes().getMasterNode() != null ? state.nodes().getMasterNode().getName() : null; assertThat("wrong master on node [" + node + "]. " + failMsgSuffix, otherMasterNodeName, equalTo(masterNode)); diff --git a/core/src/test/java/org/elasticsearch/discovery/ZenUnicastDiscoveryIT.java b/core/src/test/java/org/elasticsearch/discovery/ZenUnicastDiscoveryIT.java index 14561f255cf..3af2e32eefa 100644 --- a/core/src/test/java/org/elasticsearch/discovery/ZenUnicastDiscoveryIT.java +++ b/core/src/test/java/org/elasticsearch/discovery/ZenUnicastDiscoveryIT.java @@ -66,7 +66,7 @@ public class ZenUnicastDiscoveryIT extends ESIntegTestCase { internalCluster().startNodesAsync(currentNumNodes - unicastHostOrdinals.length).get(); if (client().admin().cluster().prepareHealth().setWaitForNodes("" + currentNumNodes).get().isTimedOut()) { - logger.info("cluster forming timed out, cluster state:\n{}", client().admin().cluster().prepareState().get().getState().prettyPrint()); + logger.info("cluster forming timed out, cluster state:\n{}", client().admin().cluster().prepareState().get().getState()); fail("timed out waiting for cluster to form with [" + currentNumNodes + "] nodes"); } } diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java index eb580716622..95fcb88a7ea 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java @@ -676,10 +676,10 @@ public class NodeJoinControllerTests extends ESTestCase { protected void assertNodesInCurrentState(List expectedNodes) { final ClusterState state = clusterService.state(); - logger.info("assert for [{}] in:\n{}", expectedNodes, state.prettyPrint()); + logger.info("assert for [{}] in:\n{}", expectedNodes, state); DiscoveryNodes discoveryNodes = state.nodes(); for (DiscoveryNode node : expectedNodes) { - assertThat("missing " + node + "\n" + discoveryNodes.prettyPrint(), discoveryNodes.get(node.getId()), equalTo(node)); + assertThat("missing " + node + "\n" + discoveryNodes, discoveryNodes.get(node.getId()), equalTo(node)); } assertThat(discoveryNodes.getSize(), equalTo(expectedNodes.size())); } diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/PublishClusterStateActionTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/PublishClusterStateActionTests.java index eb8153c8354..e1d2a226a02 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/PublishClusterStateActionTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/PublishClusterStateActionTests.java @@ -41,8 +41,6 @@ import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.DiscoverySettings; -import org.elasticsearch.discovery.zen.DiscoveryNodesProvider; -import org.elasticsearch.discovery.zen.PublishClusterStateAction; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.node.Node; import org.elasticsearch.test.ESTestCase; @@ -852,8 +850,8 @@ public class PublishClusterStateActionTests extends ESTestCase { void assertSameState(ClusterState actual, ClusterState expected) { assertThat(actual, notNullValue()); - final String reason = "\n--> actual ClusterState: " + actual.prettyPrint() + "\n" + - "--> expected ClusterState:" + expected.prettyPrint(); + final String reason = "\n--> actual ClusterState: " + actual + "\n" + + "--> expected ClusterState:" + expected; assertThat("unequal UUIDs" + reason, actual.stateUUID(), equalTo(expected.stateUUID())); assertThat("unequal versions" + reason, actual.version(), equalTo(expected.version())); } diff --git a/core/src/test/java/org/elasticsearch/index/store/CorruptedFileIT.java b/core/src/test/java/org/elasticsearch/index/store/CorruptedFileIT.java index a996c9f4bd8..d1be0d77613 100644 --- a/core/src/test/java/org/elasticsearch/index/store/CorruptedFileIT.java +++ b/core/src/test/java/org/elasticsearch/index/store/CorruptedFileIT.java @@ -60,8 +60,8 @@ import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardState; import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.indices.recovery.RecoveryFileChunkRequest; import org.elasticsearch.indices.recovery.PeerRecoveryTargetService; +import org.elasticsearch.indices.recovery.RecoveryFileChunkRequest; import org.elasticsearch.monitor.fs.FsInfo; import org.elasticsearch.node.Node; import org.elasticsearch.plugins.Plugin; @@ -178,7 +178,8 @@ public class CorruptedFileIT extends ESIntegTestCase { .timeout("5m") // sometimes due to cluster rebalacing and random settings default timeout is just not enough. .waitForNoRelocatingShards(true)).actionGet(); if (health.isTimedOut()) { - logger.info("cluster state:\n{}\n{}", client().admin().cluster().prepareState().get().getState().prettyPrint(), client().admin().cluster().preparePendingClusterTasks().get().prettyPrint()); + logger.info("cluster state:\n{}\n{}", + client().admin().cluster().prepareState().get().getState(), client().admin().cluster().preparePendingClusterTasks().get()); assertThat("timed out waiting for green state", health.isTimedOut(), equalTo(false)); } assertThat(health.getStatus(), equalTo(ClusterHealthStatus.GREEN)); @@ -284,7 +285,8 @@ public class CorruptedFileIT extends ESIntegTestCase { .health(Requests.clusterHealthRequest("test")).get(); if (response.getStatus() != ClusterHealthStatus.RED) { logger.info("Cluster turned red in busy loop: {}", didClusterTurnRed); - logger.info("cluster state:\n{}\n{}", client().admin().cluster().prepareState().get().getState().prettyPrint(), client().admin().cluster().preparePendingClusterTasks().get().prettyPrint()); + logger.info("cluster state:\n{}\n{}", + client().admin().cluster().prepareState().get().getState(), client().admin().cluster().preparePendingClusterTasks().get()); } assertThat(response.getStatus(), is(ClusterHealthStatus.RED)); ClusterState state = client().admin().cluster().prepareState().get().getState(); @@ -445,7 +447,8 @@ public class CorruptedFileIT extends ESIntegTestCase { ClusterHealthResponse actionGet = client().admin().cluster() .health(Requests.clusterHealthRequest("test").waitForGreenStatus()).actionGet(); if (actionGet.isTimedOut()) { - logger.info("ensureGreen timed out, cluster state:\n{}\n{}", client().admin().cluster().prepareState().get().getState().prettyPrint(), client().admin().cluster().preparePendingClusterTasks().get().prettyPrint()); + logger.info("ensureGreen timed out, cluster state:\n{}\n{}", + client().admin().cluster().prepareState().get().getState(), client().admin().cluster().preparePendingClusterTasks().get()); assertThat("timed out waiting for green state", actionGet.isTimedOut(), equalTo(false)); } // we are green so primaries got not corrupted. diff --git a/core/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java b/core/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java index 67a82d93c54..074b4a5d6bb 100644 --- a/core/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java +++ b/core/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java @@ -91,7 +91,7 @@ public class IndicesClusterStateServiceRandomUpdatesTests extends AbstractIndice } catch (AssertionError error) { ClusterState finalState = state; logger.error((org.apache.logging.log4j.util.Supplier) () -> - new ParameterizedMessage("failed to random change state. last good state: \n{}", finalState.prettyPrint()), error); + new ParameterizedMessage("failed to random change state. last good state: \n{}", finalState), error); throw error; } } @@ -107,7 +107,7 @@ public class IndicesClusterStateServiceRandomUpdatesTests extends AbstractIndice } catch (AssertionError error) { logger.error((org.apache.logging.log4j.util.Supplier) () -> new ParameterizedMessage( "failed to apply change on [{}].\n *** Previous state ***\n{}\n *** New state ***\n{}", - node, event.previousState().prettyPrint(), event.state().prettyPrint()), error); + node, event.previousState(), event.state()), error); throw error; } @@ -117,7 +117,7 @@ public class IndicesClusterStateServiceRandomUpdatesTests extends AbstractIndice } // TODO: check if we can go to green by starting all shards and finishing all iterations - logger.info("Final cluster state: {}", state.prettyPrint()); + logger.info("Final cluster state: {}", state); } /** diff --git a/core/src/test/java/org/elasticsearch/recovery/FullRollingRestartIT.java b/core/src/test/java/org/elasticsearch/recovery/FullRollingRestartIT.java index a180d6feb8e..4a61bebd4db 100644 --- a/core/src/test/java/org/elasticsearch/recovery/FullRollingRestartIT.java +++ b/core/src/test/java/org/elasticsearch/recovery/FullRollingRestartIT.java @@ -149,7 +149,8 @@ public class FullRollingRestartIT extends ESIntegTestCase { ClusterState state = client().admin().cluster().prepareState().get().getState(); RecoveryResponse recoveryResponse = client().admin().indices().prepareRecoveries("test").get(); for (RecoveryState recoveryState : recoveryResponse.shardRecoveryStates().get("test")) { - assertTrue("relocated from: " + recoveryState.getSourceNode() + " to: " + recoveryState.getTargetNode() + "\n" + state.prettyPrint(), recoveryState.getRecoverySource().getType() != RecoverySource.Type.PEER || recoveryState.getPrimary() == false); + assertTrue("relocated from: " + recoveryState.getSourceNode() + " to: " + recoveryState.getTargetNode() + "\n" + state, + recoveryState.getRecoverySource().getType() != RecoverySource.Type.PEER || recoveryState.getPrimary() == false); } internalCluster().restartRandomDataNode(); ensureGreen(); @@ -157,7 +158,8 @@ public class FullRollingRestartIT extends ESIntegTestCase { recoveryResponse = client().admin().indices().prepareRecoveries("test").get(); for (RecoveryState recoveryState : recoveryResponse.shardRecoveryStates().get("test")) { - assertTrue("relocated from: " + recoveryState.getSourceNode() + " to: " + recoveryState.getTargetNode()+ "-- \nbefore: \n" + state.prettyPrint() + "\nafter: \n" + afterState.prettyPrint(), recoveryState.getRecoverySource().getType() != RecoverySource.Type.PEER || recoveryState.getPrimary() == false); + assertTrue("relocated from: " + recoveryState.getSourceNode() + " to: " + recoveryState.getTargetNode()+ "-- \nbefore: \n" + state, + recoveryState.getRecoverySource().getType() != RecoverySource.Type.PEER || recoveryState.getPrimary() == false); } } } diff --git a/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java b/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java index 739371d76df..47051d9072d 100644 --- a/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java @@ -146,7 +146,7 @@ public abstract class ESAllocationTestCase extends ESTestCase { ClusterState lastClusterState; do { lastClusterState = clusterState; - logger.debug("ClusterState: {}", clusterState.getRoutingNodes().prettyPrint()); + logger.debug("ClusterState: {}", clusterState.getRoutingNodes()); clusterState = service.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); } while (lastClusterState.equals(clusterState) == false); return clusterState; diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index 1e98754a798..77c65197366 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -164,7 +164,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.BooleanSupplier; import java.util.function.Function; -import java.util.stream.Collectors; import static org.elasticsearch.client.Requests.syncedFlushRequest; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS; @@ -536,9 +535,7 @@ public abstract class ESIntegTestCase extends ESTestCase { for (Discovery discovery : internalCluster().getInstances(Discovery.class)) { if (discovery instanceof ZenDiscovery) { final ZenDiscovery zenDiscovery = (ZenDiscovery) discovery; - assertBusy(() -> assertThat(zenDiscovery.localNode().getName() + " still having pending states: \n" + - Arrays.stream(zenDiscovery.pendingClusterStates()).map(ClusterState::prettyPrint) - .collect(Collectors.joining("\n")), + assertBusy(() -> assertThat(zenDiscovery.localNode().getName() + " still having pending states", zenDiscovery.pendingClusterStates(), emptyArray())); } } @@ -758,17 +755,14 @@ public abstract class ESIntegTestCase extends ESTestCase { */ public void waitNoPendingTasksOnAll() throws Exception { assertNoTimeout(client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).get()); - assertBusy(new Runnable() { - @Override - public void run() { - for (Client client : clients()) { - ClusterHealthResponse clusterHealth = client.admin().cluster().prepareHealth().setLocal(true).get(); - assertThat("client " + client + " still has in flight fetch", clusterHealth.getNumberOfInFlightFetch(), equalTo(0)); - PendingClusterTasksResponse pendingTasks = client.admin().cluster().preparePendingClusterTasks().setLocal(true).get(); - assertThat("client " + client + " still has pending tasks " + pendingTasks.prettyPrint(), pendingTasks, Matchers.emptyIterable()); - clusterHealth = client.admin().cluster().prepareHealth().setLocal(true).get(); - assertThat("client " + client + " still has in flight fetch", clusterHealth.getNumberOfInFlightFetch(), equalTo(0)); - } + assertBusy(() -> { + for (Client client : clients()) { + ClusterHealthResponse clusterHealth = client.admin().cluster().prepareHealth().setLocal(true).get(); + assertThat("client " + client + " still has in flight fetch", clusterHealth.getNumberOfInFlightFetch(), equalTo(0)); + PendingClusterTasksResponse pendingTasks = client.admin().cluster().preparePendingClusterTasks().setLocal(true).get(); + assertThat("client " + client + " still has pending tasks " + pendingTasks, pendingTasks, Matchers.emptyIterable()); + clusterHealth = client.admin().cluster().prepareHealth().setLocal(true).get(); + assertThat("client " + client + " still has in flight fetch", clusterHealth.getNumberOfInFlightFetch(), equalTo(0)); } }); assertNoTimeout(client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).get()); @@ -872,7 +866,8 @@ public abstract class ESIntegTestCase extends ESTestCase { ClusterHealthResponse actionGet = client().admin().cluster() .health(Requests.clusterHealthRequest(indices).timeout(timeout).waitForGreenStatus().waitForEvents(Priority.LANGUID).waitForNoRelocatingShards(true)).actionGet(); if (actionGet.isTimedOut()) { - logger.info("ensureGreen timed out, cluster state:\n{}\n{}", client().admin().cluster().prepareState().get().getState().prettyPrint(), client().admin().cluster().preparePendingClusterTasks().get().prettyPrint()); + logger.info("ensureGreen timed out, cluster state:\n{}\n{}", + client().admin().cluster().prepareState().get().getState(), client().admin().cluster().preparePendingClusterTasks().get()); fail("timed out waiting for green state"); } assertThat(actionGet.getStatus(), equalTo(ClusterHealthStatus.GREEN)); @@ -899,7 +894,8 @@ public abstract class ESIntegTestCase extends ESTestCase { ClusterHealthResponse actionGet = client().admin().cluster() .health(request).actionGet(); if (actionGet.isTimedOut()) { - logger.info("waitForRelocation timed out (status={}), cluster state:\n{}\n{}", status, client().admin().cluster().prepareState().get().getState().prettyPrint(), client().admin().cluster().preparePendingClusterTasks().get().prettyPrint()); + logger.info("waitForRelocation timed out (status={}), cluster state:\n{}\n{}", status, + client().admin().cluster().prepareState().get().getState(), client().admin().cluster().preparePendingClusterTasks().get()); assertThat("timed out waiting for relocation", actionGet.isTimedOut(), equalTo(false)); } if (status != null) { @@ -996,7 +992,8 @@ public abstract class ESIntegTestCase extends ESTestCase { ClusterHealthResponse actionGet = client().admin().cluster() .health(Requests.clusterHealthRequest(indices).waitForNoRelocatingShards(true).waitForYellowStatus().waitForEvents(Priority.LANGUID)).actionGet(); if (actionGet.isTimedOut()) { - logger.info("ensureYellow timed out, cluster state:\n{}\n{}", client().admin().cluster().prepareState().get().getState().prettyPrint(), client().admin().cluster().preparePendingClusterTasks().get().prettyPrint()); + logger.info("ensureYellow timed out, cluster state:\n{}\n{}", + client().admin().cluster().prepareState().get().getState(), client().admin().cluster().preparePendingClusterTasks().get()); assertThat("timed out waiting for yellow", actionGet.isTimedOut(), equalTo(false)); } logger.debug("indices {} are yellow", indices.length == 0 ? "[_all]" : indices); @@ -1007,7 +1004,8 @@ public abstract class ESIntegTestCase extends ESTestCase { * Prints the current cluster state as debug logging. */ public void logClusterState() { - logger.debug("cluster state:\n{}\n{}", client().admin().cluster().prepareState().get().getState().prettyPrint(), client().admin().cluster().preparePendingClusterTasks().get().prettyPrint()); + logger.debug("cluster state:\n{}\n{}", + client().admin().cluster().prepareState().get().getState(), client().admin().cluster().preparePendingClusterTasks().get()); } /** @@ -1108,7 +1106,7 @@ public abstract class ESIntegTestCase extends ESTestCase { if (clusterHealthResponse.isTimedOut()) { ClusterStateResponse stateResponse = client(viaNode).admin().cluster().prepareState().get(); fail("failed to reach a stable cluster of [" + nodeCount + "] nodes. Tried via [" + viaNode + "]. last cluster state:\n" - + stateResponse.getState().prettyPrint()); + + stateResponse.getState()); } assertThat(clusterHealthResponse.isTimedOut(), is(false)); } diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java index f096e662f4a..9648eb5798e 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java @@ -318,7 +318,8 @@ public abstract class ESSingleNodeTestCase extends ESTestCase { ClusterHealthResponse actionGet = client().admin().cluster() .health(Requests.clusterHealthRequest(indices).timeout(timeout).waitForGreenStatus().waitForEvents(Priority.LANGUID).waitForNoRelocatingShards(true)).actionGet(); if (actionGet.isTimedOut()) { - logger.info("ensureGreen timed out, cluster state:\n{}\n{}", client().admin().cluster().prepareState().get().getState().prettyPrint(), client().admin().cluster().preparePendingClusterTasks().get().prettyPrint()); + logger.info("ensureGreen timed out, cluster state:\n{}\n{}", client().admin().cluster().prepareState().get().getState(), + client().admin().cluster().preparePendingClusterTasks().get()); assertThat("timed out waiting for green state", actionGet.isTimedOut(), equalTo(false)); } assertThat(actionGet.getStatus(), equalTo(ClusterHealthStatus.GREEN)); From 638353cda30d2eeacfd1d90a4efe479fcd9b5cb7 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Wed, 2 Nov 2016 13:55:52 +0100 Subject: [PATCH 093/132] `query_string` can now automatically generate phrase queries. --- .../index/query/QueryStringQueryBuilderTests.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/test/java/org/elasticsearch/index/query/QueryStringQueryBuilderTests.java b/core/src/test/java/org/elasticsearch/index/query/QueryStringQueryBuilderTests.java index 1241fde3624..73b3711701d 100644 --- a/core/src/test/java/org/elasticsearch/index/query/QueryStringQueryBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/QueryStringQueryBuilderTests.java @@ -163,7 +163,8 @@ public class QueryStringQueryBuilderTests extends AbstractQueryTestCase Date: Wed, 2 Nov 2016 14:25:08 +0100 Subject: [PATCH 094/132] Remove `lowercase_expanded_terms` and `locale` from query-parser options. (#20208) Lucene 6.2 introduces the new `Analyzer.normalize` API, which allows to apply only character-level normalization such as lowercasing or accent folding, which is exactly what is needed to process queries that operate on partial terms such as `prefix`, `wildcard` or `fuzzy` queries. As a consequence, the `lowercase_expanded_terms` option is not necessary anymore. Furthermore, the `locale` option was only needed in order to know how to perform the lowercasing, so this one can be removed as well. Closes #9978 --- .../classic/MapperQueryParser.java | 113 +++--------------- .../classic/QueryParserSettings.java | 21 ---- .../index/mapper/MapperService.java | 1 - .../index/query/QueryStringQueryBuilder.java | 80 ++++--------- .../index/query/SimpleQueryParser.java | 72 +++-------- .../index/query/SimpleQueryStringBuilder.java | 62 +++------- .../rest/action/RestActions.java | 1 - .../query/QueryStringQueryBuilderTests.java | 63 ++++++++-- .../query/SimpleQueryStringBuilderTests.java | 67 ++++++----- .../search/child/ChildQuerySearchIT.java | 4 +- .../search/query/SearchQueryIT.java | 22 ++-- .../search/query/SimpleQueryStringIT.java | 49 +------- .../query-dsl/query-string-query.asciidoc | 11 +- .../query-dsl/query-string-syntax.asciidoc | 15 ++- .../simple-query-string-query.asciidoc | 7 -- docs/reference/search/count.asciidoc | 3 - docs/reference/search/explain.asciidoc | 4 - docs/reference/search/uri-request.asciidoc | 3 - docs/reference/search/validate.asciidoc | 3 - .../resources/rest-api-spec/api/count.json | 4 - .../rest-api-spec/api/delete_by_query.json | 4 - .../resources/rest-api-spec/api/explain.json | 4 - .../api/indices.validate_query.json | 4 - .../resources/rest-api-spec/api/search.json | 4 - .../rest-api-spec/api/update_by_query.json | 4 - .../test/count/20_query_string.yaml | 9 -- .../test/explain/30_query_string.yaml | 11 -- .../20_query_string.yaml | 9 -- .../test/search/60_query_string.yaml | 9 -- 29 files changed, 193 insertions(+), 470 deletions(-) diff --git a/core/src/main/java/org/apache/lucene/queryparser/classic/MapperQueryParser.java b/core/src/main/java/org/apache/lucene/queryparser/classic/MapperQueryParser.java index bbb27aa11db..ac9770f2bc8 100644 --- a/core/src/main/java/org/apache/lucene/queryparser/classic/MapperQueryParser.java +++ b/core/src/main/java/org/apache/lucene/queryparser/classic/MapperQueryParser.java @@ -24,6 +24,7 @@ import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.analysis.tokenattributes.CharTermAttribute; import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute; import org.apache.lucene.index.Term; +import org.apache.lucene.queryparser.analyzing.AnalyzingQueryParser; import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.BoostQuery; @@ -34,6 +35,7 @@ import org.apache.lucene.search.MultiPhraseQuery; import org.apache.lucene.search.PhraseQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.SynonymQuery; +import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.automaton.RegExp; import org.elasticsearch.common.lucene.search.Queries; @@ -42,6 +44,7 @@ import org.elasticsearch.index.mapper.DateFieldMapper; import org.elasticsearch.index.mapper.LegacyDateFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.StringFieldType; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.query.support.QueryParsers; @@ -63,7 +66,7 @@ import static org.elasticsearch.common.lucene.search.Queries.fixNegativeQueryIfN * Also breaks fields with [type].[name] into a boolean query that must include the type * as well as the query on the name. */ -public class MapperQueryParser extends QueryParser { +public class MapperQueryParser extends AnalyzingQueryParser { public static final Map FIELD_QUERY_EXTENSIONS; @@ -99,11 +102,10 @@ public class MapperQueryParser extends QueryParser { setAutoGeneratePhraseQueries(settings.autoGeneratePhraseQueries()); setMaxDeterminizedStates(settings.maxDeterminizedStates()); setAllowLeadingWildcard(settings.allowLeadingWildcard()); - setLowercaseExpandedTerms(settings.lowercaseExpandedTerms()); + setLowercaseExpandedTerms(false); setPhraseSlop(settings.phraseSlop()); setDefaultOperator(settings.defaultOperator()); setFuzzyPrefixLength(settings.fuzzyPrefixLength()); - setLocale(settings.locale()); setSplitOnWhitespace(settings.splitOnWhitespace()); } @@ -330,21 +332,20 @@ public class MapperQueryParser extends QueryParser { boolean startInclusive, boolean endInclusive, QueryShardContext context) { currentFieldType = context.fieldMapper(field); if (currentFieldType != null) { - if (lowercaseExpandedTerms && currentFieldType.tokenized()) { - part1 = part1 == null ? null : part1.toLowerCase(locale); - part2 = part2 == null ? null : part2.toLowerCase(locale); - } - try { + BytesRef part1Binary = part1 == null ? null : getAnalyzer().normalize(field, part1); + BytesRef part2Binary = part2 == null ? null : getAnalyzer().normalize(field, part2); Query rangeQuery; if (currentFieldType instanceof LegacyDateFieldMapper.DateFieldType && settings.timeZone() != null) { LegacyDateFieldMapper.DateFieldType dateFieldType = (LegacyDateFieldMapper.DateFieldType) this.currentFieldType; - rangeQuery = dateFieldType.rangeQuery(part1, part2, startInclusive, endInclusive, settings.timeZone(), null, context); + rangeQuery = dateFieldType.rangeQuery(part1Binary, part2Binary, + startInclusive, endInclusive, settings.timeZone(), null, context); } else if (currentFieldType instanceof DateFieldMapper.DateFieldType && settings.timeZone() != null) { DateFieldMapper.DateFieldType dateFieldType = (DateFieldMapper.DateFieldType) this.currentFieldType; - rangeQuery = dateFieldType.rangeQuery(part1, part2, startInclusive, endInclusive, settings.timeZone(), null, context); + rangeQuery = dateFieldType.rangeQuery(part1Binary, part2Binary, + startInclusive, endInclusive, settings.timeZone(), null, context); } else { - rangeQuery = currentFieldType.rangeQuery(part1, part2, startInclusive, endInclusive, context); + rangeQuery = currentFieldType.rangeQuery(part1Binary, part2Binary, startInclusive, endInclusive, context); } return rangeQuery; } catch (RuntimeException e) { @@ -358,9 +359,6 @@ public class MapperQueryParser extends QueryParser { } protected Query getFuzzyQuery(String field, String termStr, String minSimilarity) throws ParseException { - if (lowercaseExpandedTerms) { - termStr = termStr.toLowerCase(locale); - } Collection fields = extractMultiFields(field); if (fields != null) { if (fields.size() == 1) { @@ -399,8 +397,9 @@ public class MapperQueryParser extends QueryParser { currentFieldType = context.fieldMapper(field); if (currentFieldType != null) { try { - return currentFieldType.fuzzyQuery(termStr, Fuzziness.build(minSimilarity), - fuzzyPrefixLength, settings.fuzzyMaxExpansions(), FuzzyQuery.defaultTranspositions); + BytesRef term = termStr == null ? null : getAnalyzer().normalize(field, termStr); + return currentFieldType.fuzzyQuery(term, Fuzziness.build(minSimilarity), + getFuzzyPrefixLength(), settings.fuzzyMaxExpansions(), FuzzyQuery.defaultTranspositions); } catch (RuntimeException e) { if (settings.lenient()) { return null; @@ -423,9 +422,6 @@ public class MapperQueryParser extends QueryParser { @Override protected Query getPrefixQuery(String field, String termStr) throws ParseException { - if (lowercaseExpandedTerms) { - termStr = termStr.toLowerCase(locale); - } Collection fields = extractMultiFields(field); if (fields != null) { if (fields.size() == 1) { @@ -471,8 +467,8 @@ public class MapperQueryParser extends QueryParser { setAnalyzer(context.getSearchAnalyzer(currentFieldType)); } Query query = null; - if (currentFieldType.tokenized() == false) { - query = currentFieldType.prefixQuery(termStr, multiTermRewriteMethod, context); + if (currentFieldType instanceof StringFieldType == false) { + query = currentFieldType.prefixQuery(termStr, getMultiTermRewriteMethod(), context); } if (query == null) { query = getPossiblyAnalyzedPrefixQuery(currentFieldType.name(), termStr); @@ -590,9 +586,6 @@ public class MapperQueryParser extends QueryParser { return FIELD_QUERY_EXTENSIONS.get(ExistsFieldQueryExtension.NAME).query(context, actualField); } } - if (lowercaseExpandedTerms) { - termStr = termStr.toLowerCase(locale); - } Collection fields = extractMultiFields(field); if (fields != null) { if (fields.size() == 1) { @@ -639,9 +632,8 @@ public class MapperQueryParser extends QueryParser { setAnalyzer(context.getSearchAnalyzer(currentFieldType)); } indexedNameField = currentFieldType.name(); - return getPossiblyAnalyzedWildcardQuery(indexedNameField, termStr); } - return getPossiblyAnalyzedWildcardQuery(indexedNameField, termStr); + return super.getWildcardQuery(indexedNameField, termStr); } catch (RuntimeException e) { if (settings.lenient()) { return null; @@ -652,75 +644,8 @@ public class MapperQueryParser extends QueryParser { } } - private Query getPossiblyAnalyzedWildcardQuery(String field, String termStr) throws ParseException { - if (!settings.analyzeWildcard()) { - return super.getWildcardQuery(field, termStr); - } - boolean isWithinToken = (!termStr.startsWith("?") && !termStr.startsWith("*")); - StringBuilder aggStr = new StringBuilder(); - StringBuilder tmp = new StringBuilder(); - for (int i = 0; i < termStr.length(); i++) { - char c = termStr.charAt(i); - if (c == '?' || c == '*') { - if (isWithinToken) { - try (TokenStream source = getAnalyzer().tokenStream(field, tmp.toString())) { - source.reset(); - CharTermAttribute termAtt = source.addAttribute(CharTermAttribute.class); - if (source.incrementToken()) { - String term = termAtt.toString(); - if (term.length() == 0) { - // no tokens, just use what we have now - aggStr.append(tmp); - } else { - aggStr.append(term); - } - } else { - // no tokens, just use what we have now - aggStr.append(tmp); - } - } catch (IOException e) { - aggStr.append(tmp); - } - tmp.setLength(0); - } - isWithinToken = false; - aggStr.append(c); - } else { - tmp.append(c); - isWithinToken = true; - } - } - if (isWithinToken) { - try { - try (TokenStream source = getAnalyzer().tokenStream(field, tmp.toString())) { - source.reset(); - CharTermAttribute termAtt = source.addAttribute(CharTermAttribute.class); - if (source.incrementToken()) { - String term = termAtt.toString(); - if (term.length() == 0) { - // no tokens, just use what we have now - aggStr.append(tmp); - } else { - aggStr.append(term); - } - } else { - // no tokens, just use what we have now - aggStr.append(tmp); - } - } - } catch (IOException e) { - aggStr.append(tmp); - } - } - - return super.getWildcardQuery(field, aggStr.toString()); - } - @Override protected Query getRegexpQuery(String field, String termStr) throws ParseException { - if (lowercaseExpandedTerms) { - termStr = termStr.toLowerCase(locale); - } Collection fields = extractMultiFields(field); if (fields != null) { if (fields.size() == 1) { @@ -768,7 +693,7 @@ public class MapperQueryParser extends QueryParser { Query query = null; if (currentFieldType.tokenized() == false) { query = currentFieldType.regexpQuery(termStr, RegExp.ALL, - maxDeterminizedStates, multiTermRewriteMethod, context); + getMaxDeterminizedStates(), getMultiTermRewriteMethod(), context); } if (query == null) { query = super.getRegexpQuery(field, termStr); diff --git a/core/src/main/java/org/apache/lucene/queryparser/classic/QueryParserSettings.java b/core/src/main/java/org/apache/lucene/queryparser/classic/QueryParserSettings.java index a03c84d8980..295c1ace4f6 100644 --- a/core/src/main/java/org/apache/lucene/queryparser/classic/QueryParserSettings.java +++ b/core/src/main/java/org/apache/lucene/queryparser/classic/QueryParserSettings.java @@ -24,7 +24,6 @@ import org.apache.lucene.search.MultiTermQuery; import org.elasticsearch.common.unit.Fuzziness; import org.joda.time.DateTimeZone; -import java.util.Locale; import java.util.Map; /** @@ -53,12 +52,8 @@ public class QueryParserSettings { private boolean analyzeWildcard; - private boolean lowercaseExpandedTerms; - private boolean enablePositionIncrements; - private Locale locale; - private Fuzziness fuzziness; private int fuzzyPrefixLength; private int fuzzyMaxExpansions; @@ -137,14 +132,6 @@ public class QueryParserSettings { this.allowLeadingWildcard = allowLeadingWildcard; } - public boolean lowercaseExpandedTerms() { - return lowercaseExpandedTerms; - } - - public void lowercaseExpandedTerms(boolean lowercaseExpandedTerms) { - this.lowercaseExpandedTerms = lowercaseExpandedTerms; - } - public boolean enablePositionIncrements() { return enablePositionIncrements; } @@ -269,14 +256,6 @@ public class QueryParserSettings { this.useDisMax = useDisMax; } - public void locale(Locale locale) { - this.locale = locale; - } - - public Locale locale() { - return this.locale; - } - public void timeZone(DateTimeZone timeZone) { this.timeZone = timeZone; } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/core/src/main/java/org/elasticsearch/index/mapper/MapperService.java index 3e5262f12fa..799c1baedbf 100755 --- a/core/src/main/java/org/elasticsearch/index/mapper/MapperService.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/MapperService.java @@ -54,7 +54,6 @@ import java.util.List; import java.util.Map; import java.util.Set; import java.util.function.Function; -import java.util.function.LongSupplier; import java.util.function.Supplier; import java.util.stream.Collectors; diff --git a/core/src/main/java/org/elasticsearch/index/query/QueryStringQueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/QueryStringQueryBuilder.java index d88484c98a6..2867169ecbe 100644 --- a/core/src/main/java/org/elasticsearch/index/query/QueryStringQueryBuilder.java +++ b/core/src/main/java/org/elasticsearch/index/query/QueryStringQueryBuilder.java @@ -64,7 +64,6 @@ public class QueryStringQueryBuilder extends AbstractQueryBuildertrue. - */ - public QueryStringQueryBuilder lowercaseExpandedTerms(boolean lowercaseExpandedTerms) { - this.lowercaseExpandedTerms = lowercaseExpandedTerms; - return this; - } - - public boolean lowercaseExpandedTerms() { - return this.lowercaseExpandedTerms; - } - /** * Set to true to enable position increments in result query. Defaults to * true. @@ -488,6 +478,11 @@ public class QueryStringQueryBuilder extends AbstractQueryBuildertrue to enable analysis on wildcard and prefix queries. */ @@ -500,11 +495,6 @@ public class QueryStringQueryBuilder extends AbstractQueryBuilder entry : weights.entrySet()) { + final String fieldName = entry.getKey(); try { - Query query = new FuzzyQuery(new Term(entry.getKey(), text), fuzziness); + final BytesRef term = getAnalyzer().normalize(fieldName, text); + Query query = new FuzzyQuery(new Term(fieldName, term), fuzziness); bq.add(wrapWithBoost(query, entry.getValue()), BooleanClause.Occur.SHOULD); } catch (RuntimeException e) { rethrowUnlessLenient(e); @@ -146,20 +145,19 @@ public class SimpleQueryParser extends org.apache.lucene.queryparser.simple.Simp */ @Override public Query newPrefixQuery(String text) { - if (settings.lowercaseExpandedTerms()) { - text = text.toLowerCase(settings.locale()); - } BooleanQuery.Builder bq = new BooleanQuery.Builder(); bq.setDisableCoord(true); for (Map.Entry entry : weights.entrySet()) { + final String fieldName = entry.getKey(); try { if (settings.analyzeWildcard()) { - Query analyzedQuery = newPossiblyAnalyzedQuery(entry.getKey(), text); + Query analyzedQuery = newPossiblyAnalyzedQuery(fieldName, text); if (analyzedQuery != null) { bq.add(wrapWithBoost(analyzedQuery, entry.getValue()), BooleanClause.Occur.SHOULD); } } else { - Query query = new PrefixQuery(new Term(entry.getKey(), text)); + Term term = new Term(fieldName, getAnalyzer().normalize(fieldName, text)); + Query query = new PrefixQuery(term); bq.add(wrapWithBoost(query, entry.getValue()), BooleanClause.Occur.SHOULD); } } catch (RuntimeException e) { @@ -182,11 +180,11 @@ public class SimpleQueryParser extends org.apache.lucene.queryparser.simple.Simp * of {@code TermQuery}s and {@code PrefixQuery}s */ private Query newPossiblyAnalyzedQuery(String field, String termStr) { - List> tlist = new ArrayList<> (); + List> tlist = new ArrayList<> (); // get Analyzer from superclass and tokenize the term try (TokenStream source = getAnalyzer().tokenStream(field, termStr)) { source.reset(); - List currentPos = new ArrayList<>(); + List currentPos = new ArrayList<>(); CharTermAttribute termAtt = source.addAttribute(CharTermAttribute.class); PositionIncrementAttribute posAtt = source.addAttribute(PositionIncrementAttribute.class); @@ -197,7 +195,8 @@ public class SimpleQueryParser extends org.apache.lucene.queryparser.simple.Simp tlist.add(currentPos); currentPos = new ArrayList<>(); } - currentPos.add(termAtt.toString()); + final BytesRef term = getAnalyzer().normalize(field, termAtt.toString()); + currentPos.add(term); hasMoreTokens = source.incrementToken(); } if (currentPos.isEmpty() == false) { @@ -223,7 +222,7 @@ public class SimpleQueryParser extends org.apache.lucene.queryparser.simple.Simp // build a boolean query with prefix on the last position only. BooleanQuery.Builder builder = new BooleanQuery.Builder(); for (int pos = 0; pos < tlist.size(); pos++) { - List plist = tlist.get(pos); + List plist = tlist.get(pos); boolean isLastPos = (pos == tlist.size()-1); Query posQuery; if (plist.size() == 1) { @@ -241,7 +240,7 @@ public class SimpleQueryParser extends org.apache.lucene.queryparser.simple.Simp posQuery = new SynonymQuery(terms); } else { BooleanQuery.Builder innerBuilder = new BooleanQuery.Builder(); - for (String token : plist) { + for (BytesRef token : plist) { innerBuilder.add(new BooleanClause(new PrefixQuery(new Term(field, token)), BooleanClause.Occur.SHOULD)); } @@ -257,10 +256,6 @@ public class SimpleQueryParser extends org.apache.lucene.queryparser.simple.Simp * their default values */ static class Settings { - /** Locale to use for parsing. */ - private Locale locale = SimpleQueryStringBuilder.DEFAULT_LOCALE; - /** Specifies whether parsed terms should be lowercased. */ - private boolean lowercaseExpandedTerms = SimpleQueryStringBuilder.DEFAULT_LOWERCASE_EXPANDED_TERMS; /** Specifies whether lenient query parsing should be used. */ private boolean lenient = SimpleQueryStringBuilder.DEFAULT_LENIENT; /** Specifies whether wildcards should be analyzed. */ @@ -275,29 +270,6 @@ public class SimpleQueryParser extends org.apache.lucene.queryparser.simple.Simp public Settings() { } - /** Specifies the locale to use for parsing, Locale.ROOT by default. */ - public void locale(Locale locale) { - this.locale = (locale != null) ? locale : SimpleQueryStringBuilder.DEFAULT_LOCALE; - } - - /** Returns the locale to use for parsing. */ - public Locale locale() { - return this.locale; - } - - /** - * Specifies whether to lowercase parse terms, defaults to true if - * unset. - */ - public void lowercaseExpandedTerms(boolean lowercaseExpandedTerms) { - this.lowercaseExpandedTerms = lowercaseExpandedTerms; - } - - /** Returns whether to lowercase parse terms. */ - public boolean lowercaseExpandedTerms() { - return this.lowercaseExpandedTerms; - } - /** Specifies whether to use lenient parsing, defaults to false. */ public void lenient(boolean lenient) { this.lenient = lenient; @@ -335,10 +307,7 @@ public class SimpleQueryParser extends org.apache.lucene.queryparser.simple.Simp @Override public int hashCode() { - // checking the return value of toLanguageTag() for locales only. - // For further reasoning see - // https://issues.apache.org/jira/browse/LUCENE-4021 - return Objects.hash(locale.toLanguageTag(), lowercaseExpandedTerms, lenient, analyzeWildcard, quoteFieldSuffix); + return Objects.hash(lenient, analyzeWildcard, quoteFieldSuffix); } @Override @@ -350,15 +319,8 @@ public class SimpleQueryParser extends org.apache.lucene.queryparser.simple.Simp return false; } Settings other = (Settings) obj; - - // checking the return value of toLanguageTag() for locales only. - // For further reasoning see - // https://issues.apache.org/jira/browse/LUCENE-4021 - return (Objects.equals(locale.toLanguageTag(), other.locale.toLanguageTag()) - && Objects.equals(lowercaseExpandedTerms, other.lowercaseExpandedTerms) - && Objects.equals(lenient, other.lenient) - && Objects.equals(analyzeWildcard, other.analyzeWildcard) - && Objects.equals(quoteFieldSuffix, other.quoteFieldSuffix)); + return Objects.equals(lenient, other.lenient) && Objects.equals(analyzeWildcard, other.analyzeWildcard) + && Objects.equals(quoteFieldSuffix, other.quoteFieldSuffix); } } } diff --git a/core/src/main/java/org/elasticsearch/index/query/SimpleQueryStringBuilder.java b/core/src/main/java/org/elasticsearch/index/query/SimpleQueryStringBuilder.java index 15c0e0f4f15..fd297075067 100644 --- a/core/src/main/java/org/elasticsearch/index/query/SimpleQueryStringBuilder.java +++ b/core/src/main/java/org/elasticsearch/index/query/SimpleQueryStringBuilder.java @@ -79,10 +79,6 @@ import java.util.TreeMap; * > online documentation. */ public class SimpleQueryStringBuilder extends AbstractQueryBuilder { - /** Default locale used for parsing.*/ - public static final Locale DEFAULT_LOCALE = Locale.ROOT; - /** Default for lowercasing parsed terms.*/ - public static final boolean DEFAULT_LOWERCASE_EXPANDED_TERMS = true; /** Default for using lenient query parsing.*/ public static final boolean DEFAULT_LENIENT = false; /** Default for wildcard analysis.*/ @@ -100,8 +96,10 @@ public class SimpleQueryStringBuilder extends AbstractQueryBuilder 1) { assertThat(query, instanceOf(BooleanQuery.class)); BooleanQuery boolQuery = (BooleanQuery) query; - if (queryBuilder.lowercaseExpandedTerms()) { - for (BooleanClause clause : boolQuery.clauses()) { - if (clause.getQuery() instanceof TermQuery) { - TermQuery inner = (TermQuery) clause.getQuery(); - assertThat(inner.getTerm().bytes().toString(), is(inner.getTerm().bytes().toString().toLowerCase(Locale.ROOT))); - } + for (BooleanClause clause : boolQuery.clauses()) { + if (clause.getQuery() instanceof TermQuery) { + TermQuery inner = (TermQuery) clause.getQuery(); + assertThat(inner.getTerm().bytes().toString(), is(inner.getTerm().bytes().toString().toLowerCase(Locale.ROOT))); } } assertThat(boolQuery.clauses().size(), equalTo(queryBuilder.fields().size())); @@ -336,10 +311,8 @@ public class SimpleQueryStringBuilderTests extends AbstractQueryTestCase client().prepareSearch() - .setQuery(queryStringQuery("future:[now/D TO now+2M/d]").lowercaseExpandedTerms(false)).get()); + .setQuery(queryStringQuery("future:[now/D TO now+2M/d]")).get()); assertThat(e.status(), equalTo(RestStatus.BAD_REQUEST)); assertThat(e.toString(), containsString("unit [D] not supported for date math")); } diff --git a/core/src/test/java/org/elasticsearch/search/query/SimpleQueryStringIT.java b/core/src/test/java/org/elasticsearch/search/query/SimpleQueryStringIT.java index 9502a818315..b98bc5d43cd 100644 --- a/core/src/test/java/org/elasticsearch/search/query/SimpleQueryStringIT.java +++ b/core/src/test/java/org/elasticsearch/search/query/SimpleQueryStringIT.java @@ -30,12 +30,10 @@ import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.test.ESIntegTestCase; import java.io.IOException; -import java.util.Locale; import java.util.concurrent.ExecutionException; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.index.query.QueryBuilders.boolQuery; -import static org.elasticsearch.index.query.QueryBuilders.queryStringQuery; import static org.elasticsearch.index.query.QueryBuilders.simpleQueryStringQuery; import static org.elasticsearch.index.query.QueryBuilders.termQuery; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; @@ -158,49 +156,6 @@ public class SimpleQueryStringIT extends ESIntegTestCase { assertSearchHits(searchResponse, "6", "7", "8"); } - public void testSimpleQueryStringLowercasing() { - createIndex("test"); - client().prepareIndex("test", "type1", "1").setSource("body", "Professional").get(); - refresh(); - - SearchResponse searchResponse = client().prepareSearch().setQuery(simpleQueryStringQuery("Professio*")).get(); - assertHitCount(searchResponse, 1L); - assertSearchHits(searchResponse, "1"); - - searchResponse = client().prepareSearch().setQuery( - simpleQueryStringQuery("Professio*").lowercaseExpandedTerms(false)).get(); - assertHitCount(searchResponse, 0L); - - searchResponse = client().prepareSearch().setQuery( - simpleQueryStringQuery("Professionan~1")).get(); - assertHitCount(searchResponse, 1L); - assertSearchHits(searchResponse, "1"); - - searchResponse = client().prepareSearch().setQuery( - simpleQueryStringQuery("Professionan~1").lowercaseExpandedTerms(false)).get(); - assertHitCount(searchResponse, 0L); - } - - public void testQueryStringLocale() { - createIndex("test"); - client().prepareIndex("test", "type1", "1").setSource("body", "bılly").get(); - refresh(); - - SearchResponse searchResponse = client().prepareSearch().setQuery(simpleQueryStringQuery("BILL*")).get(); - assertHitCount(searchResponse, 0L); - searchResponse = client().prepareSearch().setQuery(queryStringQuery("body:BILL*")).get(); - assertHitCount(searchResponse, 0L); - - searchResponse = client().prepareSearch().setQuery( - simpleQueryStringQuery("BILL*").locale(new Locale("tr", "TR"))).get(); - assertHitCount(searchResponse, 1L); - assertSearchHits(searchResponse, "1"); - searchResponse = client().prepareSearch().setQuery( - queryStringQuery("body:BILL*").locale(new Locale("tr", "TR"))).get(); - assertHitCount(searchResponse, 1L); - assertSearchHits(searchResponse, "1"); - } - public void testNestedFieldSimpleQueryString() throws IOException { assertAcked(prepareCreate("test") .addMapping("type1", jsonBuilder() @@ -342,7 +297,7 @@ public class SimpleQueryStringIT extends ESIntegTestCase { refresh(); SearchResponse searchResponse = client().prepareSearch() - .setQuery(simpleQueryStringQuery("Köln*").analyzeWildcard(true).field("location")).get(); + .setQuery(simpleQueryStringQuery("Köln*").field("location")).get(); assertNoFailures(searchResponse); assertHitCount(searchResponse, 1L); assertSearchHits(searchResponse, "1"); @@ -393,7 +348,7 @@ public class SimpleQueryStringIT extends ESIntegTestCase { refresh(); SearchResponse searchResponse = client().prepareSearch() - .setQuery(simpleQueryStringQuery("the*").analyzeWildcard(true).field("body")).get(); + .setQuery(simpleQueryStringQuery("the*").field("body")).get(); assertNoFailures(searchResponse); assertHitCount(searchResponse, 0L); } diff --git a/docs/reference/query-dsl/query-string-query.asciidoc b/docs/reference/query-dsl/query-string-query.asciidoc index b8ce68721a1..28d8fcffbe2 100644 --- a/docs/reference/query-dsl/query-string-query.asciidoc +++ b/docs/reference/query-dsl/query-string-query.asciidoc @@ -40,10 +40,6 @@ with default operator of `AND`, the same query is translated to |`allow_leading_wildcard` |When set, `*` or `?` are allowed as the first character. Defaults to `true`. -|`lowercase_expanded_terms` |Whether terms of wildcard, prefix, fuzzy, -and range queries are to be automatically lower-cased or not (since they -are not analyzed). Default it `true`. - |`enable_position_increments` |Set to `true` to enable position increments in result queries. Defaults to `true`. @@ -61,12 +57,12 @@ phrase matches are required. Default value is `0`. |`boost` |Sets the boost value of the query. Defaults to `1.0`. +|`auto_generate_phrase_queries` |Defaults to `false`. + |`analyze_wildcard` |By default, wildcards terms in a query string are not analyzed. By setting this value to `true`, a best effort will be made to analyze those as well. -|`auto_generate_phrase_queries` |Defaults to `false`. - |`max_determinized_states` |Limit on how many automaton states regexp queries are allowed to create. This protects against too-difficult (e.g. exponentially hard) regexps. Defaults to 10000. @@ -80,9 +76,6 @@ both>>. |`lenient` |If set to `true` will cause format based failures (like providing text to a numeric field) to be ignored. -|`locale` | Locale that should be used for string conversions. -Defaults to `ROOT`. - |`time_zone` | Time Zone to be applied to any range query related to dates. See also http://www.joda.org/joda-time/apidocs/org/joda/time/DateTimeZone.html[JODA timezone]. diff --git a/docs/reference/query-dsl/query-string-syntax.asciidoc b/docs/reference/query-dsl/query-string-syntax.asciidoc index 49a8b54c506..9c900959196 100644 --- a/docs/reference/query-dsl/query-string-syntax.asciidoc +++ b/docs/reference/query-dsl/query-string-syntax.asciidoc @@ -61,12 +61,15 @@ they match. Leading wildcards can be disabled by setting `allow_leading_wildcard` to `false`. ======= -Wildcarded terms are not analyzed by default -- they are lowercased -(`lowercase_expanded_terms` defaults to `true`) but no further analysis -is done, mainly because it is impossible to accurately analyze a word that -is missing some of its letters. However, by setting `analyze_wildcard` to -`true`, an attempt will be made to analyze wildcarded words before searching -the term list for matching terms. +Only parts of the analysis chain that operate at the character level are +applied. So for instance, if the analyzer performs both lowercasing and +stemming, only the lowercasing will be applied: it would be wrong to perform +stemming on a word that is missing some of its letters. + +By setting `analyze_wildcard` to true, queries that end with a `*` will be +analyzed and a boolean query will be built out of the different tokens, by +ensuring exact matches on the first N-1 tokens, and prefix match on the last +token. ===== Regular expressions diff --git a/docs/reference/query-dsl/simple-query-string-query.asciidoc b/docs/reference/query-dsl/simple-query-string-query.asciidoc index 59e3f727fe0..c6f70c31416 100644 --- a/docs/reference/query-dsl/simple-query-string-query.asciidoc +++ b/docs/reference/query-dsl/simple-query-string-query.asciidoc @@ -44,18 +44,11 @@ creating composite queries. |`flags` |Flags specifying which features of the `simple_query_string` to enable. Defaults to `ALL`. -|`lowercase_expanded_terms` | Whether terms of prefix and fuzzy queries should -be automatically lower-cased or not (since they are not analyzed). Defaults to -`true`. - |`analyze_wildcard` | Whether terms of prefix queries should be automatically analyzed or not. If `true` a best effort will be made to analyze the prefix. However, some analyzers will be not able to provide a meaningful results based just on the prefix of a term. Defaults to `false`. -|`locale` | Locale that should be used for string conversions. -Defaults to `ROOT`. - |`lenient` | If set to `true` will cause format based failures (like providing text to a numeric field) to be ignored. diff --git a/docs/reference/search/count.asciidoc b/docs/reference/search/count.asciidoc index 859455e89b7..d4117e4e96e 100644 --- a/docs/reference/search/count.asciidoc +++ b/docs/reference/search/count.asciidoc @@ -74,9 +74,6 @@ query. |`lenient` |If set to true will cause format based failures (like providing text to a numeric field) to be ignored. Defaults to false. -|`lowercase_expanded_terms` |Should terms be automatically lowercased or -not. Defaults to `true`. - |`analyze_wildcard` |Should wildcard and prefix queries be analyzed or not. Defaults to `false`. diff --git a/docs/reference/search/explain.asciidoc b/docs/reference/search/explain.asciidoc index 1b6560abab4..1291af702f5 100644 --- a/docs/reference/search/explain.asciidoc +++ b/docs/reference/search/explain.asciidoc @@ -136,10 +136,6 @@ This will yield the same result as the previous request. Should wildcard and prefix queries be analyzed or not. Defaults to false. -`lowercase_expanded_terms`:: - Should terms be automatically lowercased - or not. Defaults to true. - `lenient`:: If set to true will cause format based failures (like providing text to a numeric field) to be ignored. Defaults to false. diff --git a/docs/reference/search/uri-request.asciidoc b/docs/reference/search/uri-request.asciidoc index 95ce6a8ff6a..6670b9f31d5 100644 --- a/docs/reference/search/uri-request.asciidoc +++ b/docs/reference/search/uri-request.asciidoc @@ -64,9 +64,6 @@ query. |`analyzer` |The analyzer name to be used when analyzing the query string. -|`lowercase_expanded_terms` |Should terms be automatically lowercased or -not. Defaults to `true`. - |`analyze_wildcard` |Should wildcard and prefix queries be analyzed or not. Defaults to `false`. diff --git a/docs/reference/search/validate.asciidoc b/docs/reference/search/validate.asciidoc index 5fb4ad9b7ce..5b015f4e578 100644 --- a/docs/reference/search/validate.asciidoc +++ b/docs/reference/search/validate.asciidoc @@ -52,9 +52,6 @@ query. |`lenient` |If set to true will cause format based failures (like providing text to a numeric field) to be ignored. Defaults to false. -|`lowercase_expanded_terms` |Should terms be automatically lowercased or -not. Defaults to `true`. - |`analyze_wildcard` |Should wildcard and prefix queries be analyzed or not. Defaults to `false`. |======================================================================= diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/count.json b/rest-api-spec/src/main/resources/rest-api-spec/api/count.json index 9048f982712..0e2697cd524 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/count.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/count.json @@ -67,10 +67,6 @@ "lenient": { "type" : "boolean", "description" : "Specify whether format-based query failures (such as providing text to a numeric field) should be ignored" - }, - "lowercase_expanded_terms": { - "type" : "boolean", - "description" : "Specify whether query terms should be lowercased" } } }, diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/delete_by_query.json b/rest-api-spec/src/main/resources/rest-api-spec/api/delete_by_query.json index f97492aa7ab..39998fb87fe 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/delete_by_query.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/delete_by_query.json @@ -65,10 +65,6 @@ "type" : "boolean", "description" : "Specify whether format-based query failures (such as providing text to a numeric field) should be ignored" }, - "lowercase_expanded_terms": { - "type" : "boolean", - "description" : "Specify whether query terms should be lowercased" - }, "preference": { "type" : "string", "description" : "Specify the node or shard the operation should be performed on (default: random)" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/explain.json b/rest-api-spec/src/main/resources/rest-api-spec/api/explain.json index 328794ffdd0..0f0d8c132b3 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/explain.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/explain.json @@ -49,10 +49,6 @@ "type" : "boolean", "description" : "Specify whether format-based query failures (such as providing text to a numeric field) should be ignored" }, - "lowercase_expanded_terms": { - "type" : "boolean", - "description" : "Specify whether query terms should be lowercased" - }, "parent": { "type" : "string", "description" : "The ID of the parent document" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.validate_query.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.validate_query.json index 98af689833a..7a0977da194 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.validate_query.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.validate_query.json @@ -63,10 +63,6 @@ "type" : "boolean", "description" : "Specify whether format-based query failures (such as providing text to a numeric field) should be ignored" }, - "lowercase_expanded_terms": { - "type" : "boolean", - "description" : "Specify whether query terms should be lowercased" - }, "rewrite": { "type": "boolean", "description": "Provide a more detailed explanation showing the actual Lucene query that will be executed." diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/search.json b/rest-api-spec/src/main/resources/rest-api-spec/api/search.json index 5aa7a409a06..2cf359ede16 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/search.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/search.json @@ -72,10 +72,6 @@ "type" : "boolean", "description" : "Specify whether format-based query failures (such as providing text to a numeric field) should be ignored" }, - "lowercase_expanded_terms": { - "type" : "boolean", - "description" : "Specify whether query terms should be lowercased" - }, "preference": { "type" : "string", "description" : "Specify the node or shard the operation should be performed on (default: random)" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/update_by_query.json b/rest-api-spec/src/main/resources/rest-api-spec/api/update_by_query.json index 4b9e76ac59f..afa2a79570c 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/update_by_query.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/update_by_query.json @@ -65,10 +65,6 @@ "type" : "boolean", "description" : "Specify whether format-based query failures (such as providing text to a numeric field) should be ignored" }, - "lowercase_expanded_terms": { - "type" : "boolean", - "description" : "Specify whether query terms should be lowercased" - }, "pipeline": { "type" : "string", "description" : "Ingest pipeline to set on index requests made by this action. (default: none)" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/count/20_query_string.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/count/20_query_string.yaml index 933033761e9..70f402691a3 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/count/20_query_string.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/count/20_query_string.yaml @@ -58,15 +58,6 @@ count: index: test q: field:BA* - lowercase_expanded_terms: false - - - match: {count : 0} - - - do: - count: - index: test - q: field:BA* - analyze_wildcard: true - match: {count : 1} diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/explain/30_query_string.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/explain/30_query_string.yaml index 30fe6cc55b6..78ef8c4bc89 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/explain/30_query_string.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/explain/30_query_string.yaml @@ -68,17 +68,6 @@ type: test id: 1 q: field:BA* - lowercase_expanded_terms: false - - - is_false: matched - - - do: - explain: - index: test - type: test - id: 1 - q: field:BA* - analyze_wildcard: true - is_true: matched diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.validate_query/20_query_string.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.validate_query/20_query_string.yaml index c6dd323aa6c..3f96009c12a 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.validate_query/20_query_string.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.validate_query/20_query_string.yaml @@ -49,15 +49,6 @@ indices.validate_query: index: test q: field:BA* - lowercase_expanded_terms: false - - - is_true: valid - - - do: - indices.validate_query: - index: test - q: field:BA* - analyze_wildcard: true - is_true: valid diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search/60_query_string.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/search/60_query_string.yaml index 6fb93bb1044..8533cfd2668 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search/60_query_string.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search/60_query_string.yaml @@ -58,15 +58,6 @@ search: index: test q: field:BA* - lowercase_expanded_terms: false - - - match: {hits.total: 0} - - - do: - search: - index: test - q: field:BA* - analyze_wildcard: true - match: {hits.total: 1} From eac04e2adc1d09a0de2c08489ce0b0c11cfb777e Mon Sep 17 00:00:00 2001 From: Shane Connelly Date: Wed, 2 Nov 2016 06:38:50 -0700 Subject: [PATCH 095/132] Drops any plugins that haven't been updated since 2.0. Updates 5.0 links/verbiage --- docs/plugins/alerting.asciidoc | 9 ++++----- docs/plugins/analysis.asciidoc | 4 ---- docs/plugins/api.asciidoc | 7 ------- docs/plugins/discovery.asciidoc | 3 --- docs/plugins/management.asciidoc | 15 ++++++--------- docs/plugins/security.asciidoc | 11 +++++------ 6 files changed, 15 insertions(+), 34 deletions(-) diff --git a/docs/plugins/alerting.asciidoc b/docs/plugins/alerting.asciidoc index 9472dbb6382..1e365306a84 100644 --- a/docs/plugins/alerting.asciidoc +++ b/docs/plugins/alerting.asciidoc @@ -8,11 +8,10 @@ Alerting plugins allow Elasticsearch to monitor indices and to trigger alerts wh The core alerting plugins are: -link:/products/watcher[Watcher]:: +link:/products/x-pack/alerting[X-Pack]:: -Watcher is the alerting and notification product for Elasticsearch that lets -you take action based on changes in your data. It is designed around the +X-Pack contains the alerting and notification product for Elasticsearch that +lets you take action based on changes in your data. It is designed around the principle that if you can query something in Elasticsearch, you can alert on it. Simply define a query, condition, schedule, and the actions to take, and -Watcher will do the rest. - +X-Pack will do the rest. diff --git a/docs/plugins/analysis.asciidoc b/docs/plugins/analysis.asciidoc index df68be57ab9..3c3df021de5 100644 --- a/docs/plugins/analysis.asciidoc +++ b/docs/plugins/analysis.asciidoc @@ -45,11 +45,9 @@ Provides stemming for Ukrainian. A number of analysis plugins have been contributed by our community: -* https://github.com/yakaz/elasticsearch-analysis-combo/[Combo Analysis Plugin] (by Olivier Favre, Yakaz) * https://github.com/synhershko/elasticsearch-analysis-hebrew[Hebrew Analysis Plugin] (by Itamar Syn-Hershko) * https://github.com/medcl/elasticsearch-analysis-ik[IK Analysis Plugin] (by Medcl) * https://github.com/medcl/elasticsearch-analysis-mmseg[Mmseg Analysis Plugin] (by Medcl) -* https://github.com/chytreg/elasticsearch-analysis-morfologik[Morfologik (Polish) Analysis plugin] (by chytreg) * https://github.com/imotov/elasticsearch-analysis-morphology[Russian and English Morphological Analysis Plugin] (by Igor Motov) * https://github.com/medcl/elasticsearch-analysis-pinyin[Pinyin Analysis Plugin] (by Medcl) * https://github.com/duydo/elasticsearch-analysis-vietnamese[Vietnamese Analysis Plugin] (by Duy Do) @@ -67,5 +65,3 @@ include::analysis-smartcn.asciidoc[] include::analysis-stempel.asciidoc[] include::analysis-ukrainian.asciidoc[] - - diff --git a/docs/plugins/api.asciidoc b/docs/plugins/api.asciidoc index 54edcbc7f0e..a2fbc5165ac 100644 --- a/docs/plugins/api.asciidoc +++ b/docs/plugins/api.asciidoc @@ -14,10 +14,6 @@ A number of plugins have been contributed by our community: * https://github.com/wikimedia/search-extra[Elasticsearch Trigram Accelerated Regular Expression Filter]: (by Wikimedia Foundation/Nik Everett) -* https://github.com/kzwang/elasticsearch-image[Elasticsearch Image Plugin]: - Uses https://code.google.com/p/lire/[Lire (Lucene Image Retrieval)] to allow users - to index images and search for similar images (by Kevin Wang) - * https://github.com/wikimedia/search-highlighter[Elasticsearch Experimental Highlighter]: (by Wikimedia Foundation/Nik Everett) @@ -30,7 +26,4 @@ A number of plugins have been contributed by our community: * https://github.com/codelibs/elasticsearch-taste[Elasticsearch Taste Plugin]: Mahout Taste-based Collaborative Filtering implementation (by CodeLibs Project) -* https://github.com/hadashiA/elasticsearch-flavor[Elasticsearch Flavor Plugin] using - http://mahout.apache.org/[Mahout] Collaboration filtering (by hadashiA) * https://github.com/jurgc11/es-change-feed-plugin[WebSocket Change Feed Plugin] (by ForgeRock/Chris Clifton) - diff --git a/docs/plugins/discovery.asciidoc b/docs/plugins/discovery.asciidoc index 96a1c1e7b28..39afbea96dc 100644 --- a/docs/plugins/discovery.asciidoc +++ b/docs/plugins/discovery.asciidoc @@ -30,9 +30,7 @@ The File-based discovery plugin allows providing the unicast hosts list through A number of discovery plugins have been contributed by our community: -* https://github.com/grantr/elasticsearch-srv-discovery[DNS SRV Discovery Plugin] (by Grant Rodgers) * https://github.com/shikhar/eskka[eskka Discovery Plugin] (by Shikhar Bhushan) -* https://github.com/grmblfrz/elasticsearch-zookeeper[ZooKeeper Discovery Plugin] (by Sonian Inc.) * https://github.com/fabric8io/elasticsearch-cloud-kubernetes[Kubernetes Discovery Plugin] (by Jimmi Dyson, http://fabric8.io[fabric8]) include::discovery-ec2.asciidoc[] @@ -42,4 +40,3 @@ include::discovery-azure-classic.asciidoc[] include::discovery-gce.asciidoc[] include::discovery-file.asciidoc[] - diff --git a/docs/plugins/management.asciidoc b/docs/plugins/management.asciidoc index b48d29da3ab..243aaf47e33 100644 --- a/docs/plugins/management.asciidoc +++ b/docs/plugins/management.asciidoc @@ -1,19 +1,16 @@ [[management]] -== Management and Site Plugins +== Management Plugins -Management and site plugins offer UIs for managing and interacting with -Elasticsearch. +Management plugins offer UIs for managing and interacting with Elasticsearch. [float] === Core management plugins The core management plugins are: -link:/products/marvel[Marvel]:: +link:/products/x-pack/monitoring[X-Pack]:: -Marvel is a management and monitoring product for Elasticsearch. Marvel +X-Pack contains the management and monitoring features for Elasticsearch. It aggregates cluster wide statistics and events and offers a single interface to -view and analyze them. Marvel is free for development use but requires a -license to run in production. - - +view and analyze them. You can get a link:/subscriptions[free license] for basic +monitoring or a higher level license for more advanced needs. diff --git a/docs/plugins/security.asciidoc b/docs/plugins/security.asciidoc index 95ba68a6f05..d113c12bfc2 100644 --- a/docs/plugins/security.asciidoc +++ b/docs/plugins/security.asciidoc @@ -8,12 +8,12 @@ Security plugins add a security layer to Elasticsearch. The core security plugins are: -link:/products/shield[Shield]:: +link:/products/x-pack/security[X-Pack]:: -Shield is the Elastic product that makes it easy for anyone to add -enterprise-grade security to their ELK stack. Designed to address the growing security -needs of thousands of enterprises using ELK today, Shield provides peace of -mind when it comes to protecting your data. +X-Pack is the Elastic product that makes it easy for anyone to add +enterprise-grade security to their Elastic Stack. Designed to address the +growing security needs of thousands of enterprises using the Elastic Stack +today, X-Pack provides peace of mind when it comes to protecting your data. [float] === Community contributed security plugins @@ -25,4 +25,3 @@ The following plugins have been contributed by our community: * https://github.com/sscarduzio/elasticsearch-readonlyrest-plugin[Readonly REST]: High performance access control for Elasticsearch native REST API (by Simone Scarduzio) - From b6db00efd89d8bce3977baaee52ad161896cdb41 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 2 Nov 2016 09:51:55 -0400 Subject: [PATCH 096/132] Update docs after test-framework moved Closes #21263 --- .../testing/testing-framework.asciidoc | 5 +- out | 1331 +++++++++++++++++ 2 files changed, 1333 insertions(+), 3 deletions(-) create mode 100644 out diff --git a/docs/reference/testing/testing-framework.asciidoc b/docs/reference/testing/testing-framework.asciidoc index 60ea506a502..94aa6d4b42b 100644 --- a/docs/reference/testing/testing-framework.asciidoc +++ b/docs/reference/testing/testing-framework.asciidoc @@ -30,11 +30,10 @@ First, you need to include the testing dependency in your project, along with th test - org.elasticsearch - elasticsearch + org.elasticsearch.test + framework ${elasticsearch.version} test - test-jar -------------------------------------------------- diff --git a/out b/out new file mode 100644 index 00000000000..97a533d1c90 --- /dev/null +++ b/out @@ -0,0 +1,1331 @@ +{ + ".monitoring-es-2-2016.11.01" : { + "mappings" : { + "indices_stats" : { + "_all" : { + "enabled" : false + }, + "date_detection" : false, + "properties" : { + "cluster_uuid" : { + "type" : "keyword" + }, + "indices_stats" : { + "properties" : { + "_all" : { + "properties" : { + "primaries" : { + "properties" : { + "docs" : { + "properties" : { + "count" : { + "type" : "long" + } + } + }, + "indexing" : { + "properties" : { + "index_time_in_millis" : { + "type" : "long" + }, + "index_total" : { + "type" : "long" + }, + "is_throttled" : { + "type" : "boolean" + }, + "throttle_time_in_millis" : { + "type" : "long" + } + } + }, + "search" : { + "properties" : { + "query_time_in_millis" : { + "type" : "long" + }, + "query_total" : { + "type" : "long" + } + } + }, + "store" : { + "properties" : { + "size_in_bytes" : { + "type" : "long" + } + } + } + } + }, + "total" : { + "properties" : { + "docs" : { + "properties" : { + "count" : { + "type" : "long" + } + } + }, + "indexing" : { + "properties" : { + "index_time_in_millis" : { + "type" : "long" + }, + "index_total" : { + "type" : "long" + }, + "is_throttled" : { + "type" : "boolean" + }, + "throttle_time_in_millis" : { + "type" : "long" + } + } + }, + "search" : { + "properties" : { + "query_time_in_millis" : { + "type" : "long" + }, + "query_total" : { + "type" : "long" + } + } + }, + "store" : { + "properties" : { + "size_in_bytes" : { + "type" : "long" + } + } + } + } + } + } + } + } + }, + "source_node" : { + "properties" : { + "attributes" : { + "dynamic" : "true", + "properties" : { + "client" : { + "type" : "boolean" + }, + "data" : { + "type" : "boolean" + }, + "master" : { + "type" : "boolean" + } + } + }, + "host" : { + "type" : "keyword" + }, + "ip" : { + "type" : "keyword" + }, + "name" : { + "type" : "keyword" + }, + "transport_address" : { + "type" : "keyword" + }, + "uuid" : { + "type" : "keyword" + } + } + }, + "timestamp" : { + "type" : "date", + "format" : "date_time" + } + } + }, + "cluster_state" : { + "_all" : { + "enabled" : false + }, + "date_detection" : false, + "properties" : { + "cluster_state" : { + "properties" : { + "master_node" : { + "type" : "keyword" + }, + "nodes" : { + "type" : "object", + "enabled" : false + }, + "shards" : { + "type" : "object" + }, + "state_uuid" : { + "type" : "keyword" + }, + "status" : { + "type" : "keyword" + }, + "version" : { + "type" : "long" + } + } + }, + "cluster_uuid" : { + "type" : "keyword" + }, + "source_node" : { + "properties" : { + "attributes" : { + "dynamic" : "true", + "properties" : { + "client" : { + "type" : "boolean" + }, + "data" : { + "type" : "boolean" + }, + "master" : { + "type" : "boolean" + } + } + }, + "host" : { + "type" : "keyword" + }, + "ip" : { + "type" : "keyword" + }, + "name" : { + "type" : "keyword" + }, + "transport_address" : { + "type" : "keyword" + }, + "uuid" : { + "type" : "keyword" + } + } + }, + "timestamp" : { + "type" : "date", + "format" : "date_time" + } + } + }, + "node" : { + "_all" : { + "enabled" : false + }, + "date_detection" : false, + "properties" : { + "cluster_uuid" : { + "type" : "keyword" + }, + "node" : { + "properties" : { + "id" : { + "type" : "keyword" + } + } + }, + "source_node" : { + "properties" : { + "attributes" : { + "dynamic" : "true", + "properties" : { + "client" : { + "type" : "boolean" + }, + "data" : { + "type" : "boolean" + }, + "master" : { + "type" : "boolean" + } + } + }, + "host" : { + "type" : "keyword" + }, + "ip" : { + "type" : "keyword" + }, + "name" : { + "type" : "keyword" + }, + "transport_address" : { + "type" : "keyword" + }, + "uuid" : { + "type" : "keyword" + } + } + }, + "state_uuid" : { + "type" : "keyword" + }, + "timestamp" : { + "type" : "date", + "format" : "date_time" + } + } + }, + "_default_" : { + "_all" : { + "enabled" : false + }, + "date_detection" : false, + "properties" : { + "cluster_uuid" : { + "type" : "keyword" + }, + "source_node" : { + "properties" : { + "attributes" : { + "dynamic" : "true", + "properties" : { + "client" : { + "type" : "boolean" + }, + "data" : { + "type" : "boolean" + }, + "master" : { + "type" : "boolean" + } + } + }, + "host" : { + "type" : "keyword" + }, + "ip" : { + "type" : "keyword" + }, + "name" : { + "type" : "keyword" + }, + "transport_address" : { + "type" : "keyword" + }, + "uuid" : { + "type" : "keyword" + } + } + }, + "timestamp" : { + "type" : "date", + "format" : "date_time" + } + } + }, + "index_recovery" : { + "_all" : { + "enabled" : false + }, + "date_detection" : false, + "properties" : { + "cluster_uuid" : { + "type" : "keyword" + }, + "index_recovery" : { + "type" : "object", + "enabled" : false + }, + "source_node" : { + "properties" : { + "attributes" : { + "dynamic" : "true", + "properties" : { + "client" : { + "type" : "boolean" + }, + "data" : { + "type" : "boolean" + }, + "master" : { + "type" : "boolean" + } + } + }, + "host" : { + "type" : "keyword" + }, + "ip" : { + "type" : "keyword" + }, + "name" : { + "type" : "keyword" + }, + "transport_address" : { + "type" : "keyword" + }, + "uuid" : { + "type" : "keyword" + } + } + }, + "timestamp" : { + "type" : "date", + "format" : "date_time" + } + } + }, + "shards" : { + "_all" : { + "enabled" : false + }, + "date_detection" : false, + "properties" : { + "cluster_uuid" : { + "type" : "keyword" + }, + "shard" : { + "properties" : { + "index" : { + "type" : "keyword" + }, + "node" : { + "type" : "keyword" + }, + "primary" : { + "type" : "boolean" + }, + "relocating_node" : { + "type" : "keyword" + }, + "shard" : { + "type" : "long" + }, + "state" : { + "type" : "keyword" + } + } + }, + "source_node" : { + "properties" : { + "attributes" : { + "dynamic" : "true", + "properties" : { + "client" : { + "type" : "boolean" + }, + "data" : { + "type" : "boolean" + }, + "master" : { + "type" : "boolean" + } + } + }, + "host" : { + "type" : "keyword" + }, + "ip" : { + "type" : "keyword" + }, + "name" : { + "type" : "keyword" + }, + "transport_address" : { + "type" : "keyword" + }, + "uuid" : { + "type" : "keyword" + } + } + }, + "state_uuid" : { + "type" : "keyword" + }, + "timestamp" : { + "type" : "date", + "format" : "date_time" + } + } + }, + "node_stats" : { + "_all" : { + "enabled" : false + }, + "date_detection" : false, + "properties" : { + "cluster_uuid" : { + "type" : "keyword" + }, + "node_stats" : { + "properties" : { + "fs" : { + "properties" : { + "data" : { + "properties" : { + "spins" : { + "type" : "boolean" + } + } + }, + "io_stats" : { + "properties" : { + "total" : { + "properties" : { + "operations" : { + "type" : "long" + }, + "read_kilobytes" : { + "type" : "long" + }, + "read_operations" : { + "type" : "long" + }, + "write_kilobytes" : { + "type" : "long" + }, + "write_operations" : { + "type" : "long" + } + } + } + } + }, + "total" : { + "properties" : { + "available_in_bytes" : { + "type" : "long" + }, + "free_in_bytes" : { + "type" : "long" + }, + "total_in_bytes" : { + "type" : "long" + } + } + } + } + }, + "indices" : { + "properties" : { + "docs" : { + "properties" : { + "count" : { + "type" : "long" + } + } + }, + "fielddata" : { + "properties" : { + "evictions" : { + "type" : "long" + }, + "memory_size_in_bytes" : { + "type" : "long" + } + } + }, + "indexing" : { + "properties" : { + "index_time_in_millis" : { + "type" : "long" + }, + "index_total" : { + "type" : "long" + }, + "throttle_time_in_millis" : { + "type" : "long" + } + } + }, + "query_cache" : { + "properties" : { + "evictions" : { + "type" : "long" + }, + "hit_count" : { + "type" : "long" + }, + "memory_size_in_bytes" : { + "type" : "long" + }, + "miss_count" : { + "type" : "long" + } + } + }, + "request_cache" : { + "properties" : { + "evictions" : { + "type" : "long" + }, + "hit_count" : { + "type" : "long" + }, + "memory_size_in_bytes" : { + "type" : "long" + }, + "miss_count" : { + "type" : "long" + } + } + }, + "search" : { + "properties" : { + "query_time_in_millis" : { + "type" : "long" + }, + "query_total" : { + "type" : "long" + } + } + }, + "segments" : { + "properties" : { + "count" : { + "type" : "integer" + }, + "doc_values_memory_in_bytes" : { + "type" : "long" + }, + "fixed_bit_set_memory_in_bytes" : { + "type" : "long" + }, + "index_writer_memory_in_bytes" : { + "type" : "long" + }, + "memory_in_bytes" : { + "type" : "long" + }, + "norms_memory_in_bytes" : { + "type" : "long" + }, + "points_memory_in_bytes" : { + "type" : "long" + }, + "stored_fields_memory_in_bytes" : { + "type" : "long" + }, + "term_vectors_memory_in_bytes" : { + "type" : "long" + }, + "terms_memory_in_bytes" : { + "type" : "long" + }, + "version_map_memory_in_bytes" : { + "type" : "long" + } + } + }, + "store" : { + "properties" : { + "size_in_bytes" : { + "type" : "long" + }, + "throttle_time_in_millis" : { + "type" : "long" + } + } + } + } + }, + "jvm" : { + "properties" : { + "gc" : { + "properties" : { + "collectors" : { + "properties" : { + "old" : { + "properties" : { + "collection_count" : { + "type" : "long" + }, + "collection_time_in_millis" : { + "type" : "long" + } + } + }, + "young" : { + "properties" : { + "collection_count" : { + "type" : "long" + }, + "collection_time_in_millis" : { + "type" : "long" + } + } + } + } + } + } + }, + "mem" : { + "properties" : { + "heap_max_in_bytes" : { + "type" : "long" + }, + "heap_used_in_bytes" : { + "type" : "long" + }, + "heap_used_percent" : { + "type" : "long" + } + } + } + } + }, + "mlockall" : { + "type" : "boolean" + }, + "node_id" : { + "type" : "keyword" + }, + "node_master" : { + "type" : "boolean" + }, + "os" : { + "properties" : { + "cpu" : { + "properties" : { + "load_average" : { + "properties" : { + "1m" : { + "type" : "float" + } + } + } + } + } + } + }, + "process" : { + "properties" : { + "cpu" : { + "properties" : { + "percent" : { + "type" : "long" + } + } + }, + "max_file_descriptors" : { + "type" : "long" + }, + "open_file_descriptors" : { + "type" : "long" + } + } + }, + "thread_pool" : { + "properties" : { + "bulk" : { + "properties" : { + "queue" : { + "type" : "long" + }, + "rejected" : { + "type" : "long" + }, + "threads" : { + "type" : "long" + } + } + }, + "generic" : { + "properties" : { + "queue" : { + "type" : "long" + }, + "rejected" : { + "type" : "long" + }, + "threads" : { + "type" : "long" + } + } + }, + "get" : { + "properties" : { + "queue" : { + "type" : "long" + }, + "rejected" : { + "type" : "long" + }, + "threads" : { + "type" : "long" + } + } + }, + "index" : { + "properties" : { + "queue" : { + "type" : "long" + }, + "rejected" : { + "type" : "long" + }, + "threads" : { + "type" : "long" + } + } + }, + "management" : { + "properties" : { + "queue" : { + "type" : "long" + }, + "rejected" : { + "type" : "long" + }, + "threads" : { + "type" : "long" + } + } + }, + "search" : { + "properties" : { + "queue" : { + "type" : "long" + }, + "rejected" : { + "type" : "long" + }, + "threads" : { + "type" : "long" + } + } + }, + "watcher" : { + "properties" : { + "queue" : { + "type" : "long" + }, + "rejected" : { + "type" : "long" + }, + "threads" : { + "type" : "long" + } + } + } + } + } + } + }, + "source_node" : { + "properties" : { + "attributes" : { + "dynamic" : "true", + "properties" : { + "client" : { + "type" : "boolean" + }, + "data" : { + "type" : "boolean" + }, + "master" : { + "type" : "boolean" + } + } + }, + "host" : { + "type" : "keyword" + }, + "ip" : { + "type" : "keyword" + }, + "name" : { + "type" : "keyword" + }, + "transport_address" : { + "type" : "keyword" + }, + "uuid" : { + "type" : "keyword" + } + } + }, + "timestamp" : { + "type" : "date", + "format" : "date_time" + } + } + }, + "cluster_stats" : { + "_all" : { + "enabled" : false + }, + "date_detection" : false, + "properties" : { + "cluster_stats" : { + "properties" : { + "indices" : { + "properties" : { + "count" : { + "type" : "long" + }, + "docs" : { + "properties" : { + "count" : { + "type" : "long" + } + } + }, + "shards" : { + "properties" : { + "index" : { + "properties" : { + "replication" : { + "properties" : { + "min" : { + "type" : "float" + } + } + } + } + }, + "primaries" : { + "type" : "long" + }, + "replication" : { + "type" : "float" + }, + "total" : { + "type" : "long" + } + } + }, + "store" : { + "properties" : { + "size_in_bytes" : { + "type" : "long" + } + } + } + } + }, + "nodes" : { + "properties" : { + "count" : { + "properties" : { + "total" : { + "type" : "long" + } + } + }, + "fs" : { + "properties" : { + "available_in_bytes" : { + "type" : "long" + }, + "free_in_bytes" : { + "type" : "long" + }, + "total_in_bytes" : { + "type" : "long" + } + } + }, + "jvm" : { + "properties" : { + "max_uptime_in_millis" : { + "type" : "long" + }, + "mem" : { + "properties" : { + "heap_max_in_bytes" : { + "type" : "long" + }, + "heap_used_in_bytes" : { + "type" : "long" + } + } + } + } + }, + "versions" : { + "type" : "text", + "fields" : { + "keyword" : { + "type" : "keyword", + "ignore_above" : 256 + } + } + } + } + } + } + }, + "cluster_uuid" : { + "type" : "keyword" + }, + "source_node" : { + "properties" : { + "attributes" : { + "dynamic" : "true", + "properties" : { + "client" : { + "type" : "boolean" + }, + "data" : { + "type" : "boolean" + }, + "master" : { + "type" : "boolean" + } + } + }, + "host" : { + "type" : "keyword" + }, + "ip" : { + "type" : "keyword" + }, + "name" : { + "type" : "keyword" + }, + "transport_address" : { + "type" : "keyword" + }, + "uuid" : { + "type" : "keyword" + } + } + }, + "timestamp" : { + "type" : "date", + "format" : "date_time" + } + } + }, + "index_stats" : { + "_all" : { + "enabled" : false + }, + "date_detection" : false, + "properties" : { + "cluster_uuid" : { + "type" : "keyword" + }, + "index_stats" : { + "properties" : { + "index" : { + "type" : "keyword" + }, + "primaries" : { + "properties" : { + "docs" : { + "properties" : { + "count" : { + "type" : "long" + } + } + }, + "fielddata" : { + "properties" : { + "evictions" : { + "type" : "long" + }, + "memory_size_in_bytes" : { + "type" : "long" + } + } + }, + "indexing" : { + "properties" : { + "index_time_in_millis" : { + "type" : "long" + }, + "index_total" : { + "type" : "long" + }, + "throttle_time_in_millis" : { + "type" : "long" + } + } + }, + "merges" : { + "properties" : { + "total_size_in_bytes" : { + "type" : "long" + } + } + }, + "query_cache" : { + "properties" : { + "evictions" : { + "type" : "long" + }, + "hit_count" : { + "type" : "long" + }, + "memory_size_in_bytes" : { + "type" : "long" + }, + "miss_count" : { + "type" : "long" + } + } + }, + "refresh" : { + "properties" : { + "total_time_in_millis" : { + "type" : "long" + } + } + }, + "request_cache" : { + "properties" : { + "evictions" : { + "type" : "long" + }, + "hit_count" : { + "type" : "long" + }, + "memory_size_in_bytes" : { + "type" : "long" + }, + "miss_count" : { + "type" : "long" + } + } + }, + "search" : { + "properties" : { + "query_time_in_millis" : { + "type" : "long" + }, + "query_total" : { + "type" : "long" + } + } + }, + "segments" : { + "properties" : { + "count" : { + "type" : "integer" + }, + "doc_values_memory_in_bytes" : { + "type" : "long" + }, + "fixed_bit_set_memory_in_bytes" : { + "type" : "long" + }, + "index_writer_memory_in_bytes" : { + "type" : "long" + }, + "memory_in_bytes" : { + "type" : "long" + }, + "norms_memory_in_bytes" : { + "type" : "long" + }, + "points_memory_in_bytes" : { + "type" : "long" + }, + "stored_fields_memory_in_bytes" : { + "type" : "long" + }, + "term_vectors_memory_in_bytes" : { + "type" : "long" + }, + "terms_memory_in_bytes" : { + "type" : "long" + }, + "version_map_memory_in_bytes" : { + "type" : "long" + } + } + }, + "store" : { + "properties" : { + "size_in_bytes" : { + "type" : "long" + }, + "throttle_time_in_millis" : { + "type" : "long" + } + } + } + } + }, + "total" : { + "properties" : { + "docs" : { + "properties" : { + "count" : { + "type" : "long" + } + } + }, + "fielddata" : { + "properties" : { + "evictions" : { + "type" : "long" + }, + "memory_size_in_bytes" : { + "type" : "long" + } + } + }, + "indexing" : { + "properties" : { + "index_time_in_millis" : { + "type" : "long" + }, + "index_total" : { + "type" : "long" + }, + "throttle_time_in_millis" : { + "type" : "long" + } + } + }, + "merges" : { + "properties" : { + "total_size_in_bytes" : { + "type" : "long" + } + } + }, + "query_cache" : { + "properties" : { + "evictions" : { + "type" : "long" + }, + "hit_count" : { + "type" : "long" + }, + "memory_size_in_bytes" : { + "type" : "long" + }, + "miss_count" : { + "type" : "long" + } + } + }, + "refresh" : { + "properties" : { + "total_time_in_millis" : { + "type" : "long" + } + } + }, + "request_cache" : { + "properties" : { + "evictions" : { + "type" : "long" + }, + "hit_count" : { + "type" : "long" + }, + "memory_size_in_bytes" : { + "type" : "long" + }, + "miss_count" : { + "type" : "long" + } + } + }, + "search" : { + "properties" : { + "query_time_in_millis" : { + "type" : "long" + }, + "query_total" : { + "type" : "long" + } + } + }, + "segments" : { + "properties" : { + "count" : { + "type" : "integer" + }, + "doc_values_memory_in_bytes" : { + "type" : "long" + }, + "fixed_bit_set_memory_in_bytes" : { + "type" : "long" + }, + "index_writer_memory_in_bytes" : { + "type" : "long" + }, + "memory_in_bytes" : { + "type" : "long" + }, + "norms_memory_in_bytes" : { + "type" : "long" + }, + "points_memory_in_bytes" : { + "type" : "long" + }, + "stored_fields_memory_in_bytes" : { + "type" : "long" + }, + "term_vectors_memory_in_bytes" : { + "type" : "long" + }, + "terms_memory_in_bytes" : { + "type" : "long" + }, + "version_map_memory_in_bytes" : { + "type" : "long" + } + } + }, + "store" : { + "properties" : { + "size_in_bytes" : { + "type" : "long" + }, + "throttle_time_in_millis" : { + "type" : "long" + } + } + } + } + } + } + }, + "source_node" : { + "properties" : { + "attributes" : { + "dynamic" : "true", + "properties" : { + "client" : { + "type" : "boolean" + }, + "data" : { + "type" : "boolean" + }, + "master" : { + "type" : "boolean" + } + } + }, + "host" : { + "type" : "keyword" + }, + "ip" : { + "type" : "keyword" + }, + "name" : { + "type" : "keyword" + }, + "transport_address" : { + "type" : "keyword" + }, + "uuid" : { + "type" : "keyword" + } + } + }, + "timestamp" : { + "type" : "date", + "format" : "date_time" + } + } + } + } + } +} From f3ee6a8e0d0593a9c3f3dc7cc88dbdb4248427cf Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 2 Nov 2016 09:53:53 -0400 Subject: [PATCH 097/132] Remove extra file Pushed by mistake.... --- out | 1331 ----------------------------------------------------------- 1 file changed, 1331 deletions(-) delete mode 100644 out diff --git a/out b/out deleted file mode 100644 index 97a533d1c90..00000000000 --- a/out +++ /dev/null @@ -1,1331 +0,0 @@ -{ - ".monitoring-es-2-2016.11.01" : { - "mappings" : { - "indices_stats" : { - "_all" : { - "enabled" : false - }, - "date_detection" : false, - "properties" : { - "cluster_uuid" : { - "type" : "keyword" - }, - "indices_stats" : { - "properties" : { - "_all" : { - "properties" : { - "primaries" : { - "properties" : { - "docs" : { - "properties" : { - "count" : { - "type" : "long" - } - } - }, - "indexing" : { - "properties" : { - "index_time_in_millis" : { - "type" : "long" - }, - "index_total" : { - "type" : "long" - }, - "is_throttled" : { - "type" : "boolean" - }, - "throttle_time_in_millis" : { - "type" : "long" - } - } - }, - "search" : { - "properties" : { - "query_time_in_millis" : { - "type" : "long" - }, - "query_total" : { - "type" : "long" - } - } - }, - "store" : { - "properties" : { - "size_in_bytes" : { - "type" : "long" - } - } - } - } - }, - "total" : { - "properties" : { - "docs" : { - "properties" : { - "count" : { - "type" : "long" - } - } - }, - "indexing" : { - "properties" : { - "index_time_in_millis" : { - "type" : "long" - }, - "index_total" : { - "type" : "long" - }, - "is_throttled" : { - "type" : "boolean" - }, - "throttle_time_in_millis" : { - "type" : "long" - } - } - }, - "search" : { - "properties" : { - "query_time_in_millis" : { - "type" : "long" - }, - "query_total" : { - "type" : "long" - } - } - }, - "store" : { - "properties" : { - "size_in_bytes" : { - "type" : "long" - } - } - } - } - } - } - } - } - }, - "source_node" : { - "properties" : { - "attributes" : { - "dynamic" : "true", - "properties" : { - "client" : { - "type" : "boolean" - }, - "data" : { - "type" : "boolean" - }, - "master" : { - "type" : "boolean" - } - } - }, - "host" : { - "type" : "keyword" - }, - "ip" : { - "type" : "keyword" - }, - "name" : { - "type" : "keyword" - }, - "transport_address" : { - "type" : "keyword" - }, - "uuid" : { - "type" : "keyword" - } - } - }, - "timestamp" : { - "type" : "date", - "format" : "date_time" - } - } - }, - "cluster_state" : { - "_all" : { - "enabled" : false - }, - "date_detection" : false, - "properties" : { - "cluster_state" : { - "properties" : { - "master_node" : { - "type" : "keyword" - }, - "nodes" : { - "type" : "object", - "enabled" : false - }, - "shards" : { - "type" : "object" - }, - "state_uuid" : { - "type" : "keyword" - }, - "status" : { - "type" : "keyword" - }, - "version" : { - "type" : "long" - } - } - }, - "cluster_uuid" : { - "type" : "keyword" - }, - "source_node" : { - "properties" : { - "attributes" : { - "dynamic" : "true", - "properties" : { - "client" : { - "type" : "boolean" - }, - "data" : { - "type" : "boolean" - }, - "master" : { - "type" : "boolean" - } - } - }, - "host" : { - "type" : "keyword" - }, - "ip" : { - "type" : "keyword" - }, - "name" : { - "type" : "keyword" - }, - "transport_address" : { - "type" : "keyword" - }, - "uuid" : { - "type" : "keyword" - } - } - }, - "timestamp" : { - "type" : "date", - "format" : "date_time" - } - } - }, - "node" : { - "_all" : { - "enabled" : false - }, - "date_detection" : false, - "properties" : { - "cluster_uuid" : { - "type" : "keyword" - }, - "node" : { - "properties" : { - "id" : { - "type" : "keyword" - } - } - }, - "source_node" : { - "properties" : { - "attributes" : { - "dynamic" : "true", - "properties" : { - "client" : { - "type" : "boolean" - }, - "data" : { - "type" : "boolean" - }, - "master" : { - "type" : "boolean" - } - } - }, - "host" : { - "type" : "keyword" - }, - "ip" : { - "type" : "keyword" - }, - "name" : { - "type" : "keyword" - }, - "transport_address" : { - "type" : "keyword" - }, - "uuid" : { - "type" : "keyword" - } - } - }, - "state_uuid" : { - "type" : "keyword" - }, - "timestamp" : { - "type" : "date", - "format" : "date_time" - } - } - }, - "_default_" : { - "_all" : { - "enabled" : false - }, - "date_detection" : false, - "properties" : { - "cluster_uuid" : { - "type" : "keyword" - }, - "source_node" : { - "properties" : { - "attributes" : { - "dynamic" : "true", - "properties" : { - "client" : { - "type" : "boolean" - }, - "data" : { - "type" : "boolean" - }, - "master" : { - "type" : "boolean" - } - } - }, - "host" : { - "type" : "keyword" - }, - "ip" : { - "type" : "keyword" - }, - "name" : { - "type" : "keyword" - }, - "transport_address" : { - "type" : "keyword" - }, - "uuid" : { - "type" : "keyword" - } - } - }, - "timestamp" : { - "type" : "date", - "format" : "date_time" - } - } - }, - "index_recovery" : { - "_all" : { - "enabled" : false - }, - "date_detection" : false, - "properties" : { - "cluster_uuid" : { - "type" : "keyword" - }, - "index_recovery" : { - "type" : "object", - "enabled" : false - }, - "source_node" : { - "properties" : { - "attributes" : { - "dynamic" : "true", - "properties" : { - "client" : { - "type" : "boolean" - }, - "data" : { - "type" : "boolean" - }, - "master" : { - "type" : "boolean" - } - } - }, - "host" : { - "type" : "keyword" - }, - "ip" : { - "type" : "keyword" - }, - "name" : { - "type" : "keyword" - }, - "transport_address" : { - "type" : "keyword" - }, - "uuid" : { - "type" : "keyword" - } - } - }, - "timestamp" : { - "type" : "date", - "format" : "date_time" - } - } - }, - "shards" : { - "_all" : { - "enabled" : false - }, - "date_detection" : false, - "properties" : { - "cluster_uuid" : { - "type" : "keyword" - }, - "shard" : { - "properties" : { - "index" : { - "type" : "keyword" - }, - "node" : { - "type" : "keyword" - }, - "primary" : { - "type" : "boolean" - }, - "relocating_node" : { - "type" : "keyword" - }, - "shard" : { - "type" : "long" - }, - "state" : { - "type" : "keyword" - } - } - }, - "source_node" : { - "properties" : { - "attributes" : { - "dynamic" : "true", - "properties" : { - "client" : { - "type" : "boolean" - }, - "data" : { - "type" : "boolean" - }, - "master" : { - "type" : "boolean" - } - } - }, - "host" : { - "type" : "keyword" - }, - "ip" : { - "type" : "keyword" - }, - "name" : { - "type" : "keyword" - }, - "transport_address" : { - "type" : "keyword" - }, - "uuid" : { - "type" : "keyword" - } - } - }, - "state_uuid" : { - "type" : "keyword" - }, - "timestamp" : { - "type" : "date", - "format" : "date_time" - } - } - }, - "node_stats" : { - "_all" : { - "enabled" : false - }, - "date_detection" : false, - "properties" : { - "cluster_uuid" : { - "type" : "keyword" - }, - "node_stats" : { - "properties" : { - "fs" : { - "properties" : { - "data" : { - "properties" : { - "spins" : { - "type" : "boolean" - } - } - }, - "io_stats" : { - "properties" : { - "total" : { - "properties" : { - "operations" : { - "type" : "long" - }, - "read_kilobytes" : { - "type" : "long" - }, - "read_operations" : { - "type" : "long" - }, - "write_kilobytes" : { - "type" : "long" - }, - "write_operations" : { - "type" : "long" - } - } - } - } - }, - "total" : { - "properties" : { - "available_in_bytes" : { - "type" : "long" - }, - "free_in_bytes" : { - "type" : "long" - }, - "total_in_bytes" : { - "type" : "long" - } - } - } - } - }, - "indices" : { - "properties" : { - "docs" : { - "properties" : { - "count" : { - "type" : "long" - } - } - }, - "fielddata" : { - "properties" : { - "evictions" : { - "type" : "long" - }, - "memory_size_in_bytes" : { - "type" : "long" - } - } - }, - "indexing" : { - "properties" : { - "index_time_in_millis" : { - "type" : "long" - }, - "index_total" : { - "type" : "long" - }, - "throttle_time_in_millis" : { - "type" : "long" - } - } - }, - "query_cache" : { - "properties" : { - "evictions" : { - "type" : "long" - }, - "hit_count" : { - "type" : "long" - }, - "memory_size_in_bytes" : { - "type" : "long" - }, - "miss_count" : { - "type" : "long" - } - } - }, - "request_cache" : { - "properties" : { - "evictions" : { - "type" : "long" - }, - "hit_count" : { - "type" : "long" - }, - "memory_size_in_bytes" : { - "type" : "long" - }, - "miss_count" : { - "type" : "long" - } - } - }, - "search" : { - "properties" : { - "query_time_in_millis" : { - "type" : "long" - }, - "query_total" : { - "type" : "long" - } - } - }, - "segments" : { - "properties" : { - "count" : { - "type" : "integer" - }, - "doc_values_memory_in_bytes" : { - "type" : "long" - }, - "fixed_bit_set_memory_in_bytes" : { - "type" : "long" - }, - "index_writer_memory_in_bytes" : { - "type" : "long" - }, - "memory_in_bytes" : { - "type" : "long" - }, - "norms_memory_in_bytes" : { - "type" : "long" - }, - "points_memory_in_bytes" : { - "type" : "long" - }, - "stored_fields_memory_in_bytes" : { - "type" : "long" - }, - "term_vectors_memory_in_bytes" : { - "type" : "long" - }, - "terms_memory_in_bytes" : { - "type" : "long" - }, - "version_map_memory_in_bytes" : { - "type" : "long" - } - } - }, - "store" : { - "properties" : { - "size_in_bytes" : { - "type" : "long" - }, - "throttle_time_in_millis" : { - "type" : "long" - } - } - } - } - }, - "jvm" : { - "properties" : { - "gc" : { - "properties" : { - "collectors" : { - "properties" : { - "old" : { - "properties" : { - "collection_count" : { - "type" : "long" - }, - "collection_time_in_millis" : { - "type" : "long" - } - } - }, - "young" : { - "properties" : { - "collection_count" : { - "type" : "long" - }, - "collection_time_in_millis" : { - "type" : "long" - } - } - } - } - } - } - }, - "mem" : { - "properties" : { - "heap_max_in_bytes" : { - "type" : "long" - }, - "heap_used_in_bytes" : { - "type" : "long" - }, - "heap_used_percent" : { - "type" : "long" - } - } - } - } - }, - "mlockall" : { - "type" : "boolean" - }, - "node_id" : { - "type" : "keyword" - }, - "node_master" : { - "type" : "boolean" - }, - "os" : { - "properties" : { - "cpu" : { - "properties" : { - "load_average" : { - "properties" : { - "1m" : { - "type" : "float" - } - } - } - } - } - } - }, - "process" : { - "properties" : { - "cpu" : { - "properties" : { - "percent" : { - "type" : "long" - } - } - }, - "max_file_descriptors" : { - "type" : "long" - }, - "open_file_descriptors" : { - "type" : "long" - } - } - }, - "thread_pool" : { - "properties" : { - "bulk" : { - "properties" : { - "queue" : { - "type" : "long" - }, - "rejected" : { - "type" : "long" - }, - "threads" : { - "type" : "long" - } - } - }, - "generic" : { - "properties" : { - "queue" : { - "type" : "long" - }, - "rejected" : { - "type" : "long" - }, - "threads" : { - "type" : "long" - } - } - }, - "get" : { - "properties" : { - "queue" : { - "type" : "long" - }, - "rejected" : { - "type" : "long" - }, - "threads" : { - "type" : "long" - } - } - }, - "index" : { - "properties" : { - "queue" : { - "type" : "long" - }, - "rejected" : { - "type" : "long" - }, - "threads" : { - "type" : "long" - } - } - }, - "management" : { - "properties" : { - "queue" : { - "type" : "long" - }, - "rejected" : { - "type" : "long" - }, - "threads" : { - "type" : "long" - } - } - }, - "search" : { - "properties" : { - "queue" : { - "type" : "long" - }, - "rejected" : { - "type" : "long" - }, - "threads" : { - "type" : "long" - } - } - }, - "watcher" : { - "properties" : { - "queue" : { - "type" : "long" - }, - "rejected" : { - "type" : "long" - }, - "threads" : { - "type" : "long" - } - } - } - } - } - } - }, - "source_node" : { - "properties" : { - "attributes" : { - "dynamic" : "true", - "properties" : { - "client" : { - "type" : "boolean" - }, - "data" : { - "type" : "boolean" - }, - "master" : { - "type" : "boolean" - } - } - }, - "host" : { - "type" : "keyword" - }, - "ip" : { - "type" : "keyword" - }, - "name" : { - "type" : "keyword" - }, - "transport_address" : { - "type" : "keyword" - }, - "uuid" : { - "type" : "keyword" - } - } - }, - "timestamp" : { - "type" : "date", - "format" : "date_time" - } - } - }, - "cluster_stats" : { - "_all" : { - "enabled" : false - }, - "date_detection" : false, - "properties" : { - "cluster_stats" : { - "properties" : { - "indices" : { - "properties" : { - "count" : { - "type" : "long" - }, - "docs" : { - "properties" : { - "count" : { - "type" : "long" - } - } - }, - "shards" : { - "properties" : { - "index" : { - "properties" : { - "replication" : { - "properties" : { - "min" : { - "type" : "float" - } - } - } - } - }, - "primaries" : { - "type" : "long" - }, - "replication" : { - "type" : "float" - }, - "total" : { - "type" : "long" - } - } - }, - "store" : { - "properties" : { - "size_in_bytes" : { - "type" : "long" - } - } - } - } - }, - "nodes" : { - "properties" : { - "count" : { - "properties" : { - "total" : { - "type" : "long" - } - } - }, - "fs" : { - "properties" : { - "available_in_bytes" : { - "type" : "long" - }, - "free_in_bytes" : { - "type" : "long" - }, - "total_in_bytes" : { - "type" : "long" - } - } - }, - "jvm" : { - "properties" : { - "max_uptime_in_millis" : { - "type" : "long" - }, - "mem" : { - "properties" : { - "heap_max_in_bytes" : { - "type" : "long" - }, - "heap_used_in_bytes" : { - "type" : "long" - } - } - } - } - }, - "versions" : { - "type" : "text", - "fields" : { - "keyword" : { - "type" : "keyword", - "ignore_above" : 256 - } - } - } - } - } - } - }, - "cluster_uuid" : { - "type" : "keyword" - }, - "source_node" : { - "properties" : { - "attributes" : { - "dynamic" : "true", - "properties" : { - "client" : { - "type" : "boolean" - }, - "data" : { - "type" : "boolean" - }, - "master" : { - "type" : "boolean" - } - } - }, - "host" : { - "type" : "keyword" - }, - "ip" : { - "type" : "keyword" - }, - "name" : { - "type" : "keyword" - }, - "transport_address" : { - "type" : "keyword" - }, - "uuid" : { - "type" : "keyword" - } - } - }, - "timestamp" : { - "type" : "date", - "format" : "date_time" - } - } - }, - "index_stats" : { - "_all" : { - "enabled" : false - }, - "date_detection" : false, - "properties" : { - "cluster_uuid" : { - "type" : "keyword" - }, - "index_stats" : { - "properties" : { - "index" : { - "type" : "keyword" - }, - "primaries" : { - "properties" : { - "docs" : { - "properties" : { - "count" : { - "type" : "long" - } - } - }, - "fielddata" : { - "properties" : { - "evictions" : { - "type" : "long" - }, - "memory_size_in_bytes" : { - "type" : "long" - } - } - }, - "indexing" : { - "properties" : { - "index_time_in_millis" : { - "type" : "long" - }, - "index_total" : { - "type" : "long" - }, - "throttle_time_in_millis" : { - "type" : "long" - } - } - }, - "merges" : { - "properties" : { - "total_size_in_bytes" : { - "type" : "long" - } - } - }, - "query_cache" : { - "properties" : { - "evictions" : { - "type" : "long" - }, - "hit_count" : { - "type" : "long" - }, - "memory_size_in_bytes" : { - "type" : "long" - }, - "miss_count" : { - "type" : "long" - } - } - }, - "refresh" : { - "properties" : { - "total_time_in_millis" : { - "type" : "long" - } - } - }, - "request_cache" : { - "properties" : { - "evictions" : { - "type" : "long" - }, - "hit_count" : { - "type" : "long" - }, - "memory_size_in_bytes" : { - "type" : "long" - }, - "miss_count" : { - "type" : "long" - } - } - }, - "search" : { - "properties" : { - "query_time_in_millis" : { - "type" : "long" - }, - "query_total" : { - "type" : "long" - } - } - }, - "segments" : { - "properties" : { - "count" : { - "type" : "integer" - }, - "doc_values_memory_in_bytes" : { - "type" : "long" - }, - "fixed_bit_set_memory_in_bytes" : { - "type" : "long" - }, - "index_writer_memory_in_bytes" : { - "type" : "long" - }, - "memory_in_bytes" : { - "type" : "long" - }, - "norms_memory_in_bytes" : { - "type" : "long" - }, - "points_memory_in_bytes" : { - "type" : "long" - }, - "stored_fields_memory_in_bytes" : { - "type" : "long" - }, - "term_vectors_memory_in_bytes" : { - "type" : "long" - }, - "terms_memory_in_bytes" : { - "type" : "long" - }, - "version_map_memory_in_bytes" : { - "type" : "long" - } - } - }, - "store" : { - "properties" : { - "size_in_bytes" : { - "type" : "long" - }, - "throttle_time_in_millis" : { - "type" : "long" - } - } - } - } - }, - "total" : { - "properties" : { - "docs" : { - "properties" : { - "count" : { - "type" : "long" - } - } - }, - "fielddata" : { - "properties" : { - "evictions" : { - "type" : "long" - }, - "memory_size_in_bytes" : { - "type" : "long" - } - } - }, - "indexing" : { - "properties" : { - "index_time_in_millis" : { - "type" : "long" - }, - "index_total" : { - "type" : "long" - }, - "throttle_time_in_millis" : { - "type" : "long" - } - } - }, - "merges" : { - "properties" : { - "total_size_in_bytes" : { - "type" : "long" - } - } - }, - "query_cache" : { - "properties" : { - "evictions" : { - "type" : "long" - }, - "hit_count" : { - "type" : "long" - }, - "memory_size_in_bytes" : { - "type" : "long" - }, - "miss_count" : { - "type" : "long" - } - } - }, - "refresh" : { - "properties" : { - "total_time_in_millis" : { - "type" : "long" - } - } - }, - "request_cache" : { - "properties" : { - "evictions" : { - "type" : "long" - }, - "hit_count" : { - "type" : "long" - }, - "memory_size_in_bytes" : { - "type" : "long" - }, - "miss_count" : { - "type" : "long" - } - } - }, - "search" : { - "properties" : { - "query_time_in_millis" : { - "type" : "long" - }, - "query_total" : { - "type" : "long" - } - } - }, - "segments" : { - "properties" : { - "count" : { - "type" : "integer" - }, - "doc_values_memory_in_bytes" : { - "type" : "long" - }, - "fixed_bit_set_memory_in_bytes" : { - "type" : "long" - }, - "index_writer_memory_in_bytes" : { - "type" : "long" - }, - "memory_in_bytes" : { - "type" : "long" - }, - "norms_memory_in_bytes" : { - "type" : "long" - }, - "points_memory_in_bytes" : { - "type" : "long" - }, - "stored_fields_memory_in_bytes" : { - "type" : "long" - }, - "term_vectors_memory_in_bytes" : { - "type" : "long" - }, - "terms_memory_in_bytes" : { - "type" : "long" - }, - "version_map_memory_in_bytes" : { - "type" : "long" - } - } - }, - "store" : { - "properties" : { - "size_in_bytes" : { - "type" : "long" - }, - "throttle_time_in_millis" : { - "type" : "long" - } - } - } - } - } - } - }, - "source_node" : { - "properties" : { - "attributes" : { - "dynamic" : "true", - "properties" : { - "client" : { - "type" : "boolean" - }, - "data" : { - "type" : "boolean" - }, - "master" : { - "type" : "boolean" - } - } - }, - "host" : { - "type" : "keyword" - }, - "ip" : { - "type" : "keyword" - }, - "name" : { - "type" : "keyword" - }, - "transport_address" : { - "type" : "keyword" - }, - "uuid" : { - "type" : "keyword" - } - } - }, - "timestamp" : { - "type" : "date", - "format" : "date_time" - } - } - } - } - } -} From 7276737a0389c45c788874f3ee1f70590a4aea16 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Wed, 2 Nov 2016 14:58:15 +0100 Subject: [PATCH 098/132] Publishing a cluster state should clear the pending states queue (#21259) The pending cluster state queue is used to hold incoming cluster states from the master. Currently the elected master doesn't publish to itself as thus the queue is not used. Sometimes, depending on the timing of disruptions, a pending cluster state can be put on the queue (but not committed) but another master before being isolated. If this happens concurrently with a master election the elected master can have a pending cluster state in its queue. This is not really a problem but it does confuse our assertions during tests as we check to see everything was processed correctly. This commit takes a temporary step to clear (and fail) any pending cluster state on the master after it has successfully published a CS. Most notably this will happen when the master publishes the cluster state indicating it has just become the master. Long term we are working to change the publishing mechanism to make the master use the pending queue just like other nodes, which will make this a non issue. See https://elasticsearch-ci.elastic.co/job/elastic+elasticsearch+5.x+java9-periodic/509 for example. --- .../discovery/zen/ZenDiscovery.java | 9 +++ .../discovery/zen/ZenDiscoveryUnitTests.java | 56 ++++++++++++++++++- 2 files changed, 64 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java b/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java index 46f3a6fb315..a50ebab4c00 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java @@ -322,6 +322,11 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover // update the set of nodes to ping after the new cluster state has been published nodesFD.updateNodesAndPing(clusterChangedEvent.state()); + + // clean the pending cluster queue - we are currently master, so any pending cluster state should be failed + // note that we also clean the queue on master failure (see handleMasterGone) but a delayed cluster state publish + // from a stale master can still make it in the queue during the election (but not be committed) + publishClusterState.pendingStatesQueue().failAllStatesAndClear(new ElasticsearchException("elected as master")); } /** @@ -362,6 +367,10 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover return publishClusterState.pendingStatesQueue().pendingClusterStates(); } + PendingClusterStatesQueue pendingClusterStatesQueue() { + return publishClusterState.pendingStatesQueue(); + } + /** * the main function of a join thread. This function is guaranteed to join the cluster * or spawn a new join thread upon failure to do so. diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java index 8c13b5783d9..8ef4751c9cf 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java @@ -55,7 +55,9 @@ import static org.elasticsearch.discovery.zen.ElectMasterService.DISCOVERY_ZEN_M import static org.elasticsearch.discovery.zen.ZenDiscovery.shouldIgnoreOrRejectNewClusterState; import static org.elasticsearch.test.ClusterServiceUtils.createClusterService; import static org.elasticsearch.test.ClusterServiceUtils.setState; +import static org.hamcrest.Matchers.arrayWithSize; import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.emptyArray; import static org.hamcrest.Matchers.equalTo; public class ZenDiscoveryUnitTests extends ESTestCase { @@ -182,7 +184,6 @@ public class ZenDiscoveryUnitTests extends ESTestCase { toClose.add(otherZen); otherTransport.acceptIncomingRequests(); - masterTransport.connectToNode(otherNode); otherTransport.connectToNode(masterNode); @@ -213,6 +214,59 @@ public class ZenDiscoveryUnitTests extends ESTestCase { } } + public void testPendingCSQueueIsClearedWhenClusterStatePublished() throws Exception { + ThreadPool threadPool = new TestThreadPool(getClass().getName()); + // randomly make minimum_master_nodes a value higher than we have nodes for, so it will force failure + int minMasterNodes = randomBoolean() ? 3 : 1; + Settings settings = Settings.builder() + .put(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), Integer.toString(minMasterNodes)).build(); + + ArrayList toClose = new ArrayList<>(); + try { + final MockTransportService masterTransport = MockTransportService.createNewService(settings, Version.CURRENT, threadPool, null); + masterTransport.start(); + DiscoveryNode masterNode = new DiscoveryNode("master", masterTransport.boundAddress().publishAddress(), Version.CURRENT); + toClose.add(masterTransport); + masterTransport.setLocalNode(masterNode); + ClusterState state = ClusterStateCreationUtils.state(masterNode, null, masterNode); + // build the zen discovery and cluster service + ClusterService masterClusterService = createClusterService(threadPool, masterNode); + toClose.add(masterClusterService); + state = ClusterState.builder(masterClusterService.getClusterName()).nodes(state.nodes()).build(); + setState(masterClusterService, state); + ZenDiscovery masterZen = buildZenDiscovery(settings, masterTransport, masterClusterService, threadPool); + toClose.add(masterZen); + masterTransport.acceptIncomingRequests(); + + // inject a pending cluster state + masterZen.pendingClusterStatesQueue().addPending(ClusterState.builder(new ClusterName("foreign")).build()); + + // a new cluster state with a new discovery node (we will test if the cluster state + // was updated by the presence of this node in NodesFaultDetection) + ClusterState newState = ClusterState.builder(masterClusterService.state()).incrementVersion().nodes( + DiscoveryNodes.builder(masterClusterService.state().nodes()).masterNodeId(masterNode.getId()) + ).build(); + + + try { + // publishing a new cluster state + ClusterChangedEvent clusterChangedEvent = new ClusterChangedEvent("testing", newState, state); + AssertingAckListener listener = new AssertingAckListener(newState.nodes().getSize() - 1); + masterZen.publish(clusterChangedEvent, listener); + listener.await(1, TimeUnit.HOURS); + // publish was a success, check that queue as cleared + assertThat(masterZen.pendingClusterStates(), emptyArray()); + } catch (Discovery.FailedToCommitClusterStateException e) { + // not successful, so the pending queue should stay + assertThat(masterZen.pendingClusterStates(), arrayWithSize(1)); + assertThat(masterZen.pendingClusterStates()[0].getClusterName().value(), equalTo("foreign")); + } + } finally { + IOUtils.close(toClose); + terminate(threadPool); + } + } + private ZenDiscovery buildZenDiscovery(Settings settings, TransportService service, ClusterService clusterService, ThreadPool threadPool) { ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); ZenPingService zenPingService = new ZenPingService(settings, Collections.emptySet()); From 6930a4846cd49016232041ec0308780d9cdd61dc Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 2 Nov 2016 15:00:16 +0100 Subject: [PATCH 099/132] [TEST] Check static test state after suite scoped cluster is shut down (#21256) Checks on static test state are run by an @After method in ESTestCase. Suite-scoped tests in ESIntegTestCase only shut down in an @AfterClass method, which executes after the @After method in ESTestCase. The suite-scoped cluster can thus still execute actions that will violate the checks in @After without those being caught. A subsequent test executing within the same JVM will fail these checks however when @After gets called for that test. This commit adds an explicit call to check the static test state after the suite-scoped cluster has been shut down. --- .../main/java/org/elasticsearch/test/ESIntegTestCase.java | 1 + .../src/main/java/org/elasticsearch/test/ESTestCase.java | 5 +++++ 2 files changed, 6 insertions(+) diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index 77c65197366..3dcbab03352 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -1970,6 +1970,7 @@ public abstract class ESIntegTestCase extends ESTestCase { try { INSTANCE.printTestMessage("cleaning up after"); INSTANCE.afterInternal(true); + checkStaticState(); } finally { INSTANCE = null; } diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java index f75774a7364..38eeb69d179 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java @@ -237,6 +237,11 @@ public abstract class ESTestCase extends LuceneTestCase { @After public final void ensureCleanedUp() throws Exception { + checkStaticState(); + } + + // separate method so that this can be checked again after suite scoped cluster is shut down + protected static void checkStaticState() throws Exception { MockPageCacheRecycler.ensureAllPagesAreReleased(); MockBigArrays.ensureAllArraysAreReleased(); // field cache should NEVER get loaded. From c2062494a9bd91854c61b0a29bb2fbf4e5117465 Mon Sep 17 00:00:00 2001 From: Shane Connelly Date: Wed, 2 Nov 2016 07:01:21 -0700 Subject: [PATCH 100/132] Fix commit message that got into ingegrations doc --- docs/plugins/integrations.asciidoc | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/plugins/integrations.asciidoc b/docs/plugins/integrations.asciidoc index d39d02efeac..503f1274d81 100644 --- a/docs/plugins/integrations.asciidoc +++ b/docs/plugins/integrations.asciidoc @@ -201,4 +201,3 @@ These projects appear to have been abandoned: D3. * https://github.com/OlegKunitsyn/eslogd[eslogd]: Linux daemon that replicates events to a central Elasticsearch server in realtime ->>>>>>> 02602a3... Update integrations.asciidoc (#18915) From a0c094d0c1d41fd573e8441e5ed8066bd60a7745 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Tue, 18 Oct 2016 15:50:33 +0200 Subject: [PATCH 101/132] Add deprecation logging message for 'fuzzy' query This query is deprecated from 5.0 on. Similar to IndicesQueryBuilder we should log a deprecation warning whenever this query is used. Relates to #15760 --- .../org/elasticsearch/index/query/FuzzyQueryBuilder.java | 5 +++++ docs/reference/query-dsl/fuzzy-query.asciidoc | 2 ++ 2 files changed, 7 insertions(+) diff --git a/core/src/main/java/org/elasticsearch/index/query/FuzzyQueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/FuzzyQueryBuilder.java index c75ba1fda99..ef94ff16cf8 100644 --- a/core/src/main/java/org/elasticsearch/index/query/FuzzyQueryBuilder.java +++ b/core/src/main/java/org/elasticsearch/index/query/FuzzyQueryBuilder.java @@ -28,6 +28,8 @@ import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.logging.DeprecationLogger; +import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.lucene.BytesRefs; import org.elasticsearch.common.unit.Fuzziness; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -49,6 +51,8 @@ import java.util.Optional; public class FuzzyQueryBuilder extends AbstractQueryBuilder implements MultiTermQueryBuilder { public static final String NAME = "fuzzy"; + private static final DeprecationLogger DEPRECATION_LOGGER = new DeprecationLogger(Loggers.getLogger(FuzzyQueryBuilder.class)); + /** Default maximum edit distance. Defaults to AUTO. */ public static final Fuzziness DEFAULT_FUZZINESS = Fuzziness.AUTO; @@ -151,6 +155,7 @@ public class FuzzyQueryBuilder extends AbstractQueryBuilder i * @param value The value of the term */ public FuzzyQueryBuilder(String fieldName, Object value) { + DEPRECATION_LOGGER.deprecated("{} query is deprecated. Instead use the [match] query with fuzziness parameter", NAME); if (Strings.isEmpty(fieldName)) { throw new IllegalArgumentException("field name cannot be null or empty"); } diff --git a/docs/reference/query-dsl/fuzzy-query.asciidoc b/docs/reference/query-dsl/fuzzy-query.asciidoc index f320e81b579..4df30dec2f1 100644 --- a/docs/reference/query-dsl/fuzzy-query.asciidoc +++ b/docs/reference/query-dsl/fuzzy-query.asciidoc @@ -24,6 +24,7 @@ GET /_search } -------------------------------------------------- // CONSOLE +// TEST[warning:fuzzy query is deprecated. Instead use the [match] query with fuzziness parameter] Or with more advanced settings: @@ -45,6 +46,7 @@ GET /_search } -------------------------------------------------- // CONSOLE +// TEST[warning:fuzzy query is deprecated. Instead use the [match] query with fuzziness parameter] [float] ===== Parameters From b3370de71546db61c5bd0bcf86bef9e897343d54 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Wed, 2 Nov 2016 12:32:40 +0100 Subject: [PATCH 102/132] Tests: Add warning header checks to QueryBuilder tests and QueryParseContextTests This adds checks for expected warning headers to the query builder test infrastructure. Tests that are adding deprecation warnings to the response headers need to check those, otherwise the abstract base class for the test class will complain at teardown. --- .../elasticsearch/transport/TcpTransport.java | 5 ++- .../index/query/BoolQueryBuilderTests.java | 5 +++ .../query/BoostingQueryBuilderTests.java | 21 +++++---- .../query/ConstantScoreQueryBuilderTests.java | 6 ++- .../index/query/DisMaxQueryBuilderTests.java | 1 + .../index/query/FuzzyQueryBuilderTests.java | 45 +++++++++++++++++-- .../query/HasChildQueryBuilderTests.java | 5 ++- .../query/HasParentQueryBuilderTests.java | 7 ++- .../index/query/IdsQueryBuilderTests.java | 2 + .../index/query/IndicesQueryBuilderTests.java | 10 +++++ .../index/query/MatchQueryBuilderTests.java | 8 ++++ .../query/MoreLikeThisQueryBuilderTests.java | 2 + .../index/query/QueryParseContextTests.java | 32 ++++++++++++- .../index/query/RandomQueryBuilder.java | 8 ++-- .../FunctionScoreQueryBuilderTests.java | 11 ++--- .../mustache/TemplateQueryBuilderTests.java | 9 ++++ .../test/AbstractQueryTestCase.java | 41 ++++++++++++++++- 17 files changed, 186 insertions(+), 32 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/transport/TcpTransport.java b/core/src/main/java/org/elasticsearch/transport/TcpTransport.java index ab38b120c86..9e4e0262080 100644 --- a/core/src/main/java/org/elasticsearch/transport/TcpTransport.java +++ b/core/src/main/java/org/elasticsearch/transport/TcpTransport.java @@ -20,6 +20,7 @@ package org.elasticsearch.transport; import com.carrotsearch.hppc.IntHashSet; import com.carrotsearch.hppc.IntSet; + import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.logging.log4j.util.Supplier; import org.apache.lucene.util.IOUtils; @@ -53,8 +54,8 @@ import org.elasticsearch.common.network.NetworkUtils; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.BoundTransportAddress; -import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.transport.PortsRange; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.BigArrays; @@ -90,7 +91,6 @@ import java.util.Objects; import java.util.Set; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; @@ -366,6 +366,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent i return Arrays.asList(recovery, bulk, reg, state, ping); } + @Override public synchronized void close() throws IOException { closeChannels(allChannels); } diff --git a/core/src/test/java/org/elasticsearch/index/query/BoolQueryBuilderTests.java b/core/src/test/java/org/elasticsearch/index/query/BoolQueryBuilderTests.java index 720f527acd1..c5c77e027ac 100644 --- a/core/src/test/java/org/elasticsearch/index/query/BoolQueryBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/BoolQueryBuilderTests.java @@ -348,9 +348,14 @@ public class BoolQueryBuilderTests extends AbstractQueryTestCase parseQuery(query, ParseFieldMatcher.STRICT)); assertThat(ex.getMessage(), startsWith("query malformed, empty clause found at")); + checkWarningHeaders("query malformed, empty clause found at [1:27]"); } /** diff --git a/core/src/test/java/org/elasticsearch/index/query/BoostingQueryBuilderTests.java b/core/src/test/java/org/elasticsearch/index/query/BoostingQueryBuilderTests.java index 96e6959600b..a4658fbbf23 100644 --- a/core/src/test/java/org/elasticsearch/index/query/BoostingQueryBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/BoostingQueryBuilderTests.java @@ -30,9 +30,9 @@ import org.elasticsearch.test.AbstractQueryTestCase; import java.io.IOException; import java.util.Optional; +import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.instanceOf; -import static org.hamcrest.CoreMatchers.nullValue; -import static org.hamcrest.CoreMatchers.startsWith;; +import static org.hamcrest.CoreMatchers.nullValue;; public class BoostingQueryBuilderTests extends AbstractQueryTestCase { @@ -110,22 +110,27 @@ public class BoostingQueryBuilderTests extends AbstractQueryTestCase innerQueryBuilder = context.parseInnerQueryBuilder(); assertTrue(innerQueryBuilder.isPresent() == false); + checkWarningHeaders("query malformed, empty clause found at [1:36]"); + query = - "{ \"boosting\" : {" + - " \"positive\" : { \"match_all\" : {} }, " + - " \"negative\" : { }, " + - " \"negative_boost\" : 23.0" + - " }" + + "{ \"boosting\" : {\n" + + " \"positive\" : { \"match_all\" : {} },\n" + + " \"negative\" : { },\n" + + " \"negative_boost\" : 23.0\n" + + " }\n" + "}"; parser = XContentFactory.xContent(query).createParser(query); context = createParseContext(parser, ParseFieldMatcher.EMPTY); innerQueryBuilder = context.parseInnerQueryBuilder(); assertTrue(innerQueryBuilder.isPresent() == false); + checkWarningHeaders("query malformed, empty clause found at [3:20]"); + parser = XContentFactory.xContent(query).createParser(query); QueryParseContext otherContext = createParseContext(parser, ParseFieldMatcher.STRICT); IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, () -> otherContext.parseInnerQueryBuilder()); - assertThat(ex.getMessage(), startsWith("query malformed, empty clause found at")); + assertThat(ex.getMessage(), equalTo("query malformed, empty clause found at [3:20]")); + checkWarningHeaders("query malformed, empty clause found at [3:20]"); } public void testRewrite() throws IOException { diff --git a/core/src/test/java/org/elasticsearch/index/query/ConstantScoreQueryBuilderTests.java b/core/src/test/java/org/elasticsearch/index/query/ConstantScoreQueryBuilderTests.java index d1ac2c96192..7ab852bc3f6 100644 --- a/core/src/test/java/org/elasticsearch/index/query/ConstantScoreQueryBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/ConstantScoreQueryBuilderTests.java @@ -31,9 +31,9 @@ import org.elasticsearch.test.AbstractQueryTestCase; import java.io.IOException; import java.util.Optional; +import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.CoreMatchers.nullValue; -import static org.hamcrest.CoreMatchers.startsWith; import static org.hamcrest.Matchers.containsString; public class ConstantScoreQueryBuilderTests extends AbstractQueryTestCase { @@ -133,11 +133,13 @@ public class ConstantScoreQueryBuilderTests extends AbstractQueryTestCase innerQueryBuilder = context.parseInnerQueryBuilder(); assertTrue(innerQueryBuilder.isPresent() == false); + checkWarningHeaders("query malformed, empty clause found at [1:40]"); parser = XContentFactory.xContent(query).createParser(query); QueryParseContext otherContext = createParseContext(parser, ParseFieldMatcher.STRICT); IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, () -> otherContext.parseInnerQueryBuilder()); - assertThat(ex.getMessage(), startsWith("query malformed, empty clause found at")); + assertThat(ex.getMessage(), equalTo("query malformed, empty clause found at [1:40]")); + checkWarningHeaders("query malformed, empty clause found at [1:40]"); } } diff --git a/core/src/test/java/org/elasticsearch/index/query/DisMaxQueryBuilderTests.java b/core/src/test/java/org/elasticsearch/index/query/DisMaxQueryBuilderTests.java index ec542d7ccb1..114109736ea 100644 --- a/core/src/test/java/org/elasticsearch/index/query/DisMaxQueryBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/DisMaxQueryBuilderTests.java @@ -99,6 +99,7 @@ public class DisMaxQueryBuilderTests extends AbstractQueryTestCase { + private boolean testSkipped = false; + + /** + * All tests create deprecation warnings when an new FuzzyQueryBuilder is created. Instead of having to check them once + * in every single test, this is done here after each test is run + */ + @After + void checkWarningHeaders() throws IOException { + // only check that warning headers got created for tests that satisfied certain assumptions and were thus not skipped + if (testSkipped == false) { + checkWarningHeaders("fuzzy query is deprecated. Instead use the [match] query with fuzziness parameter"); + } + } + @Override protected FuzzyQueryBuilder doCreateTestQueryBuilder() { FuzzyQueryBuilder query = new FuzzyQueryBuilder(STRING_FIELD_NAME, getRandomValueForFieldName(STRING_FIELD_NAME)); @@ -98,7 +114,13 @@ public class FuzzyQueryBuilderTests extends AbstractQueryTestCase 0); + try { + assumeTrue("test runs only when at least a type is registered", getCurrentTypes().length > 0); + } catch (AssumptionViolatedException e) { + // we need to know that this test was skipped in @After checkWarningHeaders(), because no warnings will be generated + testSkipped = true; + throw e; + } String query = "{\n" + " \"fuzzy\":{\n" + " \"" + STRING_FIELD_NAME + "\":{\n" + @@ -121,7 +143,13 @@ public class FuzzyQueryBuilderTests extends AbstractQueryTestCase 0); + try { + assumeTrue("test runs only when at least a type is registered", getCurrentTypes().length > 0); + } catch (AssumptionViolatedException e) { + // we need to know that this test was skipped in @After checkWarningHeaders(), because no warnings will be generated + testSkipped = true; + throw e; + } String query = "{\n" + " \"fuzzy\":{\n" + " \"" + INT_FIELD_NAME + "\":{\n" + @@ -157,7 +185,16 @@ public class FuzzyQueryBuilderTests extends AbstractQueryTestCase parseQuery(json)); + ParsingException e = expectThrows(ParsingException.class, () -> parseQuery(json2)); assertEquals("[fuzzy] query doesn't support multiple fields, found [message1] and [message2]", e.getMessage()); String shortJson = "{\n" + diff --git a/core/src/test/java/org/elasticsearch/index/query/HasChildQueryBuilderTests.java b/core/src/test/java/org/elasticsearch/index/query/HasChildQueryBuilderTests.java index e491f64457d..aa0b82873fa 100644 --- a/core/src/test/java/org/elasticsearch/index/query/HasChildQueryBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/HasChildQueryBuilderTests.java @@ -20,6 +20,7 @@ package org.elasticsearch.index.query; import com.carrotsearch.randomizedtesting.generators.RandomPicks; + import org.apache.lucene.queries.TermsQuery; import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.BooleanQuery; @@ -58,7 +59,6 @@ import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.CoreMatchers.notNullValue; -import static org.hamcrest.CoreMatchers.startsWith; public class HasChildQueryBuilderTests extends AbstractQueryTestCase { protected static final String PARENT_TYPE = "parent"; @@ -241,7 +241,8 @@ public class HasChildQueryBuilderTests extends AbstractQueryTestCase otherContext.parseInnerQueryBuilder()); - assertThat(ex.getMessage(), startsWith("query malformed, empty clause found at")); + assertThat(ex.getMessage(), equalTo("query malformed, empty clause found at [3:17]")); + checkWarningHeaders("query malformed, empty clause found at [3:17]"); } public void testToQueryInnerQueryType() throws IOException { diff --git a/core/src/test/java/org/elasticsearch/index/query/HasParentQueryBuilderTests.java b/core/src/test/java/org/elasticsearch/index/query/HasParentQueryBuilderTests.java index cdc2be67d96..eb740835054 100644 --- a/core/src/test/java/org/elasticsearch/index/query/HasParentQueryBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/HasParentQueryBuilderTests.java @@ -45,7 +45,6 @@ import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.CoreMatchers.notNullValue; -import static org.hamcrest.CoreMatchers.startsWith; public class HasParentQueryBuilderTests extends AbstractQueryTestCase { protected static final String PARENT_TYPE = "parent"; @@ -157,6 +156,7 @@ public class HasParentQueryBuilderTests extends AbstractQueryTestCase innerQueryBuilder = context.parseInnerQueryBuilder(); assertTrue(innerQueryBuilder.isPresent() == false); + checkWarningHeaders("query malformed, empty clause found at [3:17]"); + parser = XContentFactory.xContent(query).createParser(query); QueryParseContext otherContext = createParseContext(parser, ParseFieldMatcher.STRICT); IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, () -> otherContext.parseInnerQueryBuilder()); - assertThat(ex.getMessage(), startsWith("query malformed, empty clause found at")); + assertThat(ex.getMessage(), equalTo("query malformed, empty clause found at [3:17]")); + checkWarningHeaders("query malformed, empty clause found at [3:17]"); } public void testIgnoreUnmapped() throws IOException { diff --git a/core/src/test/java/org/elasticsearch/index/query/IdsQueryBuilderTests.java b/core/src/test/java/org/elasticsearch/index/query/IdsQueryBuilderTests.java index f21df5bf759..2ad557dcc76 100644 --- a/core/src/test/java/org/elasticsearch/index/query/IdsQueryBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/IdsQueryBuilderTests.java @@ -140,6 +140,7 @@ public class IdsQueryBuilderTests extends AbstractQueryTestCase IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> parseQuery(contentString)); assertEquals("Deprecated field [_type] used, expected [type] instead", e.getMessage()); + checkWarningHeaders("Deprecated field [_type] used, expected [type] instead"); //array of types can also be called type rather than types final String contentString2 = "{\n" + @@ -153,5 +154,6 @@ public class IdsQueryBuilderTests extends AbstractQueryTestCase e = expectThrows(IllegalArgumentException.class, () -> parseQuery(contentString2)); assertEquals("Deprecated field [types] used, expected [type] instead", e.getMessage()); + checkWarningHeaders("Deprecated field [_type] used, expected [type] instead"); } } diff --git a/core/src/test/java/org/elasticsearch/index/query/IndicesQueryBuilderTests.java b/core/src/test/java/org/elasticsearch/index/query/IndicesQueryBuilderTests.java index 00bd7c4004f..3b31d17d9ba 100644 --- a/core/src/test/java/org/elasticsearch/index/query/IndicesQueryBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/IndicesQueryBuilderTests.java @@ -22,11 +22,21 @@ package org.elasticsearch.index.query; import org.apache.lucene.search.Query; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.test.AbstractQueryTestCase; +import org.junit.After; import java.io.IOException; public class IndicesQueryBuilderTests extends AbstractQueryTestCase { + /** + * All tests create deprecation warnings when an new {@link IndicesQueryBuilder} is created. + * Instead of having to check them once in every single test, this is done here after each test is run + */ + @After + void checkWarningHeaders() throws IOException { + checkWarningHeaders("indices query is deprecated. Instead search on the '_index' field"); + } + @Override protected IndicesQueryBuilder doCreateTestQueryBuilder() { String[] indices; diff --git a/core/src/test/java/org/elasticsearch/index/query/MatchQueryBuilderTests.java b/core/src/test/java/org/elasticsearch/index/query/MatchQueryBuilderTests.java index 571238e43a1..a4e202e2304 100644 --- a/core/src/test/java/org/elasticsearch/index/query/MatchQueryBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/MatchQueryBuilderTests.java @@ -313,6 +313,9 @@ public class MatchQueryBuilderTests extends AbstractQueryTestCase parseQuery(json, ParseFieldMatcher.STRICT)); assertThat(e.getMessage(), @@ -346,6 +349,9 @@ public class MatchQueryBuilderTests extends AbstractQueryTestCase parseQuery(json, ParseFieldMatcher.STRICT)); assertThat(e.getMessage(), @@ -372,6 +378,8 @@ public class MatchQueryBuilderTests extends AbstractQueryTestCase parseQuery(json, ParseFieldMatcher.STRICT)); diff --git a/core/src/test/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilderTests.java b/core/src/test/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilderTests.java index aa9462bb3e6..7d35df1fb02 100644 --- a/core/src/test/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilderTests.java @@ -367,5 +367,7 @@ public class MoreLikeThisQueryBuilderTests extends AbstractQueryTestCase parseQuery(deprecatedJson)); assertEquals("Deprecated field [mlt] used, expected [more_like_this] instead", e.getMessage()); + + checkWarningHeaders("Deprecated field [mlt] used, expected [more_like_this] instead"); } } diff --git a/core/src/test/java/org/elasticsearch/index/query/QueryParseContextTests.java b/core/src/test/java/org/elasticsearch/index/query/QueryParseContextTests.java index 5568d2fa5a7..aa8541ab956 100644 --- a/core/src/test/java/org/elasticsearch/index/query/QueryParseContextTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/QueryParseContextTests.java @@ -21,20 +21,27 @@ package org.elasticsearch.index.query; import org.elasticsearch.common.ParseFieldMatcher; import org.elasticsearch.common.ParsingException; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.indices.query.IndicesQueriesRegistry; import org.elasticsearch.search.SearchModule; import org.elasticsearch.test.ESTestCase; +import org.junit.After; +import org.junit.Before; import org.junit.BeforeClass; import java.io.IOException; +import java.util.List; import java.util.Optional; import static java.util.Collections.emptyList; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasItem; +import static org.hamcrest.Matchers.hasSize; public class QueryParseContextTests extends ESTestCase { @@ -45,6 +52,20 @@ public class QueryParseContextTests extends ESTestCase { indicesQueriesRegistry = new SearchModule(Settings.EMPTY, false, emptyList()).getQueryParserRegistry(); } + private ThreadContext threadContext; + + @Before + public void beforeTest() throws IOException { + this.threadContext = new ThreadContext(Settings.EMPTY); + DeprecationLogger.setThreadContext(threadContext); + } + + @After + public void teardown() throws IOException { + DeprecationLogger.removeThreadContext(this.threadContext); + this.threadContext.close(); + } + public void testParseTopLevelBuilder() throws IOException { QueryBuilder query = new MatchQueryBuilder("foo", "bar"); String requestBody = "{ \"query\" : " + query.toString() + "}"; @@ -89,6 +110,9 @@ public class QueryParseContextTests extends ESTestCase { QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.EMPTY); Optional emptyQuery = context.parseInnerQueryBuilder(); assertFalse(emptyQuery.isPresent()); + final List warnings = threadContext.getResponseHeaders().get(DeprecationLogger.DEPRECATION_HEADER); + assertThat(warnings, hasSize(1)); + assertThat(warnings, hasItem(equalTo("query malformed, empty clause found at [1:2]"))); } } @@ -107,6 +131,9 @@ public class QueryParseContextTests extends ESTestCase { QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.STRICT); IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> context.parseInnerQueryBuilder()); assertEquals("query malformed, empty clause found at [1:2]", exception.getMessage()); + final List warnings = threadContext.getResponseHeaders().get(DeprecationLogger.DEPRECATION_HEADER); + assertThat(warnings, hasSize(1)); + assertThat(warnings, hasItem(equalTo("query malformed, empty clause found at [1:2]"))); } source = "{ \"foo\" : \"bar\" }"; @@ -122,6 +149,9 @@ public class QueryParseContextTests extends ESTestCase { ParsingException exception = expectThrows(ParsingException.class, () -> context.parseInnerQueryBuilder()); assertEquals("no [query] registered for [foo]", exception.getMessage()); } + final List warnings = threadContext.getResponseHeaders().get(DeprecationLogger.DEPRECATION_HEADER); + assertThat(warnings, hasSize(1)); + assertThat(warnings, hasItem(equalTo("query malformed, empty clause found at [1:2]"))); } } diff --git a/core/src/test/java/org/elasticsearch/index/query/RandomQueryBuilder.java b/core/src/test/java/org/elasticsearch/index/query/RandomQueryBuilder.java index d16b225ab69..68510d4e893 100644 --- a/core/src/test/java/org/elasticsearch/index/query/RandomQueryBuilder.java +++ b/core/src/test/java/org/elasticsearch/index/query/RandomQueryBuilder.java @@ -21,6 +21,7 @@ package org.elasticsearch.index.query; import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import com.carrotsearch.randomizedtesting.generators.RandomStrings; + import org.elasticsearch.test.AbstractQueryTestCase; import java.util.Random; @@ -61,7 +62,7 @@ public class RandomQueryBuilder { // for now, only use String Rangequeries for MultiTerm test, numeric and date makes little sense // see issue #12123 for discussion MultiTermQueryBuilder multiTermQueryBuilder; - switch(RandomNumbers.randomIntBetween(r, 0, 3)) { + switch(RandomNumbers.randomIntBetween(r, 0, 2)) { case 0: RangeQueryBuilder stringRangeQuery = new RangeQueryBuilder(AbstractQueryTestCase.STRING_FIELD_NAME); stringRangeQuery.from("a" + RandomStrings.randomAsciiOfLengthBetween(r, 1, 10)); @@ -69,12 +70,9 @@ public class RandomQueryBuilder { multiTermQueryBuilder = stringRangeQuery; break; case 1: - multiTermQueryBuilder = new FuzzyQueryBuilder(AbstractQueryTestCase.STRING_FIELD_NAME, RandomStrings.randomAsciiOfLengthBetween(r, 1, 10)); - break; - case 2: multiTermQueryBuilder = new PrefixQueryBuilderTests().createTestQueryBuilder(); break; - case 3: + case 2: multiTermQueryBuilder = new WildcardQueryBuilderTests().createTestQueryBuilder(); break; default: diff --git a/core/src/test/java/org/elasticsearch/index/query/functionscore/FunctionScoreQueryBuilderTests.java b/core/src/test/java/org/elasticsearch/index/query/functionscore/FunctionScoreQueryBuilderTests.java index 88a857cccb4..50a7382e404 100644 --- a/core/src/test/java/org/elasticsearch/index/query/functionscore/FunctionScoreQueryBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/functionscore/FunctionScoreQueryBuilderTests.java @@ -20,6 +20,7 @@ package org.elasticsearch.index.query.functionscore; import com.fasterxml.jackson.core.JsonParseException; + import org.apache.lucene.index.Term; import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.Query; @@ -589,13 +590,13 @@ public class FunctionScoreQueryBuilderTests extends AbstractQueryTestCase> getPlugins() { return Arrays.asList(MustachePlugin.class, CustomScriptPlugin.class); diff --git a/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java index bc02f20197e..e65e0ab4de7 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java @@ -52,11 +52,13 @@ import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.Writeable.Reader; +import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.settings.IndexScopedSettings; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.SettingsModule; import org.elasticsearch.common.unit.Fuzziness; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; @@ -123,10 +125,12 @@ import java.util.concurrent.ExecutionException; import static java.util.Collections.emptyList; import static org.elasticsearch.test.EqualsHashCodeTestUtils.checkEqualsAndHashCode; +import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.either; -import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.hasItem; +import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.instanceOf; public abstract class AbstractQueryTestCase> extends ESTestCase { @@ -155,6 +159,10 @@ public abstract class AbstractQueryTestCase> private static String[] currentTypes; private static String[] randomTypes; + /** + * used to check warning headers of the deprecation logger + */ + private ThreadContext threadContext; protected static Index getIndex() { return index; @@ -209,6 +217,20 @@ public abstract class AbstractQueryTestCase> serviceHolder = new ServiceHolder(nodeSettings, indexSettings, getPlugins(), this); } serviceHolder.clientInvocationHandler.delegate = this; + this.threadContext = new ThreadContext(Settings.EMPTY); + DeprecationLogger.setThreadContext(threadContext); + } + + /** + * Check that there are no unaccounted warning headers. These should be checked with {@link #checkWarningHeaders(String...)} in the + * appropriate test + */ + @After + public void teardown() throws IOException { + final List warnings = threadContext.getResponseHeaders().get(DeprecationLogger.DEPRECATION_HEADER); + assertNull("unexpected warning headers", warnings); + DeprecationLogger.removeThreadContext(this.threadContext); + this.threadContext.close(); } private static SearchContext getSearchContext(String[] types, QueryShardContext context) { @@ -1009,6 +1031,23 @@ public abstract class AbstractQueryTestCase> return query; } + protected void checkWarningHeaders(String... messages) { + final List warnings = threadContext.getResponseHeaders().get(DeprecationLogger.DEPRECATION_HEADER); + assertThat(warnings, hasSize(messages.length)); + for (String msg : messages) { + assertThat(warnings, hasItem(equalTo(msg))); + } + // "clear" current warning headers by setting a new ThreadContext + DeprecationLogger.removeThreadContext(this.threadContext); + try { + this.threadContext.close(); + } catch (IOException e) { + throw new RuntimeException(e); + } + this.threadContext = new ThreadContext(Settings.EMPTY); + DeprecationLogger.setThreadContext(this.threadContext); + } + private static class ServiceHolder implements Closeable { private final IndicesQueriesRegistry indicesQueriesRegistry; From c4c4a3a504b56a272b4bb69541128f79cba4b715 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 2 Nov 2016 17:09:30 +0100 Subject: [PATCH 103/132] Fix `_shrink` test to work under a mixed version cluster (#21262) today the `_shrink` tests do relocate all shards to a single node in the cluster. Yet, that is not always possible since the only node we can safely identify in the cluster is the master and if the master is a BWC node in such a cluster we won't be able to relocate shards that have a primary on the newer version nodes since allocation deciders forbid this. This change restricts allocation for that index when the index is created to restrict allocation to the master that guarantees that all primaries are on the same node which is sufficient for the `_shrink` API to run. --- .../rest-api-spec/api/cluster.health.json | 2 +- .../test/indices.shrink/10_basic.yaml | 36 +++++++++---------- 2 files changed, 17 insertions(+), 21 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.health.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.health.json index f1a6a98217c..06828a6588a 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.health.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.health.json @@ -41,7 +41,7 @@ "wait_for_events": { "type" : "enum", "options" : ["immediate", "urgent", "high", "normal", "low", "languid"], - "description" : "Wait until all currently queued events with the given priorty are processed" + "description" : "Wait until all currently queued events with the given priority are processed" }, "wait_for_no_relocating_shards": { "type" : "boolean", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.shrink/10_basic.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.shrink/10_basic.yaml index d793816ed5d..62a75b0ff04 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.shrink/10_basic.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.shrink/10_basic.yaml @@ -1,21 +1,26 @@ --- "Shrink index via API": - - skip: - version: " - 5.0.0" - reason: this doesn't work yet with BWC tests since the master is from the old verion - # TODO we need to fix this for BWC tests to make sure we get a node that all shards can allocate on - # today if we run BWC tests and we select the master as a _shrink node but since primaries are allocated - # on the newer version nodes this fails... - # creates an index with one document. - # relocates all it's shards to one node - # shrinks it into a new index with a single shard + # creates an index with one document solely allocated on the master node + # and shrinks it into a new index with a single shard + # we don't do the relocation to a single node after the index is created + # here since in a mixed version cluster we can't identify + # which node is the one with the highest version and that is the only one that can safely + # be used to shrink the index. + - do: + cluster.state: {} + # Get master node id + + - set: { master_node: master } + - do: indices.create: index: source wait_for_active_shards: 1 body: settings: - number_of_replicas: "0" + # ensure everything is allocated on a single node + index.routing.allocation.include._id: $master + number_of_replicas: 0 - do: index: index: source @@ -34,18 +39,11 @@ - match: { _id: "1" } - match: { _source: { foo: "hello world" } } - - do: - cluster.state: {} - - # Get master node id - - set: { master_node: master } - - # relocate everything to the master node and make it read-only + # make it read-only - do: indices.put_settings: index: source body: - index.routing.allocation.include._id: $master index.blocks.write: true index.number_of_replicas: 0 @@ -53,8 +51,6 @@ cluster.health: wait_for_status: green index: source - wait_for_no_relocating_shards: true - wait_for_events: "languid" # now we do the actual shrink - do: From 799a12ad63b35faa539d50e7d8720c05036399fd Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 2 Nov 2016 12:14:47 -0400 Subject: [PATCH 104/132] Fix distribution build ordering issue Today when running gradle clean :distribution:(integ-test-zip|tar|zip):assemble, the created archive distribution will be missing the empty plugins directory. This is because the empty plugins folder created in the build folder for the copy spec task is created during configuration and then is later wiped away by the clean task. This commit addresses this issue, by pushing creation of the directory out of the configuration phase. Relates #21271 --- distribution/build.gradle | 13 +++++++------ distribution/integ-test-zip/build.gradle | 1 + distribution/tar/build.gradle | 1 + distribution/zip/build.gradle | 1 + 4 files changed, 10 insertions(+), 6 deletions(-) diff --git a/distribution/build.gradle b/distribution/build.gradle index 21467ef727f..cc4912888da 100644 --- a/distribution/build.gradle +++ b/distribution/build.gradle @@ -196,6 +196,13 @@ subprojects { * Zip and tgz configuration * *****************************************************************************/ configure(subprojects.findAll { ['zip', 'tar', 'integ-test-zip'].contains(it.name) }) { + // CopySpec does not make it easy to create an empty director so we create the directory that we want, and then point CopySpec to its + // parent to copy to the root of the distribution + File plugins = new File(buildDir, 'plugins-hack/plugins') + task createPluginsDir(type: EmptyDirTask) { + dir "${plugins}" + dirMode 0755 + } project.ext.archivesFiles = copySpec { into("elasticsearch-${version}") { with libFiles @@ -215,12 +222,6 @@ configure(subprojects.findAll { ['zip', 'tar', 'integ-test-zip'].contains(it.nam } } into('') { - // CopySpec does not make it easy to create an empty directory - // so we create the directory that we want, and then point - // CopySpec to its parent to copy to the root of the - // distribution - File plugins = new File(buildDir, 'plugins-hack/plugins') - plugins.mkdirs() from { plugins.getParent() } diff --git a/distribution/integ-test-zip/build.gradle b/distribution/integ-test-zip/build.gradle index ae4a499efd8..80da4131995 100644 --- a/distribution/integ-test-zip/build.gradle +++ b/distribution/integ-test-zip/build.gradle @@ -20,6 +20,7 @@ import org.elasticsearch.gradle.plugin.PluginBuildPlugin task buildZip(type: Zip) { + dependsOn createPluginsDir baseName = 'elasticsearch' with archivesFiles } diff --git a/distribution/tar/build.gradle b/distribution/tar/build.gradle index 8e34a6ce9ce..2140061ee48 100644 --- a/distribution/tar/build.gradle +++ b/distribution/tar/build.gradle @@ -18,6 +18,7 @@ */ task buildTar(type: Tar) { + dependsOn createPluginsDir baseName = 'elasticsearch' extension = 'tar.gz' with archivesFiles diff --git a/distribution/zip/build.gradle b/distribution/zip/build.gradle index ae4a499efd8..80da4131995 100644 --- a/distribution/zip/build.gradle +++ b/distribution/zip/build.gradle @@ -20,6 +20,7 @@ import org.elasticsearch.gradle.plugin.PluginBuildPlugin task buildZip(type: Zip) { + dependsOn createPluginsDir baseName = 'elasticsearch' with archivesFiles } From 24d5f31a545599cf6a46cec7d86668c531313e50 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 2 Nov 2016 11:29:15 -0400 Subject: [PATCH 105/132] Make painless's assertion about out of bound less brittle Instead of asserting that the message is shaped a certain way we cause the exception and catch it and assert that the messages are the same. This is the way to go because the exception message from the jvm is both local and jvm dependent. This is the CI failure that found this: https://elasticsearch-ci.elastic.co/job/elastic+elasticsearch+5.x+java9-periodic/515/consoleFull --- .../org/elasticsearch/painless/ListTests.java | 21 ++++++++++++------- 1 file changed, 13 insertions(+), 8 deletions(-) diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/ListTests.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/ListTests.java index 79112d8f58e..1ae7ca0bc4f 100644 --- a/modules/lang-painless/src/test/java/org/elasticsearch/painless/ListTests.java +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/ListTests.java @@ -21,7 +21,9 @@ package org.elasticsearch.painless; import org.hamcrest.Matcher; +import java.util.ArrayList; import java.util.Arrays; +import java.util.List; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.equalTo; @@ -56,18 +58,21 @@ public class ListTests extends ArrayLikeObjectTestCase { @Override protected Matcher outOfBoundsExceptionMessageMatcher(int index, int size) { if ("1.8".equals(Runtime.class.getPackage().getSpecificationVersion())) { - // 1.8 and below aren't as clean as 1.9+ if (index > size) { return equalTo("Index: " + index + ", Size: " + size); - } else { - Matcher m = equalTo(Integer.toString(index)); - // If we set -XX:-OmitStackTraceInFastThrow we wouldn't need this - m = anyOf(m, nullValue()); - return m; } + Matcher matcher = equalTo(Integer.toString(index)); + // If we set -XX:-OmitStackTraceInFastThrow we wouldn't need this + matcher = anyOf(matcher, nullValue()); + return matcher; } else { - // Starting with 1.9 it gets nicer - return equalTo("Index " + index + " out-of-bounds for length " + size); + // This exception is locale dependent so we attempt to reproduce it + List list = new ArrayList<>(); + for (int i = 0; i < size; i++) { + list.add(new Object()); + } + Exception e = expectThrows(IndexOutOfBoundsException.class, () -> list.get(index)); + return equalTo(e.getMessage()); } } From 83338686b9befea9785aeae296a5281052531826 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 2 Nov 2016 13:07:26 -0400 Subject: [PATCH 106/132] Remove stale install plugins assertion When installing a plugin when the plugins directory does not exist, the install plugin command outputs a line saying that it is creating this directory. The packaging tests for the archive distributions accounted for this including an assertion that this line was output. The packages have since been updated to include an empty plugins folder, so this line will no longer be output. This commit removes this stale assertion from the packaging tests. Relates #21275 --- .../scripts/module_and_plugin_test_cases.bash | 37 +++++-------------- 1 file changed, 10 insertions(+), 27 deletions(-) diff --git a/qa/vagrant/src/test/resources/packaging/scripts/module_and_plugin_test_cases.bash b/qa/vagrant/src/test/resources/packaging/scripts/module_and_plugin_test_cases.bash index fbc0f4a887a..b979f40e309 100644 --- a/qa/vagrant/src/test/resources/packaging/scripts/module_and_plugin_test_cases.bash +++ b/qa/vagrant/src/test/resources/packaging/scripts/module_and_plugin_test_cases.bash @@ -436,38 +436,21 @@ fi sudo -E -u $ESPLUGIN_COMMAND_USER "$ESHOME/bin/elasticsearch-plugin" install "file://$relativePath" > /tmp/plugin-cli-output # exclude progress line local loglines=$(cat /tmp/plugin-cli-output | grep -v "^[[:cntrl:]]" | wc -l) - if [ "$GROUP" == "TAR PLUGINS" ]; then - # tar extraction does not create the plugins directory so the plugin tool will print an additional line that the directory will be created - [ "$loglines" -eq "3" ] || { - echo "Expected 3 lines excluding progress bar but the output had $loglines lines and was:" - cat /tmp/plugin-cli-output - false - } - else - [ "$loglines" -eq "2" ] || { - echo "Expected 2 lines excluding progress bar but the output had $loglines lines and was:" - cat /tmp/plugin-cli-output - false - } - fi + [ "$loglines" -eq "2" ] || { + echo "Expected 2 lines excluding progress bar but the output had $loglines lines and was:" + cat /tmp/plugin-cli-output + false + } remove_jvm_example local relativePath=${1:-$(readlink -m jvm-example-*.zip)} sudo -E -u $ESPLUGIN_COMMAND_USER ES_JAVA_OPTS="-Des.logger.level=DEBUG" "$ESHOME/bin/elasticsearch-plugin" install "file://$relativePath" > /tmp/plugin-cli-output local loglines=$(cat /tmp/plugin-cli-output | grep -v "^[[:cntrl:]]" | wc -l) - if [ "$GROUP" == "TAR PLUGINS" ]; then - [ "$loglines" -gt "3" ] || { - echo "Expected more than 3 lines excluding progress bar but the output had $loglines lines and was:" - cat /tmp/plugin-cli-output - false - } - else - [ "$loglines" -gt "2" ] || { - echo "Expected more than 2 lines excluding progress bar but the output had $loglines lines and was:" - cat /tmp/plugin-cli-output - false - } - fi + [ "$loglines" -gt "2" ] || { + echo "Expected more than 2 lines excluding progress bar but the output had $loglines lines and was:" + cat /tmp/plugin-cli-output + false + } remove_jvm_example } From 9624fd26a8a8145d023d66c6f70f684aa8b81ecd Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Wed, 2 Nov 2016 20:02:48 +0100 Subject: [PATCH 107/132] [TEST] Update destructive operations and disable close IT tests (#21274) These tests had a single method due to the fact that es didn't support resetting settings when they were first written. They can now be rewritten to have separate methods and an after method that resets the setting that is left behind. --- .../state/CloseIndexDisableCloseAllIT.java | 89 +++-------- .../DestructiveOperationsIT.java | 147 ++++++++++++++++++ .../DestructiveOperationsIntegrationIT.java | 109 ------------- 3 files changed, 166 insertions(+), 179 deletions(-) create mode 100644 core/src/test/java/org/elasticsearch/operateAllIndices/DestructiveOperationsIT.java delete mode 100644 core/src/test/java/org/elasticsearch/operateAllIndices/DestructiveOperationsIntegrationIT.java diff --git a/core/src/test/java/org/elasticsearch/indices/state/CloseIndexDisableCloseAllIT.java b/core/src/test/java/org/elasticsearch/indices/state/CloseIndexDisableCloseAllIT.java index 8ec629dbbdc..54bdfd05008 100644 --- a/core/src/test/java/org/elasticsearch/indices/state/CloseIndexDisableCloseAllIT.java +++ b/core/src/test/java/org/elasticsearch/indices/state/CloseIndexDisableCloseAllIT.java @@ -18,99 +18,48 @@ */ package org.elasticsearch.indices.state; -import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; -import org.elasticsearch.action.admin.indices.close.CloseIndexResponse; import org.elasticsearch.action.admin.indices.close.TransportCloseIndexAction; -import org.elasticsearch.action.support.DestructiveOperations; -import org.elasticsearch.client.Client; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.test.ESIntegTestCase.ClusterScope; -import org.elasticsearch.test.ESIntegTestCase.Scope; +import org.junit.After; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.notNullValue; -@ClusterScope(scope=Scope.TEST, numDataNodes=2) public class CloseIndexDisableCloseAllIT extends ESIntegTestCase { - // Combined multiple tests into one, because cluster scope is test. - // The cluster scope is test b/c we can't clear cluster settings. - public void testCloseAllRequiresName() { - Settings clusterSettings = Settings.builder() - .put(DestructiveOperations.REQUIRES_NAME_SETTING.getKey(), true) + + @After + public void afterTest() { + Settings settings = Settings.builder().put(TransportCloseIndexAction.CLUSTER_INDICES_CLOSE_ENABLE_SETTING.getKey(), (String)null) .build(); - assertAcked(client().admin().cluster().prepareUpdateSettings().setTransientSettings(clusterSettings)); + assertAcked(client().admin().cluster().prepareUpdateSettings().setTransientSettings(settings)); + } + + public void testCloseAllRequiresName() { createIndex("test1", "test2", "test3"); - ClusterHealthResponse healthResponse = client().admin().cluster().prepareHealth().setWaitForGreenStatus().execute().actionGet(); - assertThat(healthResponse.isTimedOut(), equalTo(false)); - // Close all explicitly - try { - client().admin().indices().prepareClose("_all").execute().actionGet(); - fail(); - } catch (IllegalArgumentException e) { - } - - // Close all wildcard - try { - client().admin().indices().prepareClose("*").execute().actionGet(); - fail(); - } catch (IllegalArgumentException e) { - } - - // Close all wildcard - try { - client().admin().indices().prepareClose("test*").execute().actionGet(); - fail(); - } catch (IllegalArgumentException e) { - } - - // Close all wildcard - try { - client().admin().indices().prepareClose("*", "-test1").execute().actionGet(); - fail(); - } catch (IllegalArgumentException e) { - } - - // Close all wildcard - try { - client().admin().indices().prepareClose("*", "-test1", "+test1").execute().actionGet(); - fail(); - } catch (IllegalArgumentException e) { - } - - CloseIndexResponse closeIndexResponse = client().admin().indices().prepareClose("test3", "test2").execute().actionGet(); - assertThat(closeIndexResponse.isAcknowledged(), equalTo(true)); + assertAcked(client().admin().indices().prepareClose("test3", "test2")); assertIndexIsClosed("test2", "test3"); // disable closing - Client client = client(); createIndex("test_no_close"); - healthResponse = client.admin().cluster().prepareHealth().setWaitForGreenStatus().execute().actionGet(); - assertThat(healthResponse.isTimedOut(), equalTo(false)); - client().admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.builder().put(TransportCloseIndexAction.CLUSTER_INDICES_CLOSE_ENABLE_SETTING.getKey(), false)).get(); + Settings settings = Settings.builder().put(TransportCloseIndexAction.CLUSTER_INDICES_CLOSE_ENABLE_SETTING.getKey(), false).build(); + assertAcked(client().admin().cluster().prepareUpdateSettings().setTransientSettings(settings)); - try { - client.admin().indices().prepareClose("test_no_close").execute().actionGet(); - fail("exception expected"); - } catch (IllegalStateException ex) { - assertEquals(ex.getMessage(), "closing indices is disabled - set [cluster.indices.close.enable: true] to enable it. NOTE: closed indices still consume a significant amount of diskspace"); - } + IllegalStateException illegalStateException = expectThrows(IllegalStateException.class, + () -> client().admin().indices().prepareClose("test_no_close").get()); + assertEquals(illegalStateException.getMessage(), + "closing indices is disabled - set [cluster.indices.close.enable: true] to enable it. NOTE: closed indices still " + + "consume a significant amount of diskspace"); } private void assertIndexIsClosed(String... indices) { - checkIndexState(IndexMetaData.State.CLOSE, indices); - } - - private void checkIndexState(IndexMetaData.State state, String... indices) { ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().execute().actionGet(); for (String index : indices) { IndexMetaData indexMetaData = clusterStateResponse.getState().metaData().indices().get(index); - assertThat(indexMetaData, notNullValue()); - assertThat(indexMetaData.getState(), equalTo(state)); + assertNotNull(indexMetaData); + assertEquals(IndexMetaData.State.CLOSE, indexMetaData.getState()); } } } diff --git a/core/src/test/java/org/elasticsearch/operateAllIndices/DestructiveOperationsIT.java b/core/src/test/java/org/elasticsearch/operateAllIndices/DestructiveOperationsIT.java new file mode 100644 index 00000000000..fea2e4699d5 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/operateAllIndices/DestructiveOperationsIT.java @@ -0,0 +1,147 @@ +/* + * 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.operateAllIndices; + +import com.carrotsearch.hppc.cursors.ObjectObjectCursor; +import org.elasticsearch.action.support.DestructiveOperations; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.test.ESIntegTestCase; +import org.junit.After; + +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.hamcrest.Matchers.equalTo; + +public class DestructiveOperationsIT extends ESIntegTestCase { + + @After + public void afterTest() { + Settings settings = Settings.builder().put(DestructiveOperations.REQUIRES_NAME_SETTING.getKey(), (String)null).build(); + assertAcked(client().admin().cluster().prepareUpdateSettings().setTransientSettings(settings)); + } + + public void testDeleteIndexIsRejected() throws Exception { + Settings settings = Settings.builder() + .put(DestructiveOperations.REQUIRES_NAME_SETTING.getKey(), true) + .build(); + assertAcked(client().admin().cluster().prepareUpdateSettings().setTransientSettings(settings)); + + createIndex("index1", "1index"); + + // Should succeed, since no wildcards + assertAcked(client().admin().indices().prepareDelete("1index").get()); + + expectThrows(IllegalArgumentException.class, () -> client().admin().indices().prepareDelete("i*").get()); + expectThrows(IllegalArgumentException.class, () -> client().admin().indices().prepareDelete("_all").get()); + } + + public void testDeleteIndexDefaultBehaviour() throws Exception { + if (randomBoolean()) { + Settings settings = Settings.builder() + .put(DestructiveOperations.REQUIRES_NAME_SETTING.getKey(), false) + .build(); + assertAcked(client().admin().cluster().prepareUpdateSettings().setTransientSettings(settings)); + } + + createIndex("index1", "1index"); + + if (randomBoolean()) { + assertAcked(client().admin().indices().prepareDelete("_all").get()); + } else { + assertAcked(client().admin().indices().prepareDelete("*").get()); + } + + assertThat(client().admin().indices().prepareExists("_all").get().isExists(), equalTo(false)); + } + + public void testCloseIndexIsRejected() throws Exception { + Settings settings = Settings.builder() + .put(DestructiveOperations.REQUIRES_NAME_SETTING.getKey(), true) + .build(); + assertAcked(client().admin().cluster().prepareUpdateSettings().setTransientSettings(settings)); + + createIndex("index1", "1index"); + + // Should succeed, since no wildcards + assertAcked(client().admin().indices().prepareClose("1index").get()); + + expectThrows(IllegalArgumentException.class, () -> client().admin().indices().prepareClose("i*").get()); + expectThrows(IllegalArgumentException.class, () -> client().admin().indices().prepareClose("_all").get()); + } + + public void testCloseIndexDefaultBehaviour() throws Exception { + if (randomBoolean()) { + Settings settings = Settings.builder() + .put(DestructiveOperations.REQUIRES_NAME_SETTING.getKey(), false) + .build(); + assertAcked(client().admin().cluster().prepareUpdateSettings().setTransientSettings(settings)); + } + + createIndex("index1", "1index"); + + if (randomBoolean()) { + assertAcked(client().admin().indices().prepareClose("_all").get()); + } else { + assertAcked(client().admin().indices().prepareClose("*").get()); + } + + ClusterState state = client().admin().cluster().prepareState().get().getState(); + for (ObjectObjectCursor indexMetaDataObjectObjectCursor : state.getMetaData().indices()) { + assertEquals(IndexMetaData.State.CLOSE, indexMetaDataObjectObjectCursor.value.getState()); + } + } + + public void testOpenIndexIsRejected() throws Exception { + Settings settings = Settings.builder() + .put(DestructiveOperations.REQUIRES_NAME_SETTING.getKey(), true) + .build(); + assertAcked(client().admin().cluster().prepareUpdateSettings().setTransientSettings(settings)); + + createIndex("index1", "1index"); + assertAcked(client().admin().indices().prepareClose("1index", "index1").get()); + + expectThrows(IllegalArgumentException.class, () -> client().admin().indices().prepareOpen("i*").get()); + expectThrows(IllegalArgumentException.class, () -> client().admin().indices().prepareOpen("_all").get()); + } + + public void testOpenIndexDefaultBehaviour() throws Exception { + if (randomBoolean()) { + Settings settings = Settings.builder() + .put(DestructiveOperations.REQUIRES_NAME_SETTING.getKey(), false) + .build(); + assertAcked(client().admin().cluster().prepareUpdateSettings().setTransientSettings(settings)); + } + + createIndex("index1", "1index"); + assertAcked(client().admin().indices().prepareClose("1index", "index1").get()); + + if (randomBoolean()) { + assertAcked(client().admin().indices().prepareOpen("_all").get()); + } else { + assertAcked(client().admin().indices().prepareOpen("*").get()); + } + + ClusterState state = client().admin().cluster().prepareState().get().getState(); + for (ObjectObjectCursor indexMetaDataObjectObjectCursor : state.getMetaData().indices()) { + assertEquals(IndexMetaData.State.OPEN, indexMetaDataObjectObjectCursor.value.getState()); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/operateAllIndices/DestructiveOperationsIntegrationIT.java b/core/src/test/java/org/elasticsearch/operateAllIndices/DestructiveOperationsIntegrationIT.java deleted file mode 100644 index 28852d74696..00000000000 --- a/core/src/test/java/org/elasticsearch/operateAllIndices/DestructiveOperationsIntegrationIT.java +++ /dev/null @@ -1,109 +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.operateAllIndices; - -import org.elasticsearch.action.support.DestructiveOperations; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.test.ESIntegTestCase; - -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; -import static org.hamcrest.Matchers.equalTo; - -@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST) -public class DestructiveOperationsIntegrationIT extends ESIntegTestCase { - // One test for test performance, since cluster scope is test - // The cluster scope is test b/c we can't clear cluster settings. - public void testDestructiveOperations() throws Exception { - Settings settings = Settings.builder() - .put(DestructiveOperations.REQUIRES_NAME_SETTING.getKey(), true) - .build(); - assertAcked(client().admin().cluster().prepareUpdateSettings().setTransientSettings(settings)); - - assertAcked(client().admin().indices().prepareCreate("index1").get()); - assertAcked(client().admin().indices().prepareCreate("1index").get()); - - // Should succeed, since no wildcards - assertAcked(client().admin().indices().prepareDelete("1index").get()); - - try { - // should fail since index1 is the only index. - client().admin().indices().prepareDelete("i*").get(); - fail(); - } catch (IllegalArgumentException e) { - } - - try { - client().admin().indices().prepareDelete("_all").get(); - fail(); - } catch (IllegalArgumentException e) { - } - - settings = Settings.builder() - .put(DestructiveOperations.REQUIRES_NAME_SETTING.getKey(), false) - .build(); - assertAcked(client().admin().cluster().prepareUpdateSettings().setTransientSettings(settings)); - - assertAcked(client().admin().indices().prepareDelete("_all").get()); - assertThat(client().admin().indices().prepareExists("_all").get().isExists(), equalTo(false)); - - // end delete index: - // close index: - settings = Settings.builder() - .put(DestructiveOperations.REQUIRES_NAME_SETTING.getKey(), true) - .build(); - assertAcked(client().admin().cluster().prepareUpdateSettings().setTransientSettings(settings)); - - assertAcked(client().admin().indices().prepareCreate("index1").get()); - assertAcked(client().admin().indices().prepareCreate("1index").get()); - // Should succeed, since no wildcards - assertAcked(client().admin().indices().prepareClose("1index").get()); - - try { - client().admin().indices().prepareClose("_all").get(); - fail(); - } catch (IllegalArgumentException e) { - } - try { - assertAcked(client().admin().indices().prepareOpen("_all").get()); - fail(); - } catch (IllegalArgumentException e) { - } - try { - client().admin().indices().prepareClose("*").get(); - fail(); - } catch (IllegalArgumentException e) { - } - try { - assertAcked(client().admin().indices().prepareOpen("*").get()); - fail(); - } catch (IllegalArgumentException e) { - } - - settings = Settings.builder() - .put(DestructiveOperations.REQUIRES_NAME_SETTING.getKey(), false) - .build(); - assertAcked(client().admin().cluster().prepareUpdateSettings().setTransientSettings(settings)); - assertAcked(client().admin().indices().prepareClose("_all").get()); - assertAcked(client().admin().indices().prepareOpen("_all").get()); - - // end close index: - client().admin().indices().prepareDelete("_all").get(); - } -} From 2e652d3b577adfc14dfd87636f64f93ec2cb16f5 Mon Sep 17 00:00:00 2001 From: Igor Motov Date: Wed, 2 Nov 2016 13:21:12 -1000 Subject: [PATCH 108/132] Stored scripts and ingest node configurations should be included into a snapshot Stored scripts and ingest node configuration are important parts of the overall cluster state and should be included into a snapshot together with index templates and persistent settings if the includeGlobalState is set to true. Closes #21184 --- .../cluster/metadata/MetaData.java | 19 +++ .../elasticsearch/ingest/IngestMetadata.java | 2 +- .../elasticsearch/script/ScriptMetaData.java | 2 +- .../SharedClusterSnapshotRestoreIT.java | 121 +++++++++++++++--- 4 files changed, 126 insertions(+), 18 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java index fd7e08fec31..3ee28f1662c 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java @@ -97,10 +97,29 @@ public class MetaData implements Iterable, Diffable, Fr SNAPSHOT } + /** + * Indicates that this custom metadata will be returned as part of an API call but will not be persisted + */ public static EnumSet API_ONLY = EnumSet.of(XContentContext.API); + + /** + * Indicates that this custom metadata will be returned as part of an API call and will be persisted between + * node restarts, but will not be a part of a snapshot global state + */ public static EnumSet API_AND_GATEWAY = EnumSet.of(XContentContext.API, XContentContext.GATEWAY); + + /** + * Indicates that this custom metadata will be returned as part of an API call and stored as a part of + * a snapshot global state, but will not be persisted between node restarts + */ public static EnumSet API_AND_SNAPSHOT = EnumSet.of(XContentContext.API, XContentContext.SNAPSHOT); + /** + * Indicates that this custom metadata will be returned as part of an API call, stored as a part of + * a snapshot global state, and will be persisted between node restarts + */ + public static EnumSet ALL_CONTEXTS = EnumSet.allOf(XContentContext.class); + public interface Custom extends Diffable, ToXContent { String type(); diff --git a/core/src/main/java/org/elasticsearch/ingest/IngestMetadata.java b/core/src/main/java/org/elasticsearch/ingest/IngestMetadata.java index 9ad369e22d4..40f401ac6b4 100644 --- a/core/src/main/java/org/elasticsearch/ingest/IngestMetadata.java +++ b/core/src/main/java/org/elasticsearch/ingest/IngestMetadata.java @@ -116,7 +116,7 @@ public final class IngestMetadata implements MetaData.Custom { @Override public EnumSet context() { - return MetaData.API_AND_GATEWAY; + return MetaData.ALL_CONTEXTS; } @Override diff --git a/core/src/main/java/org/elasticsearch/script/ScriptMetaData.java b/core/src/main/java/org/elasticsearch/script/ScriptMetaData.java index 979bffb4bcc..84855da2f94 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptMetaData.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptMetaData.java @@ -131,7 +131,7 @@ public final class ScriptMetaData implements MetaData.Custom { @Override public EnumSet context() { - return MetaData.API_AND_GATEWAY; + return MetaData.ALL_CONTEXTS; } @Override diff --git a/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java b/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java index b7ce99e6ea3..0c8dbd4c42c 100644 --- a/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java +++ b/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java @@ -34,10 +34,14 @@ import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotIndexStat import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotStatus; import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotsStatusResponse; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; +import org.elasticsearch.action.admin.cluster.storedscripts.GetStoredScriptRequest; +import org.elasticsearch.action.admin.cluster.storedscripts.GetStoredScriptResponse; import org.elasticsearch.action.admin.indices.flush.FlushResponse; import org.elasticsearch.action.admin.indices.settings.get.GetSettingsResponse; import org.elasticsearch.action.admin.indices.template.get.GetIndexTemplatesResponse; import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.action.ingest.DeletePipelineRequest; +import org.elasticsearch.action.ingest.GetPipelineResponse; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.client.Client; @@ -54,6 +58,8 @@ import org.elasticsearch.cluster.metadata.MetaDataIndexStateService; import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Priority; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; @@ -64,10 +70,16 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.IndexStore; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.InvalidIndexNameException; +import org.elasticsearch.ingest.IngestTestPlugin; +import org.elasticsearch.plugins.Plugin; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.repositories.RepositoryData; import org.elasticsearch.repositories.RepositoryException; +import org.elasticsearch.script.MockScriptEngine; +import org.elasticsearch.script.ScriptService; +import org.elasticsearch.script.StoredScriptsIT; +import org.elasticsearch.snapshots.mockstore.MockRepository; import org.elasticsearch.test.junit.annotations.TestLogging; import java.nio.channels.SeekableByteChannel; @@ -76,6 +88,7 @@ import java.nio.file.Path; import java.nio.file.StandardOpenOption; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Locale; @@ -86,7 +99,9 @@ import java.util.stream.Collectors; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS; +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.index.IndexSettings.INDEX_REFRESH_INTERVAL_SETTING; +import static org.elasticsearch.index.query.QueryBuilders.boolQuery; import static org.elasticsearch.index.query.QueryBuilders.matchQuery; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAliasesExist; @@ -109,6 +124,14 @@ import static org.hamcrest.Matchers.nullValue; import static org.hamcrest.Matchers.startsWith; public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCase { + + @Override + protected Collection> nodePlugins() { + return Arrays.asList(IngestTestPlugin.class, + StoredScriptsIT.CustomScriptPlugin.class, + MockRepository.Plugin.class); + } + public void testBasicWorkFlow() throws Exception { Client client = client(); @@ -459,11 +482,39 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas assertAcked(client.admin().cluster().preparePutRepository("test-repo") .setType("fs").setSettings(Settings.builder().put("location", location))); - logger.info("--> creating test template"); - assertThat(client.admin().indices().preparePutTemplate("test-template").setTemplate("te*").addMapping("test-mapping", XContentFactory.jsonBuilder().startObject().startObject("test-mapping").startObject("properties") - .startObject("field1").field("type", "string").field("store", "yes").endObject() - .startObject("field2").field("type", "string").field("store", "yes").field("index", "not_analyzed").endObject() - .endObject().endObject().endObject()).get().isAcknowledged(), equalTo(true)); + boolean testTemplate = randomBoolean(); + boolean testPipeline = randomBoolean(); + boolean testScript = (testTemplate == false && testPipeline == false) || randomBoolean(); // At least something should be stored + + if(testTemplate) { + logger.info("--> creating test template"); + assertThat(client.admin().indices().preparePutTemplate("test-template").setTemplate("te*").addMapping("test-mapping", XContentFactory.jsonBuilder().startObject().startObject("test-mapping").startObject("properties") + .startObject("field1").field("type", "string").field("store", "yes").endObject() + .startObject("field2").field("type", "string").field("store", "yes").field("index", "not_analyzed").endObject() + .endObject().endObject().endObject()).get().isAcknowledged(), equalTo(true)); + } + + if(testPipeline) { + logger.info("--> creating test pipeline"); + BytesReference pipelineSource = jsonBuilder().startObject() + .field("description", "my_pipeline") + .startArray("processors") + .startObject() + .startObject("test") + .endObject() + .endObject() + .endArray() + .endObject().bytes(); + assertAcked(client().admin().cluster().preparePutPipeline("barbaz", pipelineSource).get()); + } + + if(testScript) { + logger.info("--> creating test script"); + assertAcked(client().admin().cluster().preparePutStoredScript() + .setScriptLang(MockScriptEngine.NAME) + .setId("foobar") + .setSource(new BytesArray("{\"script\":\"1\"}"))); + } logger.info("--> snapshot without global state"); CreateSnapshotResponse createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap-no-global-state").setIndices().setIncludeGlobalState(false).setWaitForCompletion(true).get(); @@ -477,26 +528,52 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), equalTo(0)); assertThat(client.admin().cluster().prepareGetSnapshots("test-repo").setSnapshots("test-snap-with-global-state").get().getSnapshots().get(0).state(), equalTo(SnapshotState.SUCCESS)); - logger.info("--> delete test template"); - cluster().wipeTemplates("test-template"); - GetIndexTemplatesResponse getIndexTemplatesResponse = client().admin().indices().prepareGetTemplates().get(); - assertIndexTemplateMissing(getIndexTemplatesResponse, "test-template"); + if (testTemplate) { + logger.info("--> delete test template"); + cluster().wipeTemplates("test-template"); + GetIndexTemplatesResponse getIndexTemplatesResponse = client().admin().indices().prepareGetTemplates().get(); + assertIndexTemplateMissing(getIndexTemplatesResponse, "test-template"); + } + + if (testPipeline) { + logger.info("--> delete test pipeline"); + assertAcked(client().admin().cluster().deletePipeline(new DeletePipelineRequest("barbaz")).get()); + } + + if (testScript) { + logger.info("--> delete test script"); + assertAcked(client().admin().cluster().prepareDeleteStoredScript(MockScriptEngine.NAME, "foobar").get()); + } logger.info("--> try restoring cluster state from snapshot without global state"); RestoreSnapshotResponse restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap-no-global-state").setWaitForCompletion(true).setRestoreGlobalState(true).execute().actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), equalTo(0)); logger.info("--> check that template wasn't restored"); - getIndexTemplatesResponse = client().admin().indices().prepareGetTemplates().get(); + GetIndexTemplatesResponse getIndexTemplatesResponse = client().admin().indices().prepareGetTemplates().get(); assertIndexTemplateMissing(getIndexTemplatesResponse, "test-template"); logger.info("--> restore cluster state"); restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap-with-global-state").setWaitForCompletion(true).setRestoreGlobalState(true).execute().actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), equalTo(0)); - logger.info("--> check that template is restored"); - getIndexTemplatesResponse = client().admin().indices().prepareGetTemplates().get(); - assertIndexTemplateExists(getIndexTemplatesResponse, "test-template"); + if (testTemplate) { + logger.info("--> check that template is restored"); + getIndexTemplatesResponse = client().admin().indices().prepareGetTemplates().get(); + assertIndexTemplateExists(getIndexTemplatesResponse, "test-template"); + } + + if (testPipeline) { + logger.info("--> check that pipeline is restored"); + GetPipelineResponse getPipelineResponse = client().admin().cluster().prepareGetPipeline("barbaz").get(); + assertTrue(getPipelineResponse.isFound()); + } + + if (testScript) { + logger.info("--> check that script is restored"); + GetStoredScriptResponse getStoredScriptResponse = client().admin().cluster().prepareGetStoredScript(MockScriptEngine.NAME, "foobar").get(); + assertNotNull(getStoredScriptResponse.getStoredScript()); + } createIndex("test-idx"); ensureGreen(); @@ -514,9 +591,19 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), equalTo(createSnapshotResponse.getSnapshotInfo().totalShards())); assertThat(client.admin().cluster().prepareGetSnapshots("test-repo").setSnapshots("test-snap-no-global-state-with-index").get().getSnapshots().get(0).state(), equalTo(SnapshotState.SUCCESS)); - logger.info("--> delete test template and index "); + logger.info("--> delete global state and index "); cluster().wipeIndices("test-idx"); - cluster().wipeTemplates("test-template"); + if (testTemplate) { + cluster().wipeTemplates("test-template"); + } + if (testPipeline) { + assertAcked(client().admin().cluster().deletePipeline(new DeletePipelineRequest("barbaz")).get()); + } + + if (testScript) { + assertAcked(client().admin().cluster().prepareDeleteStoredScript(MockScriptEngine.NAME, "foobar").get()); + } + getIndexTemplatesResponse = client().admin().indices().prepareGetTemplates().get(); assertIndexTemplateMissing(getIndexTemplatesResponse, "test-template"); @@ -525,9 +612,11 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); assertThat(restoreSnapshotResponse.getRestoreInfo().failedShards(), equalTo(0)); - logger.info("--> check that template wasn't restored but index was"); + logger.info("--> check that global state wasn't restored but index was"); getIndexTemplatesResponse = client().admin().indices().prepareGetTemplates().get(); assertIndexTemplateMissing(getIndexTemplatesResponse, "test-template"); + assertFalse(client().admin().cluster().prepareGetPipeline("barbaz").get().isFound()); + assertNull(client().admin().cluster().prepareGetStoredScript(MockScriptEngine.NAME, "foobar").get().getStoredScript()); assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(100L)); } From 2110221f59ebf7c17fa001afce82b3b0b308873e Mon Sep 17 00:00:00 2001 From: Areek Zillur Date: Wed, 2 Nov 2016 20:18:31 -0400 Subject: [PATCH 109/132] add EngineClosed and IndexShardClosed exceptions to assertions on executing bulk shard operation on replica --- .../action/bulk/TransportShardBulkAction.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 0f3f1f86cfa..01528c7d228 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -47,9 +47,11 @@ import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.engine.EngineClosedException; import org.elasticsearch.index.engine.VersionConflictEngineException; import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.IndexShardClosedException; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.threadpool.ThreadPool; @@ -346,7 +348,10 @@ public class TransportShardBulkAction extends TransportWriteAction Date: Mon, 31 Oct 2016 14:49:41 -1000 Subject: [PATCH 110/132] ClusterAdminClient.prepareDeletePipeline method should accept pipeline id to delete Currently there is no way to specify the pipeline id when using client().prepareDeletePipeline() method --- .../action/ingest/DeletePipelineRequestBuilder.java | 8 ++++++++ .../java/org/elasticsearch/client/ClusterAdminClient.java | 5 +++++ .../org/elasticsearch/client/support/AbstractClient.java | 5 +++++ 3 files changed, 18 insertions(+) diff --git a/core/src/main/java/org/elasticsearch/action/ingest/DeletePipelineRequestBuilder.java b/core/src/main/java/org/elasticsearch/action/ingest/DeletePipelineRequestBuilder.java index fc14e0de2df..90cbce135af 100644 --- a/core/src/main/java/org/elasticsearch/action/ingest/DeletePipelineRequestBuilder.java +++ b/core/src/main/java/org/elasticsearch/action/ingest/DeletePipelineRequestBuilder.java @@ -32,4 +32,12 @@ public class DeletePipelineRequestBuilder extends ActionRequestBuilder listener) { execute(GetPipelineAction.INSTANCE, request, listener); From 5d79eab9826ab1f1277d81ec6540c8eb0ae34541 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Thu, 3 Nov 2016 08:58:15 +0100 Subject: [PATCH 111/132] Fix the request cache keys to not hold references to the SearchContext. (#21284) Currently the request cache adds a `CacheEntity` object. It looks quite innocent but in practice it has a reference to a lambda that knows how to create a value. The issue is that this lambda has implicit references to the SearchContext object, which can be quite heavy since it holds a structured representation of aggregations for instance. This pull request splits the `CacheEntity` object from the object that generates cache values. --- .../common/io/stream/BytesStreamOutput.java | 2 +- .../AbstractIndexShardCacheEntity.java | 49 +----- .../indices/IndicesRequestCache.java | 69 +++------ .../elasticsearch/indices/IndicesService.java | 66 ++++++-- .../indices/IndicesRequestCacheTests.java | 145 +++++++++++------- 5 files changed, 170 insertions(+), 161 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/common/io/stream/BytesStreamOutput.java b/core/src/main/java/org/elasticsearch/common/io/stream/BytesStreamOutput.java index c427cbfac49..21de0c421b7 100644 --- a/core/src/main/java/org/elasticsearch/common/io/stream/BytesStreamOutput.java +++ b/core/src/main/java/org/elasticsearch/common/io/stream/BytesStreamOutput.java @@ -123,7 +123,7 @@ public class BytesStreamOutput extends StreamOutput implements BytesStream { } @Override - public void close() throws IOException { + public void close() { // empty for now. } diff --git a/core/src/main/java/org/elasticsearch/indices/AbstractIndexShardCacheEntity.java b/core/src/main/java/org/elasticsearch/indices/AbstractIndexShardCacheEntity.java index c0d929d82f5..98afd8781b4 100644 --- a/core/src/main/java/org/elasticsearch/indices/AbstractIndexShardCacheEntity.java +++ b/core/src/main/java/org/elasticsearch/indices/AbstractIndexShardCacheEntity.java @@ -19,40 +19,15 @@ package org.elasticsearch.indices; -import org.apache.lucene.index.DirectoryReader; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.cache.RemovalNotification; -import org.elasticsearch.common.io.stream.BytesStreamOutput; -import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.index.cache.request.ShardRequestCache; import org.elasticsearch.index.shard.IndexShard; -import java.io.IOException; - /** * Abstract base class for the an {@link IndexShard} level {@linkplain IndicesRequestCache.CacheEntity}. */ abstract class AbstractIndexShardCacheEntity implements IndicesRequestCache.CacheEntity { - @FunctionalInterface - public interface Loader { - void load(StreamOutput out) throws IOException; - } - - private final Loader loader; - private boolean loadedFromCache = true; - - protected AbstractIndexShardCacheEntity(Loader loader) { - this.loader = loader; - } - - /** - * When called after passing this through - * {@link IndicesRequestCache#getOrCompute(IndicesRequestCache.CacheEntity, DirectoryReader, BytesReference)} this will return whether - * or not the result was loaded from the cache. - */ - public final boolean loadedFromCache() { - return loadedFromCache; - } /** * Get the {@linkplain ShardRequestCache} used to track cache statistics. @@ -60,27 +35,7 @@ abstract class AbstractIndexShardCacheEntity implements IndicesRequestCache.Cach protected abstract ShardRequestCache stats(); @Override - public final IndicesRequestCache.Value loadValue() throws IOException { - /* BytesStreamOutput allows to pass the expected size but by default uses - * BigArrays.PAGE_SIZE_IN_BYTES which is 16k. A common cached result ie. - * a date histogram with 3 buckets is ~100byte so 16k might be very wasteful - * since we don't shrink to the actual size once we are done serializing. - * By passing 512 as the expected size we will resize the byte array in the stream - * slowly until we hit the page size and don't waste too much memory for small query - * results.*/ - final int expectedSizeInBytes = 512; - try (BytesStreamOutput out = new BytesStreamOutput(expectedSizeInBytes)) { - loader.load(out); - // for now, keep the paged data structure, which might have unused bytes to fill a page, but better to keep - // the memory properly paged instead of having varied sized bytes - final BytesReference reference = out.bytes(); - loadedFromCache = false; - return new IndicesRequestCache.Value(reference, out.ramBytesUsed()); - } - } - - @Override - public final void onCached(IndicesRequestCache.Key key, IndicesRequestCache.Value value) { + public final void onCached(IndicesRequestCache.Key key, BytesReference value) { stats().onCached(key, value); } @@ -95,7 +50,7 @@ abstract class AbstractIndexShardCacheEntity implements IndicesRequestCache.Cach } @Override - public final void onRemoval(RemovalNotification notification) { + public final void onRemoval(RemovalNotification notification) { stats().onRemoval(notification.getKey(), notification.getValue(), notification.getRemovalReason() == RemovalNotification.RemovalReason.EVICTED); } diff --git a/core/src/main/java/org/elasticsearch/indices/IndicesRequestCache.java b/core/src/main/java/org/elasticsearch/indices/IndicesRequestCache.java index a08f9ca1ad4..0fcda5c8fd5 100644 --- a/core/src/main/java/org/elasticsearch/indices/IndicesRequestCache.java +++ b/core/src/main/java/org/elasticsearch/indices/IndicesRequestCache.java @@ -41,12 +41,12 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import java.io.Closeable; -import java.io.IOException; import java.util.Collection; import java.util.Collections; import java.util.Iterator; import java.util.Set; import java.util.concurrent.ConcurrentMap; +import java.util.function.Supplier; /** * The indices request cache allows to cache a shard level request stage responses, helping with improving @@ -62,7 +62,7 @@ import java.util.concurrent.ConcurrentMap; * is functional. */ public final class IndicesRequestCache extends AbstractComponent implements RemovalListener, Closeable { + BytesReference>, Closeable { /** * A setting to enable or disable request caching on an index level. Its dynamic by default @@ -79,14 +79,14 @@ public final class IndicesRequestCache extends AbstractComponent implements Remo private final Set keysToClean = ConcurrentCollections.newConcurrentSet(); private final ByteSizeValue size; private final TimeValue expire; - private final Cache cache; + private final Cache cache; IndicesRequestCache(Settings settings) { super(settings); this.size = INDICES_CACHE_QUERY_SIZE.get(settings); this.expire = INDICES_CACHE_QUERY_EXPIRE.exists(settings) ? INDICES_CACHE_QUERY_EXPIRE.get(settings) : null; long sizeInBytes = size.getBytes(); - CacheBuilder cacheBuilder = CacheBuilder.builder() + CacheBuilder cacheBuilder = CacheBuilder.builder() .setMaximumWeight(sizeInBytes).weigher((k, v) -> k.ramBytesUsed() + v.ramBytesUsed()).removalListener(this); if (expire != null) { cacheBuilder.setExpireAfterAccess(expire); @@ -105,15 +105,16 @@ public final class IndicesRequestCache extends AbstractComponent implements Remo } @Override - public void onRemoval(RemovalNotification notification) { + public void onRemoval(RemovalNotification notification) { notification.getKey().entity.onRemoval(notification); } - BytesReference getOrCompute(CacheEntity cacheEntity, DirectoryReader reader, BytesReference cacheKey) throws Exception { + BytesReference getOrCompute(CacheEntity cacheEntity, Supplier loader, + DirectoryReader reader, BytesReference cacheKey) throws Exception { final Key key = new Key(cacheEntity, reader.getVersion(), cacheKey); - Loader loader = new Loader(cacheEntity); - Value value = cache.computeIfAbsent(key, loader); - if (loader.isLoaded()) { + Loader cacheLoader = new Loader(cacheEntity, loader); + BytesReference value = cache.computeIfAbsent(key, cacheLoader); + if (cacheLoader.isLoaded()) { key.entity.onMiss(); // see if its the first time we see this reader, and make sure to register a cleanup key CleanupKey cleanupKey = new CleanupKey(cacheEntity, reader.getVersion()); @@ -126,16 +127,18 @@ public final class IndicesRequestCache extends AbstractComponent implements Remo } else { key.entity.onHit(); } - return value.reference; + return value; } - private static class Loader implements CacheLoader { + private static class Loader implements CacheLoader { private final CacheEntity entity; + private final Supplier loader; private boolean loaded; - Loader(CacheEntity entity) { + Loader(CacheEntity entity, Supplier loader) { this.entity = entity; + this.loader = loader; } public boolean isLoaded() { @@ -143,8 +146,8 @@ public final class IndicesRequestCache extends AbstractComponent implements Remo } @Override - public Value load(Key key) throws Exception { - Value value = entity.loadValue(); + public BytesReference load(Key key) throws Exception { + BytesReference value = loader.get(); entity.onCached(key, value); loaded = true; return value; @@ -154,16 +157,12 @@ public final class IndicesRequestCache extends AbstractComponent implements Remo /** * Basic interface to make this cache testable. */ - interface CacheEntity { - /** - * Loads the actual cache value. this is the heavy lifting part. - */ - Value loadValue() throws IOException; + interface CacheEntity extends Accountable { /** - * Called after the value was loaded via {@link #loadValue()} + * Called after the value was loaded. */ - void onCached(Key key, Value value); + void onCached(Key key, BytesReference value); /** * Returns true iff the resource behind this entity is still open ie. @@ -190,32 +189,12 @@ public final class IndicesRequestCache extends AbstractComponent implements Remo /** * Called when this entity instance is removed */ - void onRemoval(RemovalNotification notification); - } - - - - static class Value implements Accountable { - final BytesReference reference; - final long ramBytesUsed; - - Value(BytesReference reference, long ramBytesUsed) { - this.reference = reference; - this.ramBytesUsed = ramBytesUsed; - } - - @Override - public long ramBytesUsed() { - return ramBytesUsed; - } - - @Override - public Collection getChildResources() { - return Collections.emptyList(); - } + void onRemoval(RemovalNotification notification); } static class Key implements Accountable { + private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(Key.class); + public final CacheEntity entity; // use as identity equality public final long readerVersion; // use the reader version to now keep a reference to a "short" lived reader until its reaped public final BytesReference value; @@ -228,7 +207,7 @@ public final class IndicesRequestCache extends AbstractComponent implements Remo @Override public long ramBytesUsed() { - return RamUsageEstimator.NUM_BYTES_OBJECT_REF + Long.BYTES + value.length(); + return BASE_RAM_BYTES_USED + entity.ramBytesUsed() + value.length(); } @Override diff --git a/core/src/main/java/org/elasticsearch/indices/IndicesService.java b/core/src/main/java/org/elasticsearch/indices/IndicesService.java index 4c7e541aafa..0d4bacefb93 100644 --- a/core/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/core/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -23,11 +23,11 @@ import com.carrotsearch.hppc.cursors.ObjectCursor; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.logging.log4j.util.Supplier; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.store.LockObtainFailedException; import org.apache.lucene.util.CollectionUtil; import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.RamUsageEstimator; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.admin.indices.stats.CommonStats; import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags; @@ -51,9 +51,11 @@ import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.io.FileSystemUtils; +import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.IndexScopedSettings; @@ -98,7 +100,6 @@ import org.elasticsearch.index.shard.IndexingOperationListener; import org.elasticsearch.index.shard.IndexingStats; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.IndexStoreConfig; -import org.elasticsearch.indices.AbstractIndexShardCacheEntity.Loader; import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.indices.cluster.IndicesClusterStateService; import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache; @@ -132,8 +133,10 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Predicate; +import java.util.function.Supplier; import java.util.stream.Collectors; import static java.util.Collections.emptyList; @@ -1110,7 +1113,7 @@ public class IndicesService extends AbstractLifecycleComponent if (shard == null) { return; } - indicesRequestCache.clear(new IndexShardCacheEntity(shard, null)); + indicesRequestCache.clear(new IndexShardCacheEntity(shard)); logger.trace("{} explicit cache clear", shard.shardId()); } @@ -1122,13 +1125,19 @@ public class IndicesService extends AbstractLifecycleComponent */ public void loadIntoContext(ShardSearchRequest request, SearchContext context, QueryPhase queryPhase) throws Exception { assert canCache(request, context); - final IndexShardCacheEntity entity = new IndexShardCacheEntity(context.indexShard(), out -> { - queryPhase.execute(context); - context.queryResult().writeToNoId(out); - }); final DirectoryReader directoryReader = context.searcher().getDirectoryReader(); - final BytesReference bytesReference = indicesRequestCache.getOrCompute(entity, directoryReader, request.cacheKey()); - if (entity.loadedFromCache()) { + + boolean[] loadedFromCache = new boolean[] { true }; + BytesReference bytesReference = cacheShardLevelResult(context.indexShard(), directoryReader, request.cacheKey(), out -> { + queryPhase.execute(context); + try { + context.queryResult().writeToNoId(out); + } catch (IOException e) { + throw new AssertionError("Could not serialize response", e); + } + loadedFromCache[0] = false; + }); + if (loadedFromCache[0]) { // restore the cached query result into the context final QuerySearchResult result = context.queryResult(); StreamInput in = new NamedWriteableAwareStreamInput(bytesReference.streamInput(), namedWriteableRegistry); @@ -1154,7 +1163,11 @@ public class IndicesService extends AbstractLifecycleComponent } BytesReference cacheKey = new BytesArray("fieldstats:" + field); BytesReference statsRef = cacheShardLevelResult(shard, searcher.getDirectoryReader(), cacheKey, out -> { - out.writeOptionalWriteable(fieldType.stats(searcher.reader())); + try { + out.writeOptionalWriteable(fieldType.stats(searcher.reader())); + } catch (IOException e) { + throw new IllegalStateException("Failed to write field stats output", e); + } }); try (StreamInput in = statsRef.streamInput()) { return in.readOptionalWriteable(FieldStats::readFrom); @@ -1173,17 +1186,33 @@ public class IndicesService extends AbstractLifecycleComponent * @param loader loads the data into the cache if needed * @return the contents of the cache or the result of calling the loader */ - private BytesReference cacheShardLevelResult(IndexShard shard, DirectoryReader reader, BytesReference cacheKey, Loader loader) + private BytesReference cacheShardLevelResult(IndexShard shard, DirectoryReader reader, BytesReference cacheKey, Consumer loader) throws Exception { - IndexShardCacheEntity cacheEntity = new IndexShardCacheEntity(shard, loader); - return indicesRequestCache.getOrCompute(cacheEntity, reader, cacheKey); + IndexShardCacheEntity cacheEntity = new IndexShardCacheEntity(shard); + Supplier supplier = () -> { + /* BytesStreamOutput allows to pass the expected size but by default uses + * BigArrays.PAGE_SIZE_IN_BYTES which is 16k. A common cached result ie. + * a date histogram with 3 buckets is ~100byte so 16k might be very wasteful + * since we don't shrink to the actual size once we are done serializing. + * By passing 512 as the expected size we will resize the byte array in the stream + * slowly until we hit the page size and don't waste too much memory for small query + * results.*/ + final int expectedSizeInBytes = 512; + try (BytesStreamOutput out = new BytesStreamOutput(expectedSizeInBytes)) { + loader.accept(out); + // for now, keep the paged data structure, which might have unused bytes to fill a page, but better to keep + // the memory properly paged instead of having varied sized bytes + return out.bytes(); + } + }; + return indicesRequestCache.getOrCompute(cacheEntity, supplier, reader, cacheKey); } static final class IndexShardCacheEntity extends AbstractIndexShardCacheEntity { + private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(IndexShardCacheEntity.class); private final IndexShard indexShard; - protected IndexShardCacheEntity(IndexShard indexShard, Loader loader) { - super(loader); + protected IndexShardCacheEntity(IndexShard indexShard) { this.indexShard = indexShard; } @@ -1201,6 +1230,13 @@ public class IndicesService extends AbstractLifecycleComponent public Object getCacheIdentity() { return indexShard; } + + @Override + public long ramBytesUsed() { + // No need to take the IndexShard into account since it is shared + // across many entities + return BASE_RAM_BYTES_USED; + } } @FunctionalInterface diff --git a/core/src/test/java/org/elasticsearch/indices/IndicesRequestCacheTests.java b/core/src/test/java/org/elasticsearch/indices/IndicesRequestCacheTests.java index 8bb8a4ddf8a..5d5584a156f 100644 --- a/core/src/test/java/org/elasticsearch/indices/IndicesRequestCacheTests.java +++ b/core/src/test/java/org/elasticsearch/indices/IndicesRequestCacheTests.java @@ -31,7 +31,7 @@ import org.apache.lucene.search.TopDocs; import org.apache.lucene.store.Directory; import org.apache.lucene.util.IOUtils; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; @@ -43,6 +43,7 @@ import org.elasticsearch.test.ESTestCase; import java.io.IOException; import java.util.Arrays; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Supplier; public class IndicesRequestCacheTests extends ESTestCase { @@ -59,23 +60,25 @@ public class IndicesRequestCacheTests extends ESTestCase { AtomicBoolean indexShard = new AtomicBoolean(true); // initial cache - TestEntity entity = new TestEntity(requestCacheStats, reader, indexShard, 0); - BytesReference value = cache.getOrCompute(entity, reader, termQuery.buildAsBytes()); + TestEntity entity = new TestEntity(requestCacheStats, indexShard); + Loader loader = new Loader(reader, 0); + BytesReference value = cache.getOrCompute(entity, loader, reader, termQuery.buildAsBytes()); assertEquals("foo", value.streamInput().readString()); assertEquals(0, requestCacheStats.stats().getHitCount()); assertEquals(1, requestCacheStats.stats().getMissCount()); assertEquals(0, requestCacheStats.stats().getEvictions()); - assertFalse(entity.loadedFromCache()); + assertFalse(loader.loadedFromCache); assertEquals(1, cache.count()); // cache hit - entity = new TestEntity(requestCacheStats, reader, indexShard, 0); - value = cache.getOrCompute(entity, reader, termQuery.buildAsBytes()); + entity = new TestEntity(requestCacheStats, indexShard); + loader = new Loader(reader, 0); + value = cache.getOrCompute(entity, loader, reader, termQuery.buildAsBytes()); assertEquals("foo", value.streamInput().readString()); assertEquals(1, requestCacheStats.stats().getHitCount()); assertEquals(1, requestCacheStats.stats().getMissCount()); assertEquals(0, requestCacheStats.stats().getEvictions()); - assertTrue(entity.loadedFromCache()); + assertTrue(loader.loadedFromCache); assertEquals(1, cache.count()); assertTrue(requestCacheStats.stats().getMemorySize().bytesAsInt() > value.length()); assertEquals(1, cache.numRegisteredCloseListeners()); @@ -91,7 +94,7 @@ public class IndicesRequestCacheTests extends ESTestCase { assertEquals(1, requestCacheStats.stats().getHitCount()); assertEquals(1, requestCacheStats.stats().getMissCount()); assertEquals(0, requestCacheStats.stats().getEvictions()); - assertTrue(entity.loadedFromCache()); + assertTrue(loader.loadedFromCache); assertEquals(0, cache.count()); assertEquals(0, requestCacheStats.stats().getMemorySize().bytesAsInt()); @@ -114,46 +117,50 @@ public class IndicesRequestCacheTests extends ESTestCase { DirectoryReader secondReader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(writer), new ShardId("foo", "bar", 1)); // initial cache - TestEntity entity = new TestEntity(requestCacheStats, reader, indexShard, 0); - BytesReference value = cache.getOrCompute(entity, reader, termQuery.buildAsBytes()); + TestEntity entity = new TestEntity(requestCacheStats, indexShard); + Loader loader = new Loader(reader, 0); + BytesReference value = cache.getOrCompute(entity, loader, reader, termQuery.buildAsBytes()); assertEquals("foo", value.streamInput().readString()); assertEquals(0, requestCacheStats.stats().getHitCount()); assertEquals(1, requestCacheStats.stats().getMissCount()); assertEquals(0, requestCacheStats.stats().getEvictions()); - assertFalse(entity.loadedFromCache()); + assertFalse(loader.loadedFromCache); assertEquals(1, cache.count()); assertTrue(requestCacheStats.stats().getMemorySize().bytesAsInt() > value.length()); final int cacheSize = requestCacheStats.stats().getMemorySize().bytesAsInt(); assertEquals(1, cache.numRegisteredCloseListeners()); // cache the second - TestEntity secondEntity = new TestEntity(requestCacheStats, secondReader, indexShard, 0); - value = cache.getOrCompute(secondEntity, secondReader, termQuery.buildAsBytes()); + TestEntity secondEntity = new TestEntity(requestCacheStats, indexShard); + loader = new Loader(secondReader, 0); + value = cache.getOrCompute(entity, loader, secondReader, termQuery.buildAsBytes()); assertEquals("bar", value.streamInput().readString()); assertEquals(0, requestCacheStats.stats().getHitCount()); assertEquals(2, requestCacheStats.stats().getMissCount()); assertEquals(0, requestCacheStats.stats().getEvictions()); - assertFalse(secondEntity.loadedFromCache()); + assertFalse(loader.loadedFromCache); assertEquals(2, cache.count()); assertTrue(requestCacheStats.stats().getMemorySize().bytesAsInt() > cacheSize + value.length()); assertEquals(2, cache.numRegisteredCloseListeners()); - secondEntity = new TestEntity(requestCacheStats, secondReader, indexShard, 0); - value = cache.getOrCompute(secondEntity, secondReader, termQuery.buildAsBytes()); + secondEntity = new TestEntity(requestCacheStats, indexShard); + loader = new Loader(secondReader, 0); + value = cache.getOrCompute(secondEntity, loader, secondReader, termQuery.buildAsBytes()); assertEquals("bar", value.streamInput().readString()); assertEquals(1, requestCacheStats.stats().getHitCount()); assertEquals(2, requestCacheStats.stats().getMissCount()); assertEquals(0, requestCacheStats.stats().getEvictions()); - assertTrue(secondEntity.loadedFromCache()); + assertTrue(loader.loadedFromCache); assertEquals(2, cache.count()); - entity = new TestEntity(requestCacheStats, reader, indexShard, 0); - value = cache.getOrCompute(entity, reader, termQuery.buildAsBytes()); + entity = new TestEntity(requestCacheStats, indexShard); + loader = new Loader(reader, 0); + value = cache.getOrCompute(entity, loader, reader, termQuery.buildAsBytes()); assertEquals("foo", value.streamInput().readString()); assertEquals(2, requestCacheStats.stats().getHitCount()); assertEquals(2, requestCacheStats.stats().getMissCount()); assertEquals(0, requestCacheStats.stats().getEvictions()); - assertTrue(entity.loadedFromCache()); + assertTrue(loader.loadedFromCache); assertEquals(2, cache.count()); // Closing the cache doesn't change returned entities @@ -161,8 +168,8 @@ public class IndicesRequestCacheTests extends ESTestCase { cache.cleanCache(); assertEquals(2, requestCacheStats.stats().getMissCount()); assertEquals(0, requestCacheStats.stats().getEvictions()); - assertTrue(entity.loadedFromCache()); - assertTrue(secondEntity.loadedFromCache()); + assertTrue(loader.loadedFromCache); + assertTrue(loader.loadedFromCache); assertEquals(1, cache.count()); assertEquals(cacheSize, requestCacheStats.stats().getMemorySize().bytesAsInt()); assertEquals(1, cache.numRegisteredCloseListeners()); @@ -178,8 +185,8 @@ public class IndicesRequestCacheTests extends ESTestCase { cache.cleanCache(); assertEquals(2, requestCacheStats.stats().getMissCount()); assertEquals(0, requestCacheStats.stats().getEvictions()); - assertTrue(entity.loadedFromCache()); - assertTrue(secondEntity.loadedFromCache()); + assertTrue(loader.loadedFromCache); + assertTrue(loader.loadedFromCache); assertEquals(0, cache.count()); assertEquals(0, requestCacheStats.stats().getMemorySize().bytesAsInt()); @@ -200,16 +207,18 @@ public class IndicesRequestCacheTests extends ESTestCase { DirectoryReader reader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(writer), new ShardId("foo", "bar", 1)); TermQueryBuilder termQuery = new TermQueryBuilder("id", "0"); - TestEntity entity = new TestEntity(requestCacheStats, reader, indexShard, 0); + TestEntity entity = new TestEntity(requestCacheStats, indexShard); + Loader loader = new Loader(reader, 0); writer.updateDocument(new Term("id", "0"), newDoc(0, "bar")); DirectoryReader secondReader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(writer), new ShardId("foo", "bar", 1)); - TestEntity secondEntity = new TestEntity(requestCacheStats, secondReader, indexShard, 0); - - BytesReference value1 = cache.getOrCompute(entity, reader, termQuery.buildAsBytes()); + TestEntity secondEntity = new TestEntity(requestCacheStats, indexShard); + Loader secondLoader = new Loader(secondReader, 0); + + BytesReference value1 = cache.getOrCompute(entity, loader, reader, termQuery.buildAsBytes()); assertEquals("foo", value1.streamInput().readString()); - BytesReference value2 = cache.getOrCompute(secondEntity, secondReader, termQuery.buildAsBytes()); + BytesReference value2 = cache.getOrCompute(secondEntity, secondLoader, secondReader, termQuery.buildAsBytes()); assertEquals("bar", value2.streamInput().readString()); size = requestCacheStats.stats().getMemorySize(); IOUtils.close(reader, secondReader, writer, dir, cache); @@ -226,24 +235,27 @@ public class IndicesRequestCacheTests extends ESTestCase { DirectoryReader reader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(writer), new ShardId("foo", "bar", 1)); TermQueryBuilder termQuery = new TermQueryBuilder("id", "0"); - TestEntity entity = new TestEntity(requestCacheStats, reader, indexShard, 0); + TestEntity entity = new TestEntity(requestCacheStats, indexShard); + Loader loader = new Loader(reader, 0); writer.updateDocument(new Term("id", "0"), newDoc(0, "bar")); DirectoryReader secondReader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(writer), new ShardId("foo", "bar", 1)); - TestEntity secondEntity = new TestEntity(requestCacheStats, secondReader, indexShard, 0); + TestEntity secondEntity = new TestEntity(requestCacheStats, indexShard); + Loader secondLoader = new Loader(secondReader, 0); writer.updateDocument(new Term("id", "0"), newDoc(0, "baz")); DirectoryReader thirdReader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(writer), new ShardId("foo", "bar", 1)); - TestEntity thirddEntity = new TestEntity(requestCacheStats, thirdReader, indexShard, 0); + TestEntity thirddEntity = new TestEntity(requestCacheStats, indexShard); + Loader thirdLoader = new Loader(thirdReader, 0); - BytesReference value1 = cache.getOrCompute(entity, reader, termQuery.buildAsBytes()); + BytesReference value1 = cache.getOrCompute(entity, loader, reader, termQuery.buildAsBytes()); assertEquals("foo", value1.streamInput().readString()); - BytesReference value2 = cache.getOrCompute(secondEntity, secondReader, termQuery.buildAsBytes()); + BytesReference value2 = cache.getOrCompute(secondEntity, secondLoader, secondReader, termQuery.buildAsBytes()); assertEquals("bar", value2.streamInput().readString()); logger.info("Memory size: {}", requestCacheStats.stats().getMemorySize()); - BytesReference value3 = cache.getOrCompute(thirddEntity, thirdReader, termQuery.buildAsBytes()); + BytesReference value3 = cache.getOrCompute(thirddEntity, thirdLoader, thirdReader, termQuery.buildAsBytes()); assertEquals("baz", value3.streamInput().readString()); assertEquals(2, cache.count()); assertEquals(1, requestCacheStats.stats().getEvictions()); @@ -262,25 +274,28 @@ public class IndicesRequestCacheTests extends ESTestCase { DirectoryReader reader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(writer), new ShardId("foo", "bar", 1)); TermQueryBuilder termQuery = new TermQueryBuilder("id", "0"); - TestEntity entity = new TestEntity(requestCacheStats, reader, indexShard, 0); + TestEntity entity = new TestEntity(requestCacheStats, indexShard); + Loader loader = new Loader(reader, 0); writer.updateDocument(new Term("id", "0"), newDoc(0, "bar")); DirectoryReader secondReader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(writer), new ShardId("foo", "bar", 1)); - TestEntity secondEntity = new TestEntity(requestCacheStats, secondReader, indexShard, 0); + TestEntity secondEntity = new TestEntity(requestCacheStats, indexShard); + Loader secondLoader = new Loader(secondReader, 0); writer.updateDocument(new Term("id", "0"), newDoc(0, "baz")); DirectoryReader thirdReader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(writer), new ShardId("foo", "bar", 1)); AtomicBoolean differentIdentity = new AtomicBoolean(true); - TestEntity thirddEntity = new TestEntity(requestCacheStats, thirdReader, differentIdentity, 0); + TestEntity thirddEntity = new TestEntity(requestCacheStats, differentIdentity); + Loader thirdLoader = new Loader(thirdReader, 0); - BytesReference value1 = cache.getOrCompute(entity, reader, termQuery.buildAsBytes()); + BytesReference value1 = cache.getOrCompute(entity, loader, reader, termQuery.buildAsBytes()); assertEquals("foo", value1.streamInput().readString()); - BytesReference value2 = cache.getOrCompute(secondEntity, secondReader, termQuery.buildAsBytes()); + BytesReference value2 = cache.getOrCompute(secondEntity, secondLoader, secondReader, termQuery.buildAsBytes()); assertEquals("bar", value2.streamInput().readString()); logger.info("Memory size: {}", requestCacheStats.stats().getMemorySize()); - BytesReference value3 = cache.getOrCompute(thirddEntity, thirdReader, termQuery.buildAsBytes()); + BytesReference value3 = cache.getOrCompute(thirddEntity, thirdLoader, thirdReader, termQuery.buildAsBytes()); assertEquals("baz", value3.streamInput().readString()); assertEquals(3, cache.count()); final long hitCount = requestCacheStats.stats().getHitCount(); @@ -289,7 +304,7 @@ public class IndicesRequestCacheTests extends ESTestCase { cache.cleanCache(); assertEquals(1, cache.count()); // third has not been validated since it's a different identity - value3 = cache.getOrCompute(thirddEntity, thirdReader, termQuery.buildAsBytes()); + value3 = cache.getOrCompute(thirddEntity, thirdLoader, thirdReader, termQuery.buildAsBytes()); assertEquals(hitCount + 1, requestCacheStats.stats().getHitCount()); assertEquals("baz", value3.streamInput().readString()); @@ -303,20 +318,39 @@ public class IndicesRequestCacheTests extends ESTestCase { StringField.TYPE_STORED)); } + private static class Loader implements Supplier { + + private final DirectoryReader reader; + private final int id; + public boolean loadedFromCache = true; + + public Loader(DirectoryReader reader, int id) { + super(); + this.reader = reader; + this.id = id; + } + + @Override + public BytesReference get() { + try (BytesStreamOutput out = new BytesStreamOutput()) { + IndexSearcher searcher = new IndexSearcher(reader); + TopDocs topDocs = searcher.search(new TermQuery(new Term("id", Integer.toString(id))), 1); + assertEquals(1, topDocs.totalHits); + Document document = reader.document(topDocs.scoreDocs[0].doc); + out.writeString(document.get("value")); + loadedFromCache = false; + return out.bytes(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + } + private class TestEntity extends AbstractIndexShardCacheEntity { private final AtomicBoolean standInForIndexShard; private final ShardRequestCache shardRequestCache; - private TestEntity(ShardRequestCache shardRequestCache, DirectoryReader reader, AtomicBoolean standInForIndexShard, int id) { - super(new Loader() { - @Override - public void load(StreamOutput out) throws IOException { - IndexSearcher searcher = new IndexSearcher(reader); - TopDocs topDocs = searcher.search(new TermQuery(new Term("id", Integer.toString(id))), 1); - assertEquals(1, topDocs.totalHits); - Document document = reader.document(topDocs.scoreDocs[0].doc); - out.writeString(document.get("value")); - } - }); + private TestEntity(ShardRequestCache shardRequestCache, AtomicBoolean standInForIndexShard) { this.standInForIndexShard = standInForIndexShard; this.shardRequestCache = shardRequestCache; } @@ -335,5 +369,10 @@ public class IndicesRequestCacheTests extends ESTestCase { public Object getCacheIdentity() { return standInForIndexShard; } + + @Override + public long ramBytesUsed() { + return 42; + } } } From cf667bcbd6557c484762e0e99da833f17074ddb1 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Thu, 3 Nov 2016 08:58:53 +0100 Subject: [PATCH 112/132] Create the QueryShardContext lazily in DocumentMapperParser. (#21287) This would allow MapperService to still be usable in contexts when a QueryShardContext cannot be obtained, for instance in the case that a MapperService needs to be created only to merge mappings. --- .../index/mapper/DocumentMapperParser.java | 2 +- .../org/elasticsearch/index/mapper/Mapper.java | 16 +++++++++++----- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapperParser.java b/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapperParser.java index 2cdeed9f040..50c7d98be92 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapperParser.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapperParser.java @@ -69,7 +69,7 @@ public class DocumentMapperParser { } public Mapper.TypeParser.ParserContext parserContext(String type) { - return new Mapper.TypeParser.ParserContext(type, indexAnalyzers, similarityService::getSimilarity, mapperService, typeParsers::get, indexVersionCreated, parseFieldMatcher, queryShardContextSupplier.get()); + return new Mapper.TypeParser.ParserContext(type, indexAnalyzers, similarityService::getSimilarity, mapperService, typeParsers::get, indexVersionCreated, parseFieldMatcher, queryShardContextSupplier); } public DocumentMapper parse(@Nullable String type, CompressedXContent source) throws MapperParsingException { diff --git a/core/src/main/java/org/elasticsearch/index/mapper/Mapper.java b/core/src/main/java/org/elasticsearch/index/mapper/Mapper.java index 1c54c2136c9..83a20e03ffe 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/Mapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/Mapper.java @@ -20,7 +20,6 @@ package org.elasticsearch.index.mapper; import org.elasticsearch.Version; -import org.elasticsearch.common.Nullable; import org.elasticsearch.common.ParseFieldMatcher; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.ToXContent; @@ -31,6 +30,7 @@ import org.elasticsearch.index.similarity.SimilarityProvider; import java.util.Map; import java.util.Objects; import java.util.function.Function; +import java.util.function.Supplier; public abstract class Mapper implements ToXContent, Iterable { @@ -93,11 +93,13 @@ public abstract class Mapper implements ToXContent, Iterable { private final ParseFieldMatcher parseFieldMatcher; - private final QueryShardContext queryShardContext; + private final Supplier queryShardContextSupplier; + private QueryShardContext queryShardContext; public ParserContext(String type, IndexAnalyzers indexAnalyzers, Function similarityLookupService, MapperService mapperService, Function typeParsers, - Version indexVersionCreated, ParseFieldMatcher parseFieldMatcher, QueryShardContext queryShardContext) { + Version indexVersionCreated, ParseFieldMatcher parseFieldMatcher, + Supplier queryShardContextSupplier) { this.type = type; this.indexAnalyzers = indexAnalyzers; this.similarityLookupService = similarityLookupService; @@ -105,7 +107,7 @@ public abstract class Mapper implements ToXContent, Iterable { this.typeParsers = typeParsers; this.indexVersionCreated = indexVersionCreated; this.parseFieldMatcher = parseFieldMatcher; - this.queryShardContext = queryShardContext; + this.queryShardContextSupplier = queryShardContextSupplier; } public String type() { @@ -137,6 +139,10 @@ public abstract class Mapper implements ToXContent, Iterable { } public QueryShardContext queryShardContext() { + // No need for synchronization, this class must be used in a single thread + if (queryShardContext == null) { + queryShardContext = queryShardContextSupplier.get(); + } return queryShardContext; } @@ -155,7 +161,7 @@ public abstract class Mapper implements ToXContent, Iterable { static class MultiFieldParserContext extends ParserContext { MultiFieldParserContext(ParserContext in) { - super(in.type(), in.indexAnalyzers, in.similarityLookupService(), in.mapperService(), in.typeParsers(), in.indexVersionCreated(), in.parseFieldMatcher(), in.queryShardContext()); + super(in.type(), in.indexAnalyzers, in.similarityLookupService(), in.mapperService(), in.typeParsers(), in.indexVersionCreated(), in.parseFieldMatcher(), in::queryShardContext); } } From be1772b70daab596bef4975dbc3926a0b1774e66 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Thu, 3 Nov 2016 09:01:53 +0100 Subject: [PATCH 113/132] pending states assertion should dump states This was removed in a cleanup assuming that Hamcrest will dump the array content. Sadly it only dumps the size. --- .../java/org/elasticsearch/test/ESIntegTestCase.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index 3dcbab03352..8def9deda3c 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -164,6 +164,8 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.BooleanSupplier; import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.elasticsearch.client.Requests.syncedFlushRequest; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS; @@ -535,8 +537,12 @@ public abstract class ESIntegTestCase extends ESTestCase { for (Discovery discovery : internalCluster().getInstances(Discovery.class)) { if (discovery instanceof ZenDiscovery) { final ZenDiscovery zenDiscovery = (ZenDiscovery) discovery; - assertBusy(() -> assertThat(zenDiscovery.localNode().getName() + " still having pending states", - zenDiscovery.pendingClusterStates(), emptyArray())); + assertBusy(() -> { + final ClusterState[] states = zenDiscovery.pendingClusterStates(); + assertThat(zenDiscovery.localNode().getName() + " still having pending states:\n" + + Stream.of(states).map(ClusterState::toString).collect(Collectors.joining("\n")), + states, emptyArray()); + }); } } } From d77d4fa63a92c2c3c9539f8f399c9e12fcb7102f Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Thu, 3 Nov 2016 10:31:35 +0100 Subject: [PATCH 114/132] Consume `full_id` request parameter early (#21270) Since we now validate all consumed request parameter, users can't specify `_cat/nodes?full_id=true|false` anymore since this parameter is consumed late. This commit adds a test for this parameter and consumes it before request is processed. Closes #21266 --- .../resources/checkstyle_suppressions.xml | 1 - .../rest/action/cat/RestNodesAction.java | 64 ++++++++++++------- .../rest-api-spec/api/cat.nodes.json | 4 ++ .../test/cat.nodes/10_basic.yaml | 25 ++++++++ 4 files changed, 71 insertions(+), 23 deletions(-) diff --git a/buildSrc/src/main/resources/checkstyle_suppressions.xml b/buildSrc/src/main/resources/checkstyle_suppressions.xml index c461a53fd88..d37e7132d68 100644 --- a/buildSrc/src/main/resources/checkstyle_suppressions.xml +++ b/buildSrc/src/main/resources/checkstyle_suppressions.xml @@ -459,7 +459,6 @@ - diff --git a/core/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java b/core/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java index 87cbe976089..b632448192d 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java @@ -85,7 +85,7 @@ public class RestNodesAction extends AbstractCatAction { clusterStateRequest.clear().nodes(true); clusterStateRequest.local(request.paramAsBoolean("local", clusterStateRequest.local())); clusterStateRequest.masterNodeTimeout(request.paramAsTime("master_timeout", clusterStateRequest.masterNodeTimeout())); - + final boolean fullId = request.paramAsBoolean("full_id", false); return channel -> client.admin().cluster().state(clusterStateRequest, new RestActionListener(channel) { @Override public void processResponse(final ClusterStateResponse clusterStateResponse) { @@ -99,7 +99,8 @@ public class RestNodesAction extends AbstractCatAction { client.admin().cluster().nodesStats(nodesStatsRequest, new RestResponseListener(channel) { @Override public RestResponse buildResponse(NodesStatsResponse nodesStatsResponse) throws Exception { - return RestTable.buildResponse(buildTable(request, clusterStateResponse, nodesInfoResponse, nodesStatsResponse), channel); + return RestTable.buildResponse(buildTable(fullId, request, clusterStateResponse, nodesInfoResponse, + nodesStatsResponse), channel); } }); } @@ -129,7 +130,8 @@ public class RestNodesAction extends AbstractCatAction { table.addCell("ram.percent", "alias:rp,ramPercent;text-align:right;desc:used machine memory ratio"); table.addCell("ram.max", "default:false;alias:rm,ramMax;text-align:right;desc:total machine memory"); table.addCell("file_desc.current", "default:false;alias:fdc,fileDescriptorCurrent;text-align:right;desc:used file descriptors"); - table.addCell("file_desc.percent", "default:false;alias:fdp,fileDescriptorPercent;text-align:right;desc:used file descriptor ratio"); + table.addCell("file_desc.percent", + "default:false;alias:fdp,fileDescriptorPercent;text-align:right;desc:used file descriptor ratio"); table.addCell("file_desc.max", "default:false;alias:fdm,fileDescriptorMax;text-align:right;desc:max file descriptors"); table.addCell("cpu", "alias:cpu;text-align:right;desc:recent cpu usage"); @@ -137,7 +139,8 @@ public class RestNodesAction extends AbstractCatAction { table.addCell("load_5m", "alias:l;text-align:right;desc:5m load avg"); table.addCell("load_15m", "alias:l;text-align:right;desc:15m load avg"); table.addCell("uptime", "default:false;alias:u;text-align:right;desc:node uptime"); - table.addCell("node.role", "alias:r,role,nodeRole;desc:m:master eligible node, d:data node, i:ingest node, -:coordinating node only"); + table.addCell("node.role", + "alias:r,role,nodeRole;desc:m:master eligible node, d:data node, i:ingest node, -:coordinating node only"); table.addCell("master", "alias:m;desc:*:current master"); table.addCell("name", "alias:n;desc:node name"); @@ -150,9 +153,12 @@ public class RestNodesAction extends AbstractCatAction { table.addCell("query_cache.evictions", "alias:qce,queryCacheEvictions;default:false;text-align:right;desc:query cache evictions"); table.addCell("request_cache.memory_size", "alias:rcm,requestCacheMemory;default:false;text-align:right;desc:used request cache"); - table.addCell("request_cache.evictions", "alias:rce,requestCacheEvictions;default:false;text-align:right;desc:request cache evictions"); - table.addCell("request_cache.hit_count", "alias:rchc,requestCacheHitCount;default:false;text-align:right;desc:request cache hit counts"); - table.addCell("request_cache.miss_count", "alias:rcmc,requestCacheMissCount;default:false;text-align:right;desc:request cache miss counts"); + table.addCell("request_cache.evictions", + "alias:rce,requestCacheEvictions;default:false;text-align:right;desc:request cache evictions"); + table.addCell("request_cache.hit_count", + "alias:rchc,requestCacheHitCount;default:false;text-align:right;desc:request cache hit counts"); + table.addCell("request_cache.miss_count", + "alias:rcmc,requestCacheMissCount;default:false;text-align:right;desc:request cache miss counts"); table.addCell("flush.total", "alias:ft,flushTotal;default:false;text-align:right;desc:number of flushes"); table.addCell("flush.total_time", "alias:ftt,flushTotalTime;default:false;text-align:right;desc:time spent in flush"); @@ -165,16 +171,20 @@ 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("indexing.delete_current", "alias:idc,indexingDeleteCurrent;default:false;text-align:right;desc:number of current deletions"); + 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"); - table.addCell("indexing.index_current", "alias:iic,indexingIndexCurrent;default:false;text-align:right;desc:number of current indexing ops"); + table.addCell("indexing.index_current", + "alias:iic,indexingIndexCurrent;default:false;text-align:right;desc:number of current indexing ops"); table.addCell("indexing.index_time", "alias:iiti,indexingIndexTime;default:false;text-align:right;desc:time spent in indexing"); table.addCell("indexing.index_total", "alias:iito,indexingIndexTotal;default:false;text-align:right;desc:number of indexing ops"); - table.addCell("indexing.index_failed", "alias:iif,indexingIndexFailed;default:false;text-align:right;desc:number of failed indexing ops"); + table.addCell("indexing.index_failed", + "alias:iif,indexingIndexFailed;default:false;text-align:right;desc:number of failed indexing ops"); table.addCell("merges.current", "alias:mc,mergesCurrent;default:false;text-align:right;desc:number of current merges"); - table.addCell("merges.current_docs", "alias:mcd,mergesCurrentDocs;default:false;text-align:right;desc:number of current merging docs"); + table.addCell("merges.current_docs", + "alias:mcd,mergesCurrentDocs;default:false;text-align:right;desc:number of current merging docs"); table.addCell("merges.current_size", "alias:mcs,mergesCurrentSize;default:false;text-align:right;desc:size of current merges"); table.addCell("merges.total", "alias:mt,mergesTotal;default:false;text-align:right;desc:number of completed merge ops"); table.addCell("merges.total_docs", "alias:mtd,mergesTotalDocs;default:false;text-align:right;desc:docs merged"); @@ -185,7 +195,8 @@ public class RestNodesAction extends AbstractCatAction { table.addCell("refresh.time", "alias:rti,refreshTime;default:false;text-align:right;desc:time spent in refreshes"); table.addCell("script.compilations", "alias:scrcc,scriptCompilations;default:false;text-align:right;desc:script compilations"); - table.addCell("script.cache_evictions", "alias:scrce,scriptCacheEvictions;default:false;text-align:right;desc:script cache evictions"); + table.addCell("script.cache_evictions", + "alias:scrce,scriptCacheEvictions;default:false;text-align:right;desc:script cache evictions"); table.addCell("search.fetch_current", "alias:sfc,searchFetchCurrent;default:false;text-align:right;desc:current fetch phase ops"); table.addCell("search.fetch_time", "alias:sfti,searchFetchTime;default:false;text-align:right;desc:time spent in fetch phase"); @@ -195,14 +206,19 @@ public class RestNodesAction extends AbstractCatAction { table.addCell("search.query_time", "alias:sqti,searchQueryTime;default:false;text-align:right;desc:time spent in query phase"); table.addCell("search.query_total", "alias:sqto,searchQueryTotal;default:false;text-align:right;desc:total query phase ops"); table.addCell("search.scroll_current", "alias:scc,searchScrollCurrent;default:false;text-align:right;desc:open scroll contexts"); - table.addCell("search.scroll_time", "alias:scti,searchScrollTime;default:false;text-align:right;desc:time scroll contexts held open"); + table.addCell("search.scroll_time", + "alias:scti,searchScrollTime;default:false;text-align:right;desc:time scroll contexts held open"); table.addCell("search.scroll_total", "alias:scto,searchScrollTotal;default:false;text-align:right;desc:completed scroll contexts"); table.addCell("segments.count", "alias:sc,segmentsCount;default:false;text-align:right;desc:number of segments"); table.addCell("segments.memory", "alias:sm,segmentsMemory;default:false;text-align:right;desc:memory used by segments"); - table.addCell("segments.index_writer_memory", "alias:siwm,segmentsIndexWriterMemory;default:false;text-align:right;desc:memory used by index writer"); - table.addCell("segments.version_map_memory", "alias:svmm,segmentsVersionMapMemory;default:false;text-align:right;desc:memory used by version map"); - table.addCell("segments.fixed_bitset_memory", "alias:sfbm,fixedBitsetMemory;default:false;text-align:right;desc:memory used by fixed bit sets for nested object field types and type filters for types referred in _parent fields"); + table.addCell("segments.index_writer_memory", + "alias:siwm,segmentsIndexWriterMemory;default:false;text-align:right;desc:memory used by index writer"); + table.addCell("segments.version_map_memory", + "alias:svmm,segmentsVersionMapMemory;default:false;text-align:right;desc:memory used by version map"); + table.addCell("segments.fixed_bitset_memory", + "alias:sfbm,fixedBitsetMemory;default:false;text-align:right;desc:memory used by fixed bit sets for nested object field types" + + " and type filters for types referred in _parent fields"); table.addCell("suggest.current", "alias:suc,suggestCurrent;default:false;text-align:right;desc:number of current suggest ops"); table.addCell("suggest.time", "alias:suti,suggestTime;default:false;text-align:right;desc:time spend in suggest"); @@ -212,8 +228,8 @@ public class RestNodesAction extends AbstractCatAction { return table; } - private Table buildTable(RestRequest req, ClusterStateResponse state, NodesInfoResponse nodesInfo, NodesStatsResponse nodesStats) { - boolean fullId = req.paramAsBoolean("full_id", false); + private Table buildTable(boolean fullId, RestRequest req, ClusterStateResponse state, NodesInfoResponse nodesInfo, + NodesStatsResponse nodesStats) { DiscoveryNodes nodes = state.getState().nodes(); String masterId = nodes.getMasterNodeId(); @@ -255,14 +271,18 @@ public class RestNodesAction extends AbstractCatAction { table.addCell(osStats == null ? null : osStats.getMem() == null ? null : osStats.getMem().getUsedPercent()); table.addCell(osStats == null ? null : osStats.getMem() == null ? null : osStats.getMem().getTotal()); table.addCell(processStats == null ? null : processStats.getOpenFileDescriptors()); - table.addCell(processStats == null ? null : calculatePercentage(processStats.getOpenFileDescriptors(), processStats.getMaxFileDescriptors())); + table.addCell(processStats == null ? null : calculatePercentage(processStats.getOpenFileDescriptors(), + processStats.getMaxFileDescriptors())); table.addCell(processStats == null ? null : processStats.getMaxFileDescriptors()); table.addCell(osStats == null ? null : Short.toString(osStats.getCpu().getPercent())); boolean hasLoadAverage = osStats != null && osStats.getCpu().getLoadAverage() != null; - table.addCell(!hasLoadAverage || osStats.getCpu().getLoadAverage()[0] == -1 ? null : String.format(Locale.ROOT, "%.2f", osStats.getCpu().getLoadAverage()[0])); - table.addCell(!hasLoadAverage || osStats.getCpu().getLoadAverage()[1] == -1 ? null : String.format(Locale.ROOT, "%.2f", osStats.getCpu().getLoadAverage()[1])); - table.addCell(!hasLoadAverage || osStats.getCpu().getLoadAverage()[2] == -1 ? null : String.format(Locale.ROOT, "%.2f", osStats.getCpu().getLoadAverage()[2])); + table.addCell(!hasLoadAverage || osStats.getCpu().getLoadAverage()[0] == -1 ? null : + String.format(Locale.ROOT, "%.2f", osStats.getCpu().getLoadAverage()[0])); + table.addCell(!hasLoadAverage || osStats.getCpu().getLoadAverage()[1] == -1 ? null : + String.format(Locale.ROOT, "%.2f", osStats.getCpu().getLoadAverage()[1])); + table.addCell(!hasLoadAverage || osStats.getCpu().getLoadAverage()[2] == -1 ? null : + String.format(Locale.ROOT, "%.2f", osStats.getCpu().getLoadAverage()[2])); table.addCell(jvmStats == null ? null : jvmStats.getUptime()); final String roles; diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.nodes.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.nodes.json index c4d0dcd5f49..1b3c1266a63 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.nodes.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.nodes.json @@ -12,6 +12,10 @@ "type" : "string", "description" : "a short version of the Accept header, e.g. json, yaml" }, + "full_id": { + "type" : "boolean", + "description" : "Return the full node ID instead of the shortened version (default: false)" + }, "local": { "type" : "boolean", "description" : "Return local information, do not retrieve the state from master node (default: false)" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.nodes/10_basic.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.nodes/10_basic.yaml index 391a7c1e6d1..9f4de56c863 100755 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.nodes/10_basic.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.nodes/10_basic.yaml @@ -57,3 +57,28 @@ - match: $body: | /^ http \n ((\d{1,3}\.){3}\d{1,3}:\d{1,5}\n)+ $/ + +--- +"Test cat nodes output with full_id set": + - skip: + version: " - 5.0.0" + reason: The full_id setting was rejected in 5.0.0 see #21266 + + + - do: + cat.nodes: + h: id + # check for a 4 char non-whitespace character string + - match: + $body: | + /^(\S{4}\n)+$/ + + - do: + cat.nodes: + h: id + full_id: true + # check for a 5+ char non-whitespace character string + - match: + $body: | + /^(\S{5,}\n)+$/ + From 6902eafbfece0fa25ca33a092a7bc4260c2ec955 Mon Sep 17 00:00:00 2001 From: Clinton Gormley Date: Thu, 3 Nov 2016 10:48:05 +0100 Subject: [PATCH 115/132] Specify date math with URI encoding in rollover API docs Closes #21181 --- docs/reference/indices/rollover-index.asciidoc | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/reference/indices/rollover-index.asciidoc b/docs/reference/indices/rollover-index.asciidoc index 04e79ca5a08..307aa1fd501 100644 --- a/docs/reference/indices/rollover-index.asciidoc +++ b/docs/reference/indices/rollover-index.asciidoc @@ -95,7 +95,8 @@ over. For instance: [source,js] -------------------------------------------------- -PUT / <1> +# PUT / with URI encoding: +PUT /%3Clogs-%7Bnow%2Fd%7D-1%3E <1> { "aliases": { "logs_write": {} From 1fb87233239dc52ed145d5e3190e793a8b3ad7b7 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Thu, 3 Nov 2016 11:22:18 +0100 Subject: [PATCH 116/132] [TEST] Add tests that combines highlighting and NOW parsing Relates to #21295 --- .../highlight/HighlighterSearchIT.java | 36 +++++++++++++++++++ 1 file changed, 36 insertions(+) diff --git a/core/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlighterSearchIT.java b/core/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlighterSearchIT.java index 14cd65335f4..ac6bc9ab359 100644 --- a/core/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlighterSearchIT.java +++ b/core/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlighterSearchIT.java @@ -24,6 +24,7 @@ import org.elasticsearch.Version; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.SearchType; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.geo.GeoPoint; @@ -40,15 +41,19 @@ import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.functionscore.FunctionScoreQueryBuilder; import org.elasticsearch.index.search.MatchQuery; +import org.elasticsearch.indices.IndicesRequestCache; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder.Field; +import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.InternalSettingsPlugin; import org.hamcrest.Matcher; import org.hamcrest.Matchers; +import org.joda.time.DateTime; +import org.joda.time.chrono.ISOChronology; import java.io.IOException; import java.util.Collection; @@ -84,6 +89,7 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHigh 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.assertSearchResponse; import static org.elasticsearch.test.hamcrest.RegexMatcher.matches; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.containsString; @@ -2948,4 +2954,34 @@ public class HighlighterSearchIT extends ESIntegTestCase { equalTo("The quick brown fox jumps over the lazy dog")); } } + + public void testHighlightQueryRewriteDatesWithNow() throws Exception { + assertAcked(client().admin().indices().prepareCreate("index-1").addMapping("type", "d", "type=date", + "field", "type=text,store=true,term_vector=with_positions_offsets") + .setSettings("index.number_of_replicas", 0, "index.number_of_shards", 2) + .get()); + DateTime now = new DateTime(ISOChronology.getInstanceUTC()); + indexRandom(true, client().prepareIndex("index-1", "type", "1").setSource("d", now, "field", "hello world"), + client().prepareIndex("index-1", "type", "2").setSource("d", now.minusDays(1), "field", "hello"), + client().prepareIndex("index-1", "type", "3").setSource("d", now.minusDays(2), "field", "world")); + ensureSearchable("index-1"); + for (int i = 0; i < 5; i++) { + final SearchResponse r1 = client().prepareSearch("index-1") + .addSort("d", SortOrder.DESC) + .setTrackScores(true) + .highlighter(highlight() + .field("field") + .preTags("") + .postTags("") + ).setQuery(QueryBuilders.boolQuery().must( + QueryBuilders.rangeQuery("d").gte("now-7d/d").lte("now").includeLower(true).includeUpper(true).boost(1.0f)) + .should(QueryBuilders.termQuery("field", "hello"))) + .get(); + + assertSearchResponse(r1); + assertThat(r1.getHits().getTotalHits(), equalTo(3L)); + assertHighlight(r1, 0, "field", 0, 1, + equalTo("hello world")); + } + } } From ef2453360ef2a88b705e169cd1ca4752ce2f05a4 Mon Sep 17 00:00:00 2001 From: Clinton Gormley Date: Thu, 3 Nov 2016 13:48:27 +0100 Subject: [PATCH 117/132] Fixed rollover docs test for date math indices --- docs/reference/indices/rollover-index.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/indices/rollover-index.asciidoc b/docs/reference/indices/rollover-index.asciidoc index 307aa1fd501..d6875909216 100644 --- a/docs/reference/indices/rollover-index.asciidoc +++ b/docs/reference/indices/rollover-index.asciidoc @@ -118,7 +118,7 @@ POST /logs_write/_rollover <2> } -------------------------------------------------- // CONSOLE -// TEST[s/\{now\//{2016.10.31||%2f/] +// TEST[s/now/2016.10.31||/] <1> Creates an index named with today's date (e.g.) `logs-2016.10.31-1` <2> Rolls over to a new index with today's date, e.g. `logs-2016.10.31-000002` if run immediately, or `logs-2016.11.01-000002` if run after 24 hours From 886eba076c1dc748fed77238ae4c67f735635069 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 3 Nov 2016 09:03:42 -0400 Subject: [PATCH 118/132] Remove node.max_local_storage_nodes from setup doc Previously node.max_local_storage_nodes defaulted to fifty, and this permitted users to start multiple instances of Elasticsearch sharing the same data folder. This can be dangerous, and usually it does not make sense to run more than one instance of Elasticsearch on a single server. Because of this, we had a note in the important settings docs advising users to set this setting to one. However, we have since changed the default value of this setting to one so this advise is no longer needed. Relates #21305 --- .../setup/important-settings.asciidoc | 26 ------------------- 1 file changed, 26 deletions(-) diff --git a/docs/reference/setup/important-settings.asciidoc b/docs/reference/setup/important-settings.asciidoc index cd957addef1..0735bfeadec 100644 --- a/docs/reference/setup/important-settings.asciidoc +++ b/docs/reference/setup/important-settings.asciidoc @@ -188,29 +188,3 @@ be thrown which will prevent the node from starting. [float] [[node.max_local_storage_nodes]] -=== `node.max_local_storage_nodes` - -It is possible to start more than one node on the same server from the same -`$ES_HOME`, just by doing the following: - -[source,sh] --------------------------------------------------- -./bin/elasticsearch -d -./bin/elasticsearch -d --------------------------------------------------- - -This works just fine: the data directory structure is designed to let multiple -nodes coexist. However, a single instance of Elasticsearch is able to use all -of the resources of a single server and it seldom makes sense to run multiple -nodes on the same server in production. - -It is, however, possible to start more than one node on the same server by -mistake and to be completely unaware that this problem exists. To prevent more -than one node from sharing the same data directory, it is advisable to add the -following setting: - -[source,yaml] --------------------------------------------------- -node.max_local_storage_nodes: 1 --------------------------------------------------- - From 548c12d60f0f4a33460bbc1beba603bf20fd695e Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 3 Nov 2016 10:14:46 -0400 Subject: [PATCH 119/132] Remove broken docs link The important settings docs previously referred to a section regarding the node.max_local_storage_nodes setting. This section was removed, but the link was not. This commit removes that link. --- docs/reference/setup/important-settings.asciidoc | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/reference/setup/important-settings.asciidoc b/docs/reference/setup/important-settings.asciidoc index 0735bfeadec..80e6c1801e9 100644 --- a/docs/reference/setup/important-settings.asciidoc +++ b/docs/reference/setup/important-settings.asciidoc @@ -12,7 +12,6 @@ configured before going into production. * <> * <> * <> -* <> [float] [[path-settings]] From 7ec51d628d89ce697e3fa293b7b3427b9b638309 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Thu, 3 Nov 2016 16:07:52 +0100 Subject: [PATCH 120/132] Make the default S3 buffer size depend on the available memory. (#21299) Currently the default S3 buffer size is 100MB, which can be a lot for small heaps. This pull request updates the default to be 100MB for heaps that are greater than 2GB and 5% of the heap size otherwise. --- docs/plugins/repository-s3.asciidoc | 3 ++- .../repositories/s3/S3Repository.java | 21 ++++++++++++++++--- .../repositories/s3/S3RepositoryTests.java | 9 ++++++++ 3 files changed, 29 insertions(+), 4 deletions(-) diff --git a/docs/plugins/repository-s3.asciidoc b/docs/plugins/repository-s3.asciidoc index a98c9a525d8..46789cf3f6e 100644 --- a/docs/plugins/repository-s3.asciidoc +++ b/docs/plugins/repository-s3.asciidoc @@ -217,7 +217,8 @@ The following settings are supported: to split the chunk into several parts, each of `buffer_size` length, and to upload each part in its own request. Note that setting a buffer size lower than `5mb` is not allowed since it will prevents the use of the - Multipart API and may result in upload errors. Defaults to `100mb`. + Multipart API and may result in upload errors. Defaults to the minimum + between `100mb` and `5%` of the heap size. `max_retries`:: diff --git a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Repository.java b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Repository.java index 34e4d78f8cf..7310b527158 100644 --- a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Repository.java +++ b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Repository.java @@ -33,6 +33,7 @@ import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.monitor.jvm.JvmInfo; import org.elasticsearch.repositories.RepositoryException; import org.elasticsearch.repositories.blobstore.BlobStoreRepository; @@ -101,14 +102,27 @@ public class S3Repository extends BlobStoreRepository { */ Setting SERVER_SIDE_ENCRYPTION_SETTING = Setting.boolSetting("repositories.s3.server_side_encryption", false, Property.NodeScope); + + /** + * Default is to use 100MB (S3 defaults) for heaps above 2GB and 5% of + * the available memory for smaller heaps. + */ + ByteSizeValue DEFAULT_BUFFER_SIZE = new ByteSizeValue( + Math.max( + ByteSizeUnit.MB.toBytes(5), // minimum value + Math.min( + ByteSizeUnit.MB.toBytes(100), + JvmInfo.jvmInfo().getMem().getHeapMax().getBytes() / 20)), + ByteSizeUnit.BYTES); + /** * repositories.s3.buffer_size: Minimum threshold below which the chunk is uploaded using a single request. Beyond this threshold, * the S3 repository will use the AWS Multipart Upload API to split the chunk into several parts, each of buffer_size length, and * to upload each part in its own request. Note that setting a buffer size lower than 5mb is not allowed since it will prevents the - * use of the Multipart API and may result in upload errors. Defaults to 100m. + * use of the Multipart API and may result in upload errors. Defaults to the minimum between 100MB and 5% of the heap size. */ Setting BUFFER_SIZE_SETTING = - Setting.byteSizeSetting("repositories.s3.buffer_size", new ByteSizeValue(100, ByteSizeUnit.MB), + Setting.byteSizeSetting("repositories.s3.buffer_size", DEFAULT_BUFFER_SIZE, new ByteSizeValue(5, ByteSizeUnit.MB), new ByteSizeValue(5, ByteSizeUnit.TB), Property.NodeScope); /** * repositories.s3.max_retries: Number of retries in case of S3 errors. Defaults to 3. @@ -195,12 +209,13 @@ public class S3Repository extends BlobStoreRepository { * @see Repositories#SERVER_SIDE_ENCRYPTION_SETTING */ Setting SERVER_SIDE_ENCRYPTION_SETTING = Setting.boolSetting("server_side_encryption", false); + /** * buffer_size * @see Repositories#BUFFER_SIZE_SETTING */ Setting BUFFER_SIZE_SETTING = - Setting.byteSizeSetting("buffer_size", new ByteSizeValue(100, ByteSizeUnit.MB), + Setting.byteSizeSetting("buffer_size", Repositories.DEFAULT_BUFFER_SIZE, new ByteSizeValue(5, ByteSizeUnit.MB), new ByteSizeValue(5, ByteSizeUnit.TB)); /** * max_retries diff --git a/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3RepositoryTests.java b/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3RepositoryTests.java index 915183888b6..14595d13448 100644 --- a/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3RepositoryTests.java +++ b/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3RepositoryTests.java @@ -32,6 +32,7 @@ import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.repositories.RepositoryException; import org.elasticsearch.test.ESTestCase; +import org.hamcrest.Matchers; import static org.elasticsearch.repositories.s3.S3Repository.Repositories; import static org.elasticsearch.repositories.s3.S3Repository.Repository; @@ -117,4 +118,12 @@ public class S3RepositoryTests extends ESTestCase { assertEquals("foo/bar/", s3repo.basePath().buildAsString()); // make sure leading `/` is removed and trailing is added } + public void testDefaultBufferSize() { + ByteSizeValue defaultBufferSize = S3Repository.Repository.BUFFER_SIZE_SETTING.get(Settings.EMPTY); + assertThat(defaultBufferSize, Matchers.lessThanOrEqualTo(new ByteSizeValue(100, ByteSizeUnit.MB))); + assertThat(defaultBufferSize, Matchers.greaterThanOrEqualTo(new ByteSizeValue(5, ByteSizeUnit.MB))); + + ByteSizeValue defaultNodeBufferSize = S3Repository.Repositories.BUFFER_SIZE_SETTING.get(Settings.EMPTY); + assertEquals(defaultBufferSize, defaultNodeBufferSize); + } } From dc6ed7b8d4208e49e2fc34e43e8c26a74f249e87 Mon Sep 17 00:00:00 2001 From: Ryan Ernst Date: Thu, 3 Nov 2016 08:20:20 -0700 Subject: [PATCH 121/132] Remove pluggability of ZenPing (#21049) Plugins: Remove pluggability of ZenPing ZenPing is the part of zen discovery which knows how to ping nodes. There is only one alternative implementation, which is just for testing. This change removes the ability to add custom zen pings, and instead hooks in the MockZenPing for tests through an overridden method in MockNode. This also folds in the ZenPingService (which was really just a single method) into ZenDiscovery, and removes the idea of having multiple ZenPing instances. Finally, this was the last usage of the ExtensionPoint classes, so that is also removed here. --- .../component/AbstractLifecycleComponent.java | 11 +- .../common/util/ExtensionPoint.java | 250 ------------------ .../discovery/DiscoveryModule.java | 74 +++--- .../discovery/zen/FaultDetection.java | 5 +- .../discovery/zen/UnicastZenPing.java | 73 ++--- .../discovery/zen/ZenDiscovery.java | 48 ++-- .../elasticsearch/discovery/zen/ZenPing.java | 22 +- .../discovery/zen/ZenPingService.java | 105 -------- .../java/org/elasticsearch/node/Node.java | 23 +- .../elasticsearch/tribe/TribeClientNode.java | 36 --- .../org/elasticsearch/tribe/TribeService.java | 4 +- .../common/util/ExtensionPointTests.java | 63 ----- .../discovery/DiscoveryModuleTests.java | 5 +- .../DiscoveryWithServiceDisruptionsIT.java | 24 +- .../discovery/zen/UnicastZenPingTests.java | 35 +-- .../discovery/zen/ZenDiscoveryUnitTests.java | 4 +- .../java/org/elasticsearch/tribe/TribeIT.java | 24 +- docs/plugins/discovery-file.asciidoc | 2 +- .../elasticsearch/tribe/TribeUnitTests.java | 4 +- .../java/org/elasticsearch/node/MockNode.java | 18 ++ .../test/discovery/MockZenPing.java | 32 +-- 21 files changed, 210 insertions(+), 652 deletions(-) delete mode 100644 core/src/main/java/org/elasticsearch/common/util/ExtensionPoint.java delete mode 100644 core/src/main/java/org/elasticsearch/discovery/zen/ZenPingService.java delete mode 100644 core/src/main/java/org/elasticsearch/tribe/TribeClientNode.java delete mode 100644 core/src/test/java/org/elasticsearch/common/util/ExtensionPointTests.java diff --git a/core/src/main/java/org/elasticsearch/common/component/AbstractLifecycleComponent.java b/core/src/main/java/org/elasticsearch/common/component/AbstractLifecycleComponent.java index e2868b23e89..2ed43ccaa24 100644 --- a/core/src/main/java/org/elasticsearch/common/component/AbstractLifecycleComponent.java +++ b/core/src/main/java/org/elasticsearch/common/component/AbstractLifecycleComponent.java @@ -21,6 +21,7 @@ package org.elasticsearch.common.component; import org.elasticsearch.common.settings.Settings; +import java.io.IOException; import java.util.List; import java.util.concurrent.CopyOnWriteArrayList; @@ -101,11 +102,17 @@ public abstract class AbstractLifecycleComponent extends AbstractComponent imple listener.beforeClose(); } lifecycle.moveToClosed(); - doClose(); + try { + doClose(); + } catch (IOException e) { + // TODO: we need to separate out closing (ie shutting down) services, vs releasing runtime transient + // structures. Shutting down services should use IOUtils.close + logger.warn("failed to close " + getClass().getName(), e); + } for (LifecycleListener listener : listeners) { listener.afterClose(); } } - protected abstract void doClose(); + protected abstract void doClose() throws IOException; } diff --git a/core/src/main/java/org/elasticsearch/common/util/ExtensionPoint.java b/core/src/main/java/org/elasticsearch/common/util/ExtensionPoint.java deleted file mode 100644 index a5dac12fab7..00000000000 --- a/core/src/main/java/org/elasticsearch/common/util/ExtensionPoint.java +++ /dev/null @@ -1,250 +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.util; - -import org.elasticsearch.common.inject.Binder; -import org.elasticsearch.common.inject.multibindings.MapBinder; -import org.elasticsearch.common.inject.multibindings.Multibinder; -import org.elasticsearch.common.settings.Settings; - -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; - -/** - * This class defines an official elasticsearch extension point. It registers - * all extensions by a single name and ensures that extensions are not registered - * more than once. - */ -public abstract class ExtensionPoint { - protected final String name; - protected final Class[] singletons; - - /** - * Creates a new extension point - * - * @param name the human readable underscore case name of the extension point. This is used in error messages etc. - * @param singletons a list of singletons to bind with this extension point - these are bound in {@link #bind(Binder)} - */ - public ExtensionPoint(String name, Class... singletons) { - this.name = name; - this.singletons = singletons; - } - - /** - * Binds the extension as well as the singletons to the given guice binder. - * - * @param binder the binder to use - */ - public final void bind(Binder binder) { - for (Class c : singletons) { - binder.bind(c).asEagerSingleton(); - } - bindExtensions(binder); - } - - /** - * Subclasses can bind their type, map or set extensions here. - */ - protected abstract void bindExtensions(Binder binder); - - /** - * A map based extension point which allows to register keyed implementations ie. parsers or some kind of strategies. - */ - public static class ClassMap extends ExtensionPoint { - protected final Class extensionClass; - protected final Map> extensions = new HashMap<>(); - private final Set reservedKeys; - - /** - * Creates a new {@link ClassMap} - * - * @param name the human readable underscore case name of the extension point. This is used in error messages etc. - * @param extensionClass the base class that should be extended - * @param singletons a list of singletons to bind with this extension point - these are bound in {@link #bind(Binder)} - * @param reservedKeys a set of reserved keys by internal implementations - */ - public ClassMap(String name, Class extensionClass, Set reservedKeys, Class... singletons) { - super(name, singletons); - this.extensionClass = extensionClass; - this.reservedKeys = reservedKeys; - } - - /** - * Returns the extension for the given key or null - */ - public Class getExtension(String type) { - return extensions.get(type); - } - - /** - * Registers an extension class for a given key. This method will thr - * - * @param key the extensions key - * @param extension the extension - * @throws IllegalArgumentException iff the key is already registered or if the key is a reserved key for an internal implementation - */ - public final void registerExtension(String key, Class extension) { - if (extensions.containsKey(key) || reservedKeys.contains(key)) { - throw new IllegalArgumentException("Can't register the same [" + this.name + "] more than once for [" + key + "]"); - } - extensions.put(key, extension); - } - - @Override - protected final void bindExtensions(Binder binder) { - MapBinder parserMapBinder = MapBinder.newMapBinder(binder, String.class, extensionClass); - for (Map.Entry> clazz : extensions.entrySet()) { - parserMapBinder.addBinding(clazz.getKey()).to(clazz.getValue()); - } - } - } - - /** - * A Type extension point which basically allows to registered keyed extensions like {@link ClassMap} - * but doesn't instantiate and bind all the registered key value pairs but instead replace a singleton based on a given setting via {@link #bindType(Binder, Settings, String, String)} - * Note: {@link #bind(Binder)} is not supported by this class - */ - public static final class SelectedType extends ClassMap { - - public SelectedType(String name, Class extensionClass) { - super(name, extensionClass, Collections.emptySet()); - } - - /** - * Binds the extension class to the class that is registered for the give configured for the settings key in - * the settings object. - * - * @param binder the binder to use - * @param settings the settings to look up the key to find the implementation to bind - * @param settingsKey the key to use with the settings - * @param defaultValue the default value if the settings do not contain the key, or null if there is no default - * @return the actual bound type key - */ - public String bindType(Binder binder, Settings settings, String settingsKey, String defaultValue) { - final String type = settings.get(settingsKey, defaultValue); - if (type == null) { - throw new IllegalArgumentException("Missing setting [" + settingsKey + "]"); - } - final Class instance = getExtension(type); - if (instance == null) { - throw new IllegalArgumentException("Unknown [" + this.name + "] type [" + type + "] possible values: " - + extensions.keySet()); - } - if (extensionClass == instance) { - binder.bind(extensionClass).asEagerSingleton(); - } else { - binder.bind(extensionClass).to(instance).asEagerSingleton(); - } - return type; - } - - } - - /** - * A set based extension point which allows to register extended classes that might be used to chain additional functionality etc. - */ - public static final class ClassSet extends ExtensionPoint { - protected final Class extensionClass; - private final Set> extensions = new HashSet<>(); - - /** - * Creates a new {@link ClassSet} - * - * @param name the human readable underscore case name of the extension point. This is used in error messages etc. - * @param extensionClass the base class that should be extended - * @param singletons a list of singletons to bind with this extension point - these are bound in {@link #bind(Binder)} - */ - public ClassSet(String name, Class extensionClass, Class... singletons) { - super(name, singletons); - this.extensionClass = extensionClass; - } - - /** - * Registers a new extension - * - * @param extension the extension to register - * @throws IllegalArgumentException iff the class is already registered - */ - public void registerExtension(Class extension) { - if (extensions.contains(extension)) { - throw new IllegalArgumentException("Can't register the same [" + this.name + "] more than once for [" + extension.getName() + "]"); - } - extensions.add(extension); - } - - @Override - protected void bindExtensions(Binder binder) { - Multibinder allocationMultibinder = Multibinder.newSetBinder(binder, extensionClass); - for (Class clazz : extensions) { - binder.bind(clazz).asEagerSingleton(); - allocationMultibinder.addBinding().to(clazz); - } - } - - public boolean isEmpty() { - return extensions.isEmpty(); - } - } - - /** - * A an instance of a map, mapping one instance value to another. Both key and value are instances, not classes - * like with other extension points. - */ - public static final class InstanceMap extends ExtensionPoint { - private final Map map = new HashMap<>(); - private final Class keyType; - private final Class valueType; - - /** - * Creates a new {@link ClassSet} - * - * @param name the human readable underscore case name of the extension point. This is used in error messages. - * @param singletons a list of singletons to bind with this extension point - these are bound in {@link #bind(Binder)} - */ - public InstanceMap(String name, Class keyType, Class valueType, Class... singletons) { - super(name, singletons); - this.keyType = keyType; - this.valueType = valueType; - } - - /** - * Registers a mapping from {@code key} to {@code value} - * - * @throws IllegalArgumentException iff the key is already registered - */ - public void registerExtension(K key, V value) { - V old = map.put(key, value); - if (old != null) { - throw new IllegalArgumentException("Cannot register [" + this.name + "] with key [" + key + "] to [" + value + "], already registered to [" + old + "]"); - } - } - - @Override - protected void bindExtensions(Binder binder) { - MapBinder mapBinder = MapBinder.newMapBinder(binder, keyType, valueType); - for (Map.Entry entry : map.entrySet()) { - mapBinder.addBinding(entry.getKey()).toInstance(entry.getValue()); - } - } - } -} diff --git a/core/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java b/core/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java index ef3018c19b7..61316a852bb 100644 --- a/core/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java +++ b/core/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java @@ -19,20 +19,6 @@ package org.elasticsearch.discovery; -import org.elasticsearch.common.inject.AbstractModule; -import org.elasticsearch.common.network.NetworkService; -import org.elasticsearch.common.settings.Setting; -import org.elasticsearch.common.settings.Setting.Property; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.ExtensionPoint; -import org.elasticsearch.discovery.zen.ZenDiscovery; -import org.elasticsearch.plugins.DiscoveryPlugin; -import org.elasticsearch.transport.TransportService; -import org.elasticsearch.discovery.zen.ZenPing; -import org.elasticsearch.discovery.zen.ZenPingService; -import org.elasticsearch.discovery.zen.UnicastHostsProvider; -import org.elasticsearch.discovery.zen.UnicastZenPing; - import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -41,6 +27,16 @@ import java.util.Objects; import java.util.function.Function; import java.util.function.Supplier; +import org.elasticsearch.common.inject.AbstractModule; +import org.elasticsearch.common.network.NetworkService; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Setting.Property; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.discovery.zen.UnicastHostsProvider; +import org.elasticsearch.discovery.zen.ZenDiscovery; +import org.elasticsearch.plugins.DiscoveryPlugin; +import org.elasticsearch.transport.TransportService; + /** * A module for loading classes for node discovery. */ @@ -52,8 +48,7 @@ public class DiscoveryModule extends AbstractModule { new Setting<>("discovery.zen.hosts_provider", DISCOVERY_TYPE_SETTING, Function.identity(), Property.NodeScope); private final Settings settings; - private final Map> unicastHostProviders; - private final ExtensionPoint.ClassSet zenPings = new ExtensionPoint.ClassSet<>("zen_ping", ZenPing.class); + private final UnicastHostsProvider hostsProvider; private final Map> discoveryTypes = new HashMap<>(); public DiscoveryModule(Settings settings, TransportService transportService, NetworkService networkService, @@ -62,16 +57,30 @@ public class DiscoveryModule extends AbstractModule { addDiscoveryType("none", NoneDiscovery.class); addDiscoveryType("zen", ZenDiscovery.class); - Map> hostProviders = new HashMap<>(); - hostProviders.put("zen", () -> Collections::emptyList); - for (DiscoveryPlugin plugin : plugins) { - plugin.getZenHostsProviders(transportService, networkService).entrySet().forEach(entry -> { - if (hostProviders.put(entry.getKey(), entry.getValue()) != null) { - throw new IllegalArgumentException("Cannot specify zen hosts provider [" + entry.getKey() + "] twice"); - } - }); + String discoveryType = DISCOVERY_TYPE_SETTING.get(settings); + if (discoveryType.equals("none") == false) { + Map> hostProviders = new HashMap<>(); + hostProviders.put("zen", () -> Collections::emptyList); + for (DiscoveryPlugin plugin : plugins) { + plugin.getZenHostsProviders(transportService, networkService).entrySet().forEach(entry -> { + if (hostProviders.put(entry.getKey(), entry.getValue()) != null) { + throw new IllegalArgumentException("Cannot specify zen hosts provider [" + entry.getKey() + "] twice"); + } + }); + } + String hostsProviderName = DISCOVERY_HOSTS_PROVIDER_SETTING.get(settings); + Supplier hostsProviderSupplier = hostProviders.get(hostsProviderName); + if (hostsProviderSupplier == null) { + throw new IllegalArgumentException("Unknown zen hosts provider [" + hostsProviderName + "]"); + } + hostsProvider = Objects.requireNonNull(hostsProviderSupplier.get()); + } else { + hostsProvider = null; } - unicastHostProviders = Collections.unmodifiableMap(hostProviders); + } + + public UnicastHostsProvider getHostsProvider() { + return hostsProvider; } /** @@ -84,10 +93,6 @@ public class DiscoveryModule extends AbstractModule { discoveryTypes.put(type, clazz); } - public void addZenPing(Class clazz) { - zenPings.registerExtension(clazz); - } - @Override protected void configure() { String discoveryType = DISCOVERY_TYPE_SETTING.get(settings); @@ -97,18 +102,7 @@ public class DiscoveryModule extends AbstractModule { } if (discoveryType.equals("none") == false) { - bind(ZenPingService.class).asEagerSingleton(); - String hostsProviderName = DISCOVERY_HOSTS_PROVIDER_SETTING.get(settings); - Supplier hostsProviderSupplier = unicastHostProviders.get(hostsProviderName); - if (hostsProviderSupplier == null) { - throw new IllegalArgumentException("Unknown zen hosts provider [" + hostsProviderName + "]"); - } - UnicastHostsProvider hostsProvider = Objects.requireNonNull(hostsProviderSupplier.get()); bind(UnicastHostsProvider.class).toInstance(hostsProvider); - if (zenPings.isEmpty()) { - zenPings.registerExtension(UnicastZenPing.class); - } - zenPings.bind(binder()); } bind(Discovery.class).to(discoveryClass).asEagerSingleton(); } diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/FaultDetection.java b/core/src/main/java/org/elasticsearch/discovery/zen/FaultDetection.java index f1f8b28ad09..715e8be03ef 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/FaultDetection.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/FaultDetection.java @@ -19,6 +19,8 @@ package org.elasticsearch.discovery.zen; +import java.io.Closeable; + import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.component.AbstractComponent; @@ -36,7 +38,7 @@ import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds; * A base class for {@link MasterFaultDetection} & {@link NodesFaultDetection}, * making sure both use the same setting. */ -public abstract class FaultDetection extends AbstractComponent { +public abstract class FaultDetection extends AbstractComponent implements Closeable { public static final Setting CONNECT_ON_NETWORK_DISCONNECT_SETTING = Setting.boolSetting("discovery.zen.fd.connect_on_network_disconnect", false, Property.NodeScope); @@ -80,6 +82,7 @@ public abstract class FaultDetection extends AbstractComponent { } } + @Override public void close() { transportService.removeConnectionListener(connectionListener); } diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/UnicastZenPing.java b/core/src/main/java/org/elasticsearch/discovery/zen/UnicastZenPing.java index bd8e2353f72..f6870cc05b6 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/UnicastZenPing.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/UnicastZenPing.java @@ -19,6 +19,26 @@ package org.elasticsearch.discovery.zen; +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; + import com.carrotsearch.hppc.cursors.ObjectCursor; import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.logging.log4j.util.Supplier; @@ -30,8 +50,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.UUIDs; -import org.elasticsearch.common.component.AbstractLifecycleComponent; -import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.Setting; @@ -56,34 +75,13 @@ import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportResponseHandler; import org.elasticsearch.transport.TransportService; -import java.io.Closeable; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Queue; -import java.util.Set; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.ThreadFactory; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Function; - import static java.util.Collections.emptyList; import static java.util.Collections.emptyMap; import static java.util.Collections.emptySet; import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.newConcurrentMap; import static org.elasticsearch.discovery.zen.ZenPing.PingResponse.readPingResponse; -public class UnicastZenPing extends AbstractLifecycleComponent implements ZenPing { +public class UnicastZenPing extends AbstractComponent implements ZenPing { public static final String ACTION_NAME = "internal:discovery/zen/unicast"; public static final Setting> DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING = @@ -125,15 +123,13 @@ public class UnicastZenPing extends AbstractLifecycleComponent implements ZenPin private volatile boolean closed = false; - @Inject public UnicastZenPing(Settings settings, ThreadPool threadPool, TransportService transportService, - UnicastHostsProvider unicastHostsProviders) { + UnicastHostsProvider unicastHostsProvider) { super(settings); this.threadPool = threadPool; this.transportService = transportService; this.clusterName = ClusterName.CLUSTER_NAME_SETTING.get(settings); - - this.hostsProvider = unicastHostsProviders; + this.hostsProvider = unicastHostsProvider; this.concurrentConnects = DISCOVERY_ZEN_PING_UNICAST_CONCURRENT_CONNECTS_SETTING.get(settings); List hosts = DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING.get(settings); @@ -190,26 +186,14 @@ public class UnicastZenPing extends AbstractLifecycleComponent implements ZenPin } @Override - protected void doStart() { - } - - @Override - protected void doStop() { - } - - @Override - protected void doClose() { + public void close() throws IOException { ThreadPool.terminate(unicastConnectExecutor, 0, TimeUnit.SECONDS); - try { - IOUtils.close(receivedResponses.values()); - } catch (IOException e) { - throw new ElasticsearchException("Error wile closing send ping handlers", e); - } + IOUtils.close(receivedResponses.values()); closed = true; } @Override - public void setPingContextProvider(PingContextProvider contextProvider) { + public void start(PingContextProvider contextProvider) { this.contextProvider = contextProvider; } @@ -501,9 +485,6 @@ public class UnicastZenPing extends AbstractLifecycleComponent implements ZenPin } private UnicastPingResponse handlePingRequest(final UnicastPingRequest request) { - if (!lifecycle.started()) { - throw new IllegalStateException("received ping request while not started"); - } temporalResponses.add(request.pingResponse); threadPool.schedule(TimeValue.timeValueMillis(request.timeout.millis() * 2), ThreadPool.Names.SAME, new Runnable() { @Override diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java b/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java index a50ebab4c00..90e7d3e2144 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java @@ -22,6 +22,7 @@ package org.elasticsearch.discovery.zen; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.logging.log4j.util.Supplier; +import org.apache.lucene.util.IOUtils; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.cluster.ClusterChangedEvent; @@ -67,6 +68,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.Set; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; @@ -105,7 +107,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover private AllocationService allocationService; private final ClusterName clusterName; private final DiscoverySettings discoverySettings; - private final ZenPingService pingService; + private final ZenPing zenPing; private final MasterFaultDetection masterFD; private final NodesFaultDetection nodesFD; private final PublishClusterStateAction publishClusterState; @@ -137,18 +139,16 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover private volatile NodeRemovalClusterStateTaskExecutor nodeRemovalExecutor; @Inject - public ZenDiscovery(Settings settings, ThreadPool threadPool, - TransportService transportService, final ClusterService clusterService, ClusterSettings clusterSettings, - ZenPingService pingService) { + public ZenDiscovery(Settings settings, ThreadPool threadPool, TransportService transportService, + ClusterService clusterService, ClusterSettings clusterSettings, ZenPing zenPing) { super(settings); this.clusterService = clusterService; this.clusterName = clusterService.getClusterName(); this.transportService = transportService; this.discoverySettings = new DiscoverySettings(settings, clusterSettings); - this.pingService = pingService; + this.zenPing = zenPing; this.electMaster = new ElectMasterService(settings); this.pingTimeout = PING_TIMEOUT_SETTING.get(settings); - this.joinTimeout = JOIN_TIMEOUT_SETTING.get(settings); this.joinRetryAttempts = JOIN_RETRY_ATTEMPTS_SETTING.get(settings); this.joinRetryDelay = JOIN_RETRY_DELAY_SETTING.get(settings); @@ -171,7 +171,6 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover this.masterFD = new MasterFaultDetection(settings, threadPool, transportService, clusterService); this.masterFD.addListener(new MasterNodeFailureListener()); - this.nodesFD = new NodesFaultDetection(settings, threadPool, transportService, clusterService.getClusterName()); this.nodesFD.addListener(new NodeFaultDetectionListener()); @@ -183,9 +182,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover new NewPendingClusterStateListener(), discoverySettings, clusterService.getClusterName()); - this.pingService.setPingContextProvider(this); this.membership = new MembershipAction(settings, transportService, this, new MembershipListener()); - this.joinThreadControl = new JoinThreadControl(threadPool); transportService.registerRequestHandler( @@ -201,7 +198,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover protected void doStart() { nodesFD.setLocalNode(clusterService.localNode()); joinThreadControl.start(); - pingService.start(); + zenPing.start(this); this.nodeJoinController = new NodeJoinController(clusterService, allocationService, electMaster, discoverySettings, settings); this.nodeRemovalExecutor = new NodeRemovalClusterStateTaskExecutor(allocationService, electMaster, this::rejoin, logger); } @@ -233,7 +230,6 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover @Override protected void doStop() { joinThreadControl.stop(); - pingService.stop(); masterFD.stop("zen disco stop"); nodesFD.stop(); DiscoveryNodes nodes = nodes(); @@ -264,10 +260,8 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover } @Override - protected void doClose() { - masterFD.close(); - nodesFD.close(); - pingService.close(); + protected void doClose() throws IOException { + IOUtils.close(masterFD, nodesFD, zenPing); } @Override @@ -871,7 +865,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover private DiscoveryNode findMaster() { logger.trace("starting to ping"); - List fullPingResponses = pingService.pingAndWait(pingTimeout).toList(); + List fullPingResponses = pingAndWait(pingTimeout).toList(); if (fullPingResponses == null) { logger.trace("No full ping responses"); return null; @@ -1013,6 +1007,28 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover } } + private ZenPing.PingCollection pingAndWait(TimeValue timeout) { + final ZenPing.PingCollection response = new ZenPing.PingCollection(); + final CountDownLatch latch = new CountDownLatch(1); + try { + zenPing.ping(pings -> { + response.addPings(pings); + latch.countDown(); + }, timeout); + } catch (Exception ex) { + logger.warn("Ping execution failed", ex); + latch.countDown(); + } + + try { + latch.await(); + return response; + } catch (InterruptedException e) { + logger.trace("pingAndWait interrupted"); + return response; + } + } + private class NewPendingClusterStateListener implements PublishClusterStateAction.NewPendingClusterStateListener { @Override diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/ZenPing.java b/core/src/main/java/org/elasticsearch/discovery/zen/ZenPing.java index be1df88d334..cb2c8cb5019 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/ZenPing.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/ZenPing.java @@ -19,15 +19,7 @@ package org.elasticsearch.discovery.zen; -import org.elasticsearch.cluster.ClusterName; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.component.LifecycleComponent; -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.unit.TimeValue; - +import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; @@ -36,11 +28,19 @@ import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicLong; +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.node.DiscoveryNode; +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.unit.TimeValue; + import static org.elasticsearch.gateway.GatewayService.STATE_NOT_RECOVERED_BLOCK; -public interface ZenPing extends LifecycleComponent { +public interface ZenPing extends Closeable { - void setPingContextProvider(PingContextProvider contextProvider); + void start(PingContextProvider contextProvider); void ping(PingListener listener, TimeValue timeout); diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/ZenPingService.java b/core/src/main/java/org/elasticsearch/discovery/zen/ZenPingService.java deleted file mode 100644 index 3aa3017f549..00000000000 --- a/core/src/main/java/org/elasticsearch/discovery/zen/ZenPingService.java +++ /dev/null @@ -1,105 +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.discovery.zen; - -import org.elasticsearch.common.component.AbstractLifecycleComponent; -import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.TimeValue; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Set; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicBoolean; - -public class ZenPingService extends AbstractLifecycleComponent { - - private List zenPings = Collections.emptyList(); - - @Inject - public ZenPingService(Settings settings, Set zenPings) { - super(settings); - this.zenPings = Collections.unmodifiableList(new ArrayList<>(zenPings)); - } - - public List zenPings() { - return this.zenPings; - } - - public void setPingContextProvider(PingContextProvider contextProvider) { - if (lifecycle.started()) { - throw new IllegalStateException("Can't set nodes provider when started"); - } - for (ZenPing zenPing : zenPings) { - zenPing.setPingContextProvider(contextProvider); - } - } - - @Override - protected void doStart() { - for (ZenPing zenPing : zenPings) { - zenPing.start(); - } - } - - @Override - protected void doStop() { - for (ZenPing zenPing : zenPings) { - zenPing.stop(); - } - } - - @Override - protected void doClose() { - for (ZenPing zenPing : zenPings) { - zenPing.close(); - } - } - - public ZenPing.PingCollection pingAndWait(TimeValue timeout) { - final ZenPing.PingCollection response = new ZenPing.PingCollection(); - final CountDownLatch latch = new CountDownLatch(zenPings.size()); - for (ZenPing zenPing : zenPings) { - final AtomicBoolean counted = new AtomicBoolean(); - try { - zenPing.ping(pings -> { - response.addPings(pings); - if (counted.compareAndSet(false, true)) { - latch.countDown(); - } - }, timeout); - } catch (Exception ex) { - logger.warn("Ping execution failed", ex); - if (counted.compareAndSet(false, true)) { - latch.countDown(); - } - } - } - try { - latch.await(); - return response; - } catch (InterruptedException e) { - logger.trace("pingAndWait interrupted"); - return response; - } - } -} diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java index 8d76ae9cbc3..f5ad4ff8772 100644 --- a/core/src/main/java/org/elasticsearch/node/Node.java +++ b/core/src/main/java/org/elasticsearch/node/Node.java @@ -76,6 +76,9 @@ import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.DiscoveryModule; import org.elasticsearch.discovery.DiscoverySettings; +import org.elasticsearch.discovery.zen.UnicastHostsProvider; +import org.elasticsearch.discovery.zen.UnicastZenPing; +import org.elasticsearch.discovery.zen.ZenPing; import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.gateway.GatewayAllocator; @@ -319,7 +322,8 @@ public class Node implements Closeable { final ClusterService clusterService = new ClusterService(settings, settingsModule.getClusterSettings(), threadPool); clusterService.add(scriptModule.getScriptService()); resourcesToClose.add(clusterService); - final TribeService tribeService = new TribeService(settings, clusterService, nodeEnvironment.nodeId(), classpathPlugins); + final TribeService tribeService = new TribeService(settings, clusterService, nodeEnvironment.nodeId(), + s -> newTribeClientNode(s, classpathPlugins)); resourcesToClose.add(tribeService); final IngestService ingestService = new IngestService(settings, threadPool, this.environment, scriptModule.getScriptService(), analysisModule.getAnalysisRegistry(), pluginsService.filterPlugins(IngestPlugin.class)); @@ -393,7 +397,10 @@ public class Node implements Closeable { b.bind(HttpServer.class).toProvider(Providers.of(null)); }; } - modules.add(new DiscoveryModule(this.settings, transportService, networkService, pluginsService.filterPlugins(DiscoveryPlugin.class))); + final DiscoveryModule discoveryModule = new DiscoveryModule(this.settings, transportService, networkService, + pluginsService.filterPlugins(DiscoveryPlugin.class)); + final ZenPing zenPing = newZenPing(settings, threadPool, transportService, discoveryModule.getHostsProvider()); + modules.add(discoveryModule); pluginsService.processModules(modules); modules.add(b -> { b.bind(IndicesQueriesRegistry.class).toInstance(searchModule.getQueryParserRegistry()); @@ -425,6 +432,7 @@ public class Node implements Closeable { b.bind(UpdateHelper.class).toInstance(new UpdateHelper(settings, scriptModule.getScriptService())); b.bind(MetaDataIndexUpgradeService.class).toInstance(new MetaDataIndexUpgradeService(settings, indicesModule.getMapperRegistry(), settingsModule.getIndexScopedSettings())); + b.bind(ZenPing.class).toInstance(zenPing); { RecoverySettings recoverySettings = new RecoverySettings(settings, settingsModule.getClusterSettings()); processRecoverySettings(settingsModule.getClusterSettings(), recoverySettings); @@ -881,4 +889,15 @@ public class Node implements Closeable { } return customNameResolvers; } + + /** Create a new ZenPing instance for use in zen discovery. */ + protected ZenPing newZenPing(Settings settings, ThreadPool threadPool, TransportService transportService, + UnicastHostsProvider hostsProvider) { + return new UnicastZenPing(settings, threadPool, transportService, hostsProvider); + } + + /** Constructs an internal node used as a client into a cluster fronted by this tribe node. */ + protected Node newTribeClientNode(Settings settings, Collection> classpathPlugins) { + return new Node(new Environment(settings), classpathPlugins); + } } diff --git a/core/src/main/java/org/elasticsearch/tribe/TribeClientNode.java b/core/src/main/java/org/elasticsearch/tribe/TribeClientNode.java deleted file mode 100644 index d9520aef768..00000000000 --- a/core/src/main/java/org/elasticsearch/tribe/TribeClientNode.java +++ /dev/null @@ -1,36 +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.tribe; - -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.env.Environment; -import org.elasticsearch.node.Node; -import org.elasticsearch.plugins.Plugin; - -import java.util.Collection; - -/** - * An internal node that connects to a remove cluster, as part of a tribe node. - */ -class TribeClientNode extends Node { - TribeClientNode(Settings settings, Collection> classpathPlugins) { - super(new Environment(settings), classpathPlugins); - } -} diff --git a/core/src/main/java/org/elasticsearch/tribe/TribeService.java b/core/src/main/java/org/elasticsearch/tribe/TribeService.java index 7871a0a6f39..69ad77fc91e 100644 --- a/core/src/main/java/org/elasticsearch/tribe/TribeService.java +++ b/core/src/main/java/org/elasticsearch/tribe/TribeService.java @@ -185,7 +185,7 @@ public class TribeService extends AbstractLifecycleComponent { private final List nodes = new CopyOnWriteArrayList<>(); public TribeService(Settings settings, ClusterService clusterService, final String tribeNodeId, - Collection> classpathPlugins) { + Function clientNodeBuilder) { super(settings); this.clusterService = clusterService; Map nodesSettings = new HashMap<>(settings.getGroups("tribe", true)); @@ -193,7 +193,7 @@ public class TribeService extends AbstractLifecycleComponent { nodesSettings.remove("on_conflict"); // remove prefix settings that don't indicate a client for (Map.Entry entry : nodesSettings.entrySet()) { Settings clientSettings = buildClientSettings(entry.getKey(), tribeNodeId, settings, entry.getValue()); - nodes.add(new TribeClientNode(clientSettings, classpathPlugins)); + nodes.add(clientNodeBuilder.apply(clientSettings)); } this.blockIndicesMetadata = BLOCKS_METADATA_INDICES_SETTING.get(settings).toArray(Strings.EMPTY_ARRAY); diff --git a/core/src/test/java/org/elasticsearch/common/util/ExtensionPointTests.java b/core/src/test/java/org/elasticsearch/common/util/ExtensionPointTests.java deleted file mode 100644 index 8fabbcc60ae..00000000000 --- a/core/src/test/java/org/elasticsearch/common/util/ExtensionPointTests.java +++ /dev/null @@ -1,63 +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.util; - -import java.util.Set; -import java.util.concurrent.atomic.AtomicInteger; - -import org.elasticsearch.common.inject.Binder; -import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.inject.Injector; -import org.elasticsearch.common.inject.Module; -import org.elasticsearch.common.inject.ModulesBuilder; -import org.elasticsearch.test.ESTestCase; - -public class ExtensionPointTests extends ESTestCase { - - public void testClassSet() { - final ExtensionPoint.ClassSet allocationDeciders = new ExtensionPoint.ClassSet<>("test_class", TestBaseClass.class, Consumer.class); - allocationDeciders.registerExtension(TestImpl.class); - Injector injector = new ModulesBuilder().add(new Module() { - @Override - public void configure(Binder binder) { - allocationDeciders.bind(binder); - } - }).createInjector(); - assertEquals(1, TestImpl.instances.get()); - - } - - public static class TestBaseClass {} - - public static class Consumer { - @Inject - public Consumer(Set deciders, TestImpl other) { - // we require the TestImpl more than once to ensure it's bound as a singleton - } - } - - public static class TestImpl extends TestBaseClass { - static final AtomicInteger instances = new AtomicInteger(0); - - @Inject - public TestImpl() { - instances.incrementAndGet(); - } - } -} diff --git a/core/src/test/java/org/elasticsearch/discovery/DiscoveryModuleTests.java b/core/src/test/java/org/elasticsearch/discovery/DiscoveryModuleTests.java index 6f0aeca9d77..28775defe45 100644 --- a/core/src/test/java/org/elasticsearch/discovery/DiscoveryModuleTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/DiscoveryModuleTests.java @@ -76,8 +76,9 @@ public class DiscoveryModuleTests extends ModuleTestCase { public void testUnknownHostsProvider() { Settings settings = Settings.builder().put(DiscoveryModule.DISCOVERY_HOSTS_PROVIDER_SETTING.getKey(), "dne").build(); - DiscoveryModule module = new DiscoveryModule(settings, null, null, Collections.emptyList()); - assertBindingFailure(module, "Unknown zen hosts provider"); + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> + new DiscoveryModule(settings, null, null, Collections.emptyList())); + assertEquals("Unknown zen hosts provider [dne]", e.getMessage()); } public void testDuplicateHostsProvider() { diff --git a/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java b/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java index 0208664033c..bbde01a6686 100644 --- a/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java +++ b/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java @@ -56,7 +56,6 @@ import org.elasticsearch.discovery.zen.PublishClusterStateAction; import org.elasticsearch.discovery.zen.UnicastZenPing; import org.elasticsearch.discovery.zen.ZenDiscovery; import org.elasticsearch.discovery.zen.ZenPing; -import org.elasticsearch.discovery.zen.ZenPingService; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.indices.store.IndicesStoreIntegrationIT; import org.elasticsearch.monitor.jvm.HotThreads; @@ -195,12 +194,9 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase { ensureStableCluster(numberOfNodes); // TODO: this is a temporary solution so that nodes will not base their reaction to a partition based on previous successful results - for (ZenPingService pingService : internalCluster().getInstances(ZenPingService.class)) { - for (ZenPing zenPing : pingService.zenPings()) { - if (zenPing instanceof UnicastZenPing) { - ((UnicastZenPing) zenPing).clearTemporalResponses(); - } - } + ZenPing zenPing = internalCluster().getInstance(ZenPing.class); + if (zenPing instanceof UnicastZenPing) { + ((UnicastZenPing) zenPing).clearTemporalResponses(); } return nodes; } @@ -858,10 +854,9 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase { // Forcefully clean temporal response lists on all nodes. Otherwise the node in the unicast host list // includes all the other nodes that have pinged it and the issue doesn't manifest - for (ZenPingService pingService : internalCluster().getInstances(ZenPingService.class)) { - for (ZenPing zenPing : pingService.zenPings()) { - ((UnicastZenPing) zenPing).clearTemporalResponses(); - } + ZenPing zenPing = internalCluster().getInstance(ZenPing.class); + if (zenPing instanceof UnicastZenPing) { + ((UnicastZenPing) zenPing).clearTemporalResponses(); } // Simulate a network issue between the unlucky node and elected master node in both directions. @@ -896,10 +891,9 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase { // Forcefully clean temporal response lists on all nodes. Otherwise the node in the unicast host list // includes all the other nodes that have pinged it and the issue doesn't manifest - for (ZenPingService pingService : internalCluster().getInstances(ZenPingService.class)) { - for (ZenPing zenPing : pingService.zenPings()) { - ((UnicastZenPing) zenPing).clearTemporalResponses(); - } + ZenPing zenPing = internalCluster().getInstance(ZenPing.class); + if (zenPing instanceof UnicastZenPing) { + ((UnicastZenPing) zenPing).clearTemporalResponses(); } // Simulate a network issue between the unicast target node and the rest of the cluster diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java index 106612f22e0..4294bdd3dd4 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.discovery.zen; +import org.apache.lucene.util.IOUtils; import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; @@ -33,10 +34,6 @@ import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; -import org.elasticsearch.discovery.zen.ElectMasterService; -import org.elasticsearch.discovery.zen.UnicastZenPing; -import org.elasticsearch.discovery.zen.PingContextProvider; -import org.elasticsearch.discovery.zen.ZenPing; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.VersionUtils; @@ -47,6 +44,7 @@ import org.elasticsearch.transport.TransportConnectionListener; import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportSettings; +import java.io.IOException; import java.net.InetSocketAddress; import java.util.Collection; import java.util.Collections; @@ -62,7 +60,7 @@ import static org.hamcrest.Matchers.greaterThan; public class UnicastZenPingTests extends ESTestCase { private static final UnicastHostsProvider EMPTY_HOSTS_PROVIDER = Collections::emptyList; - public void testSimplePings() throws InterruptedException { + public void testSimplePings() throws IOException, InterruptedException { int startPort = 11000 + randomIntBetween(0, 1000); int endPort = startPort + 10; Settings settings = Settings.builder() @@ -97,7 +95,7 @@ public class UnicastZenPingTests extends ESTestCase { Settings hostsSettingsMismatch = Settings.builder().put(hostsSettings).put(settingsMismatch).build(); UnicastZenPing zenPingA = new UnicastZenPing(hostsSettings, threadPool, handleA.transportService, EMPTY_HOSTS_PROVIDER); - zenPingA.setPingContextProvider(new PingContextProvider() { + zenPingA.start(new PingContextProvider() { @Override public DiscoveryNodes nodes() { return DiscoveryNodes.builder().add(handleA.node).localNodeId("UZP_A").build(); @@ -108,10 +106,9 @@ public class UnicastZenPingTests extends ESTestCase { return ClusterState.builder(state).blocks(ClusterBlocks.builder().addGlobalBlock(STATE_NOT_RECOVERED_BLOCK)).build(); } }); - zenPingA.start(); UnicastZenPing zenPingB = new UnicastZenPing(hostsSettings, threadPool, handleB.transportService, EMPTY_HOSTS_PROVIDER); - zenPingB.setPingContextProvider(new PingContextProvider() { + zenPingB.start(new PingContextProvider() { @Override public DiscoveryNodes nodes() { return DiscoveryNodes.builder().add(handleB.node).localNodeId("UZP_B").build(); @@ -122,7 +119,6 @@ public class UnicastZenPingTests extends ESTestCase { return state; } }); - zenPingB.start(); UnicastZenPing zenPingC = new UnicastZenPing(hostsSettingsMismatch, threadPool, handleC.transportService, EMPTY_HOSTS_PROVIDER) { @Override @@ -130,7 +126,7 @@ public class UnicastZenPingTests extends ESTestCase { return versionD; } }; - zenPingC.setPingContextProvider(new PingContextProvider() { + zenPingC.start(new PingContextProvider() { @Override public DiscoveryNodes nodes() { return DiscoveryNodes.builder().add(handleC.node).localNodeId("UZP_C").build(); @@ -141,10 +137,9 @@ public class UnicastZenPingTests extends ESTestCase { return state; } }); - zenPingC.start(); UnicastZenPing zenPingD = new UnicastZenPing(hostsSettingsMismatch, threadPool, handleD.transportService, EMPTY_HOSTS_PROVIDER); - zenPingD.setPingContextProvider(new PingContextProvider() { + zenPingD.start(new PingContextProvider() { @Override public DiscoveryNodes nodes() { return DiscoveryNodes.builder().add(handleD.node).localNodeId("UZP_D").build(); @@ -155,7 +150,6 @@ public class UnicastZenPingTests extends ESTestCase { return state; } }); - zenPingD.start(); try { logger.info("ping from UZP_A"); @@ -185,15 +179,12 @@ public class UnicastZenPingTests extends ESTestCase { assertThat(pingResponses.size(), equalTo(0)); assertCounters(handleD, handleA, handleB, handleC, handleD); } finally { - zenPingA.close(); - zenPingB.close(); - zenPingC.close(); - zenPingD.close(); - handleA.transportService.close(); - handleB.transportService.close(); - handleC.transportService.close(); - handleD.transportService.close(); - terminate(threadPool); + try { + IOUtils.close(zenPingA, zenPingB, zenPingC, zenPingD, + handleA.transportService, handleB.transportService, handleC.transportService, handleD.transportService); + } finally { + terminate(threadPool); + } } } diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java index 8ef4751c9cf..88cf23fe938 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java @@ -34,6 +34,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.zen.PublishClusterStateActionTests.AssertingAckListener; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.discovery.MockZenPing; import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; @@ -269,8 +270,7 @@ public class ZenDiscoveryUnitTests extends ESTestCase { private ZenDiscovery buildZenDiscovery(Settings settings, TransportService service, ClusterService clusterService, ThreadPool threadPool) { ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - ZenPingService zenPingService = new ZenPingService(settings, Collections.emptySet()); - ZenDiscovery zenDiscovery = new ZenDiscovery(settings, threadPool, service, clusterService, clusterSettings, zenPingService); + ZenDiscovery zenDiscovery = new ZenDiscovery(settings, threadPool, service, clusterService, clusterSettings, new MockZenPing(settings)); zenDiscovery.start(); return zenDiscovery; } diff --git a/core/src/test/java/org/elasticsearch/tribe/TribeIT.java b/core/src/test/java/org/elasticsearch/tribe/TribeIT.java index 0bc4974f285..6121b2c0c86 100644 --- a/core/src/test/java/org/elasticsearch/tribe/TribeIT.java +++ b/core/src/test/java/org/elasticsearch/tribe/TribeIT.java @@ -19,6 +19,18 @@ package org.elasticsearch.tribe; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.function.Predicate; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; + import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.support.DestructiveOperations; import org.elasticsearch.client.Client; @@ -46,18 +58,6 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; -import java.io.IOException; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Set; -import java.util.function.Consumer; -import java.util.function.Function; -import java.util.function.Predicate; -import java.util.stream.Stream; -import java.util.stream.StreamSupport; - import static java.util.stream.Collectors.toSet; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; diff --git a/docs/plugins/discovery-file.asciidoc b/docs/plugins/discovery-file.asciidoc index a848cdd6ff1..e8e1e42f867 100644 --- a/docs/plugins/discovery-file.asciidoc +++ b/docs/plugins/discovery-file.asciidoc @@ -89,5 +89,5 @@ running on the default port: ---------------------------------------------------------------- Host names are allowed instead of IP addresses (similar to -`discovery.zen.ping.unicast.hosts`), and IPv6 addresses must be +`discovery.zen.ping.unicast.hosts`), and IPv6 addresses must be specified in brackets with the port coming after the brackets. diff --git a/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java b/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java index fd54c5fadbe..34621802f55 100644 --- a/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java +++ b/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java @@ -69,14 +69,14 @@ public class TribeUnitTests extends ESTestCase { .build(); final List> mockPlugins = Arrays.asList(MockTcpTransportPlugin.class, MockZenPing.TestPlugin.class); - tribe1 = new TribeClientNode( + tribe1 = new MockNode( Settings.builder() .put(baseSettings) .put("cluster.name", "tribe1") .put("node.name", "tribe1_node") .put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), random().nextLong()) .build(), mockPlugins).start(); - tribe2 = new TribeClientNode( + tribe2 = new MockNode( Settings.builder() .put(baseSettings) .put("cluster.name", "tribe2") diff --git a/test/framework/src/main/java/org/elasticsearch/node/MockNode.java b/test/framework/src/main/java/org/elasticsearch/node/MockNode.java index 6eb28bea14a..38e8a8436b1 100644 --- a/test/framework/src/main/java/org/elasticsearch/node/MockNode.java +++ b/test/framework/src/main/java/org/elasticsearch/node/MockNode.java @@ -24,6 +24,8 @@ import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.MockBigArrays; +import org.elasticsearch.discovery.zen.UnicastHostsProvider; +import org.elasticsearch.discovery.zen.ZenPing; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.indices.recovery.RecoverySettings; @@ -33,6 +35,7 @@ import org.elasticsearch.script.ScriptService; import org.elasticsearch.search.MockSearchService; import org.elasticsearch.search.SearchService; import org.elasticsearch.search.fetch.FetchPhase; +import org.elasticsearch.test.discovery.MockZenPing; import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.Transport; @@ -96,6 +99,21 @@ public class MockNode extends Node { } } + @Override + protected ZenPing newZenPing(Settings settings, ThreadPool threadPool, TransportService transportService, + UnicastHostsProvider hostsProvider) { + if (getPluginsService().filterPlugins(MockZenPing.TestPlugin.class).isEmpty()) { + return super.newZenPing(settings, threadPool, transportService, hostsProvider); + } else { + return new MockZenPing(settings); + } + } + + @Override + protected Node newTribeClientNode(Settings settings, Collection> classpathPlugins) { + return new MockNode(settings, classpathPlugins); + } + @Override protected void processRecoverySettings(ClusterSettings clusterSettings, RecoverySettings recoverySettings) { if (false == getPluginsService().filterPlugins(RecoverySettingsChunkSizePlugin.class).isEmpty()) { diff --git a/test/framework/src/main/java/org/elasticsearch/test/discovery/MockZenPing.java b/test/framework/src/main/java/org/elasticsearch/test/discovery/MockZenPing.java index 3e02b9de0fb..d5e7de1d9bf 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/discovery/MockZenPing.java +++ b/test/framework/src/main/java/org/elasticsearch/test/discovery/MockZenPing.java @@ -20,6 +20,7 @@ package org.elasticsearch.test.discovery; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; @@ -40,7 +41,10 @@ import java.util.stream.Collectors; * A {@link ZenPing} implementation which returns results based on an static in-memory map. This allows pinging * to be immediate and can be used to speed up tests. */ -public final class MockZenPing extends AbstractLifecycleComponent implements ZenPing { +public final class MockZenPing extends AbstractComponent implements ZenPing { + + /** A marker plugin used by {@link org.elasticsearch.node.MockNode} to indicate this mock zen ping should be used. */ + public static class TestPlugin extends Plugin {} static final Map> activeNodesPerCluster = ConcurrentCollections.newConcurrentMap(); @@ -52,8 +56,11 @@ public final class MockZenPing extends AbstractLifecycleComponent implements Zen } @Override - public void setPingContextProvider(PingContextProvider contextProvider) { + public void start(PingContextProvider contextProvider) { this.contextProvider = contextProvider; + assert contextProvider != null; + boolean added = getActiveNodesForCurrentCluster().add(this); + assert added; } @Override @@ -75,33 +82,14 @@ public final class MockZenPing extends AbstractLifecycleComponent implements Zen return new PingResponse(clusterState.nodes().getLocalNode(), clusterState.nodes().getMasterNode(), clusterState); } - @Override - protected void doStart() { - assert contextProvider != null; - boolean added = getActiveNodesForCurrentCluster().add(this); - assert added; - } - private Set getActiveNodesForCurrentCluster() { return activeNodesPerCluster.computeIfAbsent(getClusterName(), clusterName -> ConcurrentCollections.newConcurrentSet()); } @Override - protected void doStop() { + public void close() { boolean found = getActiveNodesForCurrentCluster().remove(this); assert found; } - - @Override - protected void doClose() { - - } - - public static class TestPlugin extends Plugin implements DiscoveryPlugin { - - public void onModule(DiscoveryModule discoveryModule) { - discoveryModule.addZenPing(MockZenPing.class); - } - } } From 9015062dcb1b89f3dae7157a9544c2c7ac6536db Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Thu, 3 Nov 2016 16:48:55 +0100 Subject: [PATCH 122/132] Rewrite Queries/Filter in FilterAggregationBuilder and ensure client usage marks query as non-cachable (#21303) `FilterAggregationBuilder` today misses to rewrite queries which causes failures if a query that uses a client for instance to lookup terms since it must be rewritten first. This change also ensures that if a client is used from the rewrite context we mark the query as non-cacheable. Closes #21301 --- .../index/query/QueryRewriteContext.java | 2 +- .../index/query/QueryShardContext.java | 5 ++ .../filter/FilterAggregationBuilder.java | 4 +- .../query/MoreLikeThisQueryBuilderTests.java | 5 ++ .../index/query/TermsQueryBuilderTests.java | 7 ++ .../indices/IndicesRequestCacheIT.java | 6 +- .../test/search.aggregation/50_filter.yaml | 80 +++++++++++++++++++ 7 files changed, 104 insertions(+), 5 deletions(-) create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/50_filter.yaml diff --git a/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java b/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java index c8b5cf174f5..b569c36ed8f 100644 --- a/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java +++ b/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java @@ -67,7 +67,7 @@ public class QueryRewriteContext implements ParseFieldMatcherSupplier { /** * Returns a clients to fetch resources from local or remove nodes. */ - public final Client getClient() { + public Client getClient() { return client; } diff --git a/core/src/main/java/org/elasticsearch/index/query/QueryShardContext.java b/core/src/main/java/org/elasticsearch/index/query/QueryShardContext.java index 60befb9577a..4ba49e5f0e9 100644 --- a/core/src/main/java/org/elasticsearch/index/query/QueryShardContext.java +++ b/core/src/main/java/org/elasticsearch/index/query/QueryShardContext.java @@ -421,4 +421,9 @@ public class QueryShardContext extends QueryRewriteContext { return super.nowInMillis(); } + @Override + public Client getClient() { + failIfFrozen(); // we somebody uses a terms filter with lookup for instance can't be cached... + return super.getClient(); + } } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregationBuilder.java index 48be5365bb1..78d19280ce2 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregationBuilder.java @@ -72,7 +72,9 @@ public class FilterAggregationBuilder extends AbstractAggregationBuilder doBuild(AggregationContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder) throws IOException { - return new FilterAggregatorFactory(name, type, filter, context, parent, subFactoriesBuilder, metaData); + // TODO this sucks we need a rewrite phase for aggregations too + final QueryBuilder rewrittenFilter = QueryBuilder.rewriteQuery(filter, context.searchContext().getQueryShardContext()); + return new FilterAggregatorFactory(name, type, rewrittenFilter, context, parent, subFactoriesBuilder, metaData); } @Override diff --git a/core/src/test/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilderTests.java b/core/src/test/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilderTests.java index 7d35df1fb02..c0900de4de1 100644 --- a/core/src/test/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilderTests.java @@ -301,6 +301,11 @@ public class MoreLikeThisQueryBuilderTests extends AbstractQueryTestCase Date: Thu, 3 Nov 2016 18:05:18 +0100 Subject: [PATCH 123/132] Read indices options in indices upgrade API (#21281) With #21099 we removed support for the ignored allow_no_indices parameter in indices upgrade API. Truth is that ignore_unavailable and expand_wildcards were also ignored, in indices upgrade as well as upgrade status API. Those parameters are though supported internally and settable through java API, hence they should be all supported on the REST layer too. --- .../admin/indices/RestUpgradeAction.java | 24 ++++---- .../rest-api-spec/api/indices.upgrade.json | 4 ++ .../test/indices.upgrade/10_basic.yaml | 60 +++++++++++++++++-- 3 files changed, 73 insertions(+), 15 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/rest/action/admin/indices/RestUpgradeAction.java b/core/src/main/java/org/elasticsearch/rest/action/admin/indices/RestUpgradeAction.java index afde577de1e..9882b5bea3d 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/admin/indices/RestUpgradeAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/admin/indices/RestUpgradeAction.java @@ -20,9 +20,11 @@ package org.elasticsearch.rest.action.admin.indices; import org.elasticsearch.Version; +import org.elasticsearch.action.admin.indices.upgrade.get.UpgradeStatusRequest; import org.elasticsearch.action.admin.indices.upgrade.get.UpgradeStatusResponse; import org.elasticsearch.action.admin.indices.upgrade.post.UpgradeRequest; import org.elasticsearch.action.admin.indices.upgrade.post.UpgradeResponse; +import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.Tuple; @@ -68,20 +70,22 @@ public class RestUpgradeAction extends BaseRestHandler { } private RestChannelConsumer handleGet(final RestRequest request, NodeClient client) { - return channel -> client.admin().indices().prepareUpgradeStatus(Strings.splitStringByCommaToArray(request.param("index"))) - .execute(new RestBuilderListener(channel) { - @Override - public RestResponse buildResponse(UpgradeStatusResponse response, XContentBuilder builder) throws Exception { - builder.startObject(); - response.toXContent(builder, request); - builder.endObject(); - return new BytesRestResponse(OK, builder); - } - }); + UpgradeStatusRequest statusRequest = new UpgradeStatusRequest(Strings.splitStringByCommaToArray(request.param("index"))); + statusRequest.indicesOptions(IndicesOptions.fromRequest(request, statusRequest.indicesOptions())); + return channel -> client.admin().indices().upgradeStatus(statusRequest, new RestBuilderListener(channel) { + @Override + public RestResponse buildResponse(UpgradeStatusResponse response, XContentBuilder builder) throws Exception { + builder.startObject(); + response.toXContent(builder, request); + builder.endObject(); + return new BytesRestResponse(OK, builder); + } + }); } private RestChannelConsumer handlePost(final RestRequest request, NodeClient client) { UpgradeRequest upgradeReq = new UpgradeRequest(Strings.splitStringByCommaToArray(request.param("index"))); + upgradeReq.indicesOptions(IndicesOptions.fromRequest(request, upgradeReq.indicesOptions())); upgradeReq.upgradeOnlyAncientSegments(request.paramAsBoolean("only_ancient_segments", false)); return channel -> client.admin().indices().upgrade(upgradeReq, new RestBuilderListener(channel) { @Override diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.upgrade.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.upgrade.json index f83cf255165..1e2413ee723 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.upgrade.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.upgrade.json @@ -12,6 +12,10 @@ } }, "params": { + "allow_no_indices": { + "type" : "boolean", + "description" : "Whether to ignore if a wildcard indices expression resolves into no concrete indices. (This includes `_all` string or when no indices have been specified)" + }, "expand_wildcards": { "type" : "enum", "options" : ["open","closed","none","all"], diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.upgrade/10_basic.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.upgrade/10_basic.yaml index e696a5600bc..558e014e250 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.upgrade/10_basic.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.upgrade/10_basic.yaml @@ -9,14 +9,64 @@ index: number_of_replicas: 0 - - - do: - cluster.health: - wait_for_status: green - - do: indices.upgrade: index: test_index - match: {upgraded_indices.test_index.oldest_lucene_segment_version: '/(\d\.)+\d/'} - is_true: upgraded_indices.test_index.upgrade_version + +--- +"Upgrade indices ignore unavailable": + + - do: + indices.create: + index: test_index + body: + settings: + index: + number_of_shards: 1 + number_of_replicas: 0 + + - do: + indices.upgrade: + index: ["does_not_exist", "test_index"] + ignore_unavailable: true + + - match: {_shards.total: 1} + - is_true: upgraded_indices.test_index.upgrade_version + - is_false: upgraded_indices.does_not_exist + +--- +"Upgrade indices allow no indices": + + - do: + indices.upgrade: + index: test_index + ignore_unavailable: true + allow_no_indices: true + + - match: {_shards.total: 0} + +--- +"Upgrade indices disallow no indices": + + - do: + catch: missing + indices.upgrade: + index: test_index + ignore_unavailable: true + allow_no_indices: false + +--- +"Upgrade indices disallow unavailable": + - do: + indices.create: + index: test_index + + - do: + catch: missing + indices.upgrade: + index: ["test_index", "does_not_exist"] + ignore_unavailable: false + From 39f4229594f30ed82110b269e0df5a65bffc7b11 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 3 Nov 2016 18:36:07 +0100 Subject: [PATCH 124/132] Add information about in-flight requests when checking IndexShard operation counter (#21308) Our test infrastructure checks after running each test that there are no more in-flight requests on the shard level. Whenever the check fails, we only know that there were in-flight requests but don't know what requests were causing this issue. This commit adds the replication tasks that are still active at that moment to the assertion error. --- .../DiscoveryWithServiceDisruptionsIT.java | 2 +- .../elasticsearch/test/ESIntegTestCase.java | 2 +- .../test/InternalTestCluster.java | 29 +++++++++++++++++-- .../org/elasticsearch/test/TestCluster.java | 2 +- 4 files changed, 29 insertions(+), 6 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java b/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java index bbde01a6686..ca4b1c9b120 100644 --- a/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java +++ b/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java @@ -173,7 +173,7 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase { } @Override - protected void beforeIndexDeletion() { + protected void beforeIndexDeletion() throws IOException { if (disableBeforeIndexDeletion == false) { super.beforeIndexDeletion(); } diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index 8def9deda3c..7f0af14f93a 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -575,7 +575,7 @@ public abstract class ESIntegTestCase extends ESTestCase { return Collections.emptySet(); } - protected void beforeIndexDeletion() { + protected void beforeIndexDeletion() throws IOException { cluster().beforeIndexDeletion(); } diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java index 360a94c0a6d..6840eb7faec 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -31,8 +31,10 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.cluster.node.stats.NodeStats; +import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse; import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags; import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags.Flag; +import org.elasticsearch.action.support.replication.ReplicationTask; import org.elasticsearch.client.Client; import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.cluster.ClusterName; @@ -63,6 +65,8 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.PageCacheRecycler; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; @@ -87,6 +91,8 @@ import org.elasticsearch.node.service.NodeService; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.script.ScriptService; import org.elasticsearch.search.SearchService; +import org.elasticsearch.tasks.TaskInfo; +import org.elasticsearch.tasks.TaskManager; import org.elasticsearch.test.disruption.ServiceDisruptionScheme; import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.transport.MockTransportClient; @@ -1013,7 +1019,7 @@ public final class InternalTestCluster extends TestCluster { } @Override - public void beforeIndexDeletion() { + public void beforeIndexDeletion() throws IOException { // Check that the operations counter on index shard has reached 0. // The assumption here is that after a test there are no ongoing write operations. // test that have ongoing write operations after the test (for example because ttl is used @@ -1048,13 +1054,30 @@ public final class InternalTestCluster extends TestCluster { } } - private void assertShardIndexCounter() { + private void assertShardIndexCounter() throws IOException { final Collection nodesAndClients = nodes.values(); for (NodeAndClient nodeAndClient : nodesAndClients) { IndicesService indexServices = getInstance(IndicesService.class, nodeAndClient.name); for (IndexService indexService : indexServices) { for (IndexShard indexShard : indexService) { - assertThat("index shard counter on shard " + indexShard.shardId() + " on node " + nodeAndClient.name + " not 0", indexShard.getActiveOperationsCount(), equalTo(0)); + int activeOperationsCount = indexShard.getActiveOperationsCount(); + if (activeOperationsCount > 0) { + TaskManager taskManager = getInstance(TransportService.class, nodeAndClient.name).getTaskManager(); + DiscoveryNode localNode = getInstance(ClusterService.class, nodeAndClient.name).localNode(); + List taskInfos = taskManager.getTasks().values().stream() + .filter(task -> task instanceof ReplicationTask) + .map(task -> task.taskInfo(localNode, true)) + .collect(Collectors.toList()); + ListTasksResponse response = new ListTasksResponse(taskInfos, Collections.emptyList(), Collections.emptyList()); + XContentBuilder builder = XContentFactory.jsonBuilder() + .prettyPrint() + .startObject() + .value(response) + .endObject(); + throw new AssertionError("expected index shard counter on shard " + indexShard.shardId() + " on node " + + nodeAndClient.name + " to be 0 but was " + activeOperationsCount + ". Current replication tasks on node:\n" + + builder.string()); + } } } } diff --git a/test/framework/src/main/java/org/elasticsearch/test/TestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/TestCluster.java index 124960fe921..b960685777e 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/TestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/TestCluster.java @@ -82,7 +82,7 @@ public abstract class TestCluster implements Closeable { /** * Assertions that should run before the cluster is wiped should be called in this method */ - public void beforeIndexDeletion() { + public void beforeIndexDeletion() throws IOException { } /** From e6462ea4ca8d9eee8cb7aa59feac368686cdbe33 Mon Sep 17 00:00:00 2001 From: Chris Earle Date: Thu, 3 Nov 2016 15:37:37 -0400 Subject: [PATCH 125/132] [DOCS] "service" should be "elasticsearch-service" (#21311) The Windows `service.bat` script was renamed to `elasticsearch-service.bat` in ES 5.0+. --- docs/reference/setup/install/windows.asciidoc | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/reference/setup/install/windows.asciidoc b/docs/reference/setup/install/windows.asciidoc index 691794e42b5..c9d4c4f81c5 100644 --- a/docs/reference/setup/install/windows.asciidoc +++ b/docs/reference/setup/install/windows.asciidoc @@ -29,11 +29,11 @@ Download the `.zip` archive for Elasticsearch v{version} from: https://artifacts Unzip it with your favourite unzip tool. This will create a folder called +elasticsearch-{version}+, which we will refer to as `%ES_HOME%`. In a terminal -window, `CD` to the `%ES_HOME%` directory, for instance: +window, `cd` to the `%ES_HOME%` directory, for instance: ["source","sh",subs="attributes"] ---------------------------- -CD c:\elasticsearch-{version} +cd c:\elasticsearch-{version} ---------------------------- endif::[] @@ -85,7 +85,7 @@ stop the service, all from the command-line. ["source","sh",subs="attributes,callouts"] -------------------------------------------------- -c:\elasticsearch-{version}{backslash}bin>service +c:\elasticsearch-{version}{backslash}bin>elasticsearch-service Usage: elasticsearch-service.bat install|remove|start|stop|manager [SERVICE_ID] -------------------------------------------------- @@ -113,7 +113,7 @@ information is made available during install: ["source","sh",subs="attributes"] -------------------------------------------------- -c:\elasticsearch-{version}{backslash}bin>service install +c:\elasticsearch-{version}{backslash}bin>elasticsearch-service install Installing service : "elasticsearch-service-x64" Using JAVA_HOME (64-bit): "c:\jvm\jdk1.8" The service 'elasticsearch-service-x64' has been installed. From 68bed9cd3be61d1e226a2876d3650ba67e6e1fe4 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Thu, 3 Nov 2016 21:43:07 +0100 Subject: [PATCH 126/132] [TEST] Skip new tests using ingnore_unavailable in BWC tests against 5.0.0 5.0.0 nodes don't understand these parameters and will throw an exception if a 5.0.0 node will receive the request. Relates to #21281 --- .../test/indices.upgrade/10_basic.yaml | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.upgrade/10_basic.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.upgrade/10_basic.yaml index 558e014e250..9d2245b4b40 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.upgrade/10_basic.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.upgrade/10_basic.yaml @@ -18,6 +18,9 @@ --- "Upgrade indices ignore unavailable": + - skip: + version: " - 5.0.0" + reason: ignore_unavailable was added as a bugfix in 5.0.1 see #21281 - do: indices.create: @@ -40,6 +43,10 @@ --- "Upgrade indices allow no indices": + - skip: + version: " - 5.0.0" + reason: ignore_unavailable was added as a bugfix in 5.0.1 see #21281 + - do: indices.upgrade: index: test_index @@ -51,6 +58,10 @@ --- "Upgrade indices disallow no indices": + - skip: + version: " - 5.0.0" + reason: ignore_unavailable was added as a bugfix in 5.0.1 see #21281 + - do: catch: missing indices.upgrade: @@ -60,6 +71,11 @@ --- "Upgrade indices disallow unavailable": + + - skip: + version: " - 5.0.0" + reason: ignore_unavailable was added as a bugfix in 5.0.1 see #21281 + - do: indices.create: index: test_index From c3e176908cd083d37a4970d433047c6b5938f081 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 3 Nov 2016 20:40:39 -0400 Subject: [PATCH 127/132] Fix ShardInfo#toString ShardInfo#toString resorts to calling ShardInfo#toXContent via Strings#toString. However, the resulting XContent object will not start with an object, and this is a violation of the generator state machine. This commit fixes this issue by replacing the override of toString to provide simple non-toXContent output of the ShardInfo instance. Without this fix, ShardInfo#toString will instead produce "Error building toString out of XContent: com.fasterxml.jackson.core.JsonGenerationException: Can not write a field name, expecting a value" Relates #21319 --- .../replication/ReplicationResponse.java | 8 +++- .../replication/ReplicationResponseTests.java | 39 +++++++++++++++++++ 2 files changed, 45 insertions(+), 2 deletions(-) create mode 100644 core/src/test/java/org/elasticsearch/action/support/replication/ReplicationResponseTests.java diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationResponse.java b/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationResponse.java index e6ce0a5aad4..98556494191 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationResponse.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationResponse.java @@ -24,7 +24,6 @@ import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Streamable; @@ -34,6 +33,7 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.rest.RestStatus; import java.io.IOException; +import java.util.Arrays; /** * Base class for write action responses. @@ -162,7 +162,11 @@ public class ReplicationResponse extends ActionResponse { @Override public String toString() { - return Strings.toString(this); + return "ShardInfo{" + + "total=" + total + + ", successful=" + successful + + ", failures=" + Arrays.toString(failures) + + '}'; } public static ShardInfo readShardInfo(StreamInput in) throws IOException { diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/ReplicationResponseTests.java b/core/src/test/java/org/elasticsearch/action/support/replication/ReplicationResponseTests.java new file mode 100644 index 00000000000..3740f8dd5f7 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/action/support/replication/ReplicationResponseTests.java @@ -0,0 +1,39 @@ +/* + * 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.action.support.replication; + +import org.elasticsearch.test.ESTestCase; + +import java.util.Locale; + +import static org.hamcrest.CoreMatchers.equalTo; + +public class ReplicationResponseTests extends ESTestCase { + + public void testShardInfoToString() { + final int total = 5; + final int successful = randomIntBetween(1, total); + final ReplicationResponse.ShardInfo shardInfo = new ReplicationResponse.ShardInfo(total, successful); + assertThat( + shardInfo.toString(), + equalTo(String.format(Locale.ROOT, "ShardInfo{total=5, successful=%d, failures=[]}", successful))); + } + +} From 89434214940cd21e80b1b3eaa194baa1c30b59a1 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Thu, 3 Nov 2016 21:47:11 -0400 Subject: [PATCH 128/132] Only log rest connection setup once per suite (#21280) This is a bit funky to do with junit because we need per test state but we only want to log it per suite. So we use a static flag that we test per test and reset before every suite. --- .../test/rest/yaml/ClientYamlTestClient.java | 20 +++++++++++++++++-- 1 file changed, 18 insertions(+), 2 deletions(-) diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestClient.java b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestClient.java index 62b09b3ecbf..da8c54396df 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestClient.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestClient.java @@ -19,6 +19,7 @@ package org.elasticsearch.test.rest.yaml; import com.carrotsearch.randomizedtesting.RandomizedTest; + import org.apache.http.Header; import org.apache.http.HttpEntity; import org.apache.http.HttpHost; @@ -37,6 +38,7 @@ import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.test.rest.yaml.restspec.ClientYamlSuiteRestApi; import org.elasticsearch.test.rest.yaml.restspec.ClientYamlSuiteRestPath; import org.elasticsearch.test.rest.yaml.restspec.ClientYamlSuiteRestSpec; +import org.junit.BeforeClass; import java.io.IOException; import java.net.URI; @@ -59,6 +61,8 @@ public class ClientYamlTestClient { //query_string params that don't need to be declared in the spec, they are supported by default private static final Set ALWAYS_ACCEPTED_QUERY_STRING_PARAMS = Sets.newHashSet("pretty", "source", "filter_path"); + private static boolean loggedInit = false; + private final ClientYamlSuiteRestSpec restSpec; private final RestClient restClient; private final Version esVersion; @@ -70,8 +74,20 @@ public class ClientYamlTestClient { Tuple versionTuple = readMasterAndMinNodeVersion(hosts.size()); this.esVersion = versionTuple.v1(); Version masterVersion = versionTuple.v2(); - // this will be logged in each test such that if something fails we get it in the logs for each test - logger.info("initializing client, minimum es version: [{}] master version: [{}] hosts: {}", esVersion, masterVersion, hosts); + if (false == loggedInit) { + /* This will be logged once per suite which lines up with randomized runner's dumping the output of all failing suites. It'd + * be super noisy to log this once per test. We can't log it in a @BeforeClass method because we need the class variables. */ + logger.info("initializing client, minimum es version: [{}] master version: [{}] hosts: {}", esVersion, masterVersion, hosts); + loggedInit = true; + } + } + + /** + * Reset {@link #loggedInit} so we log the connection setup before this suite. + */ + @BeforeClass + public static void clearLoggedInit() { + loggedInit = false; } private Tuple readMasterAndMinNodeVersion(int numHosts) throws IOException { From 2a70f6e7b1be7fd2f3efb758ff2028e64e64076b Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Fri, 4 Nov 2016 10:34:04 +0100 Subject: [PATCH 129/132] Upgrade to lucene-6.3.0-snapshot-a66a445. (#21309) This addresses a bug that was introduced with https://issues.apache.org/jira/browse/LUCENE-7501. --- buildSrc/version.properties | 2 +- .../resources/org/elasticsearch/bootstrap/security.policy | 4 ++-- .../org/elasticsearch/bootstrap/test-framework.policy | 2 +- .../org/elasticsearch/index/engine/InternalEngineTests.java | 1 - .../lucene-analyzers-common-6.3.0-snapshot-a66a445.jar.sha1 | 1 + .../lucene-analyzers-common-6.3.0-snapshot-ed102d6.jar.sha1 | 1 - .../lucene-backward-codecs-6.3.0-snapshot-a66a445.jar.sha1 | 1 + .../lucene-backward-codecs-6.3.0-snapshot-ed102d6.jar.sha1 | 1 - .../licenses/lucene-core-6.3.0-snapshot-a66a445.jar.sha1 | 1 + .../licenses/lucene-core-6.3.0-snapshot-ed102d6.jar.sha1 | 1 - .../licenses/lucene-grouping-6.3.0-snapshot-a66a445.jar.sha1 | 1 + .../licenses/lucene-grouping-6.3.0-snapshot-ed102d6.jar.sha1 | 1 - .../lucene-highlighter-6.3.0-snapshot-a66a445.jar.sha1 | 1 + .../lucene-highlighter-6.3.0-snapshot-ed102d6.jar.sha1 | 1 - .../licenses/lucene-join-6.3.0-snapshot-a66a445.jar.sha1 | 1 + .../licenses/lucene-join-6.3.0-snapshot-ed102d6.jar.sha1 | 1 - .../licenses/lucene-memory-6.3.0-snapshot-a66a445.jar.sha1 | 1 + .../licenses/lucene-memory-6.3.0-snapshot-ed102d6.jar.sha1 | 1 - .../licenses/lucene-misc-6.3.0-snapshot-a66a445.jar.sha1 | 1 + .../licenses/lucene-misc-6.3.0-snapshot-ed102d6.jar.sha1 | 1 - .../licenses/lucene-queries-6.3.0-snapshot-a66a445.jar.sha1 | 1 + .../licenses/lucene-queries-6.3.0-snapshot-ed102d6.jar.sha1 | 1 - .../lucene-queryparser-6.3.0-snapshot-a66a445.jar.sha1 | 1 + .../lucene-queryparser-6.3.0-snapshot-ed102d6.jar.sha1 | 1 - .../licenses/lucene-sandbox-6.3.0-snapshot-a66a445.jar.sha1 | 1 + .../licenses/lucene-sandbox-6.3.0-snapshot-ed102d6.jar.sha1 | 1 - .../licenses/lucene-spatial-6.3.0-snapshot-a66a445.jar.sha1 | 1 + .../licenses/lucene-spatial-6.3.0-snapshot-ed102d6.jar.sha1 | 1 - .../lucene-spatial-extras-6.3.0-snapshot-a66a445.jar.sha1 | 1 + .../lucene-spatial-extras-6.3.0-snapshot-ed102d6.jar.sha1 | 1 - .../licenses/lucene-spatial3d-6.3.0-snapshot-a66a445.jar.sha1 | 1 + .../licenses/lucene-spatial3d-6.3.0-snapshot-ed102d6.jar.sha1 | 1 - .../licenses/lucene-suggest-6.3.0-snapshot-a66a445.jar.sha1 | 1 + .../licenses/lucene-suggest-6.3.0-snapshot-ed102d6.jar.sha1 | 1 - .../lucene-expressions-6.3.0-snapshot-a66a445.jar.sha1 | 1 + .../lucene-expressions-6.3.0-snapshot-ed102d6.jar.sha1 | 1 - .../lucene-analyzers-icu-6.3.0-snapshot-a66a445.jar.sha1 | 1 + .../lucene-analyzers-icu-6.3.0-snapshot-ed102d6.jar.sha1 | 1 - .../lucene-analyzers-kuromoji-6.3.0-snapshot-a66a445.jar.sha1 | 1 + .../lucene-analyzers-kuromoji-6.3.0-snapshot-ed102d6.jar.sha1 | 1 - .../lucene-analyzers-phonetic-6.3.0-snapshot-a66a445.jar.sha1 | 1 + .../lucene-analyzers-phonetic-6.3.0-snapshot-ed102d6.jar.sha1 | 1 - .../lucene-analyzers-smartcn-6.3.0-snapshot-a66a445.jar.sha1 | 1 + .../lucene-analyzers-smartcn-6.3.0-snapshot-ed102d6.jar.sha1 | 1 - .../lucene-analyzers-stempel-6.3.0-snapshot-a66a445.jar.sha1 | 1 + .../lucene-analyzers-stempel-6.3.0-snapshot-ed102d6.jar.sha1 | 1 - ...ucene-analyzers-morfologik-6.3.0-snapshot-a66a445.jar.sha1 | 1 + ...ucene-analyzers-morfologik-6.3.0-snapshot-ed102d6.jar.sha1 | 1 - 48 files changed, 26 insertions(+), 27 deletions(-) create mode 100644 distribution/licenses/lucene-analyzers-common-6.3.0-snapshot-a66a445.jar.sha1 delete mode 100644 distribution/licenses/lucene-analyzers-common-6.3.0-snapshot-ed102d6.jar.sha1 create mode 100644 distribution/licenses/lucene-backward-codecs-6.3.0-snapshot-a66a445.jar.sha1 delete mode 100644 distribution/licenses/lucene-backward-codecs-6.3.0-snapshot-ed102d6.jar.sha1 create mode 100644 distribution/licenses/lucene-core-6.3.0-snapshot-a66a445.jar.sha1 delete mode 100644 distribution/licenses/lucene-core-6.3.0-snapshot-ed102d6.jar.sha1 create mode 100644 distribution/licenses/lucene-grouping-6.3.0-snapshot-a66a445.jar.sha1 delete mode 100644 distribution/licenses/lucene-grouping-6.3.0-snapshot-ed102d6.jar.sha1 create mode 100644 distribution/licenses/lucene-highlighter-6.3.0-snapshot-a66a445.jar.sha1 delete mode 100644 distribution/licenses/lucene-highlighter-6.3.0-snapshot-ed102d6.jar.sha1 create mode 100644 distribution/licenses/lucene-join-6.3.0-snapshot-a66a445.jar.sha1 delete mode 100644 distribution/licenses/lucene-join-6.3.0-snapshot-ed102d6.jar.sha1 create mode 100644 distribution/licenses/lucene-memory-6.3.0-snapshot-a66a445.jar.sha1 delete mode 100644 distribution/licenses/lucene-memory-6.3.0-snapshot-ed102d6.jar.sha1 create mode 100644 distribution/licenses/lucene-misc-6.3.0-snapshot-a66a445.jar.sha1 delete mode 100644 distribution/licenses/lucene-misc-6.3.0-snapshot-ed102d6.jar.sha1 create mode 100644 distribution/licenses/lucene-queries-6.3.0-snapshot-a66a445.jar.sha1 delete mode 100644 distribution/licenses/lucene-queries-6.3.0-snapshot-ed102d6.jar.sha1 create mode 100644 distribution/licenses/lucene-queryparser-6.3.0-snapshot-a66a445.jar.sha1 delete mode 100644 distribution/licenses/lucene-queryparser-6.3.0-snapshot-ed102d6.jar.sha1 create mode 100644 distribution/licenses/lucene-sandbox-6.3.0-snapshot-a66a445.jar.sha1 delete mode 100644 distribution/licenses/lucene-sandbox-6.3.0-snapshot-ed102d6.jar.sha1 create mode 100644 distribution/licenses/lucene-spatial-6.3.0-snapshot-a66a445.jar.sha1 delete mode 100644 distribution/licenses/lucene-spatial-6.3.0-snapshot-ed102d6.jar.sha1 create mode 100644 distribution/licenses/lucene-spatial-extras-6.3.0-snapshot-a66a445.jar.sha1 delete mode 100644 distribution/licenses/lucene-spatial-extras-6.3.0-snapshot-ed102d6.jar.sha1 create mode 100644 distribution/licenses/lucene-spatial3d-6.3.0-snapshot-a66a445.jar.sha1 delete mode 100644 distribution/licenses/lucene-spatial3d-6.3.0-snapshot-ed102d6.jar.sha1 create mode 100644 distribution/licenses/lucene-suggest-6.3.0-snapshot-a66a445.jar.sha1 delete mode 100644 distribution/licenses/lucene-suggest-6.3.0-snapshot-ed102d6.jar.sha1 create mode 100644 modules/lang-expression/licenses/lucene-expressions-6.3.0-snapshot-a66a445.jar.sha1 delete mode 100644 modules/lang-expression/licenses/lucene-expressions-6.3.0-snapshot-ed102d6.jar.sha1 create mode 100644 plugins/analysis-icu/licenses/lucene-analyzers-icu-6.3.0-snapshot-a66a445.jar.sha1 delete mode 100644 plugins/analysis-icu/licenses/lucene-analyzers-icu-6.3.0-snapshot-ed102d6.jar.sha1 create mode 100644 plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-6.3.0-snapshot-a66a445.jar.sha1 delete mode 100644 plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-6.3.0-snapshot-ed102d6.jar.sha1 create mode 100644 plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-6.3.0-snapshot-a66a445.jar.sha1 delete mode 100644 plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-6.3.0-snapshot-ed102d6.jar.sha1 create mode 100644 plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-6.3.0-snapshot-a66a445.jar.sha1 delete mode 100644 plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-6.3.0-snapshot-ed102d6.jar.sha1 create mode 100644 plugins/analysis-stempel/licenses/lucene-analyzers-stempel-6.3.0-snapshot-a66a445.jar.sha1 delete mode 100644 plugins/analysis-stempel/licenses/lucene-analyzers-stempel-6.3.0-snapshot-ed102d6.jar.sha1 create mode 100644 plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-6.3.0-snapshot-a66a445.jar.sha1 delete mode 100644 plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-6.3.0-snapshot-ed102d6.jar.sha1 diff --git a/buildSrc/version.properties b/buildSrc/version.properties index 7677e8e24bb..bbf4170591d 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -1,5 +1,5 @@ elasticsearch = 6.0.0-alpha1 -lucene = 6.3.0-snapshot-ed102d6 +lucene = 6.3.0-snapshot-a66a445 # optional dependencies spatial4j = 0.6 diff --git a/core/src/main/resources/org/elasticsearch/bootstrap/security.policy b/core/src/main/resources/org/elasticsearch/bootstrap/security.policy index 310d485f98d..999f036d9f4 100644 --- a/core/src/main/resources/org/elasticsearch/bootstrap/security.policy +++ b/core/src/main/resources/org/elasticsearch/bootstrap/security.policy @@ -31,7 +31,7 @@ grant codeBase "${codebase.securesm-1.1.jar}" { //// Very special jar permissions: //// These are dangerous permissions that we don't want to grant to everything. -grant codeBase "${codebase.lucene-core-6.3.0-snapshot-ed102d6.jar}" { +grant codeBase "${codebase.lucene-core-6.3.0-snapshot-a66a445.jar}" { // needed to allow MMapDirectory's "unmap hack" (die unmap hack, die) // java 8 package permission java.lang.RuntimePermission "accessClassInPackage.sun.misc"; @@ -42,7 +42,7 @@ grant codeBase "${codebase.lucene-core-6.3.0-snapshot-ed102d6.jar}" { permission java.lang.RuntimePermission "accessDeclaredMembers"; }; -grant codeBase "${codebase.lucene-misc-6.3.0-snapshot-ed102d6.jar}" { +grant codeBase "${codebase.lucene-misc-6.3.0-snapshot-a66a445.jar}" { // needed to allow shard shrinking to use hard-links if possible via lucenes HardlinkCopyDirectoryWrapper permission java.nio.file.LinkPermission "hard"; }; diff --git a/core/src/main/resources/org/elasticsearch/bootstrap/test-framework.policy b/core/src/main/resources/org/elasticsearch/bootstrap/test-framework.policy index 2c1a963e296..1c780f96933 100644 --- a/core/src/main/resources/org/elasticsearch/bootstrap/test-framework.policy +++ b/core/src/main/resources/org/elasticsearch/bootstrap/test-framework.policy @@ -33,7 +33,7 @@ grant codeBase "${codebase.securemock-1.2.jar}" { permission java.lang.reflect.ReflectPermission "suppressAccessChecks"; }; -grant codeBase "${codebase.lucene-test-framework-6.3.0-snapshot-ed102d6.jar}" { +grant codeBase "${codebase.lucene-test-framework-6.3.0-snapshot-a66a445.jar}" { // needed by RamUsageTester permission java.lang.reflect.ReflectPermission "suppressAccessChecks"; // needed for testing hardlinks in StoreRecoveryTests since we install MockFS diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index f8621263f3f..fcc7db73a65 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -1791,7 +1791,6 @@ public class InternalEngineTests extends ESTestCase { return new Mapping(Version.CURRENT, root, new MetadataFieldMapper[0], emptyMap()); } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/21147") public void testUpgradeOldIndex() throws IOException { List indexes = new ArrayList<>(); try (DirectoryStream stream = Files.newDirectoryStream(getBwcIndicesPath(), "index-*.zip")) { diff --git a/distribution/licenses/lucene-analyzers-common-6.3.0-snapshot-a66a445.jar.sha1 b/distribution/licenses/lucene-analyzers-common-6.3.0-snapshot-a66a445.jar.sha1 new file mode 100644 index 00000000000..1626a88f4a2 --- /dev/null +++ b/distribution/licenses/lucene-analyzers-common-6.3.0-snapshot-a66a445.jar.sha1 @@ -0,0 +1 @@ +61aacb657e44a9beabf95834e106bbb96373a703 \ No newline at end of file diff --git a/distribution/licenses/lucene-analyzers-common-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-analyzers-common-6.3.0-snapshot-ed102d6.jar.sha1 deleted file mode 100644 index 7947a57e315..00000000000 --- a/distribution/licenses/lucene-analyzers-common-6.3.0-snapshot-ed102d6.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -6a96375ee285920400266dfc411342f2e3d3d623 \ No newline at end of file diff --git a/distribution/licenses/lucene-backward-codecs-6.3.0-snapshot-a66a445.jar.sha1 b/distribution/licenses/lucene-backward-codecs-6.3.0-snapshot-a66a445.jar.sha1 new file mode 100644 index 00000000000..2f45d50eeee --- /dev/null +++ b/distribution/licenses/lucene-backward-codecs-6.3.0-snapshot-a66a445.jar.sha1 @@ -0,0 +1 @@ +600de75a81e259cab0384e546d9a1d527ddba6d6 \ No newline at end of file diff --git a/distribution/licenses/lucene-backward-codecs-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-backward-codecs-6.3.0-snapshot-ed102d6.jar.sha1 deleted file mode 100644 index e395661bace..00000000000 --- a/distribution/licenses/lucene-backward-codecs-6.3.0-snapshot-ed102d6.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -aa0403f90ac23f50667477b314175e58175c8067 \ No newline at end of file diff --git a/distribution/licenses/lucene-core-6.3.0-snapshot-a66a445.jar.sha1 b/distribution/licenses/lucene-core-6.3.0-snapshot-a66a445.jar.sha1 new file mode 100644 index 00000000000..9dcdbeb40e9 --- /dev/null +++ b/distribution/licenses/lucene-core-6.3.0-snapshot-a66a445.jar.sha1 @@ -0,0 +1 @@ +188774468a56a8731ca639527d721060d26ffebd \ No newline at end of file diff --git a/distribution/licenses/lucene-core-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-core-6.3.0-snapshot-ed102d6.jar.sha1 deleted file mode 100644 index 0ac4d342184..00000000000 --- a/distribution/licenses/lucene-core-6.3.0-snapshot-ed102d6.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -6510e4e4af2b90a2d1e2e8c27e92120667c0bff8 \ No newline at end of file diff --git a/distribution/licenses/lucene-grouping-6.3.0-snapshot-a66a445.jar.sha1 b/distribution/licenses/lucene-grouping-6.3.0-snapshot-a66a445.jar.sha1 new file mode 100644 index 00000000000..14c8d7aa2b7 --- /dev/null +++ b/distribution/licenses/lucene-grouping-6.3.0-snapshot-a66a445.jar.sha1 @@ -0,0 +1 @@ +5afd9271e3d8f645440f48ff2487545ae5573e7e \ No newline at end of file diff --git a/distribution/licenses/lucene-grouping-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-grouping-6.3.0-snapshot-ed102d6.jar.sha1 deleted file mode 100644 index fddadafd79c..00000000000 --- a/distribution/licenses/lucene-grouping-6.3.0-snapshot-ed102d6.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -16fb10b3b006a4a7cb2ed9b436e59c91dc9edf47 \ No newline at end of file diff --git a/distribution/licenses/lucene-highlighter-6.3.0-snapshot-a66a445.jar.sha1 b/distribution/licenses/lucene-highlighter-6.3.0-snapshot-a66a445.jar.sha1 new file mode 100644 index 00000000000..e695284756d --- /dev/null +++ b/distribution/licenses/lucene-highlighter-6.3.0-snapshot-a66a445.jar.sha1 @@ -0,0 +1 @@ +0f575175e26d4d3b1095f6300cbefbbb3ee994cd \ No newline at end of file diff --git a/distribution/licenses/lucene-highlighter-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-highlighter-6.3.0-snapshot-ed102d6.jar.sha1 deleted file mode 100644 index 2e9d4b99abd..00000000000 --- a/distribution/licenses/lucene-highlighter-6.3.0-snapshot-ed102d6.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -47a3c05e1be7f593a0860c744c752807fb4d44fb \ No newline at end of file diff --git a/distribution/licenses/lucene-join-6.3.0-snapshot-a66a445.jar.sha1 b/distribution/licenses/lucene-join-6.3.0-snapshot-a66a445.jar.sha1 new file mode 100644 index 00000000000..ad02b0cac3b --- /dev/null +++ b/distribution/licenses/lucene-join-6.3.0-snapshot-a66a445.jar.sha1 @@ -0,0 +1 @@ +ee898c3d318681c9f29c56e6d9b52876be96d814 \ No newline at end of file diff --git a/distribution/licenses/lucene-join-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-join-6.3.0-snapshot-ed102d6.jar.sha1 deleted file mode 100644 index 4cf7214aa44..00000000000 --- a/distribution/licenses/lucene-join-6.3.0-snapshot-ed102d6.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -2bbe25506f997056555c8bfba64382663433517e \ No newline at end of file diff --git a/distribution/licenses/lucene-memory-6.3.0-snapshot-a66a445.jar.sha1 b/distribution/licenses/lucene-memory-6.3.0-snapshot-a66a445.jar.sha1 new file mode 100644 index 00000000000..0e36d650670 --- /dev/null +++ b/distribution/licenses/lucene-memory-6.3.0-snapshot-a66a445.jar.sha1 @@ -0,0 +1 @@ +ea6defd322456711394b4dabcda70a217e3caacd \ No newline at end of file diff --git a/distribution/licenses/lucene-memory-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-memory-6.3.0-snapshot-ed102d6.jar.sha1 deleted file mode 100644 index ea50210ceab..00000000000 --- a/distribution/licenses/lucene-memory-6.3.0-snapshot-ed102d6.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -76a1c370acfa5ad74beef243512a1dda64d01d79 \ No newline at end of file diff --git a/distribution/licenses/lucene-misc-6.3.0-snapshot-a66a445.jar.sha1 b/distribution/licenses/lucene-misc-6.3.0-snapshot-a66a445.jar.sha1 new file mode 100644 index 00000000000..e458570651a --- /dev/null +++ b/distribution/licenses/lucene-misc-6.3.0-snapshot-a66a445.jar.sha1 @@ -0,0 +1 @@ +ea2de7f9753a8e19a1ec9f25a3ea65d7ce909a0e \ No newline at end of file diff --git a/distribution/licenses/lucene-misc-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-misc-6.3.0-snapshot-ed102d6.jar.sha1 deleted file mode 100644 index 85a2e7bfd79..00000000000 --- a/distribution/licenses/lucene-misc-6.3.0-snapshot-ed102d6.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -ce459756eb7210c1bf7cd2a0cb25c7c5660de224 \ No newline at end of file diff --git a/distribution/licenses/lucene-queries-6.3.0-snapshot-a66a445.jar.sha1 b/distribution/licenses/lucene-queries-6.3.0-snapshot-a66a445.jar.sha1 new file mode 100644 index 00000000000..1231424e3be --- /dev/null +++ b/distribution/licenses/lucene-queries-6.3.0-snapshot-a66a445.jar.sha1 @@ -0,0 +1 @@ +0b15c6f29bfb9ec14a4615013a94bfa43a63793d \ No newline at end of file diff --git a/distribution/licenses/lucene-queries-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-queries-6.3.0-snapshot-ed102d6.jar.sha1 deleted file mode 100644 index 83e1c72fa8a..00000000000 --- a/distribution/licenses/lucene-queries-6.3.0-snapshot-ed102d6.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -1c27529566f2b8ab8856e8b0a4779ad2071588fc \ No newline at end of file diff --git a/distribution/licenses/lucene-queryparser-6.3.0-snapshot-a66a445.jar.sha1 b/distribution/licenses/lucene-queryparser-6.3.0-snapshot-a66a445.jar.sha1 new file mode 100644 index 00000000000..a367f4e45cf --- /dev/null +++ b/distribution/licenses/lucene-queryparser-6.3.0-snapshot-a66a445.jar.sha1 @@ -0,0 +1 @@ +d89d9fa1036c38144e0b8db079ae959353847c86 \ No newline at end of file diff --git a/distribution/licenses/lucene-queryparser-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-queryparser-6.3.0-snapshot-ed102d6.jar.sha1 deleted file mode 100644 index 3a2cbe25abf..00000000000 --- a/distribution/licenses/lucene-queryparser-6.3.0-snapshot-ed102d6.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -b5a9d1e55751dc193879c22a48823a863492bbe2 \ No newline at end of file diff --git a/distribution/licenses/lucene-sandbox-6.3.0-snapshot-a66a445.jar.sha1 b/distribution/licenses/lucene-sandbox-6.3.0-snapshot-a66a445.jar.sha1 new file mode 100644 index 00000000000..4c8874c0b4b --- /dev/null +++ b/distribution/licenses/lucene-sandbox-6.3.0-snapshot-a66a445.jar.sha1 @@ -0,0 +1 @@ +c003c1ab0a19a02b30156ce13372cff1001d6a7d \ No newline at end of file diff --git a/distribution/licenses/lucene-sandbox-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-sandbox-6.3.0-snapshot-ed102d6.jar.sha1 deleted file mode 100644 index b83229a5e93..00000000000 --- a/distribution/licenses/lucene-sandbox-6.3.0-snapshot-ed102d6.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -8d73ab3c889d386b3a594941992a5d942c57513a \ No newline at end of file diff --git a/distribution/licenses/lucene-spatial-6.3.0-snapshot-a66a445.jar.sha1 b/distribution/licenses/lucene-spatial-6.3.0-snapshot-a66a445.jar.sha1 new file mode 100644 index 00000000000..75dd8263828 --- /dev/null +++ b/distribution/licenses/lucene-spatial-6.3.0-snapshot-a66a445.jar.sha1 @@ -0,0 +1 @@ +a3c570bf588d7c9ca43d074db9ce9c9b8408b930 \ No newline at end of file diff --git a/distribution/licenses/lucene-spatial-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-spatial-6.3.0-snapshot-ed102d6.jar.sha1 deleted file mode 100644 index cfff1364101..00000000000 --- a/distribution/licenses/lucene-spatial-6.3.0-snapshot-ed102d6.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -85e74df9bc722f9a858775496cad7f35dce65aa0 \ No newline at end of file diff --git a/distribution/licenses/lucene-spatial-extras-6.3.0-snapshot-a66a445.jar.sha1 b/distribution/licenses/lucene-spatial-extras-6.3.0-snapshot-a66a445.jar.sha1 new file mode 100644 index 00000000000..debd8e0b873 --- /dev/null +++ b/distribution/licenses/lucene-spatial-extras-6.3.0-snapshot-a66a445.jar.sha1 @@ -0,0 +1 @@ +de54ca61f5892cf2c88ac083b3332a827beca7ff \ No newline at end of file diff --git a/distribution/licenses/lucene-spatial-extras-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-spatial-extras-6.3.0-snapshot-ed102d6.jar.sha1 deleted file mode 100644 index f3c30ce0275..00000000000 --- a/distribution/licenses/lucene-spatial-extras-6.3.0-snapshot-ed102d6.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -cc12ac933b39ef33e186e77f7f6af6bdc99f9108 \ No newline at end of file diff --git a/distribution/licenses/lucene-spatial3d-6.3.0-snapshot-a66a445.jar.sha1 b/distribution/licenses/lucene-spatial3d-6.3.0-snapshot-a66a445.jar.sha1 new file mode 100644 index 00000000000..b9eb9a0c270 --- /dev/null +++ b/distribution/licenses/lucene-spatial3d-6.3.0-snapshot-a66a445.jar.sha1 @@ -0,0 +1 @@ +cacdf81b324acd335be63798d5a3dd16e7dff9a3 \ No newline at end of file diff --git a/distribution/licenses/lucene-spatial3d-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-spatial3d-6.3.0-snapshot-ed102d6.jar.sha1 deleted file mode 100644 index 32f9ef0d6aa..00000000000 --- a/distribution/licenses/lucene-spatial3d-6.3.0-snapshot-ed102d6.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -8dfc7a42839db1e4ef69dbfc23d9ef2fb6c43ff6 \ No newline at end of file diff --git a/distribution/licenses/lucene-suggest-6.3.0-snapshot-a66a445.jar.sha1 b/distribution/licenses/lucene-suggest-6.3.0-snapshot-a66a445.jar.sha1 new file mode 100644 index 00000000000..a6517bc7d42 --- /dev/null +++ b/distribution/licenses/lucene-suggest-6.3.0-snapshot-a66a445.jar.sha1 @@ -0,0 +1 @@ +a5cb3723bc8e0db185fc43e57b648145de27fde8 \ No newline at end of file diff --git a/distribution/licenses/lucene-suggest-6.3.0-snapshot-ed102d6.jar.sha1 b/distribution/licenses/lucene-suggest-6.3.0-snapshot-ed102d6.jar.sha1 deleted file mode 100644 index 1cde0aa3992..00000000000 --- a/distribution/licenses/lucene-suggest-6.3.0-snapshot-ed102d6.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -0b065269e76aa5cf56fa30dcb0cb4641974b6bed \ No newline at end of file diff --git a/modules/lang-expression/licenses/lucene-expressions-6.3.0-snapshot-a66a445.jar.sha1 b/modules/lang-expression/licenses/lucene-expressions-6.3.0-snapshot-a66a445.jar.sha1 new file mode 100644 index 00000000000..aadc6a31524 --- /dev/null +++ b/modules/lang-expression/licenses/lucene-expressions-6.3.0-snapshot-a66a445.jar.sha1 @@ -0,0 +1 @@ +0bf61de45f8ea73a185d48572ea094f6b696a7a8 \ No newline at end of file diff --git a/modules/lang-expression/licenses/lucene-expressions-6.3.0-snapshot-ed102d6.jar.sha1 b/modules/lang-expression/licenses/lucene-expressions-6.3.0-snapshot-ed102d6.jar.sha1 deleted file mode 100644 index 5499a2d2ebf..00000000000 --- a/modules/lang-expression/licenses/lucene-expressions-6.3.0-snapshot-ed102d6.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3ad2da8a2c48520f05723050252b131c96cc3b7c \ No newline at end of file diff --git a/plugins/analysis-icu/licenses/lucene-analyzers-icu-6.3.0-snapshot-a66a445.jar.sha1 b/plugins/analysis-icu/licenses/lucene-analyzers-icu-6.3.0-snapshot-a66a445.jar.sha1 new file mode 100644 index 00000000000..29114cfcf70 --- /dev/null +++ b/plugins/analysis-icu/licenses/lucene-analyzers-icu-6.3.0-snapshot-a66a445.jar.sha1 @@ -0,0 +1 @@ +39e5761c8209a6e4e940a3aec4ba57a6b631ca00 \ No newline at end of file diff --git a/plugins/analysis-icu/licenses/lucene-analyzers-icu-6.3.0-snapshot-ed102d6.jar.sha1 b/plugins/analysis-icu/licenses/lucene-analyzers-icu-6.3.0-snapshot-ed102d6.jar.sha1 deleted file mode 100644 index 296a0b911fc..00000000000 --- a/plugins/analysis-icu/licenses/lucene-analyzers-icu-6.3.0-snapshot-ed102d6.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -2794258f69dd65abe1d991013b6ba98ae8a2ab60 \ No newline at end of file diff --git a/plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-6.3.0-snapshot-a66a445.jar.sha1 b/plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-6.3.0-snapshot-a66a445.jar.sha1 new file mode 100644 index 00000000000..2ec23fb8b2d --- /dev/null +++ b/plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-6.3.0-snapshot-a66a445.jar.sha1 @@ -0,0 +1 @@ +c4230c40a10cbb4ad54bcbe9e4265ecb598a4c25 \ No newline at end of file diff --git a/plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-6.3.0-snapshot-ed102d6.jar.sha1 b/plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-6.3.0-snapshot-ed102d6.jar.sha1 deleted file mode 100644 index 12c5e58189c..00000000000 --- a/plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-6.3.0-snapshot-ed102d6.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -7e325a21b6bde7a86861bf380b40aeaafabe8c90 \ No newline at end of file diff --git a/plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-6.3.0-snapshot-a66a445.jar.sha1 b/plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-6.3.0-snapshot-a66a445.jar.sha1 new file mode 100644 index 00000000000..27a5a67a55a --- /dev/null +++ b/plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-6.3.0-snapshot-a66a445.jar.sha1 @@ -0,0 +1 @@ +ccd0636f0df42146b5c77cac5ec57739c9ff2893 \ No newline at end of file diff --git a/plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-6.3.0-snapshot-ed102d6.jar.sha1 b/plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-6.3.0-snapshot-ed102d6.jar.sha1 deleted file mode 100644 index 2618809af12..00000000000 --- a/plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-6.3.0-snapshot-ed102d6.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -746a7888f48f3be86f7a31d95a1ca1e033cf0c3f \ No newline at end of file diff --git a/plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-6.3.0-snapshot-a66a445.jar.sha1 b/plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-6.3.0-snapshot-a66a445.jar.sha1 new file mode 100644 index 00000000000..a70cf1ae74f --- /dev/null +++ b/plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-6.3.0-snapshot-a66a445.jar.sha1 @@ -0,0 +1 @@ +17b3d2f5ffd58756b6d5bdc651eb2ea461885d0a \ No newline at end of file diff --git a/plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-6.3.0-snapshot-ed102d6.jar.sha1 b/plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-6.3.0-snapshot-ed102d6.jar.sha1 deleted file mode 100644 index 6b89f1e5386..00000000000 --- a/plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-6.3.0-snapshot-ed102d6.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -fec95fc0b5bb2e0855a9c9be3dc2982c0ab0fa16 \ No newline at end of file diff --git a/plugins/analysis-stempel/licenses/lucene-analyzers-stempel-6.3.0-snapshot-a66a445.jar.sha1 b/plugins/analysis-stempel/licenses/lucene-analyzers-stempel-6.3.0-snapshot-a66a445.jar.sha1 new file mode 100644 index 00000000000..466578a5e24 --- /dev/null +++ b/plugins/analysis-stempel/licenses/lucene-analyzers-stempel-6.3.0-snapshot-a66a445.jar.sha1 @@ -0,0 +1 @@ +d3d540a7225837e25cc0ed02aefb0c7763e0f832 \ No newline at end of file diff --git a/plugins/analysis-stempel/licenses/lucene-analyzers-stempel-6.3.0-snapshot-ed102d6.jar.sha1 b/plugins/analysis-stempel/licenses/lucene-analyzers-stempel-6.3.0-snapshot-ed102d6.jar.sha1 deleted file mode 100644 index f6e72856fd9..00000000000 --- a/plugins/analysis-stempel/licenses/lucene-analyzers-stempel-6.3.0-snapshot-ed102d6.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -8663b181f65dadffa6f0f42e56a7ab36ff447fd4 \ No newline at end of file diff --git a/plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-6.3.0-snapshot-a66a445.jar.sha1 b/plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-6.3.0-snapshot-a66a445.jar.sha1 new file mode 100644 index 00000000000..5ad5644d679 --- /dev/null +++ b/plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-6.3.0-snapshot-a66a445.jar.sha1 @@ -0,0 +1 @@ +7e711a007cd1588f8118eb02803381d448ae087c \ No newline at end of file diff --git a/plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-6.3.0-snapshot-ed102d6.jar.sha1 b/plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-6.3.0-snapshot-ed102d6.jar.sha1 deleted file mode 100644 index af4d754834c..00000000000 --- a/plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-6.3.0-snapshot-ed102d6.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -a08e5ddd375ea17a878ba077cc33ce59e68758c2 \ No newline at end of file From d53e1d213f084119fce1356b30b60d5bf55b5084 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 4 Nov 2016 10:36:38 +0100 Subject: [PATCH 130/132] Generate POM files with non-wildcard excludes (#21234) Dependencies are currently marked as non-transitive in generated POM files by adding a wildcard (*) exclusion. This breaks compatibility with the dependency manager Apache Ivy as it incorrectly translates POMs with * excludes to Ivy XML with * excludes which results in the main artifact being excluded as well (see https://issues.apache.org/jira/browse/IVY-1531). To stay compatible with the current release of Ivy this commit uses explicit excludes for each transitive artifact instead to ensure that the main artifact is not excluded. This should be revisited when we upgrade Gradle to a higher version as the current one (2.13) as Gradle automatically translates non-transitive dependencies to * excludes in 2.14+. --- .../elasticsearch/gradle/BuildPlugin.groovy | 27 ++++++++++++++----- 1 file changed, 20 insertions(+), 7 deletions(-) diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy index 36f3b36e7cd..65402290e01 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy @@ -28,6 +28,7 @@ import org.gradle.api.Task import org.gradle.api.XmlProvider import org.gradle.api.artifacts.Configuration import org.gradle.api.artifacts.ModuleDependency +import org.gradle.api.artifacts.ModuleVersionIdentifier import org.gradle.api.artifacts.ProjectDependency import org.gradle.api.artifacts.ResolvedArtifact import org.gradle.api.artifacts.dsl.RepositoryHandler @@ -294,12 +295,15 @@ class BuildPlugin implements Plugin { * Returns a closure which can be used with a MavenPom for fixing problems with gradle generated poms. * *
    - *
  • Remove transitive dependencies (using wildcard exclusions, fixed in gradle 2.14)
  • - *
  • Set compile time deps back to compile from runtime (known issue with maven-publish plugin) + *
  • Remove transitive dependencies. We currently exclude all artifacts explicitly instead of using wildcards + * as Ivy incorrectly translates POMs with * excludes to Ivy XML with * excludes which results in the main artifact + * being excluded as well (see https://issues.apache.org/jira/browse/IVY-1531). Note that Gradle 2.14+ automatically + * translates non-transitive dependencies to * excludes. We should revisit this when upgrading Gradle.
  • + *
  • Set compile time deps back to compile from runtime (known issue with maven-publish plugin)
  • *
*/ private static Closure fixupDependencies(Project project) { - // TODO: remove this when enforcing gradle 2.14+, it now properly handles exclusions + // TODO: revisit this when upgrading to Gradle 2.14+, see Javadoc comment above return { XmlProvider xml -> // first find if we have dependencies at all, and grab the node NodeList depsNodes = xml.asNode().get('dependencies') @@ -334,10 +338,19 @@ class BuildPlugin implements Plugin { continue } - // we now know we have something to exclude, so add a wildcard exclusion element - Node exclusion = depNode.appendNode('exclusions').appendNode('exclusion') - exclusion.appendNode('groupId', '*') - exclusion.appendNode('artifactId', '*') + // we now know we have something to exclude, so add exclusions for all artifacts except the main one + Node exclusions = depNode.appendNode('exclusions') + for (ResolvedArtifact artifact : artifacts) { + ModuleVersionIdentifier moduleVersionIdentifier = artifact.moduleVersion.id; + String depGroupId = moduleVersionIdentifier.group + String depArtifactId = moduleVersionIdentifier.name + // add exclusions for all artifacts except the main one + if (depGroupId != groupId || depArtifactId != artifactId) { + Node exclusion = exclusions.appendNode('exclusion') + exclusion.appendNode('groupId', depGroupId) + exclusion.appendNode('artifactId', depArtifactId) + } + } } } } From f4594d4302ed84e4ae0ec13a1c06cc1ed640c7e5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Wed, 2 Nov 2016 11:55:28 +0100 Subject: [PATCH 131/132] Removing plugin that isn't installed shouldn't trigger usage information The usage information for `elasticsearch-plugin` is quiet verbose and makes the actual error message that is shown when trying to remove an non-existing plugin hard to spot. This changes the error code to not trigger printing the usage information. Closes #21250 --- .../plugins/RemovePluginCommand.java | 27 ++++++++++--------- .../plugins/RemovePluginCommandTests.java | 20 ++++++++++++++ 2 files changed, 34 insertions(+), 13 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/plugins/RemovePluginCommand.java b/core/src/main/java/org/elasticsearch/plugins/RemovePluginCommand.java index 54cd34d6742..cf21f4cc830 100644 --- a/core/src/main/java/org/elasticsearch/plugins/RemovePluginCommand.java +++ b/core/src/main/java/org/elasticsearch/plugins/RemovePluginCommand.java @@ -19,6 +19,19 @@ package org.elasticsearch.plugins; +import joptsimple.OptionSet; +import joptsimple.OptionSpec; + +import org.apache.lucene.util.IOUtils; +import org.elasticsearch.cli.ExitCodes; +import org.elasticsearch.cli.SettingCommand; +import org.elasticsearch.cli.Terminal; +import org.elasticsearch.cli.UserException; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.env.Environment; +import org.elasticsearch.node.internal.InternalSettingsPreparer; + import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.StandardCopyOption; @@ -26,18 +39,6 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; -import joptsimple.OptionSet; -import joptsimple.OptionSpec; -import org.apache.lucene.util.IOUtils; -import org.elasticsearch.cli.ExitCodes; -import org.elasticsearch.cli.SettingCommand; -import org.elasticsearch.cli.UserException; -import org.elasticsearch.common.Strings; -import org.elasticsearch.cli.Terminal; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.env.Environment; -import org.elasticsearch.node.internal.InternalSettingsPreparer; - import static org.elasticsearch.cli.Terminal.Verbosity.VERBOSE; /** @@ -67,7 +68,7 @@ final class RemovePluginCommand extends SettingCommand { final Path pluginDir = env.pluginsFile().resolve(pluginName); if (Files.exists(pluginDir) == false) { throw new UserException( - ExitCodes.USAGE, + ExitCodes.CONFIG, "plugin " + pluginName + " not found; run 'elasticsearch-plugin list' to get list of installed plugins"); } diff --git a/qa/evil-tests/src/test/java/org/elasticsearch/plugins/RemovePluginCommandTests.java b/qa/evil-tests/src/test/java/org/elasticsearch/plugins/RemovePluginCommandTests.java index ab4f00492b0..0078c61898d 100644 --- a/qa/evil-tests/src/test/java/org/elasticsearch/plugins/RemovePluginCommandTests.java +++ b/qa/evil-tests/src/test/java/org/elasticsearch/plugins/RemovePluginCommandTests.java @@ -20,6 +20,7 @@ package org.elasticsearch.plugins; import org.apache.lucene.util.LuceneTestCase; +import org.elasticsearch.cli.ExitCodes; import org.elasticsearch.cli.MockTerminal; import org.elasticsearch.cli.UserException; import org.elasticsearch.common.settings.Settings; @@ -27,7 +28,9 @@ import org.elasticsearch.env.Environment; import org.elasticsearch.test.ESTestCase; import org.junit.Before; +import java.io.BufferedReader; import java.io.IOException; +import java.io.StringReader; import java.nio.file.DirectoryStream; import java.nio.file.Files; import java.nio.file.Path; @@ -43,6 +46,7 @@ public class RemovePluginCommandTests extends ESTestCase { private Path home; private Environment env; + @Override @Before public void setUp() throws Exception { super.setUp(); @@ -130,8 +134,24 @@ public class RemovePluginCommandTests extends ESTestCase { assertThat(terminal.getOutput(), not(containsString(expectedConfigDirPreservedMessage(configDir)))); } + public void testRemoveUninstalledPluginErrors() throws Exception { + UserException e = expectThrows(UserException.class, () -> removePlugin("fake", home)); + assertEquals(ExitCodes.CONFIG, e.exitCode); + assertEquals("plugin fake not found; run 'elasticsearch-plugin list' to get list of installed plugins", e.getMessage()); + + MockTerminal terminal = new MockTerminal(); + new RemovePluginCommand().main(new String[] { "-Epath.home=" + home, "fake" }, terminal); + try (BufferedReader reader = new BufferedReader(new StringReader(terminal.getOutput()))) { + assertEquals("-> Removing fake...", reader.readLine()); + assertEquals("ERROR: plugin fake not found; run 'elasticsearch-plugin list' to get list of installed plugins", + reader.readLine()); + assertNull(reader.readLine()); + } + } + private String expectedConfigDirPreservedMessage(final Path configDir) { return "-> Preserving plugin config files [" + configDir + "] in case of upgrade, delete manually if not needed"; } } + From 6acbefe3f7463e48a7721d5bfadaf051be4a0a16 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Fri, 4 Nov 2016 10:42:08 +0100 Subject: [PATCH 132/132] Add tests for alternative ways of writing zero offset timezones According to ISO 8601, a time zone offset of zero, can be stated numerically as "+00:00", "+0000", or "00". The Joda library also seems to allow for "-00:00", "-00" and "-0000". This adds some test to the DateMathParserTests that check that we also conform to this. Closes #21320 --- .../common/joda/DateMathParserTests.java | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/core/src/test/java/org/elasticsearch/common/joda/DateMathParserTests.java b/core/src/test/java/org/elasticsearch/common/joda/DateMathParserTests.java index 505196a97f6..ca9a6b3a1ab 100644 --- a/core/src/test/java/org/elasticsearch/common/joda/DateMathParserTests.java +++ b/core/src/test/java/org/elasticsearch/common/joda/DateMathParserTests.java @@ -82,8 +82,24 @@ public class DateMathParserTests extends ESTestCase { // timezone works within date format assertDateMathEquals("2014-05-30T20:21+02:00", "2014-05-30T18:21:00.000"); + // test alternative ways of writing zero offsets, according to ISO 8601 +00:00, +00, +0000 should work. + // joda also seems to allow for -00:00, -00, -0000 + assertDateMathEquals("2014-05-30T18:21+00:00", "2014-05-30T18:21:00.000"); + assertDateMathEquals("2014-05-30T18:21+00", "2014-05-30T18:21:00.000"); + assertDateMathEquals("2014-05-30T18:21+0000", "2014-05-30T18:21:00.000"); + assertDateMathEquals("2014-05-30T18:21-00:00", "2014-05-30T18:21:00.000"); + assertDateMathEquals("2014-05-30T18:21-00", "2014-05-30T18:21:00.000"); + assertDateMathEquals("2014-05-30T18:21-0000", "2014-05-30T18:21:00.000"); + // but also externally assertDateMathEquals("2014-05-30T20:21", "2014-05-30T18:21:00.000", 0, false, DateTimeZone.forID("+02:00")); + assertDateMathEquals("2014-05-30T18:21", "2014-05-30T18:21:00.000", 0, false, DateTimeZone.forID("+00:00")); + assertDateMathEquals("2014-05-30T18:21", "2014-05-30T18:21:00.000", 0, false, DateTimeZone.forID("+00:00")); + assertDateMathEquals("2014-05-30T18:21", "2014-05-30T18:21:00.000", 0, false, DateTimeZone.forID("+00")); + assertDateMathEquals("2014-05-30T18:21", "2014-05-30T18:21:00.000", 0, false, DateTimeZone.forID("+0000")); + assertDateMathEquals("2014-05-30T18:21", "2014-05-30T18:21:00.000", 0, false, DateTimeZone.forID("-00:00")); + assertDateMathEquals("2014-05-30T18:21", "2014-05-30T18:21:00.000", 0, false, DateTimeZone.forID("-00")); + assertDateMathEquals("2014-05-30T18:21", "2014-05-30T18:21:00.000", 0, false, DateTimeZone.forID("-0000")); // and timezone in the date has priority assertDateMathEquals("2014-05-30T20:21+03:00", "2014-05-30T17:21:00.000", 0, false, DateTimeZone.forID("-08:00"));

u@L5gp>6b!FvH@^yo+lybh^WPEG7r>fasu1a6i>0Df< zF?%(Ys(nz^_yZH9EAFdR_8ZbhjVQC)A=RAoQ~eTcGPlvqm)RtGU)d7l`Q={SUZqmd z*PR@TcQwU$r~*!wM4d&IGDt@8umy98R2`Dcl|^6+h?CtRdu$v~<}U7Y&LQ{BUB4LI zDh5%T`H<}5aljpCu9BO99sE8eJ5rt4`ZBL^J;lSKH>l|Bko)FHE;kQ!PX^xp^1n7p z%>ElE3;vCh{d;_@_!l8dqCi%A4aJ>*2FEC0Z!jpzKvIXIU+i<#&?BKX_v>Vzp#FTV z(iO-s&19;KqFpaFxs?Znsv_Wdnd&xI8KN4GHl^ojgK1LY2`EU?Azmz>FgeRc8QKu- z)+5md$QPv?&XH)o9j7pJUcT8lk~ex?$t{M1jv)Ef8-+}ArIa3-Zt@ru`3%;MvKP44 zjzAjS7XDrOV~4V21x^w6X@sZCC`5~j3GfuW3^nber01a`T}xlXVH7Dm-2|zFHqwWp zymaTaK&oVU+qSHcf?vtRIX`(gS-RoeMgYMJr3C-v$h!T(nhM!ttCenJ{)1@*#P$h8K2BUy!bv{Wp*SZn72)sU^F z%tAIC7a$S&EK}C3tT90Y7Lc{*lYKC%yb;`^`Z-iC+l7xVFJiA=BvTFSZyJ8S#x&T& zntEKBpO>aYMlhQ8?znI)wCH@B+;YQ7LUvtccGYg@X&1fQx1fJJf zQLhrpfSqnJ>o8s3ssRuv>jNmUmn@&)O}W<*0lHTHfN>PsF@ zU@?Va;&ZpE`m$uAMET@^$}o29i!z5YEnOl8W}+g4*8HbciQ-nNK9x*(@iMujfy??c zEL|o^T~W3D_!8W6iZ0PH?Bf!_?CGqmJMhwqwV6A=>(X1;2kS6i6!;70Dpd@QU(BGQ z`89ETfu4T5PjbyVuwE8#i6z_KKVjUw3RkSxZ7zbF=ee57isbl z2kLyTB?Cz?&oaetjajDnVRb^nu{2%)c9BaZ{kl$R53L9`7iP#F2NOwIPJTXh~#nC@3IAY1af6X-ra?_;EmwG>JuL2J>SqR{M0^C;B^Z?_&zhhWM7jP1p9* zja?Mr!!Zbc)p2UP5iha)w9&0o;c1l<8Yshj?@~8k68;0Jl_{(I2%t@sUYj3*z~&h8 zQ|e-OkcE~eeO7ZwP5JrVswVDq02adeT((-WEPLOyU`Kq_;2_*@saTX?P|yPRsH_oU zF=x+qfxVYS1myg_)1fBMUenQB}dIfj|bXsbk!I$mLAmO9KwhB?}nQyeSXRRtz~L({Ay&9ZO$EJbXn zGlyv9gS4l^gXCnlhBMk@H(01jBM+8Urc}Kq1Emzx<{wcn0&qgwspei)XUi3gzU7|V zR9{lbwNUTIoNTlW0CTYcs9>m<_g7?M*+v^@j;iT?xB;>~xk?LIv0mbS^kaaalLRq{ zvnX?6oF=nsfhcbfxa1|S^D8QnMkWX8;_C)b@fjS?F929Z_rRoFUvZwxtZVz)Y5XJd zAxT4Ex<9Chi!^2)--Y8czEh^57Wb&MhpS7wI&c{-ZX|1EDW?0WikGdXg<|dC;G*}d zIy;M!mrS__+}S;{{jV$~zL1hMp0KO^5Zto+nf0>)+OlXqxNKWg-B@mP)F?hQV0@`x z)@4<32c+q-I`?|0O64mCelGXk3|`Z!t$cyhl=b+MUda4dK&Z1^88;3x^2YJ5(kz|r z%6$ih2{DKdjp|4ml4%!ywdPhif2jw^-CFA77C+E+`=0To=V6A&FCCB`4a(gh&2G;j zAPH+ko}sgPMpdM^uSd62ONQFkjN`9S1(}S?1v^U({Ws3N2V7jvAT63J~tV!y8SG@m^ z2E=nGceGa)(TOK*X7x>=UD{~bC|xo;;w6{Aqq~pE za8LY_tFwU$*+#q=Og;Z*onrXZEc&4_x^+XjHP&CpQaMH=|EnRMY8rfSV66Y<%#eK5r=U+CIM!cU+r*!^=wYeZ-K{r(jL(g72shuX148tn1qqBoU z*r5^smGW~j&i3|>>>~!oBz^-8bZ$RwcST{}yLzfrhx4{3u5U2w&K8c7mEuPkWMN&? zCS+$54e6qatH}f`+-*Jfq6&j-e}4)j@uWfBzM=}NuDJKfd_|j5jFWeJHT##fGIuJv ziSap9lL|1_4$_;LX8&fQLT2(a7E*b37|MOlH3k6vNy{^*e+fx_GS_ zK$s(tzW+tChxL93h_IF@AUUPN+ z`97Lzl9`R1AJ3#4^a7}#V?ndOBL~#|5pC3FDb}G|x6}OFLgzP9U86aiXS=E7PPINa z&QJfe6_pOx`qL5s84H?VWoaHRs*Xm=4!`cBdnJ)-t$Gs->YTFri_#M}P%n>FudV$K z7~6ewB=d@2TTcaIpmC#RX0lFd2@QB-F>t%W) znG9H5KU+cRjpc^W)J~QBQJ~ut3z2PS{BSv|I5{;ba@UkD+$FTh(6f~AXE*T=ev|a> zP*IfZgqJ@r8=VhR+1S*ySjrlK!}rZVMjcK)x2jFqXAvg9Nn4y!pD#aM!YXoG==5sZ zfVWZ52Cq~jCO6uePnEOFWX2C(8t8;_8Z}vJy&QJ|ESs7`5jV0osYfDlu6!dU$^q|C zj^efymsv#d1*&+R&Z_;eG~_s0!G_}WJ8C=0_XFNYmdP69DM1H@I0|SCn^^y#Dh69Y zlOTaEMIy7$>Jasn1aDSS{mJfp6ra1m3tu$iepgg*$2_tqL_wKbLtzH$S&#y4fR7OGn-W4f;q8Er8$k(nF?^;~ z=3R7&ogIA6Cf=5b&R*71W$-o$sN&@3x-z*a@VCgLowQhOn9?Pwc6$bl*-9pt)>XEg_-d7E$dr15!Ol9n zJ3znh%Gz-dntmAUTFoZ6XVwXcOn*}!rMbfeq`-^tDuUO;u^O(ucQnPev~wTcE)Qh_uKg)z^%`Lk$ zFR{2R(`jZd_p=2U-Z$MEd6lF?gzDLH8K{N`Jx`HPdYtSI3cI_DOLtnp7Gxy(+jz z<-2PDedfk4-03-6w_RK6G_WS}-Zs*ER58ys_|nb3KyX5}yM?!c=!+E8^NsE@85TNK z{IJ5BRN#=d#>@jS^EG~;bZd~$FQ8%@dtR4}+=Ped{5=`d{3B-c{HD%7>=|`I zR(_@{+sPze+%R|>Qh9kxn%OjYiF@gvD2cBTJx6U}W>EM1y3}bA1!doMx#f2G!JAFp z3l!;->Eh(H^P=(6?GjSN;68%wRniYDXyXZd*o#m)3v55I>$<&0vg6^1_NvS4vfS>c zx*zPG)ohUx~uLpUHl`NWG0OwB$7ILK$!ko^}}x&iv8tk zI}9isZPaV-Z^n4}^Ny@ST5D3v8D0Ftkc}DsZg6|cUO#-3;-hb>Bw*6-`W&YOvvt&2 zNcF=OL}mpwOk$Pod=dgmQKqA!sa1vx@`uFv0lI=k0MRh8{wnSXb+OvSI=b6whZz;b zOsbG@O_!y;+pf{AdgK_BUES> z@2*k+f-1q$ z|9QCoUkrnNNiX*sXU@~IG<*A`gC!PMcT?w&ZTvI9A5p;FuM7ljY=zipu&=J!_x6-N z;ODrP-Ko(!+*fZWgcA6zgQHpat>F7Q-$kB$*=;8>U-!K#RVhIiO)_P^6*YGH@qn)0 z?}`l&3kCM;zyZa+s2?bA18(*mS#{-}(fP>DN8C9iT>97cXrdh4%`_0x5x&OaI$t#+;a zZX)|@jq3WC*-kTBG1gQ{tPyS6YyHw*P%I*%XLeJmh5MSzx09whS4}_tbxo(bi^_16 zP*?>lj+=glqjo4HQ#)u2aW&-sP@Rye&W<(I(nao`0R!_QSBxtSu(B)*7u80a+aGXF ziweW2)!}Qh8{+#iNOs7)Hze1hJj8v@k?j*@*N^qGyzEsZIod{UO-ZW$Ht>h+?uWy2Ye&|nfu zUtT>tjR2-hWFyZLou@PjFPam%T= zF8e3UrW7nDbJr&Y>3n=>O1o!!(Yd9QI$6rRP=+1rIekt{{ZksDY(5JZMJ*v^X;i&f z#&DM65PsOFQ5M%SR<=^N?o=`7uW7oqRHo>)QkcqO7Z|oLled)UL}fFcYUQKptLv2} zjK@gEjyS^P)C-Cxg%aD^LFWu2VgpoOwt<3Hg8a|G(l2r^*p9m!6mw4%uR?+Qf7W4b zfX;r5r0!+zCsPcWsU^<@t4(n!lsPyfH1OW4CR-5it(NIkfYgoH!5b_35^c63%8tEEQzj+xA8Ee}YLWN5 zjZE0`@MuBa>72mNC{~UOs~yG%>Y6x>SD*-}@hHSDVX@sVAcs;AUTt&z?%!eTJCZ{znKNxcL% zIzFG5!Qs2qGW|#PxQROZkjXcMxFiGb-(14q*@OsM&Ha2`S&qkRC~px5`l_4k*^R-gchSnei1l6o9-JX`>hrlWdpK#un8e-9Y`e z>aHJcQ+9fwd`^I1KnGc|<)z;%Yhl3qXq?^Wxh>PD4OTX=g*slq@?nvz!r52 zuR-_EzHU%CC+0Icyyiy_>y)@8YXep=KR>*NwCw)o-sF)C*k=Rq z96AQ$dY4S(Y~@ecGXL_A)pk+<3*#^J!u^yBy@R9JC)<9C8`*!V%HY6{bP8Jo(frO7 z8=FI|44iI2awh09ze9nkcn?`(L)9r1MyD8fFoPM^3*~lZNrA=mgDUC9_ynrg()ph% z>KSGGmdNA$2}0n`vo;8!A*ceB{A?|5@9f}+-YB;7a1kZcmz#Z?8@k#>=NnjltGYM$ zeF~>Bv6I8i%<>D$di&vk9D=Wn^}?ID+#|-PpJf{C;NDTEz56YLOr#<+QVtzsh@X$CxrdGM6PLDRw(#@sII^ zN;2xb_sfa2r3)dLEVD`kLyGyxNzT1$J@_i-C){rfo&Vk|yB{e{+J^PCX4Nka!)xr? z9QDdkT>N<1M*3dbfpa?3)Or49ha6Qti*{cXPa>=&9?5fE^hXt41bx+3h^6_wLusMQX=A;MHKQv#y&)}sf*FLE)r(!7 zkv!|Xi}mopaM?nJ(vE}KCeRaxJJ0B5E3B$}K{pxV8M2!ytdXn9H+tJ;$4RgDBQ@M7 zI${jK2_;P1wEe&rOxU{@ntn!$XLx``L6IE!HrBu=VN(8a?#W$bc)fTrpO3=etY#89>BzBbGsp#&Q1x3$ru_qD3- z_j@R7-_CtMzqnr)4_0b0-ux&kEj6zd_~GGO1LEAl)e4uodlyyA@DzJSL%R6H(c@}U z;(Q}#U!%_bV4dE(l#u_EpMjO%1S4-mDLG7nK^CEoYHmx#sxwUuJ@ zZ-K9_e>JKnjul4##<6nQs9x<#rj<2fAs~Mdlnv@-zxiBd6Q;68%x+RNp)4Sy{)n4O_Bwfj60EiG_8cCaSaE5iz3Pv@!pqyfMqBk31vwD%Rs@jWWeQMY(6EBh)u z&nZ_@CJ9X{LQuvcp*H-DRJxKB=R2yGuOnPCIVGx^QyKIkb@L910l5#CF^c?PobZ2L zd0i$rwxSBBojqw&0BR!AE>g;RNos6#=vonw#XCAyuG!K?iZq~}uX7ZtXO)dc$}Fr< zb~f2v<~CAlxtFexy%4GSeMG^^;53;|B*;b!&daPxD%6VK0h0X&AuiU8padu#q>*0? z`jyF%`IdXX;0=iyFq~{Z3e9$?utR=e{jA?H|NM{k+QK(O!j6MV_%M<(!XLBK$uaSp_%9CH> zga&jV=@Mkd^kHR|Y)*Fh0$1mE+9j$!9fK@8R5at=V4JTrbn|DrbQI469;|T1-o)FT zj%<84MY;!9sccFB3#nX98R@jmS!X^Nm}6Im^K_OXUUxb5>gZ4;mVaF@b4Nf1-hq~t z8|9_Xa-X(Hw9j?rPAZh_#b%c^Bf~~J${U&{Ypn5A)QLva!OzM30cSm^Njxpnda0Y7QdiEq%*l!^>KxvTvzV({CJla-P%p@W7^as z+Evvox%&j8h`paQHAw%>4i!onY@RF;bw>lsKQc%c{i+5NOo6Mnwc31VT8#dQ@#N;bv4CaWMpW_q-%?19!Vdj=ALmoXl&u?=F|x$gEe@~`$@>p9^5!q0s@9*>@Q$oeh;H$0 z)8L02Wy{X%Eb$PS!#Ddziae*tCtet9PxXQo+@+*S3bd0@Vd%jGrpW$af>h5BDrKzQ zKF<|5lcVocTV`cPTY;I#Y^B?6NglG9j9;odGnxG@Lq^(lwxcb9SDepeUh#9%NDkI_ zOfAhXRYyJF7%a-W0noZcD|dtSg6A21&k9J>XqG!`g#Uwua9OX;ckiYP*HRa|1oQPG zbj2dKoso95PcKVc0!+hJP*>?7^k;v+Zr`uUth*mn_)iU>C=wB|uqnk^7Hq6i3Ga|5 z7>6rh@4W0=CI8P?Fxpv@M1V>Ne^b-N71e4h-Neg{XE4(*;6{&AhZxiC?xr-?wTg2Q zWen)hnKjx+MtrM}!Gg7mqN!=@rY_oi!;O8hrp^p%nAz#4JuAm!k%xDx{BNe4GvgU* zfY`a#*G09v(x1*&*!C2Z=}R(3Czg9Uzm3zLeEsp$oHmf94X|f*?Td5KdBD6!^wEMm zPh(V+ES_mHGqLDoEYYVlufs*Kg7NQY&SSXk&a&y_BCMoe8Z@25)LGsOAD7!RfQKyt z)H`9YN(~L4rEU@C4|KYqPD^=rxev+@0D!{|rMcGRcrBDIUNrmj<7ArjN#N(_Yl^u; zi>`|s(2re};fVO&13E^28MN*ns|H&tD_DnPibH*|hIP>w1QssANu58`Tb6I$qcYjX z*u2HreCp0ogz^ArIsssEdEmGg?JOfH&RknuWij*TCPpEktQ@LhmR-PUFHDUeJc(2W zFfJ~#>Ke5!9Zn;cd*L)nRxl2((UbZ#ow>AoisfG0;4e`e%Y5FYGoK!#TP3=1=W+2l zlbPGK=u!F3JsR&HAk0DriQ=hm$SB}>nrLk)_vIx$Og&bAQe%AyHOTGYgrA?MRa(W) zs<_->ec^%h_(XblUQe=3&L_m znX?aj_5*5{WZkKX&vy;ES6Q?pgkr93(k~m3!*!Od8vKh8zrbKJdW+Ng4eI2)pU#{& zpeFXG{)lF^zdR$Jqeg0PXkzyYjMoZA<;@y@t~HvNIvjK^(xA;si~6{-DwJAp?BH1X>D*jH5frkeb=7wNfCn zLk(GJ|EWqll2Ns~i{BH}6KhqWCMgFNR!cN>-)Kx~x zh@3@l%Br&aW)WkfT*L2BZ;#i@NZD(0!?0Cvl;|50iD={&8C`+w zs5nHu6fu%DGTB+N#KUX04LMjOkL6fOPOm$||>Y27zqK$YfQ)mmT z8V)D_+@U#`N8p7TQ{xg!GnB<^F*|e#<7NIlfFwDJW+?!2vgQ5}vlQ+nPf2rVb4JO+ z6svVqXC80n^SGZ*V_H}0$V7!5Ri!_HF_620H2}lfarg|C!z>-mn`D$}KNy#|Vkm?p zQ!M{*$YLXyC!L=C)DIY3-lDQL3aD5+l{*55%gcDmdZj4|8uE#4sp|=bh^m0d zV;0?0Hd9Dw*FD)x;W4=fn?`BG!&&BWKROf7xQJp(t5l6*J|uk3M1vBihr;-D>ViRC zM&c!MZvbl94io5wbEyZ$O3Y@F*(!M^%R4^5<-+=l41?nz0gn0SLZrcORdnvG^Upiyf%X8xsGp4rTn9;y)LoJ!T2*7Rse=qPVC55KJ&pT@%Ua~kESKm7u&2AI z)mHwROWDXuoiaXypivgZtu1?yH8i;y1=U>m`J3QxLD`ViCE>nX!$&rYja=;H=^E|}$40LE!A4$jv1$&KJ4r1uQp}MHeDsR)}{o#>nzHprn1pVhF@G~$*zo) zb(;LPtPAbez_E5yWdn{;)R%%$i!WE1eU#bGWebdxh?G*zv}jNntlX)Nzop7@yL(&w zyZuCAWq^Qe!B%}W_dRX2-#|#Mb5~PY@ryD;crR6PzG2YrU}SK|^~;m?Pjs1fI(r5(;wL$4QpR9HHnP@%jv_h)0g zbZ;pD)BSc3e`4tN9xd9L%>9NT{-8lE$?mUS@srKgq?P_rU3xvprg1<^HuQ zminw7tfFnt{$c2Tv|pv-;ycdi=F|;V9_0NpsEY?r$tdl8sRbK+PPe?5jXO{aW7$Uf zE(fdfAbliAI$x>E#_QxaEoPZ~Zo@g98hlZc76T$*aepyJ95W8GG1K#tyHvy4#gU1s zRQO7zpKI`u1vRgcE;tt^Df-9|&*Gb*ij(vE4fZ{^Y>{28fwCB&zC8zzX(#Dtb&Y>p zZp=N8`iw!>Qp3Gd2BF!%qw3cAgO>m{-Mw;K0$GB_QZqa zYoVUuqFSzB|B)U^mED{#tRABFf9+MmZjcbqm1vp zE38pOmLEk}*N4@r?p_tX-2|#H&_H!Ib>oRDZ8CTol`0n(XuRDeo7Y}Ioir)^{>vpp znl+WbE#v&@nXVtb*sb#&%jC3pFK zFI=oC?o=UOa-mhAIiljgt%27xsv@-PDrX;<&T7TkqaqeJWUk08eCZ=Y;+`I6p8*=# zS;}OO(XfCv_SFW9T%w)6!7MAAZUofyzF_}PVP76EM_Hxa?^{dP(%sd4bMNitCaIf{ zX^tF=h)Z}MMlyx+F=r0BvF>iJT%k7UBkW>2v%l*bn zG-TbzSc*P0C{)2F;r&Wy*#zho>aBt`f+!ijP)G4+2j=@|gMD1`ZpUwdw7rsT&$iI{ z=nxtqmGP_x=y(Q~1aavY?u}a-J;zpuHZx3qmFA?B+i_WwHc9$G)a025xuev~x4>3r zXUS-!7bwY)7wJoCtNXCWl>@guOOntKl;5XTC`6e)(gdouNjhZM?^4383s3tStbfB;uc5q99f;02isVA zA*XGS-MmjM893l#xQJn6ul;83bM0hgPilRdA)i-0z; z+cXOM2xm2jKSsPTSy&!%ZYAp$->Q{5`8}0JSiyiD_%P5IJj7M*o%hntL{|rQ2Xqu# z#U2G_pva%z3LT?n@ZIvtu$PFHw`iEqtZ>*v3yOB?Moby!NwhKui09E+bOHs3=^Ec- zK!UP_t~Naa1|||HaYAux5v49K=uotpbkadzkyf)eqe_YLMLOLoLW-@75-oF4jRZ6xNkyH%JMR$h%#R;K!EbM=Q=c+01QwBIJo>$h+wPULGbCJ_zPMY89+7FE8N+#@!UikH&m7g$CF5x%OA(5%EJcXeXkR zN(Y<4Z|LhsS5yr`=Pao94h#`#w6jDfmC+(|(fX5PP|9zG>*zCnt#bnjI==+BDE<8o z8#H53{mTvRz$y_O6IcbnS%WcET4;|+51GTr6yBqjL(g4U?;j#O zcT(N(9EwA~&wL z<$C*j!n$J$T^WK$0L_9FzKFwgM{aTG<;@KPM)s8sMq+gS2(9D~KyL)~q1`r*(AxJ* z@soR|P|W|dS#jM_R^7GH%%>NvcVTwJUt*^V}t4FK&_k7 zD%1NayTj;%J+I68%!GTx^V46rhCI!}Biz)5OZasShS}_UD%})yPpJK>8# zVFa)0<9z&RAJ_XcvtMb{+@ohpX9ky4zhx*DFz@Y7jgpv~g!kx}XlDlq&!%j5ne=Xg z2+{Q%h0JUt1%`5A5XLH!a`yHpN3!1LF+E1Y?MXw1wMs#w@2&VKC{e5 zF;(N1(D6;FG-T6H4t7EQ(d8|6U){G8t}4ra8)54*yY3gibo)@Vgf$F`sdbI%7v;KP z#~$PKeTRW)t&RRfSR}h`U4Esmy7ndaU#eoMjxdAr1%)`uswgH=YIQR@E|y;JMm-dF z(pw!0N-Z6hi{-s0y?P~~-_zkDTc#v&vF5#QvJkKzK3t!}6ru&A?m|&Mjkb*EXPWZe z;Bl5Kd#8yl{A9LVI>%Ln^xrbT&A8UKx8Ityx z>AhA(E>?30nC-TOMOYG}1NP23D6j5(p1W^_w>o}~m*;T(a2%oo;{K|;L0B*7hvk9z zz_zB8j?mQj$Vjks3d>`(Tt6Q-4cfCk=ifB%>;YkK?eI(RReJ>40A+z6*F{`M0mZVr zdA!%uA+to+?p$Tck$pe_SgQFvgmSJk>2mXNT^{V>tDTD9^EBF9|FBQr6nZ>K&z3S9 zV`f^GtcloW8!M`){qC1yq&G+JHe)y1PM2x+SCp zq`SKt1UB7`v~);Ihjb$)4bt7xEe%rt_MHEmbM<)dMb5=FtgXv6?=x@B%=^v$1_x_0 zhC29$irdcKHcS&cf)eu$l*YqQJZ)x37=ad-h3a zs;)^>K14zF(P(w~g!Wk{dQMWOQiKc>G!#kuy=^}INVqtKX>I`5>0VYflP)5WJJ=5szjhytuy8 zm97x=f^^L+#aji#^_AO~pEm*+sQR2(;Df#L_W5?hNKsr{XpuTqkST@docI!SDrH9X z^i2_SKE>pYxZ{MiXf43Kqv%rQCgVtMnS|FP38owvp?b`lub|7aVf?bedC(8AZa7coHV|Xx4<(UR(2}tE+NKTSS zIFeP-sf!31$!6c@*;5wYn&OYY;3sW@donHKSIYLoyMVkIf{$Uv$aIW13-cSg6-Bd6 z^=_YHef1G%xR&oWotU?c`Ng!PACAB#1L6A@rWAf}y(G*ForcoEw=6gpJDEJrKts79 zz)B_2Cgfun+7=pOS;!7-A#|NDPAyj2@YX-=@vsdB=zL4dbx;d)$^j_p zwXCs$-$E4`g|>A)rozm1P2#(os4o2EjU!Ao!vNO3U|uRW=0=J_6j#DXCs~@oo5n_R z*{O_ouEo^5e2n4*`Cn`M!mQ9}#-S(W6Sh`&iiP`8>dwx}35Tnv4<>@89=V#JDf&QQ z5b9QN;T6?1R!aH+=%q>8~1Q1+v1 z5Yj)XE2d~`K09D5d!jd=*$;rSyqoPMs4YW^bI%ht_xfU2DuuJKCsl=pVMzN z44;lV^D@V5cO=qk{Mw`lcxx-1!{`!HSkon%@(_|4Xaz^j+w&M+*S+WY zZ|4Qax}UP~lq)BrSJ%Z%#_{)n#UES=1{ACDEY+N|#;EVTuANjj--&@awnNr1JyM|0 zp49hK-a2GmrN|u`@kuLIAnjB&I*qt)%qA0sK#RM+b`g)s-3D}S>PVj30$)=l;_a2I zh7iEpgNj7;8AS^ljp&ICII@bcvti+4=l(m)9j5%1tSI^IlMiFeA2s&-1R$>015DQY zo}Mt~eYulE>|FfPJ*$kbip6?;`yP3ZA94$cDN@MsRE-$Ht!7<)Mb8NhR=k9_1vT`cu4y5jHh-5U{uVuTf=#y#kzKNGRvsJthlQ_wunDtw=K-(Q^I#=%(I)sJ{=T#hn zy@nQ_LHjs3hk_?36qS-JQI4D@B2)yMt%KOHN1yGfn~0{?R8h+fgww^H6vfORAofTO zg;!XKc;CGUqI7lUN zsgft@ESi^k{^qy|+kQ;Sw>(v0!|1%BB*NG^ejFt6gJa}RLa8nxH&0RvEr;NjmPykboD`}aFvJ|@Yi4qzq;8YHiRORUe{fAxFx8qux)=5%D4Hp zb2;EqI;y(u6I^{Bg`9JZJX|fP3f~Wt_&FCWlM%3sF&j3Hz>}kCQ%7*{^GC3Nug9<^ zf|B&<&Dt}EQdYr6E)8tRf~(q-C5wn01LHdjpRqHsoBeosODux)r*XSiJZ?B=zN%0T z)O&<;?harTg$oAqrv_|{X6`)=-^|=yIbT6=sSN$1pgz@{|8-bo3t3}Lw-(3TJ^IUS z6Ui>{S}bcRR`u3C3$q;$Qv@VBL)Dsrl%On=rfgTcfr6`9>P?gcl0$;GW?#3YW^gOu zG0ZS0Xo8`UyGm!za=kQ4#%9q}#2hRu)nBNylF3JFUSYbHGB48X8bj90mbEbyivu>> zfbV=tV!|!225fZ;RFjL~zO~e$3U^Q$0o3ea-f`7QyK$dp6cL&I594Pu>^w0YJ2{7xHLm?dRDFm<*>ABzLLr8f_r+W zY3eqSFuPnhqiw#5imxz;OP!(5h@vNWQjQji4W0Dk%?ko63(P~~6Lp7AV_ts6ZS z?mK(QmXLo0!bhqCFY80@&%lZ4C-jkA>HPESM9r8Ex)w@I!xyM^9KqF=b%AzK6wbnM zzL@&5plc`+W&l7ADkL3G4(yZ!`5a_MdzI}O$BQ^VHDh%XXgi&P+8KUd8p7en>wNwA z0YjpSrQ@bCdpdU(!4pUbUV%1{i2Bepk8UW30T)M#;MN$HDGi>^A_r|lFRg{dS;W%C zQ;={>BEi64KiO}+S-IjSu7H?^{8+}1^r$}=`K3tZhY?Q73^YedWwip+n$sMb)`X^ST=g>b6mnZLVnMdL(HZa-S>=W@cdh?)q0 z&j6YQpOHg6%P+(>0viLdWrEw4y5MQvg-T+fw-uRlxc zgz~(MT%Y8WPM67n9MnVPG#9~6$H7@xj-{rieV$=F#pM`6@OEhhzMsoaRasysB1+G^ z2_!!%B#ttBa2`tpWq4iDK^=F0IN`w)Gi-S_!{?`)SUqT!e zAsaiAUZvUbsKE@i$JzHXJ~t^xQ63rTaAQGL6OF#T@M!RlVh_22XFw2*1iDtI^tmJ^OY|)E&0%J0e6$ zcWcQDg|boXrAOpY6l0!tnvHB;S%1ytE`wdk7D7Q8v%Xo=&SyJ9yNp@F^u7joGxW0s zhXU4oe2igMa}&)z>$gg$3rgycKd_nw5OkxI2A9 znta~UP$(v6H3wGYxu61tJp)%qm%TH%azioa%|2nSg6SI_?bjS6FVZaOz_cIR5BDz+6(#UU_8j^`QYnhvk-{IOsf| zgG-G=6^$&R!<{ge#Atq&zq^-CsC#u{g`(qjB7!BA-#7XNmxc5~5`3`;t9p7hmw2|o z`PjWRc0yzaSlt2bQ+W0QrCf0DGAW85WnZD`&4Hgi1;@g&z$4@3D)OiD ze#pz!4M=UtTdGbJXtQh9xf0a1Q}zbq6B+XLq8EJrfuzGG)7DAsEPM?do+_c(!7_EE zeAY`FU*IYxkltm%qVa-lEZR=MT8E3`o(NW1C5+H9C9SnHD}+lAs&0y1T0k-EEx2z+ zFUT{>>IvxZ=2K1CjxSI$gj%aKWTO~l!Zw!orgUz)c|EG>ti~KNoacVX67N*J8>URLFd=>Isj-LikqBjRmeWsUI0QO*T#72VK08xZ&Qd zc3m5n*d)eiJ(xtuP_PEJA_?7L3q6o279HmO){r9eySvE5Q% zxquhBotWAO_R$yO7!V!Gr!ZvMQ%LDZ3r|T;_`Y}rC%Z)LJyxIYh(Va+>ZpXq7{tt* zw?+zkJ3X+3Z{RLIZud0OhH>=V)x;)`eBk_X(i|j%4w$n?!WIY|h10Ozu z6TMpO=a+sTzboMcRf=2~vd@yg<=>)qs-Di+1-q)GvTxj{Rv zA^x_?gOUp;Q4IUetw2DdqXhCqY2W)1L#tQ2t( zeGm-M1CQPATyq7sJ6k9hjF6r^wI~R?Hm%s~?C`IrGe^Lzq91ET(kU>1&sY+2b;>(S zGhgC-xAIuL;%2Efhu9ihh7Lu(sga-kvLcSKu_Ku^wTli`xhBdw{^nxV&&;luAq|TV z+C@M7X>l@`8bvc6%Wl_<1%bD07<*BaseUBQ;}AP2ii~Mm{6`wNk9jS54wu8kzTOtj zUCh)Sf^-^aYr0JbC&cbN*!{nWSF3l~1|Ka;|Ris;V7T5(A(YV!)JIo~FR$R##x^Ar7jP*Q`3aw^UFy;{9 z&ey9NQ($El&u(*j-6>OR)HKadTN0t`3fVA(-JfN=7>GgM7Qlqk$JL9-gek*+0%%bk z^{q;qe*>+nh2rtpffD$NYAi+cXeLE+j7mc^s*PD(#zjglo;CZuW9ij`oa5PB?1A12dVWRiMkaASaC13P*eZS@G$RFQSkJY56e)_xg9(kE8B#Ax z$;J5}^ zShBZeUbXd$%Tx&Rv`L+52|lxT9NKBAP3aX1LSBS4+}fwr4mKg)n0TwWr@|S2M_g3k zLVydU>R_8!N1tNqCjrEI`%qpLtvfl02->9DncQ#e#2&A4#XRrfb`iNHgIF^xrx)Qy zD6k;?T*s7rR{4>sgLf*(g_2n)Sx7q1G@8B zk&oS22KrW0y7J8MX{PM%;7nR`zJol6)ofZ#Ol7p)=3Tw)ACx`sdPgEeKAx`QqGI73#5-piFM zlQZGR^^sin!c_cyZCXXki}&3|9{KrN@vi1mfXD&s;h4f0cY&s4j4LGYg5H9wfh2?c z_M;vL_MUy>UDm#F@6@&lfK2uoGeg2QT(QK>N4?#0DG{!o8M|)yID0j<(>DNf88DanTtImhe~V^x z^%Eq!EMxW;FWnZB#+kdy0dq&H~*7kr)v+M((jc_I0tv62Lw2!1j&!E9g7Fx*% z@uvXuEy7?MwI{M2m^7MmI;3SEg;lxUjgY8pY%qAI=u=*fD?X{S#wR`Mv3gAk3HMTk z1jM%tJZP^ku;JO0R8BW?ZsoW+L?(ptW}{L(CDOXQwj`9z3jXe#*bjNf$d|U+h#SJt zB(8nbPd1B2B&!dKUQvfT92g;J#3X277z5laUGSO6w4Bt#!tPIa&Uy8jEE|k+Rg!<)+Pds zu$U!4%wuHI))GVKMt?u+<>Z@!DhTSVr z&2x*2k&G&6o*BtDk9T{y4N!1(ejT7)Ar}%IKKLYS(Zyl2w+;?I+^RZa_kmeWGglL>;Orj^z98g*BsrL`yxR=JX-P z-Q1^r0o*cXP}Kc#jYcHz*h2;osGTgoYSHcQ*tztmE?o$D>+@{x;z?4CzVV4}#3K3g77lLTg zP5cm%O8nkX_8j%ylr#MOYxzf;MWD(|NqYiK&v(90B zu3Hhe>v_1l(-ToNdoPQR0lc@q#!k|M1@@nlcA)5$#IX{-7C}&m-0Jn5_m=xiYlS!j zUGVjm4ida`t6A0Bc1=5a@$|~74ws@v8Q{E*2B|75XvHVg$C>7l2^|gM4RMGPK~lLl zur0i01(VSJ;k7 z9Lx*%EU=D7vo_A=z?hhK;QHx~@-Ck`uw-8aj;68a+Ar4?KTXkG4tKU) zfxb>tj=R}3s2@_b@Dkg3LBQ7Uy6L#pf~V#{nA~gF{5iY84g3jnAQQ7u`dM#u_ApP> zhp^I`PiI+mNTeR~g@E$#@M1IL9uLFtnU|lE3j%VN`T%~*DUwroe1SWL3-zk3=h%+I zL;9YY=85aYPmp#2W6+s-+&eHBk4rsN0vghZ!-uYd2d!AZ0t_k{cSo~RP?DWA_9=ot zl^I$qs;j(5xayz?*y^qJjk^t^?UUz#@{1o^8951OpiT*Lm}@|q-Yt%`0$rnG#e)($ z8QFccF@vSD7)zfA!GRiPWYyU^LxSp(Ad6T-^)Ot+hN}~)km#}OoKK}Ql_=1m zbck=atd#LuLaN=7w9!%Yak68cupymjIW|vEMs9?yW476peKD&KEQ$J!V*|0m{-wp;bOz1TAvNX6~b$!|{~c zjs-juAq41?4+=qnEwOg2PvR^%-lD>Nm{^tDCvgzSnY|6|JndG7c3X>dweFZqijFo# z+hCY2mHG;tQO$hO5snOFyo#dH#hfrK96>ipxKgw9 zJ^2%9r%NJu=7yIiO0k_JZNc-IHR`sx0cW;_s3X&(PbFbEEY$>0!i52a3d@2VA*{`O zqeu1?$;QfZsh&7# zkow5%i&9r4`}9J@ux_&H*NiYw3cjS#Y+}!LX7rfG(2YZ@VDko6G07={gS6sSC&M{V10ND-cF_q-%NvE+ZoF zzNe);RCCgaVR_++nrNJl65i~!p!$^T_qp`YjW@{V%~qtx*o!K%R2;-_*)EA8!owKy zjlHHDX#&IBO((K1pMBNStBYx9zs9yxJc&k6q!h$DtPpeC5HaL^m)o!P4hy*L zexrCcX1NBS;ZNK3p^D~C%1*k52v;>m0933tL3X=LT`!lT@oR!CEiF-@szxi(=;2ocjorytDXOBpA^wh|RPTOG84?;=Nos?!YIP{XWr!S;VUl4dQ>>48? zu%xvZvK*Kmzcy@|2)NHHnUb z6(*q4V+TpCYN0o&@{XZ6L@$t7Yed2N1467Y97cBkP{h)}ZIu|Qd*KNJkwNQm%A{b# zh*hpSt4y-`A$E0|M<0X545(Em>m{|}5<*|4d*7D|W&8H^I(Y$KwD465c#*nUCqjHL z9DQ_7E%**6QwUzdFU;{rO1T5n(62V797}y6L}62ZnzdEnuDs(LtHGx@PbC&IA! zyg}7t5lxUzt~@_t^2unYtY;59zw`z!U9ufi48(5b6m{yv?Z~GDcGps?gW?g-qy@!s zY63l85OVM43_@TaO|T!@Tr=d z&8mX@t|A1a%OtK-YEhwNGYD=4603^LK{1SL(*|Y)3%sjU?3(Z5#20ckG_ChF_09)7 z=tw*n@8mNyZTnBG9Kku-oe#?S<@?`0YGI}4v$rQ$x9USw*c<}02u!cE2b1JKhrORtg1A!+HaH91#)^a6-nGq_awSS6)Q&&g<5T zn5F@NITH$QegS?i&TjPN{Mm!CXuzTXHvd`L(@Wum= z!hA}VQ%^qO@Ao+&F_8>a9S-dTFEW)y9*!UNDUr?XtCgXr>c!`Km+vUeRjEz$`es9` zk=|B4axd|i3>iqj8X@9+kt{*wht>?igREuXiRxz#JW>!So^d^>@eAY`v}Rz zkcEC>4%D}Q-vpV`4Wblv$@=SNKhB^pulPX~xp$;~Wb200TY4m#r63jj%teS! zzLK#IwfnrE`%hnDB%JVp~+Z6s1Pc ze54?1V^)YJFQ)p6N>!+u&I0AA4JJV$1=w6mT`c}^3ld}a;8FmH6VM!Mip zL86(-tcV+eI4N-|=dVhN9qrVQJd91(@`%idi5Rlxeq;XH!!}>^A}81-PWL^HUE%t3 z!4EooGMMUAE(Xn6Aupjr(9HSc+`^yxFBQoSB1XN{cyW&Hal+kOe?r)LT(9?57yDy! z%#)c%!z4)4l0G~!&UhuH>ut;A9657AT)k5p{vS#+8;3nmxsDz??CNUP?9j+_gHCC! z+k6-jwC`!%_OA9gVr>xxWpnanzQkQl4x21kk+B8uL0PCfV2$BF4^3A_7vpnbZZ&58 z8tK-N*nPAhR8My&F56%-Pa@3bn-6Y2cG>RkvO z5avC6%8?Kk+}10@l+MLL0ZSr|Zw_wmB!&5y>e`r1^u*^;Z)-W6>8h~o9-2xsW*c90 z+*7TN7AfF2p-HBok;BPb&tB8?@vk^*S1slw1o~5P@LG2&Cj;=X>$SbGHl~KEt338c z*u`!Qw!^+4rJ&{V^o?mmAf+B zp!11($+k8)p`VL@U6h}|wmofkO>4ak_7HY@p_NCryL1_O7(^5d$l} zRtPe#47}uA3X{l{takdax4Lu?XK9qLFQJ;pne+wS(u72*bukNc=n?loY{E{uH7~&l zb(CT#PAkM)1yXF@d@}s5j8&Vt-PwUJ+;|4UD@!`?wh^@%08o=@Bh1nhe12y&KH$kS z@H$bat_e5}*t!#Psis|YK`q@xZW9=2+pLy348szN7>Xl2grkM0E@5(Sg<^0ea>tb} z>eJfeP74%!^lD`qXz57Ca^Ll}gw;<~iRAFH9s*K8hUKyetlbqA3Pn4DioX zN0MQP+zp4K^0_83kQ=xbs;6g$l63aTryApl!q>*=x2d2^T+*j6uvG!)=`}a)7}0Y| zWk7XjGGc9lk8xZ_Z21BMczj8W8e%2w`Bv2-na&dOcsb@2B)e%Vuu0U(@AEh|H|7Zb zS%^ENB*%pU#%mXkMItp|LzFO0-9~CSl2imE8`c>4Rcr@v z-PTGwy~uF=c@N&VMS{1{VN*F)niI=xwm@A&{U$OlPy(4V0 z&;xpw-=%^d$qJT`O4+}C4ys?so}Fwgvmz_EN+m}n+LByJyZ ziMI&w6myI6f(}iILxOk*p|R}$fs ziZwrw1@ZG(t?Zg_)h#ZUh3aIF`N&l!MehoOhF4B|G4s%uWGH{LqGF@x+C$VY-)9na zPY3yvd}1|(llOBp15mEmoXqJJ>}J_tf(Ilo3o8xT$kBZeS(YR2lwM!>LNZ)!M@bfDI99Xl=Emk~ z2`sG#6T96KBwNWqmc6mwm$!?+KcchsuUe#WdDBJC9#<7$z*ECg``Tf_QzvBUGx_rQ zC*sw^vO2hY;^1g_WhFTTA|*Ut_23Q(M(?4ME4VrWOOLCmB*!@m!4j<27(s7uj2Y-u zqpdi7^Gd7uk$4`-#5UU}-6R}SBjbJ*2Zj7)pw;4neoZBgPoK{gB zh)`2j+L{Kb8=()UR~GiEscMXXhu$fYb-}JeIS~3H1G*T@2`I2i+1aRJRrvfyK8XyT zkQi%gJ_#- zAY0wI8k0)TSZXNF&m6D@$>#?mo~fz8ZyU}d8Xdo_zuC*6gQazf2pgXx{B)+pY&pi&1^bA6;DzpE)A7b5SFMpg>hkJ=Cr6Ex4SXVylH%mmQ_UJ5*di{O{KmBK zFelK*Uq5mfb0NLqh_0HiQ+!q>&}_(zGrXlT<=C4A{{@6tTh}=UR+AIGBirB3Y`-Sg zN68qUDNFrh4I7+mC$zBPydY~G{B!8eS^P(@grZVr@%+!a;+NL1@jjv}7dh|8HfQ%E zuX~Cll1ePT>t{?lU!WNvj%IHyBW25blev?=5cJ!X`3oI+U#KB>(PR}nQn zQZF8+oN2N4&n-{Oz?CVYmSxg{o}t^M*J#5Jy9h(ce)>gFaL{)d>#U_~>eiNJYv)sL z2jr`;LnyJS;aM-y3LWoFu7ygU8uR&tiJPKBq8B}yTcQ356)Hv835L3yx@EX0N%N6c zH~~5i{vrZz)meFGX@wG>KY{VgkUz5X@p0#u2x5G<8ro z4?vfo31u34h(?Hobn6Xk?{@AN3izQ~>{7XTQ`e*Mt<-78+yJkDyp_H3D z5E*eWa5NAQ5NP1HTGT`X*pWspfq#1iY+N9CAO^YsU3x2P14BD{26|H~15bZ1|!+u|AV0uy|yt6osof)qQYYkFmEJe6<=VpcY*~01>Xb% z0r~A$MRO`C-Hv?H+oX6iGbD{D?UfR2sMHJA& zs*_u3FKZ}ww2-w9wRA(rlqZo4w3eDq#IlP!uvCgty%g=Cnl7{{vlC_5G-tH9?zQF9 zKZ1qCTB2fv_-%wR!0~{9bOT5D+lT-E_56V&bhI@E7}8l->zn^JNJroRzqKC*`AxIG z%l-6&T(lqMYTE&H0fzLy_4Z9&KT+9VcKX(vJynFbEhq@cAg~eso8F8qto3v)Xj$m& z0RO3DO_^4)c0Q!Am-M$@j?!u`OELHxDb>(rhp5rH{Vka+F{3S}Qouu83iI^}p*#h^ zd7O?mRbRLwd-8vQ0$>hXN&t&nT#f9<>jV8w&RNgA!CM(ZGd%jGLRVK`BfgY~Cwe0a zry+xJ@(GU~1$r)=WLv41tF`w`>lvLjpP@6Rac07>S`#hE8%TR%>k6}wio6JwkH-mu zGEY@Y--}`vNlcRmHL_Srszq;uw<7XbigfQ2RmqE0V>x>BRziJwB;X-iZL-w%p%I#- zxuB^mi5ADNLGMGl!YqaEl;bg#`V?FS5vqD=5El_D^viKDM+vIaSfmJ)3=x7{QT~EY z{M4=nIbI5$A7i*nWToj(-4)fe*h?Ib3fJ}5r0&d(ZZEeCudK|stEQe>X)UgvP_X8;hvSWPKPlaezSDXSvOodCj?k#Kix`reH7cxN^404n=;D4gcSm@>>MbGW=mk8=!}LYd`qgx1jo@ zrV*tQ)e8(B=r*M!jiTn6qa%5_6=r#Xrg^!jn>O1U-vQf{u$#=@oB5QH6?{*^p@eH6 zP?%3WYo#Fn5-H!n{xwoCegykQ^j8G@8zLET&_@JdzsJCjJwkwB{l5StkQ@GQAAid= ziV6@QU_Tk$f9eYMyRLZu=<4^hb)OD@%nR_}b?EnfhYY|l1v36L;Qt!#k6ogF*Ja<2 zUH*|;AJR3=cU|KEGs3TA?T1KY{!XITFG~C~+5IOi{yrsr(;`XLbv5lH5D+V1hWM2NVOn|Mf zm7RsP@$a7dZBG6nb^IP(zw7+l^pdpG88P=51cdDw2*}Gng@4nz8N{y+_)p<~e4u4w zW2FNCaei~XpH6k(;^;4$9Wo_XGN3XTfKBi_nr_5@Ohcz{{^~!2{M-*%{{;Ew2c}5B z3i69lpgoCL>Iocw0I>1?Df*j5rXDK#_c-`{R{8I7@Z%bW_$L4c z_tgMzoT1dWHImrhG=Pu!Z~z0-|KbC`NB@uB|Gf#O9}Hj&+$jN9=fLp$w|VeB08+rU z)We+c8vt;t_MZU$io&Lz+nF&&5D+;1AAO+x!2pJI2B!CR!dGLINwmN)bOE;S(*T_K zLp5OZ3u%DQw+wLs=zmo}`QN)hvBbmW>l+!~*8-dwVB)*bz&#%@=Dr62k@_b9cJ~FC z&_`qS2aaA5m>je6{|^EDS)s08IbOGUcx@#d0%lgMI=6;;8#0%VXRB5zDBvI?41Qr=5uVTk%mD}g_jv_o?BVi%VP3h^ct(2& z1>9Z%jq<$()W`n`z`f%^(2dZN9+*G>KC`#ZKNLX!7c4;dp2B1Prxq|+e7O8yus~4y za!nX;b1e(p(tdA&>ZN}IaBmB2IZq7I{nP@r>kkFc`vnVxRj3yI+yZKw510Q77Qh=! zQQ8N_z&h%W7AV^KCjj@hfa0+F+|OeG@#vudy1!rn;+}3);56`)EL48{aQVMrfv*kd zNQJxGn1D66EzhHqkqp6-hw}2DW!{z^i1-PmBgqdMM zKuB?ZjDa5Le*$nX3w$d)OGSkH=K-^TA8;=CzS3xe_AmfDLt{%rD}bFg`|sPS|5AYd z2CpOn1jmfvIo+fsvrqvm}N z4H$%V+W$NY2va{4fb|!Aur4woeg>%Zzi*7Uo<3auFJyt^PgI_jz>@bE&?d zNOuYi~QRs#eWslzds)Ouj*x3`eAhc-}85eMgRNL;J+VS{8u4Y%Rd+LFUJ^xB7X6d pAI6^=`gZIi=JO9C{^da8zv~+U^4t1N4)`aG5(H!w*aHa2{{i1C(Bc39 literal 0 HcmV?d00001 diff --git a/core/src/test/resources/indices/bwc/repo-5.0.0.zip b/core/src/test/resources/indices/bwc/repo-5.0.0.zip new file mode 100644 index 0000000000000000000000000000000000000000..7145bb1ceecc4b034983131bdf96b8a5be3dd5a3 GIT binary patch literal 125399 zcmdSAW0WY-k}g=bZQFL$DciPfyH44*ZQHhO+dgH_>HA*KtiG$~cK4fKlWRrdM|>-G zMC^#2JHM2d0tSHs__t#wKvUu$H~;$t3V;LPXku?iuc8bI033*bsQz!_;tmY}2yzAt z0Pv5K{9kpHzX}llP`B76G6DU^6953>zpGo=8k@M&SsOT;I640-C?kD80D%99mkB^3 z==K+{^RK}C2RvGae+5;kN;M-zLoqW(PpR4vQKLE`H6^3cphzV_OBc?ryr4iuJ1q_P zr;%5im}jpU zMbyO=BqSAtJTx68rDa46#RZfV=!^~irGFZGbBJdYvQpLMv*Psz`uh6Ji^qrGD{;yG zRC3ZYvM)E2G`ErL)U`LUFq5)u4zdmq&n;5VwKG+WO^&Xzjt?@D)HTvHmvJ!vqUU24 z30@>^sitM7CuQbreho#Bi`UGzsxZqlyK5|NBdI4@rsY^#UuWfLLms213a20c>pznv z1Bl~2J4?A~DFHO~d7T8kgfwKmta$Y#MQ?LK2MeaTxq~LfHhC{iLsMA`#JQ5(USIsV zw9f!QD8WcY-T&pD|2)-T|J^-*bH&2Q#Oc3Gx&IvI|Mv2~!x*uPu)Df@xF`yl*jsDL zvO764n^~CsAEJ}}vyuNV=nVfQ{{J&!3h{`a+du#S-e3R#X#d;z|Cmw#dIlV;N!ekm zptRs#k+=#L7bT&>=WFy85YM{fv1Ka8+B6q~3KCS!ne*6roy0K1kGe_{2}1z_0`*r2 zLJXBqg-KFSg`rBlf?lnqpkA^mpx&g5Wz=^P=+>Q!-w(a~KHc(~eaUvJnf< zKgEo>HDOA>ilj$*_~6d4DBs!sdWkcxbWup+yTKHiO(K;FiZ3a1Dl*e;R4)1eQ`q?6 zXH#y!1a*zP;7rcb{IDNn_0bbC^jg~!&!D;8sGtuIIl_E-=*=0dq|<{lMlHwbuDEE4 z_>2^Hx_eK*bjvBR)5Ey->3n#bWe=*ZWKiw%ZI^yDZXC`!Jf_d2z)0d!)Fb+!Qp9mP zS5Zr#ebVT6@QR^+D+xq9uvi_`JE3flOc9AFkP{hnWl}2AK!VS;QMx)*Z7Vk>!?*QW zxfdT#j_xw1#&P0mas@uhd`7gtALDeVbeG+mu+(h?nx!zj#CQS`Y9R4Vp<65>TSRq~ zKNgNj2N%nsU+)%)*!M`OP{wFcte%+?Ofr#)-qj_H@O)0EaJ2SVcGFg|NUtb%ryXr3 z&27RJp99no1RjPdBm9!8Q zZoZlIbp-k?YV?`J$F2RE`=-&NqNY|W9;gPyy;GISEhQtPbykD&yM+XEltV1J)HuE} zCnH##x}-a3*Ki<9oy=>2A%`hej?rE6d~+WqVizl;?s;05rTFzwDS}w8fIQWWUoOwz zos#sKJ#8zRsrl$NAw8O8wWe13hM(Xhr(@n+K_yp9EF6HiD`Q|(0Ws_pz}C&GrKEIE zvKtj&h@hbrdz#IWD=9VL3=srei69KWKGT2KU{V08kkppJ)D+w(V8&0V5M?Bb12U$K zA7!P-mPMw*Xh9mvc4(5Sho7dld)GYXitP}aQW~9s=BR_8X2+TZRaRj8afU0d!8@&? znD?q^*JKi2@mZME<#L{;T)G{GG&xiet9MxtWwHF}>n8cZPV93HGFW{n^A~GI+2P~r zB2bf`AX|Q&NOPb;o=VA~y;IiUCvm?)F_z=l$X30=8`K~lYv`}8hYwfbxQDn5;HRS} zJq9i3VlC`v#h?O`X7Dym??nWdK^b5Q0DJg9n($RIyK({(@$g#&_K=*w%hM)FSh>$i z=`=YD@&t};ZgyvAI)#enSlWN$QJ&=L#iCQJM`aAw-bv|glZj$y+v+0S#E7*ZV;J637>qny{-7N&(glv1o2GH4-+sXGY3X?uaKg2wHNeS_poXU;kpV}*V5qoJ72L$ zPGM#k#Zosu&t51BqbHVutj27mrYY81#gMRxQ09iw=4T7!V3twBuoIGa$0LL->KQz9 zcQLIfTfqwiN=M=g@L8J+hTj>st2M!94MHC6+|1ubeOd9z+Qe`8p4+AY#Zl&I z$#dbLJwj#%3;AoUL46}4=@RcfLw(`ZK+M(U*9)ZvkHFX-}NEsrAq?eh5lp~oJp%2{tJW5_8cv{*cbg4f!)$XG}cw*2uc%aoVTDJ6zqq_}o6ce7wDQAGEFN6FRsWKL7E!Jy?|*Yp$46u+dqb zMjM)H)-gn`E;ek0$K0>Z?yz3pl$t7wn3DGZ&)g7u0_Uh3!rTJ-D%RmjGWK6%{< zWd&ZzSZkKdBK9lT+SzuN79>!t>`p|QY;5oRO=_D?CS@%XYGQh_8KLN=1-qcZ3A4`| zCQaBjwM4HXnKfgTay`WGcC4}y=($lX05Y7FVYI1PJ5^MpoHXqjN$<$mREOhgtvGht zL}Z)cBNJsXbc_mfi6#tE$wa}th;Hu0)8L0aK&KvOUXgEbsB3x}G%FldDs5b|^*fstG=SKNHC{aVJXh?jnsGzI5nwwP0#EqhWpGex6QAQ#lSKzd zdY4PFgC*^ICVt_#|MB`KCntk2Cd7WSrn1H=;BZ^AvngP!vGPgjB(vLP3E^XFnBD_;`tko3Ovl<$K$%e>%qbwj%^lcfm6uw`0W73hNS6mBdpaj|| zO|Ym}zQDsLv4??C{2nk;(}XPXD=g)R-KBMK^lqc!fHt!HyQD@*d~>EV*bsu#h2Cdz zanYM)w3%NGwPE-p}B#S?|Wa z@4hEaI{~t|IrAO0ZK`WDvb6TZi&v?ce|)7hqa70t9Els~P)$b%w@q!=?g!&MI7sFY z&eVTH1e@W)nkc102>QW<_~~Ga!qeM%y+5!Z(18wkZtvUnjUgqMP8kz$B3K(bXEP;s zDJWEo+1hiINMa~CaD*{EM3Ea-YBYgO$;VPDPOZTWw?)?ky)#Pl_RRsNV4iqIoF{~& zIT>LP4*@8eo~-^VqwuMuLLd%d4Ao`eGPy{*)Kfu8v`Xtj2xRYik&1y)I(N%3go`>! z-GndIgKEz@P5pLLb`MF{FCmV+?`Ed&#l2g6q(HyDjpqj{!&DQQER^r+JyqGxsXP@s zw6}^Id=%<8=-G3k2-ynvUv*D)H-_u=dO39h6i^)^+&*imQ9)U%-i;iD@3tN^9*lBB>PG`~=NKpz+$2P+fT-@|k@qlmx{Wx52*o za$kJ-=i*|gUNfg>k3HYE@LjKYpD`~zg_`-hz+smy2pW zuj}}4az%dIiJ5)Y}wXENZdOuUFrb6G|jO1W1v%g8cImsV(etCd@ zT(i`T?Yz2ut&917F#noq^QC=CnRuecf4v<`-Ti#u^*P(^HvRcv-}$DGeCa}b>#BYG zywCmgX~_9rQ0pZd`FWl8&3^yT`+hFf`?>M`I8%GYAIQV^`Uvg(`EKrgFHU>F-!;tW zy>H(hzy4;w&Gk9Gv+{il-2GT{`-`5>{?{BrA_I!}@{oMSy3vTWCD&KJX^^_Cqb9d_djP3hSYqt6VC@$;MVos`qK`ci&Ja+9`MJp#TQK91G-(0_`*x(4gsI+38P-@#?TjmxM7 zbTYy9IZ7YE@v7nG0{0R0;d!kvw-XB(Ko-(UFXqY8Sgahzf*v3ccOcWFjrzGcm;nk{ zGpCEHQ#s|aRw>eyO<~)5TQty_5-CHULB{IyL!BONqeD}m1`0MeabfD1vM5C{IZ%>? z63yebt{nPR+HbeY1r(gPKG)_kVK~&G84tDojjy5x1E~^fMYT-{@H3@(0u&x+ zAu|?JCK}A`h6wDf*bG-`;3Ds&ADJg2iKGv)a*4cZ5$zAzRq zW(*VD1aS**sEB_t*H)K*Z1@Ezg)zY#s&3- zlf&h$T0~1r>*mAy&?~+~WTbt!I0z1t{k00c`w(bKGHZoyL+0VgMNx}1RTO1VH3c|J zlC~oLibX@8G%$X_a^pUXh3s)2t;QuEg!oB+(E~Tvlj3tr5Lq|*baLAOlKBxkxx zbqjn$t9Z)$wzGs-8e|&F`HZROfb_C^=B>8PcQ?5u=@M0LXuFQuHD9NRLs2|+imXiU zb3{EswI}}(7;M)OO8 zN~D7*a3Jn%Jdx*3n&pzUt1{8^nThrrDlo2;phV@K?No?`>$W_bVI1Y*%1TIHu}bT_ z8PEYRd-;oGyAq9TYNMA3?ADt0g?(x99FtRJCIJt=&Sq2e+6EI0;kB4}ccZz{tB12F zuu)JAJ9EA}`6X`bu%e{CETak&bROFBhBxa_%;ibU{5#am#OuVyxNW|xLCfY8qe8}$ zpAsJKbGX9T3a+2?lm$}fl7Tvlmo9B`Uioqj^G*TP zQeYo6MhKGmj9AQYqeXg=F1%FePPSCDGYG9Hkghzoa|qAxufAkwtC5pfD;aBAuGu;! z|5UKt%;Z=42Yjen-P@Z|Nl|p0;7GEvFAJCKE4uCZ44B)%hM9$r%Iithy|EOhn9z{F zAQkPPdfKwtPjEloXwDd8$6Pe z+DY_~^#_HP=M!hAy4srR1Ec8n z`kyUV#q_nb%ssDkzHr<F7w&Iin-tE=H;>2$4!v=CssA^gU5zu*~A@N`wOD^c}@o_ z{fmRdnwD5nciZX~tNy=VX5s|y>os1!o&=6lL-Nnb>NR!q&i7@U=0BqI50g^r z)p-^!H*&SNh|n=VuQ-!JvS|HK#XVt zxn{NzOH1W~9xFgR+~gE*G`a?~o|O^3ea9etb%-Txe#u;3KzdJCfjv8kb3${7y$`rG z(S91}*%qED+?aaWx|X)DEVpHIWqV{+j^^Dg4xHj!4&Ic_e}s4It!If|bdb|UR=_xp z^WcFy3SLGpYIE62#4gC0B8s2p1;NdzR-f>s27s&deI}S|^O=XfiZ=9Z6J}kqcB)?l zBZ=>jc~YzYYJktGa$i#iw{;qByI{6rr@cix8cy0j)Ite%qLkdKrQGFl@7nxmk$9Iy zwN{-rRWiEXY&p`wYIxBkjKfS2GGy2j#F3X*W3WDqb*_3)SdhyT*?=dxD3nJF;a@!=oEtS6;qITg=aGmiGS(OAuu<)jgZ&sQ6Rre5$mnD%DpRoDA+)+O3) z#ytE4DYnL~XbZIO1qj@1{QZbc!l0=IV_47%dZeRR2&aB`4+9xaHHBJ>1rt9AZdSTO zvEpjbD6M54x#r-U>y;9Y_WHTQJ?jkqbwi{Wa&(W+#WQ+*gDP;4p-C&^%9pXO4L?#a>#GLkd4F$z9h@djAOM>Xv~}>cfUkTCH^UmH}JS ztk-Jrm-77RGHu{PAC0M!OP$-`a`6&BqbatBKvwlfA%vn*?vi8BHn>-wzwF2g4*BB) zSf`nD!!2hA_>7XQ6Jmdr-~ftU2W!51?6}739ThocWAZ!BgnHuByl{4UbJMKqQSfO> zRoB*^l=m7eXcTA>s~kJ~xLIUD2@AH1Eb12wV@c&3vBGP2#iBeA1QAZIc)ESA3vOj{ zuw`eaRoft|UknuBH(96z{o?&=wWH#hNZUzLSeZRxMN%Y`6xt3@nR4Y}V|hM*S_g9D8d16h4}x%*yy+YY#L zW}cF4WL}7+swc{CfQdRwF~p`6!##z+@lg6)E3eoDh)Ix`6TwD1wUcGEw1csD8ZfTc z#lli-Onbm`t~ow}QD`Vj;w<8OBMn<|`H5^71&d1m%dp|GuSmSmN z)%D{;vNdG0TebIeYbN9dysqYv6eE}FTZn)8DF}asK(IxUqoHgKA0sWvU@05t^39=h zt3KgkzrmCcoC8s3o)&nY!3-<=wB?(^^F5Itg$BHo?WBGmOvbQ`)_{H2UOX1}W4$S* zBm?+%-U$Y!Xc7*+rsY%ym5t;&x*yF;PZbuj1^VFWu_!(-xp?yUZG$dZvw6nNb5u|&0z52s6^@R2q{%xMpHBbiX zCNI)sveRk3;e{bti``AV>taEWIa!cH>yx^d&)fKYONLW2tQJv6vNFH~pwfWvDKJ&- z6IzJWqJ5$iSTr2<<)D;firH1R{n6u|re>0I8|!n30j$ocdBK5(H$As6BNOgxO7*^P9f z?S6R~yZ+DZ-Jm3fYSB58_j=_Pj0lK90f`jqO9Tu4{AM@M4Ioy52Ia zE?9`0XD99|bF~>RpyUH@shlQD$X7-)>CoQt$r-0hq|NU$FANa^`0>v(n!;|SPf%I$ z1%tHg5z8N4Gqep%O>NVX;gM!A^#|f1a+=ES{u3S}{);!yn zi*@2F0cTK!%$E0nlD4iKX;NYi_)2Bh>C z?H-`yB|4iClyI97_LjoR!txNd?!zc`qVj8z#G(y_=5#xC@K1dFc4tLMpCIyxwCfjxX<14%_^iwuT+|Kj=WDo6tdCfYGhj{hvbmflxkuQv08=c7 zl{mMwdV6_8i02( z?OK79&%DseH>M=e20hfZ7z(6X#D=_?5HHy3+;Nb5Vw1Qp<4{K=46s zf(2MyBmeZsp)MZ8d1`npO8B_G%e3)b)X*Tq>U}W^ z?j$R^y&ZfgdTiOgYKBF7GO>#~&}T!3<(=%x{VuSh0@kWVan^eB6vx!US>>?cYQ{K% zovCwc%z+|fmnx^C!pLxiw{BXh>D%L^yd2*~=wFH`cb(D(g=m4o6YTM@oMkL_mtQDuDp%~Bpt-sj0PqIQpl$_V)s5!Fi zqNCacIb@&#)vVR;CyXWNe8O6Hx;TE)p@n}pb!*O`T$|QGKS~S3NNs~XhI3V}q>A;% z4Z>ijoq)IPPibd&uAHfyUN3r2GxWl9+d*qIR#OIkF*c!`DRk-2o3U>+d+xm3byF}` zM2n)+xNHiVuB_?{vJ}W7SnjjHTHq?pW*}>ewT^a$r(m#VQedv0)~m9MUhbK7xl0jW}0nri`Njk5*9?!`*Q}D7aFdI#NUn ztqz~U(B>^)s63rQmSrDY?RtFeSDR8Xd&dzY`pUYOPL?%3D6Li$FuX+6T>p3F1Tu%# zL$17D^-DW5dDMfDhsJ0UOLW5r!6REZXbDL*-Rn3T^vK=DHD^JnV{&a&fNRWo*@IhG zn5N{!NSY2G8kUx!Kc^rT8>u>uN{^T13)d0QfVm5I+Z^sGk5!YjPLpnQKQ=yh~K#4O2{UX0}UY#}xoQMz( z>Rc)mv*T^68zWrhG5U@Guk&q0-n`6U^lcnE!I$)D!|nS`=Kg+}zUSu$=s#{ahp;0o z+Wcj=*#DLP$%eD-zuIvAH<}CGrBX&#MyjPHf_8#lQpQ0_dPbt+Rf@XyL3(O}X8&LA z3&&-9);wEm1ue7|nXTTww%TwXN&j5m7(`M-2GMkXoyAHryOb+SlLV!>#5m=8r9-`w z1jCsXqxI*6BfrrfXfTXDN=DFsV(i!~Wu_JUl%EHdP0&16?l)zad>xSpN^cQ9C}P#iPWpPbj4L}R_TA0Qj@K;jOV?>uBk->iHd_o@ zlcg>!6|g5GCr`>;TuE#TeWUZ%>MuQ>uY=(i^xd|{@x4iMeLn-c`g(XQPl0JTD@!^p z99eYOOSGD{<~l!+E`IG_S-3q2pUeSbvd(f3RzCFRBhq@R>=E1Cf!pK8FGl@0lsdEZ z+s<5S%Ij>Ur?Djwp8fiD@=1UZq5_lnDg+p?)N#`n#pPKYxHs{?i{&`iFJ>;%ufN9A z0%m7`AVZM21u}xSMe~aU?0BjK7*KqHy09Ke46$H>?&`pbR3rj?!L+->*I+;lZ~ffB zxYimfebE>A4}jxX?rsIIWQNQL5j@bl8t0Isec^QsVi3@HFn}II1fztQMB^OXNphC- zqo9!mK08jUcl)tc3$O7Vg<7;iqF~O`!<}o2&@R`vt&B>No)BXg2Jt#%40@&Y;OrNC zYftYvWqSu*m(p}u(xxL4L8s-Rjm{6F-6<%CC1~m%-)v;78go{~{%E-Llx#vWQCu0r zYZJ+46xM#dqn2XUnTNoi1sk~}*DhRdB(uYF*qG^V*J7PpcV8~`d?ZwM{aLGg^Rj-e z4BB)%Wu7tU9w#h;P!apH=vhzZksBFoRvxGDcFt=BFh+U!niT<34qKRQWGR5q_{BJ7 zEQjs0Xy~{JwqseCJXk=cTZGqcXyK`uIr{Ew4U$)$kU}~o(-L;XC!!tA+NLyH8C+0u zN{R+#a`95Rev$BzS_}`h5U3XLN0|gU>8BU(F=^2V8M+FtrR`AjFWbu2yQ#54ELpQfg9St+BA6!t1=7p2lJ$g0oL z1O~u3ZD=%J{C5Wsa(g6B6^mPDAsBJ{{%cX_VYBJP25V@*uPg%WggjBub!vC8J*sIS z8lWs74*0q#l3{9`U|jVg-SUdi(3NZR+Da_BYgQFv8RQ$e%CjaMzTO&z?aJ&vTo?HL zkk=mCeWHO`lqpi*wGf}Tix7@>gKXbOGDi~*(hm2x=Om>9$znU@dYt8+a6{xcNfkt5 z4cD;QjIR|Q!)fFd9*fl;!1RJE8acEjv?W#jxVMS!fc?DjyK&|dSG$!A^y;@H*4$9X zdfa$Fxu*4C-}ikkH5V&)8;M73FDK8F_WOW}n945f z1tu`=WGskDJDZy7a)SDZ8tB)v$}lL%U~vjbK?fvB`5ocJIzb|+@DK}@ozgk=D1Spp zB=|-51X^UqjJ-VJtU6Kl^sW9!%g_Cc!LxW?F9<2DF2#pW|2{o@qLrlS1PI*URN=UY zrwM)t{`BFvKm4QY6n{FlJ=3RC%*`gUL(6t%pok-#udAKmCNm+#7+E8A8?P{?#c;64 ziR(wOO&WrFgpPl3o{5a<->*g1bcK!aK+d>nJ&p2PuKEpd$b}PRiSJz;V@cZJ+x!!E zB5<{S4R`=;rA;wy5%kdAudvuGW#8-~s6D4wc_X_rgNssCL;@%HXO3$(~AtMZ^ks@OU_8t|H8ks^M$29Bv zVQRX?$Lp|J2DJN$Ri`~D-VZ#4kujZ+XoYA!U@AMcq&1&>%9!JB&in3c&Z|DP6aZ!i zW$F3l^y%`6(c6@`=q8)|(_`N<+>ZYv(f#^QUr~*tM_lLu0RTMz%KxOPS^ujh@Qd5&3kihfCoMru|^c2bg-9_HtQo_vyQd@_Lg zX8;yFeLyu!PctcNA|pjfGpiqT308&FvcRmutj5Y%zbG}u+PF&1zyQt8bO4g6tc`@R z)o*ofXlrX}Zf$L7X>GtSotJN0nVyq28CDqGOt5`8>Q7u8Gb;q$IK8Bd!=&`I1f}aF z{e%q7{N%Kp#Id6zEL=D|oV|;S0Y@I=8z{>j#>!Go8Tdc7<@UI=)Q!JOjOPDgM6CYH zh}`_|WyW4su2!60M(RO^nqpGA=G1=@6&PV!+g`UA?&l_q+39(q#l6SY;EN1pU$wnT5DiXZ{x*pN3_h9%Y{-EV}}*$I?OX1pp$xlE0aR zvo490f6!3gkkSgo7XANQIuG^zBb~MBIjb=wOB?*p4x~jr#K+RHKvIv;i%Z%}N=}MUI*-wh z$x_aaPfCj%IorWPhrq$xIyx9|{_>uIurWI|FimH@vj1M z-Q_DyY?pzbcIr`CW*6Ps(o$AiI&%5IwY-cMd+5;h^yJwcW1@iBS$vSNm;i_VUa6 zmFUee=le^$P(y0CEc z$i7N`rd*>J=_UC}l&kf&_-cK`9HwUf1YfFc(GmNAT?bp-@qnCyBqc=uCGos{2BfpN zY+@r6bDO&|Du|Om$+$z%pd}BIDrEXuu;2ywV7{h#Lj5&s=~S$p9r9k=icPeD6~oP&fspG%Natpu6~*as1QI;d>5nBwQf3fd{D9e` ziTcw=`5&+kwC+_&h~#b|-<^SGT0Y!4ELZk`b~BCS3_|k+nY0;X-r|#koHI-Q=nsJ0evN2wCYPh`a}*$Fi)yCS6)c z*XCNx2a^%lKeQTEL+xu(f*t?Ji^pH#}i{))un*gt)nb zE75R;nT(iCJ1j}}WoU>Dmngg52PzVMC2b2{f+`U#4w2`}lbf^lYGLSxUevd$;|34B zuBHpU(BFnV%?sHAG@A}PKJ{mZ*=0TZWRUjf9g1jBl9m&c#g@hk>>XjMICE$W!Z?!| z@ia=ln%cJfFoJOwGR$CSd>=^)sg@c<%3>W_6FTWVd)lN25z#0V` zochux1l|grc~O0I^!hiTQ-bzrc#CyS6r1#V&Eigih_3D}!eW+E_4~J}Fx_PAVQor+ zV3%tE6kbzn0XLqda|>Ls?MxVER->p!uzltkA?dRXu-0|LAEpf|I?kj98-;<5E=_o+ z^X|7YIT(_Vyxy}Mf0rcJJw{R80mvb-HvLfqPj~}7$5{I?E0x^(iiKO{>%+S=Di<)e z{6k)2W}6Eb;fh|r@`wY9$Bm~o8jdW3;o&UkxG-=Q(?ub|eD1U9X}|8?d@|?Ftgl4= z!8taAB*xGqMf(1e7;wShw4=Kq&mvRS2AOlpmtdG<%(d406v-NU&^IZxjzF+YDlg&5 zW&pTzB+KO~JhM%qhcHrYQm+YmXNkhN@Hf>w8*FdW1ie0Hd&PV#CwaE6dva^6fnG7h zlkHuxczF7N-p91WCtLDeYJ8~OK)Wij$rn?INk%v9jP<8-y^eI zFeKfvZ85kr7~pDN9i66b9RuMUw|Hs61?N}JVv620RU2J1E-*Vl%v8^W;vk!m3B8mB zQGo}^o!4FS1(X4La^}+Epi3s+P?ZEvfAC7#pe1A3iKT&dsh3Ln=fC~OTkAvwW|M0u zR)u2vS&maQ8}n=|QjP)zwh09Z=Tl1j2aa_aw~%IptSN9bO_6qaTKN2KKo&B$|s zFjats`nZC_O)9F$&^~$b%ezOJ#oYD+wEjbmv6Lc^0W{0zzshD@%0O|yiBaA5+;{u{ z5-M;-_xMMn6H7HHA{|9f@CwC913%Kkf{0D$l8*5NJHG=t`nN@0ac{=%kWq&51DQG7 zxNVw3zIg?$gjPOTWk(6zR6mJvXZEH2DBE2UYj^(Rwn%rWX1j@$`L7zMmUU)&gFE69 zdV)tttmP}3eoc-AK43)}b);@YuVR^~U@4=y#r&%Pi`_uebrh)!1`^4FYJGYu8iTcm zKTH_4d1c2sLKMg0KzspfuV_NS4PBE#VWYNnW4NYRf39fR0V_WVJ3|mLhDAzb`%oWC zlPDj%LZADsGp{kT3dNo7G2dB8(s1s=4Ouz^BKKlMXY$8CkWjFH0kuc1tu(+H^y*EE z9rsI=Q3RSEVvs0KNZmpGz}2y%h9BVta3Ai0cjFbiK^iH#Lo^1(4~vhfS@RfPCysS4 zRlI_MA*-z&YXq`xo&Kq)i0%g`!x1_`_=NKkB9SuRSu=B!Y9Eo5J+I`k635 zF)q;h`?CpPzKM|wSi^j192WpPBxt9UxI1Zn@A@xzBD=?OS;8U3ej}J9kvEWTQD@L# zD{)t^5y*VIIt zi5k%_#xqSJ^+<&S9x4|2!nFw>S4iER-c|2v*l>fLwf@ zpfu~sDF4;C4qfC@&XqpbNZLv5^8_uBOkQ~j5G?P;xZjg5dU<4&$a|`ZD&ix{=6uBJfOJ;?UtUy?*5=8=1m z#;^V{Is_Ud5@895`T@oq$2&nvk}!>u=GXz=6wMNjm4$M+aZfm;lcYSsQlBu7k0uQ8 zuOhEu;6ub0P$ICZc`Jv^Y?rTaoqPO8KIf9H7{?xRnGxd9k2yr2jAau-?=I}^BQTSNQcFZ z|Cl6i;)bv@^cUW!4uiR z2{gF@+1-OQm}SCYR2!=wzcpy*UBF8ZE}dFD7Jp=~gQL}4B#IUxuZ0Ge&mn=Cq~q7y zS~S)Ln&gDm*v4atqWHQ3&R7=IU|JR?pp_29!VIOcJLw8GU?`-kp6d+A?hO-MI1MY& zHQa}(Am&KpCLkpgktOVbunF}Z#W>*_m5|KN`;G_WfoVb1m-TAPinE5^iy?tc!2$Jw z#(Jwej2cL$XR8rd!Mz*S*+-r$*k6@@T8EK=a(f&>$^3yO#XSD0EznP~nAduaW5=nb z9A_!cY?cVjed{rk*AC(o4BSC)3(o=O;|xK!hDyd9_j+}uO>=1o&6CT!Qu2PwD;x4L z(bJ@bitUS&0HjbPV-yUe-)LHZAYlf|rRWmchpI7%m$@UaBNjK3mb&VPf=2yvVvw}% z4AqLO2H}Nmn3iFd!Z=|x2L6sCU^>{H-P;8tYy@!CBV02F+k_^xTLmf;Xyv}v2p~vq zIcH&Mox@bqBx?BLCVViL! z{D50WD{6}3(Z?pB1c$yt=Av`Y{0@0y4<&b!#MSqPqk>(` z13zSLg5vrWoV`nSKAix5hf>&8yEbP+AEu;SC%Co+Vz*vRub*zFn_8Wo`aWI2-aQ!Q zl5ksjhTPeZYXG-<7f#xi$3^jLSmBI1ax47Iu(EJio!bHCC`e zL*l$fb58oP@s8n93`ucHJfi~PdaH26`!bJ0vbv zI?xIlg2Yit`O#KQGJUbxze338@u(Xv|rlJ+x(!x=U$Ymj%36jff)0!yY&;;W6~ zt}uv23Ug3_xF&SxYfk$uL#3BKtlvSf6>sD&ms?!UU1UyL9RpX)vSk&~rme&>F@jGZ zDqeXm^9|QV2v5%xHX5AgWOiu}M|^?nHwshdf&|2C4VG*xb>ruNV$#o?gE0}4&f*aaV1}euzri#ekuK*> z5=YXxMNQc+%vWXd&ouno4%jQy=BOWX{A`dH4T#kn+l|{r#%lVIdS9#0%zh&gEx;7t zNxcs7&_dTucWuwUmR&uwftfm>j?C@MPmS{O#nMcao;k96Rl4cJzmCk~k>22jDh9^8 zZ489QL5DhOA*6zXs>r&=0wyYsFtD8d_PfbKx=t$6xN8ZTg(MhRA8Qq#KoIo}*x(a& zBj~21ST7lX2Ey+x#+isbLlzZi1L!~gO9f&jDJ?Z1AeBMpP%V%pgS@vmgMBCGhlGy4 zu{c8$7ih+!&KrqHiKVF}-o0J|J)quMM$rHWR9{E6#OnXeiD(po7pHFaLH7%Oa0`s2 zHlIO|tosLNJWHm|h61&Knq*j9hGDRkzgbOujaesN*b=pC-99q6c1BQxKItG|knP%$ z{L^ASJ38^ap4y}{52Q$3{u~!L-R{U^o-oqFQLUwH!4P>!!X0#8w3iCc-yPJmzy$`U zz_v3Y;)4W7*Qp>7 zl}#T;x$%kbeIx6JPSs*|4W=U?be3(_Xy)=M)_;gaL<2Chclig!nQAzULlUp)9KwAX z0U452|5wN>hlM1%N!!W^QnwTWHwi27wb0zn@ge5g5gjF^^=925SB}2gC|pfFC4(p< zS8S)fmK)gOnmuqd8{ugYs@gPUU1_C?%CU)7(XV)iC7c~wUxEsA!L_pXB0&LC+FpRh zvJ>XmIwla#cy(h?h}2I};b&PtoP~s%5E@J$B#pwNma7I9*@}_3$uST$bBKwrT>17s zw=IsAwh_qoS%LJUod%B>vg@1^OFl)`8l^p(V^H(-U^c2OKoJ&{gdFU=BOb`?)^(h4 zIH&?NN4f;%M>ZEuL4nxmQr4j-+xk7}xGs~D?b^smjC*&G@RU*lQtQjS2#Nl6baScl zQSTM6!CHHI&RM=LeGfJ;VD>90nFhdbE!Dk9Tl{>PTt0Uw()4de&!XKAMJb!0^6S&7%AY>cQJQ83`I9;!-|!qz zfOTe&5|8E}IFA|0L^O}ErO+j{Ezgv%9+z&CVuA+UXRDgIh`U!$H!&B?*c)B4d+&02~`m)LPNI) zk!R36t&Lg;tF9#|@-K75Gjk9+k==9YOIB2UfF@x}3s8vG8Sym2*4LRm%V zIS#$R;nT1T8$-#`>u=f7C*7r-$WCU3h}T2=+K;2vp-#4*37sqEenYN9QW+scbc|22uT0Q+5hHr;!JKQIr*#7hD@ihJR6N${ zI+3%8(H0xIWKv&VsjUj2Vvh|BBu}~}sxo)_7#`w$M-`KdfoL&QcER(%#-$)esa3dP z#tdXe)@vPH5lGnpUnNuxE6>qNAV%$Fi)en* zGuH-h+)j+iGPDQHa=ODPp#Y#hVy?ZX1g-r+D&@FLu1D^1kIr2u5?maQmCZk{LZ>_;_nk71j zCXs2KJ*(}{2Hqov#wdQxymv?Ci(HJ#0Zci?o@rQU4KLg0RLAMl-68d76T>0ABdW+d zgQz({|6;f({wg9A+>-=p-DPdP2Vz%$673F3;zIET!?Heu7Wb&}2aK^y+i;bMqNfMZ zTgH{+H!S%8+SUNF>-cc@XjMNZx5)rc7Jo!?@EY;iIXbkGJa&Lz^#4WKImGzFL|Oc6 z+qP}nwr$(C-LGxiwr%{|HeTCy&s)!8lG)X&Qc2}ja_gM)yQ(vgp5IBlqF1S`qSCL{ z*_ck#zAa5a@cEBau_&33@+=zxP)UA35B`mrP29xzB7#p~uSyB<9oB?T9rXViKdZB> z<3}q$bX$rqTKa@C>Dfb(8ze@S(>a>jVZx~X@VZ5Y0_Fm`#Fo@no(8EHOh-#IVbjLS z@{w+{+KPAT8>EcF0_w^)CRfgSLK*oC1MQd73!0xVwLbGXs|Q6!t1*x;|7JQP!u95#}Vs=#G1zc%VR#)TaUv z#k@s7m=nmv)Xsql<4)WVuBz5#Mqn(_WLxZomsH{c{CR&U?Wg!bPKxq7+~5z+{vYuu z2FkVAj%)Zd!e-!?!Ll^Rd_PD5lj=pK&PpA^W4X8^H@7)zX8H%&rYAM&@$~BJdMw*srZ!;qTLW4+G&fvXIv~(7;+n~e$Grf25g*zsu>CIxttNP+WSS{5SjJ774`qqslGA2hOebSYwTX)%M$=o zl`HC-p-gx)`p)u6HGD)+yn2afX_9JkN6}@8{q6( zDkkhaDpX@t!7ioBryW(JsQw0Jym-J!esVQ~Aisy|$QSgnMe2$^vL%?ieBha9hEBdi zoFn9!eUXBu7mRNURI*syxdY`!Cqi^kn%I^V%51k#b}2aTBSzw`fBfKGenRzJ|qFrG|jM1b;$`P;}n&XxY2!&=T9eFHbaohoS4IXeR@dVN41z{PGU z4_wdlZk`AQub{;er}(lZ!g~G1*0Lb79iy0fJYYd6Z0lZG8r)y#hmKn(&wqOheRd2r zO+gwxsScVyWVpopnuT*p861{I0}Jc&$;xd6-{WM05iQ~v4l<005>j-w5|Sv{)B)#I zC^3_1cFlDMuRnd}luo!jRMX2ti(Z1?dx(w$A0!~;Q?+3hJM3_XMoxT$S+x_~D=?1~ zfPnXz;1(gmZ%nqEmv0Ccq~}OZzOY3OCo8y-t{XP?*u@1{C}PW6su*v<@<)E`44k#$ z0YYcoPEIg&=AQ%2e?JUTPFv*hn$1knlxOZmLe~#M#8MVXRKe%l zWr@z!UtLL&RD@s1+Iqm+UDBP0BbMnO*FXI>7Tsnm+&&YGI-k$1Cc zRw`u20HZRwZ035P8pnOhogP5o))!Q?G7#SF8+!=%f`@aj&-VSMpXynlrcPIse4p^a zv5%mBRnz??27rtt92V|VWC(z!eQ2}0+tIXE>ii5;;FJU8+3OeH8>?A?(+Vn zMJ7;&A@F#e!^{s+CYn*y?Ych1}z9RZxy?< zfF*1}`3uvUxTUt26FW@%DzXJQgI=)*JNi1`gq$V}o5f+^CdntRy(oc7lN!^E!)86rX znJ$}_#zT1LJ_aSUqEcATq91=Xk08a1{C&vwvYfE%q#!lm3tJ78AR-%w!b=;9dN85d zmK>I?6q2PAE(Q%P6*rqth6Zs7BfYRA=$?$|uWcakDHfe9pQ z7!}f;E9^gs)_tqjIYqRRRh5lNS*6siK}P6LsQ)CqP*>UMVE*u3<^9*6m<2qWXqocm zyh`dBVfwsK*S_c zg6k(d?<5C)0Qv;NsLUE1bf&WiH)oXZ6S7pm-L)fU?_my?JVi{2eJg)KD+i+fC7=2C zxMoiUD$-Xlza;;xiN`0@c&okP4GXtOc{!sw4cT2t7bF|kmCIF09x7#)UHEW>aYL$6 zE8S0i_ja1~p=o!Ojy!YsFjd6AdsX=_2z`=Xn1`=Xx(4oPi4;0hE|$nCCEXK=9(btnsF@LwtpF#__ND8HYYroCY(2C#%g;pbPXJ>|!mY{lc$8+GTB zjbkNn2cy6Hujd7(=9pe^xqNW#!rI10<1LL#dV9=nB7dSB}rJ z7r7IHzkkTWMXr4RdN-)7s9#)diswqJbJ*^7soPYlZA-jjer5zdDGhHujLmmeyL;~4 zf7cB@GtFW`oe&SM96%`7c=N)l#OA5ciK7Yeyww3A8#N7opAYQ*8I~c0zQzJvBO?U} zIyiSz=PAn5i{u=cF+n_bI=jU1lP{|Pis3+Y&}KMxOR8aeMY6gKT(a|#LgyN8Vcc7$ zqzy`cYCNA+j}Y>(FL?b&#WIAzPh6ThCU3Z<+e`uRoGu%s8v_`E-YUBN4K6OA)>f;c zz6mP11Gl?iU3YPkX!9c6Oug%v3nbtiy2+ra#0$cmAIXrBI#DO0Wy3Exw;}+RbMyiB zj;?2(m1w)eTUS${y`0fs^ppl7UVVr{R?DD&eRUudO}vlHLFc@mADf>aAUvlXx+WTB|D9X zHmWz(EV;q~I&O&FheF9&`?_hSA(B^i<4zrspq@=~L*^*vx<@S;^spz+M3bE7`n)OX zhIwoeNEum9_+rO02P5f6%8S;naVaJ(BgdC22QPGzn`XCG7;2EEi2VJELT^3bMSH z>pw&5vX9obu(q~T%V=6ZK>dUcW6*l5nL&!Ig-Shc6Zwc~n}a?E*`LzkCXmIS>j9*C z%oF+8XKYLozG1m$%}p*V;2)6e;E@Yoz@riVz1SNdR~|`b z#kTR8M3gRN{YCcB^e#}0aY(3X>Wwf;69_*9_u7XfXzDc11r@tzGS`Vn)7NVm-F~EX z6P#nm9_~D&)eYICOtwn<)1pWlqb$U3mKPw|f_dS7rzh?Xdga}5k~(cH6`8lR$@N;e zUeTpe3-DtFA+Zf>ZqYTzbSNbGMvnc#av!Vf@Zw)ud4}i7umdNS&o8ow7`(Pjk~P93 zz9n=LWGSZd5$&$qxG_;}3fr_B#dlDtHVjapPXj*n0uzg%%Z*Fsqd-|pyw4X|hfR-; zUPwk@=~jr`^g~HFDKAjA#i~<5u45~zXF1%lL1r&r*E#P90%0EKlBKUh(%VvFdL!MJ_{O<`Iu!X2JapqhhWm=7-J zDjH_JOK?cGOZI6SoIh%pHH?L%n<&h-LgyE zPJPT8%Xfp~XQwkJXmp2+U${Q8#iBf+fqAzTZA^q4Yih5U9g5CX*(zE>qlgvr*tEK{ zIyTfQeadS+a(0w~4-vorG(j^jTtgWZ*B=)_dMde0-3I>i zSLb0TS1kMRQti;_8@T0&VClVfj4VPaND4gIi^)*M4{4avTvgwf#eQb>hypUM`4QsV9ee% z~fT1t1VC34*C$78nlcSusrUx=m@5YV=nzfXx*l1WID<@M! z7cKpll|jU@ec}W+){e_R9KN8fsB(c}c8{mSXyYluW70TFNNRkh0_8DL@r0%!HpB#xZY{n;J!PjA=xUDsU`D zoyJ&pg9I}P|Ak*?^z>zF`l=bc{6HeN8qEQi`wZdHR8nQvJeBH7`G@8Sg#Wmbm6rQD z2bY3`lk)+I5$9e(`E9iB>E6pG&Ll+3yiLBHO?y!xMogAROhTrlX zv?(Ic1K-}i%qaCiSP==H`@rD)qtTyT8g*%{l6l17vp0aDwoc%ttOy&Yg?dmPy3Pdc z?%}HU7%R?A7)Qbto<-psa~#*P_Q;S%pJ=jxr1OdVn*bdY~z%4Qr=h`;}W zZ+}t6Ic}2OwOZ|ZPDN|%KYOPaF5Yw$JU?5X&S&~Sfa!2G!1hd6gLkX=Uedu};p>QP z1s8naOk`EPbeyEQS2odFq{)MKjTh=!F+k-)nFKSlGeDCXD&xQ6U+x{g(PW&~_=)3K zwzEguhSWdq>v;57WafZ20!V?JXYL@CJE;m%&X{9Y4$XGT%W|?DP=>y+HF->~NV~KY zO`4J=D~r!o$y987V1P)loVU0&Lo)BIv99tG`sxcH3%9!rbVo*-=i|AOu_lRPeghW@6{%~}3#YDXzDT*~OHUTS( zL3cSPhOV9f@Dg0qbFjL1FvH6S^pimOLiHu;lmLl$j%wW7@>ENPC)qAVXYy2o86`I1 zG&_C&eGsKgZ6*8+v}tzounb>$IT*sacWNDkn?$MPkc8$n?82=2o$ub7^aITN@`w_5!Cs!~_uI3-5Yu2oos z$O_ujapj5?n>R@7g_jG#o4-h(e5!xKB?Bkne=rto;W_{IFgQt)C3cih8Vp0Rzv~zi znS_fjiC|xW9c!0@R;DKOn-QEF<==(NCEoUuoMZ@2wBvlGr#t8pk)0#RuzJp=>orO# zu$V#I&e`lb&m;o)+jrrabF|FkeZD521UvVs+bzP1W;3_Pi&!YUAL|^o9l?8&ICczQ z5jI^ufT$|aT|&Xtw8nc}Ql23-D&|6ZAoOrL1{RyI?r56wG*Y(mwfcPsE6_n^JmvYR zkJ3bj_ZPLNM1CE86HMtngp zgjWfU#|y@bLyuQmYWr9F!TCEC-CxM{lvTZfuwpViI^sD8OBf<5&W>qXaG_0;4@5aX2!L?1e5D#UPx9N@xjkL8Nq8kK4->kR0;cLiTPP6aVLQB+P8kugK#=b~A~CZg^a`dnO|Q^j3Q9ka zvn)2S!EYV2;jkSFSZ5G$2~!i+cidsw$xK;B!=Vz04>6&&G?ZJZb_w^=l~lg|^Xj@O za}JghS?+~BB&Y0znNKGorm7>?X_-Vx@b*5{Qx&i-O0qC_s zC^T5ac|9MCIe_3M1|nFLmlPikcqavl+YEC#e!?aNvv_n>gT^fcbO}-+ZL%1=P8QHYuH4s zoGMp@>r&0$J~?SsuC4(H#Vpfvdq#ACsi~QlS-?E0W_O8*iYJZSjrSO-pb1wMZ~WCF ze`~BLv+fQl*kXuo^o-*mu?lw1I^hq}tEY_F%D8x>#$V`$co8IDehONA<>Co3vL%kn zcmw?zqo(^|X6>$84XRsaGxIbp1K7CGY<0vo+DveC??2A{e3j=1{CutL3;f>xzRqdn zgSEdX^nH}>7d$BZ_Z#r@S)KU3GVx<=8o<`nkG)~==c-=(CI2`6GpOi-NKfhFlJD6d z;2)&*g0EEneN6RCBKj0(Sh25DBS4{EWAR>t-Igp{`Kd0y#Mo5MZi~4R)Es$$B_DNC&25M!lwVZ08@Q*?_()^@8jHH zJ0RiN_cZqN1P43U%Yey0WBJ!d<+n}ii_b)w>Dfn5T1JxTcIDFAPnknaOl_tft+(on zP_>@MU@5t;ubeU*K=rCBvcT5bcbaGZ3+QG1R(%&&k_{_UGcc%KyWabW(_&RiCAV7E zcs6a)|7~@+{NNxT9eVq~e{B_3D6DrIRWVQqnorI1*;998%Ur$;;L>sklh&9i(fW1s zyU(jDKdTybsD6I^^YP)3c5%Dr@=TQKUq^Yd7GAYi?6VdihopY)75>3@lx_Wua6|Qa zv&4II{xn@A-~T~wHP7e9hFXNv9PnIlfbiq_h4543`#ZRd@SFbo;_xHB`nf-N`(@g7 zeHm`-cXMnI`#m0@^RZMve)DTO{&*YUP|H8w_ljZi`uo*>`%5s||F~~^%@Od~@9)U= z?)E8=jsCkY#P0cf5&y%P{qtwYfe+K?`T6a;bNcrClECHFQpzn^TwX|Nw(78jJm6Q~ z@81dciZv-e{CpN#kPosymJ#}8@_!YT`o7`hdVVCT&=k^sJ|qH)f7*YC^Z$IG}5K9JQ4)_miwvGyG)P(FG9=# ztwcQ|)cn@YX}&%>2<^ZB>il?I@%+LjuNm$JbA?Uj8#{w0I}mY&oM2}C@{zaCA6s&Y zMStwAq$U*Yu~-RLQFz^6I^6t*nhVMls7NNi#`>sBQyG+ff<)g6v^3@7E2?&UeE5hb zC6(>gEocA!c>occ%u##^JA3`$c&*``cZ*m4I`FvExycky0#1nDh7r4Z?PI!7)2lyC z!e@JcBO_6pU3+!CZ0fh7d`4x|{&%-Y2I-bz)m`4QRN`x07GXLXa+gi4k;al10=|qd z-w*-!JOMvPw|th}zsZ$}0yLQd6F(sVO5d-)KMcOsjR*QB*FyiwZZhCqLg0M?f#CS& zS1au^D^Kc?IY6_hrk~?UuM2*5MwVKdQc6l*>&3=EMSST@PTFDg3(4=BWT!?$_?1dp z;LY!aBP~_H^UmS+*Xd;~K(XhS@KxZ8_4Padr#JQE_t67?1`J;|;8$qnu^r&w&F}N8 zAl>?}T0aB61f1Nn@a4qIO_loWRs&AgT$idUeFO?f%TT4NJ_vSM`ei1B6_v`IrLy|# zm-l2YsOps!*ia5D7W{KOV4Po6J0o+dt@Un@Y-zF*Of`ihj*7Dg`mEqp6gs-Sm( z@m9Y4v?;<^`joE3{`f0gR+7f3zb$+7=-o=F6N;d7&oczP+jK=1;RF;^grn^}@!8;= z5AL0yetsvsf9{>NloWo7?R>e7P64eHuPrH1GSqCUuJ}}X+f3f7@Gt*z!v#~DMvm$3 zl{OFnD)9PW+5_GSo;dp7uLn4^qURjGwO%3wX0!Hw?q3OV{C+7y^-c01RIDUk+J9Je zzPg~*i$Iviv<$w@nn*(Oe-D3C-(#8f1HWmX4^9yz!dF9T_gr!Y-WcL748Fg#4EWy; z3_cL9cy0+h47V|6^Vw#*07 zXyt=WP_;UDKoz2sbpeI~Ht;GcNhYl5uzX3UmcTV zZ6V7T@mn{(-l}m4F|W^q`6t8U?&FPcti#ge1*CG1-GM!I?G^U70rVd+8irHzg-r-q zL2^arbNmn%$?=)@P<@~fj378J0Yn~u;yeA`X0+`UxtjqvPh1p4{s8g<1|)gDgOA+? zxNWeAg~jaoF{9#}tVAD--C?*>9P?Ht-nKR?fpHB?6FM(F--y;Y4NaTXJSVUPA1EXj zeBx3NcF^{-soOzD9|WB*ZbZYnW*bg3g8Q5$`~zmznt4}pDnh^&ZBAnz(FC5b!#N#C zjB|_mb5zCV+Tgl)wSedr;*H}Wi93`}&fsQTgGg+g3ra)WZu&m13Eh??4buqwxp>qv zMVo{{KzRPMy(WYeSlT{X&3;X5m{q0@s98Q;6A!GyJfqC_93x^6$Tz^c!g27Jgq$w8 z;lH`gGA>6Bg8`K+dw@Nj35@$+@`Q=+A-c>C!s{M|E81rT z0$}VRPv*Hxet}D~L^HW$*i7(BLQh?7BKETd8`(R++JY2(AE|I8_IOoJDEFr8m^>l6 zZG&os57W+|aQP@^HVXu0^5g+RjVyxGL#{DK>#5J>llcdWzh?wL0FO1k$*u>Xc!sRJyq*yhOW`?2q%f=adcuoi z?ey}Fsoz2SLL_+4n5A>>am%##f@?A9;kc!esAricdb!KMU@*iAJ)*kr?1Oy$usS4jf|vG<@SNe6PSn#n&k`$08KHAip0Q0yANz^4Sdk z-cgrlw~fVb!w-tsLskVpw=LlA9H65Y`r%OAfv&D|zIE&P3hf?g4z|l$x4pctQ$ipw zZL@Y^lg)0Ql+|8Ku5Y`5CSRdt<$j@1j`e=*rXRJ6B*2B)-C7~qyhV{Y?1zK@CYgb5 zG2<#l`~s;FtRc%=~|&b*`EnA#vW4p4nuI-HNX=5Gi&1VDlHG zr;x7r!M3<5iPdPE;hp^hMK3d311yu>a)s_uXPrLmaRdS_Lfhl?#cM*+lWmjg=3PP0 zMVT1?&>%X=8QJH`A@@D}nZniAgdiI4WQkNJV$jqhYukY49imXgJ_J~~E^R|%@QzH- z@ea=^Qpp8+4%dLhx{pKKXocdeV)(`x8^tT+G-BlpUEzXmDm(ORk12CE&XVZ|`3-UF zTJ~b}h?mTtOxX@&|COM14m}P16bmM+ddbiIedGVJmn`?3aG zHxc%^^c(c+D1-`sJzzkXY@KTck)Dk?8Pp~Nl%740|(Q-4%r|4r*_zbTfY(BzLozL^T<89PU5HRGspOT1$o zL+>=4M7J=8b1A82jYHbs2>7E^p^rp;^>C2q2R(_*5gSIgimG#%FJgEzU0H5sizMNQ zb2RKZv+D?v@EWkMU*sq*h!5={eR{UAJ7}|v@50y3`q$>l6J`U+^WVq?BGkMUImHxDy_s2*Q%)A| z+BQ)W+z)5icW{;JjtOK5&M7_Q0{Hu$-e*+cQ^gkSlis6R5JWz@+=F9&(Yh!Nb3>RD zEVU*2{O-WPIwnaxA!aq*K)kV*uzLK(;Ok}>$=(vR+kp5!w+Vx#DysB~XIy6&%-H_V zspjxiqaZ$29d|>_*(W${JzQOc%pTZEs6{$~tw#ajCQI}t`EfjIjwi>$Um|e=qQL>| z!9|egS#Bj>{(b(nUksuCO`KRH@rYLr;MaVqt^ z5o7!Ks)zr|wXT!yh^{hBzo_W8Q84tdA%BpC!x z*sO>QKFNHdAX4T(SlvnE7EYSod!KNHulTmYR6Iq8%#!a+Q2znI`~P#eNMkfhmxKxj z?BhyknjGskyqL#}yz*-6b?r90MZz#tdb^Msf9>4(O9R;)wIp5%K>YGFw8>;QcT9Z< zkgOy%$V{uRD$lap&}fcgJ(-d3V~_A8NE~1ScX>tVeaCTm3Ku=2wg>(hB174dJ{{q< zaa&OyU3tyw50;y+iW$Ok`HO=$$7p>`>5s7Lf9S;i$L{+J{uL<8q9O6ZJGZtG5@{Bx zr*H($AxZaNCKlGO44d09l?Rr4_+0EvR!&&{nL0VaE+B}+(iJ+4>?SSk7ZR50KvsJE zD*8R*J|%j&UtFQBzbI2%=c~Q8w6Sd_4mlTlk%;O*Z&9!t(W-YpVgpJ;$LOZrzm3g0kf7wbp>Vi>5_Hcm4J+#vNuc)VV>R7vF-y1bZ* zb#`Elb?de!NJLqj@gdoQhGiU|`KtVarW+3SvDhlF#Nr)NWb;1@(F?nWksS^nX+s}z zaYGOUuaz#-(RB>H;`7xIlLVtbTq+%Lswle$cu+};B(q~ictH;_q~?&E|Q`O=5-H3^!sUl%4lkxG_@G%LLsa zWslKM+7PbkY9wv)aSq^OH`py>zF6SBshBPRlFL6bRQ#Wvv<_)V>GH(jrTBVauC16( z=vAU9X%Re1e+}zY-|zsd!p^w!FBmz)0ojW7G1GBGflXe;EmmxUybS_HOxyzW=c}L& z$ZSW%o+q{88<2FARjoD{Ek#$N{dpcNELq`Y~}aY^Rroq8*% zCZSiJ#XFW3+VgnE83&OP6PTPn6-wFy^BxF<$u|-9&@EN*N*8v73+~bz{I~;Po)JeE zE4(RqU`ve;ynO8e6;~vPu(y92gUu9O<34We0qsMwFvnjypV0P>(AC}WCQNMc*+!`o z?hNnPQ@5jzL1>IGHVwX!lMB?1ToT#6CXlK|t+WT!_r?d(HU&OfS^*0!=iG$h<)}~_A(3jj<-?PAz{q`vA~pgK{LmB z^NVAR&^Y2#4nTEJ@mL!teuK^PNNuoOLV1H4bsF+ zk(;M;*2Dnc%#t^a-#F{&bC=UZibR9)Gl7jwP65T%Gssi?Y8ZZ^8CQYy^evLkUc#qc zA7P7Txns0kGwKM+OAwFv<;WVBxvYU0KgDM^Xj18BesOD}A>8r^Dd;OC+p$qvo^Wm| z#X5bLe8bme@Hng`{Exvx8hmzZy1uPJD^lnmm9qV5jiY<^axUqQ$?ZqruqgXwhx z!>R!+p?^dWA7kn%VPkq8;V5PjZ#qay+l<)W^aee2_rh1IJsRzE!@jSG)XpUsB|8tm zeuKc5vy}e2LDsMIzOMrNj0uk?@EOJ_VScaLKxVjKQt?jv36q9x-x`z65oe6Yaf} zo?eprp1*SkOJBN3KVP}f>Ito3BdO+>|HjJx#gvghl%Y;GV-;8F|Y!5TXr7?~xN>ithLPAqf7B#gWn#0GsH0a$~>&)N|Zqq`2)>*Y=IbxBbCIh)g@zgkU%WsAoGG5^Z ziQ47(fF~-1;)68pBHd&SK}lmSQGtdz{g8AJ7<-0qnUKh86_sQjELDHiEeFV&mdQCr zjJzDTZ2u4exP?5RYx7iuPv+)5gYw^zR=H&=Sq8I{X&prd#$I1F{Xj*+nB;Foi* z(*s{+EcyWc)iis|{NXcP)(c}A2{Yy5hJ-`?MjKk5Vr!Z~GWu{`5(lT(kJ9!=xepfb z(_YmU_&~k2%U?)VU*{lI$8O8?kO%UK`9?Sc@b6rBpuR+=VZ&~2Qt&0>H9WTdSj|8cKP8@WAuH%=kPl&6J?Tr(<};F#B2Bv zkoF4F0_J4XHiBGUpyAx0B7Xq`vVNpo?UKsf`#=BX&TXJ|jcWy~jzf~;i=5$Z%_zNu zehs>yD#vJ|qzxwafmbhSKY#>4&T7nOcAMLjh#Dw2nMs~^gCF_H6wnr(rm%XvME1aU zGG28R68RnDGF`6R(QzJtZb54u8-G&o$lME<^$nm?vD_LC@`1*O`5{t78Nqvg@>K&=QC858Y&=BE-j#*FG?XK}9h(>$ zm2l+;J$S*rb=c5G#lbn+54~;?V?q{eJ zgr}>%k!VNRjkh4k+XOskxRt7UMmc&rB;8cM;0lkoB)Na8T>1oMC7DkXM#G!}B6t8H zm#b7$WWsq?9p`ksi6tm&(XNgp?nCyBwJ4iJiMn2aLdfY`6zT-!z4|8|*RPfKnqJw( zWkE-ip1J>X^%WI)?yQO7^`On~Kwh&AQJp>08a3}%7zuDAJo)4tttFTF5^r-{M6MG3 zuA_Q388Paj@gq1pXQV_5p*zaXR??E5hPv6 zGqZ5Ts5hTCn#oAOu%f2L5QTqy)GAR6N`bGo=&qMUiFN*XK3k<%sFB7g8PX_mW5Fle zk~S5;tz)zNB_LmLuXcsEOwU$zcM$3WH*pZB$aowmT{ptcKE(6~FRx(b{V#m3m|L2a zMTVnY(lh^{6n68Dby|~TU|ImcwmYH8@I2h+wFQE`IpZc&NSZ<_*7}yNNf$l!oGo%IoZnocyh(IKkC*hN9l+xl(Q{(Ru!|IE|DsL# zEFbJ02b=KqX)Bx&5Z&QUiuP9ye%$7jv&c!Mkc!hYEx8 z4pg=H3O69LeiAr(afJGUyGAJNZl^J%Zj?MGLA}*{CpjyLYuB8{gT)_)eMQ$_*+I_o zpuJbOi4D94-8mh=p6vU@bpg zSM`>~yVd*Gv}|l^p7Ph1XV&9`q0nX_aT66(IWi=K$w`u5YkU?YQ=$gFH(L zA)s{(z4C_Vgi3-x!9A6thHt#oRE2A|2Yd{hf5@yXS0tPLpQ%fhp7E5;rR<5!k~Iwr zr8~?EI0bEegv#<7(_+fy1-4d$8Q2fwY%aj~9R-&6SVYe!l+r|2_xK+_r2yMqL%jnm zGH1T%(!{pjKA5!sCx_ZFLp@!!=qXO7XLz4oC(yI(xGv`a63iV2n_~mnjIFZmO-DS8 z@h%f~fzG9-*iD%r>tt~Arf=4KfHSO4ZJ(nc-c5V_O5%-;6+h%ev#T)*E=oQ9xDcmr51KWJ?Grt;$CtSUM$9c@!Khq&& zP~nT#zR})dSw`3Gzd93tq{)WHNVgtcE^?C3&}z2Iks=aNRn{4%A8|{!>*Vgt0sZWv zNcHSzoQ61uesGIIhHFN3qTL@zYXO0O^0c>GCb~luw0S5_aW$nz%2aVi{A^;HCJ7&& zgHP1Z0HA$lh&MyMxjEuXC0fiNz!LUIrNY`rFwZ^6bd=Ip^l~Qx$3CZurIfs;c;@?q z*Y)ZU30%2cSLKM>vsJ|Lc{ZrM*c)PqqOJ!T#Aal}qK7@AF^(?yLO-8Gjh6zdb?oDKL)DToD???N4LgjhV_)TP~wb=pF);=`^TBx0~zM@ zi%n|c0XcDl5>5F&ycejG^wwAxcY<%k4>r;zE#TaRtqVU$3umw?!8f$(3#yFrTz7#k z`jj1f9CEJ7A+c6P`eAMou}EI*!iCN;9?DIiFe|5$2mBKE(q`%#WFDrowf-1|QXi?l zPUt9C&49Lwmm-rjx(P^O}Y9wYKhOZ63W+4G`Ks@)y3k_iZ9wzp+ha>SyjM)@C^_!y~uMMLK$m zW%TeW)^O@lpRgE}QpHJc5hQy{1!m-LxuWG{|M8Pim^)MvS2UOZhRPgNtDB80MDofEEcNc&p~!YvB;u5d*2+C03Wtmcpi;G)6|&z^_G z-dl9jFhv$wt&cbQykqGq3&J0c**C;Z{2JTf?uSP^HGG*{hkU?m?2pkY&J}YDh39Mv z6SHD@k)NXE|_Z;T1Mmr7YLFK@qeh7fO4>XulcIBb-|ICl@$B65{luYz~<{S8)bU zUPGM3{sn!VQbT%y85nm+t;3+-+X^1-k}ASkQL`@Fg7?UI@>?1TaxM#RCd?ih-w&5w zN`O(Wr1ub>k!=rP>m_b-IwNse3OT+y0iGWmIP1QPA_Wg+CUa*TWBv&%TA>rg`0p3e zM}&|T#bloXw+TV;J5}VLMzAWsIiT0P{BVqU4_Nohzp(|}jzO6xV)OL-U{k^%P@0!w z^+|4R=0UuQBV3~UF4JG1^G8yXwP7^Iq)qu>+~M_i(Ek!m?}?$+_St3Ae*|~`A>14S z=9&~bT$c&s4BKm@kOCx+A2LqfOjuSr~~)vq3yWbx14;EPf2>PtKndR(r=- z4z>IEzC-C6vK(}fhXPpEf!&8X&l}! zgJ=TC+_2xMH6)nzgWLfN*ZT|wKH$$BC7@!%2DMzmCXQoX5>gae5p8d7L-8zS*GK%K z=#8HN-@8u%Z#DwI$*8t_zXAUXzqqzMTA{x7-+#m0K>shitL6WNU(D%+NWmLjVcS&8 z?Y`}0<73IfF)zV~Dp`P>@6GKdDbeXv~$Y`HVd_1{zJ^ms=6Ej-t;im5iXlK!&>pZjB>-8FUnNvllGuG7_L9?&Z z@0sSRJ!@TwvA};>&Nr{^wLU?sJxi6S%kuh)2;g>qZbCiP*Xy-i)F;^GvAJ$~dO*1K zGkb2zvMc`^EmgN22dQ@BezV-TrteSD)4L*`iMm zK<)4I|I5Fo5MvO)V4TlmGt~GLt%3o_6!5vgkk1lu7-|`W@?tf*9E6g{bzQuFTl-SM zzzqq&3s{V5{`Yh&Rz5maL^>MoeyN=1a8}pCmTBp{4`Ik_J1*ak+ZOT?psDbjx?&k{ zo}bYb0cgc;p`FEYORwCn636GVYhfiuZA)paXaKZk%9u0C;=t~2#n1$>!@5Y#s5YE( z7F7KHCCkMd(FAqstPISV8Glr?cy;a%;|^<-J`Pu+2@Xto6f|g34+yhLJ>Lb6?UF$; z4$HN>&t}FtdqjQ#8O&62_wA%6*!hzyu~8@;i_mWTm>3#V0BbYlyn~I7eDgVF$0WG% zw+IZ~x!jGZl34rXT6Nd(HLco=8#r6*=(UP*4n~3euvc01C5nt;s~<}LN|&4=gXk2i zKLZP*Q|pd_9w_lnC#4(>^gAo!$nNXRsVov`@%vMRyyEvw#hEU2k3*d~cp+<187HOK zDC5-T0S`K2y2GdpJxXTdhgd{QU!INHb_b8|)+=714n9R=cQ4RuqtBd32KC+>^%*WT zi|Ks9>4n{79Xca32fRP*k45olIXRf3jhpb?OuMZe;0Q-ED~u0#qq(&W8I3Bjej$N) z+%YZ;OvkVPNX)oD^$n5tCTG2_0=?q3Ee|%T3||>M{VE?1SY-sd#TRg}HIe6^EV?PD zU5n|4z{8grfO0Eq*~jVJWLSC%4`U?j9z1aDHSneFRjUWRKab;b5uk!=>&C0_!y6gU zGScGJ9ZJv0x;!|G8TCr*f2urJ7sCc^0IVxCClzm^4D6h6+K$E12-B#r9V;UtqJ<`? zupxjjjx%i%ZXpyIxl%M2fd6dH9<|@71CedN8mj=8c;!>M2GeSp(BtAB^%ud6LI6b6 zr0PA7n3ZwD2t};d;4P|u4Y8}tYtngEN0t_%oIoU_*F6U3xFc@)`SxNX>m9K)%{%*J zvKDmN?0(CCj7a*l>7nN}VgKP4^Q^E+ew0e?@aTMUZxpE4z0ddk{>VH9yk#jMR$saSi9d?T)Fsv*WFVd3*L!8= z$l5eg=hb1ZVrwf+IqDnk#t5606;6U(jcz$7GF7J<*)Mfc@6N(-Sk5qhc}B(U@kW#B zx)zYEo;?*Ra9??!jIXii$*$Sd)!tuPdc7EBUgJ<)B}9WPsI7!XU1HiCTC=M;#VN(w zUAyP_3oTe)^#uy)S-cG%p^bXZ+#emm?o8LKsY>3HXvSdvy+SF|w5ze1ou9?AT`-?= z#BIkyfGZ#VP{z}kWpge29{^%Noxd{mcM6uK(x3dA3q^xk=<<{6Ft9hsxod?tn0|a> zMd^$F&e}--K;NPthbX$M6Q~)D&$BUs7`Qkxce!uYZp2DLAorM1%z zCegZ17wa$;QGf|ft|Un|F&e2=(fM?)K1wgnoERGDHw;4C7tQ5WU4tX7(6GD!3j0y0wC8*G%{uIe z8%)moSanZox>w9o>4F5> zvdE6(VkHa?BtCntf|2X1(hH}M#x#bSfcnD3uH)@2T?ZK{mXOB3fr-65#?R|!>$smh zkF~d(i*Zp%-1rjB(Yg5wg!FC!p+8G5ek_YbGyYbK z&E@SBI!F_5^woy*1iCrChC3PSE2gP_s&hl!z^d+#$MU_pv8MSST5OHdPO>$S^Doxb z^;|4w%$;xWlXRRm@3z|o?5V~-(r$E@ z=YEl4(z2Qy=Zum1;Ln&abcTs@4+x#F9;6E~W=?-aoaeB<;54IM6xTz~J-=B%hl^r^ zoEqB~?wGqlo>gx_uHSX&X*}CA?rYqbGmI zq-XseSI!w!*D6wbwj6erO(!3e^L?JB+e5uLqf|Nr^aualSd~Ccuc7f*bg>4j5}y^+ zEyZ3U(xAqPhk#UHu$5=SZ66e9C3!NfZ6zaeUo<)~Es)&SAd9UN*Gb#|@6Z zj;rXR*Q&a2SHv97I0kKo?gtAXp?-xfpas3r01D8@)4hMwquUc}5Jg}BfVST;2$uCM z>_yb4GkpB@1jZB>YcuL&{ET5znKv-Kkyi`?d97%L4vfc@_401>@t7BfF-MA9H1G%G zTh{n&rQOMH-csYv00hExAy0o-oEsRkHHep=AFJqMeY0X?aRuCkt%nZo>|fP^F{+gv zCy5z4V~ZS)2k$h=5HrTrhU0uK5J#W0R)|fZH4#GkaW|yP&(tw6 zxlc%Cx`DGPLFYL`%*9J>*HY)=!99j%&uq|1XENv@J@yEW!zn-n0;D1OxEEWCJoXzR5o&Py9)HQ zld3KUf|~=KbkJa}6>VYh2P|hDa9x|$>{d{^(Rr%6P7!zF``&2s-UeQ^o{u-$Ax0cJ zcUQGKoCn`g`Fj)SZ7&~7!(7Z!bnDHg*j44N5ciqZrRYDmKXvpLLDX5qB-?vr@h(?P zi(g&8Y?tdknm=dlvIDMf;g6NOT;II+H5>#_dlmn>V%q#!>p|W={1X@*{p;0|yv?R_ zt@&VkyIl8lOl`hSKC8}o<)eEA49R_qZ7$lPJZ41_TtkTSnQ3m65DnX2)ECkyN+2>$D z2lK3;Uy&Lm+{Zp4=spw&&}k%SUTm^a1w34YgbV*pKWO0GNX!mjqkx3i@)v7J4@Yxw z*;++uVATBtlQNXY1rs82`BF5yocvAJU5 zCkf6piV|cD$AQhK=0(BmHS!Waw{fQwZ=v3CN z2`r2zuJM9XRC@C`p|kO=U^Q8{s@j}XO@EG2wNzSFdjiwYpzxQZ;#`Xni|}thoW_(! zh`-1YiT?S`!zmH(^i-4(1-q!{_v`Yom*E3&)l%r+>=lilOZTi7 z3ouFF)%8Y9YY;QNGuW^~X19?ZX!_|=qUPsnG$^g$C>v{>)x)J=Vt1+_vo5a;r)eK> z+_{=k&!9r04384e;Ju5bk_2KZ=ka!w@1*Kh9kdaXoZ(HyzB+3px2t}>#szsh*ih#Y zuEz&SnN`e_F=fR8re;6I$(FRTlQD`X10!#zw(|lY-A^7-DwWeUmT;Jbdb)?+E;aFJ?+T9Z~(F?Z6VtS+tr!AAG8d z;cFBS#M7a_FEr@2T0mO-7Y-@77(!LVC*y&^CY1n&drFnRMFiZkxG<|a|BOK^*ur}x zIP!ogpVKZmC6E* zUfjA<2j0YQd6x=Vx7@TMzHed(R=a+7r>=`l8bEFz=qqFNntt{I@XIF-v~YH_>gBVU zWAS=&j0@{ffY$!Os#Z1$b~-^u`&FZz2T=B2_zc5rS;Kh@32hn7I#c-hi@4EJhP93H zIYWmPsAGl0_U!=H_B)5fYeyJ?%Pdkm<}E<1OPFrIpowk`g8&0}K{t(QjtTf;kHL22 zB`~XsuQ`wg4Ax?1upM~G3!mcrDI&YpgdRUV-!QB_bQ&3&RC+ev6;pG*0aWp!L!#qc zQ{i2NVC&&)AZvGa>3qCNPArqNW;jz7=XuOr2#S|(024@wHHXkxUib*ZQ_OLepWoi4 zTgQ^Sfei;8yw3F)K+tAC|E`G@0&X)0OQWLk_4Ho#&W%9Zy7w+dV*{>R)M5pp#t|(&lf{+F3}dE+|N`m@sdO zI9V`r2Z&yd>9)7)BEe`SOIhhmvc59LFI=#|Jv4Q7iLT%25xavGUS_q!4$eRa6W$q) zigPsnf}HmF4Wpe!ge``9#d=VS!)$R~1*3Jg*Uq3zv_^5K3LF6nznt;;{kX<@3$$)u zi04LLYv$7!Um<9*oF@F_Jc!jLMzxhKWnS^7De=Q6jt>z9qZ`!Kr40H)8O}Rv0Kkl@ zVx5Z#gx2!2m%NJm3N&I{e#zgQb_$E3K>^h&r*cx(eetQh9fl8^I{&+k4>yM+C}ltT zt`1H69xzIH3M^CPca!RW8f!JX3`(TwOk>9X80Cdx|K6&*ZBvXhKy$_!BVm{=!LoV8 ztqn&om|h0;3^_i}!_Rk6C9^1))g)Vuhl2!88Y9o>d{GxYWd{{%?kb0O8d&$n3DuD> z>b%#Z@y9E47U&IvjgvGA!BgCiF2z!@r5kTbM5c5HmFcq^?JzQ`kjfj0h%4@k8XxD= zYd9_34+dDX&m?WZ+$D@3V3GZ}s*W@9;eDFeL$6>MGB!0R#*H(+jA@R#pc=p8V}V(% zyh~$OlJ~&OE4U)fAT&>q3yMi$x_SaLUmn0EEkoyP9SpMVdQvf&XJ~#W6!F17Y{+ zB-j~7>1B`P+t{TGSL{rVmYbxq8IH&cbRDNv4cji3_8WX&KP}N#aRT?!s|Z}x(|0UC zG5|`V#jN74ukxJ?IvoDq5dNUiD9$m&bI^0aox1n{0N;WI!~1IDM|Q1M{Mr@o;v;@K z-VX-{4X-%Ai=-M0#Jxr3PxKQ$_alPegC?a9UqP(=yqHJ#r%@ycOGfz{JhE1?M1%U z?KM)s$?R&o6Q78MxN)fFe$OPYL_PVw>37}(i|>-=vK3!l5x?qV^`PJII}h8s_@Aw6 z?$;cyO~T?MZ|G%~4)a!XW~zSt2dm384PBc1d!Dr!-A^ zewNkb3I3N}Xn)Ir5!k<%o1ei8{MFIzze~>F!iVr9Dx^0~;^NVX27h%dy;$Wh+lZT0 zEZb%e7!g(|`%I67c8_y$VvCzySo#x9sBl`XV+`h6LhPa|X0yi+Kiz9^yGbW*gtT2c zC`P*Q;dtGo*raJ;hda57`;xPj-={=Zjz_U&A84N0WZGe{9Lk2)EL|UIz~g({vFpqi*0#AX1=Q=WM1` z@$2h}I?V?U+vl7^;@2gSn)f;pFdST6{vqkusyQ-f7tdKQ56&iX)=KcNeL(7nm7UX) zbc~(=9(MLe@oez0!~TyvEDitfQMWYxcODi@-5hkAp!|as**~dR2#p4kB{oOgGGME5 z9imtG%Sg@+2e=tb62QpP9D=;grZ9q}NGWkCP$8{NtO^hU{~Zi-X?!PQyizbT0W6hx}Orq zAY?0L1*9)0XxlDXVN7v1NQIaGlk>GwOgwNLE~jUJi0gn`n+Z%%tW~^3-o6pwhxl$C zhNn{L(U?P=s18m8ba}9QVk}%1C@seHQOpVneT$BX#dBv%WUJ#Z8I?Kx>Fq)JmPl6-n zOy{?7seg(92SO~bdf{p9gh-zkJY>*!7p1kkMZ&nwET<$7C8o<_Gk`sW=@GS;Ii_< z>=Ys`#6k;c(g-olQM~MG5{?E4qfbzd_oy_Bb4al8Z!dm^SZY0*#@cy8$UEp&$po~D zU)o;kN(!6gM3we8OKO!bD_8Pf06+r|x1gHxwsO_h@z!=i}*2A-aC?VvU4{8T4zN0D* zw;5EEAHS^Wjr3xJbkuMTFbF$@(Ob&ev1H82St>CI=?EwQdaSnN=VJj`u?DK{d@jBM zgNR?}1(EL|Q#u@gAJ~wMv!J^^MnLxxM~VgR?Y;~eew#*q=rgr_i+~;Rbd`fs7hohs zuO@Z|InAbl=Y7@E+3!{IHJr-XaDtKdf@z#Qf6sulMIf~Rb5O-|6e7tO{OnRZ?3pTt zI5=_z0NmtlqaESRbpAq#12S6pHehUivx+ls;RBL0Gzr+E zB^B|Kz%^@KoFSGBsz+azZqj&Ob!=`vRV~k0t0fb5x9E{(DSQ(1hUKDLpCz~h)O`wM z1S3{Yv4#Q=L>(}iTs1h^E+9d*M&fV?vu?de&j1v?P4z-XxW|0ZY)9c0F!!Z?bhy&O zz5E#WGw#uGZVh-uT1Jy3?&UXY1Ru5sK+K?7KzyYCiM4V_{47RG0SY2I2B_y&PUnCf zTiNz1m+Ra-rmr1EP)UAxw_a(*-#5j0ivjfFNlyZI@2=CmrUdyC#-3IoFMY|@&PWu| zBH|BvYC+L9=6=f&i|ZJ?lN-1jmZW`=NA3~#Deh%Y(Ped7ICQ3wUucjA#)Yl|p)eom zE3fc^lPW|t-2F^(QiGX$>VD_JZk?aCIcVqp%q9<-VkUJ=_|I&8Fv{NVmeU%&pz)Jx z&W(U4_u#T8syhF(SppYw7(xL6R9b5lEU%NRliXj}UbK!m>lO1Xff*g68EigN*Av22 zD*JT<=Mz5>W1tw*LL#H-jP*z7YHU_TF0k8GK<=v?7_l+xs2OTS`{?Xc)g|l!juy?r z@I1pL6v--Hx|9@_35*$@iMvLZn`JPHIhR{v8;UsHP#K9r3YK|E8OKWT#WexeuGrhF zuv$8QQI)qlk@MeGoD^_qcYG^|i2bocw{lm&T=D7v>x=H}!V)9DxNS6Ob}Fa0#CA}C;ro^!0;+n+-5PoKPJum( z>-@E`&JtKV=uI$oosaJx^))urdvyM{KKj;=2mR<^-)IC2wKB#HG5Hxeo6Lo+c9cQl zq+!9!ReoVJKyAD?xp%+={cH_%>h%@=u`7LdzIvVZ6p`aRZJ_Nc4d zXKB)0ZZ(E%7slx5zL6;J?4W8s#J1|5Z-~JjR*(Bf-9h~5F;ddLG05jIBJ*opn0g8d zJkVX65{0lmdl}?^*hz(zrwjHFq)3MgVyuq>=10CE%!VPp)Khhj883eBGxJP*RemV| z=!;ESx4+jEtWJRP6So9uwEDevp0=H#W-(XsLDd0RCtK}2Bfsf06&l%HpbyuK_k!aX zxgf+Jai^tWi$1V4oK|Xf^Eyi>g)i35^11B zwuOXkus`B}Zf1^vLtW#Lc*_Mb*5E%FCpo}&YCBGzfQdPFD-y^vH0J)s=0`a+my<0& zzSbeaMl05Y9RJ)Gr!<%~WVPdrjCQ3mD#d;RiL<`aBYGwp_S;fhIW0p3V_3b|-vlZ; zM`-+4RvErk-1V;5q`@Q-wxj4+UPen{82>nqSr2vl$@!Sfsl5bIppYt)df15OoWjjx zw)Ue3CmZDAbk7k!;ErM%Q_EvlJGDyF0H{=~rFB2)(R8f6YGs`(F7=q{wFG{|=Qt9R zE$C2vun)+Rt?C?93%Yno_^iSppl{V!3W$47C+x4;{#>(ve5Yv zJ)>%^72{j?(3zPsKRp->{&$;0?Cu)|Krt5M(XOb6h8K@o=KNrrdpgLielP{jqAXaX zO2P(m1&THo6z%%rPp_^AxuUGcmHV7i{CSR~Xz}dzi|Hps!*2g*-fNP7#j|km`e`y^ zcEur*=;2?lt)^(&{8uFHi(aFHtp#(y;v|^^Gc5k}+IulO{QpSU!t}pH&eHs!jIE;# z#zE&_d6NHN$-gLDl!febbBLCO66{d)49qPApMXg#X%Hn&Qj|*@%<|F%QP6^zPAUO7 zs07$qLBD83qeO^85raW|wS@0^0jME!w18nrOos#9e`<#R33GX@~J^*8Ep14wl84hVbd9;(d8A#;a(zfe?7fTZKnZ9 z*kA^;NU}Q|gOhp5I&vrv6&>>H5Y31RsVgfG%-dCEC`lJ_VL=cO(T?&j2<954z(?UU8>5d@&!5Joy+C4f0YOjuXhs3mRa@ zpoKR}w^udmObvL0IfvL@UIq~oyes)}AGXvRM=1t$3f|s&K?A6!h|$pwN#H4*^@IKr z*9V&cHxu1t@^0PiAZVoz+r&0Zt=f)5I#;}BO7-5OidfgiXYoD%ob4;u8ewLLAbD;Vmk~CMYs@y4DWkTU3p2FTwlC z6oD(SB2O-fHdt;*(#2akeJ}7?D=i1bc#@Aqh0Lbv?HZ*`fI@z{hYoT7BUn?hy{hxb zmHa%|LAnn5!l!t7!b^7Go00dD8pq>qgsCz74uGXylrs{-0LZi^b6O1}VZ#KRDVgdL z#+_SK@_v|4i;+0$kYgh`rA*(=T85WwuTg5=5-hqXpzMhygiu&NWlwy;W#*fPAAui{ zb*i+o9*ESTHcTx35&`+!{YDwthWZd^YrqySt3PDw}ad|xkNROdxG@iL)KDXSd4$wb0cj=V_LGN9FsL2Xs(ZTd!oRzt=rrl`3~~`cmQSH@%*6(29W$x;(Y*lf zV|&C6Oz<~N=TXHX@X|Sn96S%RX>K9+25{9MPEdo#?O-CQEUlV z^CCWQr7E-Kk_t_8UkDZ%$Df1Y-Hwy7Cii$2KqtM@Eqz-Hp>y`@)oolwsGXFU%%-<8 zfFBI7mu^xCuFS%sk1$=SNz&H93KqNS2HT#GhMk)o{AllPCZSi7f-s6h?ONsjZpucW zc>QP0c9M+};5DUF>n<$Ln*`l10J&@uG_s`#hNp7=mp+0YFKXnY&~dazkEbcN09B%! zYX-Yo`ZStxh4ZY66>sj?-;SeMuFiLAk-W|$ScxUD*#QPLk2&iSnte#GaA^`&yWH)k z338wW|KWg+m9CFwjsaTKO%x6flLLNKAAMyD$W)x%V%GP%{rIH;-CpDoDRy8(4z;=Y zN6XK*c69su-7?;Ih5*e3ZLbGE2Vr%9Y3)|c*&AQ|paqOp4^HLsFuPIu1W1kx0@{Bf zin6~c&+&VNRH_e|tXFD?EUWnWtyr#ql2nVp-4}cKMW*COFxtg^IxM|el_QK*l0r`E z`8@6yf2T})n#y$7gY+UU1EqyJqCZfOMQkzZ?NGwf-aam$gE|Rc8b4i{KZR^-Xz)>^ie8Mc&qAXM?&HGo-{v{btZ3e$v$1;2%A}354N*NW)D4RkWzuYcCiFjY{|`O6+fF_H7u7)ug?ct z`ROmAuOlqvbScyMIW^*+tz=J~|J5e_OCf?ZcyCza=gP@-zha5818gYh^lISZ%Q3F; zc|t6QH1FIL3qU`<+dmQ~6szD9O&O%W%)=kdx~bx)>oAi)9t&og?y3saWsPvRns^?| z=yeL)Tm8YEO=0#iV>ug1!sJUHK+Ul|Gz`yU9aS6RAd{KLZtW*u95DDFWY%njQ{Iy} zUqF|B&B9f9VSLN5Rc$5Y1qW)_Rc!!aiV09)r;Hg5do{_gw@72=9}GWvY`ozIKQ^l} z=S@O}dO*?oMOYfrMzU6p%kEIQ@j5yGaVN2LKu6Yfr^4&Wvt0aStYKlTlVcq) z7j_>~A-`ae(AhJUQr#=V2@tCIOvG$R?>BpNe!WdH>Fl8L3EhC?z1erWz|WE>U!+NX zwszUf>puK*84-&qu0{!pWkifZVgHP>m8py+&$ZvD!c#NrYLE9oFnxO19+rg+@{wT!JO{tG|*k21Eb)BaaP?cF5&Cp$}sdVy~Ql6Da{n+5I^ zQet5l{Fy{iuzlkWJ8TVQK$cln=5UIo*_D;agGE-By3JBlR;E5?q}TworxIo!BJU|d zLTxZfOMf>lZW*DOMNuv=6hD$wUnnQJWggdfnU?IgcR%s5-LI_#~GZD};L;8Qn4fWgxB`3y5#0!AKZ*Fh#v! z@c@IY7JWehZZnj}5?}&uoWeGv*J!rnxbAAj%vn3_SX}ZMLd8Udiky>2`(#od;0<1S zk2dP)ZG~sypLdbtuAbZq-WaY`wgkDQtP?CAYc$)`he@dn6CLtXo2+!xXr~lh19X*M z!3%&+=g6$bI6sx|nq5ch@x%B zDG|O^TxNOUY9_7t1j|o7VjwpACp*FSj*egQ2dkY@{0Q2cga!&0-1I&v3(<$4B#Yi_)i1fc=wOJ~-CZ_-%aDqc$8+ z=_f^+FwD++g5kPVotkFHx)#uN@Fb@!x+>5!|09*C)PsBVb`YBond@Du zS}=(%QF!!T*!SRas~sd488BdHs^y|%&3G|hjXd4)4@{o|Lfk7IQsH5?FsiIU>5d;e z5cZqZN(&M%<_mn#gZOcmMyX8ds47n}TRDf(ZWJpq0U$j55l8yie`c2{?Dc#Pa0Fb{ zm3PbML;Tz>jQ}_1{%NX#JcVmA;SMXuxj^PTl21JJXmX$8RjRsr#05;;1X4)QsF9bc z6T^8%v;Kkxyt{)6!bo=VES2nr3`q4rCwyCR9Ec~FRkCllZavNE`7N?Y85m(bI};Vp^$bSe+P>y@b_N7F+>+9m2GgY+*TYvdzcURmVU`@QR=j|VH$6xc9lxRn zz^F`D-pG~8mOX;fbAz2!zjL6edx0Ec*EJJUi}rUa4A*-|c(RMyH=a-BY`+A3uFFehn!4>oxkt2HUwK2Ji=Q>%W5juxQ& zQ0U?0k+{fbyP6$3(lMG-~fAh<`MYm=Si`L358@>zdoAy&5m%K z#f&PW3o42LyhW{0=}1$iU&dSDl5aV>IJCFs z9A@)B^-68qZ}|C}o^F5QNGHc#y7-K#^d;XnN0L0oEZCQh6Q0I~qwr*?|9i*-V0{sH z`YHHvustUi&Ttg{GbyCBuT&%wb!#;$4q4p5fBW*Y@Z~@4HTYXyl7v0Q`Gr*PW}YcV z(tv0fB{yOLxxXGS&eFKGR#pq}8~S9{2}m4Y%g9;MO}{4q&W+8His3K-sJ!Og#H>GmZa=o$}C8_Q1&j0L~$|%P(w*O-0A2k*bk-=;T|*ng<#IyHWad zKFKT@E0}SMrptOk_;~?xJ;{<1__iVREx5({29cA;!{*E_eK@opSX|=mF%DKZ}?hWD_Jq0;=bC{;2NdRo8E@r;n zz=P~o~i6d#4ovvx(b8fAb zzHab+sKl$DlB*xEReW5px~oZ;SYNiK1DtrdquX~+CY5h*r}8bSRKkK?rZi$PXjR?X z5a2E|FCT*&cgP`ewsX>5>fXb z74fFQ%s*M}NG{@sdSn)K7?BpMLg%f)N{ys1=euaOZRFY!7@Zjgm!r$+y6w zWbM~3cT9_C;a``31j4iuL~H)B>u&?mS}#$(r}hUMnMDAX=bZ(W`5hrJrlfoAbZ(t^bn180G)Yx5B9X@8PgK z{wLu|!>G(sjABW-WG3Sr&>;-yVA7G$X($9&@o8)%W0GNUat3&LSswu!x5{ekff-soQ;hca4?mm=K zID~?8eqiv?%8le?6|2NL5f@Qt&boog>UQ$&VHr_s{7A}E1Ys>vG0#pj5EQ z6oMnGM-?qKU@b3wN_FPKCN0C;Os99)mO)f0NqQNhR7GnqXn$h5WPHeoMlR=bWh`X2 z0^3Vf7fHXaE-T059^DHix2*$CL(Q$myN5S2*ay4=K@Rh7Xg^A~^NOa-P=|JDQG7)v z4NZ!|+f+XVy!O(wWkR6aGQ6xz7A%0}(q63P@Ei$?Dbo}zoXN?BT}?12FzW09S%O3k z&KqSl)-0uwlF7Bf0=82t{rQuW*I%UsFU*z2%5=6j@th^31uYnJ!@STS*gu?RH!`m{ zpWuOg3y?DfSZ$+?IvP`T9Z(2{rS zjN)loNI+_ovJxv;Of~P+6UC2kr$)9?m9LOsw0xjC1&ULOwpWzF6Pj77l?aHm*)UgQhq`{eY2DuI+0lM4l1oWuC3)vwy(E*;13!&6E^O*7<_mLh8 z8r>d2<$(3DM{DQwew0WxOY+j;m^$Xf)Z zP+1eF{HVU<>()rt2>>GPU)wSQ5aAzQzOBqQ8*AVsg%12wbF2N?dj@+&)_8tISIKPK z@Vk1|oly}Q$ZOK4wPQ*q@^Oz^6WWP?(2J)?a&j6~r0hz)Yd)p(g=IP`k(Wno-CkZ7 zuW77F_VQy3_jw)E`D07>i)G+!cv(MLz!m#rtDOhS8J_tiquNTRGcP@kydRjC6Z>>~ z8{(c&gBecboXJ!P=*aj7I#g3#RWFd*(EUsojJy=>H-$VY&ZvU#-APHnATCb=sCDp8 zc%iD>65`r1%zU=d&Z6raoo!`ONe>_-|MeKGeDi16qOQxS5`){ zC}{|dc{1E~zs}s7D$JlXLjBh^PNp*;<&7-l_`ESDk5@N zJ)oqiulMNuXjwEuT1kAWi&-~gU2U&IOYCn-M5wx;D2~nhBGnzqVki$yd|=gvv(Bkp zTx-4Gk>*<)_&n~KY67Ej@Mmnf$*^xsSwCwuAh7f6ldoeEnO@c z{yufu0Euj8$%Z}%`&dJ=zk|k#buKeOxjQduy8T62_1VgC_G!+{zSc+ox<;s-11%!b|m5LRQJ^fRu*8>f^5b04|E_5$13t*P_Nx=I$QxN09c{(#Y`%KcGL zp4VEk=fz@1A;ww2rq@GK!~Cnz3|Bj~W|Aj$?k% z17Z7_t@8;c(JdY7BzS`JGo@Ytc9&873k+E}kyF+PP{du63~pA+j8z597P5V~&+74` zt{)uk>--cpn%V5$tyBKKVxCY1=sTgRu`U-Tp9Gu6v z*pbX>%&3iy&Ih`wBba*~^+tp00U7J^bT9uEd7wC4;Q)XC!2~LASOY`F=M8yH7nxQJ@=f)qAr6D`yl!=^g z3S_gRjT-yC>HJQyeyfOiAau9s)!{Ii3%%3^SM3J%e96!#+5QoQ$hNzl^M@-`%i0*% z&eE^cb)oTUD}xY2e$1y}zAeY<5>LUoaUd1xt-IBX06>^Y*rSj zt(-}%L)Kp_(;NTSY{fMAFVV0f{`YLfB>#_W#W4LRJ#>6Bi zWEfb$aTH^+xCQ<}zEY(@ls7D1P)EBYiPsI!#B7Ar^n^yLI)%zUU>cN3ROaP}Es(S9McKpS9$j(w&}l-I zzI+dTBoC?S1}hW7D8-!Imawo?li#MIL5*DW5`w1Ba4+~aeQsZTk#R&G#_Z6SG*!&w zm1buRFymU8jX)8>?N!iHnG>z_7a&jHl9UdxRmRo4>`Ya53Iw0`(?40THan{AFq<>l zpM)M3lLo^V^c7EY5So`LCOJm|7(7+77xN_zN6Fx%c$Fz6kIKCUDc>CABro;^&p4S8+dh^e18?bwe%RIPz3yVTc zRI*!KtMORbn4S`87{a62UB`(u%KV{6kkplRLM7GZuey3`!oQi^9`2 zN~@}L2oa-u?EmErAh;U;n@!sH zCY4EViJ#XE2o*l(8;)*o(q!23DA&t25q)YTSQEv0hGv~cxvI>5RPmBysh6m^Gen?6 z{Yb3Gx$JpGxbqPjcWB|fa|u9rx`9_(`H7tWkTkrJFkBJi3j0YR!O!PiE_2!XLWPLr zcTJst+f`4J{XtsUbDCfPQ|Uqwq0R^lnCI0T96TGgB)%V(^;fRWpRSd|(`N{!${JKU zFjIxh{?(Q-i!(fNuE84AbL9n?^62@|#PXWPuc}g$5UFjpBiOpDE>SKmGr$Tt1%{O|mwtjEud`XoW3^>0x#LrwNd!XRQe4I=z zb>FtUK*&xCH+Ny7CghI%R;5jW+f< zFD0_+Ph_t{Ov5D>W=J}0(lWD@D%uXR`F5TQFz|rsN0(X}WfM`+a2%e@$$ex}%e=)@ zEH-FWS}FBVgA6@b`Z0&|BYKjT$!lQMUibuYOY>34Vs<)I@>iYea7y94 z-E?L?+=ruIb-+Y-(<|;mP3+rCJ)C4>&hU(_y(^tMe;m{#wn$kI6e<<_9$KBYv6W z7+-U+KxNag9o1^Z9bnVs0?3~k@bjY@27pox%Z@K1Uf!49i#3)9H0kjYg`#t3YYL@A zvQuffQ)p23QKObF+rz9-Ybc;mbG+r}ms!9})G@*wE((kDVd=r1Or0hSFS@7<>FwRi zpHyYKAm$WSh_7pIE;Bw;WAaozxD4bO=QZ)|N(G-ps^(awjwS2oi)&cqDXX}jGQHpg zd8Zi2EI+BtM4_&v5tehkhg*=?3DoYOhILcOAiV7ET-;dG#ld#fc|wo~Sh#ESFlfNx z4N=tupVdzVd>N60`+9Y@i>5A?KL5%WT4Enlu?~0JQTjr}?OJ0^K3XkHlShf%KjpR{5u>7*|vR!EMo+XZQVYIex zj^wc{ZQk9J^z7;ipVV;HE7nhJKR%+NTcB1HZeGgxNi;R2+Wq)L4A(){D0!*;rA~Qp zl<C1}w^!DW)qo+W$E344kHsA`C=SFYoD21cGY0TR6&ifB^4)xT+(@3t_0w*QCIYvx z4ykzq#&|IvMP;(qXeoJ6+@^}pH!x6(3_sb~*Tt>W>U6RrRmYjIp%ft9bsQ}dN@B=> zBJge|z*qYDhHj6i2oq}t8RIKG%3Gi8`047ZE}A_xamj=(`}H0_BDXPB2maCGSYi|) z`kC#gA7Pl+5y(;8o4fqtBC6E;Ddm8CqR(Rs^P%iB0p94d@n!pvueNAiL)EX1!H)hR*e5`!UI2H&=YH#gmP4DNNyoCGd^>7}2yHkS$oF6Z;d z(%qk^%9H#OU5fKMg@0|L%;niv2TB;@Y5Z9^iO+DQ34+P*=^=eH$!Zt*(_G`L>#`R) zM2<=9Yd>CX7u4{OO;bv5J)1(QR@Aa-u41m3JW_Pz@;RVK`WHj|mQp96`O>=AE;{q@ zE|=;hew)WjuT@{f8h-Kcc!Lu7@v%+dOFnfX`N%3Eswg?UH8+6!V7~;((P_4WqRf;}K0dUV{@l@Gc+@_2%MRuDR zqAnF91v_-L!r5Y(qDVk2NZHgVoTV5y0KY|@M`65TicqDUN6NC7N2&7cQAJkOYt->~ zby+?@9v+#_Cn=kkT1@&t@d3Ij>wh-)dW^(qB|g0+fXGB@!2^*AW~v$n=Sk_~nF+ue z$Ow?aWGbTU2mpRml8h#G;>b>rF4R*vE{<-ZOu%BaAWbQIoTOJ#H+y|GMj-(nfWkUW zt2QarMkP$t8+d_+Q)L}ojk?U;$jaUqR!wUsR0fAJsZk_Db}yNujKN^Gspd={iqey1 zdt%&&KT%o%lez$4bvKeHWuLTux%Yv*VPA2&Hc|wzl^VnMpUBNt)NA;;$(2b-k+F7A zP`!Bh0np>k0+w{4Cd((oJ9Wd74S1O|L-mrK3?~IrejMK&2B&FSBR9&}e?FoT(y!ww z?XWD0TPh77o&s&N2vC5LH@}fhW8yAd_mhPhctfy+8dU+R-EZQK@!Ed^QYr#SDPM32 z#~heX;jYxt1K6QP3)c(R%g}2A0AWC$zs6Tqwa52rvf1jKvf;yysxPap^0E`9D?3O~ zw?zt*P;Ti#wycSB&H&DBQ*J9IG8UA-nk>?(%Pw^r%o+v>4rc2zYov%evK!kDstl6B zO8Buw8!v#;{KPiE)yg#U8<`g{O5v;*ROgfsk1LywIoqkcOy2~$m*^lXHx-~p^yOsL z@pB*wNQ@0kK3(S;_3>h!p$nNd&-~Suge%%*rm0j;sH~}5l8Ez;x$q_ zs7_}Gyy{zqtPqhrIFrG#hFq*m*u+-8rR9fTcNmRxOtZ(kWtlA$GIdIz`n7(bfO(Ww zcN>ghx(*lq1*|xbl}tb~;v%CqT!hOR-_vlGvwHX}_v88a!th+R%>FFq$UOAfRM_q( zIA#CP6!XCWAFo0~Q(WPMD$Kgjfjz5kQjwLL`1KKXI{{nP_zw+rw7pqzyDC0sg^3GRt;@6)*g`Be6XBai~>A(M`)0j3}4WBI6O{3JLF})@@i^7h$Nt>@3+S zP3Kpb(xI658)5_lxlyzWumPRl@6qEy`P_?p#1$5!u71%NPu7<~uBg@44G>mPjtIBY z#1qw8KZJp09N$t8EPUZ6O`?cabUIY!Nq|{5*Zkn;Hdey}#%NBIYjd%s1zG&qA;*-# zMw|Zx5E;4U0!D<&?h@+6#~WRvMU5?RQy~Oq(m1aOiwzp$6w*iXU&jby~8MwYDNcq!T0v%Ci*jq-;y+b9i zLrJi}B3@@`2DO!JHMfO9ZBD&kun!C3yzU46M8+g0zYdU+1Q5@`Sjk#A? z{C28DwLabLr#;lr&nz1WoysXrW}P`8dp}TnK1zg3v!&8SRu63s_IN6#0LU&pb$-7= z@$0tZXRBSE-K&-3Th?-tTCJz=Q2$2)W|(02_{L}}$;A~vYXjEjuJjC2X0{kGxbc%- z8TtnrtC1hk3!WkqM-Ak%Axr0D`ZpJoi65G*kwXZ!8l&?IyCe-esVXj{1kJBlem0|t z#e52{v?wfnFNo&_-oAOnj(B4nZzL^4+yX=|gr9xZq{dmds(yYEi2R9l(hGB88TT{U z4jKmA`Ybs9l8Q0vpu1IN&M$5HLk>os-U`uuWk57BDg>_T(Hq2BNyq$s{Al4yD+ZX{ z-J`_t>_BUzqlHs_iY=uFsJ01D|5fij|KrBhX-|gCsj7krRMnjA6t;JmfgFksZey0Cjbp0 zw&EPYWL#{FMjWa-oF`NRKiMYOdq>SLs7VXpGU4G?cDu^Q%3Wiip1Q14CXNK1jy+Mg zkF@Dgt)dt;@Luw5N0Uj3x>)B{-ETX5968*){D@u+5BmVq_+5?O+bQd~@roUJg;Uj8 zk6QNLp+Nk1DO149o~~}b-6w-plIb@GI2Fkf*8P2rn7~e$ln*WOW0Ub;+ZqmBM()-z zWiHHh4u4`xXIsXdhBjC6vrp-oeV#_;O_k@X>`NwqHLq(pC5Ff8>`;Sx)$F1w&H^re ziW}}y)2O0?f~a*F+jW254TV9C!&U%Wbmn5h8m-!J(0PILPY>g6UNp;AAEU&3oU>ox z;xRI(zcl^uA`K@GH|bgZly$z$sWdZi24ey>O1aANlkMx>qRN%Zn;KguE73+Kv=doC z0eXZtB0n{C!Q!XXN$(uqExtF&;CF29~G2 zVL<1rsxo_rJk))fVr7GVei*di4_a~&{oM4@oiz|U_JGba=&bEkYDaygRc%u0#~w{e z+jo5+H;q3i$o!B#4gYS5r(7%wYI?V-rrYOeGM^^CRrP|)P3rEPIn-TF6SvaR{ylVJ zg`3cc1vR;8cdKRHh1<0S$DBY8>8r8qWvk1fuFX9vcc=(;YvyHz!xuoHe{SmHi%^aC z>(oZbKGbZ7*?L@X>L8V|wX)~ApVz=)=DX?)DB4febiQPYit)dR0QvJ=gyZlzml=L2 zAYaR!>XbOYTNSTiqO#XzQ=WU;2H)BQ_JWea$MMe?&gmReFaCf*u8`-$bnj7lcbk%u zKi!1%`C*M~VoO6f_|+3rxP?Dvk2S>A1J&Uof*>m6$m5~de)Lo?bQb{#)5Tmp>2u|& zUX7A1ruArIpivvnVLimf4W>@(`Kl$m!*`-|kyVDnZrkOKX|mhm^+c#{JK%cR(*A(! z4_PorvMh>c?Q_K;t8i|gJ1AN4wgc{%w&0Bo#IhF5*+B6#YS6LE9f#aEXTxncxazg_ z@|SNAfjeZ?>t&MT92~svqxt`r;;{JtPRsJ}f8FEkA3RQomTmsFj2%wlusD?bSMsvC z%5WUb1uyGJ5JXi$N%H{*bQYN7=Xj`xmXQdBQ_J@p97mx=? zo_cA8P~2O#qG?QaY>-}vb{ODJWQOFr1%z1xj!Qkdsp|-cLP<;ri&G^DHRy-%C{5hN zkEjGKSjwtS*``>$U?*Yb^rTz0^U_e9Yh7D-AcAP*LHYf)k)7DI;;}MvUW{8o@h*tU2;alVt$0S4w$|`>B89o0`V=*Gd_`$?<}+C1v;!B{IqV9+oL(aW8X!NV}(z z8KTm}yDE5(JNan#T`T&lBvhEnyDEP2d8p}K)LJvTKocj}%)Z`|$Nb>S1_YlncQ2@N z>jKSBCJBNf754){R=tq|6D3Dbf3jy81w@C#490JN;kevcq+uE8epca?{`^5*ROtI4 zt?7|ex-?DZTWzHNCF_;iy_H7z0|3JFAi$rJAax^f`nN4FyowSqot2IsKA>xU`gybJ zV)CV`v4=#PRF7^|Idn7^KhTBkxqkW^)39bpnApntHFmd#5pQHITt2R<^AkK+7t4VG z*j;6oZp))y{=7_Dbpwl5mtx(elH}{7ehElu%X*ke-O!w(I4!ulauM@~qSmD3sh z2B7x$uFh}mR|fUW{CgG1#J3DEzMGkNqnFt$nOI^0%lQ?xi{qPR7OPwo=1=q^Xp_Yw6r332 z)+!zrzQ>vQs~XVCS-e}quKX;{zoa{lDb}`@JgnQlm)(=xO}6+;l{K@+@L?E;k*11A zeo<50s||6p2YUjcJd+#BU2=mpfHZ=T7`<)5FE(K!I%fhf{Hj)MC2zT61&Hj4&5ZlDd#d7XmrV{jD=k2EQU$-zAuiwub4_s^*|G1c z;xPztbcsqxlwS2unz$d5v@L6Q0s=QLm$8#SxZ-$|HFNxG1J>88{Z$ysnU@vs>DZ5X0!@UXI$?DP@ag`!^iu+uCw6^<+GDsSkCw?fq}BNGH}tmpqICI z5$1xr^=e?8Jr&?i=Wde zRERqc-?Y&pJ-8B>6jS?QFT~Dm1uB=89a;j2eP2j~<<^CU36iSf&1 zzsvBD3Qoub7fvG#oP-kZM@lki2x6 zS?1&|XX5HftR5dqb-$}%oYu+q7B%ujy zx(YHD<03KLg06s)8Q0@fc_SUGwUZ9b z#ImxVgaAqG1JWL=iJ87(U#j6Q6l=pHx9a8ACQgUFv?XPp40`^q#y^-yLbMQzEveZ$ zKUSvk2~Nh0-_<0;^NWFzAc=wJM9qep!SO%JuMYsZ1ui+ds!IC@tqP%>L6ILj7k&cv zCFa+w4g_`?WjlY|>j&@m8zL35Ky`f3`Kdw3?yFTFrd;Pc)@ehC%H&>lV3qE%t16G% zKdMmsjLNfhO7*dyCoeAav-lX;{~lF5Z!=39*ABvuVHHdPgk!oFY2uA;=)oUDzfLir z?Vx0v*(OfO;IJC0z|Y_4*8ONowd(Y1UicG3B8NA+{qX)Ny8X9aakr{ezHW%qffOej zeuSfZ^44S=VHEi=v1)4ro*M1w==_!;n##(jThxiN5y9@#Vf9TFvuC=)+wke!U9T8h z_Jgf%;zpbp&b^6)vu` zSR;GV_5 zY>_U_SiuCH_({LyXwPvk{tT%u%MU_}I;+VY<$zOnRmH^iU_u*Yr^DAbfN~wO>J8at<39Sy6|GgTKe6n9I}V|9 z#LDJ^I^nvx^72)0Y*>7Nq*dT(^ski%fTO*EyU-v1YqDYRUt4edM{C6VKavfVyz?)5 z6$BE6LrGAc;T)w1Iv_+-0~{F2fOGbZVs_|b3I+<`=`pfB%5Vl0J}#wvgjvc+Qf@g` z1Z8&&xu+E6u{FXq&~Krv5QCm^sqmfL#}G;?$iwj=xgQC%^)Q$ale<<^tBwFf036L= z&~DRZ{w~>Fui%6T$d>@yf;q&As=QCFkzkQ!#kQbSai=Q#3NN5S`x@uR5#|}nfcEyy zJM6H9oHwH9xCFn@GRqlwAvJTNn&3uqmnwq=ASBYEJ~_)NE9g6_#u`+=Znomm<4L__ zHbRKXevE4En$-*S!gz&YeNC$4pR!mU8$Lqm0 zW%izRzSd4<#4df7(TB?pe3WFrrJ+z`@^&hdECZ*+jGxijv$T;AwG>Sspf=CX@Xeb{ z$$Sj^&qg~kUu{XfrVHO}ye_ zE#AcO7R>_scniZ|ZIeLxCYmuE!JNQY_yO&LK(3|bj7`acg@iEiat{I82bK;^QYE0C z;la~E2;L-;Ld8qLTPh75(WrCx`gjMC@Nz9YE}m6Bfw1U?`~H>ReXVEG!K zFLA|i5IxWN^Hd$!1o|cIW-X_8?BbJ6$)jqR!4*vB*P0}|E6Ds3LyaGkgI??p$_Pd$ zex%o0q2OYFomnt839XU+q(5p~Ccjr29FRtNW}1xLiXp=@4cn(=55B9MwB_VqzDM=) zE)@&{X0{zhyQ{EUx{Mi31NZy7Ooqg;jf7FKy9!zOicZg2PFv$)F8(HHuO=5~p(b=@ zSY^f7LI~?68Z*CO(k4Q4$%!#+fPDbzc;PjQGZTwzCKL0Z_Df{~B+o}=tCiwPf!Dwp zaYzlYYbE0^t1bw@N>H~MOc(d5xDWp3CEF95u1DJ#`29bOiLC%oe~T)!3JVX-i4X@r z(`!x_#`L=?ZalXOKOCeU;H$Xk;^uIX5CT7g<0e<)Ch@M0ZoSCKbjpM@=`dOj7PFlB z@mzr3s8)jBtuV0vqk)#cKmo%=F6fFaVM)<&*&isO({gRxuLHI^->y1XQhSsYO&=R0 zVN8)hwiBm(mVf+28a4NMyGPtvfVOzk(1<h?lau0M!0 zAe_;q@r9*l7XtUkUS|EK)(*p;jnTyoR&`253v-pK*pGrIoWb>WnCFdZGnlLJFO4C7 zm_Z1Ff;`G>cTW5o?9F}Me5pJN-W#^?=*mnE4+fVduhnTl=}0{GlU`%ek4 z=%Sht`~DsuM-BGHitKVRy}{ofC;7qOfxin2OQ}j5DSm1|+@mrZaO|6Xy12Br>W;7S z?Yk+A^mtdh$Vn_e)G5L>A2jW?em@0+YZeg4!KyAgqUu(fe)bk-_ilA0>~x51j_Z*C zeznPKErR3KW;^Um0HD39rp_&!ynSw*7j7-pNWPnjv%9Ifc%TA400ep17~sMm={_Cg z7pT&Yo`-Kin0M0%VYhue2>MLp56YdLPbdD^5EYx7r$Z%m$XCD@G)m=)N#uP!M5uxR zOjKttJm~nr+7yB@VlS$h8mHl!Vp3IO1CmP6N>Ah_oDb{NLfH@R#?)&*Jxa3fcUqkkA_m$_?p4JH7=%sbLW=HH`LWap z0b~L<+JvdPNG%mGSxx>wx@2o_x|pqR_d&b<#+f<_8UE>|>e3`~p=j@2+|o#AUi<5otmDg3A3< zL=(J**|SVPt3k4Eq;~ttlp@EwEwyP86h4)ukc;Fsk(V#Q`Ck{0w$1F=lpJQr^oE$XoKTzsW7)n55OBavrVx_ zi-C&z`9-eo<1H&LX0_jCRxC!soGPSk93X%Zu2M-{8pWQpX~A6bOW(`!$(FlV@cUd? zphocr+~}Z+&c8JvrwXc!Nw8;strtj;j8&Y2UIu4!k$Rq0 z-Ql&vVyvd~M~%@UX=orQG7Ga@1jNbrY0Bl47FadE*qT|>+aqBi=l5M#j017|`GBA8 z2?Yn@Ptlf%Eof44NQK$k`r@r_!+wd2GqLW!(#yZ7J1=qT65!4~0W1DtV9a?FV30p9 z9mA_@elZ(U^nQPh64_`S0Eqo`3%^)h)A@4@>(gqI(|?Iz)@P@*qYiF^Z?6s0D2sPp zW_;NDDcb2dSmcznjP`LR3PqWMAYundTDEMTJ4!-zeG6oZsMji)VR%R~TXMBkbL3Xf z_sv~R7rZ}r{bF1Y%jWdB*@oK=Cpx#5xY~lb>p|eA;nuk4A@l$Dj`i7ps{xC%f2U

+J2; zy6hyRJL#kkz)9yK+QIfQo6cH}FLGZ1_PwBL`)eRO^^fqiGhB+^{!|KijeUUH#`Wi2AQzM{gXd7OZ!nW=$RX+}_0C*=O9N%K_ z38z#~?^D8>tk>HE*=ep}_7;Ubz7`U{ZuRb{!Ir`ew6($o*IUywQN|i1A^z%?rRoU+eH6VW=IPl_aW@ir&++ZGbt^U*19{xZ)qROGY z<=XI9+|d0=8PW>aBWSxRfScL@Rnfe_Zp}>Q5p03aN&R;Ftui=qehLCOyS?ZhfS;yh zAMFQM!W!ZH36Y`O!4|^$3 zdtKq2oDD6wFX#7rFLqCR{)v^H#b?>Qqt6wyKbIr7Q=X-Fm{9|d3NjM6*d`fI@$Ams z4?aA*m)h$yoY)%*A$Ld$p1BEcKN_I&&yS>~H9XUs2_{q>d1g(>$CZX=a>Yy&nPj+8 zahS_9NzM%XmKyyut!?dNWx0E(0%G8ZJlq%N)7JGD!$&``vScfLwfdS?n#G0)M@WG- zMsaChKlDk*A>zHTxpiLg>9-yA4^k^B;42{gy5!Q%anh>c(uwmG5x*0 zS;WYF3dbDfJ2EdYF}L3{!ohal>P*uYJwuKh80v;2GLc#oTr?{#hit?gZY#7RJgi5$?#QA z@a1WVnmtjL^@#PUs)5}ZX(g^_3Z7#|*aqNJ_`Tn7w)CfzxrEeFm48qE$Z~HWXrweC;q=lD3B7FvHM#hKFk*;fR)nj+O z4o2>ED`gMS6o>_2R#Gd9Oteb-DqcicMnoFe<(-<^#n$6~mGa^RuSCkY4b+5$>b;5B zy`(4B?niixT`EtQk}ik$&<{jIyxQerrVTn7E%lH^NYcIxHP8Pci)7kr>nfPFh)9+9 zc6p~p+G{A>b>|z$o~o0&KTMPE^w6cYlR~Z4B(At1E6KO91G}yGp!cO#6YFQABqJv)QUzUu$Ali)GI#I{SF7(mCL6bVh_F7#SX7IPLkpa}@89 z=1T(Rr|P*P&s=O7oMCNw6q0CHGsiN`M$|^F1?Hi5=Qv)|7wN)x1k;+ zbE>oM@eK4-Dd(M$^Z?Aay_;&15mIe`Kz&h#uHrAk>U2%Iz6|tgzM^Ww*KTR;;RPiL zeED+Dg_iQH*u^C0n^tInB^@z`bc|KCXM@`B|_OyP%t;EchhS zHQWb`TK_YV&_klMq`Z*x&z~GyjY&^bZkOZaRn%o)<6(KaFvPw>#N($bL{4 zaxN>U*nU3*c5hx6ipz%eU<3k?B&DU~* zZ_DAW+?65B5Fa9?=w2ox$knlMB~0Zn8FOO>;|9Os zVgck#DmK(pJ$983Ef6LB@K%y$p0I*n*1_?!pRV6aC|JbGA%yXG$;z6hk#WotR2MlF z0E@FD6He=rTkOjh9t2<3R8~7d>Mylzl2l6`uD0{@`FO5{UR5tX2@h$1K_%iRJkF}J zST=S2-m0du5+tqx3Jx72Of>ydIM#5UT4pLuW$e*}wry;}IAL})b3 zGyuGGGln-UyD|A;ko~|;*v?z+Oc({w#pv8LC#%nyBGrGLIKvO`fgwA@_}NkkIgJV2 zM;4pdlTOo_GrICdNTEeis4AmiW&AT`w_oM!OroWuq0xw{nfga9SF$;Um=DKC!kjJ2 zv7EmTpwIg7+~&fasyfHQybUR8Z4WLVr}ZX$-S-zZDU9txFqXOaGASsdr%a*lSD3aT zbhW4Zi!Xv+blKMaclnrOIu#2`q(Ta{Gof4!VN1I4N56Zb^PD?jRB&5>H zU&du4`Qq*NhrAtNk$NWn#=X&hKAHf6yhEq$1!sj+@Lw-SW zwlf#smxMdW*k%SQs+^Rrk}%q}$=fy1xpE4T#u@EOPYHN+6;+qNebgjdQEzdz`dmHd zy#8kaO)DLe`RR(8p{KXZyJXd<`r%7xfY9@YU)h4JGv`rfB%4$s5@m&}carF2TuH6X z-RxNnwH`WFF=Sqf0!BRH-~EF~*JW{-!q=OJ#bmhEHX-?=E*)IXH6C?*J@RY=>vv1) z9_dZgL;iMt2M$kSFNa4_T^H$Y;?-}Tn`|!DJ%7D48|gi7gna#-*}}GW+G_p$H2>-) zx%^w#5!tPvk!j3nx*NrOvgmVCZ2FZ$m;Ix66@&kt>(IK->ceU0+>?Ksz@y0g?#sba z-I4#}`pd*oXxHYFYhJaW(q-dIh#{)ditDyRyWhL1`MLDW{S<$nlqe(^_GqagEm;w_ zE_Gxj<7gi-JWc42(R|?v&a?syZ%sL&A_Zw*f!7t(TrL*(+vTdM7zSTC?(hDw>m1}TuF50g(W`Jv+|lHD%xrwmH6KgtQ?c_ zjOK1(!?Grh*m$bqw$-jCbP&rG&8ME>9Zotj1X^~5{XW^P&pFWUy6PRdEQKwomBPJ)oyaR^mTTgy@YxUX$fq}$@p-3Frvc^mr1-oIZHfn6x7n-WE80$ zJyMx@C?ehhRdo+63rhI0%u%iV+jzF|E5Y`A!`ujI30g>qi#46RqNeVFn9`+intM(q zrdbU0Cj^eniVstyv` z>+hSoRI%q`6Hg5PEw)=>MW4f4##-s0h}PY9T0g7XWKm6BOdg=Z?`3vubXh~MzMgSE zAJNsL9e$Rnkix^CXT3OJsXPzuUAO`%6Tv-TC5e$)&3d+ekp9#Hi(b~CIATk-T+j#a zzE14%P)sY&mM!PR3zB=YT^YoWULdSH%qCTKa#OdZs#V728PoDYM_WZQuT&9@ZY|zf zn*KuQ$Pv2Z0pjL{Zb;1h@$ZshFmcf&4$&hG-W(PCrKVYsvgLN+x}gUa z2ZT_clp|ds1P+FRzZW{clq2^?h!UzZW-&V12pEOtkI3pFTPGxGmKo=y-%bENShklu zzSm;G+EC8NhAsmzG}nPGax7d`KXGRZFd&Ru&M^rhj;F+Pp5ta*Y|y4<9O zT@SXb?uoK#Rpp%PdO5Za6p|SP$3OWP!Zwg{d0qb2$Ot)k9AV%Nr_|INByZS5LQ>Ad zqc43XO#~#ZX5G^7!IoqsD-k$SZscVux%CGbgoV$NcIub-d8f4H-9YX*Rd9llKM=if ziy7HZ2Nxb)|&Y65V!@uYU6YzFX#D-PD}h4w;f|q*UeifRJfR1#r@?;k>8?5>B-j z(8&ujueM29jZw<(PQbo|0i_eG%*wqMMcLRijQSYdSnXGp1pA&@JzeqKRy#_#4IN#t zPgQI0rFhJPM9}T7CnCks{i)6oM376>x z72@Q6RWgN+6FYuP(oaBNADW zrer05`f263)Yru@XauhobZCMl#e~r_Sb-W;P}rZZ9%E#R7F{IE>`9(M2z2Ra!A#Fn zTInOoMwQFp2yWk^mQP@K$z!`b;Rqj>s(z3RXixyFnzZWkfkH$(wd#OhyoVVAbhX$| zO@l#hx#Xo$^O0;>7Ll2;O!7Gi#hxa$kY@5C316qWX1zWlYlR&Emlr6na$(sntExxz z;G?HTKM0NrJ`>*|N)l4K%4tu!T>kQCrbqh~MBJ`AEmV;BD2sJTA!)xxyf}J7XrR>* zn?lI?O~v)b7g>BXLY>?X`23}x{c;W%kul5fr>sqH+~A`iUQ!bF075R$R#W?u%WR4t zbB;6WSPjP^ZZ}S{1Tw4DFd+h{uWJ<`I_^{9OG+-J66Yt`BzxQ_c>jiNbJ+$-{iM?9A+(cH^m_Sqa=eFCprFJ$hB;ku^^KSd0; zrw^e|27L%gA%Qec=K6u`$+*E{rEFn>EuGmUDaD2+s^O*LEJVB!YZ@+1`KqBZJ>hds z8N5XqvAV2JT1EC!`V*n(QKkGb9bl4M8kwF?6#dty_#bF0ZP+$pV=1GGH(K&Gu|-4j z_=7Cv>~l$C2O|gUej|J-5qG0!PrQ|pyJW{qY&o+syQW%-3 zs(wSMWn-Q)PANHQX{4_H41g6aNXq3s zB2LHS^?Hxomz$x-d9Cj}&rv%O)XRHfh0{a#X)kwWZc(DHff4-G_SsHR>cD6No>FR2u*dtiX-#+(o%lSQYv-)yWzV7%0 zM10wLdUJ`i%CNUbajHe?ICeCcitutj6!R9|W~|@ThmMiDuU_;Q*DpCO${AV5ah~V* zuN5~8M2}id?iYt@xgwgFIAu`*2n1jKD+_X=4L-?`79Iata|=1lWFR={M@zUMFjWG$ zMj7-S!^W?%zy|6D%m3j@AI<$>_>=(Zv)L`hbCltjdxfx596qq3=&xqcl5c&jWJ4v( zrI7#dL18zBZuDMyc3q$>Xku!=7Vq$*1AdA`W|g@05?6vx$$rgXh=NMvPpjM!hS)Em zA@~tz`d}KTkxHAY_uUp-V;RA8(HPZTD)~=KWxn^(x)4-JF~X0X?b}k|jVT%aNw-K! zpcSPjgNSq;8cCV$BG@%wNyCQXs(9?K0>#0%lGwy}YB*H0ONMu&|EPqOF}vlf0Z>8= zh89s8-la=Vejh>2a1cVzFKFEq@zu!2WPo>E!CS2}@ijvQ?LuXs?>tyG-}1FqiL!5N z**7EWydnH(TwyRwYdXl`Dg5rRbzgT^h`v@Lg#s=Qtng zA+ZVuu)_gj{rV>IgcJRVNVw^%p23YLelG*34bkkUFlf+D^^|(iH~uk(*aF#5-+o0B zoL?NXEbgU1eSMRkX3EWC1T#%?;^9bpl_UznE7jr-uU%29H0Vn@&^u$TqznzPvQs$@ z>$G1*^GTgyM7OTw39lf`{#2Sj&h;Oz^;VDm#wEsq@2aBrtHpaj1+Vl#9SVSm&MTKS z^Bp(^rgJNSy%Le~ZHei1gQ`Z$zc}R-WO%OIAoY=fb>lrCN_?1wX=Ii-<*+}q@fpi3 zkd;ipH#h}{S0#(MYCsT6yJ1i!Q16#;G+ssSs5I6%Zbd#cq!PILK~<&9MscY8uqtXk zvFbr`Tk*S^XjzGN9gJWeOnD{A+Kn@#-{qSp1AeoF9$nimkDR*9k8c^di=9btoCr&M zqcby3rA6XflTNkABFYWoBL(Y_6`vcQS~u({jt}owI{F!CAj$ratS&KTU;}x$sP97M8OG;}z?H!_o^9zpJ0RK=nKsp+f2+08P6y@K? za1Jw8OSbN%(-HX%FKMCh!V*CdAUshm`amioBXxvZzxYQy=$f|*!={t9A=V7;2pJin zsz!cGww6Le`2G;gc0p1V2k2T-)?#N8UN!aDvP2dkDB;}5nLMTp@b6I1Z*=;hp7y2Z zmAmDJx}3)9by>ylU`BmDwa788iONV}fz$FF0Ts2XuSqQlwnd;HP~59XR!9;*d{d#N zmKc!&frpgj`YY?(m%iCo9Pj;!%c6bi7|}8d=Zt)z!dG69MYdYTiyuzw493-2DpJ&`m>F$JlgS zXoloD5%iv2BYCnq5jr3jn~<8kP1!}LLnHy#@mX0R|P;82Un3xM-(aL-4sT^F}V`h3?!tv z1*uK{w)wOIKLUyGo)P7%tjG?@2MY{+#=G8Uae=UWF{ZX-(2(6)Sh604R)cwb`5X_N z{~WxKR@CA*5#1zw;RIlc7&P!q6|Uh^3YXIazAefKWA5biDg$Z%AJ*P6xYD-U7VdO9 zNyoN2wz8<~{!0*PL}<8e`1K z8H567BpmAAWfG(jEZW2rNXX3&S%o$281njk_nBT22HQYVaPLCz`(IM46J^gtGR4b_ zDohL(W(Yf1=FPA%k~lLiIy1I4p@gtuw62Ax4V;2%$%0EcaS=o(9z|xqCQ%mZj!G9A zMQaiLjdFRw(-f21gyS^(z1OP_9q2}WCf)7uV~yNJkxHS&8rGCofDcU0PKSA!b44X0 z2t#_fE6x$ZSQ>pcxZF@kr$!)Q!LHmfvQnbP$#x@`AX%0me(ucu5cj>epgdMyim3`FW>J-I!b|?v zzwHZ^d(T{)n|Mf3HN&;CdHShSg|-DJ))+@#P)c|dUgc4RdwGU`h5Vj3Kli` z;`MDyTLx!u>V2N!lbaQ<&%E}T{Ub=jA?n@X^V!>`u*X=f~&4CtJ^kN{`nW z^xrJ~*Tt0^_$_WPEA;twB{PQtFo@5$bJBvxt5==>EF`g=Cr_pM(zv;Q{r>xEM&tjB zn(<$1pY#7v`=F$!W~Qa8{3CLk`v0q{E`Bkv{xa70{l+IuTll80oHgD)3w2rEUn|&A zJKvu0Mo93P+rOW#2zK-U!1A2ay$8H<~V8Jc{Tru)~|=cku3 zM1er2x`Cpi6I7F0|Kk06MRKiia(GC9g8z2hTkbqc)A)RPf_~Fm?s`8`$NF1>A}!g{ zcFfcEHx(81AW*D$o!_8QeEA0wEdNE1_1nJ&e)CI<)$o;||GuWlnAX79=-1qc!`q?0@esCYsu616u<+ zSGZ(0kkO)qg5Lqa=-&ravUf%MWC8(dmcN({7*d~G5bPH3Wm|EDI)8Ox5KdZwFWB_v zCPUOEkU^W5und!-%Jvk<_8`QF7hiMn=J<(GyHfyx2&^{Diqpu&-0-DwE|Bv*b2D5} ziRiUK;343p0?1SZyM(gvOizB6e&$Fk-<-VO{?QU@ zU4JY)?o~K1FB*v0O@Y{!+C6?BZ70RcC7X^XL>4k<@ zG-rRDjNsP(B!-U+NV&gsIa>Fa0z8hpTmkkn@aNgeJxq;r8MSJ{A6ekIS5knq(MhW$ z!b@d77i`kpf5+*xc^@`(I^DLGI(wXFk0)l1-!Fvgnf3Bc$=4U12nDcHT+#*V6YaqO(4MGS+Sg2 z%aTqjpf(n<=Nhh+^HxZKO2m9+8|kT)sA`+kwe~Fx4%OP2`b#N#@aWNpZKIK)1WSP; zWWcybu0qM$w$%wB*4pJ03+r|p5w(_qhE(1l)F#ees#6>VOH!J_w8>$jxCf=A-t>GZpLj26Vr3l+7N(OCvE{q70eUy!Eb z8r>gvkco@LLEbLY@8nFboLb%V~Se-Aiv52w!oNo5R+XG*|B zSm=A~cIfLNZLt^ad3_=O$q5JT1U70wW%Zy@8!xm)26<2(gIsVhF-&$by z(dg$AkK0#{Au86NR^-J8Vu zT1C(3HwE-e+f3NAfvBQczV|N#)K3(d?Dy@FQ#%7TdK;W3D zhYTH6M{9ch-@)41u`w|l4>9qLsii2q;H+*N0?m`G-LcJ^p)TV7_DoyAqN%|ybis>c zLBlL*6qhIo47>(ugOTy%uKNj5L(xz=ij1>=hTu z`Gc_D`kHBj8E0eXa=N)nYF+ zbLTlWkuBoAJ2}x&OnY^lA_6Fau-`T1U$a8MY-7r(nmCNi^d|J-?0CMc=aopeRmNMB zRJ~O_ZoSs!&p!sa26iIUi+0o3Gc@+ey8)GxS`6I{%@VP@l{DU(1lKKR3dG!Zj(0G3 zl9!>X?yQgfzozoAbwsuVw>nJBh?T>}?K_5}7QMf5%ImXGO3Zm>rq&B%5-U}5!=NPd ziN=siH46{Q=J@^^K>^TE9cW6+8>Pz)rZJb<|4g1^%$ekJdi(%+axKwTp|F}_CZi0! zewdR{etH^L1OPr3#OqiVQVt2nO#IGi>OA-tK~Dlpe=W)V%0GnuQjd7p`Wuq4{nbwF< z@K4cnt{z0+UcLA=b5HYhDF*^BKxLiUh!ovd4?83|Q z)?DBB8b*#%id|yDjT7q3QvkercTEiRH`i8`x!+ta*FvyWVXsehj}t$3vBBcP5FB9X z6f^`7^|ut4wk4WEH4r`^Am~0_-Y`ByKKs&?M?4)}GWPDEOs>AYW_};iR*w2(N+*i; zdl$8a&R_c)GAJy{b{c>HJL4qH)mF?lO9h5th9pk3!>Nb<{=)~_;Md^>zsRz>?NzG^ zm+?F3ln=8s<>C14i?Fk}m7b{@ z(egbZ@m6nhu7~#8qZjawNXUK%URou&2y;v@xZ=12OP8EL~+mH{fgHqVmkE^=9 zxPZO60)a&37vuBXr<%J*hgL)xKj^BAvW(msl_^>{#RfVY*_)dN&*d;ANydK75cQBV z{_}7o$&fYvc5Xqsel3Daxa$vu$3Ru68k~&VHso}1#KoQM%i)i9e~-BIu!70J^O-}4 zu(Qn}fj6a)E}ip*%3yxW;3v%xx`3w(1Rc2RLtKRT%v~9Qw-x>$LnX`Sf9<*K zk%&O1smOc3yBs??c3Ozh!_h(x?#FRG$?*EPHt3h!)nhb>*Py5)*A{928)t?t_dWvC2lFkDxhUVCV`6$%f+_9y=3Dz?p$p5&@RJ1 z_VJMOVeWrgI$vO-$_1<#BK3wMCK*W52T+C8&!7&_ptd9?RTAg3{qZSvB=1`;54plT zup12F70js1_eC@cp%Fyc{W;^G!k7Px2LUw%Oz@lg1MZ~V$-LU>&y1Ie+|TTc&G&~~ z{-*mkVAUu3f8JCq8lbQQeyvxPz7+bx|9#^O9Z4qxBWoiTW*vH3!+&qwU^FK~?r(=P zbbWN>R!(9B!9oaNKi{U$EmVGZJFo6b@ZZn2w2}c(CsceOdr*y?Bf~ z?m}vDB}{3QPoJ4tI4N}^AI&>*ed@BNvXp9KX%}A_l^I)t&H^^u{ojk69g6!aeY~#$ zxM_vemiZN`CJ=`vR2u$+V5n-|| zrb^6h+*xy|#xP(M$u;;^~R5V}WBl%GU~e7yEy(2>z}A#Rc&B3e-n?Y3*dgtSFNmGWZB$%j4*bbgJuazWw@|PJ}WG=+zhf-q&@_o%KGnXY;{&mUY=4f7%vyIQ7Kz z(M9q?_8HyvvCx#5^O!S+&g^rBJe2hsIoQVappEo?+V=S9L-h*00*`3=WyfIgp?=r; z*?pAtc^Lk=XVb|v<}uv=2wJD}8VLUp^R`8QsnK<-!fXG5dGflg^L)_tE{y=%iTCn! z%Q_&a|gfmd0UD940eZ%|9QppDs#_s`*p$reejr4_UdqDS^Rl|?R6MF%(l$) z>EiX-%ke&X#l~a#a)2+))Bm2aBl8A$@OfWstN3$QF!<_oSNAbF#XNMi<=ORm5B9qC z8hRhxwNdn1C~>Pni~qjzbmg^m5kLKH^F!seXzLT=xZpJr|7}a>dBz9oefdhW7hl|SyX$QT z|NRb~xA4v>Yt!p%P1|?yPBE1HvE6s|e&F+v^_hE=<@wOIY0do;;iGrGNFhCuvb^g% zwReOidTb{1bTT2v_hYb|!pk`~#%eLTwP!=Cw%Zh`n}+m-Kz?8-t!>ET-cCjv@~*|- zW0neAYa5D=I14UKV2r%l-^kUV?5*!V%H3T|*^|JZm3fuCQq>E11o8J&;7Mt4>ujrU>1O|t5`&V0EYZR`|F@9G|Z)U<%n^lr3#eDHD>7&Xt)k| z13DsnRy(!_oE@@HLuOI{-eg_?QNUP_^RKy1ziN$R2rKCNUst4?OY`+3WYpNxq{k2i%*aD@7kcfs%)W}aa-(03n5#;cZt-Vj0WF6*m3huAN zgCDf6@q5&pFfIQg-e~eOxpZp>wZ5Eo!!tzR7!uZ0_VLin8usot=|oOabt}1c2-vHl zm8{j*NLP$wL{(Xd5NBvF=02;HGABzNwrVd)2^x`FL^%EIz#gl$$FI?34FxtyYjUX0 zTKY-<=~k~Mt)~Gg1HxU{X;nW^py%yq;(9&RjMtm*77#6e2Lpr9c|D#TUFsMD;c}MG zI|H@yD=T{E`MMP{)UMm)9{FHY!n|Wd6)l6#?#xiTi71vHqvuRVFK_Z-tbHqV)?;ZCNF05r zpQT!kcs4E7$EuGh@O!S8_gf(Nb-}$*=Ov@bWZh&I8NmAraIu@{EecbeDGPuIw%zDN zpA?Z2-)Q%5rEjmrZXDE09lP!Y8BMgGMu)7OTsECOdAA*2@=HH=MGQqJ-CFofvdy8jwkQ$Xs}pzzb`c zz%{IzZ?0|Ax=bjy_1f0#7xq7Ahq({hCcdl{?Rc>Z<$&iY#g29#Yum0lIAv8c1(;=e z&4+oGtg!PHr)~Got)$%AsaTT|=KxVy+uJvqT|*$yTH?8)=_8rvc~qe|_K4QF_~Q!n zcphKZ(Cx?z)t;P)=at6XEFc&W+yr#HZXoGrP1JyB&?()*~Jt@hR! z#abE~1+`Q(Y7!cLT%A*m%!vxf>q<;0rB9xLvpUV6eQApqf85-p$ncYFdw-ZzMIpJm^93WHz88Zp=dJGE)>*_J2?9|m{GBruM< zV_U+(0v1Vp1Ze=ugkmec4Ds)JLhy9JVH2Wnd#*@Z@C|1Z@<+2+GTV+EQDHGRT=Qx{W9p|@|y9z%<}1}jJgq|aLqA^%1kpCdISESS-Oh9!al@yGMo1& z1oAMY9U$K5uQW7EKf{)OX>+oDSBL>BNNZvRTlZZO=0oBuk3>DxD^0D@8+V|B|CFbh zOXF`G!mrYTfomzm17+moF-UA|@#7j-Y=VSwhfjhLo4PPYFonwtBGL1Ut}k(IwoHmO zT^2nHcd?qdqd#+!vr0PjjPkmrH*D~_Lm7R7TJTsu;DoO{H1Gj8qeHXs-2H)_hvWcN zE!Bn`O&gskY^H>wkZlv&rhd0_L!ne0Xj;|*w%XJ>ZaG0gQ9}{Ihj&c;Y*<}|V)NV~ zi=|0}6T)MN;B=Pyct)U+Qz6aMC-bJ{$~9dG0orMu0q*w#5^7Llk(WDrV?7y3g9e)t z5$%s|Q>c~GsP9JD7p%F2iD>8!x#lRZ-}H^lLD=U<8+7a17QVx-Je zc0(;aFrcY8ytLeXoI6Y8hJ!6>KOp0Tie^OgaOU^{F(~>%n#-yy`CAcpH+P+CZ_POb zys;U*x?U?sQwIkrDKoHhDuS)#RT*B>;iS0%o?Cge9j_XDN^pg4q>O}&*cl@TW!CAW zy5to-XReRIw-6}O@ngkl&C1|%*~w^YG{J{iyk|WG$0%5UXB?D-CUDInb+Z5mU_Bfhay+@*OlB8CcMl z&LJQ;1c-5UvxmQ-L3EHuko$7j)ifNJH3ZYLrgSy6q|eQrRKg(7ITJ z26cfNMSTpOPJ7H!eq2Hr#y_MAnKut{@jQ(g8EKRp@G?a?&!; zbYaoedTc_I&OrpQMP^&7Nw=BdlMQS!d%}tjoia^{W4nzq%D$iK$st5k;!Gy2Lz4wm zphX5Zt7;tg#$p4|-q&XB3L_w`hW0sXC9Df!@2>(H?!k9uyt8 z77m-EJ{CedpV!SR%booyTB-*859C+0QPjE4AA=a=4+rl8-TU<<;@s26r%%d92{4C~ z5|rVK)P;LCTtFb))I{W^}Be~7SBqLyHtWb80sd?upa{aq)^Fzod z|FF$t&ph;q$*+6t?-*_eXF5)!17e$)mpBD$E@?E3=|y7cO7E;^bOa;%T8ia>nKG|B zVwgK8tFTnaG|_@MCFYt*fO+?mpA_11TutJHY~7Ms9|%Lvos5dq(&}1J_&`i0t_M$E zqSGSzz3GJ+?|O+q-maUQ41MA-1qV619dlDr$XPl>Bdd4C1B@PoOU&9{$InzOQS%oyH^RaT7yjmxv&(wfSR zG8ZzotBr~KWrxJbRKExgSsmk0QA)2olZ} ze>#c*GTBq)5ys>7Bfig7%sR_+sJQcLES5ZzI{eM(!IcG=LcFY> z^*A|aRsOd5FsIT_g0fi{o?hf!?Q?@k+&IaEupK)s2})9klo`*nB$Uu32kITEzmCT3 zfpM+2M>sTX|5=MJeIKuPFjqw=B|Tp~)$;H|ytW^%(SH!L1u!cB<>kL!rGpd;PInmoY0Dg#(mU-p=ym8g{S@&EdKOmF>Mj`IVb{pjnaY~ zS~qYn^P)AFafD*5-Ff26q)zLeezD<^sEpTTfJ$O2H$QGgbsAk{W?Z$G<}C_omj>Jbo>CuRu zcvi18e8_?Pw$1>IrA2>DojeujrHlIgC12-?5zGo!Cx^+lSZSd$&ger^3L}|(e>A(x z3vu?|KbwKt$ZS)6Vgq;0mleVt;}iSgTZNn7a%9Nu)BW|5gHl%&{7tC;E=qc&t1UH8 zIzSF@xlyg`Mh&O5A{tCoE1tq`oqO|Iv3Iy2gHeWq+DNWA7WNs#krYhB+COLo9aOU4 z@J=Af2B7MB%2j^=t`EtHPnhjYgt6H%%JtdU5IJSm%jq=L$5D9X{Ds{>Q4oXku) zJR%#1VvBH{siJADUSG%DY2_@EIa|-u8U{3+COZ?78`QY3qL&A9k8lM9+W=}{tG{I4 z;rl51&~X%b4jTThe#aDFe#qG1;H7ecm|Lk(FcP2!fXda_ES+q>K2LD8b_dALw$;?N zRQ?)zqv(13ZoLy%IhceX)io}iqmg>t7EIiHy?MV`3m~U@R#{wvFz;XG`#R;rZm