Add support for waiting until a refresh occurs

This adds support for setting the refresh request parameter to
`wait_for` in the `index`, `delete`, `update`, and `bulk` APIs. When
`refresh=wait_for` is set those APIs will not return until their
results have been made visible to search by a refresh.

Also it adds a `forced_refresh` field to the response of `index`,
`delete`, `update`, and to each item in a bulk response. This will
be true for requests with `?refresh` or `?refresh=true` and will be
true for some requests (see below) with `refresh=wait_for` but ought
to otherwise always be false.

`refresh=wait_for` is implemented as a list of
`Tuple<Translog.Location, Consumer<Boolean>>`s in the new `RefreshListeners`
class that is managed by `IndexShard`. The dynamic, index scoped
`index.max_refresh_listeners` setting controls a maximum number of
listeners allowed in any shard. If more than that many listeners
accumulate in the engine then a refresh will be forced, the thread that
adds the listener will be blocked until the refresh completes, and then the
listener will be called with a `forcedRefresh` flag so it knows that it was
the "straw that broke the camel's back". These listeners are only used by
`refresh=wait_for` and that flag manifests itself as `forced_refresh` being
`true` in the response.

About half of this change comes from piping async-ness down to the appropriate
layer in a way that is compatible with the ongoing with with sequence ids.

Closes #1063

You can look up the winding story of all the commits here:
https://github.com/elastic/elasticsearch/pull/17986

Here are the commit messages in case they are intersting to you:
commit 59a753b891
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon Jun 6 10:18:23 2016 -0400

    Replace a method reference with implementing an interface

    Saves a single allocation and forces more commonality
    between the WriteResults.

commit 31f7861a85
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon Jun 6 10:07:55 2016 -0400

    Revert "Replace static method that takes consumer with delegate class that takes an interface"

    This reverts commit 777e23a659.

commit 777e23a659
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon Jun 6 09:29:35 2016 -0400

    Replace static method that takes consumer with delegate class that takes an interface

    Same number of allocations, much less code duplication.

commit 9b49a480ca
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon Jun 6 08:25:38 2016 -0400

    Patch from boaz

commit c2bc36524f
Author: Nik Everett <nik9000@gmail.com>
Date:   Thu Jun 2 14:46:27 2016 -0400

    Fix docs

    After updating to master we are actually testing them.

commit 03975ac056
Author: Nik Everett <nik9000@gmail.com>
Date:   Thu Jun 2 14:20:11 2016 -0400

    Cleanup after merge from master

commit 9c9a1deb00
Author: Nik Everett <nik9000@gmail.com>
Date:   Thu Jun 2 14:09:14 2016 -0400

    Breaking changes notes

commit 1c3e64ae06
Merge: 9e63ad6 f67e580
Author: Nik Everett <nik9000@gmail.com>
Date:   Thu Jun 2 14:00:05 2016 -0400

    Merge branch 'master' into block_until_refresh2

commit 9e63ad6de5
Author: Nik Everett <nik9000@gmail.com>
Date:   Thu Jun 2 13:21:27 2016 -0400

    Test for TransportWriteAction

commit 522ecb59d3
Author: Nik Everett <nik9000@gmail.com>
Date:   Thu Jun 2 10:30:18 2016 -0400

    Document deprecation

commit 0cd67b947f
Author: Nik Everett <nik9000@gmail.com>
Date:   Thu Jun 2 10:26:23 2016 -0400

    Deprecate setRefresh(boolean)

    Users should use `setRefresh(RefreshPolicy)` instead.

commit aeb1be3f2c
Author: Nik Everett <nik9000@gmail.com>
Date:   Thu Jun 2 10:12:27 2016 -0400

    Remove checkstyle suppression

    It is fixed

commit 00d09a9caa
Author: Nik Everett <nik9000@gmail.com>
Date:   Thu Jun 2 10:08:28 2016 -0400

    Improve comment

commit 788164b898
Author: Nik Everett <nik9000@gmail.com>
Date:   Thu Jun 2 10:01:01 2016 -0400

    S/ReplicatedWriteResponse/WriteResponse/

    Now it lines up with WriteRequest.

commit b74cf3fe77
Author: Nik Everett <nik9000@gmail.com>
Date:   Wed Jun 1 18:27:52 2016 -0400

    Preserve `?refresh` behavior

    `?refresh` means the same things as `?refresh=true`.

commit 30f972bdae
Author: Nik Everett <nik9000@gmail.com>
Date:   Wed Jun 1 17:39:05 2016 -0400

    Handle hanging documents

    If a document is added to the index during a refresh we weren't properly
    firing its refresh listener. This happened because the way we detect
    whether a refresh makes something visible or not is imperfect. It is
    ok because it always errs on the side of thinking that something isn't
    yet visible.

    So when a document arrives during a refresh the refresh listeners
    won't think it made it into a refresh when, often, it does. The way
    we work around this is by telling Elasticsearch that it ought to
    trigger a refresh if there are any pending refresh listeners even
    if there aren't pending documents to update. Lucene short circuits
    the refresh so it doesn't take that much effort, but the refresh
    listeners still get the signal that a refresh has come in and they
    still pick up the change and notify the listener.

    This means that the time that a listener can wait is actually slightly
    longer than the refresh interval.

commit d523b5702b
Author: Nik Everett <nik9000@gmail.com>
Date:   Wed Jun 1 14:34:01 2016 -0400

    Explain Integer.MAX_VALUE

commit 4ffb7c0e95
Author: Nik Everett <nik9000@gmail.com>
Date:   Wed Jun 1 14:27:39 2016 -0400

    Fire all refresh listeners in a single thread

    Rather than queueing a runnable each.

commit 19606ec3bb
Author: Nik Everett <nik9000@gmail.com>
Date:   Wed Jun 1 14:09:52 2016 -0400

    Assert translog ordering

commit 6bb4e5c75e
Author: Nik Everett <nik9000@gmail.com>
Date:   Wed Jun 1 13:17:44 2016 -0400

    Support null RefreshListeners in InternalEngine

    Just skip using it.

commit 74be1480d6
Author: Nik Everett <nik9000@gmail.com>
Date:   Tue May 31 18:02:03 2016 -0400

    Move funny ShardInfo hack for bulk into bulk

    This should make it easier to understand because it is closer to where it
    matters....

commit 2b771f8dab
Author: Nik Everett <nik9000@gmail.com>
Date:   Tue May 31 17:39:46 2016 -0400

    Pull listener out into an inner class with javadoc and stuff

commit 058481ad72
Author: Nik Everett <nik9000@gmail.com>
Date:   Tue May 31 17:33:42 2016 -0400

    Fix javadoc links

commit d2123b1cab
Author: Nik Everett <nik9000@gmail.com>
Date:   Tue May 31 17:28:09 2016 -0400

    Make more stuff final

commit 8453fc4f78
Author: Nik Everett <nik9000@gmail.com>
Date:   Tue May 31 17:26:48 2016 -0400

    Javadoc

commit fb16d2fc70
Author: Nik Everett <nik9000@gmail.com>
Date:   Tue May 31 16:14:48 2016 -0400

    Rewrite refresh docs

commit 5797d1b1c4
Author: Nik Everett <nik9000@gmail.com>
Date:   Tue May 31 15:02:34 2016 -0400

    Fix forced_refresh flag

    It wasn't being set.

commit 43ce50a1de
Author: Nik Everett <nik9000@gmail.com>
Date:   Tue May 31 14:02:56 2016 -0400

    Delay translog sync and flush until after refresh

    The sync might have occurred for us during the refresh so we
    have less work to do. Maybe.

commit bb2739202e
Author: Nik Everett <nik9000@gmail.com>
Date:   Tue May 31 13:08:08 2016 -0400

    Remove duplication in WritePrimaryResult and WriteReplicaResult

commit 2f579f89b4
Author: Nik Everett <nik9000@gmail.com>
Date:   Tue May 31 12:19:05 2016 -0400

    Clean up registration of RefreshListeners

commit 87ab6e60ca
Author: Nik Everett <nik9000@gmail.com>
Date:   Tue May 31 11:28:30 2016 -0400

    Shorten lock time in RefreshListeners

    Also use null to represent no listeners rather than an empty list.
    This saves allocating a new ArrayList every refresh cycle on every
    index.

commit 0d49d9c572
Author: Nik Everett <nik9000@gmail.com>
Date:   Tue May 24 10:46:18 2016 -0400

    Flip relationship between RefreshListeners and Engine

    Now RefreshListeners comes to Engine from EngineConfig.

commit b2704b8a39
Author: Nik Everett <nik9000@gmail.com>
Date:   Tue May 24 09:37:58 2016 -0400

    Remove unused imports

    Maybe I added them?

commit 04343a2264
Author: Nik Everett <nik9000@gmail.com>
Date:   Tue May 24 09:37:52 2016 -0400

    Javadoc

commit da1e765678
Author: Nik Everett <nik9000@gmail.com>
Date:   Tue May 24 09:26:35 2016 -0400

    Reply with non-null

    Also move the fsync and flush to before the refresh listener stuff.

commit 5d8eecd0d9
Author: Nik Everett <nik9000@gmail.com>
Date:   Tue May 24 08:58:47 2016 -0400

    Remove funky synchronization in AsyncReplicaAction

commit 1ec71eea0f
Author: Nik Everett <nik9000@gmail.com>
Date:   Tue May 24 08:01:14 2016 -0400

    s/LinkedTransferQueue/ArrayList/

commit 7da36a4cee
Author: Nik Everett <nik9000@gmail.com>
Date:   Tue May 24 07:46:38 2016 -0400

    More cleanup for RefreshListeners

commit 957e9b7700
Author: Nik Everett <nik9000@gmail.com>
Date:   Tue May 24 07:34:13 2016 -0400

    /Consumer<Runnable>/Executor/

commit 4d8bf5d4a7
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon May 23 22:20:42 2016 -0400

    explain

commit 15d948a348
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon May 23 22:17:59 2016 -0400

    Better....

commit dc28951d02
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon May 23 21:09:20 2016 -0400

    Javadocs and compromises

commit 8eebaa89c0
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon May 23 20:52:49 2016 -0400

    Take boaz's changes to their logic conclusion and unbreak important stuff like bulk

commit 7056b96ea4
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon May 23 15:49:32 2016 -0400

    Patch from boaz

commit 87be7eaed0
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon May 23 15:49:13 2016 -0400

    Revert "Move async parts of replica operation outside of the lock"

    This reverts commit 13807ad10b.

commit 13807ad10b
Author: Nik Everett <nik9000@gmail.com>
Date:   Fri May 20 22:53:15 2016 -0400

    Move async parts of replica operation outside of the lock

commit b8cadcef56
Author: Nik Everett <nik9000@gmail.com>
Date:   Fri May 20 16:17:20 2016 -0400

    Docs

commit 91149e0580
Author: Nik Everett <nik9000@gmail.com>
Date:   Fri May 20 15:17:40 2016 -0400

    Finally!

commit 1ff50c2faf
Author: Nik Everett <nik9000@gmail.com>
Date:   Fri May 20 15:01:53 2016 -0400

    Remove Translog#lastWriteLocation

    I wasn't being careful enough with locks so it wasn't right anyway.
    Instead this builds a synthetic Tranlog.Location when you call
    getWriteLocation with much more relaxed equality guarantees. Rather
    than being equal to the last Translog.Location returned it is
    simply guaranteed to be greater than the last translog returned
    and less than the next.

commit 55596ea68b
Author: Nik Everett <nik9000@gmail.com>
Date:   Fri May 20 14:40:06 2016 -0400

    Remove listener from shardOperationOnPrimary

    Create instead asyncShardOperationOnPrimary which is called after
    all of the replica operations are started to handle any async
    operations.

commit 3322e26211
Author: Nik Everett <nik9000@gmail.com>
Date:   Tue May 17 17:20:02 2016 -0400

    Increase default maximum number of listeners to 1000

commit 88171a8322
Author: Nik Everett <nik9000@gmail.com>
Date:   Tue May 17 16:40:57 2016 -0400

    Rename test

commit 179c27c4f8
Author: Nik Everett <nik9000@gmail.com>
Date:   Tue May 17 16:35:27 2016 -0400

    Move refresh listeners into their own class

    They still live at the IndexShard level but they live on their
    own in RefreshListeners which interacts with IndexShard using a
    couple of callbacks and a registration method. This lets us test
    the listeners without standing up an entire IndexShard. We still
    test the listeners against an InternalEngine, because the interplay
    between InternalEngine, Translog, and RefreshListeners is complex
    and important to get right.

commit d8926d5fc1
Author: Nik Everett <nik9000@gmail.com>
Date:   Tue May 17 11:02:38 2016 -0400

    Move refresh listeners into IndexShard

commit df91cde398
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon May 16 16:01:03 2016 -0400

    unused import

commit 066da45b08
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon May 16 15:54:11 2016 -0400

    Remove RefreshListener interface

    Just pass a Translog.Location and a Consumer<Boolean> when registering.

commit b971d6d330
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon May 16 14:41:06 2016 -0400

    Docs for setForcedRefresh

commit 6c43be821e
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon May 16 14:34:39 2016 -0400

    Rename refresh setter and getter

commit e61b7391f9
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon Apr 25 22:48:09 2016 -0400

    Trigger listeners even when there is no refresh

    Each refresh gives us an opportunity to pick up any listeners we may
    have left behind.

commit 0c9b047708
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon Apr 25 20:30:06 2016 -0400

    REST

commit 8250343240
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon Apr 25 19:34:22 2016 -0400

    Switch to estimated count

    We don't need a linear time count of the number of listeners - a volatile
    variable is good enough to guess. It probably undercounts more than it
    overcounts but it isn't a huge problem.

commit bd531167fe
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon Apr 25 18:21:02 2016 -0400

    Don't try and set forced refresh on bulk items without a response

    NullPointerExceptions are bad. If the entire request fails then the user
    has worse problems then "did these force a refresh".

commit bcfded1151
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon Apr 25 18:14:20 2016 -0400

    Replace LinkedList and synchronized with LinkedTransferQueue

commit 8a80cc70a7
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon Apr 25 17:38:24 2016 -0400

    Support for update

commit 1f36966742
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon Apr 25 15:46:06 2016 -0400

    Cleanup translog tests

commit 8d121bf35e
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon Apr 25 15:40:53 2016 -0400

    Cleanup listener implementation

    Much more testing too!

commit 2058f4a808
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon Apr 25 11:45:55 2016 -0400

    Pass back information about whether we refreshed

commit e445cb0cb9
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon Apr 25 11:03:31 2016 -0400

    Javadoc

commit 611cbeeaeb
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon Apr 25 11:01:40 2016 -0400

    Move ReplicationResponse

    now it is in the same package as its request

commit 9919758b64
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon Apr 25 11:00:14 2016 -0400

    Oh boy that wasn't working

commit 247cb483c4
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon Apr 25 10:29:37 2016 -0400

    Basic block_until_refresh exposed to java client

    and basic "is it plugged in" style tests.

commit 46c855c997
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon Apr 25 10:11:10 2016 -0400

    Move test to own class

commit a5ffd892d0
Author: Nik Everett <nik9000@gmail.com>
Date:   Mon Apr 25 07:44:25 2016 -0400

    WIP

commit 213bebb6ec
Author: Nik Everett <nik9000@gmail.com>
Date:   Fri Apr 22 21:35:52 2016 -0400

    Add refresh listeners

commit a2bc7f30e6
Author: Nik Everett <nik9000@gmail.com>
Date:   Fri Apr 22 21:11:55 2016 -0400

    Return last written location from refresh

commit 85033a8755
Author: Nik Everett <nik9000@gmail.com>
Date:   Fri Apr 22 20:28:21 2016 -0400

    Never reply to replica actions while you have the operation lock

    This last thing was causing periodic test failures because we were
    replying while we had the operation lock. Now, we probably could get
    away with that in most cases but the tests don't like it and it isn't
    a good idea to do network io while you have a lock anyway. So this
    prevents it.

commit 1f25cf35e7
Author: Nik Everett <nik9000@gmail.com>
Date:   Fri Apr 22 19:56:18 2016 -0400

    Cleanup

commit 52c5f7c3f0
Author: Nik Everett <nik9000@gmail.com>
Date:   Fri Apr 22 19:33:00 2016 -0400

    Add a listener to shard operations

commit 5b142dc331
Author: Nik Everett <nik9000@gmail.com>
Date:   Fri Apr 22 18:03:52 2016 -0400

    Cleanup

commit 3d22b2d7ce
Author: Nik Everett <nik9000@gmail.com>
Date:   Fri Apr 22 17:59:55 2016 -0400

    Push the listener into shardOperationOnPrimary

commit 34b378943b
Author: Nik Everett <nik9000@gmail.com>
Date:   Fri Apr 22 17:48:47 2016 -0400

    Doc

commit b42b8da968
Author: Nik Everett <nik9000@gmail.com>
Date:   Fri Apr 22 17:45:40 2016 -0400

    Don't finish early if the primary finishes early

    We use a "fake" pending shard that we resolve when the replicas have
    all started.

commit 0fc045b56e
Author: Nik Everett <nik9000@gmail.com>
Date:   Fri Apr 22 17:30:06 2016 -0400

    Make performOnPrimary asyncS

    Instead of returning Tuple<Response, ReplicaRequest> it returns
    ReplicaRequest and takes a ActionListener<Response> as an argument.
    We call the listener immediately to preserve backwards compatibility
    for now.

commit 80119b9a26
Author: Nik Everett <nik9000@gmail.com>
Date:   Fri Apr 22 16:51:53 2016 -0400

    Factor out common code in shardOperationOnPrimary

commit 0642083676
Author: Nik Everett <nik9000@gmail.com>
Date:   Fri Apr 22 16:32:29 2016 -0400

    Factor out common code from shardOperationOnReplica

commit 8bdc415fed
Author: Nik Everett <nik9000@gmail.com>
Date:   Fri Apr 22 16:23:28 2016 -0400

    Create ReplicatedMutationRequest

    Superclass for index, delete, and bulkShard requests.

commit 0f8fa846a2
Author: Nik Everett <nik9000@gmail.com>
Date:   Fri Apr 22 16:10:30 2016 -0400

    Create TransportReplicatedMutationAction

    It is the superclass of replication actions that mutate data: index, delete,
    and shardBulk. shardFlush and shardRefresh are replication actions but they
    do not extend TransportReplicatedMutationAction because they don't change
    the data, only shuffle it around.
This commit is contained in:
Nik Everett 2016-06-06 10:45:44 -04:00
parent 200d76e6f0
commit d8056c8213
87 changed files with 2542 additions and 548 deletions

View file

@ -19,7 +19,6 @@
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]Action.java" checks="LineLength" /> <suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]Action.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]ActionModule.java" checks="LineLength" /> <suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]ActionModule.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]ActionRequestBuilder.java" checks="LineLength" /> <suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]ActionRequestBuilder.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]ReplicationResponse.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]cluster[/\\]health[/\\]ClusterHealthRequestBuilder.java" checks="LineLength" /> <suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]cluster[/\\]health[/\\]ClusterHealthRequestBuilder.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]cluster[/\\]health[/\\]TransportClusterHealthAction.java" checks="LineLength" /> <suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]cluster[/\\]health[/\\]TransportClusterHealthAction.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]cluster[/\\]node[/\\]hotthreads[/\\]NodesHotThreadsRequestBuilder.java" checks="LineLength" /> <suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]cluster[/\\]node[/\\]hotthreads[/\\]NodesHotThreadsRequestBuilder.java" checks="LineLength" />
@ -101,7 +100,6 @@
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]indices[/\\]open[/\\]TransportOpenIndexAction.java" checks="LineLength" /> <suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]indices[/\\]open[/\\]TransportOpenIndexAction.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]indices[/\\]recovery[/\\]TransportRecoveryAction.java" checks="LineLength" /> <suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]indices[/\\]recovery[/\\]TransportRecoveryAction.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]indices[/\\]refresh[/\\]TransportRefreshAction.java" checks="LineLength" /> <suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]indices[/\\]refresh[/\\]TransportRefreshAction.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]indices[/\\]refresh[/\\]TransportShardRefreshAction.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]indices[/\\]segments[/\\]IndexSegments.java" checks="LineLength" /> <suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]indices[/\\]segments[/\\]IndexSegments.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]indices[/\\]segments[/\\]IndicesSegmentResponse.java" checks="LineLength" /> <suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]indices[/\\]segments[/\\]IndicesSegmentResponse.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]indices[/\\]segments[/\\]IndicesSegmentsRequestBuilder.java" checks="LineLength" /> <suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]indices[/\\]segments[/\\]IndicesSegmentsRequestBuilder.java" checks="LineLength" />

View file

@ -18,10 +18,15 @@
*/ */
package org.elasticsearch.action; package org.elasticsearch.action;
import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.action.support.WriteResponse;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.action.support.replication.ReplicationResponse;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.StatusToXContent; import org.elasticsearch.common.xcontent.StatusToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.rest.RestStatus; import org.elasticsearch.rest.RestStatus;
@ -30,12 +35,13 @@ import java.io.IOException;
/** /**
* A base class for the response of a write operation that involves a single doc * A base class for the response of a write operation that involves a single doc
*/ */
public abstract class DocWriteResponse extends ReplicationResponse implements StatusToXContent { public abstract class DocWriteResponse extends ReplicationResponse implements WriteResponse, StatusToXContent {
private ShardId shardId; private ShardId shardId;
private String id; private String id;
private String type; private String type;
private long version; private long version;
private boolean forcedRefresh;
public DocWriteResponse(ShardId shardId, String type, String id, long version) { public DocWriteResponse(ShardId shardId, String type, String id, long version) {
this.shardId = shardId; this.shardId = shardId;
@ -84,6 +90,20 @@ public abstract class DocWriteResponse extends ReplicationResponse implements St
return this.version; return this.version;
} }
/**
* Did this request force a refresh? Requests that set {@link WriteRequest#setRefreshPolicy(RefreshPolicy)} to
* {@link RefreshPolicy#IMMEDIATE} will always return true for this. Requests that set it to {@link RefreshPolicy#WAIT_UNTIL} will
* only return true here if they run out of refresh listener slots (see {@link IndexSettings#MAX_REFRESH_LISTENERS_PER_SHARD}).
*/
public boolean forcedRefresh() {
return forcedRefresh;
}
@Override
public void setForcedRefresh(boolean forcedRefresh) {
this.forcedRefresh = forcedRefresh;
}
/** returns the rest status for this response (based on {@link ShardInfo#status()} */ /** returns the rest status for this response (based on {@link ShardInfo#status()} */
public RestStatus status() { public RestStatus status() {
return getShardInfo().status(); return getShardInfo().status();
@ -97,6 +117,7 @@ public abstract class DocWriteResponse extends ReplicationResponse implements St
type = in.readString(); type = in.readString();
id = in.readString(); id = in.readString();
version = in.readZLong(); version = in.readZLong();
forcedRefresh = in.readBoolean();
} }
@Override @Override
@ -106,6 +127,7 @@ public abstract class DocWriteResponse extends ReplicationResponse implements St
out.writeString(type); out.writeString(type);
out.writeString(id); out.writeString(id);
out.writeZLong(version); out.writeZLong(version);
out.writeBoolean(forcedRefresh);
} }
static final class Fields { static final class Fields {
@ -121,7 +143,8 @@ public abstract class DocWriteResponse extends ReplicationResponse implements St
builder.field(Fields._INDEX, shardId.getIndexName()) builder.field(Fields._INDEX, shardId.getIndexName())
.field(Fields._TYPE, type) .field(Fields._TYPE, type)
.field(Fields._ID, id) .field(Fields._ID, id)
.field(Fields._VERSION, version); .field(Fields._VERSION, version)
.field("forced_refresh", forcedRefresh);
shardInfo.toXContent(builder, params); shardInfo.toXContent(builder, params);
return builder; return builder;
} }

View file

@ -19,9 +19,9 @@
package org.elasticsearch.action.admin.indices.flush; package org.elasticsearch.action.admin.indices.flush;
import org.elasticsearch.action.ReplicationResponse;
import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.replication.ReplicationResponse;
import org.elasticsearch.action.support.replication.TransportBroadcastReplicationAction; import org.elasticsearch.action.support.replication.TransportBroadcastReplicationAction;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterService;

View file

@ -19,14 +19,13 @@
package org.elasticsearch.action.admin.indices.flush; package org.elasticsearch.action.admin.indices.flush;
import org.elasticsearch.action.ReplicationResponse;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.replication.ReplicationResponse;
import org.elasticsearch.action.support.replication.TransportReplicationAction; import org.elasticsearch.action.support.replication.TransportReplicationAction;
import org.elasticsearch.cluster.action.shard.ShardStateAction; import org.elasticsearch.cluster.action.shard.ShardStateAction;
import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShard;
@ -55,18 +54,19 @@ public class TransportShardFlushAction extends TransportReplicationAction<ShardF
} }
@Override @Override
protected Tuple<ReplicationResponse, ShardFlushRequest> shardOperationOnPrimary(ShardFlushRequest shardRequest) { protected PrimaryResult shardOperationOnPrimary(ShardFlushRequest shardRequest) {
IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId().getIndex()).getShard(shardRequest.shardId().id()); IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId().getIndex()).getShard(shardRequest.shardId().id());
indexShard.flush(shardRequest.getRequest()); indexShard.flush(shardRequest.getRequest());
logger.trace("{} flush request executed on primary", indexShard.shardId()); logger.trace("{} flush request executed on primary", indexShard.shardId());
return new Tuple<>(new ReplicationResponse(), shardRequest); return new PrimaryResult(shardRequest, new ReplicationResponse());
} }
@Override @Override
protected void shardOperationOnReplica(ShardFlushRequest request) { protected ReplicaResult shardOperationOnReplica(ShardFlushRequest request) {
IndexShard indexShard = indicesService.indexServiceSafe(request.shardId().getIndex()).getShard(request.shardId().id()); IndexShard indexShard = indicesService.indexServiceSafe(request.shardId().getIndex()).getShard(request.shardId().id());
indexShard.flush(request.getRequest()); indexShard.flush(request.getRequest());
logger.trace("{} flush request executed on replica", indexShard.shardId()); logger.trace("{} flush request executed on replica", indexShard.shardId());
return new ReplicaResult();
} }
@Override @Override

View file

@ -19,10 +19,10 @@
package org.elasticsearch.action.admin.indices.refresh; package org.elasticsearch.action.admin.indices.refresh;
import org.elasticsearch.action.ReplicationResponse;
import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.replication.BasicReplicationRequest; import org.elasticsearch.action.support.replication.BasicReplicationRequest;
import org.elasticsearch.action.support.replication.ReplicationResponse;
import org.elasticsearch.action.support.replication.TransportBroadcastReplicationAction; import org.elasticsearch.action.support.replication.TransportBroadcastReplicationAction;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterService;

View file

@ -19,15 +19,14 @@
package org.elasticsearch.action.admin.indices.refresh; package org.elasticsearch.action.admin.indices.refresh;
import org.elasticsearch.action.ReplicationResponse;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.replication.BasicReplicationRequest; import org.elasticsearch.action.support.replication.BasicReplicationRequest;
import org.elasticsearch.action.support.replication.ReplicationResponse;
import org.elasticsearch.action.support.replication.TransportReplicationAction; import org.elasticsearch.action.support.replication.TransportReplicationAction;
import org.elasticsearch.cluster.action.shard.ShardStateAction; import org.elasticsearch.cluster.action.shard.ShardStateAction;
import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShard;
@ -36,10 +35,8 @@ import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportService;
/** public class TransportShardRefreshAction
* extends TransportReplicationAction<BasicReplicationRequest, BasicReplicationRequest, ReplicationResponse> {
*/
public class TransportShardRefreshAction extends TransportReplicationAction<BasicReplicationRequest, BasicReplicationRequest, ReplicationResponse> {
public static final String NAME = RefreshAction.NAME + "[s]"; public static final String NAME = RefreshAction.NAME + "[s]";
@ -47,8 +44,8 @@ public class TransportShardRefreshAction extends TransportReplicationAction<Basi
public TransportShardRefreshAction(Settings settings, TransportService transportService, ClusterService clusterService, public TransportShardRefreshAction(Settings settings, TransportService transportService, ClusterService clusterService,
IndicesService indicesService, ThreadPool threadPool, ShardStateAction shardStateAction, IndicesService indicesService, ThreadPool threadPool, ShardStateAction shardStateAction,
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) { ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) {
super(settings, NAME, transportService, clusterService, indicesService, threadPool, shardStateAction, super(settings, NAME, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters,
actionFilters, indexNameExpressionResolver, BasicReplicationRequest::new, BasicReplicationRequest::new, ThreadPool.Names.REFRESH); indexNameExpressionResolver, BasicReplicationRequest::new, BasicReplicationRequest::new, ThreadPool.Names.REFRESH);
} }
@Override @Override
@ -57,19 +54,20 @@ public class TransportShardRefreshAction extends TransportReplicationAction<Basi
} }
@Override @Override
protected Tuple<ReplicationResponse, BasicReplicationRequest> shardOperationOnPrimary(BasicReplicationRequest shardRequest) { protected PrimaryResult shardOperationOnPrimary(BasicReplicationRequest shardRequest) {
IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId().getIndex()).getShard(shardRequest.shardId().id()); IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId().getIndex()).getShard(shardRequest.shardId().id());
indexShard.refresh("api"); indexShard.refresh("api");
logger.trace("{} refresh request executed on primary", indexShard.shardId()); logger.trace("{} refresh request executed on primary", indexShard.shardId());
return new Tuple<>(new ReplicationResponse(), shardRequest); return new PrimaryResult(shardRequest, new ReplicationResponse());
} }
@Override @Override
protected void shardOperationOnReplica(BasicReplicationRequest request) { protected ReplicaResult shardOperationOnReplica(BasicReplicationRequest request) {
final ShardId shardId = request.shardId(); final ShardId shardId = request.shardId();
IndexShard indexShard = indicesService.indexServiceSafe(shardId.getIndex()).getShard(shardId.id()); IndexShard indexShard = indicesService.indexServiceSafe(shardId.getIndex()).getShard(shardId.id());
indexShard.refresh("api"); indexShard.refresh("api");
logger.trace("{} refresh request executed on replica", indexShard.shardId()); logger.trace("{} refresh request executed on replica", indexShard.shardId());
return new ReplicaResult();
} }
@Override @Override

View file

@ -26,6 +26,7 @@ import org.elasticsearch.action.IndicesRequest;
import org.elasticsearch.action.WriteConsistencyLevel; import org.elasticsearch.action.WriteConsistencyLevel;
import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.action.update.UpdateRequest;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
@ -54,16 +55,21 @@ import static org.elasticsearch.action.ValidateActions.addValidationError;
* Note that we only support refresh on the bulk request not per item. * Note that we only support refresh on the bulk request not per item.
* @see org.elasticsearch.client.Client#bulk(BulkRequest) * @see org.elasticsearch.client.Client#bulk(BulkRequest)
*/ */
public class BulkRequest extends ActionRequest<BulkRequest> implements CompositeIndicesRequest { public class BulkRequest extends ActionRequest<BulkRequest> implements CompositeIndicesRequest, WriteRequest<BulkRequest> {
private static final int REQUEST_OVERHEAD = 50; private static final int REQUEST_OVERHEAD = 50;
/**
* Requests that are part of this request. It is only possible to add things that are both {@link ActionRequest}s and
* {@link WriteRequest}s to this but java doesn't support syntax to declare that everything in the array has both types so we declare
* the one with the least casts.
*/
final List<ActionRequest<?>> requests = new ArrayList<>(); final List<ActionRequest<?>> requests = new ArrayList<>();
List<Object> payloads = null; List<Object> payloads = null;
protected TimeValue timeout = BulkShardRequest.DEFAULT_TIMEOUT; protected TimeValue timeout = BulkShardRequest.DEFAULT_TIMEOUT;
private WriteConsistencyLevel consistencyLevel = WriteConsistencyLevel.DEFAULT; private WriteConsistencyLevel consistencyLevel = WriteConsistencyLevel.DEFAULT;
private boolean refresh = false; private RefreshPolicy refreshPolicy = RefreshPolicy.NONE;
private long sizeInBytes = 0; private long sizeInBytes = 0;
@ -437,18 +443,15 @@ public class BulkRequest extends ActionRequest<BulkRequest> implements Composite
return this.consistencyLevel; return this.consistencyLevel;
} }
/** @Override
* Should a refresh be executed post this bulk operation causing the operations to public BulkRequest setRefreshPolicy(RefreshPolicy refreshPolicy) {
* be searchable. Note, heavy indexing should not set this to <tt>true</tt>. Defaults this.refreshPolicy = refreshPolicy;
* to <tt>false</tt>.
*/
public BulkRequest refresh(boolean refresh) {
this.refresh = refresh;
return this; return this;
} }
public boolean refresh() { @Override
return this.refresh; public RefreshPolicy getRefreshPolicy() {
return refreshPolicy;
} }
/** /**
@ -483,7 +486,7 @@ public class BulkRequest extends ActionRequest<BulkRequest> implements Composite
* @return Whether this bulk request contains index request with an ingest pipeline enabled. * @return Whether this bulk request contains index request with an ingest pipeline enabled.
*/ */
public boolean hasIndexRequestsWithPipelines() { public boolean hasIndexRequestsWithPipelines() {
for (ActionRequest actionRequest : requests) { for (ActionRequest<?> actionRequest : requests) {
if (actionRequest instanceof IndexRequest) { if (actionRequest instanceof IndexRequest) {
IndexRequest indexRequest = (IndexRequest) actionRequest; IndexRequest indexRequest = (IndexRequest) actionRequest;
if (Strings.hasText(indexRequest.getPipeline())) { if (Strings.hasText(indexRequest.getPipeline())) {
@ -503,10 +506,9 @@ public class BulkRequest extends ActionRequest<BulkRequest> implements Composite
} }
for (ActionRequest<?> request : requests) { for (ActionRequest<?> request : requests) {
// We first check if refresh has been set // We first check if refresh has been set
if ((request instanceof DeleteRequest && ((DeleteRequest)request).refresh()) || if (((WriteRequest<?>) request).getRefreshPolicy() != RefreshPolicy.NONE) {
(request instanceof UpdateRequest && ((UpdateRequest)request).refresh()) || validationException = addValidationError(
(request instanceof IndexRequest && ((IndexRequest)request).refresh())) { "RefreshPolicy is not supported on an item request. Set it on the BulkRequest instead.", validationException);
validationException = addValidationError("Refresh is not supported on an item request, set the refresh flag on the BulkRequest instead.", validationException);
} }
ActionRequestValidationException ex = request.validate(); ActionRequestValidationException ex = request.validate();
if (ex != null) { if (ex != null) {
@ -541,7 +543,7 @@ public class BulkRequest extends ActionRequest<BulkRequest> implements Composite
requests.add(request); requests.add(request);
} }
} }
refresh = in.readBoolean(); refreshPolicy = RefreshPolicy.readFrom(in);
timeout = TimeValue.readTimeValue(in); timeout = TimeValue.readTimeValue(in);
} }
@ -560,7 +562,7 @@ public class BulkRequest extends ActionRequest<BulkRequest> implements Composite
} }
request.writeTo(out); request.writeTo(out);
} }
out.writeBoolean(refresh); refreshPolicy.writeTo(out);
timeout.writeTo(out); timeout.writeTo(out);
} }
} }

View file

@ -25,6 +25,7 @@ import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.action.delete.DeleteRequestBuilder; import org.elasticsearch.action.delete.DeleteRequestBuilder;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.support.WriteRequestBuilder;
import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.action.update.UpdateRequest;
import org.elasticsearch.action.update.UpdateRequestBuilder; import org.elasticsearch.action.update.UpdateRequestBuilder;
import org.elasticsearch.client.ElasticsearchClient; import org.elasticsearch.client.ElasticsearchClient;
@ -35,7 +36,8 @@ import org.elasticsearch.common.unit.TimeValue;
* A bulk request holds an ordered {@link IndexRequest}s and {@link DeleteRequest}s and allows to executes * A bulk request holds an ordered {@link IndexRequest}s and {@link DeleteRequest}s and allows to executes
* it in a single batch. * it in a single batch.
*/ */
public class BulkRequestBuilder extends ActionRequestBuilder<BulkRequest, BulkResponse, BulkRequestBuilder> { public class BulkRequestBuilder extends ActionRequestBuilder<BulkRequest, BulkResponse, BulkRequestBuilder>
implements WriteRequestBuilder<BulkRequestBuilder> {
public BulkRequestBuilder(ElasticsearchClient client, BulkAction action) { public BulkRequestBuilder(ElasticsearchClient client, BulkAction action) {
super(client, action, new BulkRequest()); super(client, action, new BulkRequest());
@ -116,16 +118,6 @@ public class BulkRequestBuilder extends ActionRequestBuilder<BulkRequest, BulkRe
return this; return this;
} }
/**
* Should a refresh be executed post this bulk operation causing the operations to
* be searchable. Note, heavy indexing should not set this to <tt>true</tt>. Defaults
* to <tt>false</tt>.
*/
public BulkRequestBuilder setRefresh(boolean refresh) {
request.refresh(refresh);
return this;
}
/** /**
* A timeout to wait if the index operation can't be performed immediately. Defaults to <tt>1m</tt>. * A timeout to wait if the index operation can't be performed immediately. Defaults to <tt>1m</tt>.
*/ */

View file

@ -19,7 +19,7 @@
package org.elasticsearch.action.bulk; package org.elasticsearch.action.bulk;
import org.elasticsearch.action.support.replication.ReplicationRequest; import org.elasticsearch.action.support.replication.ReplicatedWriteRequest;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
@ -31,23 +31,17 @@ import java.util.List;
/** /**
* *
*/ */
public class BulkShardRequest extends ReplicationRequest<BulkShardRequest> { public class BulkShardRequest extends ReplicatedWriteRequest<BulkShardRequest> {
private BulkItemRequest[] items; private BulkItemRequest[] items;
private boolean refresh;
public BulkShardRequest() { public BulkShardRequest() {
} }
BulkShardRequest(BulkRequest bulkRequest, ShardId shardId, boolean refresh, BulkItemRequest[] items) { BulkShardRequest(BulkRequest bulkRequest, ShardId shardId, RefreshPolicy refreshPolicy, BulkItemRequest[] items) {
super(shardId); super(shardId);
this.items = items; this.items = items;
this.refresh = refresh; setRefreshPolicy(refreshPolicy);
}
boolean refresh() {
return this.refresh;
} }
BulkItemRequest[] items() { BulkItemRequest[] items() {
@ -77,7 +71,6 @@ public class BulkShardRequest extends ReplicationRequest<BulkShardRequest> {
out.writeBoolean(false); out.writeBoolean(false);
} }
} }
out.writeBoolean(refresh);
} }
@Override @Override
@ -89,7 +82,6 @@ public class BulkShardRequest extends ReplicationRequest<BulkShardRequest> {
items[i] = BulkItemRequest.readBulkItem(in); items[i] = BulkItemRequest.readBulkItem(in);
} }
} }
refresh = in.readBoolean();
} }
@Override @Override
@ -97,8 +89,15 @@ public class BulkShardRequest extends ReplicationRequest<BulkShardRequest> {
// This is included in error messages so we'll try to make it somewhat user friendly. // This is included in error messages so we'll try to make it somewhat user friendly.
StringBuilder b = new StringBuilder("BulkShardRequest to ["); StringBuilder b = new StringBuilder("BulkShardRequest to [");
b.append(index).append("] containing [").append(items.length).append("] requests"); b.append(index).append("] containing [").append(items.length).append("] requests");
if (refresh) { switch (getRefreshPolicy()) {
case IMMEDIATE:
b.append(" and a refresh"); b.append(" and a refresh");
break;
case WAIT_UNTIL:
b.append(" blocking until refresh");
break;
case NONE:
break;
} }
return b.toString(); return b.toString();
} }

View file

@ -19,7 +19,9 @@
package org.elasticsearch.action.bulk; package org.elasticsearch.action.bulk;
import org.elasticsearch.action.ReplicationResponse; import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.support.WriteResponse;
import org.elasticsearch.action.support.replication.ReplicationResponse;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
@ -29,7 +31,7 @@ import java.io.IOException;
/** /**
* *
*/ */
public class BulkShardResponse extends ReplicationResponse { public class BulkShardResponse extends ReplicationResponse implements WriteResponse {
private ShardId shardId; private ShardId shardId;
private BulkItemResponse[] responses; private BulkItemResponse[] responses;
@ -50,6 +52,20 @@ public class BulkShardResponse extends ReplicationResponse {
return responses; return responses;
} }
@Override
public void setForcedRefresh(boolean forcedRefresh) {
/*
* Each DocWriteResponse already has a location for whether or not it forced a refresh so we just set that information on the
* response.
*/
for (BulkItemResponse response : responses) {
DocWriteResponse r = response.getResponse();
if (r != null) {
r.setForcedRefresh(forcedRefresh);
}
}
}
@Override @Override
public void readFrom(StreamInput in) throws IOException { public void readFrom(StreamInput in) throws IOException {
super.readFrom(in); super.readFrom(in);

View file

@ -344,7 +344,8 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
for (Map.Entry<ShardId, List<BulkItemRequest>> entry : requestsByShard.entrySet()) { for (Map.Entry<ShardId, List<BulkItemRequest>> entry : requestsByShard.entrySet()) {
final ShardId shardId = entry.getKey(); final ShardId shardId = entry.getKey();
final List<BulkItemRequest> requests = entry.getValue(); final List<BulkItemRequest> requests = entry.getValue();
BulkShardRequest bulkShardRequest = new BulkShardRequest(bulkRequest, shardId, bulkRequest.refresh(), requests.toArray(new BulkItemRequest[requests.size()])); BulkShardRequest bulkShardRequest = new BulkShardRequest(bulkRequest, shardId, bulkRequest.getRefreshPolicy(),
requests.toArray(new BulkItemRequest[requests.size()]));
bulkShardRequest.consistencyLevel(bulkRequest.consistencyLevel()); bulkShardRequest.consistencyLevel(bulkRequest.consistencyLevel());
bulkShardRequest.timeout(bulkRequest.timeout()); bulkShardRequest.timeout(bulkRequest.timeout());
if (task != null) { if (task != null) {

View file

@ -30,7 +30,8 @@ import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.action.index.TransportIndexAction; import org.elasticsearch.action.index.TransportIndexAction;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.replication.ReplicationRequest; import org.elasticsearch.action.support.replication.ReplicationRequest;
import org.elasticsearch.action.support.replication.TransportReplicationAction; import org.elasticsearch.action.support.replication.TransportWriteAction;
import org.elasticsearch.action.support.replication.ReplicationResponse.ShardInfo;
import org.elasticsearch.action.update.UpdateHelper; import org.elasticsearch.action.update.UpdateHelper;
import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.action.update.UpdateRequest;
import org.elasticsearch.action.update.UpdateResponse; import org.elasticsearch.action.update.UpdateResponse;
@ -53,6 +54,7 @@ import org.elasticsearch.index.engine.VersionConflictEngineException;
import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.Translog.Location;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.rest.RestStatus; import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
@ -67,7 +69,7 @@ import static org.elasticsearch.action.support.replication.ReplicationOperation.
/** /**
* Performs the index operation. * Performs the index operation.
*/ */
public class TransportShardBulkAction extends TransportReplicationAction<BulkShardRequest, BulkShardRequest, BulkShardResponse> { public class TransportShardBulkAction extends TransportWriteAction<BulkShardRequest, BulkShardResponse> {
private final static String OP_TYPE_UPDATE = "update"; private final static String OP_TYPE_UPDATE = "update";
private final static String OP_TYPE_DELETE = "delete"; private final static String OP_TYPE_DELETE = "delete";
@ -83,9 +85,8 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
IndicesService indicesService, ThreadPool threadPool, ShardStateAction shardStateAction, IndicesService indicesService, ThreadPool threadPool, ShardStateAction shardStateAction,
MappingUpdatedAction mappingUpdatedAction, UpdateHelper updateHelper, ActionFilters actionFilters, MappingUpdatedAction mappingUpdatedAction, UpdateHelper updateHelper, ActionFilters actionFilters,
IndexNameExpressionResolver indexNameExpressionResolver) { IndexNameExpressionResolver indexNameExpressionResolver) {
super(settings, ACTION_NAME, transportService, clusterService, indicesService, threadPool, shardStateAction, super(settings, ACTION_NAME, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters,
actionFilters, indexNameExpressionResolver, indexNameExpressionResolver, BulkShardRequest::new, ThreadPool.Names.BULK);
BulkShardRequest::new, BulkShardRequest::new, ThreadPool.Names.BULK);
this.updateHelper = updateHelper; this.updateHelper = updateHelper;
this.allowIdGeneration = settings.getAsBoolean("action.allow_id_generation", true); this.allowIdGeneration = settings.getAsBoolean("action.allow_id_generation", true);
this.mappingUpdatedAction = mappingUpdatedAction; this.mappingUpdatedAction = mappingUpdatedAction;
@ -107,10 +108,9 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
} }
@Override @Override
protected Tuple<BulkShardResponse, BulkShardRequest> shardOperationOnPrimary(BulkShardRequest request) { protected WriteResult<BulkShardResponse> onPrimaryShard(BulkShardRequest request, IndexShard indexShard) throws Exception {
ShardId shardId = request.shardId(); ShardId shardId = request.shardId();
final IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex()); final IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
final IndexShard indexShard = indexService.getShard(shardId.getId());
final IndexMetaData metaData = indexService.getIndexSettings().getIndexMetaData(); final IndexMetaData metaData = indexService.getIndexSettings().getIndexMetaData();
long[] preVersions = new long[request.items().length]; long[] preVersions = new long[request.items().length];
@ -121,13 +121,13 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
location = handleItem(metaData, request, indexShard, preVersions, preVersionTypes, location, requestIndex, item); location = handleItem(metaData, request, indexShard, preVersions, preVersionTypes, location, requestIndex, item);
} }
processAfterWrite(request.refresh(), indexShard, location);
BulkItemResponse[] responses = new BulkItemResponse[request.items().length]; BulkItemResponse[] responses = new BulkItemResponse[request.items().length];
BulkItemRequest[] items = request.items(); BulkItemRequest[] items = request.items();
for (int i = 0; i < items.length; i++) { for (int i = 0; i < items.length; i++) {
responses[i] = items[i].getPrimaryResponse(); responses[i] = items[i].getPrimaryResponse();
} }
return new Tuple<>(new BulkShardResponse(request.shardId(), responses), request); BulkShardResponse response = new BulkShardResponse(request.shardId(), responses);
return new WriteResult<>(response, location);
} }
private Translog.Location handleItem(IndexMetaData metaData, BulkShardRequest request, IndexShard indexShard, long[] preVersions, VersionType[] preVersionTypes, Translog.Location location, int requestIndex, BulkItemRequest item) { private Translog.Location handleItem(IndexMetaData metaData, BulkShardRequest request, IndexShard indexShard, long[] preVersions, VersionType[] preVersionTypes, Translog.Location location, int requestIndex, BulkItemRequest item) {
@ -154,9 +154,9 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
preVersionTypes[requestIndex] = indexRequest.versionType(); preVersionTypes[requestIndex] = indexRequest.versionType();
try { try {
WriteResult<IndexResponse> result = shardIndexOperation(request, indexRequest, metaData, indexShard, true); WriteResult<IndexResponse> result = shardIndexOperation(request, indexRequest, metaData, indexShard, true);
location = locationToSync(location, result.location); location = locationToSync(location, result.getLocation());
// add the response // add the response
IndexResponse indexResponse = result.response(); IndexResponse indexResponse = result.getResponse();
setResponse(item, new BulkItemResponse(item.id(), indexRequest.opType().lowercase(), indexResponse)); setResponse(item, new BulkItemResponse(item.id(), indexRequest.opType().lowercase(), indexResponse));
} catch (Throwable e) { } catch (Throwable e) {
// rethrow the failure if we are going to retry on primary and let parent failure to handle it // rethrow the failure if we are going to retry on primary and let parent failure to handle it
@ -197,8 +197,8 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
try { try {
// add the response // add the response
final WriteResult<DeleteResponse> writeResult = TransportDeleteAction.executeDeleteRequestOnPrimary(deleteRequest, indexShard); final WriteResult<DeleteResponse> writeResult = TransportDeleteAction.executeDeleteRequestOnPrimary(deleteRequest, indexShard);
DeleteResponse deleteResponse = writeResult.response(); DeleteResponse deleteResponse = writeResult.getResponse();
location = locationToSync(location, writeResult.location); location = locationToSync(location, writeResult.getLocation());
setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_DELETE, deleteResponse)); setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_DELETE, deleteResponse));
} catch (Throwable e) { } catch (Throwable e) {
// rethrow the failure if we are going to retry on primary and let parent failure to handle it // rethrow the failure if we are going to retry on primary and let parent failure to handle it
@ -237,16 +237,17 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
} }
if (updateResult.success()) { if (updateResult.success()) {
if (updateResult.writeResult != null) { if (updateResult.writeResult != null) {
location = locationToSync(location, updateResult.writeResult.location); location = locationToSync(location, updateResult.writeResult.getLocation());
} }
switch (updateResult.result.operation()) { switch (updateResult.result.operation()) {
case UPSERT: case UPSERT:
case INDEX: case INDEX:
@SuppressWarnings("unchecked")
WriteResult<IndexResponse> result = updateResult.writeResult; WriteResult<IndexResponse> result = updateResult.writeResult;
IndexRequest indexRequest = updateResult.request(); IndexRequest indexRequest = updateResult.request();
BytesReference indexSourceAsBytes = indexRequest.source(); BytesReference indexSourceAsBytes = indexRequest.source();
// add the response // add the response
IndexResponse indexResponse = result.response(); IndexResponse indexResponse = result.getResponse();
UpdateResponse updateResponse = new UpdateResponse(indexResponse.getShardInfo(), indexResponse.getShardId(), indexResponse.getType(), indexResponse.getId(), indexResponse.getVersion(), indexResponse.isCreated()); UpdateResponse updateResponse = new UpdateResponse(indexResponse.getShardInfo(), indexResponse.getShardId(), indexResponse.getType(), indexResponse.getId(), indexResponse.getVersion(), indexResponse.isCreated());
if (updateRequest.fields() != null && updateRequest.fields().length > 0) { if (updateRequest.fields() != null && updateRequest.fields().length > 0) {
Tuple<XContentType, Map<String, Object>> sourceAndContent = XContentHelper.convertToMap(indexSourceAsBytes, true); Tuple<XContentType, Map<String, Object>> sourceAndContent = XContentHelper.convertToMap(indexSourceAsBytes, true);
@ -256,8 +257,9 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_UPDATE, updateResponse)); setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_UPDATE, updateResponse));
break; break;
case DELETE: case DELETE:
@SuppressWarnings("unchecked")
WriteResult<DeleteResponse> writeResult = updateResult.writeResult; WriteResult<DeleteResponse> writeResult = updateResult.writeResult;
DeleteResponse response = writeResult.response(); DeleteResponse response = writeResult.getResponse();
DeleteRequest deleteRequest = updateResult.request(); DeleteRequest deleteRequest = updateResult.request();
updateResponse = new UpdateResponse(response.getShardInfo(), response.getShardId(), response.getType(), response.getId(), response.getVersion(), false); updateResponse = new UpdateResponse(response.getShardInfo(), response.getShardId(), response.getType(), response.getId(), response.getVersion(), false);
updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, request.index(), response.getVersion(), updateResult.result.updatedSourceAsMap(), updateResult.result.updateSourceContentType(), null)); updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, request.index(), response.getVersion(), updateResult.result.updatedSourceAsMap(), updateResult.result.updateSourceContentType(), null));
@ -326,10 +328,13 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
request.setPrimaryResponse(response); request.setPrimaryResponse(response);
if (response.isFailed()) { if (response.isFailed()) {
request.setIgnoreOnReplica(); request.setIgnoreOnReplica();
} else {
// Set the ShardInfo to 0 so we can safely send it to the replicas. We won't use it in the real response though.
response.getResponse().setShardInfo(new ShardInfo());
} }
} }
private WriteResult shardIndexOperation(BulkShardRequest request, IndexRequest indexRequest, IndexMetaData metaData, private WriteResult<IndexResponse> shardIndexOperation(BulkShardRequest request, IndexRequest indexRequest, IndexMetaData metaData,
IndexShard indexShard, boolean processed) throws Throwable { IndexShard indexShard, boolean processed) throws Throwable {
MappingMetaData mappingMd = metaData.mappingOrDefault(indexRequest.type()); MappingMetaData mappingMd = metaData.mappingOrDefault(indexRequest.type());
@ -431,12 +436,8 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
} }
} }
@Override @Override
protected void shardOperationOnReplica(BulkShardRequest request) { protected Location onReplicaShard(BulkShardRequest request, IndexShard indexShard) {
final ShardId shardId = request.shardId();
IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
IndexShard indexShard = indexService.getShard(shardId.id());
Translog.Location location = null; Translog.Location location = null;
for (int i = 0; i < request.items().length; i++) { for (int i = 0; i < request.items().length; i++) {
BulkItemRequest item = request.items()[i]; BulkItemRequest item = request.items()[i];
@ -472,8 +473,7 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
throw new IllegalStateException("Unexpected index operation: " + item.request()); throw new IllegalStateException("Unexpected index operation: " + item.request());
} }
} }
return location;
processAfterWrite(request.refresh(), indexShard, location);
} }
private void applyVersion(BulkItemRequest item, long version, VersionType versionType) { private void applyVersion(BulkItemRequest item, long version, VersionType versionType) {

View file

@ -21,7 +21,7 @@ package org.elasticsearch.action.delete;
import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.DocumentRequest; import org.elasticsearch.action.DocumentRequest;
import org.elasticsearch.action.support.replication.ReplicationRequest; import org.elasticsearch.action.support.replication.ReplicatedWriteRequest;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
@ -43,7 +43,7 @@ import static org.elasticsearch.action.ValidateActions.addValidationError;
* @see org.elasticsearch.client.Client#delete(DeleteRequest) * @see org.elasticsearch.client.Client#delete(DeleteRequest)
* @see org.elasticsearch.client.Requests#deleteRequest(String) * @see org.elasticsearch.client.Requests#deleteRequest(String)
*/ */
public class DeleteRequest extends ReplicationRequest<DeleteRequest> implements DocumentRequest<DeleteRequest> { public class DeleteRequest extends ReplicatedWriteRequest<DeleteRequest> implements DocumentRequest<DeleteRequest> {
private String type; private String type;
private String id; private String id;
@ -51,7 +51,6 @@ public class DeleteRequest extends ReplicationRequest<DeleteRequest> implements
private String routing; private String routing;
@Nullable @Nullable
private String parent; private String parent;
private boolean refresh;
private long version = Versions.MATCH_ANY; private long version = Versions.MATCH_ANY;
private VersionType versionType = VersionType.INTERNAL; private VersionType versionType = VersionType.INTERNAL;
@ -165,20 +164,6 @@ public class DeleteRequest extends ReplicationRequest<DeleteRequest> implements
return this.routing; return this.routing;
} }
/**
* Should a refresh be executed post this index operation causing the operation to
* be searchable. Note, heavy indexing should not set this to <tt>true</tt>. Defaults
* to <tt>false</tt>.
*/
public DeleteRequest refresh(boolean refresh) {
this.refresh = refresh;
return this;
}
public boolean refresh() {
return this.refresh;
}
/** /**
* Sets the version, which will cause the delete operation to only be performed if a matching * Sets the version, which will cause the delete operation to only be performed if a matching
* version exists and no changes happened on the doc since then. * version exists and no changes happened on the doc since then.
@ -208,7 +193,6 @@ public class DeleteRequest extends ReplicationRequest<DeleteRequest> implements
id = in.readString(); id = in.readString();
routing = in.readOptionalString(); routing = in.readOptionalString();
parent = in.readOptionalString(); parent = in.readOptionalString();
refresh = in.readBoolean();
version = in.readLong(); version = in.readLong();
versionType = VersionType.fromValue(in.readByte()); versionType = VersionType.fromValue(in.readByte());
} }
@ -220,7 +204,6 @@ public class DeleteRequest extends ReplicationRequest<DeleteRequest> implements
out.writeString(id); out.writeString(id);
out.writeOptionalString(routing()); out.writeOptionalString(routing());
out.writeOptionalString(parent()); out.writeOptionalString(parent());
out.writeBoolean(refresh);
out.writeLong(version); out.writeLong(version);
out.writeByte(versionType.getValue()); out.writeByte(versionType.getValue());
} }

View file

@ -19,6 +19,7 @@
package org.elasticsearch.action.delete; package org.elasticsearch.action.delete;
import org.elasticsearch.action.support.WriteRequestBuilder;
import org.elasticsearch.action.support.replication.ReplicationRequestBuilder; import org.elasticsearch.action.support.replication.ReplicationRequestBuilder;
import org.elasticsearch.client.ElasticsearchClient; import org.elasticsearch.client.ElasticsearchClient;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
@ -27,7 +28,8 @@ import org.elasticsearch.index.VersionType;
/** /**
* A delete document action request builder. * A delete document action request builder.
*/ */
public class DeleteRequestBuilder extends ReplicationRequestBuilder<DeleteRequest, DeleteResponse, DeleteRequestBuilder> { public class DeleteRequestBuilder extends ReplicationRequestBuilder<DeleteRequest, DeleteResponse, DeleteRequestBuilder>
implements WriteRequestBuilder<DeleteRequestBuilder> {
public DeleteRequestBuilder(ElasticsearchClient client, DeleteAction action) { public DeleteRequestBuilder(ElasticsearchClient client, DeleteAction action) {
super(client, action, new DeleteRequest()); super(client, action, new DeleteRequest());
@ -71,16 +73,6 @@ public class DeleteRequestBuilder extends ReplicationRequestBuilder<DeleteReques
return this; return this;
} }
/**
* Should a refresh be executed post this index operation causing the operation to
* be searchable. Note, heavy indexing should not set this to <tt>true</tt>. Defaults
* to <tt>false</tt>.
*/
public DeleteRequestBuilder setRefresh(boolean refresh) {
request.refresh(refresh);
return this;
}
/** /**
* Sets the version, which will cause the delete operation to only be performed if a matching * Sets the version, which will cause the delete operation to only be performed if a matching
* version exists and no changes happened on the doc since then. * version exists and no changes happened on the doc since then.

View file

@ -27,19 +27,19 @@ import org.elasticsearch.action.admin.indices.create.CreateIndexResponse;
import org.elasticsearch.action.admin.indices.create.TransportCreateIndexAction; import org.elasticsearch.action.admin.indices.create.TransportCreateIndexAction;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.AutoCreateIndex; import org.elasticsearch.action.support.AutoCreateIndex;
import org.elasticsearch.action.support.replication.TransportReplicationAction; import org.elasticsearch.action.support.replication.TransportWriteAction;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.action.shard.ShardStateAction; import org.elasticsearch.cluster.action.shard.ShardStateAction;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.translog.Translog.Location;
import org.elasticsearch.indices.IndexAlreadyExistsException; import org.elasticsearch.indices.IndexAlreadyExistsException;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.Task;
@ -49,7 +49,7 @@ import org.elasticsearch.transport.TransportService;
/** /**
* Performs the delete operation. * Performs the delete operation.
*/ */
public class TransportDeleteAction extends TransportReplicationAction<DeleteRequest, DeleteRequest, DeleteResponse> { public class TransportDeleteAction extends TransportWriteAction<DeleteRequest, DeleteResponse> {
private final AutoCreateIndex autoCreateIndex; private final AutoCreateIndex autoCreateIndex;
private final TransportCreateIndexAction createIndexAction; private final TransportCreateIndexAction createIndexAction;
@ -60,9 +60,8 @@ public class TransportDeleteAction extends TransportReplicationAction<DeleteRequ
TransportCreateIndexAction createIndexAction, ActionFilters actionFilters, TransportCreateIndexAction createIndexAction, ActionFilters actionFilters,
IndexNameExpressionResolver indexNameExpressionResolver, IndexNameExpressionResolver indexNameExpressionResolver,
AutoCreateIndex autoCreateIndex) { AutoCreateIndex autoCreateIndex) {
super(settings, DeleteAction.NAME, transportService, clusterService, indicesService, threadPool, shardStateAction, super(settings, DeleteAction.NAME, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters,
actionFilters, indexNameExpressionResolver, indexNameExpressionResolver, DeleteRequest::new, ThreadPool.Names.INDEX);
DeleteRequest::new, DeleteRequest::new, ThreadPool.Names.INDEX);
this.createIndexAction = createIndexAction; this.createIndexAction = createIndexAction;
this.autoCreateIndex = autoCreateIndex; this.autoCreateIndex = autoCreateIndex;
} }
@ -119,11 +118,13 @@ public class TransportDeleteAction extends TransportReplicationAction<DeleteRequ
} }
@Override @Override
protected Tuple<DeleteResponse, DeleteRequest> shardOperationOnPrimary(DeleteRequest request) { protected WriteResult<DeleteResponse> onPrimaryShard(DeleteRequest request, IndexShard indexShard) {
IndexShard indexShard = indicesService.indexServiceSafe(request.shardId().getIndex()).getShard(request.shardId().id()); return executeDeleteRequestOnPrimary(request, indexShard);
final WriteResult<DeleteResponse> result = executeDeleteRequestOnPrimary(request, indexShard); }
processAfterWrite(request.refresh(), indexShard, result.location);
return new Tuple<>(result.response, request); @Override
protected Location onReplicaShard(DeleteRequest request, IndexShard indexShard) {
return executeDeleteRequestOnReplica(request, indexShard).getTranslogLocation();
} }
public static WriteResult<DeleteResponse> executeDeleteRequestOnPrimary(DeleteRequest request, IndexShard indexShard) { public static WriteResult<DeleteResponse> executeDeleteRequestOnPrimary(DeleteRequest request, IndexShard indexShard) {
@ -134,9 +135,8 @@ public class TransportDeleteAction extends TransportReplicationAction<DeleteRequ
request.version(delete.version()); request.version(delete.version());
assert request.versionType().validateVersionForWrites(request.version()); assert request.versionType().validateVersionForWrites(request.version());
return new WriteResult<>( DeleteResponse response = new DeleteResponse(indexShard.shardId(), request.type(), request.id(), delete.version(), delete.found());
new DeleteResponse(indexShard.shardId(), request.type(), request.id(), delete.version(), delete.found()), return new WriteResult<>(response, delete.getTranslogLocation());
delete.getTranslogLocation());
} }
public static Engine.Delete executeDeleteRequestOnReplica(DeleteRequest request, IndexShard indexShard) { public static Engine.Delete executeDeleteRequestOnReplica(DeleteRequest request, IndexShard indexShard) {
@ -144,13 +144,4 @@ public class TransportDeleteAction extends TransportReplicationAction<DeleteRequ
indexShard.delete(delete); indexShard.delete(delete);
return delete; return delete;
} }
@Override
protected void shardOperationOnReplica(DeleteRequest request) {
final ShardId shardId = request.shardId();
IndexShard indexShard = indicesService.indexServiceSafe(shardId.getIndex()).getShard(shardId.id());
Engine.Delete delete = executeDeleteRequestOnReplica(request, indexShard);
processAfterWrite(request.refresh(), indexShard, delete.getTranslogLocation());
}
} }

View file

@ -24,7 +24,7 @@ import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.DocumentRequest; import org.elasticsearch.action.DocumentRequest;
import org.elasticsearch.action.RoutingMissingException; import org.elasticsearch.action.RoutingMissingException;
import org.elasticsearch.action.TimestampParsingException; import org.elasticsearch.action.TimestampParsingException;
import org.elasticsearch.action.support.replication.ReplicationRequest; import org.elasticsearch.action.support.replication.ReplicatedWriteRequest;
import org.elasticsearch.client.Requests; import org.elasticsearch.client.Requests;
import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData;
import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.MetaData;
@ -67,7 +67,7 @@ import static org.elasticsearch.action.ValidateActions.addValidationError;
* @see org.elasticsearch.client.Requests#indexRequest(String) * @see org.elasticsearch.client.Requests#indexRequest(String)
* @see org.elasticsearch.client.Client#index(IndexRequest) * @see org.elasticsearch.client.Client#index(IndexRequest)
*/ */
public class IndexRequest extends ReplicationRequest<IndexRequest> implements DocumentRequest<IndexRequest> { public class IndexRequest extends ReplicatedWriteRequest<IndexRequest> implements DocumentRequest<IndexRequest> {
/** /**
* Operation type controls if the type of the index operation. * Operation type controls if the type of the index operation.
@ -145,7 +145,6 @@ public class IndexRequest extends ReplicationRequest<IndexRequest> implements Do
private OpType opType = OpType.INDEX; private OpType opType = OpType.INDEX;
private boolean refresh = false;
private long version = Versions.MATCH_ANY; private long version = Versions.MATCH_ANY;
private VersionType versionType = VersionType.INTERNAL; private VersionType versionType = VersionType.INTERNAL;
@ -542,20 +541,6 @@ public class IndexRequest extends ReplicationRequest<IndexRequest> implements Do
return this.opType; return this.opType;
} }
/**
* Should a refresh be executed post this index operation causing the operation to
* be searchable. Note, heavy indexing should not set this to <tt>true</tt>. Defaults
* to <tt>false</tt>.
*/
public IndexRequest refresh(boolean refresh) {
this.refresh = refresh;
return this;
}
public boolean refresh() {
return this.refresh;
}
/** /**
* Sets the version, which will cause the index operation to only be performed if a matching * Sets the version, which will cause the index operation to only be performed if a matching
* version exists and no changes happened on the doc since then. * version exists and no changes happened on the doc since then.
@ -652,7 +637,6 @@ public class IndexRequest extends ReplicationRequest<IndexRequest> implements Do
source = in.readBytesReference(); source = in.readBytesReference();
opType = OpType.fromId(in.readByte()); opType = OpType.fromId(in.readByte());
refresh = in.readBoolean();
version = in.readLong(); version = in.readLong();
versionType = VersionType.fromValue(in.readByte()); versionType = VersionType.fromValue(in.readByte());
pipeline = in.readOptionalString(); pipeline = in.readOptionalString();
@ -674,7 +658,6 @@ public class IndexRequest extends ReplicationRequest<IndexRequest> implements Do
} }
out.writeBytesReference(source); out.writeBytesReference(source);
out.writeByte(opType.id()); out.writeByte(opType.id());
out.writeBoolean(refresh);
out.writeLong(version); out.writeLong(version);
out.writeByte(versionType.getValue()); out.writeByte(versionType.getValue());
out.writeOptionalString(pipeline); out.writeOptionalString(pipeline);

View file

@ -19,6 +19,7 @@
package org.elasticsearch.action.index; package org.elasticsearch.action.index;
import org.elasticsearch.action.support.WriteRequestBuilder;
import org.elasticsearch.action.support.replication.ReplicationRequestBuilder; import org.elasticsearch.action.support.replication.ReplicationRequestBuilder;
import org.elasticsearch.client.ElasticsearchClient; import org.elasticsearch.client.ElasticsearchClient;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
@ -33,7 +34,8 @@ import java.util.Map;
/** /**
* An index document action request builder. * An index document action request builder.
*/ */
public class IndexRequestBuilder extends ReplicationRequestBuilder<IndexRequest, IndexResponse, IndexRequestBuilder> { public class IndexRequestBuilder extends ReplicationRequestBuilder<IndexRequest, IndexResponse, IndexRequestBuilder>
implements WriteRequestBuilder<IndexRequestBuilder> {
public IndexRequestBuilder(ElasticsearchClient client, IndexAction action) { public IndexRequestBuilder(ElasticsearchClient client, IndexAction action) {
super(client, action, new IndexRequest()); super(client, action, new IndexRequest());
@ -220,16 +222,6 @@ public class IndexRequestBuilder extends ReplicationRequestBuilder<IndexRequest,
return this; return this;
} }
/**
* Should a refresh be executed post this index operation causing the operation to
* be searchable. Note, heavy indexing should not set this to <tt>true</tt>. Defaults
* to <tt>false</tt>.
*/
public IndexRequestBuilder setRefresh(boolean refresh) {
request.refresh(refresh);
return this;
}
/** /**
* Sets the version, which will cause the index operation to only be performed if a matching * Sets the version, which will cause the index operation to only be performed if a matching
* version exists and no changes happened on the doc since then. * version exists and no changes happened on the doc since then.

View file

@ -27,7 +27,7 @@ import org.elasticsearch.action.admin.indices.create.TransportCreateIndexAction;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.AutoCreateIndex; import org.elasticsearch.action.support.AutoCreateIndex;
import org.elasticsearch.action.support.replication.ReplicationOperation; import org.elasticsearch.action.support.replication.ReplicationOperation;
import org.elasticsearch.action.support.replication.TransportReplicationAction; import org.elasticsearch.action.support.replication.TransportWriteAction;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.action.index.MappingUpdatedAction; import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
import org.elasticsearch.cluster.action.shard.ShardStateAction; import org.elasticsearch.cluster.action.shard.ShardStateAction;
@ -36,16 +36,14 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData;
import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.Mapping;
import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.mapper.SourceToParse;
import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.Translog.Location;
import org.elasticsearch.indices.IndexAlreadyExistsException; import org.elasticsearch.indices.IndexAlreadyExistsException;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.Task;
@ -62,7 +60,7 @@ import org.elasticsearch.transport.TransportService;
* <li><b>allowIdGeneration</b>: If the id is set not, should it be generated. Defaults to <tt>true</tt>. * <li><b>allowIdGeneration</b>: If the id is set not, should it be generated. Defaults to <tt>true</tt>.
* </ul> * </ul>
*/ */
public class TransportIndexAction extends TransportReplicationAction<IndexRequest, IndexRequest, IndexResponse> { public class TransportIndexAction extends TransportWriteAction<IndexRequest, IndexResponse> {
private final AutoCreateIndex autoCreateIndex; private final AutoCreateIndex autoCreateIndex;
private final boolean allowIdGeneration; private final boolean allowIdGeneration;
@ -78,7 +76,7 @@ public class TransportIndexAction extends TransportReplicationAction<IndexReques
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
AutoCreateIndex autoCreateIndex) { AutoCreateIndex autoCreateIndex) {
super(settings, IndexAction.NAME, transportService, clusterService, indicesService, threadPool, shardStateAction, super(settings, IndexAction.NAME, transportService, clusterService, indicesService, threadPool, shardStateAction,
actionFilters, indexNameExpressionResolver, IndexRequest::new, IndexRequest::new, ThreadPool.Names.INDEX); actionFilters, indexNameExpressionResolver, IndexRequest::new, ThreadPool.Names.INDEX);
this.mappingUpdatedAction = mappingUpdatedAction; this.mappingUpdatedAction = mappingUpdatedAction;
this.createIndexAction = createIndexAction; this.createIndexAction = createIndexAction;
this.autoCreateIndex = autoCreateIndex; this.autoCreateIndex = autoCreateIndex;
@ -141,26 +139,13 @@ public class TransportIndexAction extends TransportReplicationAction<IndexReques
} }
@Override @Override
protected Tuple<IndexResponse, IndexRequest> shardOperationOnPrimary(IndexRequest request) throws Exception { protected WriteResult<IndexResponse> onPrimaryShard(IndexRequest request, IndexShard indexShard) throws Exception {
return executeIndexRequestOnPrimary(request, indexShard, mappingUpdatedAction);
IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
IndexShard indexShard = indexService.getShard(request.shardId().id());
final WriteResult<IndexResponse> result = executeIndexRequestOnPrimary(request, indexShard, mappingUpdatedAction);
final IndexResponse response = result.response;
final Translog.Location location = result.location;
processAfterWrite(request.refresh(), indexShard, location);
return new Tuple<>(response, request);
} }
@Override @Override
protected void shardOperationOnReplica(IndexRequest request) { protected Location onReplicaShard(IndexRequest request, IndexShard indexShard) {
final ShardId shardId = request.shardId(); return executeIndexRequestOnReplica(request, indexShard).getTranslogLocation();
IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
IndexShard indexShard = indexService.getShard(shardId.id());
final Engine.Index operation = executeIndexRequestOnReplica(request, indexShard);
processAfterWrite(request.refresh(), indexShard, operation.getTranslogLocation());
} }
/** /**
@ -188,11 +173,8 @@ public class TransportIndexAction extends TransportReplicationAction<IndexReques
return indexShard.prepareIndexOnPrimary(sourceToParse, request.version(), request.versionType()); return indexShard.prepareIndexOnPrimary(sourceToParse, request.version(), request.versionType());
} }
/** public static WriteResult<IndexResponse> executeIndexRequestOnPrimary(IndexRequest request, IndexShard indexShard,
* Execute the given {@link IndexRequest} on a primary shard, throwing a MappingUpdatedAction mappingUpdatedAction) throws Exception {
* {@link ReplicationOperation.RetryOnPrimaryException} if the operation needs to be re-tried.
*/
public static WriteResult<IndexResponse> executeIndexRequestOnPrimary(IndexRequest request, IndexShard indexShard, MappingUpdatedAction mappingUpdatedAction) throws Exception {
Engine.Index operation = prepareIndexOperationOnPrimary(request, indexShard); Engine.Index operation = prepareIndexOperationOnPrimary(request, indexShard);
Mapping update = operation.parsedDoc().dynamicMappingsUpdate(); Mapping update = operation.parsedDoc().dynamicMappingsUpdate();
final ShardId shardId = indexShard.shardId(); final ShardId shardId = indexShard.shardId();
@ -214,8 +196,8 @@ public class TransportIndexAction extends TransportReplicationAction<IndexReques
assert request.versionType().validateVersionForWrites(request.version()); assert request.versionType().validateVersionForWrites(request.version());
return new WriteResult<>(new IndexResponse(shardId, request.type(), request.id(), request.version(), created), operation.getTranslogLocation()); IndexResponse response = new IndexResponse(shardId, request.type(), request.id(), request.version(), created);
return new WriteResult<>(response, operation.getTranslogLocation());
} }
} }

View file

@ -162,7 +162,7 @@ public final class IngestActionFilter extends AbstractComponent implements Actio
return bulkRequest; return bulkRequest;
} else { } else {
BulkRequest modifiedBulkRequest = new BulkRequest(); BulkRequest modifiedBulkRequest = new BulkRequest();
modifiedBulkRequest.refresh(bulkRequest.refresh()); modifiedBulkRequest.setRefreshPolicy(bulkRequest.getRefreshPolicy());
modifiedBulkRequest.consistencyLevel(bulkRequest.consistencyLevel()); modifiedBulkRequest.consistencyLevel(bulkRequest.consistencyLevel());
modifiedBulkRequest.timeout(bulkRequest.timeout()); modifiedBulkRequest.timeout(bulkRequest.timeout());

View file

@ -0,0 +1,109 @@
/*
* 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;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.bulk.BulkRequest;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.replication.ReplicatedWriteRequest;
import org.elasticsearch.action.update.UpdateRequest;
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.io.stream.Writeable;
import java.io.IOException;
/**
* Interface implemented by requests that modify the documents in an index like {@link IndexRequest}, {@link UpdateRequest}, and
* {@link BulkRequest}. Rather than implement this directly most implementers should extend {@link ReplicatedWriteRequest}.
*/
public interface WriteRequest<R extends WriteRequest<R>> extends Streamable {
/**
* Should this request trigger a refresh ({@linkplain RefreshPolicy#IMMEDIATE}), wait for a refresh (
* {@linkplain RefreshPolicy#WAIT_UNTIL}), or proceed ignore refreshes entirely ({@linkplain RefreshPolicy#NONE}, the default).
*/
R setRefreshPolicy(RefreshPolicy refreshPolicy);
/**
* Parse the refresh policy from a string, only modifying it if the string is non null. Convenient to use with request parsing.
*/
@SuppressWarnings("unchecked")
default R setRefreshPolicy(String refreshPolicy) {
if (refreshPolicy != null) {
setRefreshPolicy(RefreshPolicy.parse(refreshPolicy));
}
return (R) this;
}
/**
* Should this request trigger a refresh ({@linkplain RefreshPolicy#IMMEDIATE}), wait for a refresh (
* {@linkplain RefreshPolicy#WAIT_UNTIL}), or proceed ignore refreshes entirely ({@linkplain RefreshPolicy#NONE}, the default).
*/
RefreshPolicy getRefreshPolicy();
ActionRequestValidationException validate();
enum RefreshPolicy implements Writeable {
/**
* Don't refresh after this request. The default.
*/
NONE,
/**
* Force a refresh as part of this request. This refresh policy does not scale for high indexing or search throughput but is useful
* to present a consistent view to for indices with very low traffic. And it is wonderful for tests!
*/
IMMEDIATE,
/**
* Leave this request open until a refresh has made the contents of this request visible to search. This refresh policy is
* compatible with high indexing and search throughput but it causes the request to wait to reply until a refresh occurs.
*/
WAIT_UNTIL;
/**
* Parse the string representation of a refresh policy, usually from a request parameter.
*/
public static RefreshPolicy parse(String string) {
switch (string) {
case "false":
return NONE;
/*
* Empty string is IMMEDIATE because that makes "POST /test/test/1?refresh" perform a refresh which reads well and is what folks
* are used to.
*/
case "":
case "true":
return IMMEDIATE;
case "wait_for":
return WAIT_UNTIL;
}
throw new IllegalArgumentException("Unknown value for refresh: [" + string + "].");
}
public static RefreshPolicy readFrom(StreamInput in) throws IOException {
return RefreshPolicy.values()[in.readByte()];
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeByte((byte) ordinal());
}
}
}

View file

@ -0,0 +1,50 @@
/*
* 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;
import org.elasticsearch.Version;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
public interface WriteRequestBuilder<B extends WriteRequestBuilder<B>> {
WriteRequest<?> request();
/**
* Should this request trigger a refresh ({@linkplain RefreshPolicy#IMMEDIATE}), wait for a refresh (
* {@linkplain RefreshPolicy#WAIT_UNTIL}), or proceed ignore refreshes entirely ({@linkplain RefreshPolicy#NONE}, the default).
*/
@SuppressWarnings("unchecked")
default B setRefreshPolicy(RefreshPolicy refreshPolicy) {
request().setRefreshPolicy(refreshPolicy);
return (B) this;
}
/**
* If set to true then this request will force an immediate refresh. Backwards compatibility layer for Elasticsearch's old
* {@code setRefresh} calls.
*
* @deprecated use {@link #setRefreshPolicy(RefreshPolicy)} with {@link RefreshPolicy#IMMEDIATE} or {@link RefreshPolicy#NONE} instead.
* Will be removed in 6.0.
*/
@Deprecated
default B setRefresh(boolean refresh) {
assert Version.CURRENT.major < 6 : "Remove setRefresh(boolean) in 6.0";
return setRefreshPolicy(refresh ? RefreshPolicy.IMMEDIATE : RefreshPolicy.NONE);
}
}

View file

@ -0,0 +1,40 @@
/*
* 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;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.bulk.BulkResponse;
import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.action.update.UpdateResponse;
import org.elasticsearch.index.IndexSettings;
/**
* Interface implemented by responses for actions that modify the documents in an index like {@link IndexResponse}, {@link UpdateResponse},
* and {@link BulkResponse}. Rather than implement this directly most implementers should extend {@link DocWriteResponse}.
*/
public interface WriteResponse {
/**
* Mark the response as having forced a refresh? Requests that set {@link WriteRequest#setRefreshPolicy(RefreshPolicy)} to
* {@link RefreshPolicy#IMMEDIATE} should always mark this as true. Requests that set it to {@link RefreshPolicy#WAIT_UNTIL} will only
* set this to true if they run out of refresh listener slots (see {@link IndexSettings#MAX_REFRESH_LISTENERS_PER_SHARD}).
*/
public abstract void setForcedRefresh(boolean forcedRefresh);
}

View file

@ -0,0 +1,72 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.action.support.replication;
import org.elasticsearch.action.bulk.BulkShardRequest;
import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.index.shard.ShardId;
import java.io.IOException;
/**
* Requests that are both {@linkplain ReplicationRequest}s (run on a shard's primary first, then the replica) and {@linkplain WriteRequest}
* (modify documents on a shard), for example {@link BulkShardRequest}, {@link IndexRequest}, and {@link DeleteRequest}.
*/
public abstract class ReplicatedWriteRequest<R extends ReplicatedWriteRequest<R>> extends ReplicationRequest<R> implements WriteRequest<R> {
private RefreshPolicy refreshPolicy = RefreshPolicy.NONE;
/**
* Constructor for deserialization.
*/
public ReplicatedWriteRequest() {
}
public ReplicatedWriteRequest(ShardId shardId) {
super(shardId);
}
@Override
@SuppressWarnings("unchecked")
public R setRefreshPolicy(RefreshPolicy refreshPolicy) {
this.refreshPolicy = refreshPolicy;
return (R) this;
}
@Override
public RefreshPolicy getRefreshPolicy() {
return refreshPolicy;
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
refreshPolicy = RefreshPolicy.readFrom(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
refreshPolicy.writeTo(out);
}
}

View file

@ -21,7 +21,6 @@ package org.elasticsearch.action.support.replication;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ReplicationResponse;
import org.elasticsearch.action.UnavailableShardsException; import org.elasticsearch.action.UnavailableShardsException;
import org.elasticsearch.action.WriteConsistencyLevel; import org.elasticsearch.action.WriteConsistencyLevel;
import org.elasticsearch.action.support.TransportActions; import org.elasticsearch.action.support.TransportActions;
@ -29,7 +28,6 @@ import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.cluster.routing.IndexRoutingTable;
import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.index.engine.VersionConflictEngineException; import org.elasticsearch.index.engine.VersionConflictEngineException;
@ -47,28 +45,41 @@ import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Consumer; import java.util.function.Consumer;
import java.util.function.Supplier; import java.util.function.Supplier;
public class ReplicationOperation<Request extends ReplicationRequest<Request>, ReplicaRequest extends ReplicationRequest<ReplicaRequest>, public class ReplicationOperation<
Response extends ReplicationResponse> { Request extends ReplicationRequest<Request>,
ReplicaRequest extends ReplicationRequest<ReplicaRequest>,
PrimaryResultT extends ReplicationOperation.PrimaryResult<ReplicaRequest>
> {
final private ESLogger logger; final private ESLogger logger;
final private Request request; final private Request request;
final private Supplier<ClusterState> clusterStateSupplier; final private Supplier<ClusterState> clusterStateSupplier;
final private String opType; final private String opType;
final private AtomicInteger totalShards = new AtomicInteger(); final private AtomicInteger totalShards = new AtomicInteger();
/**
* The number of pending sub-operations in this operation. This is incremented when the following operations start and decremented when
* they complete:
* <ul>
* <li>The operation on the primary</li>
* <li>The operation on each replica</li>
* <li>Coordination of the operation as a whole. This prevents the operation from terminating early if we haven't started any replica
* operations and the primary finishes.</li>
* </ul>
*/
final private AtomicInteger pendingShards = new AtomicInteger(); final private AtomicInteger pendingShards = new AtomicInteger();
final private AtomicInteger successfulShards = new AtomicInteger(); final private AtomicInteger successfulShards = new AtomicInteger();
final private boolean executeOnReplicas; final private boolean executeOnReplicas;
final private boolean checkWriteConsistency; final private boolean checkWriteConsistency;
final private Primary<Request, ReplicaRequest, Response> primary; final private Primary<Request, ReplicaRequest, PrimaryResultT> primary;
final private Replicas<ReplicaRequest> replicasProxy; final private Replicas<ReplicaRequest> replicasProxy;
final private AtomicBoolean finished = new AtomicBoolean(); final private AtomicBoolean finished = new AtomicBoolean();
final protected ActionListener<Response> finalResponseListener; final protected ActionListener<PrimaryResultT> resultListener;
private volatile Response finalResponse = null; private volatile PrimaryResultT primaryResult = null;
private final List<ReplicationResponse.ShardInfo.Failure> shardReplicaFailures = Collections.synchronizedList(new ArrayList<>()); private final List<ReplicationResponse.ShardInfo.Failure> shardReplicaFailures = Collections.synchronizedList(new ArrayList<>());
ReplicationOperation(Request request, Primary<Request, ReplicaRequest, Response> primary, ReplicationOperation(Request request, Primary<Request, ReplicaRequest, PrimaryResultT> primary,
ActionListener<Response> listener, ActionListener<PrimaryResultT> listener,
boolean executeOnReplicas, boolean checkWriteConsistency, boolean executeOnReplicas, boolean checkWriteConsistency,
Replicas<ReplicaRequest> replicas, Replicas<ReplicaRequest> replicas,
Supplier<ClusterState> clusterStateSupplier, ESLogger logger, String opType) { Supplier<ClusterState> clusterStateSupplier, ESLogger logger, String opType) {
@ -76,7 +87,7 @@ public class ReplicationOperation<Request extends ReplicationRequest<Request>, R
this.executeOnReplicas = executeOnReplicas; this.executeOnReplicas = executeOnReplicas;
this.replicasProxy = replicas; this.replicasProxy = replicas;
this.primary = primary; this.primary = primary;
this.finalResponseListener = listener; this.resultListener = listener;
this.logger = logger; this.logger = logger;
this.request = request; this.request = request;
this.clusterStateSupplier = clusterStateSupplier; this.clusterStateSupplier = clusterStateSupplier;
@ -85,28 +96,27 @@ public class ReplicationOperation<Request extends ReplicationRequest<Request>, R
void execute() throws Exception { void execute() throws Exception {
final String writeConsistencyFailure = checkWriteConsistency ? checkWriteConsistency() : null; final String writeConsistencyFailure = checkWriteConsistency ? checkWriteConsistency() : null;
final ShardId shardId = primary.routingEntry().shardId(); final ShardRouting primaryRouting = primary.routingEntry();
final ShardId primaryId = primaryRouting.shardId();
if (writeConsistencyFailure != null) { if (writeConsistencyFailure != null) {
finishAsFailed(new UnavailableShardsException(shardId, finishAsFailed(new UnavailableShardsException(primaryId,
"{} Timeout: [{}], request: [{}]", writeConsistencyFailure, request.timeout(), request)); "{} Timeout: [{}], request: [{}]", writeConsistencyFailure, request.timeout(), request));
return; return;
} }
totalShards.incrementAndGet(); totalShards.incrementAndGet();
pendingShards.incrementAndGet(); // increase by 1 until we finish all primary coordination pendingShards.incrementAndGet();
Tuple<Response, ReplicaRequest> primaryResponse = primary.perform(request); primaryResult = primary.perform(request);
successfulShards.incrementAndGet(); // mark primary as successful final ReplicaRequest replicaRequest = primaryResult.replicaRequest();
finalResponse = primaryResponse.v1();
ReplicaRequest replicaRequest = primaryResponse.v2();
assert replicaRequest.primaryTerm() > 0 : "replicaRequest doesn't have a primary term"; assert replicaRequest.primaryTerm() > 0 : "replicaRequest doesn't have a primary term";
if (logger.isTraceEnabled()) { if (logger.isTraceEnabled()) {
logger.trace("[{}] op [{}] completed on primary for request [{}]", shardId, opType, request); 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 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 // 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. // to the recovery target. If we use an old cluster state, we may miss a relocation that has started since then.
// If the index gets deleted after primary operation, we skip replication // If the index gets deleted after primary operation, we skip replication
List<ShardRouting> shards = getShards(shardId, clusterStateSupplier.get()); final List<ShardRouting> shards = getShards(primaryId, clusterStateSupplier.get());
final String localNodeId = primary.routingEntry().currentNodeId(); final String localNodeId = primary.routingEntry().currentNodeId();
for (final ShardRouting shard : shards) { for (final ShardRouting shard : shards) {
if (executeOnReplicas == false || shard.unassigned()) { if (executeOnReplicas == false || shard.unassigned()) {
@ -125,8 +135,8 @@ public class ReplicationOperation<Request extends ReplicationRequest<Request>, R
} }
} }
// decrement pending and finish (if there are no replicas, or those are done) successfulShards.incrementAndGet();
decPendingAndFinishIfNeeded(); // incremented in the beginning of this method decPendingAndFinishIfNeeded();
} }
private void performOnReplica(final ShardRouting shard, final ReplicaRequest replicaRequest) { private void performOnReplica(final ShardRouting shard, final ReplicaRequest replicaRequest) {
@ -241,19 +251,19 @@ public class ReplicationOperation<Request extends ReplicationRequest<Request>, R
failuresArray = new ReplicationResponse.ShardInfo.Failure[shardReplicaFailures.size()]; failuresArray = new ReplicationResponse.ShardInfo.Failure[shardReplicaFailures.size()];
shardReplicaFailures.toArray(failuresArray); shardReplicaFailures.toArray(failuresArray);
} }
finalResponse.setShardInfo(new ReplicationResponse.ShardInfo( primaryResult.setShardInfo(new ReplicationResponse.ShardInfo(
totalShards.get(), totalShards.get(),
successfulShards.get(), successfulShards.get(),
failuresArray failuresArray
) )
); );
finalResponseListener.onResponse(finalResponse); resultListener.onResponse(primaryResult);
} }
} }
private void finishAsFailed(Throwable throwable) { private void finishAsFailed(Throwable throwable) {
if (finished.compareAndSet(false, true)) { if (finished.compareAndSet(false, true)) {
finalResponseListener.onFailure(throwable); resultListener.onFailure(throwable);
} }
} }
@ -284,22 +294,31 @@ public class ReplicationOperation<Request extends ReplicationRequest<Request>, R
} }
interface Primary<Request extends ReplicationRequest<Request>, ReplicaRequest extends ReplicationRequest<ReplicaRequest>, interface Primary<
Response extends ReplicationResponse> { Request extends ReplicationRequest<Request>,
ReplicaRequest extends ReplicationRequest<ReplicaRequest>,
PrimaryResultT extends PrimaryResult<ReplicaRequest>
> {
/** routing entry for this primary */ /**
* routing entry for this primary
*/
ShardRouting routingEntry(); ShardRouting routingEntry();
/** fail the primary, typically due to the fact that the operation has learned the primary has been demoted by the master */ /**
* fail the primary, typically due to the fact that the operation has learned the primary has been demoted by the master
*/
void failShard(String message, Throwable throwable); void failShard(String message, Throwable throwable);
/** /**
* Performs the given request on this primary * Performs the given request on this primary. Yes, this returns as soon as it can with the request for the replicas and calls a
* listener when the primary request is completed. Yes, the primary request might complete before the method returns. Yes, it might
* also complete after. Deal with it.
* *
* @return A tuple containing not null values, as first value the result of the primary operation and as second value * @param request the request to perform
* the request to be executed on the replica shards. * @return the request to send to the repicas
*/ */
Tuple<Response, ReplicaRequest> perform(Request request) throws Exception; PrimaryResultT perform(Request request) throws Exception;
} }
@ -316,6 +335,7 @@ public class ReplicationOperation<Request extends ReplicationRequest<Request>, R
/** /**
* Fail the specified shard, removing it from the current set of active shards * Fail the specified shard, removing it from the current set of active shards
*
* @param replica shard to fail * @param replica shard to fail
* @param primary the primary shard that requested the failure * @param primary the primary shard that requested the failure
* @param message a (short) description of the reason * @param message a (short) description of the reason
@ -345,4 +365,11 @@ public class ReplicationOperation<Request extends ReplicationRequest<Request>, R
super(in); super(in);
} }
} }
interface PrimaryResult<R extends ReplicationRequest<R>> {
R replicaRequest();
void setShardInfo(ReplicationResponse.ShardInfo shardInfo);
}
} }

View file

@ -23,6 +23,8 @@ import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.IndicesRequest;
import org.elasticsearch.action.WriteConsistencyLevel; import org.elasticsearch.action.WriteConsistencyLevel;
import org.elasticsearch.action.admin.indices.refresh.TransportShardRefreshAction;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
@ -38,7 +40,8 @@ import java.util.concurrent.TimeUnit;
import static org.elasticsearch.action.ValidateActions.addValidationError; import static org.elasticsearch.action.ValidateActions.addValidationError;
/** /**
* * Requests that are run on a particular replica, first on the primary and then on the replicas like {@link IndexRequest} or
* {@link TransportShardRefreshAction}.
*/ */
public abstract class ReplicationRequest<Request extends ReplicationRequest<Request>> extends ActionRequest<Request> public abstract class ReplicationRequest<Request extends ReplicationRequest<Request>> extends ActionRequest<Request>
implements IndicesRequest { implements IndicesRequest {
@ -65,7 +68,6 @@ public abstract class ReplicationRequest<Request extends ReplicationRequest<Requ
} }
/** /**
* Creates a new request with resolved shard id * Creates a new request with resolved shard id
*/ */

View file

@ -17,10 +17,12 @@
* under the License. * under the License.
*/ */
package org.elasticsearch.action; package org.elasticsearch.action.support.replication;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
@ -79,14 +81,16 @@ public class ReplicationResponse extends ActionResponse {
} }
/** /**
* @return the total number of shards the write should go to (replicas and primaries). This includes relocating shards, so this number can be higher than the number of shards. * @return the total number of shards the write should go to (replicas and primaries). This includes relocating shards, so this
* number can be higher than the number of shards.
*/ */
public int getTotal() { public int getTotal() {
return total; return total;
} }
/** /**
* @return the total number of shards the write succeeded on (replicas and primaries). This includes relocating shards, so this number can be higher than the number of shards. * @return the total number of shards the write succeeded on (replicas and primaries). This includes relocating shards, so this
* number can be higher than the number of shards.
*/ */
public int getSuccessful() { public int getSuccessful() {
return successful; return successful;

View file

@ -22,7 +22,6 @@ package org.elasticsearch.action.support.replication;
import com.carrotsearch.hppc.cursors.IntObjectCursor; import com.carrotsearch.hppc.cursors.IntObjectCursor;
import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ReplicationResponse;
import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.DefaultShardOperationFailedException; import org.elasticsearch.action.support.DefaultShardOperationFailedException;

View file

@ -22,7 +22,6 @@ package org.elasticsearch.action.support.replication;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionListenerResponseHandler; import org.elasticsearch.action.ActionListenerResponseHandler;
import org.elasticsearch.action.ReplicationResponse;
import org.elasticsearch.action.UnavailableShardsException; import org.elasticsearch.action.UnavailableShardsException;
import org.elasticsearch.action.WriteConsistencyLevel; import org.elasticsearch.action.WriteConsistencyLevel;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
@ -41,7 +40,6 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
@ -53,7 +51,6 @@ import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.IndexShardState; import org.elasticsearch.index.shard.IndexShardState;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.node.NodeClosedException; import org.elasticsearch.node.NodeClosedException;
import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.Task;
@ -66,6 +63,7 @@ import org.elasticsearch.transport.TransportException;
import org.elasticsearch.transport.TransportRequestHandler; import org.elasticsearch.transport.TransportRequestHandler;
import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportRequestOptions;
import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportResponse;
import org.elasticsearch.transport.TransportResponse.Empty;
import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportService;
import java.io.IOException; import java.io.IOException;
@ -81,9 +79,11 @@ import java.util.function.Supplier;
* primary node to validate request before primary operation followed by sampling state again for resolving * primary node to validate request before primary operation followed by sampling state again for resolving
* nodes with replica copies to perform replication. * nodes with replica copies to perform replication.
*/ */
public abstract class TransportReplicationAction<Request extends ReplicationRequest<Request>, public abstract class TransportReplicationAction<
Request extends ReplicationRequest<Request>,
ReplicaRequest extends ReplicationRequest<ReplicaRequest>, ReplicaRequest extends ReplicationRequest<ReplicaRequest>,
Response extends ReplicationResponse> extends TransportAction<Request, Response> { Response extends ReplicationResponse
> extends TransportAction<Request, Response> {
final protected TransportService transportService; final protected TransportService transportService;
final protected ClusterService clusterService; final protected ClusterService clusterService;
@ -149,17 +149,17 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
} }
/** /**
* Primary operation on node with primary copy * Primary operation on node with primary copy.
* *
* @return A tuple containing not null values, as first value the result of the primary operation and as second value * @param shardRequest the request to the primary shard
* the request to be executed on the replica shards.
*/ */
protected abstract Tuple<Response, ReplicaRequest> shardOperationOnPrimary(Request shardRequest) throws Exception; protected abstract PrimaryResult shardOperationOnPrimary(Request shardRequest) throws Exception;
/** /**
* Replica operation on nodes with replica copies * Synchronous replica operation on nodes with replica copies. This is done under the lock form
* {@link #acquireReplicaOperationLock(ShardId, long)}.
*/ */
protected abstract void shardOperationOnReplica(ReplicaRequest shardRequest); protected abstract ReplicaResult shardOperationOnReplica(ReplicaRequest shardRequest);
/** /**
* True if write consistency should be checked for an implementation * True if write consistency should be checked for an implementation
@ -198,26 +198,6 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
|| TransportActions.isShardNotAvailableException(e); || TransportActions.isShardNotAvailableException(e);
} }
protected static class WriteResult<T extends ReplicationResponse> {
public final T response;
public final Translog.Location location;
public WriteResult(T response, Translog.Location location) {
this.response = response;
this.location = location;
}
@SuppressWarnings("unchecked")
public <T extends ReplicationResponse> T response() {
// this sets total, pending and failed to 0 and this is ok, because we will embed this into the replica
// request and not use it
response.setShardInfo(new ReplicationResponse.ShardInfo());
return (T) response;
}
}
class OperationTransportHandler implements TransportRequestHandler<Request> { class OperationTransportHandler implements TransportRequestHandler<Request> {
@Override @Override
public void messageReceived(final Request request, final TransportChannel channel, Task task) throws Exception { public void messageReceived(final Request request, final TransportChannel channel, Task task) throws Exception {
@ -289,7 +269,17 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
final IndexMetaData indexMetaData = clusterService.state().getMetaData().index(request.shardId().getIndex()); final IndexMetaData indexMetaData = clusterService.state().getMetaData().index(request.shardId().getIndex());
final boolean executeOnReplicas = (indexMetaData == null) || shouldExecuteReplication(indexMetaData.getSettings()); final boolean executeOnReplicas = (indexMetaData == null) || shouldExecuteReplication(indexMetaData.getSettings());
final ActionListener<Response> listener = createResponseListener(channel, replicationTask, primaryShardReference); final ActionListener<Response> listener = createResponseListener(channel, replicationTask, primaryShardReference);
createReplicatedOperation(request, listener, primaryShardReference, executeOnReplicas).execute(); createReplicatedOperation(request, new ActionListener<PrimaryResult>() {
@Override
public void onResponse(PrimaryResult result) {
result.respond(listener);
}
@Override
public void onFailure(Throwable e) {
listener.onFailure(e);
}
}, primaryShardReference, executeOnReplicas).execute();
success = true; success = true;
} }
} finally { } finally {
@ -299,8 +289,8 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
} }
} }
protected ReplicationOperation<Request, ReplicaRequest, Response> protected ReplicationOperation<Request, ReplicaRequest, PrimaryResult> createReplicatedOperation(
createReplicatedOperation(Request request, ActionListener<Response> listener, Request request, ActionListener<PrimaryResult> listener,
PrimaryShardReference primaryShardReference, boolean executeOnReplicas) { PrimaryShardReference primaryShardReference, boolean executeOnReplicas) {
return new ReplicationOperation<>(request, primaryShardReference, listener, return new ReplicationOperation<>(request, primaryShardReference, listener,
executeOnReplicas, checkWriteConsistency(), replicasProxy, clusterService::state, logger, actionName executeOnReplicas, checkWriteConsistency(), replicasProxy, clusterService::state, logger, actionName
@ -339,6 +329,41 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
} }
} }
protected class PrimaryResult implements ReplicationOperation.PrimaryResult<ReplicaRequest> {
final ReplicaRequest replicaRequest;
final Response finalResponse;
public PrimaryResult(ReplicaRequest replicaRequest, Response finalResponse) {
this.replicaRequest = replicaRequest;
this.finalResponse = finalResponse;
}
@Override
public ReplicaRequest replicaRequest() {
return replicaRequest;
}
@Override
public void setShardInfo(ReplicationResponse.ShardInfo shardInfo) {
finalResponse.setShardInfo(shardInfo);
}
public void respond(ActionListener<Response> listener) {
listener.onResponse(finalResponse);
}
}
protected class ReplicaResult {
/**
* Public constructor so subclasses can call it.
*/
public ReplicaResult() {}
public void respond(ActionListener<TransportResponse.Empty> listener) {
listener.onResponse(TransportResponse.Empty.INSTANCE);
}
}
class ReplicaOperationTransportHandler implements TransportRequestHandler<ReplicaRequest> { class ReplicaOperationTransportHandler implements TransportRequestHandler<ReplicaRequest> {
@Override @Override
public void messageReceived(final ReplicaRequest request, final TransportChannel channel) throws Exception { public void messageReceived(final ReplicaRequest request, final TransportChannel channel) throws Exception {
@ -426,15 +451,35 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
protected void doRun() throws Exception { protected void doRun() throws Exception {
setPhase(task, "replica"); setPhase(task, "replica");
assert request.shardId() != null : "request shardId must be set"; assert request.shardId() != null : "request shardId must be set";
ReplicaResult result;
try (Releasable ignored = acquireReplicaOperationLock(request.shardId(), request.primaryTerm())) { try (Releasable ignored = acquireReplicaOperationLock(request.shardId(), request.primaryTerm())) {
shardOperationOnReplica(request); result = shardOperationOnReplica(request);
}
result.respond(new ResponseListener());
}
/**
* Listens for the response on the replica and sends the response back to the primary.
*/
private class ResponseListener implements ActionListener<TransportResponse.Empty> {
@Override
public void onResponse(Empty response) {
if (logger.isTraceEnabled()) { if (logger.isTraceEnabled()) {
logger.trace("action [{}] completed on shard [{}] for request [{}]", transportReplicaAction, request.shardId(), logger.trace("action [{}] completed on shard [{}] for request [{}]", transportReplicaAction, request.shardId(),
request); request);
} }
}
setPhase(task, "finished"); setPhase(task, "finished");
channel.sendResponse(TransportResponse.Empty.INSTANCE); try {
channel.sendResponse(response);
} catch (Exception e) {
onFailure(e);
}
}
@Override
public void onFailure(Throwable e) {
responseWithFailure(e);
}
} }
} }
@ -722,7 +767,7 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
return IndexMetaData.isIndexUsingShadowReplicas(settings) == false; return IndexMetaData.isIndexUsingShadowReplicas(settings) == false;
} }
class PrimaryShardReference implements ReplicationOperation.Primary<Request, ReplicaRequest, Response>, Releasable { class PrimaryShardReference implements ReplicationOperation.Primary<Request, ReplicaRequest, PrimaryResult>, Releasable {
private final IndexShard indexShard; private final IndexShard indexShard;
private final Releasable operationLock; private final Releasable operationLock;
@ -751,9 +796,9 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
} }
@Override @Override
public Tuple<Response, ReplicaRequest> perform(Request request) throws Exception { public PrimaryResult perform(Request request) throws Exception {
Tuple<Response, ReplicaRequest> result = shardOperationOnPrimary(request); PrimaryResult result = shardOperationOnPrimary(request);
result.v2().primaryTerm(indexShard.getPrimaryTerm()); result.replicaRequest().primaryTerm(indexShard.getPrimaryTerm());
return result; return result;
} }
@ -805,20 +850,6 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
} }
} }
protected final void processAfterWrite(boolean refresh, IndexShard indexShard, Translog.Location location) {
if (refresh) {
try {
indexShard.refresh("refresh_flag_index");
} catch (Throwable e) {
// ignore
}
}
if (indexShard.getTranslogDurability() == Translog.Durability.REQUEST && location != null) {
indexShard.sync(location);
}
indexShard.maybeFlush();
}
/** /**
* Sets the current phase on the task if it isn't null. Pulled into its own * Sets the current phase on the task if it isn't null. Pulled into its own
* method because its more convenient that way. * method because its more convenient that way.

View file

@ -0,0 +1,227 @@
/*
* 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.action.ActionListener;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.action.support.WriteResponse;
import org.elasticsearch.cluster.action.shard.ShardStateAction;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexService;
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.indices.IndicesService;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportResponse;
import org.elasticsearch.transport.TransportService;
import java.util.function.Supplier;
/**
* Base class for transport actions that modify data in some shard like index, delete, and shardBulk.
*/
public abstract class TransportWriteAction<
Request extends ReplicatedWriteRequest<Request>,
Response extends ReplicationResponse & WriteResponse
> extends TransportReplicationAction<Request, Request, Response> {
protected TransportWriteAction(Settings settings, String actionName, TransportService transportService,
ClusterService clusterService, IndicesService indicesService, ThreadPool threadPool, ShardStateAction shardStateAction,
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, Supplier<Request> request,
String executor) {
super(settings, actionName, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters,
indexNameExpressionResolver, request, request, executor);
}
/**
* Called on the primary with a reference to the {@linkplain IndexShard} to modify.
*/
protected abstract WriteResult<Response> onPrimaryShard(Request request, IndexShard indexShard) throws Exception;
/**
* Called once per replica with a reference to the {@linkplain IndexShard} to modify.
*
* @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);
@Override
protected final WritePrimaryResult shardOperationOnPrimary(Request request) throws Exception {
IndexShard indexShard = indexShard(request);
WriteResult<Response> result = onPrimaryShard(request, indexShard);
return new WritePrimaryResult(request, result.getResponse(), result.getLocation(), indexShard);
}
@Override
protected final WriteReplicaResult shardOperationOnReplica(Request request) {
IndexShard indexShard = indexShard(request);
Translog.Location location = onReplicaShard(request, indexShard);
return new WriteReplicaResult(indexShard, request, location);
}
/**
* Fetch the IndexShard for the request. Protected so it can be mocked in tests.
*/
protected IndexShard indexShard(Request request) {
final ShardId shardId = request.shardId();
IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
return indexService.getShard(shardId.id());
}
/**
* Simple result from a write action. Write actions have static method to return these so they can integrate with bulk.
*/
public static class WriteResult<Response extends ReplicationResponse> {
private final Response response;
private final Translog.Location location;
public WriteResult(Response response, @Nullable Location location) {
this.response = response;
this.location = location;
}
public Response getResponse() {
return response;
}
public Translog.Location getLocation() {
return location;
}
}
/**
* Result of taking the action on the primary.
*/
class WritePrimaryResult extends PrimaryResult implements RespondingWriteResult {
boolean finishedAsyncActions;
ActionListener<Response> listener = null;
public WritePrimaryResult(Request request, Response finalResponse,
@Nullable Translog.Location location,
IndexShard indexShard) {
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.
*/
postWriteActions(indexShard, request, location, this, logger);
}
@Override
public synchronized void respond(ActionListener<Response> listener) {
this.listener = listener;
respondIfPossible();
}
/**
* Respond if the refresh has occurred and the listener is ready. Always called while synchronized on {@code this}.
*/
protected void respondIfPossible() {
if (finishedAsyncActions && listener != null) {
super.respond(listener);
}
}
@Override
public synchronized void respondAfterAsyncAction(boolean forcedRefresh) {
finalResponse.setForcedRefresh(forcedRefresh);
finishedAsyncActions = true;
respondIfPossible();
}
}
/**
* Result of taking the action on the replica.
*/
class WriteReplicaResult extends ReplicaResult implements RespondingWriteResult {
boolean finishedAsyncActions;
private ActionListener<TransportResponse.Empty> listener;
public WriteReplicaResult(IndexShard indexShard, ReplicatedWriteRequest<?> request, Translog.Location location) {
postWriteActions(indexShard, request, location, this, logger);
}
@Override
public void respond(ActionListener<TransportResponse.Empty> listener) {
this.listener = listener;
respondIfPossible();
}
/**
* Respond if the refresh has occurred and the listener is ready. Always called while synchronized on {@code this}.
*/
protected void respondIfPossible() {
if (finishedAsyncActions && listener != null) {
super.respond(listener);
}
}
@Override
public synchronized void respondAfterAsyncAction(boolean forcedRefresh) {
finishedAsyncActions = true;
respondIfPossible();
}
}
private interface RespondingWriteResult {
void respondAfterAsyncAction(boolean forcedRefresh);
}
static void postWriteActions(final IndexShard indexShard,
final WriteRequest<?> request,
@Nullable final Translog.Location location,
final RespondingWriteResult respond,
final ESLogger logger) {
boolean pendingOps = false;
boolean immediateRefresh = false;
switch (request.getRefreshPolicy()) {
case IMMEDIATE:
indexShard.refresh("refresh_flag_index");
immediateRefresh = true;
break;
case WAIT_UNTIL:
if (location != null) {
pendingOps = true;
indexShard.addRefreshListener(location, forcedRefresh -> {
logger.warn("block_until_refresh request ran out of slots and forced a refresh: [{}]", request);
respond.respondAfterAsyncAction(forcedRefresh);
});
}
break;
case NONE:
break;
}
boolean fsyncTranslog = indexShard.getTranslogDurability() == Translog.Durability.REQUEST && location != null;
if (fsyncTranslog) {
indexShard.sync(location);
}
indexShard.maybeFlush();
if (pendingOps == false) {
respond.respondAfterAsyncAction(immediateRefresh);
}
}
}

View file

@ -26,6 +26,7 @@ import org.elasticsearch.action.RoutingMissingException;
import org.elasticsearch.action.admin.indices.create.CreateIndexRequest; import org.elasticsearch.action.admin.indices.create.CreateIndexRequest;
import org.elasticsearch.action.admin.indices.create.CreateIndexResponse; import org.elasticsearch.action.admin.indices.create.CreateIndexResponse;
import org.elasticsearch.action.admin.indices.create.TransportCreateIndexAction; import org.elasticsearch.action.admin.indices.create.TransportCreateIndexAction;
import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.delete.DeleteResponse;
import org.elasticsearch.action.delete.TransportDeleteAction; import org.elasticsearch.action.delete.TransportDeleteAction;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
@ -187,6 +188,7 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
} else { } else {
update.setGetResult(null); update.setGetResult(null);
} }
update.setForcedRefresh(response.forcedRefresh());
listener.onResponse(update); listener.onResponse(update);
} }
@ -219,6 +221,7 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
public void onResponse(IndexResponse response) { public void onResponse(IndexResponse response) {
UpdateResponse update = new UpdateResponse(response.getShardInfo(), response.getShardId(), response.getType(), response.getId(), response.getVersion(), response.isCreated()); UpdateResponse update = new UpdateResponse(response.getShardInfo(), response.getShardId(), response.getType(), response.getId(), response.getVersion(), response.isCreated());
update.setGetResult(updateHelper.extractGetResult(request, request.concreteIndex(), response.getVersion(), result.updatedSourceAsMap(), result.updateSourceContentType(), indexSourceBytes)); update.setGetResult(updateHelper.extractGetResult(request, request.concreteIndex(), response.getVersion(), result.updatedSourceAsMap(), result.updateSourceContentType(), indexSourceBytes));
update.setForcedRefresh(response.forcedRefresh());
listener.onResponse(update); listener.onResponse(update);
} }
@ -241,11 +244,13 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
}); });
break; break;
case DELETE: case DELETE:
deleteAction.execute(result.action(), new ActionListener<DeleteResponse>() { DeleteRequest deleteRequest = result.action();
deleteAction.execute(deleteRequest, new ActionListener<DeleteResponse>() {
@Override @Override
public void onResponse(DeleteResponse response) { public void onResponse(DeleteResponse response) {
UpdateResponse update = new UpdateResponse(response.getShardInfo(), response.getShardId(), response.getType(), response.getId(), response.getVersion(), false); UpdateResponse update = new UpdateResponse(response.getShardInfo(), response.getShardId(), response.getType(), response.getId(), response.getVersion(), false);
update.setGetResult(updateHelper.extractGetResult(request, request.concreteIndex(), response.getVersion(), result.updatedSourceAsMap(), result.updateSourceContentType(), null)); update.setGetResult(updateHelper.extractGetResult(request, request.concreteIndex(), response.getVersion(), result.updatedSourceAsMap(), result.updateSourceContentType(), null));
update.setForcedRefresh(response.forcedRefresh());
listener.onResponse(update); listener.onResponse(update);
} }

View file

@ -131,7 +131,7 @@ public class UpdateHelper extends AbstractComponent {
// it has to be a "create!" // it has to be a "create!"
.create(true) .create(true)
.ttl(ttl) .ttl(ttl)
.refresh(request.refresh()) .setRefreshPolicy(request.getRefreshPolicy())
.routing(request.routing()) .routing(request.routing())
.parent(request.parent()) .parent(request.parent())
.consistencyLevel(request.consistencyLevel()); .consistencyLevel(request.consistencyLevel());
@ -229,12 +229,13 @@ public class UpdateHelper extends AbstractComponent {
.version(updateVersion).versionType(request.versionType()) .version(updateVersion).versionType(request.versionType())
.consistencyLevel(request.consistencyLevel()) .consistencyLevel(request.consistencyLevel())
.timestamp(timestamp).ttl(ttl) .timestamp(timestamp).ttl(ttl)
.refresh(request.refresh()); .setRefreshPolicy(request.getRefreshPolicy());
return new Result(indexRequest, Operation.INDEX, updatedSourceAsMap, updateSourceContentType); return new Result(indexRequest, Operation.INDEX, updatedSourceAsMap, updateSourceContentType);
} else if ("delete".equals(operation)) { } else if ("delete".equals(operation)) {
DeleteRequest deleteRequest = Requests.deleteRequest(request.index()).type(request.type()).id(request.id()).routing(routing).parent(parent) DeleteRequest deleteRequest = Requests.deleteRequest(request.index()).type(request.type()).id(request.id()).routing(routing).parent(parent)
.version(updateVersion).versionType(request.versionType()) .version(updateVersion).versionType(request.versionType())
.consistencyLevel(request.consistencyLevel()); .consistencyLevel(request.consistencyLevel())
.setRefreshPolicy(request.getRefreshPolicy());
return new Result(deleteRequest, Operation.DELETE, updatedSourceAsMap, updateSourceContentType); return new Result(deleteRequest, Operation.DELETE, updatedSourceAsMap, updateSourceContentType);
} else if ("none".equals(operation)) { } else if ("none".equals(operation)) {
UpdateResponse update = new UpdateResponse(shardId, getResult.getType(), getResult.getId(), getResult.getVersion(), false); UpdateResponse update = new UpdateResponse(shardId, getResult.getType(), getResult.getId(), getResult.getVersion(), false);

View file

@ -23,6 +23,7 @@ import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.DocumentRequest; import org.elasticsearch.action.DocumentRequest;
import org.elasticsearch.action.WriteConsistencyLevel; import org.elasticsearch.action.WriteConsistencyLevel;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.action.support.single.instance.InstanceShardOperationRequest; import org.elasticsearch.action.support.single.instance.InstanceShardOperationRequest;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.ParseFieldMatcher; import org.elasticsearch.common.ParseFieldMatcher;
@ -53,7 +54,8 @@ import static org.elasticsearch.action.ValidateActions.addValidationError;
/** /**
*/ */
public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest> implements DocumentRequest<UpdateRequest> { public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
implements DocumentRequest<UpdateRequest>, WriteRequest<UpdateRequest> {
private String type; private String type;
private String id; private String id;
@ -72,7 +74,7 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
private VersionType versionType = VersionType.INTERNAL; private VersionType versionType = VersionType.INTERNAL;
private int retryOnConflict = 0; private int retryOnConflict = 0;
private boolean refresh = false; private RefreshPolicy refreshPolicy = RefreshPolicy.NONE;
private WriteConsistencyLevel consistencyLevel = WriteConsistencyLevel.DEFAULT; private WriteConsistencyLevel consistencyLevel = WriteConsistencyLevel.DEFAULT;
@ -422,18 +424,15 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
return this.versionType; return this.versionType;
} }
/** @Override
* Should a refresh be executed post this update operation causing the operation to public UpdateRequest setRefreshPolicy(RefreshPolicy refreshPolicy) {
* be searchable. Note, heavy indexing should not set this to <tt>true</tt>. Defaults this.refreshPolicy = refreshPolicy;
* to <tt>false</tt>.
*/
public UpdateRequest refresh(boolean refresh) {
this.refresh = refresh;
return this; return this;
} }
public boolean refresh() { @Override
return this.refresh; public RefreshPolicy getRefreshPolicy() {
return refreshPolicy;
} }
public WriteConsistencyLevel consistencyLevel() { public WriteConsistencyLevel consistencyLevel() {
@ -730,7 +729,7 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
script = new Script(in); script = new Script(in);
} }
retryOnConflict = in.readVInt(); retryOnConflict = in.readVInt();
refresh = in.readBoolean(); refreshPolicy = RefreshPolicy.readFrom(in);
if (in.readBoolean()) { if (in.readBoolean()) {
doc = new IndexRequest(); doc = new IndexRequest();
doc.readFrom(in); doc.readFrom(in);
@ -767,7 +766,7 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
script.writeTo(out); script.writeTo(out);
} }
out.writeVInt(retryOnConflict); out.writeVInt(retryOnConflict);
out.writeBoolean(refresh); refreshPolicy.writeTo(out);
if (doc == null) { if (doc == null) {
out.writeBoolean(false); out.writeBoolean(false);
} else { } else {

View file

@ -21,6 +21,7 @@ package org.elasticsearch.action.update;
import org.elasticsearch.action.WriteConsistencyLevel; import org.elasticsearch.action.WriteConsistencyLevel;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.WriteRequestBuilder;
import org.elasticsearch.action.support.single.instance.InstanceShardOperationRequestBuilder; import org.elasticsearch.action.support.single.instance.InstanceShardOperationRequestBuilder;
import org.elasticsearch.client.ElasticsearchClient; import org.elasticsearch.client.ElasticsearchClient;
import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.bytes.BytesReference;
@ -32,9 +33,8 @@ import org.elasticsearch.script.Script;
import java.util.Map; import java.util.Map;
/** public class UpdateRequestBuilder extends InstanceShardOperationRequestBuilder<UpdateRequest, UpdateResponse, UpdateRequestBuilder>
*/ implements WriteRequestBuilder<UpdateRequestBuilder> {
public class UpdateRequestBuilder extends InstanceShardOperationRequestBuilder<UpdateRequest, UpdateResponse, UpdateRequestBuilder> {
public UpdateRequestBuilder(ElasticsearchClient client, UpdateAction action) { public UpdateRequestBuilder(ElasticsearchClient client, UpdateAction action) {
super(client, action, new UpdateRequest()); super(client, action, new UpdateRequest());
@ -121,17 +121,6 @@ public class UpdateRequestBuilder extends InstanceShardOperationRequestBuilder<U
return this; return this;
} }
/**
* Should a refresh be executed post this update operation causing the operation to
* be searchable. Note, heavy indexing should not set this to <tt>true</tt>. Defaults
* to <tt>false</tt>.
*/
public UpdateRequestBuilder setRefresh(boolean refresh) {
request.refresh(refresh);
return this;
}
/** /**
* Sets the consistency level of write. Defaults to {@link org.elasticsearch.action.WriteConsistencyLevel#DEFAULT} * Sets the consistency level of write. Defaults to {@link org.elasticsearch.action.WriteConsistencyLevel#DEFAULT}
*/ */

View file

@ -115,6 +115,7 @@ public final class IndexScopedSettings extends AbstractScopedSettings {
IndexSettings.QUERY_STRING_LENIENT_SETTING, IndexSettings.QUERY_STRING_LENIENT_SETTING,
IndexSettings.ALLOW_UNMAPPED, IndexSettings.ALLOW_UNMAPPED,
IndexSettings.INDEX_CHECK_ON_STARTUP, IndexSettings.INDEX_CHECK_ON_STARTUP,
IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD,
ShardsLimitAllocationDecider.INDEX_TOTAL_SHARDS_PER_NODE_SETTING, ShardsLimitAllocationDecider.INDEX_TOTAL_SHARDS_PER_NODE_SETTING,
IndexSettings.INDEX_GC_DELETES_SETTING, IndexSettings.INDEX_GC_DELETES_SETTING,
IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING, IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING,

View file

@ -115,6 +115,11 @@ public final class IndexSettings {
public static final Setting<TimeValue> INDEX_GC_DELETES_SETTING = public static final Setting<TimeValue> INDEX_GC_DELETES_SETTING =
Setting.timeSetting("index.gc_deletes", DEFAULT_GC_DELETES, new TimeValue(-1, TimeUnit.MILLISECONDS), Property.Dynamic, Setting.timeSetting("index.gc_deletes", DEFAULT_GC_DELETES, new TimeValue(-1, TimeUnit.MILLISECONDS), Property.Dynamic,
Property.IndexScope); Property.IndexScope);
/**
* The maximum number of refresh listeners allows on this shard.
*/
public static final Setting<Integer> MAX_REFRESH_LISTENERS_PER_SHARD = Setting.intSetting("index.max_refresh_listeners", 1000, 0,
Property.Dynamic, Property.IndexScope);
private final Index index; private final Index index;
private final Version version; private final Version version;
@ -145,6 +150,10 @@ public final class IndexSettings {
private volatile int maxResultWindow; private volatile int maxResultWindow;
private volatile int maxRescoreWindow; private volatile int maxRescoreWindow;
private volatile boolean TTLPurgeDisabled; private volatile boolean TTLPurgeDisabled;
/**
* The maximum number of refresh listeners allows on this shard.
*/
private volatile int maxRefreshListeners;
/** /**
* Returns the default search field for this index. * Returns the default search field for this index.
@ -229,6 +238,7 @@ public final class IndexSettings {
maxResultWindow = scopedSettings.get(MAX_RESULT_WINDOW_SETTING); maxResultWindow = scopedSettings.get(MAX_RESULT_WINDOW_SETTING);
maxRescoreWindow = scopedSettings.get(MAX_RESCORE_WINDOW_SETTING); maxRescoreWindow = scopedSettings.get(MAX_RESCORE_WINDOW_SETTING);
TTLPurgeDisabled = scopedSettings.get(INDEX_TTL_DISABLE_PURGE_SETTING); TTLPurgeDisabled = scopedSettings.get(INDEX_TTL_DISABLE_PURGE_SETTING);
maxRefreshListeners = scopedSettings.get(MAX_REFRESH_LISTENERS_PER_SHARD);
this.mergePolicyConfig = new MergePolicyConfig(logger, this); this.mergePolicyConfig = new MergePolicyConfig(logger, this);
assert indexNameMatcher.test(indexMetaData.getIndex().getName()); assert indexNameMatcher.test(indexMetaData.getIndex().getName());
@ -251,6 +261,7 @@ public final class IndexSettings {
scopedSettings.addSettingsUpdateConsumer(INDEX_GC_DELETES_SETTING, this::setGCDeletes); scopedSettings.addSettingsUpdateConsumer(INDEX_GC_DELETES_SETTING, this::setGCDeletes);
scopedSettings.addSettingsUpdateConsumer(INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING, this::setTranslogFlushThresholdSize); scopedSettings.addSettingsUpdateConsumer(INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING, this::setTranslogFlushThresholdSize);
scopedSettings.addSettingsUpdateConsumer(INDEX_REFRESH_INTERVAL_SETTING, this::setRefreshInterval); scopedSettings.addSettingsUpdateConsumer(INDEX_REFRESH_INTERVAL_SETTING, this::setRefreshInterval);
scopedSettings.addSettingsUpdateConsumer(MAX_REFRESH_LISTENERS_PER_SHARD, this::setMaxRefreshListeners);
} }
private void setTranslogFlushThresholdSize(ByteSizeValue byteSizeValue) { private void setTranslogFlushThresholdSize(ByteSizeValue byteSizeValue) {
@ -499,6 +510,16 @@ public final class IndexSettings {
return scopedSettings.get(setting); return scopedSettings.get(setting);
} }
/**
* The maximum number of refresh listeners allows on this shard.
*/
public int getMaxRefreshListeners() {
return maxRefreshListeners;
}
private void setMaxRefreshListeners(int maxRefreshListeners) {
this.maxRefreshListeners = maxRefreshListeners;
}
IndexScopedSettings getScopedSettings() { return scopedSettings;} IndexScopedSettings getScopedSettings() { return scopedSettings;}
} }

View file

@ -607,6 +607,7 @@ public abstract class Engine implements Closeable {
* Synchronously refreshes the engine for new search operations to reflect the latest * Synchronously refreshes the engine for new search operations to reflect the latest
* changes. * changes.
*/ */
@Nullable
public abstract void refresh(String source) throws EngineException; public abstract void refresh(String source) throws EngineException;
/** /**
@ -999,6 +1000,9 @@ public abstract class Engine implements Closeable {
public static final GetResult NOT_EXISTS = new GetResult(false, Versions.NOT_FOUND, null); public static final GetResult NOT_EXISTS = new GetResult(false, Versions.NOT_FOUND, null);
/**
* Build a realtime get result from the translog.
*/
public GetResult(boolean exists, long version, @Nullable Translog.Source source) { public GetResult(boolean exists, long version, @Nullable Translog.Source source) {
this.source = source; this.source = source;
this.exists = exists; this.exists = exists;
@ -1007,6 +1011,9 @@ public abstract class Engine implements Closeable {
this.searcher = null; this.searcher = null;
} }
/**
* Build a non-realtime get result from the searcher.
*/
public GetResult(Searcher searcher, Versions.DocIdAndVersion docIdAndVersion) { public GetResult(Searcher searcher, Versions.DocIdAndVersion docIdAndVersion) {
this.exists = true; this.exists = true;
this.source = null; this.source = null;

View file

@ -25,14 +25,15 @@ import org.apache.lucene.index.SnapshotDeletionPolicy;
import org.apache.lucene.search.QueryCache; import org.apache.lucene.search.QueryCache;
import org.apache.lucene.search.QueryCachingPolicy; import org.apache.lucene.search.QueryCachingPolicy;
import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.search.similarities.Similarity;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.codec.CodecService;
import org.elasticsearch.index.shard.RefreshListeners;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.TranslogRecoveryPerformer; import org.elasticsearch.index.shard.TranslogRecoveryPerformer;
import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.Store;
@ -40,8 +41,6 @@ import org.elasticsearch.index.translog.TranslogConfig;
import org.elasticsearch.indices.IndexingMemoryController; import org.elasticsearch.indices.IndexingMemoryController;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import java.util.function.Function;
/* /*
* Holds all the configuration that is used to create an {@link Engine}. * Holds all the configuration that is used to create an {@link Engine}.
* Once {@link Engine} has been created with this object, changes to this * Once {@link Engine} has been created with this object, changes to this
@ -66,6 +65,8 @@ public final class EngineConfig {
private final Engine.EventListener eventListener; private final Engine.EventListener eventListener;
private final QueryCache queryCache; private final QueryCache queryCache;
private final QueryCachingPolicy queryCachingPolicy; private final QueryCachingPolicy queryCachingPolicy;
@Nullable
private final RefreshListeners refreshListeners;
/** /**
* Index setting to change the low level lucene codec used for writing new segments. * Index setting to change the low level lucene codec used for writing new segments.
@ -99,7 +100,7 @@ public final class EngineConfig {
MergePolicy mergePolicy,Analyzer analyzer, MergePolicy mergePolicy,Analyzer analyzer,
Similarity similarity, CodecService codecService, Engine.EventListener eventListener, Similarity similarity, CodecService codecService, Engine.EventListener eventListener,
TranslogRecoveryPerformer translogRecoveryPerformer, QueryCache queryCache, QueryCachingPolicy queryCachingPolicy, TranslogRecoveryPerformer translogRecoveryPerformer, QueryCache queryCache, QueryCachingPolicy queryCachingPolicy,
TranslogConfig translogConfig, TimeValue flushMergesAfter) { TranslogConfig translogConfig, TimeValue flushMergesAfter, RefreshListeners refreshListeners) {
if (openMode == null) { if (openMode == null) {
throw new IllegalArgumentException("openMode must not be null"); throw new IllegalArgumentException("openMode must not be null");
} }
@ -125,6 +126,7 @@ public final class EngineConfig {
this.translogConfig = translogConfig; this.translogConfig = translogConfig;
this.flushMergesAfter = flushMergesAfter; this.flushMergesAfter = flushMergesAfter;
this.openMode = openMode; this.openMode = openMode;
this.refreshListeners = refreshListeners;
} }
/** /**
@ -303,4 +305,10 @@ public final class EngineConfig {
OPEN_INDEX_AND_TRANSLOG; OPEN_INDEX_AND_TRANSLOG;
} }
/**
* {@linkplain RefreshListeners} instance to configure.
*/
public RefreshListeners getRefreshListeners() {
return refreshListeners;
}
} }

View file

@ -154,6 +154,10 @@ public class InternalEngine extends Engine {
this.versionMap.setManager(searcherManager); this.versionMap.setManager(searcherManager);
// don't allow commits until we are done with recovering // don't allow commits until we are done with recovering
allowCommits.compareAndSet(true, openMode != EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG); allowCommits.compareAndSet(true, openMode != EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG);
if (engineConfig.getRefreshListeners() != null) {
searcherManager.addListener(engineConfig.getRefreshListeners());
engineConfig.getRefreshListeners().setTranslog(translog);
}
success = true; success = true;
} finally { } finally {
if (success == false) { if (success == false) {

View file

@ -30,7 +30,6 @@ import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.ReleasableLock; import org.elasticsearch.common.util.concurrent.ReleasableLock;
import org.elasticsearch.index.shard.TranslogRecoveryPerformer;
import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.Translog;
import java.io.IOException; import java.io.IOException;
@ -68,6 +67,9 @@ public class ShadowEngine extends Engine {
public ShadowEngine(EngineConfig engineConfig) { public ShadowEngine(EngineConfig engineConfig) {
super(engineConfig); super(engineConfig);
if (engineConfig.getRefreshListeners() != null) {
throw new IllegalArgumentException("ShadowEngine doesn't support RefreshListeners");
}
SearcherFactory searcherFactory = new EngineSearcherFactory(engineConfig); SearcherFactory searcherFactory = new EngineSearcherFactory(engineConfig);
final long nonexistentRetryTime = engineConfig.getIndexSettings().getSettings() final long nonexistentRetryTime = engineConfig.getIndexSettings().getSettings()
.getAsTime(NONEXISTENT_INDEX_RETRY_WAIT, DEFAULT_NONEXISTENT_INDEX_RETRY_WAIT) .getAsTime(NONEXISTENT_INDEX_RETRY_WAIT, DEFAULT_NONEXISTENT_INDEX_RETRY_WAIT)

View file

@ -131,6 +131,7 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Consumer;
import java.util.function.BiConsumer; import java.util.function.BiConsumer;
public class IndexShard extends AbstractIndexShardComponent { public class IndexShard extends AbstractIndexShardComponent {
@ -203,6 +204,12 @@ public class IndexShard extends AbstractIndexShardComponent {
* IndexingMemoryController}). * IndexingMemoryController}).
*/ */
private final AtomicBoolean active = new AtomicBoolean(); private final AtomicBoolean active = new AtomicBoolean();
/**
* Allows for the registration of listeners that are called when a change becomes visible for search. This is nullable because
* {@linkplain ShadowIndexShard} doesn't support this.
*/
@Nullable
private final RefreshListeners refreshListeners;
public IndexShard(ShardRouting shardRouting, IndexSettings indexSettings, ShardPath path, Store store, IndexCache indexCache, public IndexShard(ShardRouting shardRouting, IndexSettings indexSettings, ShardPath path, Store store, IndexCache indexCache,
MapperService mapperService, SimilarityService similarityService, IndexFieldDataService indexFieldDataService, MapperService mapperService, SimilarityService similarityService, IndexFieldDataService indexFieldDataService,
@ -255,6 +262,7 @@ public class IndexShard extends AbstractIndexShardComponent {
suspendableRefContainer = new SuspendableRefContainer(); suspendableRefContainer = new SuspendableRefContainer();
searcherWrapper = indexSearcherWrapper; searcherWrapper = indexSearcherWrapper;
primaryTerm = indexSettings.getIndexMetaData().primaryTerm(shardId.id()); primaryTerm = indexSettings.getIndexMetaData().primaryTerm(shardId.id());
refreshListeners = buildRefreshListeners();
persistMetadata(shardRouting, null); persistMetadata(shardRouting, null);
} }
@ -579,6 +587,7 @@ public class IndexShard extends AbstractIndexShardComponent {
*/ */
public void refresh(String source) { public void refresh(String source) {
verifyNotClosed(); verifyNotClosed();
if (canIndex()) { if (canIndex()) {
long bytes = getEngine().getIndexBufferRAMBytesUsed(); long bytes = getEngine().getIndexBufferRAMBytesUsed();
writingBytes.addAndGet(bytes); writingBytes.addAndGet(bytes);
@ -1530,7 +1539,7 @@ public class IndexShard extends AbstractIndexShardComponent {
return new EngineConfig(openMode, shardId, return new EngineConfig(openMode, shardId,
threadPool, indexSettings, warmer, store, deletionPolicy, indexSettings.getMergePolicy(), threadPool, indexSettings, warmer, store, deletionPolicy, indexSettings.getMergePolicy(),
mapperService.indexAnalyzer(), similarityService.similarity(mapperService), codecService, shardEventListener, translogRecoveryPerformer, indexCache.query(), cachingPolicy, translogConfig, mapperService.indexAnalyzer(), similarityService.similarity(mapperService), codecService, shardEventListener, translogRecoveryPerformer, indexCache.query(), cachingPolicy, translogConfig,
IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING.get(indexSettings.getSettings())); IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING.get(indexSettings.getSettings()), refreshListeners);
} }
public Releasable acquirePrimaryOperationLock() { public Releasable acquirePrimaryOperationLock() {
@ -1626,6 +1635,17 @@ public class IndexShard extends AbstractIndexShardComponent {
return false; return false;
} }
/**
* Build {@linkplain RefreshListeners} for this shard. Protected so {@linkplain ShadowIndexShard} can override it to return null.
*/
protected RefreshListeners buildRefreshListeners() {
return new RefreshListeners(
indexSettings::getMaxRefreshListeners,
() -> refresh("too_many_listeners"),
threadPool.executor(ThreadPool.Names.LISTENER)::execute,
logger);
}
/** /**
* Simple struct encapsulating a shard failure * Simple struct encapsulating a shard failure
* *
@ -1651,14 +1671,26 @@ public class IndexShard extends AbstractIndexShardComponent {
} }
/** /**
* Returns <code>true</code> iff one or more changes to the engine are not visible to via the current searcher. * Returns <code>true</code> iff one or more changes to the engine are not visible to via the current searcher *or* there are pending
* refresh listeners.
* Otherwise <code>false</code>. * Otherwise <code>false</code>.
* *
* @throws EngineClosedException if the engine is already closed * @throws EngineClosedException if the engine is already closed
* @throws AlreadyClosedException if the internal indexwriter in the engine is already closed * @throws AlreadyClosedException if the internal indexwriter in the engine is already closed
*/ */
public boolean isRefreshNeeded() { public boolean isRefreshNeeded() {
return getEngine().refreshNeeded(); return getEngine().refreshNeeded() || (refreshListeners != null && refreshListeners.refreshNeeded());
}
/**
* Add a listener for refreshes.
*
* @param location the location to listen for
* @param listener for the refresh. Called with true if registering the listener ran it out of slots and forced a refresh. Called with
* false otherwise.
*/
public void addRefreshListener(Translog.Location location, Consumer<Boolean> listener) {
refreshListeners.addOrNotify(location, listener);
} }
private class IndexShardRecoveryPerformer extends TranslogRecoveryPerformer { private class IndexShardRecoveryPerformer extends TranslogRecoveryPerformer {

View file

@ -0,0 +1,208 @@
/*
* 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.shard;
import org.apache.lucene.search.ReferenceManager;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.index.translog.Translog;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.Executor;
import java.util.function.Consumer;
import java.util.function.IntSupplier;
import static java.util.Objects.requireNonNull;
/**
* Allows for the registration of listeners that are called when a change becomes visible for search. This functionality is exposed from
* {@link IndexShard} but kept here so it can be tested without standing up the entire thing.
*/
public final class RefreshListeners implements ReferenceManager.RefreshListener {
private final IntSupplier getMaxRefreshListeners;
private final Runnable forceRefresh;
private final Executor listenerExecutor;
private final ESLogger logger;
/**
* List of refresh listeners. Defaults to null and built on demand because most refresh cycles won't need it. Entries are never removed
* from it, rather, it is nulled and rebuilt when needed again. The (hopefully) rare entries that didn't make the current refresh cycle
* are just added back to the new list. Both the reference and the contents are always modified while synchronized on {@code this}.
*/
private volatile List<Tuple<Translog.Location, Consumer<Boolean>>> refreshListeners = null;
/**
* The translog location that was last made visible by a refresh.
*/
private volatile Translog.Location lastRefreshedLocation;
public RefreshListeners(IntSupplier getMaxRefreshListeners, Runnable forceRefresh, Executor listenerExecutor, ESLogger logger) {
this.getMaxRefreshListeners = getMaxRefreshListeners;
this.forceRefresh = forceRefresh;
this.listenerExecutor = listenerExecutor;
this.logger = logger;
}
/**
* Add a listener for refreshes, calling it immediately if the location is already visible. If this runs out of listener slots then it
* forces a refresh and calls the listener immediately as well.
*
* @param location the location to listen for
* @param listener for the refresh. Called with true if registering the listener ran it out of slots and forced a refresh. Called with
* false otherwise.
*/
public void addOrNotify(Translog.Location location, Consumer<Boolean> listener) {
requireNonNull(listener, "listener cannot be null");
requireNonNull(location, "location cannot be null");
if (lastRefreshedLocation != null && lastRefreshedLocation.compareTo(location) >= 0) {
// Location already visible, just call the listener
listener.accept(false);
return;
}
synchronized (this) {
if (refreshListeners == null) {
refreshListeners = new ArrayList<>();
}
if (refreshListeners.size() < getMaxRefreshListeners.getAsInt()) {
// We have a free slot so register the listener
refreshListeners.add(new Tuple<>(location, listener));
return;
}
}
// No free slot so force a refresh and call the listener in this thread
forceRefresh.run();
listener.accept(true);
}
/**
* Returns true if there are pending listeners.
*/
public boolean refreshNeeded() {
// No need to synchronize here because we're doing a single volatile read
return refreshListeners != null;
}
/**
* Setup the translog used to find the last refreshed location.
*/
public void setTranslog(Translog translog) {
this.translog = translog;
}
// Implementation of ReferenceManager.RefreshListener that adapts Lucene's RefreshListener into Elasticsearch's refresh listeners.
private Translog translog;
/**
* Snapshot of the translog location before the current refresh if there is a refresh going on or null. Doesn't have to be volatile
* because when it is used by the refreshing thread.
*/
private Translog.Location currentRefreshLocation;
@Override
public void beforeRefresh() throws IOException {
currentRefreshLocation = translog.getLastWriteLocation();
}
@Override
public void afterRefresh(boolean didRefresh) throws IOException {
/*
* We intentionally ignore didRefresh here because our timing is a little off. It'd be a useful flag if we knew everything that made
* it into the refresh, but the way we snapshot the translog position before the refresh, things can sneak into the refresh that we
* don't know about.
*/
if (null == currentRefreshLocation) {
/*
* The translog had an empty last write location at the start of the refresh so we can't alert anyone to anything. This
* usually happens during recovery. The next refresh cycle out to pick up this refresh.
*/
return;
}
// First check if we've actually moved forward. If not then just bail immediately.
assert lastRefreshedLocation == null || currentRefreshLocation.compareTo(lastRefreshedLocation) >= 0;
if (lastRefreshedLocation != null && currentRefreshLocation.compareTo(lastRefreshedLocation) == 0) {
return;
}
/*
* Set the lastRefreshedLocation so listeners that come in for locations before that will just execute inline without messing
* around with refreshListeners or synchronizing at all.
*/
lastRefreshedLocation = currentRefreshLocation;
/*
* Grab the current refresh listeners and replace them with null while synchronized. Any listeners that come in after this won't be
* in the list we iterate over and very likely won't be candidates for refresh anyway because we've already moved the
* lastRefreshedLocation.
*/
List<Tuple<Translog.Location, Consumer<Boolean>>> candidates;
synchronized (this) {
candidates = refreshListeners;
// No listeners to check so just bail early
if (candidates == null) {
return;
}
refreshListeners = null;
}
// Iterate the list of listeners, copying the listeners to fire to one list and those to preserve to another list.
List<Consumer<Boolean>> listenersToFire = null;
List<Tuple<Translog.Location, Consumer<Boolean>>> preservedListeners = null;
for (Tuple<Translog.Location, Consumer<Boolean>> tuple : candidates) {
Translog.Location location = tuple.v1();
Consumer<Boolean> listener = tuple.v2();
if (location.compareTo(currentRefreshLocation) <= 0) {
if (listenersToFire == null) {
listenersToFire = new ArrayList<>();
}
listenersToFire.add(listener);
} else {
if (preservedListeners == null) {
preservedListeners = new ArrayList<>();
}
preservedListeners.add(tuple);
}
}
/*
* Now add any preserved listeners back to the running list of refresh listeners while under lock. We'll try them next time. While
* we were iterating the list of listeners new listeners could have come in. That means that adding all of our preserved listeners
* might push our list of listeners above the maximum number of slots allowed. This seems unlikely because we expect few listeners
* to be preserved. And the next listener while we're full will trigger a refresh anyway.
*/
if (preservedListeners != null) {
synchronized (this) {
if (refreshListeners == null) {
refreshListeners = new ArrayList<>();
}
refreshListeners.addAll(preservedListeners);
}
}
// Lastly, fire the listeners that are ready on the listener thread pool
if (listenersToFire != null) {
final List<Consumer<Boolean>> finalListenersToFire = listenersToFire;
listenerExecutor.execute(() -> {
for (Consumer<Boolean> listener : finalListenersToFire) {
try {
listener.accept(false);
} catch (Throwable t) {
logger.warn("Error firing refresh listener", t);
}
}
});
}
}
}

View file

@ -31,12 +31,14 @@ import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.merge.MergeStats; import org.elasticsearch.index.merge.MergeStats;
import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.TranslogStats; import org.elasticsearch.index.translog.TranslogStats;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import java.io.IOException; import java.io.IOException;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.function.Consumer;
/** /**
* ShadowIndexShard extends {@link IndexShard} to add file synchronization * ShadowIndexShard extends {@link IndexShard} to add file synchronization
@ -86,6 +88,12 @@ public final class ShadowIndexShard extends IndexShard {
return engineFactory.newReadOnlyEngine(config); return engineFactory.newReadOnlyEngine(config);
} }
@Override
protected RefreshListeners buildRefreshListeners() {
// ShadowEngine doesn't have a translog so it shouldn't try to support RefreshListeners.
return null;
}
@Override @Override
public boolean shouldFlush() { public boolean shouldFlush() {
// we don't need to flush since we don't write - all dominated by the primary // we don't need to flush since we don't write - all dominated by the primary
@ -96,4 +104,9 @@ public final class ShadowIndexShard extends IndexShard {
public TranslogStats translogStats() { public TranslogStats translogStats() {
return null; // shadow engine has no translog return null; // shadow engine has no translog
} }
@Override
public void addRefreshListener(Translog.Location location, Consumer<Boolean> listener) {
throw new UnsupportedOperationException("Can't listen for a refresh on a shadow engine because it doesn't have a translog");
}
} }

View file

@ -447,6 +447,21 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
} }
} }
/**
* The a {@linkplain Location} that will sort after the {@linkplain Location} returned by the last write but before any locations which
* can be returned by the next write.
*/
public Location getLastWriteLocation() {
try (ReleasableLock lock = readLock.acquire()) {
/*
* We use position = current - 1 and size = Integer.MAX_VALUE here instead of position current and size = 0 for two reasons:
* 1. Translog.Location's compareTo doesn't actually pay attention to size even though it's equals method does.
* 2. It feels more right to return a *position* that is before the next write's position rather than rely on the size.
*/
return new Location(current.generation, current.sizeInBytes() - 1, Integer.MAX_VALUE);
}
}
boolean assertBytesAtLocation(Translog.Location location, BytesReference expectedBytes) throws IOException { boolean assertBytesAtLocation(Translog.Location location, BytesReference expectedBytes) throws IOException {
// tests can override this // tests can override this
ByteBuffer buffer = ByteBuffer.allocate(location.size); ByteBuffer buffer = ByteBuffer.allocate(location.size);

View file

@ -26,6 +26,7 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.Channels; import org.elasticsearch.common.io.Channels;
import org.elasticsearch.common.logging.ESLoggerFactory;
import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;

View file

@ -84,7 +84,7 @@ public class RestBulkAction extends BaseRestHandler {
bulkRequest.consistencyLevel(WriteConsistencyLevel.fromString(consistencyLevel)); bulkRequest.consistencyLevel(WriteConsistencyLevel.fromString(consistencyLevel));
} }
bulkRequest.timeout(request.paramAsTime("timeout", BulkShardRequest.DEFAULT_TIMEOUT)); bulkRequest.timeout(request.paramAsTime("timeout", BulkShardRequest.DEFAULT_TIMEOUT));
bulkRequest.refresh(request.paramAsBoolean("refresh", bulkRequest.refresh())); bulkRequest.setRefreshPolicy(request.param("refresh"));
bulkRequest.add(request.content(), defaultIndex, defaultType, defaultRouting, defaultFields, defaultPipeline, null, allowExplicitIndex); bulkRequest.add(request.content(), defaultIndex, defaultType, defaultRouting, defaultFields, defaultPipeline, null, allowExplicitIndex);
client.bulk(bulkRequest, new RestBuilderListener<BulkResponse>(channel) { client.bulk(bulkRequest, new RestBuilderListener<BulkResponse>(channel) {

View file

@ -51,7 +51,7 @@ public class RestDeleteAction extends BaseRestHandler {
deleteRequest.routing(request.param("routing")); deleteRequest.routing(request.param("routing"));
deleteRequest.parent(request.param("parent")); // order is important, set it after routing, so it will set the routing deleteRequest.parent(request.param("parent")); // order is important, set it after routing, so it will set the routing
deleteRequest.timeout(request.paramAsTime("timeout", DeleteRequest.DEFAULT_TIMEOUT)); deleteRequest.timeout(request.paramAsTime("timeout", DeleteRequest.DEFAULT_TIMEOUT));
deleteRequest.refresh(request.paramAsBoolean("refresh", deleteRequest.refresh())); deleteRequest.setRefreshPolicy(request.param("refresh"));
deleteRequest.version(RestActions.parseVersion(request)); deleteRequest.version(RestActions.parseVersion(request));
deleteRequest.versionType(VersionType.fromString(request.param("version_type"), deleteRequest.versionType())); deleteRequest.versionType(VersionType.fromString(request.param("version_type"), deleteRequest.versionType()));

View file

@ -80,7 +80,7 @@ public class RestIndexAction extends BaseRestHandler {
indexRequest.setPipeline(request.param("pipeline")); indexRequest.setPipeline(request.param("pipeline"));
indexRequest.source(request.content()); indexRequest.source(request.content());
indexRequest.timeout(request.paramAsTime("timeout", IndexRequest.DEFAULT_TIMEOUT)); indexRequest.timeout(request.paramAsTime("timeout", IndexRequest.DEFAULT_TIMEOUT));
indexRequest.refresh(request.paramAsBoolean("refresh", indexRequest.refresh())); indexRequest.setRefreshPolicy(request.param("refresh"));
indexRequest.version(RestActions.parseVersion(request)); indexRequest.version(RestActions.parseVersion(request));
indexRequest.versionType(VersionType.fromString(request.param("version_type"), indexRequest.versionType())); indexRequest.versionType(VersionType.fromString(request.param("version_type"), indexRequest.versionType()));
String sOpType = request.param("op_type"); String sOpType = request.param("op_type");

View file

@ -58,7 +58,7 @@ public class RestUpdateAction extends BaseRestHandler {
updateRequest.routing(request.param("routing")); updateRequest.routing(request.param("routing"));
updateRequest.parent(request.param("parent")); updateRequest.parent(request.param("parent"));
updateRequest.timeout(request.paramAsTime("timeout", updateRequest.timeout())); updateRequest.timeout(request.paramAsTime("timeout", updateRequest.timeout()));
updateRequest.refresh(request.paramAsBoolean("refresh", updateRequest.refresh())); updateRequest.setRefreshPolicy(request.param("refresh"));
String consistencyLevel = request.param("consistency"); String consistencyLevel = request.param("consistency");
if (consistencyLevel != null) { if (consistencyLevel != null) {
updateRequest.consistencyLevel(WriteConsistencyLevel.fromString(consistencyLevel)); updateRequest.consistencyLevel(WriteConsistencyLevel.fromString(consistencyLevel));

View file

@ -24,6 +24,7 @@ import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.action.update.UpdateRequest;
import org.elasticsearch.client.Requests; import org.elasticsearch.client.Requests;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
@ -180,22 +181,22 @@ public class BulkRequestTests extends ESTestCase {
public void testBulkRequestWithRefresh() throws Exception { public void testBulkRequestWithRefresh() throws Exception {
BulkRequest bulkRequest = new BulkRequest(); BulkRequest bulkRequest = new BulkRequest();
// We force here a "id is missing" validation error // We force here a "id is missing" validation error
bulkRequest.add(new DeleteRequest("index", "type", null).refresh(true)); bulkRequest.add(new DeleteRequest("index", "type", null).setRefreshPolicy(RefreshPolicy.IMMEDIATE));
// We force here a "type is missing" validation error // We force here a "type is missing" validation error
bulkRequest.add(new DeleteRequest("index", null, "id")); bulkRequest.add(new DeleteRequest("index", null, "id"));
bulkRequest.add(new DeleteRequest("index", "type", "id").refresh(true)); bulkRequest.add(new DeleteRequest("index", "type", "id").setRefreshPolicy(RefreshPolicy.IMMEDIATE));
bulkRequest.add(new UpdateRequest("index", "type", "id").doc("{}").refresh(true)); bulkRequest.add(new UpdateRequest("index", "type", "id").doc("{}").setRefreshPolicy(RefreshPolicy.IMMEDIATE));
bulkRequest.add(new IndexRequest("index", "type", "id").source("{}").refresh(true)); bulkRequest.add(new IndexRequest("index", "type", "id").source("{}").setRefreshPolicy(RefreshPolicy.IMMEDIATE));
ActionRequestValidationException validate = bulkRequest.validate(); ActionRequestValidationException validate = bulkRequest.validate();
assertThat(validate, notNullValue()); assertThat(validate, notNullValue());
assertThat(validate.validationErrors(), not(empty())); assertThat(validate.validationErrors(), not(empty()));
assertThat(validate.validationErrors(), contains( assertThat(validate.validationErrors(), contains(
"Refresh is not supported on an item request, set the refresh flag on the BulkRequest instead.", "RefreshPolicy is not supported on an item request. Set it on the BulkRequest instead.",
"id is missing", "id is missing",
"type is missing", "type is missing",
"Refresh is not supported on an item request, set the refresh flag on the BulkRequest instead.", "RefreshPolicy is not supported on an item request. Set it on the BulkRequest instead.",
"Refresh is not supported on an item request, set the refresh flag on the BulkRequest instead.", "RefreshPolicy is not supported on an item request. Set it on the BulkRequest instead.",
"Refresh is not supported on an item request, set the refresh flag on the BulkRequest instead.")); "RefreshPolicy is not supported on an item request. Set it on the BulkRequest instead."));
} }
// issue 15120 // issue 15120

View file

@ -19,6 +19,7 @@
package org.elasticsearch.action.bulk; package org.elasticsearch.action.bulk;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
@ -28,9 +29,11 @@ public class BulkShardRequestTests extends ESTestCase {
public void testToString() { public void testToString() {
String index = randomSimpleString(random(), 10); String index = randomSimpleString(random(), 10);
int count = between(1, 100); int count = between(1, 100);
BulkShardRequest r = new BulkShardRequest(null, new ShardId(index, "ignored", 0), false, new BulkItemRequest[count]); BulkShardRequest r = new BulkShardRequest(null, new ShardId(index, "ignored", 0), RefreshPolicy.NONE, new BulkItemRequest[count]);
assertEquals("BulkShardRequest to [" + index + "] containing [" + count + "] requests", r.toString()); assertEquals("BulkShardRequest to [" + index + "] containing [" + count + "] requests", r.toString());
r = new BulkShardRequest(null, new ShardId(index, "ignored", 0), true, new BulkItemRequest[count]); r = new BulkShardRequest(null, new ShardId(index, "ignored", 0), RefreshPolicy.IMMEDIATE, new BulkItemRequest[count]);
assertEquals("BulkShardRequest to [" + index + "] containing [" + count + "] requests and a refresh", r.toString()); assertEquals("BulkShardRequest to [" + index + "] containing [" + count + "] requests and a refresh", r.toString());
r = new BulkShardRequest(null, new ShardId(index, "ignored", 0), RefreshPolicy.WAIT_UNTIL, new BulkItemRequest[count]);
assertEquals("BulkShardRequest to [" + index + "] containing [" + count + "] requests blocking until refresh", r.toString());
} }
} }

View file

@ -21,7 +21,6 @@ package org.elasticsearch.action.support.replication;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.NoShardAvailableActionException; import org.elasticsearch.action.NoShardAvailableActionException;
import org.elasticsearch.action.ReplicationResponse;
import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.UnavailableShardsException; import org.elasticsearch.action.UnavailableShardsException;
import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.flush.FlushRequest;

View file

@ -21,16 +21,15 @@ package org.elasticsearch.action.support.replication;
import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.CorruptIndexException;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ReplicationResponse;
import org.elasticsearch.action.UnavailableShardsException; import org.elasticsearch.action.UnavailableShardsException;
import org.elasticsearch.action.WriteConsistencyLevel; import org.elasticsearch.action.WriteConsistencyLevel;
import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.action.support.replication.ReplicationResponse.ShardInfo;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.logging.ESLogger;
@ -102,7 +101,7 @@ public class ReplicationOperationTests extends ESTestCase {
} }
Request request = new Request(shardId); Request request = new Request(shardId);
PlainActionFuture<Response> listener = new PlainActionFuture<>(); PlainActionFuture<TestPrimary.Result> listener = new PlainActionFuture<>();
final ClusterState finalState = state; final ClusterState finalState = state;
final TestReplicaProxy replicasProxy = new TestReplicaProxy(expectedFailures); final TestReplicaProxy replicasProxy = new TestReplicaProxy(expectedFailures);
final TestReplicationOperation op = new TestReplicationOperation(request, final TestReplicationOperation op = new TestReplicationOperation(request,
@ -114,7 +113,7 @@ public class ReplicationOperationTests extends ESTestCase {
assertThat(request.processedOnReplicas, equalTo(expectedReplicas)); assertThat(request.processedOnReplicas, equalTo(expectedReplicas));
assertThat(replicasProxy.failedReplicas, equalTo(expectedFailedShards)); assertThat(replicasProxy.failedReplicas, equalTo(expectedFailedShards));
assertTrue("listener is not marked as done", listener.isDone()); assertTrue("listener is not marked as done", listener.isDone());
Response.ShardInfo shardInfo = listener.actionGet().getShardInfo(); ShardInfo shardInfo = listener.actionGet().getShardInfo();
assertThat(shardInfo.getFailed(), equalTo(expectedFailedShards.size())); assertThat(shardInfo.getFailed(), equalTo(expectedFailedShards.size()));
assertThat(shardInfo.getFailures(), arrayWithSize(expectedFailedShards.size())); assertThat(shardInfo.getFailures(), arrayWithSize(expectedFailedShards.size()));
assertThat(shardInfo.getSuccessful(), equalTo(1 + expectedReplicas.size() - expectedFailures.size())); assertThat(shardInfo.getSuccessful(), equalTo(1 + expectedReplicas.size() - expectedFailures.size()));
@ -135,7 +134,7 @@ public class ReplicationOperationTests extends ESTestCase {
final ShardRouting primaryShard = indexShardRoutingTable.primaryShard(); final ShardRouting primaryShard = indexShardRoutingTable.primaryShard();
Request request = new Request(shardId); Request request = new Request(shardId);
PlainActionFuture<Response> listener = new PlainActionFuture<>(); PlainActionFuture<TestPrimary.Result> listener = new PlainActionFuture<>();
final TestReplicationOperation op = new TestReplicationOperation(request, final TestReplicationOperation op = new TestReplicationOperation(request,
new TestPrimary(primaryShard, primaryTerm), listener, false, false, new TestPrimary(primaryShard, primaryTerm), listener, false, false,
new TestReplicaProxy(), () -> state, logger, "test"); new TestReplicaProxy(), () -> state, logger, "test");
@ -143,7 +142,7 @@ public class ReplicationOperationTests extends ESTestCase {
assertThat("request was not processed on primary", request.processedOnPrimary.get(), equalTo(true)); assertThat("request was not processed on primary", request.processedOnPrimary.get(), equalTo(true));
assertThat(request.processedOnReplicas, equalTo(Collections.emptySet())); assertThat(request.processedOnReplicas, equalTo(Collections.emptySet()));
assertTrue("listener is not marked as done", listener.isDone()); assertTrue("listener is not marked as done", listener.isDone());
Response.ShardInfo shardInfo = listener.actionGet().getShardInfo(); ShardInfo shardInfo = listener.actionGet().getShardInfo();
assertThat(shardInfo.getFailed(), equalTo(0)); assertThat(shardInfo.getFailed(), equalTo(0));
assertThat(shardInfo.getFailures(), arrayWithSize(0)); assertThat(shardInfo.getFailures(), arrayWithSize(0));
assertThat(shardInfo.getSuccessful(), equalTo(1)); assertThat(shardInfo.getSuccessful(), equalTo(1));
@ -172,7 +171,7 @@ public class ReplicationOperationTests extends ESTestCase {
expectedFailures.put(failedReplica, new CorruptIndexException("simulated", (String) null)); expectedFailures.put(failedReplica, new CorruptIndexException("simulated", (String) null));
Request request = new Request(shardId); Request request = new Request(shardId);
PlainActionFuture<Response> listener = new PlainActionFuture<>(); PlainActionFuture<TestPrimary.Result> listener = new PlainActionFuture<>();
final ClusterState finalState = state; final ClusterState finalState = state;
final TestReplicaProxy replicasProxy = new TestReplicaProxy(expectedFailures) { final TestReplicaProxy replicasProxy = new TestReplicaProxy(expectedFailures) {
@Override @Override
@ -233,16 +232,16 @@ public class ReplicationOperationTests extends ESTestCase {
final ShardRouting primaryShard = state.get().routingTable().shardRoutingTable(shardId).primaryShard(); final ShardRouting primaryShard = state.get().routingTable().shardRoutingTable(shardId).primaryShard();
final TestPrimary primary = new TestPrimary(primaryShard, primaryTerm) { final TestPrimary primary = new TestPrimary(primaryShard, primaryTerm) {
@Override @Override
public Tuple<Response, Request> perform(Request request) throws Exception { public Result perform(Request request) throws Exception {
final Tuple<Response, Request> tuple = super.perform(request); Result result = super.perform(request);
state.set(changedState); state.set(changedState);
logger.debug("--> state after primary operation:\n{}", state.get().prettyPrint()); logger.debug("--> state after primary operation:\n{}", state.get().prettyPrint());
return tuple; return result;
} }
}; };
Request request = new Request(shardId); Request request = new Request(shardId);
PlainActionFuture<Response> listener = new PlainActionFuture<>(); PlainActionFuture<TestPrimary.Result> listener = new PlainActionFuture<>();
final TestReplicationOperation op = new TestReplicationOperation(request, primary, listener, final TestReplicationOperation op = new TestReplicationOperation(request, primary, listener,
new TestReplicaProxy(), state::get); new TestReplicaProxy(), state::get);
op.execute(); op.execute();
@ -296,7 +295,7 @@ public class ReplicationOperationTests extends ESTestCase {
state.prettyPrint()); state.prettyPrint());
final long primaryTerm = state.metaData().index(index).primaryTerm(shardId.id()); final long primaryTerm = state.metaData().index(index).primaryTerm(shardId.id());
final IndexShardRoutingTable shardRoutingTable = state.routingTable().index(index).shard(shardId.id()); final IndexShardRoutingTable shardRoutingTable = state.routingTable().index(index).shard(shardId.id());
PlainActionFuture<Response> listener = new PlainActionFuture<>(); PlainActionFuture<TestPrimary.Result> listener = new PlainActionFuture<>();
final ShardRouting primaryShard = shardRoutingTable.primaryShard(); final ShardRouting primaryShard = shardRoutingTable.primaryShard();
final TestReplicationOperation op = new TestReplicationOperation(request, final TestReplicationOperation op = new TestReplicationOperation(request,
new TestPrimary(primaryShard, primaryTerm), new TestPrimary(primaryShard, primaryTerm),
@ -362,10 +361,7 @@ public class ReplicationOperationTests extends ESTestCase {
} }
} }
static class Response extends ReplicationResponse { static class TestPrimary implements ReplicationOperation.Primary<Request, Request, TestPrimary.Result> {
}
static class TestPrimary implements ReplicationOperation.Primary<Request, Request, Response> {
final ShardRouting routing; final ShardRouting routing;
final long term; final long term;
@ -385,12 +381,35 @@ public class ReplicationOperationTests extends ESTestCase {
} }
@Override @Override
public Tuple<Response, Request> perform(Request request) throws Exception { public Result perform(Request request) throws Exception {
if (request.processedOnPrimary.compareAndSet(false, true) == false) { if (request.processedOnPrimary.compareAndSet(false, true) == false) {
fail("processed [" + request + "] twice"); fail("processed [" + request + "] twice");
} }
request.primaryTerm(term); request.primaryTerm(term);
return new Tuple<>(new Response(), request); return new Result(request);
}
static class Result implements ReplicationOperation.PrimaryResult<Request> {
private final Request request;
private ShardInfo shardInfo;
public Result(Request request) {
this.request = request;
}
@Override
public Request replicaRequest() {
return request;
}
@Override
public void setShardInfo(ShardInfo shardInfo) {
this.shardInfo = shardInfo;
}
public ShardInfo getShardInfo() {
return shardInfo;
}
} }
} }
@ -436,15 +455,15 @@ public class ReplicationOperationTests extends ESTestCase {
} }
} }
class TestReplicationOperation extends ReplicationOperation<Request, Request, Response> { class TestReplicationOperation extends ReplicationOperation<Request, Request, TestPrimary.Result> {
public TestReplicationOperation(Request request, Primary<Request, Request, Response> primary, ActionListener<Response> listener, public TestReplicationOperation(Request request, Primary<Request, Request, TestPrimary.Result> primary,
Replicas<Request> replicas, Supplier<ClusterState> clusterStateSupplier) { ActionListener<TestPrimary.Result> listener, Replicas<Request> replicas, Supplier<ClusterState> clusterStateSupplier) {
this(request, primary, listener, true, false, replicas, clusterStateSupplier, ReplicationOperationTests.this.logger, "test"); this(request, primary, listener, true, false, replicas, clusterStateSupplier, ReplicationOperationTests.this.logger, "test");
} }
public TestReplicationOperation(Request request, Primary<Request, Request, Response> primary, ActionListener<Response> listener, public TestReplicationOperation(Request request, Primary<Request, Request, TestPrimary.Result> primary,
boolean executeOnReplicas, boolean checkWriteConsistency, Replicas<Request> replicas, ActionListener<TestPrimary.Result> listener, boolean executeOnReplicas, boolean checkWriteConsistency,
Supplier<ClusterState> clusterStateSupplier, ESLogger logger, String opType) { Replicas<Request> replicas, Supplier<ClusterState> clusterStateSupplier, ESLogger logger, String opType) {
super(request, primary, listener, executeOnReplicas, checkWriteConsistency, replicas, clusterStateSupplier, logger, opType); super(request, primary, listener, executeOnReplicas, checkWriteConsistency, replicas, clusterStateSupplier, logger, opType);
} }
} }

View file

@ -20,7 +20,6 @@ package org.elasticsearch.action.support.replication;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ReplicationResponse;
import org.elasticsearch.action.UnavailableShardsException; import org.elasticsearch.action.UnavailableShardsException;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.PlainActionFuture;
@ -43,7 +42,6 @@ import org.elasticsearch.cluster.routing.allocation.AllocationService;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasable;
@ -155,7 +153,7 @@ public class TransportReplicationActionTests extends ESTestCase {
ClusterBlocks.Builder block = ClusterBlocks.builder() ClusterBlocks.Builder block = ClusterBlocks.builder()
.addGlobalBlock(new ClusterBlock(1, "non retryable", false, true, RestStatus.SERVICE_UNAVAILABLE, ClusterBlockLevel.ALL)); .addGlobalBlock(new ClusterBlock(1, "non retryable", false, true, RestStatus.SERVICE_UNAVAILABLE, ClusterBlockLevel.ALL));
setState(clusterService, ClusterState.builder(clusterService.state()).blocks(block)); setState(clusterService, ClusterState.builder(clusterService.state()).blocks(block));
TransportReplicationAction.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener); Action.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
reroutePhase.run(); reroutePhase.run();
assertListenerThrows("primary phase should fail operation", listener, ClusterBlockException.class); assertListenerThrows("primary phase should fail operation", listener, ClusterBlockException.class);
assertPhase(task, "failed"); assertPhase(task, "failed");
@ -199,7 +197,7 @@ public class TransportReplicationActionTests extends ESTestCase {
Request request = new Request(shardId).timeout("1ms"); Request request = new Request(shardId).timeout("1ms");
PlainActionFuture<Response> listener = new PlainActionFuture<>(); PlainActionFuture<Response> listener = new PlainActionFuture<>();
TransportReplicationAction.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener); Action.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
reroutePhase.run(); reroutePhase.run();
assertListenerThrows("unassigned primary didn't cause a timeout", listener, UnavailableShardsException.class); assertListenerThrows("unassigned primary didn't cause a timeout", listener, UnavailableShardsException.class);
assertPhase(task, "failed"); assertPhase(task, "failed");
@ -245,7 +243,7 @@ public class TransportReplicationActionTests extends ESTestCase {
Request request = new Request(shardId).timeout("1ms").routedBasedOnClusterVersion(clusterService.state().version() + 1); Request request = new Request(shardId).timeout("1ms").routedBasedOnClusterVersion(clusterService.state().version() + 1);
PlainActionFuture<Response> listener = new PlainActionFuture<>(); PlainActionFuture<Response> listener = new PlainActionFuture<>();
TransportReplicationAction.ReroutePhase reroutePhase = action.new ReroutePhase(null, request, listener); Action.ReroutePhase reroutePhase = action.new ReroutePhase(null, request, listener);
reroutePhase.run(); reroutePhase.run();
assertListenerThrows("cluster state too old didn't cause a timeout", listener, UnavailableShardsException.class); assertListenerThrows("cluster state too old didn't cause a timeout", listener, UnavailableShardsException.class);
@ -285,7 +283,7 @@ public class TransportReplicationActionTests extends ESTestCase {
PlainActionFuture<Response> listener = new PlainActionFuture<>(); PlainActionFuture<Response> listener = new PlainActionFuture<>();
ReplicationTask task = maybeTask(); ReplicationTask task = maybeTask();
TransportReplicationAction.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener); Action.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
reroutePhase.run(); reroutePhase.run();
assertListenerThrows("must throw index not found exception", listener, IndexNotFoundException.class); assertListenerThrows("must throw index not found exception", listener, IndexNotFoundException.class);
assertPhase(task, "failed"); assertPhase(task, "failed");
@ -312,7 +310,7 @@ public class TransportReplicationActionTests extends ESTestCase {
PlainActionFuture<Response> listener = new PlainActionFuture<>(); PlainActionFuture<Response> listener = new PlainActionFuture<>();
ReplicationTask task = maybeTask(); ReplicationTask task = maybeTask();
TransportReplicationAction.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener); Action.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
reroutePhase.run(); reroutePhase.run();
CapturingTransport.CapturedRequest[] capturedRequests = transport.getCapturedRequestsAndClear(); CapturingTransport.CapturedRequest[] capturedRequests = transport.getCapturedRequestsAndClear();
assertThat(capturedRequests, arrayWithSize(1)); assertThat(capturedRequests, arrayWithSize(1));
@ -364,7 +362,7 @@ public class TransportReplicationActionTests extends ESTestCase {
Request request = new Request(shardId); Request request = new Request(shardId);
PlainActionFuture<Response> listener = new PlainActionFuture<>(); PlainActionFuture<Response> listener = new PlainActionFuture<>();
TransportReplicationAction.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener); Action.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
reroutePhase.run(); reroutePhase.run();
assertThat(request.shardId(), equalTo(shardId)); assertThat(request.shardId(), equalTo(shardId));
logger.info("--> primary is assigned to [{}], checking request forwarded", primaryNodeId); logger.info("--> primary is assigned to [{}], checking request forwarded", primaryNodeId);
@ -393,8 +391,8 @@ public class TransportReplicationActionTests extends ESTestCase {
AtomicBoolean executed = new AtomicBoolean(); AtomicBoolean executed = new AtomicBoolean();
Action.PrimaryOperationTransportHandler primaryPhase = action.new PrimaryOperationTransportHandler() { Action.PrimaryOperationTransportHandler primaryPhase = action.new PrimaryOperationTransportHandler() {
@Override @Override
protected ReplicationOperation createReplicatedOperation(Request request, ActionListener<Response> actionListener, protected ReplicationOperation<Request, Request, Action.PrimaryResult> createReplicatedOperation(Request request,
Action.PrimaryShardReference primaryShardReference, ActionListener<Action.PrimaryResult> actionListener, Action.PrimaryShardReference primaryShardReference,
boolean executeOnReplicas) { boolean executeOnReplicas) {
return new NoopReplicationOperation(request, actionListener) { return new NoopReplicationOperation(request, actionListener) {
public void execute() throws Exception { public void execute() throws Exception {
@ -448,8 +446,8 @@ public class TransportReplicationActionTests extends ESTestCase {
AtomicBoolean executed = new AtomicBoolean(); AtomicBoolean executed = new AtomicBoolean();
Action.PrimaryOperationTransportHandler primaryPhase = action.new PrimaryOperationTransportHandler() { Action.PrimaryOperationTransportHandler primaryPhase = action.new PrimaryOperationTransportHandler() {
@Override @Override
protected ReplicationOperation createReplicatedOperation(Request request, ActionListener<Response> actionListener, protected ReplicationOperation<Request, Request, Action.PrimaryResult> createReplicatedOperation(Request request,
Action.PrimaryShardReference primaryShardReference, ActionListener<Action.PrimaryResult> actionListener, Action.PrimaryShardReference primaryShardReference,
boolean executeOnReplicas) { boolean executeOnReplicas) {
return new NoopReplicationOperation(request, actionListener) { return new NoopReplicationOperation(request, actionListener) {
public void execute() throws Exception { public void execute() throws Exception {
@ -478,9 +476,9 @@ public class TransportReplicationActionTests extends ESTestCase {
}; };
Action.PrimaryShardReference primary = action.new PrimaryShardReference(shard, releasable); Action.PrimaryShardReference primary = action.new PrimaryShardReference(shard, releasable);
final Request request = new Request(); final Request request = new Request();
Tuple<Response, Request> result = primary.perform(request); Request replicaRequest = primary.perform(request).replicaRequest;
assertThat(result.v2().primaryTerm(), equalTo(primaryTerm)); assertThat(replicaRequest.primaryTerm(), equalTo(primaryTerm));
final ElasticsearchException exception = new ElasticsearchException("testing"); final ElasticsearchException exception = new ElasticsearchException("testing");
primary.failShard("test", exception); primary.failShard("test", exception);
@ -582,8 +580,8 @@ public class TransportReplicationActionTests extends ESTestCase {
setState(clusterService, state); setState(clusterService, state);
Action.PrimaryOperationTransportHandler primaryPhase = action.new PrimaryOperationTransportHandler() { Action.PrimaryOperationTransportHandler primaryPhase = action.new PrimaryOperationTransportHandler() {
@Override @Override
protected ReplicationOperation createReplicatedOperation(Request request, ActionListener<Response> actionListener, protected ReplicationOperation<Request, Request, Action.PrimaryResult> createReplicatedOperation(Request request,
Action.PrimaryShardReference primaryShardReference, ActionListener<Action.PrimaryResult> actionListener, Action.PrimaryShardReference primaryShardReference,
boolean executeOnReplicas) { boolean executeOnReplicas) {
assertFalse(executeOnReplicas); assertFalse(executeOnReplicas);
return new NoopReplicationOperation(request, actionListener); return new NoopReplicationOperation(request, actionListener);
@ -608,8 +606,8 @@ public class TransportReplicationActionTests extends ESTestCase {
Action.PrimaryOperationTransportHandler primaryPhase = action.new PrimaryOperationTransportHandler() { Action.PrimaryOperationTransportHandler primaryPhase = action.new PrimaryOperationTransportHandler() {
@Override @Override
protected ReplicationOperation createReplicatedOperation(Request request, ActionListener<Response> listener, protected ReplicationOperation<Request, Request, Action.PrimaryResult> createReplicatedOperation(Request request,
Action.PrimaryShardReference primaryShardReference, ActionListener<Action.PrimaryResult> listener, Action.PrimaryShardReference primaryShardReference,
boolean executeOnReplicas) { boolean executeOnReplicas) {
assertIndexShardCounter(1); assertIndexShardCounter(1);
if (throwExceptionOnCreation) { if (throwExceptionOnCreation) {
@ -623,7 +621,7 @@ public class TransportReplicationActionTests extends ESTestCase {
if (throwExceptionOnRun) { if (throwExceptionOnRun) {
throw new ElasticsearchException("simulated exception, during performOnPrimary"); throw new ElasticsearchException("simulated exception, during performOnPrimary");
} else if (respondWithError) { } else if (respondWithError) {
this.finalResponseListener.onFailure(new ElasticsearchException("simulated exception, as a response")); this.resultListener.onFailure(new ElasticsearchException("simulated exception, as a response"));
} else { } else {
super.execute(); super.execute();
} }
@ -667,13 +665,13 @@ public class TransportReplicationActionTests extends ESTestCase {
final ReplicationTask task = maybeTask(); final ReplicationTask task = maybeTask();
Action action = new Action(Settings.EMPTY, "testActionWithExceptions", transportService, clusterService, threadPool) { Action action = new Action(Settings.EMPTY, "testActionWithExceptions", transportService, clusterService, threadPool) {
@Override @Override
protected void shardOperationOnReplica(Request request) { protected ReplicaResult shardOperationOnReplica(Request request) {
assertIndexShardCounter(1); assertIndexShardCounter(1);
assertPhase(task, "replica"); assertPhase(task, "replica");
if (throwException) { if (throwException) {
throw new ElasticsearchException("simulated"); throw new ElasticsearchException("simulated");
} }
super.shardOperationOnReplica(request); return new ReplicaResult();
} }
}; };
final Action.ReplicaOperationTransportHandler replicaOperationTransportHandler = action.new ReplicaOperationTransportHandler(); final Action.ReplicaOperationTransportHandler replicaOperationTransportHandler = action.new ReplicaOperationTransportHandler();
@ -765,15 +763,16 @@ public class TransportReplicationActionTests extends ESTestCase {
} }
@Override @Override
protected Tuple<Response, Request> shardOperationOnPrimary(Request shardRequest) throws Exception { protected PrimaryResult shardOperationOnPrimary(Request shardRequest) throws Exception {
boolean executedBefore = shardRequest.processedOnPrimary.getAndSet(true); boolean executedBefore = shardRequest.processedOnPrimary.getAndSet(true);
assert executedBefore == false : "request has already been executed on the primary"; assert executedBefore == false : "request has already been executed on the primary";
return new Tuple<>(new Response(), shardRequest); return new PrimaryResult(shardRequest, new Response());
} }
@Override @Override
protected void shardOperationOnReplica(Request request) { protected ReplicaResult shardOperationOnReplica(Request request) {
request.processedOnReplicas.incrementAndGet(); request.processedOnReplicas.incrementAndGet();
return new ReplicaResult();
} }
@Override @Override
@ -822,15 +821,14 @@ public class TransportReplicationActionTests extends ESTestCase {
} }
} }
class NoopReplicationOperation extends ReplicationOperation<Request, Request, Response> { class NoopReplicationOperation extends ReplicationOperation<Request, Request, Action.PrimaryResult> {
public NoopReplicationOperation(Request request, ActionListener<Action.PrimaryResult> listener) {
public NoopReplicationOperation(Request request, ActionListener<Response> listener) {
super(request, null, listener, true, true, null, null, TransportReplicationActionTests.this.logger, "noop"); super(request, null, listener, true, true, null, null, TransportReplicationActionTests.this.logger, "noop");
} }
@Override @Override
public void execute() throws Exception { public void execute() throws Exception {
this.finalResponseListener.onResponse(new Response()); this.resultListener.onResponse(action.new PrimaryResult(null, new Response()));
} }
} }

View file

@ -0,0 +1,190 @@
/*
* 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.action.ActionListener;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.action.support.WriteResponse;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
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;
import org.junit.Before;
import org.mockito.ArgumentCaptor;
import java.util.HashSet;
import java.util.function.BiConsumer;
import java.util.function.Consumer;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.verify;
public class TransportWriteActionTests extends ESTestCase {
private IndexShard indexShard;
private Translog.Location location;
@Before
public void initCommonMocks() {
indexShard = mock(IndexShard.class);
location = mock(Translog.Location.class);
}
public void testPrimaryNoRefreshCall() throws Exception {
noRefreshCall(TestAction::shardOperationOnPrimary, TestAction.WritePrimaryResult::respond);
}
public void testReplicaNoRefreshCall() throws Exception {
noRefreshCall(TestAction::shardOperationOnReplica, TestAction.WriteReplicaResult::respond);
}
private <Result, Response> void noRefreshCall(ThrowingBiFunction<TestAction, TestRequest, Result> action,
BiConsumer<Result, CapturingActionListener<Response>> responder)
throws Exception {
TestRequest request = new TestRequest();
request.setRefreshPolicy(RefreshPolicy.NONE); // The default, but we'll set it anyway just to be explicit
Result result = action.apply(new TestAction(), request);
CapturingActionListener<Response> listener = new CapturingActionListener<>();
responder.accept(result, listener);
assertNotNull(listener.response);
verify(indexShard, never()).refresh(any());
verify(indexShard, never()).addRefreshListener(any(), any());
}
public void testPrimaryImmediateRefresh() throws Exception {
immediateRefresh(TestAction::shardOperationOnPrimary, TestAction.WritePrimaryResult::respond, r -> assertTrue(r.forcedRefresh));
}
public void testReplicaImmediateRefresh() throws Exception {
immediateRefresh(TestAction::shardOperationOnReplica, TestAction.WriteReplicaResult::respond, r -> {});
}
private <Result, Response> void immediateRefresh(ThrowingBiFunction<TestAction, TestRequest, Result> action,
BiConsumer<Result, CapturingActionListener<Response>> responder,
Consumer<Response> responseChecker) throws Exception {
TestRequest request = new TestRequest();
request.setRefreshPolicy(RefreshPolicy.IMMEDIATE);
Result result = action.apply(new TestAction(), request);
CapturingActionListener<Response> listener = new CapturingActionListener<>();
responder.accept(result, listener);
assertNotNull(listener.response);
responseChecker.accept(listener.response);
verify(indexShard).refresh("refresh_flag_index");
verify(indexShard, never()).addRefreshListener(any(), any());
}
public void testPrimaryWaitForRefresh() throws Exception {
waitForRefresh(TestAction::shardOperationOnPrimary, TestAction.WritePrimaryResult::respond,
(r, forcedRefresh) -> assertEquals(forcedRefresh, r.forcedRefresh));
}
public void testReplicaWaitForRefresh() throws Exception {
waitForRefresh(TestAction::shardOperationOnReplica, TestAction.WriteReplicaResult::respond, (r, forcedRefresh) -> {});
}
private <Result, Response> void waitForRefresh(ThrowingBiFunction<TestAction, TestRequest, Result> action,
BiConsumer<Result, CapturingActionListener<Response>> responder,
BiConsumer<Response, Boolean> resultChecker) throws Exception {
TestRequest request = new TestRequest();
request.setRefreshPolicy(RefreshPolicy.WAIT_UNTIL);
Result result = action.apply(new TestAction(), request);
CapturingActionListener<Response> listener = new CapturingActionListener<>();
responder.accept(result, listener);
assertNull(listener.response); // Haven't reallresponded yet
@SuppressWarnings({ "unchecked", "rawtypes" })
ArgumentCaptor<Consumer<Boolean>> refreshListener = ArgumentCaptor.forClass((Class) Consumer.class);
verify(indexShard, never()).refresh(any());
verify(indexShard).addRefreshListener(any(), refreshListener.capture());
// Now we can fire the listener manually and we'll get a response
boolean forcedRefresh = randomBoolean();
refreshListener.getValue().accept(forcedRefresh);
assertNotNull(listener.response);
resultChecker.accept(listener.response, forcedRefresh);
}
private class TestAction extends TransportWriteAction<TestRequest, TestResponse> {
protected TestAction() {
super(Settings.EMPTY, "test", mock(TransportService.class), null, null, null, null, new ActionFilters(new HashSet<>()),
new IndexNameExpressionResolver(Settings.EMPTY), TestRequest::new, ThreadPool.Names.SAME);
}
@Override
protected IndexShard indexShard(TestRequest request) {
return indexShard;
}
@Override
protected WriteResult<TestResponse> onPrimaryShard(TestRequest request, IndexShard indexShard) throws Exception {
return new WriteResult<>(new TestResponse(), location);
}
@Override
protected Location onReplicaShard(TestRequest request, IndexShard indexShard) {
return location;
}
@Override
protected TestResponse newResponseInstance() {
return new TestResponse();
}
}
private static class TestRequest extends ReplicatedWriteRequest<TestRequest> {
public TestRequest() {
setShardId(new ShardId("test", "test", 1));
}
}
private static class TestResponse extends ReplicationResponse implements WriteResponse {
boolean forcedRefresh;
@Override
public void setForcedRefresh(boolean forcedRefresh) {
this.forcedRefresh = forcedRefresh;
}
}
private static class CapturingActionListener<R> implements ActionListener<R> {
private R response;
@Override
public void onResponse(R response) {
this.response = response;
}
@Override
public void onFailure(Throwable e) {
throw new RuntimeException(e);
}
}
private interface ThrowingBiFunction<A, B, R> {
R apply(A a, B b) throws Exception;
}
}

View file

@ -29,6 +29,7 @@ import org.elasticsearch.action.admin.indices.alias.get.GetAliasesResponse;
import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder; import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder;
import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.AliasAction; import org.elasticsearch.cluster.metadata.AliasAction;
import org.elasticsearch.cluster.metadata.AliasMetaData; import org.elasticsearch.cluster.metadata.AliasMetaData;
@ -175,10 +176,15 @@ public class IndexAliasesIT extends ESIntegTestCase {
assertAcked(admin().indices().prepareAliases().addAlias("test", "tests", termQuery("name", "test"))); assertAcked(admin().indices().prepareAliases().addAlias("test", "tests", termQuery("name", "test")));
logger.info("--> indexing against [test]"); logger.info("--> indexing against [test]");
client().index(indexRequest("test").type("type1").id("1").source(source("1", "foo test")).refresh(true)).actionGet(); client().index(indexRequest("test").type("type1").id("1").source(source("1", "foo test")).setRefreshPolicy(RefreshPolicy.IMMEDIATE))
client().index(indexRequest("test").type("type1").id("2").source(source("2", "bar test")).refresh(true)).actionGet(); .actionGet();
client().index(indexRequest("test").type("type1").id("3").source(source("3", "baz test")).refresh(true)).actionGet(); client().index(indexRequest("test").type("type1").id("2").source(source("2", "bar test")).setRefreshPolicy(RefreshPolicy.IMMEDIATE))
client().index(indexRequest("test").type("type1").id("4").source(source("4", "something else")).refresh(true)).actionGet(); .actionGet();
client().index(indexRequest("test").type("type1").id("3").source(source("3", "baz test")).setRefreshPolicy(RefreshPolicy.IMMEDIATE))
.actionGet();
client().index(
indexRequest("test").type("type1").id("4").source(source("4", "something else")).setRefreshPolicy(RefreshPolicy.IMMEDIATE))
.actionGet();
logger.info("--> checking single filtering alias search"); logger.info("--> checking single filtering alias search");
SearchResponse searchResponse = client().prepareSearch("foos").setQuery(QueryBuilders.matchAllQuery()).get(); SearchResponse searchResponse = client().prepareSearch("foos").setQuery(QueryBuilders.matchAllQuery()).get();

View file

@ -22,6 +22,7 @@ package org.elasticsearch.cluster.allocation;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.action.admin.cluster.reroute.ClusterRerouteResponse; import org.elasticsearch.action.admin.cluster.reroute.ClusterRerouteResponse;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.cluster.health.ClusterHealthStatus;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
@ -225,7 +226,7 @@ public class ClusterRerouteIT extends ESIntegTestCase {
assertThat(state.getRoutingNodes().unassigned().size(), equalTo(1)); assertThat(state.getRoutingNodes().unassigned().size(), equalTo(1));
assertThat(state.getRoutingNodes().node(state.nodes().resolveNode(node_1).getId()).iterator().next().state(), equalTo(ShardRoutingState.STARTED)); assertThat(state.getRoutingNodes().node(state.nodes().resolveNode(node_1).getId()).iterator().next().state(), equalTo(ShardRoutingState.STARTED));
client().prepareIndex("test", "type", "1").setSource("field", "value").setRefresh(true).execute().actionGet(); client().prepareIndex("test", "type", "1").setSource("field", "value").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
final Index index = resolveIndex("test"); final Index index = resolveIndex("test");
logger.info("--> closing all nodes"); logger.info("--> closing all nodes");

View file

@ -19,7 +19,6 @@
package org.elasticsearch.document; package org.elasticsearch.document;
import org.elasticsearch.action.ReplicationResponse;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse; import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse;
import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkItemResponse;
@ -27,6 +26,7 @@ import org.elasticsearch.action.bulk.BulkRequestBuilder;
import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.bulk.BulkResponse;
import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.delete.DeleteResponse;
import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.action.support.replication.ReplicationResponse;
import org.elasticsearch.action.update.UpdateResponse; import org.elasticsearch.action.update.UpdateResponse;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;

View file

@ -0,0 +1,217 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index;
import org.elasticsearch.action.ListenableActionFuture;
import org.elasticsearch.action.bulk.BulkItemResponse;
import org.elasticsearch.action.bulk.BulkRequestBuilder;
import org.elasticsearch.action.bulk.BulkResponse;
import org.elasticsearch.action.delete.DeleteResponse;
import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.action.update.UpdateResponse;
import org.elasticsearch.common.network.NetworkModule;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.script.ExecutableScript;
import org.elasticsearch.script.NativeScriptFactory;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptModule;
import org.elasticsearch.script.ScriptService.ScriptType;
import org.elasticsearch.test.ESIntegTestCase;
import org.junit.Before;
import java.util.Collection;
import java.util.Map;
import java.util.concurrent.ExecutionException;
import static java.util.Collections.emptyMap;
import static java.util.Collections.singleton;
import static java.util.Collections.singletonMap;
import static org.elasticsearch.index.query.QueryBuilders.matchQuery;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoSearchHits;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchHits;
/**
* Tests that requests with RefreshPolicy.WAIT_UNTIL will be visible when they return.
*/
public class WaitUntilRefreshIT extends ESIntegTestCase {
@Override
protected Settings nodeSettings(int nodeOrdinal) {
return Settings.builder().put(super.nodeSettings(nodeOrdinal)).put(NetworkModule.HTTP_ENABLED.getKey(), true).build();
}
@Override
public Settings indexSettings() {
// Use a shorter refresh interval to speed up the tests. We'll be waiting on this interval several times.
return Settings.builder().put(super.indexSettings()).put("index.refresh_interval", "40ms").build();
}
@Before
public void createTestIndex() {
createIndex("test");
}
public void testIndex() {
IndexResponse index = client().prepareIndex("test", "index", "1").setSource("foo", "bar").setRefreshPolicy(RefreshPolicy.WAIT_UNTIL)
.get();
assertEquals(RestStatus.CREATED, index.status());
assertFalse("request shouldn't have forced a refresh", index.forcedRefresh());
assertSearchHits(client().prepareSearch("test").setQuery(matchQuery("foo", "bar")).get(), "1");
}
public void testDelete() throws InterruptedException, ExecutionException {
// Index normally
indexRandom(true, client().prepareIndex("test", "test", "1").setSource("foo", "bar"));
assertSearchHits(client().prepareSearch("test").setQuery(matchQuery("foo", "bar")).get(), "1");
// Now delete with blockUntilRefresh
DeleteResponse delete = client().prepareDelete("test", "test", "1").setRefreshPolicy(RefreshPolicy.WAIT_UNTIL).get();
assertTrue("document was deleted", delete.isFound());
assertFalse("request shouldn't have forced a refresh", delete.forcedRefresh());
assertNoSearchHits(client().prepareSearch("test").setQuery(matchQuery("foo", "bar")).get());
}
public void testUpdate() throws InterruptedException, ExecutionException {
// Index normally
indexRandom(true, client().prepareIndex("test", "test", "1").setSource("foo", "bar"));
assertSearchHits(client().prepareSearch("test").setQuery(matchQuery("foo", "bar")).get(), "1");
// Update with RefreshPolicy.WAIT_UNTIL
UpdateResponse update = client().prepareUpdate("test", "test", "1").setDoc("foo", "baz").setRefreshPolicy(RefreshPolicy.WAIT_UNTIL)
.get();
assertEquals(2, update.getVersion());
assertFalse("request shouldn't have forced a refresh", update.forcedRefresh());
assertSearchHits(client().prepareSearch("test").setQuery(matchQuery("foo", "baz")).get(), "1");
// Upsert with RefreshPolicy.WAIT_UNTIL
update = client().prepareUpdate("test", "test", "2").setDocAsUpsert(true).setDoc("foo", "cat")
.setRefreshPolicy(RefreshPolicy.WAIT_UNTIL).get();
assertEquals(1, update.getVersion());
assertFalse("request shouldn't have forced a refresh", update.forcedRefresh());
assertSearchHits(client().prepareSearch("test").setQuery(matchQuery("foo", "cat")).get(), "2");
// Update-becomes-delete with RefreshPolicy.WAIT_UNTIL
update = client().prepareUpdate("test", "test", "2").setScript(new Script("delete_plz", ScriptType.INLINE, "native", emptyMap()))
.setRefreshPolicy(RefreshPolicy.WAIT_UNTIL).get();
assertEquals(2, update.getVersion());
assertFalse("request shouldn't have forced a refresh", update.forcedRefresh());
assertNoSearchHits(client().prepareSearch("test").setQuery(matchQuery("foo", "cat")).get());
}
public void testBulk() {
// Index by bulk with RefreshPolicy.WAIT_UNTIL
BulkRequestBuilder bulk = client().prepareBulk().setRefreshPolicy(RefreshPolicy.WAIT_UNTIL);
bulk.add(client().prepareIndex("test", "test", "1").setSource("foo", "bar"));
assertBulkSuccess(bulk.get());
assertSearchHits(client().prepareSearch("test").setQuery(matchQuery("foo", "bar")).get(), "1");
// Update by bulk with RefreshPolicy.WAIT_UNTIL
bulk = client().prepareBulk().setRefreshPolicy(RefreshPolicy.WAIT_UNTIL);
bulk.add(client().prepareUpdate("test", "test", "1").setDoc("foo", "baz"));
assertBulkSuccess(bulk.get());
assertSearchHits(client().prepareSearch("test").setQuery(matchQuery("foo", "baz")).get(), "1");
// Delete by bulk with RefreshPolicy.WAIT_UNTIL
bulk = client().prepareBulk().setRefreshPolicy(RefreshPolicy.WAIT_UNTIL);
bulk.add(client().prepareDelete("test", "test", "1"));
assertBulkSuccess(bulk.get());
assertNoSearchHits(client().prepareSearch("test").setQuery(matchQuery("foo", "bar")).get());
// Update makes a noop
bulk = client().prepareBulk().setRefreshPolicy(RefreshPolicy.WAIT_UNTIL);
bulk.add(client().prepareDelete("test", "test", "1"));
assertBulkSuccess(bulk.get());
}
/**
* Tests that an explicit request makes block_until_refresh return. It doesn't check that block_until_refresh doesn't return until the
* explicit refresh if the interval is -1 because we don't have that kind of control over refresh. It can happen all on its own.
*/
public void testNoRefreshInterval() throws InterruptedException, ExecutionException {
client().admin().indices().prepareUpdateSettings("test").setSettings(singletonMap("index.refresh_interval", -1)).get();
ListenableActionFuture<IndexResponse> index = client().prepareIndex("test", "index", "1").setSource("foo", "bar")
.setRefreshPolicy(RefreshPolicy.WAIT_UNTIL).execute();
while (false == index.isDone()) {
client().admin().indices().prepareRefresh("test").get();
}
assertEquals(RestStatus.CREATED, index.get().status());
assertFalse("request shouldn't have forced a refresh", index.get().forcedRefresh());
assertSearchHits(client().prepareSearch("test").setQuery(matchQuery("foo", "bar")).get(), "1");
}
private void assertBulkSuccess(BulkResponse response) {
assertNoFailures(response);
for (BulkItemResponse item : response) {
assertFalse("request shouldn't have forced a refresh", item.getResponse().forcedRefresh());
}
}
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
return singleton(DeletePlzPlugin.class);
}
public static class DeletePlzPlugin extends Plugin {
@Override
public String name() {
return "delete_please";
}
@Override
public String description() {
return "adds a script that converts any update into a delete for testing";
}
public void onModule(ScriptModule scriptModule) {
scriptModule.registerScript("delete_plz", DeletePlzFactory.class);
}
}
public static class DeletePlzFactory implements NativeScriptFactory {
@Override
public ExecutableScript newScript(Map<String, Object> params) {
return new ExecutableScript() {
private Map<String, Object> ctx;
@Override
@SuppressWarnings("unchecked") // Elasicsearch convention
public void setNextVar(String name, Object value) {
if (name.equals("ctx")) {
ctx = (Map<String, Object>) value;
}
}
@Override
public Object run() {
ctx.put("op", "delete");
return null;
}
};
}
@Override
public boolean needsScores() {
return false;
}
}
}

View file

@ -170,6 +170,8 @@ public class InternalEngineTests extends ESTestCase {
.put(IndexSettings.INDEX_GC_DELETES_SETTING, "1h") // make sure this doesn't kick in on us .put(IndexSettings.INDEX_GC_DELETES_SETTING, "1h") // make sure this doesn't kick in on us
.put(EngineConfig.INDEX_CODEC_SETTING.getKey(), codecName) .put(EngineConfig.INDEX_CODEC_SETTING.getKey(), codecName)
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
.put(IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD,
between(10, 10 * IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.get(Settings.EMPTY)))
.build()); // TODO randomize more settings .build()); // TODO randomize more settings
threadPool = new ThreadPool(getClass().getName()); threadPool = new ThreadPool(getClass().getName());
store = createStore(); store = createStore();
@ -200,7 +202,7 @@ public class InternalEngineTests extends ESTestCase {
return new EngineConfig(openMode, config.getShardId(), config.getThreadPool(), config.getIndexSettings(), config.getWarmer(), return new EngineConfig(openMode, config.getShardId(), config.getThreadPool(), config.getIndexSettings(), config.getWarmer(),
config.getStore(), config.getDeletionPolicy(), config.getMergePolicy(), config.getAnalyzer(), config.getSimilarity(), config.getStore(), config.getDeletionPolicy(), config.getMergePolicy(), config.getAnalyzer(), config.getSimilarity(),
new CodecService(null, logger), config.getEventListener(), config.getTranslogRecoveryPerformer(), config.getQueryCache(), new CodecService(null, logger), config.getEventListener(), config.getTranslogRecoveryPerformer(), config.getQueryCache(),
config.getQueryCachingPolicy(), config.getTranslogConfig(), config.getFlushMergesAfter()); config.getQueryCachingPolicy(), config.getTranslogConfig(), config.getFlushMergesAfter(), config.getRefreshListeners());
} }
@Override @Override
@ -291,14 +293,16 @@ public class InternalEngineTests extends ESTestCase {
} catch (IOException e) { } catch (IOException e) {
throw new ElasticsearchException("can't find index?", e); throw new ElasticsearchException("can't find index?", e);
} }
EngineConfig config = new EngineConfig(openMode, shardId, threadPool, indexSettings Engine.EventListener listener = new Engine.EventListener() {
, null, store, createSnapshotDeletionPolicy(), mergePolicy,
iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(null, logger), new Engine.EventListener() {
@Override @Override
public void onFailedEngine(String reason, @Nullable Throwable t) { public void onFailedEngine(String reason, @Nullable Throwable t) {
// we don't need to notify anybody in this test // we don't need to notify anybody in this test
} }
}, new TranslogHandler(shardId.getIndexName(), logger), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5)); };
EngineConfig config = new EngineConfig(openMode, shardId, threadPool, indexSettings, null, store, createSnapshotDeletionPolicy(),
mergePolicy, iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(null, logger), listener,
new TranslogHandler(shardId.getIndexName(), logger), IndexSearcher.getDefaultQueryCache(),
IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5), null);
return config; return config;
} }
@ -315,12 +319,14 @@ public class InternalEngineTests extends ESTestCase {
assertThat(engine.segmentsStats(false).getCount(), equalTo(0L)); assertThat(engine.segmentsStats(false).getCount(), equalTo(0L));
assertThat(engine.segmentsStats(false).getMemoryInBytes(), equalTo(0L)); assertThat(engine.segmentsStats(false).getMemoryInBytes(), equalTo(0L));
// create a doc and refresh // create two docs and refresh
ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null); ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null);
engine.index(new Engine.Index(newUid("1"), doc)); Engine.Index first = new Engine.Index(newUid("1"), doc);
engine.index(first);
ParsedDocument doc2 = testParsedDocument("2", "2", "test", null, -1, -1, testDocumentWithTextField(), B_2, null); ParsedDocument doc2 = testParsedDocument("2", "2", "test", null, -1, -1, testDocumentWithTextField(), B_2, null);
engine.index(new Engine.Index(newUid("2"), doc2)); Engine.Index second = new Engine.Index(newUid("2"), doc2);
engine.index(second);
assertThat(second.getTranslogLocation(), greaterThan(first.getTranslogLocation()));
engine.refresh("test"); engine.refresh("test");
segments = engine.segments(false); segments = engine.segments(false);
@ -2064,10 +2070,11 @@ public class InternalEngineTests extends ESTestCase {
/* create a TranslogConfig that has been created with a different UUID */ /* create a TranslogConfig that has been created with a different UUID */
TranslogConfig translogConfig = new TranslogConfig(shardId, translog.location(), config.getIndexSettings(), BigArrays.NON_RECYCLING_INSTANCE); TranslogConfig translogConfig = new TranslogConfig(shardId, translog.location(), config.getIndexSettings(), BigArrays.NON_RECYCLING_INSTANCE);
EngineConfig brokenConfig = new EngineConfig(EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG, shardId, threadPool, config.getIndexSettings() EngineConfig brokenConfig = new EngineConfig(EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG, shardId, threadPool,
, null, store, createSnapshotDeletionPolicy(), newMergePolicy(), config.getIndexSettings(), null, store, createSnapshotDeletionPolicy(), newMergePolicy(), config.getAnalyzer(),
config.getAnalyzer(), config.getSimilarity(), new CodecService(null, logger), config.getEventListener() config.getSimilarity(), new CodecService(null, logger), config.getEventListener(), config.getTranslogRecoveryPerformer(),
, config.getTranslogRecoveryPerformer(), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5)); IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig,
TimeValue.timeValueMinutes(5), config.getRefreshListeners());
try { try {
InternalEngine internalEngine = new InternalEngine(brokenConfig); InternalEngine internalEngine = new InternalEngine(brokenConfig);

View file

@ -55,6 +55,7 @@ import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.internal.SourceFieldMapper; import org.elasticsearch.index.mapper.internal.SourceFieldMapper;
import org.elasticsearch.index.mapper.internal.UidFieldMapper; import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import org.elasticsearch.index.shard.RefreshListeners;
import org.elasticsearch.index.shard.DocsStats; import org.elasticsearch.index.shard.DocsStats;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardUtils; import org.elasticsearch.index.shard.ShardUtils;
@ -213,7 +214,7 @@ public class ShadowEngineTests extends ESTestCase {
} }
protected ShadowEngine createShadowEngine(IndexSettings indexSettings, Store store) { protected ShadowEngine createShadowEngine(IndexSettings indexSettings, Store store) {
return new ShadowEngine(config(indexSettings, store, null, null)); return new ShadowEngine(config(indexSettings, store, null, null, null));
} }
protected InternalEngine createInternalEngine(IndexSettings indexSettings, Store store, Path translogPath) { protected InternalEngine createInternalEngine(IndexSettings indexSettings, Store store, Path translogPath) {
@ -221,11 +222,12 @@ public class ShadowEngineTests extends ESTestCase {
} }
protected InternalEngine createInternalEngine(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy) { protected InternalEngine createInternalEngine(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy) {
EngineConfig config = config(indexSettings, store, translogPath, mergePolicy); EngineConfig config = config(indexSettings, store, translogPath, mergePolicy, null);
return new InternalEngine(config); return new InternalEngine(config);
} }
public EngineConfig config(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy) { public EngineConfig config(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy,
RefreshListeners refreshListeners) {
IndexWriterConfig iwc = newIndexWriterConfig(); IndexWriterConfig iwc = newIndexWriterConfig();
final EngineConfig.OpenMode openMode; final EngineConfig.OpenMode openMode;
try { try {
@ -237,14 +239,17 @@ public class ShadowEngineTests extends ESTestCase {
} catch (IOException e) { } catch (IOException e) {
throw new ElasticsearchException("can't find index?", e); throw new ElasticsearchException("can't find index?", e);
} }
TranslogConfig translogConfig = new TranslogConfig(shardId, translogPath, indexSettings, BigArrays.NON_RECYCLING_INSTANCE); Engine.EventListener eventListener = new Engine.EventListener() {
EngineConfig config = new EngineConfig(openMode, shardId, threadPool, indexSettings
, null, store, createSnapshotDeletionPolicy(), mergePolicy,
iwc.getAnalyzer(), iwc.getSimilarity() , new CodecService(null, logger), new Engine.EventListener() {
@Override @Override
public void onFailedEngine(String reason, @Nullable Throwable t) { public void onFailedEngine(String reason, @Nullable Throwable t) {
// we don't need to notify anybody in this test // we don't need to notify anybody in this test
}}, null, IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5)); }
};
TranslogConfig translogConfig = new TranslogConfig(shardId, translogPath, indexSettings, BigArrays.NON_RECYCLING_INSTANCE);
EngineConfig config = new EngineConfig(openMode, shardId, threadPool, indexSettings, null, store, createSnapshotDeletionPolicy(),
mergePolicy, iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(null, logger), eventListener, null,
IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig,
TimeValue.timeValueMinutes(5), refreshListeners);
return config; return config;
} }
@ -1011,4 +1016,11 @@ public class ShadowEngineTests extends ESTestCase {
assertEquals(0, docStats.getDeleted()); assertEquals(0, docStats.getDeleted());
primaryEngine.forceMerge(randomBoolean(), 1, false, false, false); primaryEngine.forceMerge(randomBoolean(), 1, false, false, false);
} }
public void testRefreshListenersFails() throws IOException {
EngineConfig config = config(defaultSettings, store, createTempDir(), newMergePolicy(),
new RefreshListeners(null, null, null, logger));
Exception e = expectThrows(IllegalArgumentException.class, () -> new ShadowEngine(config));
assertEquals("ShadowEngine doesn't support RefreshListeners", e.getMessage());
}
} }

View file

@ -19,6 +19,7 @@
package org.elasticsearch.index.mapper.all; package org.elasticsearch.index.mapper.all;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.client.Client; import org.elasticsearch.client.Client;
import org.elasticsearch.index.query.MatchPhraseQueryBuilder; import org.elasticsearch.index.query.MatchPhraseQueryBuilder;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
@ -87,7 +88,7 @@ public class AllFieldMapperPositionIncrementGapTests extends ESSingleNodeTestCas
private static void testGap(Client client, String indexName, private static void testGap(Client client, String indexName,
String type, int positionIncrementGap) throws IOException { String type, int positionIncrementGap) throws IOException {
client.prepareIndex(indexName, type, "position_gap_test") client.prepareIndex(indexName, type, "position_gap_test")
.setSource("string1", "one", "string2", "two three").setRefresh(true).get(); .setSource("string1", "one", "string2", "two three").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
// Baseline - phrase query finds matches in the same field value // Baseline - phrase query finds matches in the same field value
assertHitCount(client.prepareSearch(indexName) assertHitCount(client.prepareSearch(indexName)

View file

@ -0,0 +1,292 @@
/*
* 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.shard;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy;
import org.apache.lucene.index.SnapshotDeletionPolicy;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
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.common.util.concurrent.FutureUtils;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.codec.CodecService;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.engine.EngineConfig;
import org.elasticsearch.index.engine.InternalEngine;
import org.elasticsearch.index.engine.InternalEngineTests.TranslogHandler;
import org.elasticsearch.index.fieldvisitor.SingleFieldsVisitor;
import org.elasticsearch.index.mapper.ParseContext.Document;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import org.elasticsearch.index.store.DirectoryService;
import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.translog.TranslogConfig;
import org.elasticsearch.test.DummyShardLock;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.IndexSettingsModule;
import org.elasticsearch.threadpool.ThreadPool;
import org.junit.After;
import org.junit.Before;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Locale;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Consumer;
import static org.elasticsearch.common.unit.TimeValue.timeValueMillis;
/**
* Tests how {@linkplain RefreshListeners} interacts with {@linkplain InternalEngine}.
*/
public class RefreshListenersTests extends ESTestCase {
private RefreshListeners listeners;
private Engine engine;
private volatile int maxListeners;
private ThreadPool threadPool;
private Store store;
@Before
public void setupListeners() throws Exception {
// Setup dependencies of the listeners
maxListeners = randomIntBetween(1, 1000);
listeners = new RefreshListeners(
() -> maxListeners,
() -> engine.refresh("too-many-listeners"),
// Immediately run listeners rather than adding them to the listener thread pool like IndexShard does to simplify the test.
Runnable::run,
logger
);
// Now setup the InternalEngine which is much more complicated because we aren't mocking anything
threadPool = new ThreadPool(getTestName());
IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("index", Settings.EMPTY);
ShardId shardId = new ShardId(new Index("index", "_na_"), 1);
Directory directory = newDirectory();
DirectoryService directoryService = new DirectoryService(shardId, indexSettings) {
@Override
public Directory newDirectory() throws IOException {
return directory;
}
@Override
public long throttleTimeInNanos() {
return 0;
}
};
store = new Store(shardId, indexSettings, directoryService, new DummyShardLock(shardId));
IndexWriterConfig iwc = newIndexWriterConfig();
TranslogConfig translogConfig = new TranslogConfig(shardId, createTempDir("translog"), indexSettings,
BigArrays.NON_RECYCLING_INSTANCE);
Engine.EventListener eventListener = new Engine.EventListener() {
@Override
public void onFailedEngine(String reason, @Nullable Throwable t) {
// we don't need to notify anybody in this test
}
};
EngineConfig config = new EngineConfig(EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG, shardId, threadPool, indexSettings, null,
store, new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy()), newMergePolicy(), iwc.getAnalyzer(),
iwc.getSimilarity(), new CodecService(null, logger), eventListener, new TranslogHandler(shardId.getIndexName(), logger),
IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig,
TimeValue.timeValueMinutes(5), listeners);
engine = new InternalEngine(config);
}
@After
public void tearDownListeners() throws Exception {
IOUtils.close(engine, store);
terminate(threadPool);
}
public void testTooMany() throws Exception {
assertFalse(listeners.refreshNeeded());
Engine.Index index = index("1");
// Fill the listener slots
List<DummyRefreshListener> nonForcedListeners = new ArrayList<>(maxListeners);
for (int i = 0; i < maxListeners; i++) {
DummyRefreshListener listener = new DummyRefreshListener();
nonForcedListeners.add(listener);
listeners.addOrNotify(index.getTranslogLocation(), listener);
assertTrue(listeners.refreshNeeded());
}
// We shouldn't have called any of them
for (DummyRefreshListener listener : nonForcedListeners) {
assertNull("Called listener too early!", listener.forcedRefresh.get());
}
// Add one more listener which should cause a refresh.
DummyRefreshListener forcingListener = new DummyRefreshListener();
listeners.addOrNotify(index.getTranslogLocation(), forcingListener);
assertTrue("Forced listener wasn't forced?", forcingListener.forcedRefresh.get());
// That forces all the listeners through. It would be on the listener ThreadPool but we've made all of those execute immediately.
for (DummyRefreshListener listener : nonForcedListeners) {
assertEquals("Expected listener called with unforced refresh!", Boolean.FALSE, listener.forcedRefresh.get());
}
assertFalse(listeners.refreshNeeded());
}
public void testAfterRefresh() throws Exception {
Engine.Index index = index("1");
engine.refresh("I said so");
if (randomBoolean()) {
index(randomFrom("1" /* same document */, "2" /* different document */));
if (randomBoolean()) {
engine.refresh("I said so");
}
}
DummyRefreshListener listener = new DummyRefreshListener();
listeners.addOrNotify(index.getTranslogLocation(), listener);
assertFalse(listener.forcedRefresh.get());
}
/**
* Attempts to add a listener at the same time as a refresh occurs by having a background thread force a refresh as fast as it can while
* adding listeners. This can catch the situation where a refresh happens right as the listener is being added such that the listener
* misses the refresh and has to catch the next one. If the listener wasn't able to properly catch the next one then this would fail.
*/
public void testConcurrentRefresh() throws Exception {
AtomicBoolean run = new AtomicBoolean(true);
Thread refresher = new Thread(() -> {
while (run.get()) {
engine.refresh("test");
}
});
refresher.start();
try {
for (int i = 0; i < 100; i++) {
Engine.Index index = index("1");
DummyRefreshListener listener = new DummyRefreshListener();
listeners.addOrNotify(index.getTranslogLocation(), listener);
assertBusy(() -> assertNotNull(listener.forcedRefresh.get()));
assertFalse(listener.forcedRefresh.get());
}
} finally {
run.set(false);
refresher.join();
}
}
/**
* Uses a bunch of threads to index, wait for refresh, and non-realtime get documents to validate that they are visible after waiting
* regardless of what crazy sequence of events causes the refresh listener to fire.
*/
public void testLotsOfThreads() throws Exception {
int threadCount = between(3, 10);
maxListeners = between(1, threadCount * 2);
// This thread just refreshes every once in a while to cause trouble.
ScheduledFuture<?> refresher = threadPool.scheduleWithFixedDelay(() -> engine.refresh("because test"), timeValueMillis(100));
// These threads add and block until the refresh makes the change visible and then do a non-realtime get.
Thread[] indexers = new Thread[threadCount];
for (int thread = 0; thread < threadCount; thread++) {
final String threadId = String.format(Locale.ROOT, "%04d", thread);
indexers[thread] = new Thread(() -> {
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());
DummyRefreshListener listener = new DummyRefreshListener();
listeners.addOrNotify(index.getTranslogLocation(), listener);
assertBusy(() -> assertNotNull("listener never called", listener.forcedRefresh.get()));
if (threadCount < maxListeners) {
assertFalse(listener.forcedRefresh.get());
}
Engine.Get get = new Engine.Get(false, index.uid());
try (Engine.GetResult getResult = engine.get(get)) {
assertTrue("document not found", getResult.exists());
assertEquals(iteration, getResult.version());
SingleFieldsVisitor visitor = new SingleFieldsVisitor("test");
getResult.docIdAndVersion().context.reader().document(getResult.docIdAndVersion().docId, visitor);
assertEquals(Arrays.asList(testFieldValue), visitor.fields().get("test"));
}
} catch (Throwable t) {
throw new RuntimeException("failure on the [" + iteration + "] iteration of thread [" + threadId + "]", t);
}
}
});
indexers[thread].start();
}
for (Thread indexer: indexers) {
indexer.join();
}
FutureUtils.cancel(refresher);
}
private Engine.Index index(String id) {
return index(id, "test");
}
private Engine.Index index(String id, String testFieldValue) {
String type = "test";
String uid = type + ":" + id;
Document document = new Document();
document.add(new TextField("test", testFieldValue, Field.Store.YES));
Field uidField = new Field("_uid", type + ":" + id, UidFieldMapper.Defaults.FIELD_TYPE);
Field versionField = new NumericDocValuesField("_version", Versions.MATCH_ANY);
document.add(uidField);
document.add(versionField);
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;
}
private static class DummyRefreshListener implements Consumer<Boolean> {
/**
* When the listener is called this captures it's only argument.
*/
private AtomicReference<Boolean> forcedRefresh = new AtomicReference<>();
@Override
public void accept(Boolean forcedRefresh) {
assertNotNull(forcedRefresh);
Boolean oldValue = this.forcedRefresh.getAndSet(forcedRefresh);
assertNull("Listener called twice", oldValue);
}
}
}

View file

@ -20,6 +20,7 @@
package org.elasticsearch.index.translog; package org.elasticsearch.index.translog;
import com.carrotsearch.randomizedtesting.generators.RandomPicks; import com.carrotsearch.randomizedtesting.generators.RandomPicks;
import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.index.Term; import org.apache.lucene.index.Term;
import org.apache.lucene.mockfile.FilterFileChannel; import org.apache.lucene.mockfile.FilterFileChannel;
@ -44,6 +45,7 @@ import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.index.VersionType; import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.translog.Translog.Location;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.IndexSettingsModule; import org.elasticsearch.test.IndexSettingsModule;
import org.hamcrest.Matchers; import org.hamcrest.Matchers;
@ -204,18 +206,40 @@ public class TranslogTests extends ESTestCase {
} }
public void testRead() throws IOException { public void testRead() throws IOException {
Location loc0 = translog.getLastWriteLocation();
assertNotNull(loc0);
Translog.Location loc1 = translog.add(new Translog.Index("test", "1", new byte[]{1})); Translog.Location loc1 = translog.add(new Translog.Index("test", "1", new byte[]{1}));
assertThat(loc1, greaterThan(loc0));
assertThat(translog.getLastWriteLocation(), greaterThan(loc1));
Translog.Location loc2 = translog.add(new Translog.Index("test", "2", new byte[]{2})); Translog.Location loc2 = translog.add(new Translog.Index("test", "2", new byte[]{2}));
assertThat(loc2, greaterThan(loc1));
assertThat(translog.getLastWriteLocation(), greaterThan(loc2));
assertThat(translog.read(loc1).getSource().source.toBytesArray(), equalTo(new BytesArray(new byte[]{1}))); assertThat(translog.read(loc1).getSource().source.toBytesArray(), equalTo(new BytesArray(new byte[]{1})));
assertThat(translog.read(loc2).getSource().source.toBytesArray(), equalTo(new BytesArray(new byte[]{2}))); assertThat(translog.read(loc2).getSource().source.toBytesArray(), equalTo(new BytesArray(new byte[]{2})));
Translog.Location lastLocBeforeSync = translog.getLastWriteLocation();
translog.sync(); translog.sync();
assertEquals(lastLocBeforeSync, translog.getLastWriteLocation());
assertThat(translog.read(loc1).getSource().source.toBytesArray(), equalTo(new BytesArray(new byte[]{1}))); assertThat(translog.read(loc1).getSource().source.toBytesArray(), equalTo(new BytesArray(new byte[]{1})));
assertThat(translog.read(loc2).getSource().source.toBytesArray(), equalTo(new BytesArray(new byte[]{2}))); assertThat(translog.read(loc2).getSource().source.toBytesArray(), equalTo(new BytesArray(new byte[]{2})));
Translog.Location loc3 = translog.add(new Translog.Index("test", "2", new byte[]{3})); Translog.Location loc3 = translog.add(new Translog.Index("test", "2", new byte[]{3}));
assertThat(loc3, greaterThan(loc2));
assertThat(translog.getLastWriteLocation(), greaterThan(loc3));
assertThat(translog.read(loc3).getSource().source.toBytesArray(), equalTo(new BytesArray(new byte[]{3}))); assertThat(translog.read(loc3).getSource().source.toBytesArray(), equalTo(new BytesArray(new byte[]{3})));
lastLocBeforeSync = translog.getLastWriteLocation();
translog.sync(); translog.sync();
assertEquals(lastLocBeforeSync, translog.getLastWriteLocation());
assertThat(translog.read(loc3).getSource().source.toBytesArray(), equalTo(new BytesArray(new byte[]{3}))); assertThat(translog.read(loc3).getSource().source.toBytesArray(), equalTo(new BytesArray(new byte[]{3})));
translog.prepareCommit(); translog.prepareCommit();
/*
* The commit adds to the lastWriteLocation even though is isn't really a write. This is just an implementation artifact but it can
* safely be ignored because the lastWriteLocation continues to be greater than the Location returned from the last write operation
* and less than the location of the next write operation.
*/
assertThat(translog.getLastWriteLocation(), greaterThan(lastLocBeforeSync));
assertThat(translog.read(loc3).getSource().source.toBytesArray(), equalTo(new BytesArray(new byte[]{3}))); assertThat(translog.read(loc3).getSource().source.toBytesArray(), equalTo(new BytesArray(new byte[]{3})));
translog.commit(); translog.commit();
assertNull(translog.read(loc1)); assertNull(translog.read(loc1));

View file

@ -21,6 +21,7 @@ package org.elasticsearch.routing;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchType; import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
@ -45,7 +46,7 @@ public class AliasRoutingIT extends ESIntegTestCase {
assertAcked(admin().indices().prepareAliases().addAliasAction(newAddAliasAction("test", "alias0").routing("0"))); assertAcked(admin().indices().prepareAliases().addAliasAction(newAddAliasAction("test", "alias0").routing("0")));
logger.info("--> indexing with id [1], and routing [0] using alias"); logger.info("--> indexing with id [1], and routing [0] using alias");
client().prepareIndex("alias0", "type1", "1").setSource("field", "value1").setRefresh(true).execute().actionGet(); client().prepareIndex("alias0", "type1", "1").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
logger.info("--> verifying get with no routing, should not find anything"); logger.info("--> verifying get with no routing, should not find anything");
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false)); assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false));
@ -72,7 +73,7 @@ public class AliasRoutingIT extends ESIntegTestCase {
logger.info("--> deleting with no routing, should not delete anything"); logger.info("--> deleting with no routing, should not delete anything");
client().prepareDelete("test", "type1", "1").setRefresh(true).execute().actionGet(); client().prepareDelete("test", "type1", "1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false)); assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false));
assertThat(client().prepareGet("test", "type1", "1").setRouting("0").execute().actionGet().isExists(), equalTo(true)); assertThat(client().prepareGet("test", "type1", "1").setRouting("0").execute().actionGet().isExists(), equalTo(true));
@ -80,7 +81,7 @@ public class AliasRoutingIT extends ESIntegTestCase {
} }
logger.info("--> deleting with routing alias, should delete"); logger.info("--> deleting with routing alias, should delete");
client().prepareDelete("alias0", "type1", "1").setRefresh(true).execute().actionGet(); client().prepareDelete("alias0", "type1", "1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false)); assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false));
assertThat(client().prepareGet("test", "type1", "1").setRouting("0").execute().actionGet().isExists(), equalTo(false)); assertThat(client().prepareGet("test", "type1", "1").setRouting("0").execute().actionGet().isExists(), equalTo(false));
@ -88,7 +89,7 @@ public class AliasRoutingIT extends ESIntegTestCase {
} }
logger.info("--> indexing with id [1], and routing [0] using alias"); logger.info("--> indexing with id [1], and routing [0] using alias");
client().prepareIndex("alias0", "type1", "1").setSource("field", "value1").setRefresh(true).execute().actionGet(); client().prepareIndex("alias0", "type1", "1").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
logger.info("--> verifying get with no routing, should not find anything"); logger.info("--> verifying get with no routing, should not find anything");
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false)); assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false));
@ -110,7 +111,7 @@ public class AliasRoutingIT extends ESIntegTestCase {
.addAliasAction(newAddAliasAction("test", "alias01").searchRouting("0,1"))); .addAliasAction(newAddAliasAction("test", "alias01").searchRouting("0,1")));
logger.info("--> indexing with id [1], and routing [0] using alias"); logger.info("--> indexing with id [1], and routing [0] using alias");
client().prepareIndex("alias0", "type1", "1").setSource("field", "value1").setRefresh(true).execute().actionGet(); client().prepareIndex("alias0", "type1", "1").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
logger.info("--> verifying get with no routing, should not find anything"); logger.info("--> verifying get with no routing, should not find anything");
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false)); assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false));
@ -142,7 +143,7 @@ public class AliasRoutingIT extends ESIntegTestCase {
} }
logger.info("--> indexing with id [2], and routing [1] using alias"); logger.info("--> indexing with id [2], and routing [1] using alias");
client().prepareIndex("alias1", "type1", "2").setSource("field", "value1").setRefresh(true).execute().actionGet(); client().prepareIndex("alias1", "type1", "2").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
logger.info("--> search with no routing, should fine two"); logger.info("--> search with no routing, should fine two");
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
@ -207,7 +208,7 @@ public class AliasRoutingIT extends ESIntegTestCase {
.addAliasAction(newAddAliasAction("test-b", "alias-ab").searchRouting("1"))); .addAliasAction(newAddAliasAction("test-b", "alias-ab").searchRouting("1")));
ensureGreen(); // wait for events again to make sure we got the aliases on all nodes ensureGreen(); // wait for events again to make sure we got the aliases on all nodes
logger.info("--> indexing with id [1], and routing [0] using alias to test-a"); logger.info("--> indexing with id [1], and routing [0] using alias to test-a");
client().prepareIndex("alias-a0", "type1", "1").setSource("field", "value1").setRefresh(true).execute().actionGet(); client().prepareIndex("alias-a0", "type1", "1").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
logger.info("--> verifying get with no routing, should not find anything"); logger.info("--> verifying get with no routing, should not find anything");
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
assertThat(client().prepareGet("test-a", "type1", "1").execute().actionGet().isExists(), equalTo(false)); assertThat(client().prepareGet("test-a", "type1", "1").execute().actionGet().isExists(), equalTo(false));
@ -218,7 +219,7 @@ public class AliasRoutingIT extends ESIntegTestCase {
} }
logger.info("--> indexing with id [0], and routing [1] using alias to test-b"); logger.info("--> indexing with id [0], and routing [1] using alias to test-b");
client().prepareIndex("alias-b1", "type1", "1").setSource("field", "value1").setRefresh(true).execute().actionGet(); client().prepareIndex("alias-b1", "type1", "1").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
logger.info("--> verifying get with no routing, should not find anything"); logger.info("--> verifying get with no routing, should not find anything");
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
assertThat(client().prepareGet("test-a", "type1", "1").execute().actionGet().isExists(), equalTo(false)); assertThat(client().prepareGet("test-a", "type1", "1").execute().actionGet().isExists(), equalTo(false));
@ -261,9 +262,9 @@ public class AliasRoutingIT extends ESIntegTestCase {
.addAliasAction(newAddAliasAction("index", "index_1").routing("1"))); .addAliasAction(newAddAliasAction("index", "index_1").routing("1")));
logger.info("--> indexing on index_1 which is an alias for index with routing [1]"); logger.info("--> indexing on index_1 which is an alias for index with routing [1]");
client().prepareIndex("index_1", "type1", "1").setSource("field", "value1").setRefresh(true).execute().actionGet(); client().prepareIndex("index_1", "type1", "1").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
logger.info("--> indexing on index_2 which is a concrete index"); logger.info("--> indexing on index_2 which is a concrete index");
client().prepareIndex("index_2", "type2", "2").setSource("field", "value2").setRefresh(true).execute().actionGet(); client().prepareIndex("index_2", "type2", "2").setSource("field", "value2").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
logger.info("--> search all on index_* should find two"); logger.info("--> search all on index_* should find two");
@ -286,9 +287,9 @@ public class AliasRoutingIT extends ESIntegTestCase {
.addAliasAction(newAddAliasAction("index", "index_1").routing("1"))); .addAliasAction(newAddAliasAction("index", "index_1").routing("1")));
logger.info("--> indexing on index_1 which is an alias for index with routing [1]"); logger.info("--> indexing on index_1 which is an alias for index with routing [1]");
client().prepareIndex("index_1", "type1", "1").setSource("field", "value1").setRefresh(true).execute().actionGet(); client().prepareIndex("index_1", "type1", "1").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
logger.info("--> indexing on index_2 which is a concrete index"); logger.info("--> indexing on index_2 which is a concrete index");
client().prepareIndex("index_2", "type2", "2").setSource("field", "value2").setRefresh(true).execute().actionGet(); client().prepareIndex("index_2", "type2", "2").setSource("field", "value2").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
SearchResponse searchResponse = client().prepareSearch("index_*").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(1).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet(); SearchResponse searchResponse = client().prepareSearch("index_*").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(1).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet();
@ -307,7 +308,7 @@ public class AliasRoutingIT extends ESIntegTestCase {
.addAliasAction(newAddAliasAction("test", "alias").routing("3"))); .addAliasAction(newAddAliasAction("test", "alias").routing("3")));
logger.info("--> indexing with id [0], and routing [3]"); logger.info("--> indexing with id [0], and routing [3]");
client().prepareIndex("alias", "type1", "0").setSource("field", "value1").setRefresh(true).execute().actionGet(); client().prepareIndex("alias", "type1", "0").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
logger.info("--> verifying get with no routing, should not find anything"); logger.info("--> verifying get with no routing, should not find anything");
logger.info("--> verifying get and search with routing, should find"); logger.info("--> verifying get and search with routing, should find");
@ -332,7 +333,7 @@ public class AliasRoutingIT extends ESIntegTestCase {
.addAliasAction(newAddAliasAction("test", "alias").searchRouting("3,4").indexRouting("4"))); .addAliasAction(newAddAliasAction("test", "alias").searchRouting("3,4").indexRouting("4")));
logger.info("--> indexing with id [1], and routing [4]"); logger.info("--> indexing with id [1], and routing [4]");
client().prepareIndex("alias", "type1", "1").setSource("field", "value2").setRefresh(true).execute().actionGet(); client().prepareIndex("alias", "type1", "1").setSource("field", "value2").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
logger.info("--> verifying get with no routing, should not find anything"); logger.info("--> verifying get with no routing, should not find anything");
logger.info("--> verifying get and search with routing, should find"); logger.info("--> verifying get and search with routing, should find");

View file

@ -28,6 +28,7 @@ import org.elasticsearch.action.explain.ExplainResponse;
import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.get.GetResponse;
import org.elasticsearch.action.get.MultiGetRequest; import org.elasticsearch.action.get.MultiGetRequest;
import org.elasticsearch.action.get.MultiGetResponse; import org.elasticsearch.action.get.MultiGetResponse;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.action.termvectors.MultiTermVectorsResponse; import org.elasticsearch.action.termvectors.MultiTermVectorsResponse;
import org.elasticsearch.action.termvectors.TermVectorsRequest; import org.elasticsearch.action.termvectors.TermVectorsRequest;
import org.elasticsearch.action.termvectors.TermVectorsResponse; import org.elasticsearch.action.termvectors.TermVectorsResponse;
@ -56,7 +57,8 @@ public class SimpleRoutingIT extends ESIntegTestCase {
ensureGreen(); ensureGreen();
logger.info("--> indexing with id [1], and routing [0]"); logger.info("--> indexing with id [1], and routing [0]");
client().prepareIndex("test", "type1", "1").setRouting("0").setSource("field", "value1").setRefresh(true).execute().actionGet(); client().prepareIndex("test", "type1", "1").setRouting("0").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE)
.get();
logger.info("--> verifying get with no routing, should not find anything"); logger.info("--> verifying get with no routing, should not find anything");
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false)); assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false));
@ -67,21 +69,22 @@ public class SimpleRoutingIT extends ESIntegTestCase {
} }
logger.info("--> deleting with no routing, should not delete anything"); logger.info("--> deleting with no routing, should not delete anything");
client().prepareDelete("test", "type1", "1").setRefresh(true).execute().actionGet(); client().prepareDelete("test", "type1", "1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false)); assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false));
assertThat(client().prepareGet("test", "type1", "1").setRouting("0").execute().actionGet().isExists(), equalTo(true)); assertThat(client().prepareGet("test", "type1", "1").setRouting("0").execute().actionGet().isExists(), equalTo(true));
} }
logger.info("--> deleting with routing, should delete"); logger.info("--> deleting with routing, should delete");
client().prepareDelete("test", "type1", "1").setRouting("0").setRefresh(true).execute().actionGet(); client().prepareDelete("test", "type1", "1").setRouting("0").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false)); assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false));
assertThat(client().prepareGet("test", "type1", "1").setRouting("0").execute().actionGet().isExists(), equalTo(false)); assertThat(client().prepareGet("test", "type1", "1").setRouting("0").execute().actionGet().isExists(), equalTo(false));
} }
logger.info("--> indexing with id [1], and routing [0]"); logger.info("--> indexing with id [1], and routing [0]");
client().prepareIndex("test", "type1", "1").setRouting("0").setSource("field", "value1").setRefresh(true).execute().actionGet(); client().prepareIndex("test", "type1", "1").setRouting("0").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE)
.get();
logger.info("--> verifying get with no routing, should not find anything"); logger.info("--> verifying get with no routing, should not find anything");
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false)); assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false));
@ -97,7 +100,8 @@ public class SimpleRoutingIT extends ESIntegTestCase {
ensureGreen(); ensureGreen();
logger.info("--> indexing with id [1], and routing [0]"); logger.info("--> indexing with id [1], and routing [0]");
client().prepareIndex("test", "type1", "1").setRouting("0").setSource("field", "value1").setRefresh(true).execute().actionGet(); client().prepareIndex("test", "type1", "1").setRouting("0").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE)
.get();
logger.info("--> verifying get with no routing, should not find anything"); logger.info("--> verifying get with no routing, should not find anything");
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false)); assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false));
@ -125,7 +129,7 @@ public class SimpleRoutingIT extends ESIntegTestCase {
} }
logger.info("--> indexing with id [2], and routing [1]"); logger.info("--> indexing with id [2], and routing [1]");
client().prepareIndex("test", "type1", "2").setRouting("1").setSource("field", "value1").setRefresh(true).execute().actionGet(); client().prepareIndex("test", "type1", "2").setRouting("1").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
logger.info("--> search with no routing, should fine two"); logger.info("--> search with no routing, should fine two");
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
@ -165,12 +169,13 @@ public class SimpleRoutingIT extends ESIntegTestCase {
ensureGreen(); ensureGreen();
logger.info("--> indexing with id [1], and routing [0]"); logger.info("--> indexing with id [1], and routing [0]");
client().prepareIndex(indexOrAlias(), "type1", "1").setRouting("0").setSource("field", "value1").setRefresh(true).execute().actionGet(); client().prepareIndex(indexOrAlias(), "type1", "1").setRouting("0").setSource("field", "value1")
.setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
logger.info("--> verifying get with no routing, should fail"); logger.info("--> verifying get with no routing, should fail");
logger.info("--> indexing with id [1], with no routing, should fail"); logger.info("--> indexing with id [1], with no routing, should fail");
try { try {
client().prepareIndex(indexOrAlias(), "type1", "1").setSource("field", "value1").setRefresh(true).execute().actionGet(); client().prepareIndex(indexOrAlias(), "type1", "1").setSource("field", "value1").get();
fail("index with missing routing when routing is required should fail"); fail("index with missing routing when routing is required should fail");
} catch (ElasticsearchException e) { } catch (ElasticsearchException e) {
assertThat(e.unwrapCause(), instanceOf(RoutingMissingException.class)); assertThat(e.unwrapCause(), instanceOf(RoutingMissingException.class));
@ -183,7 +188,7 @@ public class SimpleRoutingIT extends ESIntegTestCase {
logger.info("--> deleting with no routing, should fail"); logger.info("--> deleting with no routing, should fail");
try { try {
client().prepareDelete(indexOrAlias(), "type1", "1").setRefresh(true).execute().actionGet(); client().prepareDelete(indexOrAlias(), "type1", "1").get();
fail("delete with missing routing when routing is required should fail"); fail("delete with missing routing when routing is required should fail");
} catch (ElasticsearchException e) { } catch (ElasticsearchException e) {
assertThat(e.unwrapCause(), instanceOf(RoutingMissingException.class)); assertThat(e.unwrapCause(), instanceOf(RoutingMissingException.class));
@ -223,7 +228,7 @@ public class SimpleRoutingIT extends ESIntegTestCase {
assertThat(getResponse.getSourceAsMap().get("field"), equalTo("value2")); assertThat(getResponse.getSourceAsMap().get("field"), equalTo("value2"));
} }
client().prepareDelete(indexOrAlias(), "type1", "1").setRouting("0").setRefresh(true).execute().actionGet(); client().prepareDelete(indexOrAlias(), "type1", "1").setRouting("0").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
try { try {
@ -320,7 +325,8 @@ public class SimpleRoutingIT extends ESIntegTestCase {
logger.info("--> indexing with id [1], and routing [0]"); logger.info("--> indexing with id [1], and routing [0]");
client().prepareIndex(indexOrAlias(), "type1", "1").setRouting("0").setSource("field", "value1").get(); client().prepareIndex(indexOrAlias(), "type1", "1").setRouting("0").setSource("field", "value1").get();
logger.info("--> indexing with id [2], and routing [0]"); logger.info("--> indexing with id [2], and routing [0]");
client().prepareIndex(indexOrAlias(), "type1", "2").setRouting("0").setSource("field", "value2").setRefresh(true).get(); client().prepareIndex(indexOrAlias(), "type1", "2").setRouting("0").setSource("field", "value2")
.setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
logger.info("--> verifying get with id [1] with routing [0], should succeed"); logger.info("--> verifying get with id [1] with routing [0], should succeed");
assertThat(client().prepareGet(indexOrAlias(), "type1", "1").setRouting("0").execute().actionGet().isExists(), equalTo(true)); assertThat(client().prepareGet(indexOrAlias(), "type1", "1").setRouting("0").execute().actionGet().isExists(), equalTo(true));

View file

@ -26,6 +26,7 @@ import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchType; import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.common.lucene.search.function.CombineFunction; import org.elasticsearch.common.lucene.search.function.CombineFunction;
import org.elasticsearch.common.lucene.search.function.FiltersFunctionScoreQuery; import org.elasticsearch.common.lucene.search.function.FiltersFunctionScoreQuery;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
@ -754,10 +755,11 @@ public class ChildQuerySearchIT extends ESIntegTestCase {
assertNoFailures(response); assertNoFailures(response);
assertThat(response.getHits().totalHits(), equalTo(0L)); assertThat(response.getHits().totalHits(), equalTo(0L));
client().prepareIndex("test", "child1").setSource(jsonBuilder().startObject().field("text", "value").endObject()).setRefresh(true) client().prepareIndex("test", "child1").setSource(jsonBuilder().startObject().field("text", "value").endObject())
.get(); .setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
response = client().prepareSearch("test").setQuery(QueryBuilders.hasChildQuery("child", matchQuery("text", "value"), ScoreMode.None)).get(); response = client().prepareSearch("test")
.setQuery(QueryBuilders.hasChildQuery("child", matchQuery("text", "value"), ScoreMode.None)).get();
assertNoFailures(response); assertNoFailures(response);
assertThat(response.getHits().totalHits(), equalTo(0L)); assertThat(response.getHits().totalHits(), equalTo(0L));

View file

@ -31,6 +31,7 @@ import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.Fuzziness; import org.elasticsearch.common.unit.Fuzziness;
@ -473,7 +474,7 @@ public class CompletionSuggestSearch2xIT extends ESIntegTestCase {
.setSettings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, PRE2X_VERSION.id)) .setSettings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, PRE2X_VERSION.id))
.addMapping(TYPE, mapping)); .addMapping(TYPE, mapping));
client().prepareIndex(INDEX, TYPE, "1") client().prepareIndex(INDEX, TYPE, "1")
.setRefresh(true) .setRefreshPolicy(RefreshPolicy.IMMEDIATE)
.setSource(jsonBuilder().startObject().field(FIELD, "Foo Fighters").endObject()).get(); .setSource(jsonBuilder().startObject().field(FIELD, "Foo Fighters").endObject()).get();
ensureGreen(INDEX); ensureGreen(INDEX);
@ -496,7 +497,7 @@ public class CompletionSuggestSearch2xIT extends ESIntegTestCase {
).execute().actionGet(); ).execute().actionGet();
assertSuggestions(suggestResponse, "suggs"); assertSuggestions(suggestResponse, "suggs");
client().prepareIndex(INDEX, TYPE, "1").setRefresh(true) client().prepareIndex(INDEX, TYPE, "1").setRefreshPolicy(RefreshPolicy.IMMEDIATE)
.setSource(jsonBuilder().startObject().field(FIELD, "Foo Fighters").endObject()).get(); .setSource(jsonBuilder().startObject().field(FIELD, "Foo Fighters").endObject()).get();
ensureGreen(INDEX); ensureGreen(INDEX);
@ -522,7 +523,7 @@ public class CompletionSuggestSearch2xIT extends ESIntegTestCase {
.addMapping(TYPE, mapping) .addMapping(TYPE, mapping)
.setSettings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, PRE2X_VERSION.id))); .setSettings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, PRE2X_VERSION.id)));
client().prepareIndex(INDEX, TYPE, "1") client().prepareIndex(INDEX, TYPE, "1")
.setRefresh(true) .setRefreshPolicy(RefreshPolicy.IMMEDIATE)
.setSource(jsonBuilder().startObject().field(FIELD, "Foo Fighters").endObject()).get(); .setSource(jsonBuilder().startObject().field(FIELD, "Foo Fighters").endObject()).get();
ensureGreen(INDEX); ensureGreen(INDEX);
@ -545,7 +546,7 @@ public class CompletionSuggestSearch2xIT extends ESIntegTestCase {
).execute().actionGet(); ).execute().actionGet();
assertSuggestions(suggestResponse, "suggs"); assertSuggestions(suggestResponse, "suggs");
client().prepareIndex(INDEX, TYPE, "1").setRefresh(true) client().prepareIndex(INDEX, TYPE, "1").setRefreshPolicy(RefreshPolicy.IMMEDIATE)
.setSource(jsonBuilder().startObject().field(FIELD, "Foo Fighters").endObject()).get(); .setSource(jsonBuilder().startObject().field(FIELD, "Foo Fighters").endObject()).get();
ensureGreen(INDEX); ensureGreen(INDEX);
@ -731,10 +732,10 @@ public class CompletionSuggestSearch2xIT extends ESIntegTestCase {
assertThat(putMappingResponse.isAcknowledged(), is(true)); assertThat(putMappingResponse.isAcknowledged(), is(true));
// Index two entities // Index two entities
client().prepareIndex(INDEX, TYPE, "1").setRefresh(true) client().prepareIndex(INDEX, TYPE, "1").setRefreshPolicy(RefreshPolicy.IMMEDIATE)
.setSource(jsonBuilder().startObject().field(FIELD, "Foo Fighters").field(otherField, "WHATEVER").endObject()) .setSource(jsonBuilder().startObject().field(FIELD, "Foo Fighters").field(otherField, "WHATEVER").endObject())
.get(); .get();
client().prepareIndex(INDEX, TYPE, "2").setRefresh(true) client().prepareIndex(INDEX, TYPE, "2").setRefreshPolicy(RefreshPolicy.IMMEDIATE)
.setSource(jsonBuilder().startObject().field(FIELD, "Bar Fighters").field(otherField, "WHATEVER2").endObject()) .setSource(jsonBuilder().startObject().field(FIELD, "Bar Fighters").field(otherField, "WHATEVER2").endObject())
.get(); .get();
@ -1040,7 +1041,7 @@ public class CompletionSuggestSearch2xIT extends ESIntegTestCase {
.startArray("input").value(str).endArray() .startArray("input").value(str).endArray()
.field("output", "foobar") .field("output", "foobar")
.endObject().endObject() .endObject().endObject()
).setRefresh(true).get(); ).setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
// need to flush and refresh, because we keep changing the same document // need to flush and refresh, because we keep changing the same document
// we have to make sure that segments without any live documents are deleted // we have to make sure that segments without any live documents are deleted
flushAndRefresh(); flushAndRefresh();
@ -1074,7 +1075,7 @@ public class CompletionSuggestSearch2xIT extends ESIntegTestCase {
.startArray("input").value(longString).endArray() .startArray("input").value(longString).endArray()
.field("output", "foobar") .field("output", "foobar")
.endObject().endObject() .endObject().endObject()
).setRefresh(true).get(); ).setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
} }
@ -1096,7 +1097,7 @@ public class CompletionSuggestSearch2xIT extends ESIntegTestCase {
.startArray("input").value(string).endArray() .startArray("input").value(string).endArray()
.field("output", "foobar") .field("output", "foobar")
.endObject().endObject() .endObject().endObject()
).setRefresh(true).get(); ).setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
fail("expected MapperParsingException"); fail("expected MapperParsingException");
} catch (MapperParsingException expected) {} } catch (MapperParsingException expected) {}
} }
@ -1116,7 +1117,7 @@ public class CompletionSuggestSearch2xIT extends ESIntegTestCase {
.startObject() .startObject()
.field(FIELD, string) .field(FIELD, string)
.endObject() .endObject()
).setRefresh(true).get(); ).setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
try { try {
client().prepareSearch(INDEX).addAggregation(AggregationBuilders.terms("suggest_agg").field(FIELD) client().prepareSearch(INDEX).addAggregation(AggregationBuilders.terms("suggest_agg").field(FIELD)
@ -1148,11 +1149,11 @@ public class CompletionSuggestSearch2xIT extends ESIntegTestCase {
ensureGreen(); ensureGreen();
client().prepareIndex(INDEX, TYPE, "1").setSource(FIELD, "strings make me happy", FIELD + "_1", "nulls make me sad") client().prepareIndex(INDEX, TYPE, "1").setSource(FIELD, "strings make me happy", FIELD + "_1", "nulls make me sad")
.setRefresh(true).get(); .setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
try { try {
client().prepareIndex(INDEX, TYPE, "2").setSource(FIELD, null, FIELD + "_1", "nulls make me sad") client().prepareIndex(INDEX, TYPE, "2").setSource(FIELD, null, FIELD + "_1", "nulls make me sad")
.setRefresh(true).get(); .setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
fail("Expected MapperParsingException for null value"); fail("Expected MapperParsingException for null value");
} catch (MapperParsingException e) { } catch (MapperParsingException e) {
// make sure that the exception has the name of the field causing the error // make sure that the exception has the name of the field causing the error

View file

@ -15,6 +15,8 @@ This section describes the following CRUD APIs:
.Multi-document APIs .Multi-document APIs
* <<docs-multi-get>> * <<docs-multi-get>>
* <<docs-bulk>> * <<docs-bulk>>
* <<docs-update-by-query>>
* <<docs-reindex>>
NOTE: All CRUD APIs are single-index APIs. The `index` parameter accepts a single NOTE: All CRUD APIs are single-index APIs. The `index` parameter accepts a single
index name, or an `alias` which points to a single index. index name, or an `alias` which points to a single index.
@ -42,3 +44,5 @@ include::docs/reindex.asciidoc[]
include::docs/termvectors.asciidoc[] include::docs/termvectors.asciidoc[]
include::docs/multi-termvectors.asciidoc[] include::docs/multi-termvectors.asciidoc[]
include::docs/refresh.asciidoc[]

View file

@ -167,12 +167,8 @@ are the same).
[[bulk-refresh]] [[bulk-refresh]]
=== Refresh === Refresh
The `refresh` parameter can be set to `true` in order to refresh the relevant Control when the changes made by this request are visible to search. See
primary and replica shards immediately after the bulk operation has occurred <<docs-refresh>>.
and make it searchable, instead of waiting for the normal refresh interval to
expire. Setting it to `true` can trigger additional load, and may slow down
indexing. Due to its costly nature, the `refresh` parameter is set on the bulk request level
and is not supported on each individual bulk item.
[float] [float]
[[bulk-update]] [[bulk-update]]

View file

@ -113,11 +113,9 @@ is the same).
[[delete-refresh]] [[delete-refresh]]
=== Refresh === Refresh
The `refresh` parameter can be set to `true` in order to refresh the relevant Control when the changes made by this request are visible to search. See
primary and replica shards after the delete operation has occurred and make it <<docs-refresh>>.
searchable. Setting it to `true` should be done after careful thought and
verification that this does not cause a heavy load on the system (and slows
down indexing).
[float] [float]
[[delete-timeout]] [[delete-timeout]]

View file

@ -30,7 +30,8 @@ The result of the above index operation is:
"_type" : "tweet", "_type" : "tweet",
"_id" : "1", "_id" : "1",
"_version" : 1, "_version" : 1,
"created" : true "created" : true,
"forced_refresh": false
} }
-------------------------------------------------- --------------------------------------------------
// TESTRESPONSE[s/"successful" : 2/"successful" : 1/] // TESTRESPONSE[s/"successful" : 2/"successful" : 1/]
@ -221,7 +222,8 @@ The result of the above index operation is:
"_type" : "tweet", "_type" : "tweet",
"_id" : "6a8ca01c-7896-48e9-81cc-9f70661fcb32", "_id" : "6a8ca01c-7896-48e9-81cc-9f70661fcb32",
"_version" : 1, "_version" : 1,
"created" : true "created" : true,
"forced_refresh": false
} }
-------------------------------------------------- --------------------------------------------------
// TESTRESPONSE[s/6a8ca01c-7896-48e9-81cc-9f70661fcb32/$body._id/ s/"successful" : 2/"successful" : 1/] // TESTRESPONSE[s/6a8ca01c-7896-48e9-81cc-9f70661fcb32/$body._id/ s/"successful" : 2/"successful" : 1/]
@ -385,13 +387,8 @@ replication group have indexed the document (sync replication).
[[index-refresh]] [[index-refresh]]
=== Refresh === Refresh
To refresh the shard (not the whole index) immediately after the operation Control when the changes made by this request are visible to search. See
occurs, so that the document appears in search results immediately, the <<docs-refresh>>.
`refresh` parameter can be set to `true`. Setting this option to `true` should
*ONLY* be done after careful thought and verification that it does not lead to
poor performance, both from an indexing and a search standpoint. Note, getting
a document using the get API is completely realtime and doesn't require a
refresh.
[float] [float]
[[index-noop]] [[index-noop]]

View file

@ -0,0 +1,109 @@
[[docs-refresh]]
== `?refresh`
The <<docs-index_,Index>>, <<docs-update,Update>>, <<docs-delete,Delete>>, and
<<docs-bulk,Bulk>> APIs support setting `refresh` to control when changes made
by this request are made visible to search. These are the allowed values:
Empty string or `true`::
Refresh the relevant primary and replica shards (not the whole index)
immediately after the operation occurs, so that the updated document appears
in search results immediately. This should *ONLY* be done after careful thought
and verification that it does not lead to poor performance, both from an
indexing and a search standpoint.
`wait_for`::
Wait for the changes made by the request to be made visible by a refresh before
replying. This doesn't force an immediate refresh, rather, it waits for a
refresh happen. Elasticsearch automatically refreshes shards that have changed
every `index.refresh_interval` which defaults to one second. That setting is
<<dynamic-index-settings,dynamic>>. The <<indices-refresh>> API will also
cause the request to return, as will setting `refresh` to `true` on any of the
APIs that support it.
`false` (the default)::
Take no refresh related actions. The changes made by this request will be made
visible at some point after the request returns.
=== Choosing which setting to use
Unless you have a good reason to wait for the change to become visible always
use `refresh=false`, or, because that is the default, just leave the `refresh`
parameter out of the URL. That is the simplest and fastest choice.
If you absolutely must have the changes made by a request visible synchronously
with the request then you must get to pick between putting more load on
Elasticsearch (`true`) and waiting longer for the response (`wait_for`). Here
are a few points that should inform that decision:
* The more changes being made to the index the more work `wait_for` saves
compared to `true`. In the case that the index is only changed once every
`index.refresh_interval` then it saves no work.
* `true` creates less efficient indexes constructs (tiny segments) that must
later be merged into more efficient index constructs (larger segments). Meaning
that the cost of `true` is payed at index time to create the tiny segment, at
search time to search the tiny segment, and at merge time to make the larger
segments.
* Never start multiple `refresh=wait_for` requests in a row. Instead batch them
into a single bulk request with `refresh=wait_for` and Elasticsearch will start
them all in parallel and return only when they have all finished.
* If the refresh interval is set to `-1`, disabling the automatic refreshes,
then requests with `refresh=wait_for` will wait indefinitely until some action
causes a refresh. Conversely, setting `index.refresh_interval` to something
shorter than the default like `200ms` will make `refresh=wait_for` come back
faster, but it'll still generate inefficient segments.
* `refresh=wait_for` only affects the request that it is on, but, by forcing a
refresh immediately, `refresh=true` will affect other ongoing request. In
general, if you have a running system you don't wish to disturb then
`refresh=wait_for` is a smaller modification.
=== `refresh=wait_for` Can Force a Refresh
If a `refresh=wait_for` request comes in when there are already
`index.max_refresh_listeners` (defaults to 1000) requests waiting for a refresh
on that shard then that request will behave just as though it had `refresh` set
to `true` instead: it will force a refresh. This keeps the promise that when a
`refresh=wait_for` request returns that its changes are visible for search
while preventing unchecked resource usage for blocked requests. If a request
forced a refresh because it ran out of listener slots then its response will
contain `"forced_refresh": true`.
Bulk requests only take up one slot on each shard that they touch no matter how
many times they modify the shard.
=== Examples
These will create a document and immediately refresh the index so it is visible:
[source,js]
--------------------------------------------------
PUT /test/test/1?refresh
{"test": "test"}
PUT /test/test/2?refresh=true
{"test": "test"}
--------------------------------------------------
// CONSOLE
These will create a document without doing anything to make it visible for
search:
[source,js]
--------------------------------------------------
PUT /test/test/3
{"test": "test"}
PUT /test/test/4?refresh=true
{"test": "test"}
--------------------------------------------------
// CONSOLE
This will create a document and wait for it to become visible for search:
[source,js]
--------------------------------------------------
PUT /test/test/4?refresh=wait_for
{"test": "test"}
--------------------------------------------------
// CONSOLE

View file

@ -235,9 +235,8 @@ The write consistency of the index/delete operation.
`refresh`:: `refresh`::
Refresh the relevant primary and replica shards (not the whole index) Control when the changes made by this request are visible to search. See
immediately after the operation occurs, so that the updated document appears <<docs-refresh>>.
in search results immediately.
`fields`:: `fields`::

View file

@ -136,6 +136,11 @@ specific index module:
experimental[] Disables the purge of <<mapping-ttl-field,expired docs>> on experimental[] Disables the purge of <<mapping-ttl-field,expired docs>> on
the current index. the current index.
`index.max_refresh_listeners`::
Maximum number of refresh listeners available on each shard of the index.
These listeners are used to implement <<docs-refresh,`refresh=wait_for`>>.
[float] [float]
=== Settings in other index modules === Settings in other index modules

View file

@ -1,6 +1,16 @@
[[breaking_50_document_api_changes]] [[breaking_50_document_api_changes]]
=== Document API changes === Document API changes
==== `?refresh` no longer supports truthy and falsy values
The `?refresh` request parameter used to accept any value other than `false`,
`0`, `off`, and `no` to mean "make the changes from this request visible for
search immediately." Now it only accepts `?refresh` and `?refresh=true` to
mean that. You can set it to `?refresh=false` and the request will take no
refresh-related action. The same is true if you leave `refresh` off of the
url entirely. If you add `?refresh=wait_for` Elasticsearch will wait for the
changes to become visible before replying to the request but won't take any
immediate refresh related action. See <<docs-refresh>>.
==== Reindex and Update By Query ==== Reindex and Update By Query
Before 5.0.0 `_reindex` and `_update_by_query` only retried bulk failures so Before 5.0.0 `_reindex` and `_update_by_query` only retried bulk failures so
they used the following response format: they used the following response format:

View file

@ -304,3 +304,8 @@ The `setQuery(BytesReference)` method have been removed in favor of using `setQu
Removed the `getMemoryAvailable` method from `OsStats`, which could be previously accessed calling Removed the `getMemoryAvailable` method from `OsStats`, which could be previously accessed calling
`clusterStatsResponse.getNodesStats().getOs().getMemoryAvailable()`. `clusterStatsResponse.getNodesStats().getOs().getMemoryAvailable()`.
=== setRefresh(boolean) has been deprecated
`setRefresh(boolean)` has been deprecated in favor of `setRefreshPolicy(RefreshPolicy)` because there
are now three options. It will be removed in 5.0.

View file

@ -29,6 +29,7 @@ import org.elasticsearch.action.get.GetResponse;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.action.update.UpdateRequest;
import org.elasticsearch.action.update.UpdateRequestBuilder; import org.elasticsearch.action.update.UpdateRequestBuilder;
import org.elasticsearch.action.update.UpdateResponse; import org.elasticsearch.action.update.UpdateResponse;
@ -584,7 +585,7 @@ public class BulkTests extends ESIntegTestCase {
.add(new IndexRequest("test", "type", "4").source("{ \"title\" : \"Great Title of doc 4\" }")) .add(new IndexRequest("test", "type", "4").source("{ \"title\" : \"Great Title of doc 4\" }"))
.add(new IndexRequest("test", "type", "5").source("{ \"title\" : \"Great Title of doc 5\" }")) .add(new IndexRequest("test", "type", "5").source("{ \"title\" : \"Great Title of doc 5\" }"))
.add(new IndexRequest("test", "type", "6").source("{ \"title\" : \"Great Title of doc 6\" }")) .add(new IndexRequest("test", "type", "6").source("{ \"title\" : \"Great Title of doc 6\" }"))
.setRefresh(true) .setRefreshPolicy(RefreshPolicy.IMMEDIATE)
.get(); .get();
assertNoFailures(indexBulkItemResponse); assertNoFailures(indexBulkItemResponse);
@ -622,7 +623,7 @@ public class BulkTests extends ESIntegTestCase {
.add(new IndexRequest("bulkindex2", "index2_type").source("text", "hallo2")) .add(new IndexRequest("bulkindex2", "index2_type").source("text", "hallo2"))
.add(new UpdateRequest("bulkindex2", "index2_type", "2").doc("foo", "bar")) .add(new UpdateRequest("bulkindex2", "index2_type", "2").doc("foo", "bar"))
.add(new DeleteRequest("bulkindex2", "index2_type", "3")) .add(new DeleteRequest("bulkindex2", "index2_type", "3"))
.refresh(true); .setRefreshPolicy(RefreshPolicy.IMMEDIATE);
client().bulk(bulkRequest).get(); client().bulk(bulkRequest).get();
SearchResponse searchResponse = client().prepareSearch("bulkindex*").get(); SearchResponse searchResponse = client().prepareSearch("bulkindex*").get();
@ -643,10 +644,10 @@ public class BulkTests extends ESIntegTestCase {
client().prepareIndex("bulkindex1", "index1_type", "1").setSource("text", "test").get(); client().prepareIndex("bulkindex1", "index1_type", "1").setSource("text", "test").get();
assertAcked(client().admin().indices().prepareClose("bulkindex1")); assertAcked(client().admin().indices().prepareClose("bulkindex1"));
BulkRequest bulkRequest = new BulkRequest(); BulkRequest bulkRequest = new BulkRequest().setRefreshPolicy(RefreshPolicy.IMMEDIATE);
bulkRequest.add(new IndexRequest("bulkindex1", "index1_type", "1").source("text", "hallo1")) bulkRequest.add(new IndexRequest("bulkindex1", "index1_type", "1").source("text", "hallo1"))
.add(new UpdateRequest("bulkindex1", "index1_type", "1").doc("foo", "bar")) .add(new UpdateRequest("bulkindex1", "index1_type", "1").doc("foo", "bar"))
.add(new DeleteRequest("bulkindex1", "index1_type", "1")).refresh(true); .add(new DeleteRequest("bulkindex1", "index1_type", "1"));
BulkResponse bulkResponse = client().bulk(bulkRequest).get(); BulkResponse bulkResponse = client().bulk(bulkRequest).get();
assertThat(bulkResponse.hasFailures(), is(true)); assertThat(bulkResponse.hasFailures(), is(true));

View file

@ -26,6 +26,7 @@ import org.elasticsearch.action.admin.indices.alias.Alias;
import org.elasticsearch.action.admin.indices.alias.IndicesAliasesResponse; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesResponse;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.client.Requests; import org.elasticsearch.client.Requests;
import org.elasticsearch.common.geo.builders.ShapeBuilders; import org.elasticsearch.common.geo.builders.ShapeBuilders;
import org.elasticsearch.common.lucene.search.function.CombineFunction; import org.elasticsearch.common.lucene.search.function.CombineFunction;
@ -294,8 +295,8 @@ public class PercolatorIT extends ESIntegTestCase {
.field("color", "blue") .field("color", "blue")
.field("query", termQuery("field1", "value1")) .field("query", termQuery("field1", "value1"))
.endObject()) .endObject())
.setRefresh(true) .setRefreshPolicy(RefreshPolicy.IMMEDIATE)
.execute().actionGet(); .get();
cluster().wipeIndices("test"); cluster().wipeIndices("test");
createIndex("test"); createIndex("test");
@ -308,8 +309,8 @@ public class PercolatorIT extends ESIntegTestCase {
.field("color", "blue") .field("color", "blue")
.field("query", termQuery("field1", "value1")) .field("query", termQuery("field1", "value1"))
.endObject()) .endObject())
.setRefresh(true) .setRefreshPolicy(RefreshPolicy.IMMEDIATE)
.execute().actionGet(); .get();
} }
// see #2814 // see #2814
@ -338,8 +339,8 @@ public class PercolatorIT extends ESIntegTestCase {
.field("source", "productizer") .field("source", "productizer")
.field("query", QueryBuilders.constantScoreQuery(QueryBuilders.queryStringQuery("filingcategory:s"))) .field("query", QueryBuilders.constantScoreQuery(QueryBuilders.queryStringQuery("filingcategory:s")))
.endObject()) .endObject())
.setRefresh(true) .setRefreshPolicy(RefreshPolicy.IMMEDIATE)
.execute().actionGet(); .get();
refresh(); refresh();
PercolateResponse percolate = preparePercolate(client()) PercolateResponse percolate = preparePercolate(client())
@ -417,8 +418,8 @@ public class PercolatorIT extends ESIntegTestCase {
.field("color", "blue") .field("color", "blue")
.field("query", termQuery("field1", "value1")) .field("query", termQuery("field1", "value1"))
.endObject()) .endObject())
.setRefresh(true) .setRefreshPolicy(RefreshPolicy.IMMEDIATE)
.execute().actionGet(); .get();
logger.info("--> register a query 2"); logger.info("--> register a query 2");
client().prepareIndex(INDEX_NAME, TYPE_NAME, "bubu") client().prepareIndex(INDEX_NAME, TYPE_NAME, "bubu")
@ -426,8 +427,8 @@ public class PercolatorIT extends ESIntegTestCase {
.field("color", "green") .field("color", "green")
.field("query", termQuery("field1", "value2")) .field("query", termQuery("field1", "value2"))
.endObject()) .endObject())
.setRefresh(true) .setRefreshPolicy(RefreshPolicy.IMMEDIATE)
.execute().actionGet(); .get();
PercolateResponse percolate = preparePercolate(client()) PercolateResponse percolate = preparePercolate(client())
.setIndices(INDEX_NAME).setDocumentType("type1") .setIndices(INDEX_NAME).setDocumentType("type1")
@ -461,8 +462,8 @@ public class PercolatorIT extends ESIntegTestCase {
.field("color", "blue") .field("color", "blue")
.field("query", termQuery("field1", "value1")) .field("query", termQuery("field1", "value1"))
.endObject()) .endObject())
.setRefresh(true) .setRefreshPolicy(RefreshPolicy.IMMEDIATE)
.execute().actionGet(); .get();
PercolateResponse percolate = preparePercolate(client()) PercolateResponse percolate = preparePercolate(client())
.setIndices(INDEX_NAME).setDocumentType("type1") .setIndices(INDEX_NAME).setDocumentType("type1")
@ -478,8 +479,8 @@ public class PercolatorIT extends ESIntegTestCase {
.field("color", "green") .field("color", "green")
.field("query", termQuery("field1", "value2")) .field("query", termQuery("field1", "value2"))
.endObject()) .endObject())
.setRefresh(true) .setRefreshPolicy(RefreshPolicy.IMMEDIATE)
.execute().actionGet(); .get();
percolate = preparePercolate(client()) percolate = preparePercolate(client())
.setIndices(INDEX_NAME).setDocumentType("type1") .setIndices(INDEX_NAME).setDocumentType("type1")
@ -495,8 +496,8 @@ public class PercolatorIT extends ESIntegTestCase {
.field("color", "red") .field("color", "red")
.field("query", termQuery("field1", "value2")) .field("query", termQuery("field1", "value2"))
.endObject()) .endObject())
.setRefresh(true) .setRefreshPolicy(RefreshPolicy.IMMEDIATE)
.execute().actionGet(); .get();
PercolateSourceBuilder sourceBuilder = new PercolateSourceBuilder() PercolateSourceBuilder sourceBuilder = new PercolateSourceBuilder()
.setDoc(docBuilder().setDoc(jsonBuilder().startObject().field("field1", "value2").endObject())) .setDoc(docBuilder().setDoc(jsonBuilder().startObject().field("field1", "value2").endObject()))
@ -510,7 +511,7 @@ public class PercolatorIT extends ESIntegTestCase {
assertThat(convertFromTextArray(percolate.getMatches(), INDEX_NAME), arrayContaining("susu")); assertThat(convertFromTextArray(percolate.getMatches(), INDEX_NAME), arrayContaining("susu"));
logger.info("--> deleting query 1"); logger.info("--> deleting query 1");
client().prepareDelete(INDEX_NAME, TYPE_NAME, "kuku").setRefresh(true).execute().actionGet(); client().prepareDelete(INDEX_NAME, TYPE_NAME, "kuku").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
percolate = preparePercolate(client()) percolate = preparePercolate(client())
.setIndices(INDEX_NAME).setDocumentType("type1") .setIndices(INDEX_NAME).setDocumentType("type1")
@ -1461,8 +1462,8 @@ public class PercolatorIT extends ESIntegTestCase {
.must(QueryBuilders.queryStringQuery("root")) .must(QueryBuilders.queryStringQuery("root"))
.must(QueryBuilders.termQuery("message", "tree")))) .must(QueryBuilders.termQuery("message", "tree"))))
.endObject()) .endObject())
.setRefresh(true) .setRefreshPolicy(RefreshPolicy.IMMEDIATE)
.execute().actionGet(); .get();
refresh(); refresh();
PercolateResponse percolate = preparePercolate(client()) PercolateResponse percolate = preparePercolate(client())

View file

@ -22,8 +22,9 @@
"description" : "Explicit write consistency setting for the operation" "description" : "Explicit write consistency setting for the operation"
}, },
"refresh": { "refresh": {
"type" : "boolean", "type" : "enum",
"description" : "Refresh the index after performing the operation" "options": ["true", "false", "wait_for"],
"description" : "If `true` then refresh the effected shards to make this operation visible to search, if `wait_for` then wait for a refresh to make this operation visible to search, if `false` (the default) then do nothing with refreshes."
}, },
"routing": { "routing": {
"type" : "string", "type" : "string",

View file

@ -33,8 +33,9 @@
"description" : "ID of parent document" "description" : "ID of parent document"
}, },
"refresh": { "refresh": {
"type" : "boolean", "type" : "enum",
"description" : "Refresh the index after performing the operation" "options": ["true", "false", "wait_for"],
"description" : "If `true` then refresh the effected shards to make this operation visible to search, if `wait_for` then wait for a refresh to make this operation visible to search, if `false` (the default) then do nothing with refreshes."
}, },
"routing": { "routing": {
"type" : "string", "type" : "string",

View file

@ -38,8 +38,9 @@
"description" : "ID of the parent document" "description" : "ID of the parent document"
}, },
"refresh": { "refresh": {
"type" : "boolean", "type" : "enum",
"description" : "Refresh the index after performing the operation" "options": ["true", "false", "wait_for"],
"description" : "If `true` then refresh the effected shards to make this operation visible to search, if `wait_for` then wait for a refresh to make this operation visible to search, if `false` (the default) then do nothing with refreshes."
}, },
"routing": { "routing": {
"type" : "string", "type" : "string",

View file

@ -41,8 +41,9 @@
"description": "ID of the parent document. Is is only used for routing and when for the upsert request" "description": "ID of the parent document. Is is only used for routing and when for the upsert request"
}, },
"refresh": { "refresh": {
"type": "boolean", "type" : "enum",
"description": "Refresh the index after performing the operation" "options": ["true", "false", "wait_for"],
"description" : "If `true` then refresh the effected shards to make this operation visible to search, if `wait_for` then wait for a refresh to make this operation visible to search, if `false` (the default) then do nothing with refreshes."
}, },
"retry_on_conflict": { "retry_on_conflict": {
"type": "number", "type": "number",

View file

@ -0,0 +1,48 @@
---
"refresh=true immediately makes changes are visible in search":
- do:
bulk:
refresh: true
body: |
{"index": {"_index": "test_index", "_type": "test_type", "_id": "test_id"}}
{"f1": "v1", "f2": 42}
{"index": {"_index": "test_index", "_type": "test_type", "_id": "test_id2"}}
{"f1": "v2", "f2": 47}
- do:
count:
index: test_index
- match: {count: 2}
---
"refresh=empty string immediately makes changes are visible in search":
- do:
bulk:
refresh: ""
body: |
{"index": {"_index": "test_index", "_type": "test_type", "_id": "test_id"}}
{"f1": "v1", "f2": 42}
{"index": {"_index": "test_index", "_type": "test_type", "_id": "test_id2"}}
{"f1": "v2", "f2": 47}
- do:
count:
index: test_index
- match: {count: 2}
---
"refresh=wait_for waits until changes are visible in search":
- do:
bulk:
refresh: wait_for
body: |
{"index": {"_index": "test_index", "_type": "test_type", "_id": "test_id"}}
{"f1": "v1", "f2": 42}
{"index": {"_index": "test_index", "_type": "test_type", "_id": "test_id2"}}
{"f1": "v2", "f2": 47}
- do:
count:
index: test_index
- match: {count: 2}

View file

@ -33,8 +33,9 @@
index: test_1 index: test_1
type: test type: test
id: 2 id: 2
refresh: 1 refresh: true
body: { foo: bar } body: { foo: bar }
- is_true: forced_refresh
- do: - do:
search: search:
@ -44,3 +45,42 @@
query: { term: { _id: 2 }} query: { term: { _id: 2 }}
- match: { hits.total: 1 } - match: { hits.total: 1 }
---
"When refresh url parameter is an empty string that means \"refresh immediately\"":
- do:
create:
index: test_1
type: test
id: 1
refresh: ""
body: { foo: bar }
- is_true: forced_refresh
- do:
search:
index: test_1
type: test
body:
query: { term: { _id: 1 }}
- match: { hits.total: 1 }
---
"refresh=wait_for waits until changes are visible in search":
- do:
index:
index: test_1
type: test
id: 1
body: { foo: bar }
refresh: wait_for
- is_false: forced_refresh
- do:
search:
index: test_1
type: test
body:
query: { term: { _id: 1 }}
- match: { hits.total: 1 }

View file

@ -19,7 +19,7 @@
type: test type: test
id: 1 id: 1
body: { foo: bar } body: { foo: bar }
refresh: 1 refresh: true
# If you wonder why this document get 3 as an id instead of 2, it is because the # If you wonder why this document get 3 as an id instead of 2, it is because the
# current routing algorithm would route 1 and 2 to the same shard while we need # current routing algorithm would route 1 and 2 to the same shard while we need
@ -30,7 +30,8 @@
type: test type: test
id: 3 id: 3
body: { foo: bar } body: { foo: bar }
refresh: 1 refresh: true
- is_true: forced_refresh
- do: - do:
search: search:
@ -61,7 +62,7 @@
index: test_1 index: test_1
type: test type: test
id: 3 id: 3
refresh: 1 refresh: true
# If a replica shard where doc 1 is located gets initialized at this point, doc 1 # If a replica shard where doc 1 is located gets initialized at this point, doc 1
# won't be found by the following search as the shard gets automatically refreshed # won't be found by the following search as the shard gets automatically refreshed
@ -75,3 +76,72 @@
query: { terms: { _id: [1,3] }} query: { terms: { _id: [1,3] }}
- match: { hits.total: 1 } - match: { hits.total: 1 }
---
"When refresh url parameter is an empty string that means \"refresh immediately\"":
- do:
index:
index: test_1
type: test
id: 1
body: { foo: bar }
refresh: true
- is_true: forced_refresh
- do:
search:
index: test_1
type: test
body:
query: { term: { _id: 1 }}
- match: { hits.total: 1 }
- do:
delete:
index: test_1
type: test
id: 1
refresh: ""
- do:
search:
index: test_1
type: test
body:
query: { term: { _id: 1 }}
- match: { hits.total: 0 }
---
"refresh=wait_for waits until changes are visible in search":
- do:
index:
index: test_1
type: test
id: 1
body: { foo: bar }
refresh: true
- is_true: forced_refresh
- do:
search:
index: test_1
type: test
body:
query: { term: { _id: 1 }}
- match: { hits.total: 1 }
- do:
delete:
index: test_1
type: test
id: 1
refresh: wait_for
- is_false: forced_refresh
- do:
search:
index: test_1
type: test
body:
query: { term: { _id: 1 }}
- match: { hits.total: 0 }

View file

@ -33,8 +33,9 @@
index: test_1 index: test_1
type: test type: test
id: 2 id: 2
refresh: 1 refresh: true
body: { foo: bar } body: { foo: bar }
- is_true: forced_refresh
- do: - do:
search: search:
@ -44,3 +45,42 @@
query: { term: { _id: 2 }} query: { term: { _id: 2 }}
- match: { hits.total: 1 } - match: { hits.total: 1 }
---
"When refresh url parameter is an empty string that means \"refresh immediately\"":
- do:
index:
index: test_1
type: test
id: 1
refresh: ""
body: { foo: bar }
- is_true: forced_refresh
- do:
search:
index: test_1
type: test
body:
query: { term: { _id: 1 }}
- match: { hits.total: 1 }
---
"refresh=wait_for waits until changes are visible in search":
- do:
index:
index: test_1
type: test
id: 1
body: { foo: bar }
refresh: wait_for
- is_false: forced_refresh
- do:
search:
index: test_1
type: test
body:
query: { term: { _id: 1 }}
- match: { hits.total: 1 }

View file

@ -35,10 +35,11 @@
index: test_1 index: test_1
type: test type: test
id: 2 id: 2
refresh: 1 refresh: true
body: body:
doc: { foo: baz } doc: { foo: baz }
upsert: { foo: bar } upsert: { foo: bar }
- is_true: forced_refresh
- do: - do:
search: search:
@ -48,3 +49,70 @@
query: { term: { _id: 2 }} query: { term: { _id: 2 }}
- match: { hits.total: 1 } - match: { hits.total: 1 }
---
"When refresh url parameter is an empty string that means \"refresh immediately\"":
- do:
index:
index: test_1
type: test
id: 1
refresh: true
body: { foo: bar }
- is_true: forced_refresh
- do:
update:
index: test_1
type: test
id: 1
refresh: ""
body:
doc: {cat: dog}
- is_true: forced_refresh
- do:
search:
index: test_1
type: test
body:
query: { term: { cat: dog }}
- match: { hits.total: 1 }
---
"refresh=wait_for waits until changes are visible in search":
- do:
index:
index: test_1
type: test
id: 1
body: { foo: bar }
refresh: true
- is_true: forced_refresh
- do:
search:
index: test_1
type: test
body:
query: { term: { _id: 1 }}
- match: { hits.total: 1 }
- do:
update:
index: test_1
type: test
id: 1
refresh: wait_for
body:
doc: { test: asdf }
- is_false: forced_refresh
- do:
search:
index: test_1
type: test
body:
query: { match: { test: asdf } }
- match: { hits.total: 1 }