diff --git a/.thumbs.yml b/.thumbs.yml
new file mode 100644
index 00000000..5408ebc6
--- /dev/null
+++ b/.thumbs.yml
@@ -0,0 +1,9 @@
+minimum_reviewers: 2
+build_steps:
+ - make clean
+ - make test
+ - make xref
+ - make dialyzer
+merge: false
+org_mode: true
+timeout: 1790
diff --git a/Makefile b/Makefile
index 035e7384..1cf2cb8c 100644
--- a/Makefile
+++ b/Makefile
@@ -26,13 +26,6 @@ clean:
distclean: clean
$(REBAR) delete-deps
-# You should 'clean' before your first run of this target
-# so that deps get built with PULSE where needed.
-pulse:
- ./rebar compile -D PULSE
- ./rebar eunit -D PULSE skip_deps=true suite=$(PULSE_TESTS)
-
-
##
## Dialyzer
##
diff --git a/debug/yz_perf.erl b/debug/yz_perf.erl
deleted file mode 100644
index c2ceb628..00000000
--- a/debug/yz_perf.erl
+++ /dev/null
@@ -1,74 +0,0 @@
-%% -------------------------------------------------------------------
-%% Copyright (c) 2016 Basho Technologies, Inc. All Rights Reserved.
-%%
-%% This file is provided 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.
-%%
-%% -------------------------------------------------------------------
--module(yz_perf).
-
-%% API
--export([resize_workers/1, resize_helpers/1]).
-
-
-%% @doc Resize the number of queues. For debugging/testing only,
-%% this will briefly cause the worker that queues remap to
-%% to change so updates may be out of order briefly.
--spec resize_workers(NewSize :: pos_integer()) -> yz_solrq:size_resps().
-resize_workers(NewSize) when NewSize > 0 ->
- do_child_resize(NewSize, yz_solrq:num_worker_specs(), worker).
-
-%% @doc Resize the number of helpers. For debugging/testing only,
-%% this will briefly cause the worker that queues remap to
-%% to change so updates may be out of order briefly.
--spec resize_helpers(NewSize :: pos_integer()) -> yz_solrq:size_resps().
-resize_helpers(NewSize) when NewSize > 0 ->
- do_child_resize(NewSize, yz_solrq:num_helper_specs(), helper).
-
--spec do_child_resize(NewSize :: pos_integer(),
- OldSize :: non_neg_integer(),
- ChildType :: helper | worker)
- -> yz_solrq:size_resps().
-do_child_resize(OldSize, OldSize, _ChildType) ->
- same_size;
-do_child_resize(NewSize, OldSize, ChildType) when NewSize < OldSize->
- %% Reduce down to the new size before killing
- set_tuple(ChildType, NewSize),
- lists:foreach(fun(I) ->
- Name = reg_name(ChildType, I),
- _ = supervisor:terminate_child(yz_solrq_sup, Name),
- ok = supervisor:delete_child(yz_solrq_sup, Name)
- end,
- lists:seq(NewSize + 1, OldSize)),
- {shrank, OldSize - NewSize};
-do_child_resize(NewSize, OldSize, ChildType) ->
- lists:foreach(fun(I) ->
- supervisor:start_child(yz_solrq_sup,
- yz_solrq_sup:child_spec(ChildType, reg_name(ChildType, I)))
- end,
- lists:seq(OldSize + 1, NewSize)),
- set_tuple(ChildType, NewSize),
- {grew, NewSize - OldSize}.
-
-set_tuple(helper, Size) ->
- yz_solrq:set_solrq_helper_tuple(Size);
-set_tuple(worker, Size) ->
- yz_solrq:set_solrq_worker_tuple(Size).
-
-reg_name(helper, Name) ->
- yz_solrq:helper_regname(Name);
-reg_name(worker, Name) ->
- yz_solrq:worker_regname(Name).
-
-
diff --git a/dialyzer.ignore-warnings b/dialyzer.ignore-warnings
index d3c6e352..8b4c2978 100644
--- a/dialyzer.ignore-warnings
+++ b/dialyzer.ignore-warnings
@@ -57,10 +57,14 @@ Unknown functions:
riakc_pb_socket:start_link/2
riakc_pb_socket:stop/1
riakc_pb_socket:update_type/4
+ riakc_pb_socket:create_search_index/3
+ riakc_pb_socket:get_search_index/3
+ riakc_pb_socket:set_options/2
riakc_register:set/2
riakc_set:add_element/2
riakc_set:new/0
riakc_set:to_op/1
+ rt:admin/2
rt:build_cluster/1
rt:build_cluster/2
rt:clean_data_dir/2
@@ -100,6 +104,8 @@ Unknown functions:
rt:partition/2
rt:wait_until_transfers_complete/1
rt:log_to_nodes/2
+ rt:start/1
+ rt:upgrade/4
yz_solr_orig:index_batch_orig/2
yz_solrq_drain_mgr_orig:drain_orig/1
yz_solrq_drain_mgr_orig:unlink_and_kill_orig/2
diff --git a/docs/BATCHING.md b/docs/BATCHING.md
index 3291c005..8e8960ad 100644
--- a/docs/BATCHING.md
+++ b/docs/BATCHING.md
@@ -1,44 +1,34 @@
# Introduction
-In Yokozuna versions prior to 2.0.7, update operations on Solr (notably, `add` and `delete` operations) are synchronous blocking operations and are performed once-at-a-time in Solr. In particular, calls to the `yz_kv:index/3` Erlang function block until the associated data is written to Solr, and each such call results in an HTTP POST with a single Solr operation.
+In Yokozuna versions prior to 2.0.4 (e.g., Riak 2.0.7), update operations on Solr (notably, `add` and `delete` operations) are synchronous blocking operations and are performed once-at-a-time in Solr. In particular, calls to the `yz_kv:index/3` Erlang function block until the associated data is written to Solr, and each such call results in an HTTP POST with a single Solr operation.
-Yokozuna version 2.0.7 introduces batching and asynchronous delivery of Solr operations. The primary objective of this work is to decouple update operations in Solr from the Riak vnodes that are responsible for managment of replicas of Riak objects across the cluster. Without batching and asynchronous delivery, Riak vnodes have to wait for Solr operations to complete, sometimes an inordinate amount of time, which can have an impact on read and write operations in a Riak cluster, even for Riak objects that aren't being indexed through Yokozuna! This feature is intended to free up Riak vnodes to do other work, thus allowing Riak vnodes to service requests from more clients concurrently, and increasing operational throughput throughout the cluster.
+Yokozuna version 2.0.4 introduces batching and asynchronous delivery of Solr operations. The primary objective of this work is to decouple update operations in Solr from the Riak vnodes that are responsible for managment of replicas of Riak objects across the cluster. Without batching and asynchronous delivery, Riak vnodes have to wait for Solr operations to complete, sometimes an inordinate amount of time, which can have an impact on read and write operations in a Riak cluster, even for Riak objects that aren't being indexed through Yokozuna! This feature is intended to free up Riak vnodes to do other work, thus allowing Riak vnodes to service requests from more clients concurrently, and increasing operational throughput throughout the cluster.
De-coupling indexing operations from Riak vnode activity, however introduces some complexity into the system, as there is now the possibility for indeterminate latency between the time that an object is available in Riak versus its availability in Yokozuna and Solr. This latency can introduce divergence between the Riak AAE trees stored in Riak/KV and the AAE trees stored in Yokozuna, with which the Riak/KV trees are compared.
-This document describes the batching and asynchronous delivery subsystem introduced in Yokozuna 2.0.7, from both the end-users' and implementors' point of view, as well as the methods by which divergence between Riak and Solr is mitigated.
+This document describes the batching and asynchronous delivery subsystem introduced in Yokozuna 2.0.4, from both the end-users' and implementors' point of view, as well as the methods by which divergence between Riak and Solr is mitigated.
# Overview
The Yokozuna batching subsystem introduces two sets of additional `gen_server` processes into each Riak node, a set of "workers", and a set of "helpers". The job of the worker processes is to enqueue updates from components inside of Riak -- typically Riak vnodes, but also sometimes parts of the Yokozuna AAE subsystem -- and to make enqueued objects available to helper processes, which, in turn, are responsible for dispatching batches of operations to Solr.
-For each Riak node, there is a fixed, but configurable, number of helper and worker processes, and by default, 10 of each are created. A change to the number of workers or helpers requires a restart of the Yokozuna OTP application.
+For each vnode and Solr index on a Riak node, there is a pair of helper and worker processes which are responsible for enqueing and dispatching batches of operations into Solr. For example, on a Riak node with 12 vnodes and 5 indices, there will be 60 such pairs. Each pair of helper and worker processes has an associated supervisor, which oversees the lifecycle of each pair of `gen_server` processes. Each such supervisor is itself supervised by a supervisor for the entire batching subsytem, which in turn is supervised by the supervision hierarchy for the Yokozuna application.
-When enqueuing an object, a worker process is selected by hashing the Solr index (core) with which the operation is associated, along with the Riak bucket and key (bkey). This hash, modulo the number of workers, determines the worker process on which the datum is enqueued. When a batch of objects is dequeued from a worker process, a helper is randomly selected from among the set of configured helpers using a uniform PRNG. These algorithms for selecting workers and helpers is designed to provide as even a distribution of load across all workers and helpers as possible.
-
-Once a batch is dequeued from a worker, the Riak objects are transformed into Solr operations, and a single HTTP request is formed, containing a batch of Solr operations (encoded in a JSON payload). The batch is then delivered to Solr.
+When enqueuing an object, a worker and helper process pair is selected based on the Solr index (core) with which the operation is associated, along with the Riak partition on which the Riak object is stored. The associated helper process will periodically dequeued batches of objects from a worker process, translate the objects into a set of Solr "operations", and dispatch those operations in batches of HTTP requests to the Solr server running on the same node.
The following diagram illustrates the relationship between Riak vnodes, Yokozuna workers and helpers, and Apache Solr:
-![YZ Batching Overview](https://raw.githubusercontent.com/basho/yokozuna/2.0/docs/yz-batching-overview.png)
+![YZ Batching Overview](https://raw.githubusercontent.com/basho/yokozuna/docs/yz-batching-overview.png)
Each helper process is stateless; a helper's only role is to dequeue batches from the workers, to transform those batches into something that Solr can understand, and to dispatch the transformed batches to Solr (and report the results back to the worker process, on whose behalf it is doing the work).
The worker processes, on the other hand, maintain the queues of objects that are ready for update in Solr, in addition to managing the run-time behavior between the batching subsystem and the collection of vnodes and other processes that are communicating with the workers.
-In order to batch operations into a single HTTP POST into Solr, all operations must be organized under the same Solr core, i.e. Riak search index. As a consequence, each Yokozuna worker process maintains a table (`dict`) of "indexq" structures, keyed off the Riak search index. These indexq structures include, most critically, the enqueued object
-
-Indexq structures are created on-demand in each worker process, as data is added to the system. In most cases, all worker processes will contain entries for each Riak serch index, but it is sometimes possible for a worker process to be missing an entry for given index because, for example, it has not yet seen an object that needs to be updated for a given search index. This is expected behavior.
-
-The relationship between Riak search indices and indexq structures within each worker process is illustrated in the following diagram:
-
-![YZ Batching Worker](https://raw.githubusercontent.com/basho/yokozuna/2.0/docs/yz-batching-worker.png)
-
## Batching Parameters
-When an update message is sent to a worker process, it is immediately enqueued in the indexq structure associated with the index for which the operation is destined.
+When an update message is sent to a worker process, it is immediately enqueued in the worker associated with the index and Riak partition for which the operation is destined.
The Yokozuna batching subsystem provides configuration parameters that drive batching benavior, including configuration of:
@@ -52,7 +42,7 @@ If when enqueing an update operation the number of batched messages is smaller t
## Backpressure
-Each worker process is configured with a high water mark (10, by default), which represents the total number of messages that may be enqueued across all indexq structures in a given worker process before calls into the batching subsystem (update/index) will block calling vnodes. If the total number of enqueued messages exceeds this threshold, calling vnodes (and parts of the AAE subsystem) will block until data is successfully written to Solr, or it is purged, in a manner described below.
+Each worker process is configured with a high water mark (1000, by default), which represents the total number of messages that may be enqueued in a given worker process before calls into the batching subsystem (update/index) will block calling vnodes. If the total number of enqueued messages exceeds this threshold, calling vnodes (and parts of the AAE subsystem) will block until data is successfully written to Solr, or it is purged, in a manner described below.
This way, the batching subsystem exerts back-pressure on the vnode and AAE systems, in the case where Solr is being driven beyond its operational capacity.
@@ -64,13 +54,12 @@ In the undesirable cases where Solr becomes unresponsive (e.g., data corruption,
## Purge Strategies
-If a Solr core has become unresponsive and the specified error threshold has been traversed, and if, in addition, the high water mark has been exceeded, then the yokozuna batching system has a mechanism for automatically purging enqueued entries, so as to allow vnodes to continue servicing requests, as well as to allow update operations to occur for indices that are not in a pathological state.
+If a Solr core has become unresponsive and the specified error threshold has been traversed, and if, in addition, the high water mark has been exceeded for a particular `yz_solrq_worker` process, then the yokozuna batching system has a mechanism for automatically purging enqueued entries, so as to allow vnodes to continue servicing requests, as well as to allow update operations to occur for indices that are not in a pathological state.
-The yokozuna batching subsystem supports 4 different purge strategies:
+The yokozuna batching subsystem supports 3 different purge strategies:
-* `purge_one` (default behavior): Purge the oldest entry from a randomly selected indexq structure among the set of search indexes which have crossed the error threshold;
-* `purge_index`: Purge all entries from a randomly selected indexq structure among the set of search indexes which have crossed the error threshold;
-* `purge_all`: Purge all entries from all indexq structure among the set of search indexes which have crossed the error threshold;
+* `purge_one` (default behavior): Purge the oldest entry from the `yz_solrq_worker`;
+* `purge_index`: Purge all entries from the `yz_solrq_worker`;
* `off`: Perform no purges. This has the effect of blocking vnodes indefinitely, and is not recommended for production servers.
Note that purging enqueued messages should be considered safe, as long as Active Anti-Entropy (AAE) is enabled, as the AAE subsystem will eventually detect missing entries in Solr, and will correct for the difference.
@@ -93,29 +82,39 @@ Recall that a Riak object is indexed in Solr if it is in a bucket which is assoc
In order for Riak/KV and YZ AAE trees to be comparable, they must represent the same replica sets, where a replica set is determined by its initial Riak partition and replication factor (`n_val`). Because AAE trees are ignorant of buckets -- they are based entirely on the ring topology and replication factor, YZ AAE trees need to contain entries not only for Riak objects that are indexed in Solr, but also Riak object that are not. If YZ AAE trees did not contain hashes for entries that are not indexed in Solr, the comparison with Riak/KV AAE trees would always show data missing in Solr, and thus repairs would always be attempted.
-# Configuration and Statistics
+# Commands, Configuration, and Statistics
The batching subsystem is designed to be primarily invisible to the user, except perhaps for improved throughput under high load. However, some parameters of the batching subsystem are tunable via Cuttlefish configuration properties in the `riak.conf` configuration file (or, alternatively, via the `advanced.config` file). Changes to this file require a restart of Riak (or, alternatively, of just the Yokozuna application, via the Riak console).
The batching subsystem also introduces a set of statistics, which provides operators visibility into such measurements as available queue capacity, averages and histograms for batch sizes, AAE activity, and so forth.
-This section describes the configuration parameters and statistics of the batching subsystem, from the user's point of view.
+This section describes the commands, configuration parameters, and statistics of the batching subsystem, from the user's point of view.
-## Configuration
+## Commands
-The behavior of the batching subsystem may be controlled via the following Cuttlefish configuration parameters, as defined in `riak.conf`. Consult the Cuttlefish schema (TODO add link) for the associated configuration settings in the Riak `advanced.config` file.
+The `riak-admin` command may be used to control the participation of a node in distributed query. This command can be useful, for example, if a node is down for repair or reindexing. The node can be temporarily removed from coverage plans, so that it is not consulted as part of a distributed query.
-* `search.queue.batch.minimum` (default: 1) The minimum batch size, in number of Riak objects. Any batches that are smaller than this amount will not be immediately flushed to Solr, but are guaranteed to be flushed within the value specified in `search.queue.batch.flush_interval`.
+Here are sample usages of this command:
-* `search.queue.batch.maximum` (default: 100) The maximum batch size, in number of Riak objects. Any batches that are larger than this amount will be split, where the first `search.queue.batch.maximum` objects will be flushed to Solr, and the remaining objects enqueued for that index will be retained until the next batch is delivered. This parameter ensures that at most `search.queue.batch.maximum` objects will be delivered into Solr in any given request.
+ shell$ riak-admin set search.dist_query=off # disable distributed query for this node
+ shell$ riak-admin set search.dist_query=on # enable distributed query for this node
+ shell$ riak-admin show search.dist_query # get the status of distributed query for this node
-* `search.queue.batch.flush_interval` (default: 1 second) The maximum delay between notification to flush batches to Solr. This setting is used to increase or decrease the frequency of batch delivery into Solr, specifically for relatively low-volume input into Riak. This setting ensures that data will be delivered into Solr in accordance with the `search.queue.batch.maximum` and `search.queue.batch.maximum` settings within the specified interval. Batches that are smaller than `search.queue.batch.maximum` will be delivered to Solr within this interval. This setting will generally have no effect on heavily loaded systems.
+> Note. that even if a node is removed from a distributed query, it's search endpoint may still be consulted for query. If distributed query is disabled on the node and the search endpoint is used for query, only the other available nodes in the cluster will be consulted as part of a distributed query.
-* `search.queue.high_watermark` (default: 10) The queue high water mark. If the total number of queued messages in a Solrq worker instance exceeds this limit, then the calling vnode will be blocked until the total number falls below this limit. This parameter exercises flow control between Riak and the Yokozuna batching subsystem, if writes into Solr start to fall behind.
+Using this command will only temporarily enable or disable distributed query until explicitly disabling or re-enabling via the same command, or after restart. See the `search.dist_query` configuration setting to control a node's participation in drributed queries across restarts of a Riak server.
-* `search.queue.worker_count` (default: 10) The number of solr queue workers to instantiate in the Yokozuna application. Solr queue workers are responsible for enqueing objects for insertion or update into Solr. Increasing the number of solr queue workers distributes the queuing of objects, and can lead to greater throughput under high load, potentially at the expense of smaller batch sizes.
+## Configuration
+
+The behavior of the batching subsystem may be controlled via the following Cuttlefish configuration parameters, as defined in `riak.conf`. Consult the Cuttlefish schema (TODO add link) for the associated configuration settings in the Riak `advanced.config` file.
+
+* `search.queue.batch.minimum` (default: 10) The minimum batch size, in number of Riak objects. Any batches that are smaller than this amount will not be immediately flushed to Solr, but are guaranteed to be flushed within the value specified in `search.queue.batch.flush_interval`.
-* `search.queue.helper_count` (default: 10) The number of solr queue helpers to instantiate in the Yokozuna application. Solr queue helpers are responsible for delivering batches of data into Solr. Increasing the number of solr queue helpers may increase concurrent writes into Solr.
+* `search.queue.batch.maximum` (default: 500) The maximum batch size, in number of Riak objects. Any batches that are larger than this amount will be split, where the first `search.queue.batch.maximum` objects will be flushed to Solr, and the remaining objects enqueued for that index will be retained until the next batch is delivered. This parameter ensures that at most `search.queue.batch.maximum` objects will be delivered into Solr in any given request.
+
+* `search.queue.batch.flush_interval` (default: 500ms) The maximum delay between notification to flush batches to Solr. This setting is used to increase or decrease the frequency of batch delivery into Solr, specifically for relatively low-volume input into Riak. This setting ensures that data will be delivered into Solr in accordance with the `search.queue.batch.maximum` and `search.queue.batch.maximum` settings within the specified interval. Batches that are smaller than `search.queue.batch.maximum` will be delivered to Solr within this interval. This setting will generally have no effect on heavily loaded systems.
+
+* `search.queue.high_watermark` (default: 1000) The queue high water mark. If the total number of queued messages in a Solrq worker instance exceeds this limit, then the calling vnode will be blocked until the total number falls below this limit. This parameter exercises flow control between Riak and the Yokozuna batching subsystem, if writes into Solr start to fall behind.
* `search.index.error_threshold.failure_count` (default: 3) The number of failures within the specified `search.index.error_threshold.failure_interval` before writes into Solr will be short-circuited. Once the error threshold is crossed for a given Riak index (i.e., Solr core), Yokozuna will make no further attempts to write to Solr for objects destined for that index until the error is reset.
@@ -123,8 +122,22 @@ The behavior of the batching subsystem may be controlled via the following Cuttl
* `search.index.error_threshold.reset_interval` (default: 30 seconds) The amount of time it takes for a an error error threashold traversal associated with a Solr core to reset. If `search.index.error_threshold.failure_count` failures occur within `search.index.error_threshold.failure_interval`, requests to Solr for that core are short-circuited for this interval of time.
-* `search.queue.high_watermark.purge_strategy` (default: `purge_one`) The high watermrk purge strategy. If a Solr core threshold is traversed, and if the number of enqueued messages in a solr worker exceeds `search.queue.high_watermark`, then Yokozuna will use the defined purge strategy to purge enqueued messages. Valid values are `purge_one`, `purge_index`, `purge_all`, and `off`.
+* `search.queue.high_watermark.purge_strategy` (default: `purge_one`) The high watermrk purge strategy. If a Solr core threshold is traversed, and if the number of enqueued messages in a solr worker exceeds `search.queue.high_watermark`, then Yokozuna will use the defined purge strategy to purge enqueued messages. Valid values are `purge_one`, `purge_index`, and `off`.
+
+* `search.anti_entropy.throttle` (default: on) Whether the throttle for Yokozuna active anti-entropy is enabled.
+* `search.anti_entropy.throttle.$tier.solrq_queue_length` Sets the throttling tiers for active anti-entropy. Each tier is a minimum solrq queue size and a time-delay that the throttle should observe at that size and above. For example:
+
+ search.anti_entropy.throttle.tier1.solrq_queue_length = 0
+ search.anti_entropy.throttle.tier1.delay = 0ms
+ search.anti_entropy.throttle.tier2.solrq_queue_length = 40
+ search.anti_entropy.throttle.tier2.delay = 5ms
+
+ will introduce a 5 millisecond sleep for any queues of length 40 or higher. If configured, there must be a tier which includes a mailbox size of 0. Both `.solrq_queue_length` and `.delay` must be set for each tier. There is no limit to the number of tiers that may be specified.
+
+* `search.anti_entropy.throttle.$tier.delay` See above.
+
+* `search.dist_query` (Default: on) Enable or disable this node in distributed query plans. If enabled, this node will participate in distributed Solr queries. If disabled, the node will be excluded from yokozuna cover plans, and will therefore never be consulted in a distributed query. Note that this node may still be used to execute a query. Use this flag if you have a long running administrative operation (e.g., reindexing) which requires that the node be removed from query plans, and which would otherwise result in inconsistent search results.
The following options are hidden from the default `riak.conf` file:
@@ -171,6 +184,10 @@ The Yokozuna batching subsystem maintains a set of statistics that provide visib
* `search_detected_repairs_count` The total number of AAE repairs that have been detected when comparing YZ and Riak/KV AAE trees. Note that this statistic is a measurement of the differences found in the AAE trees; there may be some latency between the time the trees are compared and the time that the repair is written to Solr.
+* `search_index_bad_entry_(count|one)` The number of writes to Solr that have resulted in an error due to the format of the data (e.g., non-unicode data) since the last restart of Riak.
+
+* `search_index_extract_fail_(count|one)` The number of failures that have occurred extracting data into a format suitable to insert into Solr (e.g., badly formatted JSON) since the last start of Riak.
+
The following statistics refer to query operations, and are not impacted by Yokozuna batching. They are included here for completeness:
* `search_query_throughput_(count|one)` The total count of queries, per Riak node, and the count of queries within the metric measurement window.
@@ -178,176 +195,3 @@ The following statistics refer to query operations, and are not impacted by Yoko
* `search_query_latency_(min|mean|max|median|95|99|999)` The minimum, mean, maximum, median, 95th percentile, 99th percentile, and 99.9th percentile measurements of querying latency, as measured from the time it takes to send a request to Solr to the time the response is received from Solr.
* `search_query_fail_(count|one)` The total count of failed queries, per Riak node, and the count of query failures within the metric measurement window.
-
-
-# Implementation Notes
-
-This section describes the internal components that form the batching subsystem. It is targeted primarily at a developer audience. By convention, the Erlang modules that form this subsystem contain the `solrq` moniker in their names.
-
-In the remainder of this document, we generalize update and delete operations that are posted to solr as simply "Solr operations", without considering whether the operations originated as the result of a Riak put operations, delete operation, or AAE and/or read repair.
-
-## Supervision Tree
-
-The Solrq subsystem contains a supervision hierarchy that branches off the `yz_general_sup` supervisor in the Yokozuna supervision tree. The `yz_solrq_sup` is the top-level supervisor in this hierarchy.
-
-The `yz_solrq_sup` supervisor monitors a pool of Solrq workers and a pool of Solrq helpers. The objects in this pool are OTP registered gen_server processes, taking on the names `yz_solrq_i` and `yz_solrq_helper_j`, respectively, where i and j are padded integer values in the range {1..n} and {1..m}, respectively, where n and m are defined in config via the `num_solrq` and `num_solrq_helpers` Yokozuna configuration properties, respectively. These values are configurable, but both default to 10.
-
-The following diagram illustrates this supervision hierarchy:
-
-![YZ Batching Supervision Tree](https://github.com/basho/internal_wiki/blob/master/images/yokozuna/yz-solrq-supervision-tree.png)
-
-There will generally be a `yz_solrq_sup` per `yokozuna` application instance.
-
-> **Note.** The `yz_solr_sup` module also provides an entrypoint API for components in the batching subsystem. Beware that these APIs are simply convenience functions through a common module, and that all calls through these APIs are sequential Erlang functions, and are not executed in the context of the `yz_solrq_sup` supervisor OTP process.
-
-## Solrq Component Overview
-
-The `yz_solrq_sup`, `yz_solrq`, and `yz_solrq_helper` processes form the major components in the Yokozuna batching subsystem.
-
-When a `riak_kv_vnode` services a put or delete request, an API call is made into the `yz_solrq_sup` module to locate a `yz_solrq` instance from the pool of Solrq workers, by taking the (portable) hash of the index and the bucket and key (bkey) associated with the Riak object, and "dividing" that hash space by the number of workers in the pool, n:
-
- hash({Index, BKey}) mod n -> yz_solrq_i
-
-This way, we get a roughly even distribution of load to all workers in the pool, assuming an even distribution of writes/deletes to indices and partitions.
-
-Once a worker is located, the Riak Object and associated operational data is enqueued onto the queue associated with the index on the worker. (The internal structure of the Solrq worker process is discussed below in more detail.)
-
-It is the job of the `yz_solrq_helper` process to periodically pull batches of data that have been enqueued on the worker queues, to prepare the data for Solr, including extracting fields via Yokozuna extractors, as well as translation to Solr operations, and to dispatch Solr operations to Solr via HTTP POST operations.
-
-The `yz_solrq_helper` instances form a pool, as well, and an instance of a helper is selected by using a uniform random distribution.
-
-The following diagram illustrates the relationship between these components:
-
-![YZ Batching Overview](https://github.com/basho/internal_wiki/blob/master/images/yokozuna/yz-batching-overview.png)
-
-The following subsections describe these components in more detail.
-
-### The Solrq Worker Process(es)
-
-The Yokozuna Solrq batching subsystem maintains a fixed-size (but configurable) pool of worker processes, which are responsible for enqueuing messages to be delivered to Solr.
-
-The Solrq worker processes provide caching of (add/delete) operations written to Solr. As gen_server processes, they maintain state about what data is cached for batching, as well as book keeping information about how the queues are configured. The Solrq helper processes (see below) coordinate with the worker processes, and do the actual writes to Solr. But it is the worker processes that are the first line of caching, between Yokozuna and Solr.
-
-Solr workers are gen_server processes that form a pool of batching/caching resources. As a pool of resources, they provide increased throughput under high load, as indexing/delete operations can be distributed evenly across workers in the pool. However, when posting an operation (add/delete) to Solr, all operations (or any batched operations thereof) need to take place under the umbrella of a single Solr index (or "core"). Specifically, the HTTP POST operation to Solr to update an index contains the Solr index as part of its URL, even if that POST operation contains multiple documents.
-
-As a consequence, the Solrq workers must partition the operations by index, so that when batches are delivered to Solr, each batch is POSTed under the URL of a single core.
-
-Each Solrq process manages this partitioning by maintaining a dictionary of index -> `indexq` mappings, where the index key is a Solr index name, and an `indexq` is a data structure that holds the queue of operations to be POSTed to Solr under that index.
-
-> **Note.** This mapping is encpasulated in an Erlang `dict` structure, but the population of entries in this dictionary are done lazily. Initially, the dictionary on a `yz_solrq` instance is empty, and as operations are performed on an index, entries are added. Eventually, a given worker instance may contain an entry for each index configured by the user, but due to the luck of the draw, there may be some solrq instances that do not contain entries for some indices (e.g., if objects map to a strict subset of partitons on a given node, for example).
-
-The `indexq` structure contains book keeping information, such as the queue of data for that index, its length, cached configuration information for that queue, such as its minimum and maximum batch size, and other state information described in more detail below.
-
-The Yokozuna Solrq worker process is illustrated in the following diagram:
-
-![YZ Solrq Worker](https://github.com/basho/internal_wiki/blob/master/images/yokozuna/yz-solrq-worker.png)
-
-### Solrq Helper Process(es) and Batching Protocol
-
-The `yz_solrq_helper` process is a stateless `gen_server` OTP process, whose only role is to dequeue data from `yz_solrq` instances, to dispatch the data to Solr, and to reply back to the worker process when a batch has completed. As a stateless object, there is not much to describe about a helper process, except for the protocol of messages that are sent between the `riak_kv_vnode`, `yz_solrq` worker, and `yz_solrq_helper` processes.
-
-When an indexing operation (add/delete) is requested from a `riak_kv_vnode`, a `yz_solrq` worker process is located (as described above), and a syncronous index message is delivered to the worker, via a `gen_server:call`:
-
- {index, Index, {BKey, Docs, Reason, P}}
-
-If the total number of queued messages for this worker is not above the configured high water mark (See the Backpressure section, below), an `ok` is immediately delivered back to the vnode, and the requested data is enqueued onto the `yz_solrq` worker, keyed off the supplied index.
-
-If the number of queued messages is above the configured minimum (default: 1), and if there are currently no pending helpers who have been told to request a batch, the `yz_solrq` worker will locate a `yz_solrq_helper` as described above, and send it an asynchronous message (via `gen_server:cast`), telling it that the worker is ready to deliver a batch:
-
- {ready, Index, QPid}
-
-> **Note.** This asynchronous message is delivered via a cast to the worker. No further work is required on the part of the worker, and it does not synchronously wait for a response from the helper.
-
-Upon receipt of the `ready` message, the selected helper will send an asynchronous message back to the worker, requesting a batch for the specified index:
-
- {request_batch, Index, HPid}
-
-Upon receipt of this message, the worker will select a batch of messages to deliver, typically less than or equal to the size of the configured maximum batch size, and send a batch message back to the helper:
-
- {batch, Index, BatchMax, QPid, Entries}
-
-> **Note.** The number of entries in the batch may exceed the configured batch maximum in the case where the queues are being drained. See the "Draining" section below for more information.
-
-> **Comment.** I'd clear up what async means here. We're still "synchronously" waiting for the solr http update to finish, via an ibrowse call, before we return the cast message, right?
-
-Upon receipt of the `batch` message, the helper dispatches the batch (or batches, in the case of draining) to Solr, and then sends an asynchronous message back to the worker, indicating that the batch (or batches) have completed, the number of messages that were delivered into Solr, and a return status, indicating whether the writes to Solr were successful or not:
-
- {batch_complete, NumDelivered, Result}
-
-> *Note.* The `batch_complete` message is new, and has not been shipped with any of the patches to date.
-
-Upon receipt of the `batch_complete` message, the worker will:
-
-* Decrement the number of queued messages by the number of delivered messages
-* Unblock any waiting vnodes if the total number of queued messages is below the high water mark
-* Mark the pending state of the batch back to false, so that new batches for the same index can proceed
-* Request a helper, if there is still data left in the queues to be flushed and if the number of queued messages is above the configured minimum.
-* Re-queue (but pre-pend) any undelivered messages, in case of any failures in delivery
-
-The Yokozuna batching protocol between vnodes, workers, and helpers is illustrated in the following diagram.
-
-![YZ Solrq Batching Protocol](https://github.com/basho/internal_wiki/blob/master/images/yokozuna/yz-batching-sequence.png)
-
-
-
-### Flushing
-
-In the above scenario, data enqueued for a specific index is written to Solr as soon as the number of queued messages exceeds the configured minimum, which defaults to 1. Hence, in many cases, enqueued messages get sent almost immediately after being enqueued, but asynchronously from the perspective of vnodes.
-
-> *Note*. This does not entail that batch sizes are always 1; because the messaging protocol between workers and helpers is asynchronous, more messages may arrive in the queue between the time that a worker is notified that a batch is ready and the time that the batch is actually retrieved from the queues. In heavily loaded systems, the queues typically grow in size in this time.
-
-What happens, however, if the user configures a relatively high minimum batch size, but data drips into the queues at a relatively slow rate?
-
-In this case, the batching subsystem will set a timer, specified by the `solrq_delayms_max` configuration setting (default: 1000ms). If no data has been flushed within this time interval (which would happen, for example, as the result of a write into the queue), then a batch will automatically be initiated, along the lines of the protocol described above.
-
-## Backpressure
-
-Each `yz_solrq` worker maintains a record of how many messages are enqueued in all of the `indexq` structures held in the worker. When data is sent to a worker from a vnode, the worker will increment the count of queued messages. If the total count exceeds the configured high water mark (`solrq_queue_hwm`; default: 10000), a reply to the calling vnode is deferred until the total count of queued messages again falls below this limit.
-
-The number of enqueued messages is decremented once the `batch_complete` message is received from the associated helper process, where the message received back from the helper contains the number of messages that have been delivered to Solr.
-
-> **Note.** In previous implementations of the Yokozuna batching patch, the total number of enqueued messages was decremented when a batch was delivered to the helper for dispatch to the worker, but this behavior has changed with the introduction of the `batch_complete` message.
-
-> **Note.** Setting the `solrq_batch_min` and `solrq_batch_max` values to 1 results in immediate backpressure on the vnode, until the message has been successfully delivered to Solr. This special case of the Yokozuna batching patch simulates the `dw` semantics of the pre-batching Yokozuna implementation.
-
-### Fuses
-
-Yokozuna batching makes use of the [Fuse](https://github.com/jlouis/fuse) library, an OTP application which supports a circuit-breaker pattern. Every time some condition applies (at the discretion of the application), a fuse is "melted" slightly. If some configured number of melts occur within a configured time interval, then the fuse "blows", and applications can then modify their behavior, with the knowledge that some operation is likely to be fruitless. Fuses eventually "heal", after a configured amount of time, presumably while not under load.
-
-Yokozuna uses this pattern in its handling of communication with Solr. For each Solr index, a fuse is created, which represents the connection to Solr. If requests to Solr continuously fail (for a given index), the fuse for that index is blown, and any subsequent Solr requests for that index are short-circuited. Once a fuse heals, batching resumes.
-
-The Fuse library makes use of Erlang alarm handling, whereby fuse events (fuse "trips" and "heals") are delivered through Erlang alarm handlers. The `yz_events` event handler subscribes to these events, and will notify the batching subsystem when a fuse blows or heals.
-
-If a fuse trips, this has the effect of effectively pausing the delivery of any batches to Solr, for a given Solr index (the index associated with the tripped fuse). Once a fuse heals, delivery of batches is resumed.
-
-If delivery of batches to Solr is paused, then it is possible that enqueued messages will pile up, potentially reaching the configured high water mark (and thus causing back-pressure on the calling vnodes). If the `purge_blown_indices` yokozuna configuration variable is set to true (its default value), then when the high water mark is reached, then entries will be discarded on indices that are blocked. A warning will be logged to the system logs, indicating which index has had data discarded (but not the data contents).
-
-> Note. The rationale for defaulting this value to true is that AAE is also enabled by default on Riak systems, and that any divergence between Riak and Yokozuna can be eventually repaired by AAE.
-
-If the `purge_blown_indices` yokozuna configuration variable is set to false, then no data will be automatically purged from indices that have blown fuses, and the only way to resume batching for *all* indices (and therefore to relieve back pressure on the vnodes on the Riak node) is for the fuse to heal. For pathological indices, this may take a long time! This variable should only be set to false if Yokozuna AAE is not enabled, and if Solr indices are known to be well-behaved. Otherwise, there is a non-trivial risk that all of Riak can wedge while fuses heal.
-
-## Draining
-
-Some applications have the need to not only to periodically flush idle queues, but also to completely drain the contents of all queues on demand, or at least all of the messages associated with a given Riak partition. These applications include stopping the Yokozuna application (to ensure everything in memory is flushed to Solr), as well as the YZ AAE subsystem, which, when exchanging hash trees between Riak K/V and Yokozuna, snapshots the Riak K/V hash tree, drains the queues, and then snapshots the Yokozuna hash tree, in order to minimize divergence between the two hash trees.
-
-Draining is provided as a `drain` function in the `yz_solrq_mgr` module, which, when invoked, will spawn and monitor a `gen_fsm` process, `yz_solrq_drain_fsm`. The role of the FSM is to trigger a drain on all of the `yz_solrq` workers in the pool, and then to wait for all of the drains to complete. Once all of the queues have been drained, the FSM terminates, and the calling process will receive a `'DOWN'` message from the FSM it is monitoring, and the drain function can then return to the caller. If not all of the queues have drained within a configured timeout, the drain function will return `{error, timeout}`.
-
-In the current implementation, there may only be one Drain FSM active at time. An attempt to call the `drain` funtion while a drain is active will result in a return value of `{error, in_progress}`
-
-The `yz_solrq_drain_fsm` has two states:
-
-* *prepare* In this state, the Drain FSM will iterate over all `yz_solrq` instances, generate a token (via `erlang:make_ref()`) for each instance, and send the `{drain, DPid, Token, Partition}` message to each worker, where `DPid` is the PID of the drain FSM, and `Token` is the generated token, and `Partition` is the desired Riak partition to drain (or `undefined`, if all messages should be drained). It will then enter the *wait* state.
-
-* *wait* In this state, the Drain FSM will wait for `{drain_complete, Token}` messages back from each `yz_solrq` worker instance. Once all delivered tokens have been received, the Drain FSM will terminate normally.
-
-When a `yz_solrq` worker receives a `{drain, DPid, Token, Partition}` message, it will iterate over all of it `indexq` structures, set the `draining` flag on each structure, and initiate the batching protocol with an associated helper, as described above. However, unlike the normal batching case, when the worker is in the draining state, it will deliver *all* of its enqueued messages that match the specified partition (or all messages, if `Partition` is `undefined`) to the helper, along with the configured maximum batch size. In this case, the helper will iterate over all of the messages it has been sent, and form batches of messages for delivery to Solr. For example, if the batch size is 100 and 1027 messages have been enqueued, the helper will end up sequentially delivering 10 batches of 100, and then an 11th of size 27.
-
-While in the draining state, and messages that get enqueued get put onto a special "auxiliary queue" (`aux_queue`), which prevents them from getting sent in any batches to Solr. This prevents messages new messages delivered during the drain phase from getting written to Solr. When the drain for an `indexq` is completed, the `draining` flag is reset to false, and any messages on the `aux_queue` are moved back to the normal queue, for subsequent dispatch into Solr.
-
-Once the batch has completed and a `batch_complete` message is sent back to the worker, the worker records which index has been drained. Once all of the indices have been drained in the worker, it sends the `{drain_complete, Token}` message back to the Drain FSM, thus completing the drain for that `yz_solrq` worker instance. It will not, however, continue with batching until it receives a `batch_complete` message back from the `yz_solrq_drain_fsm`. This blocks writes into Solr until all queues have been drained.
-
-The `yz_solrq_drain_fsm` will stay in the waiting state until all of the tokens it has delivered to the `yz_solrq` instances have been returned. Once all of the tokens are returned, it will update the YZ index hashtree before firing off a `drain_complete` message to all of the `solrqs`, indicating that they can proceed with normal batching operations. Any messages that have been cached on the auxiliary queues are then moved to the normal queue, and batching proceeds as usual. The `yz_solrq_drain_fsm` then terminates, indicating to the caller that all queues have drained.
-
-The relationship between the caller of the `drain` function (in this example, `yz_exchange_fsm`), the `yz_solrq_drain_fsm`, and the Solrq workers and helpers is illustrated in the following sequence diagram:
-
-![YZ Solrq Draining](https://github.com/basho/internal_wiki/blob/master/images/yokozuna/yz-solrq-draining.png)
diff --git a/docs/yz-batching-overview.graffle b/docs/yz-batching-overview.graffle
index cceb03c7..ea6a2726 100644
--- a/docs/yz-batching-overview.graffle
+++ b/docs/yz-batching-overview.graffle
@@ -14,7 +14,7 @@
BackgroundGraphic
Bounds
- {{0, 0}, {1466, 576}}
+ {{0, 0}, {733, 576}}
Class
SolidGraphic
ID
@@ -42,111 +42,35 @@
ColumnSpacing
36
CreationDate
- 2015-12-23 01:06:49 +0000
+ 2016-11-07 18:20:22 +0000
Creator
Fred Dushin
DisplayScale
- 1 0/72 in = 1.0000 in
+ 1 0/72 in = 1 0/72 in
GraphDocumentVersion
8
GraphicsList
Bounds
- {{393.00000184774399, 238.50006103515625}, {98, 24}}
+ {{508, 454}, {55, 14}}
Class
ShapedGraphic
FitText
YES
Flow
Resize
- FontInfo
-
- Color
-
- w
- 0
-
- Font
- Helvetica
- Size
- 12
-
ID
- 313
- Line
-
- ID
- 307
- Position
- 0.49700599908828735
- RotationType
- 0
-
+ 37
Shape
Rectangle
Style
- shadow
-
- Draws
- NO
-
- stroke
+ fill
Draws
NO
-
- Text
-
- Text
- {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf340
-\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
-{\colortbl;\red255\green255\blue255;}
-\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
-
-\f0\fs24 \cf0 dequeue (batch)}
-
- Wrap
- NO
-
-
- Bounds
- {{99.087750594127698, 219.72650554174598}, {101, 24}}
- Class
- ShapedGraphic
- FitText
- YES
- Flow
- Resize
- FontInfo
-
- Color
-
- w
- 0
-
- Font
- Helvetica
- Size
- 12
-
- ID
- 312
- Line
-
- ID
- 302
- Position
- 0.44889700412750244
- RotationType
- 0
-
- Shape
- Rectangle
- Style
-
shadow
Draws
@@ -160,20 +84,24 @@
Text
+ Pad
+ 0
Text
- {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf340
+ {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf470
\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
{\colortbl;\red255\green255\blue255;}
\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
-\f0\fs24 \cf0 enqueue (object)}
+\f0\fs24 \cf0 Riak node}
+ VerticalPad
+ 0
Wrap
NO
Bounds
- {{587, 315.00003051757812}, {11, 14}}
+ {{413, 415}, {11, 14}}
Class
ShapedGraphic
FitText
@@ -181,7 +109,7 @@
Flow
Resize
ID
- 310
+ 36
Shape
Rectangle
Style
@@ -207,7 +135,7 @@
Pad
0
Text
- {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf340
+ {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf470
\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
{\colortbl;\red255\green255\blue255;}
\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
@@ -220,100 +148,74 @@
NO
- Bounds
- {{587, 180.50003051757812}, {11, 14}}
Class
- ShapedGraphic
- FitText
- YES
- Flow
- Resize
+ LineGraphic
+ Head
+
+ ID
+ 23
+ Info
+ 4
+
ID
- 309
- Shape
- Rectangle
+ 35
+ Points
+
+ {411, 387}
+ {423, 387}
+
Style
- fill
-
- Draws
- NO
-
- shadow
-
- Draws
- NO
-
stroke
- Draws
- NO
+ HeadArrow
+ 0
+ Legacy
+
+ TailArrow
+ 0
- Text
+ Tail
- Pad
- 0
- Text
- {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf340
-\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
-{\colortbl;\red255\green255\blue255;}
-\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
-
-\f0\fs24 \cf0 ...}
- VerticalPad
- 0
+ ID
+ 22
+ Info
+ 3
- Wrap
- NO
- Bounds
- {{279, 180.50003051757812}, {11, 14}}
Class
- ShapedGraphic
- FitText
- YES
- Flow
- Resize
+ LineGraphic
+ Head
+
+ ID
+ 21
+
ID
- 308
- Shape
- Rectangle
+ 34
+ Points
+
+ {411, 351}
+ {423, 351}
+
Style
- fill
-
- Draws
- NO
-
- shadow
-
- Draws
- NO
-
stroke
- Draws
- NO
+ HeadArrow
+ 0
+ Legacy
+
+ TailArrow
+ 0
- Text
+ Tail
- Pad
- 0
- Text
- {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf340
-\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
-{\colortbl;\red255\green255\blue255;}
-\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
-
-\f0\fs24 \cf0 ...}
- VerticalPad
- 0
+ ID
+ 20
- Wrap
- NO
Class
@@ -321,31 +223,31 @@
Head
ID
- 294
+ 19
ID
- 307
+ 33
Points
- {359, 250.50006103515625}
- {526, 250.50006103515625}
+ {411, 298}
+ {423, 298}
Style
stroke
HeadArrow
- FilledArrow
+ 0
Legacy
TailArrow
- FilledArrow
+ 0
Tail
ID
- 293
+ 18
Info
3
@@ -353,12 +255,17 @@
Class
LineGraphic
+ Head
+
+ ID
+ 14
+
ID
- 306
+ 32
Points
- {451, 423}
- {517, 423}
+ {411, 262}
+ {423, 262}
Style
@@ -370,80 +277,69 @@
TailArrow
0
- Width
- 0.25
+ Tail
+
+ ID
+ 13
+ Info
+ 3
+
- Bounds
- {{428.50005722045898, 327.50005185604095}, {110.99988555908203, 24}}
Class
- ShapedGraphic
- FitText
- Vertical
- Flow
- Resize
- FontInfo
-
- Color
-
- w
- 0
-
- Font
- Helvetica
- Size
- 12
-
- ID
- 305
- Line
+ LineGraphic
+ Head
ID
- 303
- Position
- 0.74924027919769287
- RotationType
+ 17
+ Info
4
- Rotation
- 90
- Shape
- Rectangle
+ ID
+ 31
+ Points
+
+ {411, 211}
+ {423, 211}
+
Style
- shadow
-
- Draws
- NO
-
stroke
- Draws
- NO
+ HeadArrow
+ 0
+ Legacy
+
+ TailArrow
+ 0
- Text
+ Tail
- Text
- {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf340
-\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
-{\colortbl;\red255\green255\blue255;}
-\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
-
-\f0\fs24 \cf0 m-many helpers}
+ ID
+ 16
+ Info
+ 3
Class
LineGraphic
+ Head
+
+ ID
+ 12
+ Info
+ 4
+
ID
- 304
+ 30
Points
- {455, 94}
- {521, 94}
+ {411, 173}
+ {423, 173}
Style
@@ -455,20 +351,30 @@
TailArrow
0
- Width
- 0.25
+ Tail
+
+ ID
+ 11
+ Info
+ 3
+
Class
LineGraphic
+ Head
+
+ ID
+ 22
+
ID
- 303
+ 29
Points
- {484, 93}
- {484, 422}
+ {208, 305}
+ {288, 387}
Style
@@ -479,44 +385,40 @@
Legacy
TailArrow
- FilledArrow
- Width
- 0.5
+ 0
+ Tail
+
+ ID
+ 6
+ Info
+ 3
+
Class
LineGraphic
- ControlPoints
-
- {-0.5, -87.0001220703125}
- {-22, -7.50006103515625}
-
Head
ID
- 293
+ 20
ID
- 302
+ 28
Points
- {106.5, 284.0001220703125}
- {226, 250.50006103515625}
+ {208, 305}
+ {288, 351}
Style
stroke
- Bezier
-
HeadArrow
FilledArrow
Legacy
- LineType
- 1
TailArrow
0
@@ -524,153 +426,243 @@
Tail
ID
- 254
+ 6
Info
- 2
+ 3
- Bounds
- {{349.50011825561523, 156.0000018030405}, {95.999763488769531, 24}}
Class
- ShapedGraphic
- FitText
- Vertical
- Flow
- Resize
- FontInfo
+ LineGraphic
+ Head
+
+ ID
+ 18
+
+ ID
+ 27
+ Points
+
+ {208, 271}
+ {288, 298}
+
+ Style
- Color
+ stroke
- w
+ HeadArrow
+ FilledArrow
+ Legacy
+
+ TailArrow
0
- Font
- Helvetica
- Size
- 12
- ID
- 300
- Line
+ Tail
ID
- 297
- Position
- 0.22188450396060944
- RotationType
- 4
+ 5
+ Info
+ 3
- Rotation
- 90
- Shape
- Rectangle
+
+
+ Class
+ LineGraphic
+ Head
+
+ ID
+ 13
+
+ ID
+ 26
+ Points
+
+ {208, 271}
+ {288, 262}
+
Style
- shadow
-
- Draws
- NO
-
stroke
- Draws
- NO
+ HeadArrow
+ FilledArrow
+ Legacy
+
+ TailArrow
+ 0
- Text
+ Tail
- Text
- {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf340
-\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
-{\colortbl;\red255\green255\blue255;}
-\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
-
-\f0\fs24 \cf0 n-many workers}
+ ID
+ 5
+ Info
+ 3
+
+
+
+ Class
+ LineGraphic
+ Head
+
+ ID
+ 16
+ Info
+ 4
-
-
- Class
- LineGraphic
ID
- 299
+ 25
Points
- {365, 423}
- {431, 423}
+ {208, 237}
+ {288, 211}
Style
stroke
HeadArrow
- 0
+ FilledArrow
Legacy
TailArrow
0
- Width
- 0.25
+ Tail
+
+ ID
+ 4
+ Info
+ 3
+
Class
LineGraphic
+ Head
+
+ ID
+ 11
+ Info
+ 4
+
ID
- 298
+ 24
Points
- {365, 94}
- {431, 94}
+ {208, 237}
+ {288, 173}
Style
stroke
HeadArrow
- 0
+ FilledArrow
Legacy
TailArrow
0
- Width
- 0.25
+ Tail
+
+ ID
+ 4
+ Info
+ 3
+
+ Bounds
+ {{423, 374}, {123, 26}}
Class
- LineGraphic
+ ShapedGraphic
ID
- 297
- Points
+ 23
+ Magnets
- {397.5, 95}
- {397.5, 424}
+ {0, 1}
+ {0, -1}
+ {1, 0}
+ {-1, 0}
- Style
+ Shape
+ Rectangle
+ Text
- stroke
-
- HeadArrow
- FilledArrow
- Legacy
-
- TailArrow
- FilledArrow
- Width
- 0.5
-
+ Text
+ {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf470
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs24 \cf0 helper 228359, B}
+
+
+
+ Bounds
+ {{288, 374}, {123, 26}}
+ Class
+ ShapedGraphic
+ ID
+ 22
+ Magnets
+
+ {0, 1}
+ {0, -1}
+ {1, 0}
+ {-1, 0}
+
+ Shape
+ Rectangle
+ Text
+
+ Text
+ {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf470
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs24 \cf0 worker 228359, B}
+
+
+
+ Bounds
+ {{423, 338}, {123, 26}}
+ Class
+ ShapedGraphic
+ ID
+ 21
+ Magnets
+
+ {0, 1}
+ {0, -1}
+ {1, 0}
+ {-1, 0}
+
+ Shape
+ Rectangle
+ Text
+
+ Text
+ {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf470
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs24 \cf0 helper 228359, A}
Bounds
- {{526, 363}, {133, 61}}
+ {{288, 338}, {123, 26}}
Class
ShapedGraphic
ID
- 296
+ 20
Magnets
{0, 1}
@@ -683,21 +675,21 @@
Text
Text
- {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf340
+ {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf470
\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
{\colortbl;\red255\green255\blue255;}
\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
-\f0\fs24 \cf0 Solrq Helper m}
+\f0\fs24 \cf0 worker 228359, A}
Bounds
- {{226, 363}, {133, 61}}
+ {{423, 285}, {123, 26}}
Class
ShapedGraphic
ID
- 295
+ 19
Magnets
{0, 1}
@@ -710,21 +702,21 @@
Text
Text
- {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf340
+ {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf470
\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
{\colortbl;\red255\green255\blue255;}
\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
-\f0\fs24 \cf0 Solrq Worker n}
+\f0\fs24 \cf0 helper 114179, B}
Bounds
- {{526, 220.00006103515625}, {133, 61}}
+ {{288, 285}, {123, 26}}
Class
ShapedGraphic
ID
- 294
+ 18
Magnets
{0, 1}
@@ -737,21 +729,21 @@
Text
Text
- {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf340
+ {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf470
\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
{\colortbl;\red255\green255\blue255;}
\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
-\f0\fs24 \cf0 Solrq Helper j}
+\f0\fs24 \cf0 worker 114179, B}
Bounds
- {{226, 220.00006103515625}, {133, 61}}
+ {{423, 198}, {123, 26}}
Class
ShapedGraphic
ID
- 293
+ 17
Magnets
{0, 1}
@@ -764,21 +756,21 @@
Text
Text
- {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf340
+ {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf470
\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
{\colortbl;\red255\green255\blue255;}
\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
-\f0\fs24 \cf0 Solrq Worker i}
+\f0\fs24 \cf0 helper 0, B}
Bounds
- {{526, 92}, {133, 61}}
+ {{288, 198}, {123, 26}}
Class
ShapedGraphic
ID
- 292
+ 16
Magnets
{0, 1}
@@ -791,17 +783,17 @@
Text
Text
- {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf340
+ {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf470
\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
{\colortbl;\red255\green255\blue255;}
\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
-\f0\fs24 \cf0 Solrq Helper 1}
+\f0\fs24 \cf0 worker 0, B}
Bounds
- {{287, 315.00003051757812}, {11, 14}}
+ {{141, 326}, {11, 14}}
Class
ShapedGraphic
FitText
@@ -809,7 +801,7 @@
Flow
Resize
ID
- 291
+ 15
Shape
Rectangle
Style
@@ -835,7 +827,7 @@
Pad
0
Text
- {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf340
+ {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf470
\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
{\colortbl;\red255\green255\blue255;}
\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
@@ -849,117 +841,178 @@
Bounds
- {{707.02464470537973, 219.07510029874931}, {48, 38}}
+ {{423, 249}, {123, 26}}
Class
ShapedGraphic
- FitText
- YES
- Flow
- Resize
- FontInfo
-
- Color
-
- w
- 0
-
- Font
- Helvetica
- Size
- 12
-
ID
- 261
- Line
+ 14
+ Magnets
+
+ {0, 1}
+ {0, -1}
+ {1, 0}
+ {-1, 0}
+
+ Shape
+ Rectangle
+ Text
- ID
- 260
- Position
- 0.51475578546524048
- RotationType
- 0
+ Text
+ {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf470
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs24 \cf0 helper 114179, A}
+
+
+ Bounds
+ {{288, 249}, {123, 26}}
+ Class
+ ShapedGraphic
+ ID
+ 13
+ Magnets
+
+ {0, 1}
+ {0, -1}
+ {1, 0}
+ {-1, 0}
+
Shape
Rectangle
- Style
+ Text
- shadow
-
- Draws
- NO
-
- stroke
-
- Draws
- NO
-
+ Text
+ {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf470
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs24 \cf0 worker 114179, A}
+
+
+ Bounds
+ {{423, 160}, {123, 26}}
+ Class
+ ShapedGraphic
+ ID
+ 12
+ Magnets
+
+ {0, 1}
+ {0, -1}
+ {1, 0}
+ {-1, 0}
+
+ Shape
+ Rectangle
Text
Text
- {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf340
+ {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf470
\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
{\colortbl;\red255\green255\blue255;}
\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
-\f0\fs24 \cf0 POST\
-(batch)}
+\f0\fs24 \cf0 helper 0, A}
- Wrap
- NO
+ Bounds
+ {{288, 160}, {123, 26}}
Class
- LineGraphic
- ControlPoints
+ ShapedGraphic
+ ID
+ 11
+ Magnets
- {60, -46.50006103515625}
- {-81, 2.82421875}
+ {0, 1}
+ {0, -1}
+ {1, 0}
+ {-1, 0}
- Head
+ Shape
+ Rectangle
+ Text
- ID
- 259
+ Text
+ {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf470
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs24 \cf0 worker 0, A}
+
+
+ Bounds
+ {{124, 130}, {54, 41}}
+ Class
+ ShapedGraphic
ID
- 260
- Points
+ 10
+ Magnets
- {659, 250.50006103515625}
- {815, 257.00006103515625}
+ {0, 1}
+ {0, -1}
+ {1, 0}
+ {-1, 0}
- Style
+ Shape
+ NoteShape
+ Text
- stroke
-
- Bezier
-
- HeadArrow
- FilledArrow
- Legacy
-
- LineType
- 1
- TailArrow
- 0
-
+ Text
+ {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf470
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs24 \cf0 index\
+'B'}
- Tail
+
+
+ Bounds
+ {{56, 130}, {54, 41}}
+ Class
+ ShapedGraphic
+ ID
+ 9
+ Magnets
+
+ {0, 1}
+ {0, -1}
+ {1, 0}
+ {-1, 0}
+
+ Shape
+ NoteShape
+ Text
- ID
- 294
+ Text
+ {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf470
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs24 \cf0 index\
+'A'}
Bounds
- {{815, 185.00009155273438}, {143.99993896484375, 143.99993896484375}}
+ {{85, 292}, {123, 26}}
Class
ShapedGraphic
ID
- 259
+ 6
Magnets
- {-0.5, 0}
+ {0, 1}
{0, -1}
{1, 0}
{-1, 0}
@@ -969,22 +1022,21 @@
Text
Text
- {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf340
+ {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf470
\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
{\colortbl;\red255\green255\blue255;}
\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
-\f0\fs24 \cf0 Apache\
-Solr}
+\f0\fs24 \cf0 vnode 228359}
Bounds
- {{61, 284.0001220703125}, {91, 61}}
+ {{85, 258}, {123, 26}}
Class
ShapedGraphic
ID
- 254
+ 5
Magnets
{0, 1}
@@ -997,21 +1049,21 @@ Solr}
Text
Text
- {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf340
+ {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf470
\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
{\colortbl;\red255\green255\blue255;}
\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
-\f0\fs24 \cf0 riak_kv_vnode}
+\f0\fs24 \cf0 vnode 114179}
Bounds
- {{226, 94}, {133, 61}}
+ {{85, 224}, {123, 26}}
Class
ShapedGraphic
ID
- 253
+ 4
Magnets
{0, 1}
@@ -1024,23 +1076,21 @@ Solr}
Text
Text
- {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf340
+ {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf470
\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
{\colortbl;\red255\green255\blue255;}
\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
-\f0\fs24 \cf0 Solrq Worker 1}
+\f0\fs24 \cf0 vnode 0}
Bounds
- {{50, 64}, {639, 423}}
+ {{35, 111}, {538, 367}}
Class
ShapedGraphic
- HFlip
- YES
ID
- 311
+ 3
Magnets
{0, 1}
@@ -1065,23 +1115,9 @@ Solr}
stroke
Pattern
- 2
+ 1
- Text
-
- Align
- 2
- Text
- {\rtf1\ansi\ansicpg1252\cocoartf1404\cocoasubrtf340
-\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
-{\colortbl;\red255\green255\blue255;}
-\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qr
-
-\f0\fs24 \cf0 Riak Node Process Boundary}
-
- TextPlacement
- 2
GridInfo
@@ -1091,7 +1127,7 @@ Solr}
GuidesVisible
YES
HPages
- 2
+ 1
ImageCounter
1
KeepToScale
@@ -1131,7 +1167,7 @@ Solr}
MasterSheets
ModificationDate
- 2016-04-28 19:05:52 +0000
+ 2016-11-07 19:32:26 +0000
Modifier
Fred Dushin
NotesVisible
@@ -1212,7 +1248,7 @@ Solr}
Frame
- {{161, 201}, {1233, 882}}
+ {{123, 100}, {1215, 1018}}
ListView
OutlineWidth
@@ -1226,7 +1262,7 @@ Solr}
SidebarWidth
120
VisibleRegion
- {{0, -74}, {1098, 725}}
+ {{-174, -150}, {1080, 876}}
Zoom
1
ZoomValues
diff --git a/docs/yz-batching-overview.png b/docs/yz-batching-overview.png
index ddfeb9f2..c65d2024 100644
Binary files a/docs/yz-batching-overview.png and b/docs/yz-batching-overview.png differ
diff --git a/include/yokozuna.hrl b/include/yokozuna.hrl
index becac1e0..9cd15e32 100644
--- a/include/yokozuna.hrl
+++ b/include/yokozuna.hrl
@@ -113,7 +113,10 @@
%% take care of it.
-type component() :: search | index.
--type solr_entry() :: {bkey(), obj(), write_reason(), p(), short_preflist(),
+
+-type object_pair() :: {obj(), obj() | no_old_object}.
+
+-type solr_entry() :: {bkey(), object_pair(), write_reason(), p(), short_preflist(),
hash()}.
-type solr_entries() :: [solr_entry()].
@@ -263,6 +266,15 @@
-define(YZ_ENTROPY_LOCK_TIMEOUT,
app_helper:get_env(?YZ_APP_NAME, anti_entropy_lock_timeout, 10000)).
+-define(YZ_ENABLE_DIST_QUERY,
+ app_helper:get_env(?YZ_APP_NAME, enable_dist_query, true)).
+
+-define(YZ_ENTROPY_THROTTLE_KEY, aae_throttle).
+-define(YZ_ENTROPY_THROTTLE_LIMITS_KEY, aae_throttle_limits).
+-define(YZ_ENTROPY_THROTTLE_DEFAULT_LIMITS,
+ [{-1,0}, {500,10}, {1000,50}, {5000,250}, {10000,1000}, {50000,3000}]).
+-define(YZ_ENTROPY_THROTTLE_ENABLED_KEY, aae_throttle_enabled).
+
-type hashtree() :: hashtree:hashtree().
-type exchange() :: {p(), {p(), n()}}.
-type exchange_mode() :: automatic | manual.
@@ -358,6 +370,7 @@
-define(TOMBSTONE, <<>>).
-define(YZ_INDEX_TOMBSTONE, <<"_dont_index_">>).
+-define(YZ_SHOULD_INDEX(Index), Index =/= ?YZ_INDEX_TOMBSTONE).
-define(YZ_INDEX, search_index).
%%%===================================================================
@@ -498,12 +511,13 @@
%%%===================================================================
-define(SOLRQ_BATCH_MIN, solrq_batch_min).
+-define(SOLRQ_BATCH_MIN_DEFAULT, 10).
-define(SOLRQ_BATCH_MAX, solrq_batch_max).
+-define(SOLRQ_BATCH_MAX_DEFAULT, 500).
-define(SOLRQ_BATCH_FLUSH_INTERVAL, solrq_batch_flush_interval).
+-define(SOLRQ_BATCH_FLUSH_INTERVAL_DEFAULT, 500).
-define(SOLRQ_HWM, solrq_hwm).
--define(SOLRQ_HWM_PURGE, solrq_hwm_purge).
--define(SOLRQ_WORKER_COUNT, solrq_worker_count).
--define(SOLRQ_HELPER_COUNT, solrq_helper_count).
+-define(SOLRQ_HWM_DEFAULT, 1000).
-define(SOLRQ_HWM_PURGE_STRATEGY, solrq_hwm_purge_strategy).
-define(PURGE_NONE, off).
-define(PURGE_ONE, purge_one).
@@ -514,7 +528,7 @@
-type solrq_batch_max() :: pos_integer().
-type solrq_batch_flush_interval() :: non_neg_integer()|infinity.
-type solrq_hwm() :: non_neg_integer().
--type purge_strategy() :: ?PURGE_NONE|?PURGE_ONE|?PURGE_IDX|?PURGE_ALL.
+-type purge_strategy() :: ?PURGE_NONE|?PURGE_ONE|?PURGE_IDX.
%%%===================================================================
%%% draining
@@ -524,8 +538,12 @@
-define(YZ_INDEX_HASHTREE_PARAMS, yz_index_hashtree_update_params).
-define(DRAIN_PARTITION, drain_partition).
-define(SOLRQ_DRAIN_TIMEOUT, solrq_drain_timeout).
+-define(SOLRQ_DRAIN_TIMEOUT_DEFAULT, 60000).
-define(SOLRQ_DRAIN_ENABLE, solrq_drain_enable).
+-define(SOLRQ_DRAIN_ENABLE_DEFAULT, true).
-define(SOLRQ_DRAIN_CANCEL_TIMEOUT, solrq_drain_cancel_timeout).
+-define(SOLRQ_DRAIN_CANCEL_TIMEOUT_DEFAULT, 5000).
+
-type drain_param() ::
{?EXCHANGE_FSM_PID, pid()} |
diff --git a/priv/conf/solrconfig.xml b/priv/conf/solrconfig.xml
index 04113997..a4008a38 100644
--- a/priv/conf/solrconfig.xml
+++ b/priv/conf/solrconfig.xml
@@ -178,7 +178,7 @@
-->
- 1000
+ 500
">>),
+ file:close(File),
+ ok.
+
+
+%%
+%% Simple wrapper around yz_rt:create_indexed_bucket_type/4, because
+%% the pmap callback only takes a tuple
+%%
+create_indexed_bucket_type({Cluster, BucketType, Index, BucketProperties}) ->
+ ok = yz_rt:create_indexed_bucket_type(Cluster, BucketType, Index, BucketProperties).
+
+
+%%
+%% write and verify one one go
+%%
+write_and_verify_data(Cluster, Bucket, Index, Start, Num, Expected) ->
+ true = write_data(Cluster, Bucket, Start, Num),
+ ok = verify_data(Cluster, Index, Expected).
diff --git a/riak_test/yz_solrq_test.erl b/riak_test/yz_solrq_test.erl
index ef17632f..27db111a 100644
--- a/riak_test/yz_solrq_test.erl
+++ b/riak_test/yz_solrq_test.erl
@@ -58,18 +58,15 @@
?INDEX6, ?INDEX7, ?INDEX8, ?INDEX9, ?INDEX10,
?INDEX11, ?INDEX12]).
--define(NUM_SOLRQ, 3).
--define(NUM_SOLRQ_HELPERS, 3).
-define(SOLRQ_DELAYMS_MAX, 3000).
-define(SOLRQ_BATCH_MIN_SETTING, 4).
-define(SOLRQ_BATCH_MAX_SETTING, 8).
-define(MELT_RESET_REFRESH, 1000).
-define(SOLRQ_HWM_SETTING, 20).
+-define(RING_SIZE, 8).
-define(CONFIG,
[{yokozuna,
[{enabled, true},
- {?SOLRQ_WORKER_COUNT, ?NUM_SOLRQ},
- {?SOLRQ_WORKER_COUNT, ?NUM_SOLRQ_HELPERS},
{?SOLRQ_BATCH_FLUSH_INTERVAL, ?SOLRQ_DELAYMS_MAX},
{?SOLRQ_BATCH_MIN, ?SOLRQ_BATCH_MIN_SETTING},
{?SOLRQ_BATCH_MAX, ?SOLRQ_BATCH_MAX_SETTING},
@@ -78,7 +75,8 @@
{?ERR_THRESH_RESET_INTERVAL, ?MELT_RESET_REFRESH},
{?SOLRQ_DRAIN_ENABLE, true},
{anti_entropy, {off, []}}
- ]}]).
+ ]},
+ {riak_core, [{ring_creation_size, ?RING_SIZE}]}]).
-compile(export_all).
@@ -113,17 +111,18 @@ confirm() ->
pass.
confirm_drain_fsm_failure(Cluster) ->
+ lager:info("Starting confirm_drain_fsm_failure"),
yz_stat:reset(),
try
yz_rt:load_intercept_code(Cluster),
- yz_rt:add_intercept(Cluster, yz_solrq_drain_fsm, prepare, 2, prepare_crash),
+ yz_rt:add_intercepts(Cluster, yz_solrq_drain_fsm, [{{prepare, 2}, prepare_crash}]),
%% drain solrqs and wait until the drain failure stats are touched
yz_rt:drain_solrqs(Cluster),
yz_rt:wait_until(Cluster, fun check_drain_failure_stats/1),
lager:info("confirm_drain_fsm_failure ok")
after
- yz_rt:add_intercept(Cluster, yz_solrq_drain_fsm, prepare, 2, prepare_orig)
+ yz_rt:add_intercepts(Cluster, yz_solrq_drain_fsm, [{{prepare, 2}, prepare_orig}])
end.
check_drain_failure_stats(Node) ->
@@ -140,19 +139,20 @@ check_drain_failure_stats(Node) ->
yz_rt:check_stat_values(Stats, Pairs).
confirm_drain_fsm_timeout(Cluster) ->
+ lager:info("Starting confirm_drain_fsm_timeout"),
yz_stat:reset(),
[rpc:call(
- Node, application, set_env, [?YZ_APP_NAME, ?SOLRQ_DRAIN_TIMEOUT, 500])
+ Node, application, set_env, [?YZ_APP_NAME, ?SOLRQ_DRAIN_TIMEOUT, 250])
|| Node <- Cluster],
try
yz_rt:load_intercept_code(Cluster),
- yz_rt:add_intercept(Cluster, yz_solrq_drain_fsm, prepare, 2, prepare_sleep_1s),
+ yz_rt:add_intercepts(Cluster, yz_solrq_drain_fsm, [{{resume_workers, 1}, resume_workers_sleep_1s}]),
yz_rt:drain_solrqs(Cluster),
yz_rt:wait_until(Cluster, fun check_drain_timeout_stats/1),
lager:info("confirm_drain_fsm_timeout ok")
after
- yz_rt:add_intercept(Cluster, yz_solrq_drain_fsm, prepare, 2, prepare_orig),
+ yz_rt:add_intercepts(Cluster, yz_solrq_drain_fsm, [{{resume_workers, 1}, resume_workers_orig}]),
[rpc:call(
Node, application, set_env, [?YZ_APP_NAME, ?SOLRQ_DRAIN_TIMEOUT, 60000])
|| Node <- Cluster]
@@ -172,17 +172,21 @@ check_drain_timeout_stats(Node) ->
yz_rt:check_stat_values(Stats, Pairs).
confirm_drain_fsm_kill(Cluster) ->
+ lager:info("Starting confirm_drain_fsm_kill"),
[rpc:call(
Node, application, set_env, [?YZ_APP_NAME, ?SOLRQ_DRAIN_TIMEOUT, 10])
|| Node <- Cluster],
+ %% technically not needed for this test (because the cancel intercept will
+ %% just return timeout), but added for completeness
[rpc:call(
Node, application, set_env, [?YZ_APP_NAME, ?SOLRQ_DRAIN_CANCEL_TIMEOUT, 10])
|| Node <- Cluster],
try
yz_test_listener:start(),
yz_rt:load_intercept_code(Cluster),
- yz_rt:add_intercept(Cluster, yz_solrq_drain_fsm, prepare, 2, prepare_sleep_5s),
- yz_rt:add_intercept(Cluster, yz_solrq_drain_mgr, unlink_and_kill, 2, count_unlink_and_kill),
+ yz_rt:add_intercepts(Cluster, yz_solrq_drain_fsm, [{{resume_workers, 1}, resume_workers_sleep_1s},
+ {{cancel, 2}, cancel_timeout}]),
+ yz_rt:add_intercepts(Cluster, yz_solrq_drain_mgr, [{{unlink_and_kill, 2}, count_unlink_and_kill}]),
yz_rt:drain_solrqs(Cluster),
yz_rt:wait_until(Cluster, fun check_drain_cancel_timeout_stats/1),
@@ -190,8 +194,9 @@ confirm_drain_fsm_kill(Cluster) ->
lager:info("confirm_drain_fsm_kill ok")
after
- yz_rt:add_intercept(Cluster, yz_solrq_drain_fsm, prepare, 2, prepare_orig),
- yz_rt:add_intercept(Cluster, yz_solrq_drain_mgr, unlink_and_kill, 2, unlink_and_kill_orig),
+ yz_rt:add_intercepts(Cluster, yz_solrq_drain_fsm, [{{resume_workers, 1}, resume_workers_orig},
+ {{cancel, 2}, cancel_orig}]),
+ yz_rt:add_intercepts(Cluster, yz_solrq_drain_mgr, [{{unlink_and_kill, 2}, unlink_and_kill_orig}]),
yz_test_listener:stop(),
[rpc:call(
Node, application, set_env, [?YZ_APP_NAME, ?SOLRQ_DRAIN_TIMEOUT, 60000])
@@ -216,6 +221,7 @@ check_drain_cancel_timeout_stats(Node) ->
confirm_batch_size(Cluster, PBConn, BKey, Index) ->
+ lager:info("Starting confirm_batch_size"),
%% First, put one less than the min batch size and expect that there are no
%% search results (because the index operations are queued).
Count = ?SOLRQ_BATCH_MIN_SETTING - 1,
@@ -248,11 +254,12 @@ confirm_batch_size(Cluster, PBConn, BKey, Index) ->
ok.
confirm_hwm(Cluster, PBConn, Bucket, Index, HWM) ->
+ lager:info("Starting confirm_hwm"),
yz_rt:drain_solrqs(Cluster),
{OldMin, OldMax, OldDelay} = set_index(Cluster, Index, 1, 100, 100),
try
yz_rt:load_intercept_code(Cluster),
- yz_rt:intercept_index_batch(Cluster, index_batch_throw_exception),
+ yz_rt:intercept_index_batch(Cluster, index_batch_returns_other_error),
yz_rt:set_hwm(Cluster, HWM),
yz_rt:set_purge_strategy(Cluster, ?PURGE_NONE),
?assertEqual(HWM, put_objects(PBConn, Bucket, HWM + 1))
@@ -268,9 +275,10 @@ confirm_hwm(Cluster, PBConn, Bucket, Index, HWM) ->
-spec gteq(number(), number()) -> boolean().
gteq(A, B) -> A >= B.
-confirm_draining(Cluster, PBConn, BKey, Index) ->
+confirm_draining(Cluster, PBConn, Bucket, Index) ->
+ lager:info("Starting confirm_draining"),
Count = ?SOLRQ_BATCH_MIN_SETTING - 1,
- Count = put_objects(PBConn, BKey, Count),
+ Count = put_objects(PBConn, Bucket, Count),
yz_rt:commit(Cluster, Index),
verify_search_count(PBConn, Index, 0),
yz_rt:drain_solrqs(Cluster),
@@ -280,14 +288,15 @@ confirm_draining(Cluster, PBConn, BKey, Index) ->
ok.
confirm_requeue_undelivered([Node|_] = Cluster, PBConn, BKey, Index) ->
+ lager:info("Starting confirm_requeue_undelivered"),
yz_rt:load_intercept_code(Node),
- yz_rt:intercept_index_batch(Node, index_batch_throw_exception),
+ yz_rt:intercept_index_batch(Node, index_batch_returns_other_error),
Count = ?SOLRQ_BATCH_MIN_SETTING,
Count = put_objects(PBConn, BKey, Count),
yz_rt:commit(Cluster, Index),
- %% Because the index_batch_throw_exception intercept simulates a Solr
+ %% Because the index_batch_returns_other_error intercept simulates a Solr
%% failure, none of the objects should have been indexed at this point.
verify_search_count(PBConn, Index, 0),
@@ -302,6 +311,7 @@ confirm_requeue_undelivered([Node|_] = Cluster, PBConn, BKey, Index) ->
ok.
confirm_no_contenttype_data(Cluster, PBConn, BKey, Index) ->
+ lager:info("Starting confirm_no_contenttype_data"),
yz_rt:set_index(Cluster, Index, 1, 100, 100),
Count = 1,
Count = put_no_contenttype_objects(PBConn, BKey, Count),
@@ -311,34 +321,23 @@ confirm_no_contenttype_data(Cluster, PBConn, BKey, Index) ->
ok.
confirm_purge_strategy(Cluster, PBConn) ->
+ lager:info("Starting confirm_purge_strategy"),
confirm_purge_one_strategy(Cluster, PBConn,
- {?BUCKET5, ?INDEX5},
- {?BUCKET6, ?INDEX6}),
+ {?BUCKET5, ?INDEX5}),
confirm_purge_idx_strategy(Cluster, PBConn,
- {?BUCKET7, ?INDEX7},
- {?BUCKET8, ?INDEX8}),
- confirm_purge_all_strategy(Cluster, PBConn,
- {?BUCKET9, ?INDEX9},
- {?BUCKET10, ?INDEX10}),
+ {?BUCKET7, ?INDEX7}),
confirm_purge_none_strategy(Cluster, PBConn,
- {?BUCKET11, ?INDEX11},
- {?BUCKET12, ?INDEX12}),
+ {?BUCKET11, ?INDEX11}),
ok.
-confirm_purge_one_strategy(Cluster, PBConn, Bucket1Index1, Bucket2Index2) ->
- PurgeResults = do_purge(Cluster, PBConn, Bucket1Index1, Bucket2Index2,
- ?PURGE_ONE),
+confirm_purge_one_strategy(Cluster, PBConn, Bucket1Index1) ->
+ PurgeResults = do_purge(Cluster, PBConn, Bucket1Index1, ?PURGE_ONE),
check_one_purged(PurgeResults),
lager:info("confirm_purge_one_strategy ok"),
ok.
-check_one_purged({{Index1Written, Index1SearchResults},
- {Index2Written, Index2SearchResults}} = TestResults) ->
- Condition =
- equal(Index1Written, Index1SearchResults)
- andalso first_purged(Index2Written, Index2SearchResults)
- orelse first_purged(Index1Written, Index1SearchResults)
- andalso equal(Index2Written, Index2SearchResults),
+check_one_purged({Index1Written, Index1SearchResults} = TestResults) ->
+ Condition = first_purged(Index1Written, Index1SearchResults),
case Condition of
false ->
lager:error("check_one_purged error: ~p", [TestResults]);
@@ -347,15 +346,13 @@ check_one_purged({{Index1Written, Index1SearchResults},
?assertEqual(Condition, true),
ok.
-confirm_purge_idx_strategy(Cluster, PBConn, Bucket1Index1, Bucket2Index2) ->
- PurgeResults = do_purge(Cluster, PBConn, Bucket1Index1, Bucket2Index2,
- ?PURGE_IDX),
+confirm_purge_idx_strategy(Cluster, PBConn, Bucket1Index1) ->
+ PurgeResults = do_purge(Cluster, PBConn, Bucket1Index1, ?PURGE_IDX),
check_idx_purged(PurgeResults),
lager:info("confirm_purge_idx_strategy ok"),
ok.
-check_idx_purged({{[_K1, _K2, K3] = Index1Written, Index1SearchResults},
- {Index2Written, Index2SearchResults}} = TestResults) ->
+check_idx_purged({[_K1, _K2, _K3, _K4, K5] = _Index1Written, Index1SearchResults} = TestResults) ->
%%
%% Note the second condition, because we wrote to Index1
%% but that was the purge trigger, so the last entry will
@@ -363,11 +360,7 @@ check_idx_purged({{[_K1, _K2, K3] = Index1Written, Index1SearchResults},
%% Otherwise, it was the second indexq, and nothing should
%% have been pending for that indexq, so they all get deleted.
%%
- Condition =
- equal(Index1Written, Index1SearchResults)
- andalso equal([], Index2SearchResults)
- orelse equal(Index1SearchResults, [K3])
- andalso equal(Index2Written, Index2SearchResults),
+ Condition = equal(Index1SearchResults, [K5]),
case Condition of
false ->
lager:error("check_idx_purged error: ~p", [TestResults]);
@@ -376,44 +369,16 @@ check_idx_purged({{[_K1, _K2, K3] = Index1Written, Index1SearchResults},
?assertEqual(Condition, true),
ok.
-confirm_purge_all_strategy(Cluster, PBConn, Bucket1Index1, Bucket2Index2) ->
- PurgeResults = do_purge(Cluster, PBConn, Bucket1Index1, Bucket2Index2,
- ?PURGE_ALL),
- check_all_purged(PurgeResults),
- lager:info("confirm_purge_all_strategy ok"),
- ok.
-
-check_all_purged({{[_K1, _K2, K3] = _Index1Written, Index1SearchResults},
- {_Index2Written, Index2SearchResults}} = TestResults) ->
- %%
- %% Note the first condition, because we wrote to Index1
- %% but that was the purge trigger, so the last entry will
- %% NOT have been purged, of that indexq was the one chosen.
- %% Otherwise, it was the second indexq, and nothing should
- %% have been pending for that indexq, so they all get deleted.
- %%
- Condition =
- equal(Index1SearchResults, [K3]) andalso equal([], Index2SearchResults),
- case Condition of
- false ->
- lager:error("check_all_purged error: ~p", [TestResults]);
- _ -> ok
- end,
- ?assertEqual(Condition, true),
- ok.
-
-confirm_purge_none_strategy(Cluster, PBConn, Bucket1Index1, Bucket2Index2) ->
- PurgeResults = do_purge(Cluster, PBConn, Bucket1Index1, Bucket2Index2,
+confirm_purge_none_strategy(Cluster, PBConn, Bucket1Index1) ->
+ PurgeResults = do_purge(Cluster, PBConn, Bucket1Index1,
?PURGE_NONE),
check_none_purged(PurgeResults),
lager:info("confirm_purge_none_strategy ok"),
ok.
-check_none_purged({{Index1Written, Index1SearchResults},
- {Index2Written, Index2SearchResults}} = TestResults) ->
+check_none_purged({Index1Written, Index1SearchResults} = TestResults) ->
Condition =
- equal(Index1Written, Index1SearchResults)
- andalso equal(Index2Written, Index2SearchResults),
+ equal(Index1Written, Index1SearchResults),
case Condition of
false ->
lager:error("check_none_purged error: ~p", [TestResults]);
@@ -447,49 +412,52 @@ first_purged([_H|T] = _Written, Searched) ->
%% of this function.
%%
do_purge([Node|_] = Cluster, PBConn,
- {Bucket1, Index1},
- {Bucket2, Index2},
+ {Bucket, Index},
PurgeStrategy) ->
yz_rt:set_purge_strategy(Cluster, PurgeStrategy),
- yz_rt:set_index(Cluster, Index1, 1, 100, 99999),
- yz_rt:set_index(Cluster, Index2, 1, 100, 99999),
+ yz_rt:set_index(Cluster, Index, 1, 100, 99999),
yz_rt:set_hwm(Cluster, 4),
+ TargetPartition = 1096126227998177188652763624537212264741949407232,
%%
- %% Find a list of representative keys for each Index.
+ %% Find a list of representative keys.
%% Each representative in the list is a unique key
- %% that hashes to yz_solrq_001.
+ %% that hashes to the target partition.
%%
- Index1BKeys = find_representative_bkeys(Node, Index1, Bucket1),
- Index2BKeys = find_representative_bkeys(Node, Index2, Bucket2),
- Index1BKey1 = lists:nth(1, Index1BKeys),
- Index1BKey2 = lists:nth(2, Index1BKeys),
- Index1BKey3 = lists:nth(3, Index1BKeys),
- Index2BKey1 = lists:nth(1, Index2BKeys),
- Index2BKey2 = lists:nth(2, Index2BKeys),
+ RepresentativeKeys = find_representative_bkeys(TargetPartition, Index, Bucket),
+ Key1 = lists:nth(1, RepresentativeKeys),
+ Key2 = lists:nth(2, RepresentativeKeys),
+ Key3 = lists:nth(3, RepresentativeKeys),
+ Key4 = lists:nth(4, RepresentativeKeys),
+ Key5 = lists:nth(5, RepresentativeKeys),
try
yz_rt:load_intercept_code(Node),
- yz_rt:intercept_index_batch(Node, index_batch_throw_exception),
+ yz_rt:intercept_index_batch(Node, index_batch_returns_other_error),
%%
- %% Send two messages through each indexq on the solrq, which
- %% will trip the fuse on both; however
+ %% Send messages through the index on the solrq, which
+ %% will trip the fuse; however
%% because fuse blown events are handled asynchronously,
- %% we need to wait until the solrqs are blown.
+ %% we need to wait until the solrq is blown.
%%
- [Index1BKey1] = put_bkey_objects(PBConn, [Index1BKey1]),
- [Index1BKey2] = put_bkey_objects(PBConn, [Index1BKey2]),
- [Index2BKey1] = put_bkey_objects(PBConn, [Index2BKey1]),
- [Index2BKey2] = put_bkey_objects(PBConn, [Index2BKey2]),
- yz_rt:wait_until_fuses_blown(Node, yz_solrq_worker_0001, [Index1, Index2]),
+ [Key1] = put_bkey_objects(PBConn, [Key1]),
+ [Key2] = put_bkey_objects(PBConn, [Key2]),
+ [Key3] = put_bkey_objects(PBConn, [Key3]),
+ [Key4] = put_bkey_objects(PBConn, [Key4]),
+ yz_rt:wait_until_fuses_blown(Node, TargetPartition, [Index]),
%%
- %% At this point, the two indexqs in yz_solrq_worker_001 corresponding
- %% to Index1 and Index2, respectively, should be blown.
- %% Send one more message through one of the Indexqs, which
+ %% At this point, the target solrq corresponding
+ %% to {TargetPartition, Index1}, should be blown.
+ %% Send one more message through the Indexq, which
%% will trigger a purge.
%%
+ yz_rt:set_index(Cluster, Index, 1, 100, 99999),
F = fun() ->
- [Index1BKey3] = put_bkey_objects(PBConn, [Index1BKey3])
+ [Key5] = put_bkey_objects(PBConn, [Key5])
end,
case PurgeStrategy of
+ %% If testing PURGE_NONE, all vnodes are blocked
+ %% spawn the call so we don't block the test.of
+ %% Note that this will result in an error showing up
+ %% in the test log, but that's ok as we expect it.
?PURGE_NONE ->
spawn(F);
_ ->
@@ -501,22 +469,17 @@ do_purge([Node|_] = Cluster, PBConn,
%% fuse to reset. Commit to Solr so that we can run a query.
%%
yz_rt:intercept_index_batch(Node, index_batch_call_orig),
- yz_rt:wait_until_fuses_reset(Node, yz_solrq_worker_0001, [Index1, Index2]),
+ yz_rt:wait_until_fuses_reset(Node, TargetPartition, [Index]),
yz_rt:drain_solrqs(Node),
- yz_rt:commit(Cluster, Index1),
- yz_rt:commit(Cluster, Index2)
+ yz_rt:commit(Cluster, Index)
end,
%%
- %% Return the search results for Index1 and Index2.
- %% The first list is the set of bkeys we wrote for each index.
+ %% Return the search results for the index.
+ %% The first list is the set of bkeys we wrote.
%% The second list is the set that are available for search.
%%
- Index1SearchBKeys = search_bkeys(PBConn, Index1),
- Index2SearchBKeys = search_bkeys(PBConn, Index2),
- {
- {[Index1BKey1, Index1BKey2, Index1BKey3], Index1SearchBKeys},
- {[Index2BKey1, Index2BKey2], Index2SearchBKeys}
- }.
+ KeysFound = search_bkeys(PBConn, Index),
+ {[Key1, Key2, Key3, Key4, Key5], KeysFound}.
-spec search_bkeys(pid(), index_name()) -> [bkey()].
search_bkeys(PBConn, Index) ->
@@ -530,36 +493,24 @@ search_bkeys(PBConn, Index) ->
end,
SearchResults).
--spec find_representative_bkeys(node(), index_name(), bucket()) -> [bkey()].
-find_representative_bkeys(Node, Index, Bucket) ->
- find_representative_bkeys(Node, Index, Bucket, yz_solrq_worker_0001).
-
--spec find_representative_bkeys(node(),
+-spec find_representative_bkeys(p(),
index_name(),
- bucket(), module()) -> [bkey()].
-find_representative_bkeys(Node, Index, Bucket, Solrq) ->
- {ok, BKeys} = dict:find(Solrq, find_representatives(Node, Index, Bucket)),
+ bucket()) -> [bkey()].
+find_representative_bkeys(Partition, Index, Bucket) ->
+ Solrq = yz_solrq:worker_regname(Index, Partition),
+ Representatives = find_representatives(Index, Bucket),
+ {ok, BKeys} = dict:find(Solrq, Representatives),
BKeys.
--spec find_representatives(node(), index_name(), bucket()) -> dict().
-find_representatives(Node, Index, Bucket) ->
+-spec find_representatives(index_name(), bucket()) -> dict().
+find_representatives(Index, Bucket) ->
BKeys =
lists:map(
fun(I) ->
{Bucket, erlang:list_to_binary(erlang:integer_to_list(I))}
end,
- lists:seq(1, 100)),
-
- lists:foldl(
- fun({Solrq, BKey}, Accum) ->
- dict:append(Solrq, BKey, Accum)
- end,
- dict:new(),
- [{get_solrq(Node, Index, BKey), BKey} || BKey <- BKeys]).
-
--spec get_solrq(node(), index_name(), bkey()) -> module().
-get_solrq(Node, Index, BKey) ->
- rpc:call(Node, yz_solrq, worker_regname, [erlang:phash2({Index, BKey})]).
+ lists:seq(1, 1000)),
+ yz_rt:find_representatives(Index, BKeys, ?RING_SIZE).
-spec put_no_contenttype_objects(pid(), bucket(), non_neg_integer()) -> non_neg_integer().
put_no_contenttype_objects(PBConn, Bucket, Count) ->
@@ -583,10 +534,12 @@ put_objects(PBConn, Bucket, Count, ContentType) ->
end || Val <- RandVals],
lager:info("Writing ~p objects to Riak...", [length(Objects)]),
Results = [riakc_pb_socket:put(PBConn, Obj, [return_head, {timeout, 1000}]) || Obj <- Objects],
- length(lists:filter(fun({Result, _}) ->
- ok =:= Result
- end,
- Results)).
+ NoWritten = length(lists:filter(fun({Result, _}) ->
+ ok =:= Result
+ end,
+ Results)),
+ lager:info("Wrote ~p objects to Riak...", [NoWritten]),
+ NoWritten.
-spec put_bkey_objects(pid(), [bkey()]) -> [bkey()].
put_bkey_objects(PBConn, BKeys) ->
@@ -600,7 +553,16 @@ put_bkey_objects(PBConn, BKeys) ->
%%lager:info("Results: ~p", [Results]),
lists:map(
fun({BKey, {_Result, _}}) -> BKey end,
- lists:filter(fun({_BKey, {Result, _}}) -> ok =:= Result end, Results)).
+ lists:filter(
+ fun({BKey, {Result, _}}) ->
+ case Result of
+ ok -> true;
+ _ ->
+ lager:info("The result of a put was not ok. BKey: ~p Result: ~p", [BKey, Result]),
+ false
+ end
+ end,
+ Results)).
verify_search_count(PBConn, Index, Count) ->
{ok, {search_results, _R, _Score, Found}} =
diff --git a/riak_test/yz_stat_test.erl b/riak_test/yz_stat_test.erl
index 1be9a676..d626a514 100644
--- a/riak_test/yz_stat_test.erl
+++ b/riak_test/yz_stat_test.erl
@@ -15,6 +15,7 @@
-define(NUM_NODES, 1).
-define(RING_SIZE, 8).
-define(NUM_ENTRIES, 10).
+-define(EVENT_TICK_INTERVAL, 1000).
-define(CFG, [
{riak_core, [
@@ -29,7 +30,6 @@
]},
{yokozuna, [
{enabled, true},
- {?SOLRQ_WORKER_COUNT, 1},
{?SOLRQ_BATCH_MIN, 6},
{?SOLRQ_BATCH_FLUSH_INTERVAL, 100000},
{?SOLRQ_HWM, 10},
@@ -39,7 +39,12 @@
%% allow AAE to build trees and exchange rapidly
{anti_entropy_tick, 1000},
{anti_entropy_build_limit, {100, 1000}},
- {anti_entropy_concurrency, 8}
+ {anti_entropy_concurrency, 8},
+ %% Force a complete check of indexes after every tick
+ %% to make yokozuna remove/add indexes faster
+ %% Needed for recreate test
+ {events_full_check_after, 1},
+ {events_tick_interval, ?EVENT_TICK_INTERVAL}
]}
]).
@@ -55,16 +60,16 @@ prepare_cluster(NumNodes) ->
Cluster.
confirm_stats(Cluster) ->
- {Host, Port} = yz_rt:select_random(
- [yz_rt:riak_pb(I) || {_,I} <- rt:connection_info(Cluster)]
- ),
+ PBConn0 = get_pb_connection(Cluster),
Index = <<"yz_stat_test">>,
Bucket = {Index, <<"b1">>},
+ create_indexed_bucket(Cluster, Bucket, Index, PBConn0),
- {ok, PBConn} = riakc_pb_socket:start_link(Host, Port),
- yz_rt:create_indexed_bucket(PBConn, Cluster, Bucket, Index, ?N_VAL),
- {ok, BProps} = riakc_pb_socket:get_bucket(PBConn, Bucket),
- ?assertEqual(?N_VAL, proplists:get_value(n_val, BProps)),
+ %% because the cluster will die w/o fix for this confirm step,
+ %% it closes the PB connection. Will get another once it's done
+ confirm_recreate_indexed_bucket(Cluster, Bucket, Index, PBConn0),
+
+ PBConn = get_pb_connection(Cluster),
%%
%% Clear the yz and kv hashtrees, because we have created a new
%% bucket type with a different n_vals.
@@ -76,16 +81,41 @@ confirm_stats(Cluster) ->
yz_rt:set_yz_aae_mode(Cluster, manual),
+ Values = confirm_index_stats(Cluster, PBConn, Bucket, Index),
+
+ confirm_query_stats(Cluster, PBConn, Index, Values),
+
+ confirm_aae_repair_and_stats(Cluster, Index, Bucket, Values),
+
+ confirm_extract_fail_stats(Cluster, PBConn, Bucket),
+
+ confirm_fuse_and_purge_stats(Cluster, PBConn, Index, Bucket),
+
+ riakc_pb_socket:stop(PBConn).
+
+get_pb_connection(Cluster) ->
+ {Host, Port} = yz_rt:select_random(
+ [yz_rt:riak_pb(I) || {_, I} <- rt:connection_info(Cluster)]
+ ),
+ {ok, PBConn} = riakc_pb_socket:start_link(Host, Port),
+ PBConn.
+
+create_indexed_bucket(Cluster, Bucket, Index, PBConn) ->
+ yz_rt:create_indexed_bucket(PBConn, Cluster, Bucket, Index, ?N_VAL),
+ {ok, BProps} = riakc_pb_socket:get_bucket(PBConn, Bucket),
+ ?assertEqual(?N_VAL, proplists:get_value(n_val, BProps)).
+
+confirm_fuse_and_purge_stats(Cluster, PBConn, Index, Bucket) ->
yz_rt:reset_stats(Cluster),
- Values = populate_data_and_wait(PBConn, Cluster, Bucket, Index, ?NUM_ENTRIES),
- yz_rt:verify_num_match(yokozuna, Cluster, Index, ?NUM_ENTRIES),
- yz_rt:verify_num_match(solr, Cluster, Index, ?NUM_ENTRIES * ?N_VAL),
- yz_rt:wait_until(Cluster, fun check_index_stats/1),
+ blow_fuses(Cluster, PBConn, Index, Bucket),
+ yz_rt:wait_until(Cluster, fun check_fuse_and_purge_stats/1).
+confirm_extract_fail_stats(Cluster, PBConn, Bucket) ->
yz_rt:reset_stats(Cluster),
- search_values(PBConn, Index, Values),
- yz_rt:wait_until(Cluster, fun check_query_stats/1),
+ write_bad_json(Cluster, PBConn, Bucket, 1),
+ yz_rt:wait_until(Cluster, fun check_index_extract_fail_stats/1).
+confirm_aae_repair_and_stats(Cluster, Index, Bucket, Values) ->
yz_rt:reset_stats(Cluster),
[delete_key_in_solr(Cluster, Index, {Bucket, K}) || K <- Values],
yz_rt:verify_num_match(yokozuna, Cluster, Index, 0),
@@ -94,20 +124,22 @@ confirm_stats(Cluster) ->
yz_rt:wait_for_full_exchange_round(Cluster),
yz_rt:drain_solrqs(Cluster),
yz_rt:wait_until(Cluster, fun check_aae_stats/1),
-
yz_rt:reset_stats(Cluster),
yz_rt:verify_num_match(yokozuna, Cluster, Index, ?NUM_ENTRIES),
- yz_rt:verify_num_match(solr, Cluster, Index, ?NUM_ENTRIES * ?N_VAL),
+ yz_rt:verify_num_match(solr, Cluster, Index, ?NUM_ENTRIES * ?N_VAL).
+confirm_query_stats(Cluster, PBConn, Index, Values) ->
yz_rt:reset_stats(Cluster),
- write_bad_json(Cluster, PBConn, Bucket, 1),
- yz_rt:wait_until(Cluster, fun check_index_fail_stats/1),
+ search_values(PBConn, Index, Values),
+ yz_rt:wait_until(Cluster, fun check_query_stats/1).
+confirm_index_stats(Cluster, PBConn, Bucket, Index) ->
yz_rt:reset_stats(Cluster),
- blow_fuses(Cluster, PBConn, Index, Bucket),
- yz_rt:wait_until(Cluster, fun check_fuse_and_purge_stats/1),
-
- riakc_pb_socket:stop(PBConn).
+ Values = populate_data_and_wait(PBConn, Cluster, Bucket, Index, ?NUM_ENTRIES),
+ yz_rt:verify_num_match(yokozuna, Cluster, Index, ?NUM_ENTRIES),
+ yz_rt:verify_num_match(solr, Cluster, Index, ?NUM_ENTRIES * ?N_VAL),
+ yz_rt:wait_until(Cluster, fun check_index_stats/1),
+ Values.
clear_hashtrees(Cluster) ->
yz_rt:clear_kv_trees(Cluster),
@@ -121,11 +153,15 @@ populate_data(_, _, 0, Acc) ->
Acc;
populate_data(Pid, Bucket, Count, Acc)->
KV = gen_random_name(16),
- PO = riakc_obj:new(Bucket, KV, KV, "text/plain"),
- ok = riakc_pb_socket:put(Pid, PO, []),
- lager:info("Wrote Bucket ~p key ~p", [Bucket, KV]),
+ write_key(Pid, Bucket, KV, KV),
populate_data(Pid, Bucket, Count - 1, [KV|Acc]).
+write_key(Pid, Bucket, Key, Value) ->
+ PO = riakc_obj:new(Bucket, Key, Value, "text/plain"),
+ ok = riakc_pb_socket:put(Pid, PO, []),
+ lager:info("Wrote Bucket ~p key ~p", [Bucket, Key]).
+
+
populate_data_and_wait(Pid, Cluster, Bucket, Index, Count) ->
Values = populate_data(Pid, Bucket, Count, []),
yz_rt:wait_until(Cluster, fun check_queue_total_length_stats/1),
@@ -169,9 +205,10 @@ blow_fuses(Cluster, PBConn, Index, Bucket) ->
yz_rt:set_index(Cluster, Index, 1, 99999, 99999),
yz_rt:set_hwm(Cluster, 1),
yz_rt:set_purge_strategy(Cluster, ?PURGE_ONE),
+ Key = <<"blow_a_fuse">>,
try
[yz_rt:load_intercept_code(Node) || Node <- Cluster],
- yz_rt:intercept_index_batch(Cluster, index_batch_throw_exception),
+ yz_rt:intercept_index_batch(Cluster, index_batch_returns_other_error),
%%
%% Send a message through each the indexq on the solrq, which
%% will trip the fuse; however
@@ -179,8 +216,8 @@ blow_fuses(Cluster, PBConn, Index, Bucket) ->
%% we need to wait until the solrqs are blown.
%%
lager:info("Writing one entry to blow fuse..."),
- populate_data(PBConn, Bucket, 1),
- yz_rt:wait_until_fuses_blown(Cluster, yz_solrq_worker_0001, [Index]),
+ write_key(PBConn, Bucket, Key, Key),
+ yz_rt:wait_until_fuses_blown(Cluster, 0, [Index]),
%%
%% At this point, the indexq in yz_solrq_worker_0001 corresponding
%% to the Index should be blown.
@@ -188,16 +225,16 @@ blow_fuses(Cluster, PBConn, Index, Bucket) ->
%% will trigger a purge.
%%
lager:info("Writing next entry to purge previous entry..."),
- populate_data(PBConn, Bucket, 1)
+ write_key(PBConn, Bucket, Key, Key)
after
%%
%% Revert the intercept, and drain, giving time for the
%% fuse to reset. Commit to Solr so that we can run a query.
%%
yz_rt:intercept_index_batch(Cluster, index_batch_call_orig),
- yz_rt:wait_until_fuses_reset(Cluster, yz_solrq_worker_0001, [Index]),
+ yz_rt:wait_until_fuses_reset(Cluster, 0, [Index]),
lager:info("Writing one last entry to set the threshold ok stat ..."),
- populate_data(PBConn, Bucket, 1),
+ write_key(PBConn, Bucket, Key, Key),
yz_rt:drain_solrqs(Cluster),
yz_rt:commit(Cluster, Index)
end.
@@ -267,16 +304,16 @@ check_index_stats(Node) ->
],
yz_rt:check_stat_values(Stats, Pairs).
-check_index_fail_stats(Node) ->
+check_index_extract_fail_stats(Node) ->
Stats = rpc:call(Node, yz_stat, get_stats, []),
- IFail = proplists:get_value(?STAT_NAME([index, fail]), Stats),
+ IFail = proplists:get_value(?STAT_NAME([index, extract, fail]), Stats),
IFailCount = proplists:get_value(count, IFail),
IFailOne = proplists:get_value(one, IFail),
Pairs = [
- {index_fail_count, IFailCount, '>', 0},
- {index_fail_one, IFailOne, '>', 0}
+ {index_extract_fail_count, IFailCount, '>', 0},
+ {index_extract_fail_one, IFailOne, '>', 0}
],
yz_rt:check_stat_values(Stats, Pairs).
@@ -370,3 +407,26 @@ check_fuse_and_purge_stats(Node) ->
{error_threshold_recovered_one, ErrorThresholdRecoveredOneValue, '>', 0}
],
yz_rt:check_stat_values(Stats, Pairs).
+
+%% @doc This test exists to test the ability to remove and recreate an index.
+%% Specifically, there were issues using the `fuse_stats_exometer' plugin
+%% as in its `init' function it always calls `exometer:new' rather than
+%% `re_register' which would have not crashed. We will be submitting a PR
+%% to fuse to get this resolved, but in the mean time there's a workaround
+%% in `yz_fuse' to remove the exometer statistics on fuse removal.
+confirm_recreate_indexed_bucket(Cluster, Bucket, Index, PBConn) ->
+ rt:setup_log_capture(Cluster),
+ lager:info("Removing index ~p", [Index]),
+ yz_rt:really_remove_index(Cluster, Bucket, Index, PBConn),
+ ?assert(rt:expect_in_log(hd(Cluster), "Delta: Removed: \\[<<\"yz_stat_test\">>\\] Added: \\[\\] Same: \\[\\]")),
+ lager:info("Recreating index ~p", [Index]),
+ ok = riakc_pb_socket:create_search_index(PBConn, Index, <<>>, [{n_val, ?N_VAL}]),
+ %% Stop the PB Connection so it doesn't crash our test
+ riakc_pb_socket:stop(PBConn),
+ yz_rt:wait_for_index(Cluster, Index),
+ Props = [{?YZ_INDEX, Index}],
+ lager:info("Adding index ~p back to bucket ~p", [Index, Bucket]),
+ rpc:call(hd(Cluster), riak_core_bucket, set_bucket, [Bucket, Props]),
+ %% Allow yz_events to tick before continuing to ensure index is removed
+ timer:sleep(?EVENT_TICK_INTERVAL + 100),
+ ?assert(rt:expect_not_in_logs(hd(Cluster), "gen_server fuse_server terminated with reason: exists")).
\ No newline at end of file
diff --git a/solr-patches/no-stale-check-4.7.0.patch b/solr-patches/no-stale-check-4.10.4.patch
similarity index 56%
rename from solr-patches/no-stale-check-4.7.0.patch
rename to solr-patches/no-stale-check-4.10.4.patch
index 70b01c66..f26c3e14 100644
--- a/solr-patches/no-stale-check-4.7.0.patch
+++ b/solr-patches/no-stale-check-4.10.4.patch
@@ -1,73 +1,59 @@
-From a2e77eaa262bd24c8bf3250d85fa0b3cab7d33e7 Mon Sep 17 00:00:00 2001
-From: Ryan Zezeski
-Date: Tue, 12 Feb 2013 11:54:21 -0500
-Subject: [PATCH] Disable stale check and nagle
+commit 13449f7dc2ac0d205180611f2de6e1371cd88a4b
+Author: Zeeshan Lakhani
+Date: Thu May 7 00:20:08 2015 +0900
-* Apache HTTP's stale conn check causes additionall latency and
- is not recommended for high-throughput/low-latency scenarios.
-
-* Disabling the stale check requires adding a periodic background
- task which clears idles connections. This prevents the client
- from pulling a conn closed by the server which causes an IOException.
-
-* Disable nagle as it's meant for protocols that use many small messages.
----
- .../solr/handler/component/HttpShardHandlerFactory.java | 3 +++
- .../org/apache/solr/client/solrj/impl/HttpClientUtil.java | 13 +++++++++++++
- 2 files changed, 16 insertions(+)
+ Update patches for Solr 4.10.4
diff --git a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
-index 3f91f17..e8b018e 100644
+index f574f29..42168df 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
-@@ -17,6 +17,7 @@ package org.apache.solr.handler.component;
- */
+@@ -29,6 +29,7 @@ import java.util.concurrent.ThreadPoolExecutor;
+ import java.util.concurrent.TimeUnit;
import org.apache.commons.lang.StringUtils;
+import org.apache.http.params.HttpConnectionParams;
import org.apache.http.client.HttpClient;
- import org.apache.solr.client.solrj.SolrServerException;
- import org.apache.solr.client.solrj.impl.HttpClientUtil;
-@@ -153,6 +154,8 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
- clientParams.set(HttpClientUtil.PROP_CONNECTION_TIMEOUT, connectionTimeout);
- clientParams.set(HttpClientUtil.PROP_USE_RETRY, false);
+ import org.apache.http.impl.client.DefaultHttpClient;
+ import org.apache.http.impl.client.DefaultHttpRequestRetryHandler;
+@@ -163,7 +164,8 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
+ clientParams.set(HttpClientUtil.PROP_USE_RETRY, false);
+ }
this.defaultClient = HttpClientUtil.createClient(clientParams);
+-
+ this.defaultClient.getParams().setParameter(HttpConnectionParams.STALE_CONNECTION_CHECK, false);
+ this.defaultClient.getParams().setParameter(HttpConnectionParams.TCP_NODELAY, true);
- this.loadbalancer = createLoadbalancer(defaultClient);
- }
-
+ // must come after createClient
+ if (useRetries) {
+ // our default retry handler will never retry on IOException if the request has been sent already,
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientUtil.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientUtil.java
-index b7c0680..8ca13f3 100644
+index fcbc80d..c98c982 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientUtil.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientUtil.java
-@@ -18,6 +18,9 @@ package org.apache.solr.client.solrj.impl;
+@@ -17,6 +17,9 @@
+ package org.apache.solr.client.solrj.impl;
import java.io.IOException;
- import java.io.InputStream;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+ import java.io.InputStream;
import java.util.zip.GZIPInputStream;
import java.util.zip.InflaterInputStream;
-
-@@ -110,6 +113,16 @@ public class HttpClientUtil {
+@@ -115,6 +118,16 @@ public class HttpClientUtil {
logger.debug("Creating new http client, config:" + config);
}
final DefaultHttpClient httpClient = new SystemDefaultHttpClient();
+ final org.apache.http.conn.ClientConnectionManager mgr = httpClient.getConnectionManager();
+
-+ // NOTE: The sweeper task is assuming hard-coded Jetty max-idle of 50s.
++ // Note: The sweeper task is assuming hard-coded Jetty max-idle of 50s.
+ final Runnable sweeper = new Runnable() {
-+ public void run() {
-+ mgr.closeIdleConnections(40, TimeUnit.SECONDS);
-+ }
-+ };
++ public void run() {
++ mgr.closeIdleConnections(40, TimeUnit.SECONDS);
++ }
++ };
+ final ScheduledExecutorService stp = Executors.newScheduledThreadPool(1);
+ stp.scheduleWithFixedDelay(sweeper, 5, 5, TimeUnit.SECONDS);
configureClient(httpClient, config);
return httpClient;
}
---
-1.9.0
-
diff --git a/src/yz_app.erl b/src/yz_app.erl
index a00f7d53..7f31afa3 100644
--- a/src/yz_app.erl
+++ b/src/yz_app.erl
@@ -60,6 +60,7 @@ start(_StartType, _StartArgs) ->
%% Now everything is started, permit usage by KV/query
enable_components(),
+ clique:register([yz_console]),
{ok, Pid};
Error ->
Error
@@ -126,8 +127,6 @@ maybe_setup(true) ->
yz_rs_migration:strip_rs_hooks(RSEnabled, Ring),
Routes = yz_wm_search:routes() ++ yz_wm_extract:routes() ++
yz_wm_index:routes() ++ yz_wm_schema:routes(),
- yz_misc:add_routes(Routes),
- maybe_register_pb(RSEnabled),
ok = yz_events:add_guarded_handler(yz_events, []),
yz_fuse:setup(),
setup_stats(),
@@ -141,6 +140,8 @@ maybe_setup(true) ->
ok = riak_core:register(search, [{permissions, ['query',admin]}]),
ok = yz_schema:setup_schema_bucket(),
ok = set_ibrowse_config(),
+ yz_misc:add_routes(Routes),
+ maybe_register_pb(RSEnabled),
ok.
%% @doc Conditionally register PB service IFF Riak Search is not
@@ -176,4 +177,4 @@ set_ibrowse_config() ->
?YZ_CONFIG_IBROWSE_MAX_PIPELINE_SIZE,
?YZ_CONFIG_IBROWSE_MAX_PIPELINE_SIZE_DEFAULT)}
],
- yz_solr:set_ibrowse_config(Config).
+ yz_solr:set_ibrowse_config(Config).
\ No newline at end of file
diff --git a/src/yz_console.erl b/src/yz_console.erl
index b96e4d0d..abd02713 100644
--- a/src/yz_console.erl
+++ b/src/yz_console.erl
@@ -1,6 +1,6 @@
%% -------------------------------------------------------------------
%%
-%% Copyright (c) 2014 Basho Technologies, Inc. All Rights Reserved.
+%% Copyright (c) 2014-2016 Basho Technologies, Inc. All Rights Reserved.
%%
%% This file is provided to you under the Apache License,
%% Version 2.0 (the "License"); you may not use this file
@@ -18,10 +18,27 @@
%%
%% -------------------------------------------------------------------
-module(yz_console).
+
+-behavior(clique_handler).
+
-include("yokozuna.hrl").
+
+%% New clique CLI code:
+-export([register_cli/0,
+ format_dist_query_value/1,
+ dist_query_cfg_change/2]).
+
+%% Old pre-clique CLI callbacks:
-export([aae_status/1,
switch_to_new_search/1]).
+-spec register_cli() -> ok.
+register_cli() ->
+ clique:register_config_whitelist(["search.dist_query"]),
+ clique:register_formatter(["search.dist_query"], fun format_dist_query_value/1),
+ clique:register_config(["search", "dist_query"], fun dist_query_cfg_change/2),
+ ok.
+
%% @doc Print the Active Anti-Entropy status to stdout.
-spec aae_status([]) -> ok.
aae_status([]) ->
@@ -51,3 +68,22 @@ switch_to_new_search([]) ->
io:format(standard_error, "The following nodes could not be reached: ~s", [DownStr]),
{error, {nodes_down, Down}}
end.
+
+
+%% @doc Callback for changes to dist_query enabled flag. When this flag is set to "on",
+%% then this node participates in distributed queries and will be included in
+%% cover plans when queries are made through yokozuna. When disabled, the node
+%% will be excluded in cover plans, meaning that it will not be consulted as part
+%% of a distributed query. Note that you can still query though this node;
+%% the node, however, will not be consulted in a Solr distrubuted query.
+dist_query_cfg_change(["search", "dist_query"], "on") ->
+ set_dist_query(true);
+dist_query_cfg_change(["search", "dist_query"], "off") ->
+ set_dist_query(false).
+
+set_dist_query(Val) ->
+ {ok, OldVal} = yz_solr_proc:set_dist_query(Val),
+ io_lib:format("Previous value: ~p", [format_dist_query_value(OldVal)]).
+
+format_dist_query_value(true) -> "on";
+format_dist_query_value(false) -> "off".
diff --git a/src/yz_doc.erl b/src/yz_doc.erl
index 6e7c80c9..8ee99398 100644
--- a/src/yz_doc.erl
+++ b/src/yz_doc.erl
@@ -68,6 +68,8 @@ doc_id(O, Partition, Sibling) ->
end.
%% @doc grab all siblings' vtags from Object contents
+sibling_vtags(Cs) when length(Cs) == 1 ->
+ [none];
sibling_vtags(Cs) -> [get_vtag(MD) || {MD, _V} <- Cs].
%% @doc count of Object contents that are siblings and not tombstones
@@ -152,18 +154,15 @@ extract_fields({MD, V}) ->
ExtractorDef = yz_extractor:get_def(CT, [check_default]),
case yz_extractor:run(V, ExtractorDef) of
{error, Reason} ->
- %% TODO is this an index failure, or should we track this with a different stat?
- yz_stat:index_fail(),
+ yz_stat:index_extract_fail(),
?ERROR("Failed to extract fields from value with reason ~p. Value: ~p", [Reason, V]),
[{?YZ_ERR_FIELD_S, 1}];
Fields ->
Fields
end
catch _:Err ->
- %% TODO is this an index failure, or should we track this with a different stat?
- yz_stat:index_fail(),
- Trace = erlang:get_stacktrace(),
- ?ERROR("An exception occurred extracting fields from value with reason ~p. Trace: ~p. Value: ~p", [Err, Trace, V]),
+ yz_stat:index_extract_fail(),
+ ?ERROR("An exception occurred extracting fields from value with reason ~p. Value: ~p", [Err, V]),
[{?YZ_ERR_FIELD_S, 1}]
end.
diff --git a/src/yz_dt_extractor.erl b/src/yz_dt_extractor.erl
index 19fbb911..8b5e3bc6 100644
--- a/src/yz_dt_extractor.erl
+++ b/src/yz_dt_extractor.erl
@@ -36,6 +36,10 @@
%% {<<"set">>, <<"Riak">>},
%% {<<"set">>, <<"Voldemort">>}]
%% '''
+%% HyperLogLog example:
+%% ```
+%% [{<<"hll">>, <<"10004">>}]
+%% '''
%% Map example (note the output of embedded types and conversion of
%% module to symbolic type):
%% ```
@@ -72,7 +76,7 @@
}).
-type state() :: #state{}.
-type field_path_name() :: undefined | {binary() | undefined, binary()}.
--type datatype() :: map | set | counter | register | flag.
+-type datatype() :: map | set | counter | register | flag | hll.
-spec extract(binary()) -> fields() | {error, any()}.
extract(Value) ->
extract(Value, ?NO_OPTIONS).
@@ -88,7 +92,8 @@ extract(Value0, Opts) ->
-spec extract_fields(#crdt{}, state()) -> fields().
extract_fields(#crdt{mod=Mod, value=Data}, S) ->
- #state{fields=F} = extract_fields(undefined, riak_kv_crdt:from_mod(Mod), Mod:value(Data), S),
+ #state{fields=F} = extract_fields(undefined, riak_kv_crdt:from_mod(Mod),
+ Mod:value(Data), S),
F.
-spec extract_fields(field_path_name(), datatype(), term(), state()) -> state().
@@ -106,9 +111,15 @@ extract_fields(Name, register, Value, #state{fields=Fields, field_separator=Sep}
State#state{fields=[{FieldName, Value}|Fields]};
extract_fields(Name, flag, Value, #state{fields=Fields, field_separator=Sep}=State) ->
FieldName = field_name(Name, flag, Sep),
- State#state{fields=[{FieldName, Value}|Fields]}.
+ State#state{fields=[{FieldName, Value}|Fields]};
+
+extract_fields(Name, hll=Type, Value, #state{fields=Fields,
+ field_separator=Sep}=State) ->
+ FieldName = field_name(Name, Type, Sep),
+ State#state{fields=[{FieldName, ?INT_TO_BIN(Value)}|Fields]}.
--spec field_name(field_path_name(), datatype(), binary()) -> undefined | binary().
+-spec field_name(field_path_name(), datatype(), binary()) -> undefined |
+ binary().
field_name(undefined, map, _Sep) ->
undefined;
field_name(undefined, Type, _Sep) ->
diff --git a/src/yz_entropy_mgr.erl b/src/yz_entropy_mgr.erl
index 4a2cea9c..a5fa8178 100644
--- a/src/yz_entropy_mgr.erl
+++ b/src/yz_entropy_mgr.erl
@@ -216,11 +216,13 @@ handle_cast({exchange_status, Pid, Index, {StartIdx, N}, Status}, S) ->
{noreply, S2};
handle_cast(clear_trees, S) ->
+ lager:info("Clearing YZ hashtrees and stopping all current exchanges."),
clear_all_exchanges(S#state.exchanges),
clear_all_trees(S#state.trees),
{noreply, S};
handle_cast(expire_trees, S) ->
+ lager:info("Expiring YZ hashtrees."),
ok = expire_all_trees(S#state.trees),
{noreply, S};
@@ -629,9 +631,9 @@ maybe_exchange(Ring, S) ->
S2;
{NextExchange, S2} ->
{Index, IndexN} = NextExchange,
- case already_exchanging(Index, S) of
+ case already_exchanging(Index, S2) of
true ->
- requeue_exchange(Index, IndexN, S2);
+ S2;
false ->
case start_exchange(Index, IndexN, Ring, S2) of
{ok, S3} -> S3;
diff --git a/src/yz_events.erl b/src/yz_events.erl
index adbcf100..bc240faa 100644
--- a/src/yz_events.erl
+++ b/src/yz_events.erl
@@ -59,6 +59,8 @@
-define(DEFAULT_EVENTS_FULL_CHECK_AFTER, 60).
-define(DEFAULT_EVENTS_TICK_INTERVAL, 1000).
+-define(UNKNOWN_QUEUE_LENGTH, -1).
+
%%%===================================================================
%%% API
@@ -260,6 +262,7 @@ sync_indexes() ->
lager:info("Delta: Removed: ~p Added: ~p Same: ~p",
[Removed, Added, Same])
end,
+ ok = yz_solrq_sup:sync_active_queue_pairs(),
ok = sync_indexes(Removed, Added, Same);
{error, _Reason} ->
ok
diff --git a/src/yz_exchange_fsm.erl b/src/yz_exchange_fsm.erl
index 3db8cec5..d8eb87df 100644
--- a/src/yz_exchange_fsm.erl
+++ b/src/yz_exchange_fsm.erl
@@ -34,6 +34,8 @@
built :: integer(),
timeout :: pos_integer()}).
+-type repair_count() :: {DeleteCount::non_neg_integer(), RepairCount::non_neg_integer()}.
+
%%%===================================================================
%%% API
%%%===================================================================
@@ -62,9 +64,9 @@ drain_error(Pid, Reason) ->
%% Use at your own risk (or ignore).
%% @end
%%
--spec update_yz_index_hashtree(pid(), tree(), p(), short_preflist()) -> ok.
-update_yz_index_hashtree(Pid, YZTree, Index, IndexN) ->
- do_update(Pid, yz_index_hashtree, YZTree, Index, IndexN).
+-spec update_yz_index_hashtree(pid(), tree(), p(), short_preflist(), fun() | undefined) -> ok.
+update_yz_index_hashtree(Pid, YZTree, Index, IndexN, Callback) ->
+ do_update(Pid, yz_index_hashtree, YZTree, Index, IndexN, Callback).
%%%===================================================================
%%% gen_fsm callbacks
@@ -91,9 +93,12 @@ handle_event(_Event, StateName, S) ->
handle_sync_event(_Event, _From, StateName, S) ->
{reply, ok, StateName, S}.
-handle_info({'DOWN', _, _, _, _}, _StateName, S) ->
+handle_info({'DOWN', _, _, _, _} = Msg, _StateName, S) ->
%% Either the entropy manager, local hashtree, or remote hashtree has
%% exited. Stop exchange.
+ lager:notice(
+ "YZ Exchange FSM received a DOWN message from a process it was "
+ "monitoring. The received message is: ~p", [Msg]),
{stop, normal, S};
handle_info(_Info, StateName, S) ->
@@ -186,24 +191,37 @@ key_exchange(timeout, S=#state{index=Index,
(_, _) ->
ok
end,
-
- AccFun = fun(KeyDiff, Count) ->
- lists:foldl(fun(Diff, InnerCount) ->
- case repair(Index, Diff) of
- full_repair -> InnerCount + 1;
- _ -> InnerCount
- end
- end, Count, KeyDiff)
- end,
- case yz_index_hashtree:compare(IndexN, Remote, AccFun, 0, YZTree) of
- 0 ->
+ AccFun = fun(KeyDiffs, Accum) ->
+ hashtree_compare_accum_fun(Index, KeyDiffs, Accum)
+ end,
+ async_do_compare(IndexN, Remote, AccFun, YZTree),
+ {next_state, key_exchange, S};
+
+key_exchange({compare_complete, CompareResult}, State=#state{index=Index,
+ index_n=IndexN}) ->
+ case CompareResult of
+ {error, Reason} ->
+ lager:error("An error occurred comparing hashtrees. Error: ~p", [Reason]);
+ {0, 0} ->
yz_kv:update_aae_exchange_stats(Index, IndexN, 0);
- Count ->
- yz_stat:detected_repairs(Count),
- lager:info("Will repair ~b keys of partition ~p for preflist ~p",
- [Count, Index, IndexN])
+ {YZDeleteCount, YZRepairCount} ->
+ yz_stat:detected_repairs(YZDeleteCount + YZRepairCount),
+ lager:info("Will delete ~p keys and repair ~b keys of partition ~p for preflist ~p",
+ [YZDeleteCount, YZRepairCount, Index, IndexN])
end,
- {stop, normal, S}.
+ {stop, normal, State}.
+
+async_do_compare(IndexN, Remote, AccFun, YZTree) ->
+ ExchangePid = self(),
+ spawn_link(
+ fun() ->
+ CompareResult = yz_index_hashtree:compare(IndexN, Remote, AccFun, {0, 0}, YZTree),
+ compare_complete(ExchangePid, CompareResult)
+ end).
+
+compare_complete(ExchangePid, CompareResult) ->
+ gen_fsm:send_event(ExchangePid, {compare_complete, CompareResult}).
+
%%%===================================================================
%%% Internal functions
@@ -217,6 +235,30 @@ exchange_bucket_kv(Tree, IndexN, Level, Bucket) ->
exchange_segment_kv(Tree, IndexN, Segment) ->
riak_kv_index_hashtree:exchange_segment(IndexN, Segment, Tree).
+-spec hashtree_compare_accum_fun(p(), [keydiff()], repair_count()) ->
+ repair_count().
+hashtree_compare_accum_fun(Index, KeyDiffs, Accum) ->
+ lists:foldl(
+ fun(KeyDiff, InnerAccum) ->
+ repair_fold_func(Index, KeyDiff, InnerAccum)
+ end,
+ Accum,
+ KeyDiffs
+ ).
+
+-spec repair_fold_func(p(), keydiff(), repair_count()) ->
+ repair_count().
+repair_fold_func(Index, KeyDiff, Accum) ->
+ RepairResult = repair(Index, KeyDiff),
+ update_repair_func_accum(RepairResult, KeyDiff, Accum).
+
+update_repair_func_accum(full_repair, _KeyDiff={remote_missing, _KeyBin}, {YZDeleteCount, YZRepairCount}) ->
+ {YZDeleteCount + 1, YZRepairCount};
+update_repair_func_accum(full_repair, _KeyDiff, {YZDeleteCount, YZRepairCount}) ->
+ {YZDeleteCount, YZRepairCount + 1};
+update_repair_func_accum(_RepairType, _KeyDiff, Accum) ->
+ Accum.
+
%% @private
%%
%% @doc If Yokozuna gets {remote_missing, _} b/c yz has it, but kv doesn't.
@@ -228,16 +270,11 @@ repair(Partition, {remote_missing, KeyBin}) ->
BKey = binary_to_term(KeyBin),
Index = yz_kv:get_index(BKey),
FakeObj = fake_kv_object(BKey),
- case yz_kv:should_index(Index) of
- true ->
- Repair = full_repair,
- yz_solrq:index(Index, BKey, FakeObj, {anti_entropy_delete, Repair}, Partition),
- Repair;
- false ->
- Repair = tree_repair,
- yz_solrq:index(Index, BKey, FakeObj, {anti_entropy_delete, Repair}, Partition),
- Repair
- end;
+ Repair = determine_repair_type(Index),
+ yz_solrq:index(Index, BKey, {FakeObj, no_old_object},
+ {anti_entropy_delete, Repair}, Partition),
+ Repair;
+
repair(Partition, {_Reason, KeyBin}) ->
%% Either Yokozuna is missing the key or the hash doesn't
%% match. In either case the object must be re-indexed.
@@ -247,18 +284,10 @@ repair(Partition, {_Reason, KeyBin}) ->
%% node is owner.
case yz_kv:local_get(Partition, BKey) of
{ok, Obj} ->
- case yz_kv:should_index(Index) of
- true ->
- Repair = full_repair,
- yz_solrq:index(Index, BKey, Obj, {anti_entropy, Repair},
- Partition),
- Repair;
- false ->
- Repair = tree_repair,
- yz_solrq:index(Index, BKey, Obj, {anti_entropy, Repair},
- Partition),
- Repair
- end;
+ Repair = determine_repair_type(Index),
+ yz_solrq:index(Index, BKey, {Obj, no_old_object},
+ {anti_entropy, Repair}, Partition),
+ Repair;
_Other ->
%% In most cases Other will be `{error, notfound}' which
%% is fine because hashtree updates are async and the
@@ -270,14 +299,18 @@ repair(Partition, {_Reason, KeyBin}) ->
failed_repair
end.
+determine_repair_type(Index) ->
+ case yz_kv:should_index(Index) of
+ true ->
+ full_repair;
+ false ->
+ tree_repair
+ end.
+
%% @private
fake_kv_object({Bucket, Key}) ->
riak_object:new(Bucket, Key, <<"fake object">>).
-%% @private
-do_update(ToWhom, Module, Tree, Index, IndexN) ->
- do_update(ToWhom, Module, Tree, Index, IndexN, undefined).
-
%% @private
do_update(ToWhom, Module, Tree, Index, IndexN, Callback) ->
UpdateResult = module_update(Module, IndexN, Tree, Callback),
@@ -291,8 +324,8 @@ do_update(ToWhom, Module, Tree, Index, IndexN, Callback) ->
%% @private
module_update(riak_kv_index_hashtree, Index, Tree, Callback) ->
riak_kv_index_hashtree:update(Index, Tree, Callback);
-module_update(yz_index_hashtree, Index, Tree, undefined) ->
- yz_index_hashtree:update(Index, Tree).
+module_update(yz_index_hashtree, Index, Tree, Callback) ->
+ yz_index_hashtree:update(Index, Tree, Callback).
%% @private
update_request(Module, Tree, Index, IndexN, Callback) ->
diff --git a/src/yz_fuse.erl b/src/yz_fuse.erl
index 412ad25e..4f08949a 100644
--- a/src/yz_fuse.erl
+++ b/src/yz_fuse.erl
@@ -93,6 +93,7 @@ remove(Index) ->
FuseName = fuse_name_for_index(Index),
fuse:remove(FuseName),
yz_stat:delete_dynamic_stats(Index, ?DYNAMIC_STATS),
+ remove_fuse_stats(FuseName),
ok.
-spec reset(index_name()) -> ok.
@@ -230,3 +231,14 @@ round_trip_through_fuse_name(IndexName) ->
FuseName = fuse_name_for_index(IndexName),
index_for_fuse_name(FuseName).
-endif.
+
+%% @doc Remove exometer stats for fuse `Name'.
+remove_fuse_stats(Name) ->
+ _ = exometer:delete(metric(Name, ok)),
+ _ = exometer:delete(metric(Name, blown)),
+ _ = exometer:delete(metric(Name, melt)),
+ ok.
+
+%% Internal.
+metric(Name, Counter) ->
+ [fuse, Name, Counter].
\ No newline at end of file
diff --git a/src/yz_index_hashtree.erl b/src/yz_index_hashtree.erl
index d955a6c4..0ffc766c 100644
--- a/src/yz_index_hashtree.erl
+++ b/src/yz_index_hashtree.erl
@@ -50,33 +50,70 @@ start(Index, RPs) ->
start_link(Index, RPs) ->
gen_server:start_link(?MODULE, [Index, RPs], []).
+%% @doc Insert the given `Key' and `Hash' pair on `Tree' for the given
+%% `Id'. The result of this call is not useful, as it may fail
+%% but is protected by a `try/catch'
+-spec insert({p(),n()}, bkey(), binary(), tree(), list()) ->
+ ok | term().
+insert(Id, BKey, Hash, Tree, Options) ->
+ SyncOrAsync = hashtree_call_mode(),
+ insert(SyncOrAsync, Id, BKey, Hash, Tree, Options).
+
%% @doc Insert the given `Key' and `Hash' pair on `Tree' for the given
%% `Id'. The result of a sync call should be ignored since it
%% uses `catch'.
+%% WARNING: In almost all cases, the caller should use `insert/5'
+%% rather than calling `insert/6' directly to prevent mailbox
+%% overload of the hashtree.
-spec insert(async | sync, {p(),n()}, bkey(), binary(), tree(), list()) ->
ok | term().
insert(async, Id, BKey, Hash, Tree, Options) ->
gen_server:cast(Tree, {insert, Id, BKey, Hash, Options});
insert(sync, Id, BKey, Hash, Tree, Options) ->
- catch gen_server:call(Tree, {insert, Id, BKey, Hash, Options}, infinity).
+ try
+ gen_server:call(Tree, {insert, Id, BKey, Hash, Options}, infinity)
+ catch
+ Error ->
+ lager:error("Synchronous insert into hashtree failed for reason ~p", [Error])
+ end.
%% @doc Delete the `BKey' from `Tree'. The id will be determined from
%% `BKey'. The result of the sync call should be ignored since
%% it uses catch.
+-spec delete({p(),n()}, bkey(), tree()) -> ok.
+delete(Id, BKey, Tree) ->
+ SyncOrAsync = hashtree_call_mode(),
+ delete(SyncOrAsync, Id, BKey, Tree).
+
+%% @doc Delete the `BKey' from `Tree'. The id will be determined from
+%% `BKey'. The result of the sync call should be ignored since
+%% it uses catch.
+%% WARNING: In almost all cases, the caller should use `delete/3'
+%% rather than calling `delete/4' directly to prevent mailbox
+%% overload of the hashtree.
-spec delete(async | sync, {p(),n()}, bkey(), tree()) -> ok.
delete(async, Id, BKey, Tree) ->
gen_server:cast(Tree, {delete, Id, BKey});
delete(sync, Id, BKey, Tree) ->
- catch gen_server:call(Tree, {delete, Id, BKey}, infinity).
+ try
+ gen_server:call(Tree, {delete, Id, BKey}, infinity)
+ catch
+ Error ->
+ lager:error("Synchronous delete from hashtree failed for reason ~p", [Error])
+ end.
--spec update({p(),n()}, tree()) -> ok.
+-spec update({p(), n()}, tree()) -> ok.
update(Id, Tree) ->
- gen_server:call(Tree, {update_tree, Id}, infinity).
+ gen_server:call(Tree, {update_tree, Id, undefined}, infinity).
+
+-spec update({p(),n()}, tree(), fun()) -> ok.
+update(Id, Tree, Callback) ->
+ gen_server:call(Tree, {update_tree, Id, Callback}, infinity).
-spec compare({p(),n()}, hashtree:remote_fun(),
- undefined | hashtree:acc_fun(T), term(), tree()) -> T.
+ undefined | hashtree:acc_fun(T), term(), tree()) -> T | {error, Reason::term()}.
compare(Id, Remote, AccFun, Acc, Tree) ->
gen_server:call(Tree, {compare, Id, Remote, AccFun, Acc}, infinity).
@@ -182,7 +219,7 @@ handle_call({get_lock, Type, Pid}, _From, S) ->
{Reply, S2} = do_get_lock(Type, Pid, S),
{reply, Reply, S2};
-handle_call({update_tree, Id}, From, S) ->
+handle_call({update_tree, Id, Callback}, From, S) ->
lager:debug("Updating tree for partition ~p preflist ~p",
[S#state.index, Id]),
apply_tree(Id,
@@ -192,6 +229,7 @@ handle_call({update_tree, Id}, From, S) ->
Self = self(),
spawn_link(
fun() ->
+ catch maybe_callback(Callback),
hashtree:update_perform(SnapTree),
gen_server:cast(Self, {updated, Id}),
gen_server:reply(From, ok)
@@ -201,8 +239,8 @@ handle_call({update_tree, Id}, From, S) ->
S);
handle_call({compare, Id, Remote, AccFun, Acc}, From, S) ->
- do_compare(Id, Remote, AccFun, Acc, From, S),
- {noreply, S};
+ S2 = do_compare(Id, Remote, AccFun, Acc, From, S),
+ {noreply, S2};
handle_call(destroy, _From, S) ->
S2 = destroy_trees(S),
@@ -255,11 +293,13 @@ handle_cast(expire, S) ->
handle_cast(_Msg, S) ->
{noreply, S}.
+handle_info({'EXIT', _SomeOtherProc, normal}, S) ->
+ {noreply, S};
handle_info({'DOWN', Ref, _, _, _}, S) ->
S2 = maybe_release_lock(Ref, S),
{noreply, S2};
-
-handle_info(_Info, S) ->
+handle_info(Message, S) ->
+ lager:error("Received unhandled message with message ~p", [Message]),
{noreply, S}.
terminate(_Reason, S) ->
@@ -322,7 +362,7 @@ fold_keys(Partition, Tree, Indexes) ->
%% TODO: return _yz_fp from iterator and use that for
%% more efficient get_index_N
IndexN = get_index_n(BKey),
- insert(async, IndexN, BKey, Hash, Tree, [if_missing])
+ insert(IndexN, BKey, Hash, Tree, [if_missing])
end,
Filter = [{partition, LogicalPartition}],
[yz_entropy:iterate_entropy_data(I, Filter, F) || I <- Indexes].
@@ -427,6 +467,40 @@ do_delete(Id, Key, S=#state{trees=Trees}) ->
handle_unexpected_key(Id, Key, S)
end.
+%% @private
+%%
+%% @doc Determine the method used to make the hashtree update. Most
+%% updates will be performed in async manner but want to occasionally
+%% use a blocking call to avoid overloading the hashtree.
+%%
+%% NOTE: This uses the process dictionary and thus is another function
+%% which relies running on a long-lived process. In this case that
+%% process is the KV vnode. In the future this should probably use
+%% cast only + sidejob for overload protection.
+-spec hashtree_call_mode() -> async | sync.
+hashtree_call_mode() ->
+ case get(yz_hashtree_tokens) of
+ undefined ->
+ put(yz_hashtree_tokens, max_hashtree_tokens() - 1),
+ async;
+ N when N > 0 ->
+ put(yz_hashtree_tokens, N - 1),
+ async;
+ _ ->
+ put(yz_hashtree_tokens, max_hashtree_tokens() - 1),
+ sync
+ end.
+
+%% @private
+%%
+%% @doc Return the max number of async hashtree calls that may be
+%% performed before requiring a blocking call.
+-spec max_hashtree_tokens() -> pos_integer().
+max_hashtree_tokens() ->
+ %% Use same max as riak_kv if no override provided
+ app_helper:get_env(yokozuna, anti_entropy_max_async,
+ app_helper:get_env(riak_kv, anti_entropy_max_async, 90)).
+
-spec handle_unexpected_key({p(),n()}, binary(), state()) -> state().
handle_unexpected_key(Id, Key, S=#state{index=Partition}) ->
RP = riak_kv_util:responsible_preflists(Partition),
@@ -484,17 +558,26 @@ do_compare(Id, Remote, AccFun, Acc, From, S) ->
%% This case shouldn't happen, but might as well safely handle it.
lager:warning("Tried to compare nonexistent tree "
"(vnode)=~p (preflist)=~p", [S#state.index, Id]),
- gen_server:reply(From, []);
+ gen_server:reply(From, []),
+ S;
{ok, Tree} ->
spawn_link(
- fun() ->
- Remote(init, self()),
- Result = hashtree:compare(Tree, Remote, AccFun, Acc),
- Remote(final, self()),
- gen_server:reply(From, Result)
- end)
+ fun() -> async_do_compare(Remote, Tree, AccFun, Acc, From) end
+ ),
+ S
+ end.
+
+async_do_compare(Remote, Tree, AccFun, Acc, From) ->
+ Result = try
+ Remote(init, self()),
+ CompareResult = hashtree:compare(Tree, Remote, AccFun, Acc),
+ Remote(final, self()),
+ CompareResult
+ catch
+ _:Reason ->
+ {error, Reason}
end,
- ok.
+ gen_server:reply(From, Result).
%% TODO: OMG cache this with entry in proc dict, use `_yz_fp' as Index
%% and keep an orddict(Bucket,N) in proc dict
@@ -529,7 +612,7 @@ clear_tree(S=#state{index=Index}) ->
S3#state{built=false, expired=false}.
destroy_trees(S) ->
- S2 = close_trees(S),
+ S2 = close_trees(S, true),
{_,Tree0} = hd(S#state.trees), % deliberately using state with live db ref
hashtree:destroy(Tree0),
S2.
@@ -547,7 +630,11 @@ maybe_build(S) ->
%%
%% @doc Flush and close the trees if not closed already.
-spec close_trees(#state{}) -> #state{}.
-close_trees(S=#state{trees=Trees, closed=false}) ->
+close_trees(State) ->
+ close_trees(State, false).
+
+-spec close_trees(#state{}, SkipUpdate::boolean()) -> #state{}.
+close_trees(S=#state{trees=Trees, closed=false}, _WillDestroy=false) ->
Trees2 = [begin
NewTree =
try hashtree:next_rebuild(Tree) of
@@ -568,11 +655,19 @@ close_trees(S=#state{trees=Trees, closed=false}) ->
end,
{IdxN, NewTree}
end || {IdxN, Tree} <- Trees],
- Trees3 = [{IdxN, hashtree:close(Tree)} || {IdxN, Tree} <- Trees2],
- S#state{trees=Trees3, closed=true};
-close_trees(S) ->
+ really_close_trees(Trees2, S);
+close_trees(#state{trees=Trees, closed=false}=State, _WillDestroy=true) ->
+ really_close_trees(Trees, State);
+close_trees(S, _) ->
S.
+really_close_trees(Trees2, S) ->
+ lists:foreach(fun really_close_tree/1, Trees2),
+ S#state{trees = undefined, closed = true}.
+
+really_close_tree({_IdxN, Tree}) ->
+ hashtree:close(Tree).
+
-spec build_or_rehash(pid(), state()) -> ok.
build_or_rehash(Tree, S) ->
Type = case load_built(S) of
@@ -676,3 +771,8 @@ handle_iter_keys(Tree, Index, IterKeys) ->
lager:debug("YZ AAE tree build failed: ~p", [Index]),
gen_server:cast(Tree, build_failed)
end.
+
+maybe_callback(undefined) ->
+ ok;
+maybe_callback(Callback) ->
+ Callback().
diff --git a/src/yz_kv.erl b/src/yz_kv.erl
index 1bb7f7ae..c8a77c6d 100644
--- a/src/yz_kv.erl
+++ b/src/yz_kv.erl
@@ -30,8 +30,6 @@
-include_lib("eunit/include/eunit.hrl").
-endif.
--type delops() :: []|[{id, _}]|[{siblings, _}].
-
%%%===================================================================
%%% TODO: move to riak_core
%%%===================================================================
@@ -141,8 +139,15 @@ get_md_entry(MD, Key) ->
-spec get_index(bkey()) -> index_name().
get_index({Bucket, _}) ->
BProps = riak_core_bucket:get_bucket(Bucket),
+ get_index_from_bucket_props(BProps).
+
+get_index_from_bucket_props(BProps) ->
proplists:get_value(?YZ_INDEX, BProps, ?YZ_INDEX_TOMBSTONE).
+is_search_enabled_for_bucket(BucketProps) ->
+ get_index_from_bucket_props(BucketProps) =/=
+ ?YZ_INDEX_TOMBSTONE.
+
%% @doc Called by KV vnode to determine if handoff should start or
%% not. Yokozuna needs to make sure that the bucket types have been
%% transfered first. Otherwise the bucket-to-index associations may
@@ -195,31 +200,38 @@ has_indexes(RemoteNode) ->
_ -> false
end.
-index(Obj, Reason, P) ->
- try
- case yokozuna:is_enabled(index) andalso ?YZ_ENABLED of
- true ->
- BKey = {riak_object:bucket(Obj), riak_object:key(Obj)},
- Index = yz_kv:get_index(BKey),
-
- yz_solrq:index(Index, BKey, Obj, Reason, P);
- false ->
- ok
- end
- catch
- _:Err ->
- yz_stat:index_fail(),
- Trace = erlang:get_stacktrace(),
- ?ERROR("Index failed - ~p\nat: ~p", [Err, Trace]),
- {error, Err}
+
+%% @doc Index the data supplied in the Riak Object.
+%% The Riak Object should be a serialized object (a binary,
+%% which has been serialized using riak_object:to_binary/1)
+-spec index_binary(bucket(), key(), binary(), write_reason(), p()) -> ok.
+index_binary(Bucket, Key, Bin, Reason, P) ->
+ case yokozuna:is_enabled(index) andalso ?YZ_ENABLED of
+ true ->
+ RObj = riak_object:from_binary(Bucket, Key, Bin),
+ index(
+ {RObj, no_old_object}, Reason, P
+ );
+ _ -> ok
+ end.
+
+%% @doc Index the data supplied in the Riak Object.
+-spec index(object_pair(), write_reason(), p()) -> ok.
+index({Obj, _OldObj}=Objects, Reason, P) ->
+ case yokozuna:is_enabled(index) andalso ?YZ_ENABLED of
+ true ->
+ BKey = {riak_object:bucket(Obj), riak_object:key(Obj)},
+ Index = yz_kv:get_index(BKey),
+
+ yz_solrq:index(Index, BKey, Objects, Reason, P);
+ false ->
+ ok
end.
%% @doc Should the content be indexed?
-spec should_index(index_name()) -> boolean().
-should_index(?YZ_INDEX_TOMBSTONE) ->
- false;
-should_index(_) ->
- true.
+should_index(Index) ->
+ ?YZ_SHOULD_INDEX(Index).
%% @doc Perform a local KV get for `BKey' stored under `Index'. This
%% avoids spawning a coordinator and performing quorum.
@@ -260,51 +272,17 @@ update_hashtree(Action, Partition, IdxN, BKey) ->
not_registered ->
ok;
Tree ->
- Method = get_method(),
case Action of
{insert, ObjHash} ->
- yz_index_hashtree:insert(Method, IdxN, BKey,
+ yz_index_hashtree:insert(IdxN, BKey,
ObjHash, Tree, []),
ok;
delete ->
- yz_index_hashtree:delete(Method, IdxN, BKey, Tree),
+ yz_index_hashtree:delete(IdxN, BKey, Tree),
ok
end
end.
-%% @private
-%%
-%% @doc Determine the method used to make the hashtree update. Most
-%% updates will be performed in async manner but want to occasionally
-%% use a blocking call to avoid overloading the hashtree.
-%%
-%% NOTE: This uses the process dictionary and thus is another function
-%% which relies running on a long-lived process. In this case that
-%% process is the KV vnode. In the future this should probably use
-%% cast only + sidejob for overload protection.
--spec get_method() -> async | sync.
-get_method() ->
- case get(yz_hashtree_tokens) of
- undefined ->
- put(yz_hashtree_tokens, max_hashtree_tokens() - 1),
- async;
- N when N > 0 ->
- put(yz_hashtree_tokens, N - 1),
- async;
- _ ->
- put(yz_hashtree_tokens, max_hashtree_tokens() - 1),
- sync
- end.
-
-%% @private
-%%
-%% @doc Return the max number of async hashtree calls that may be
-%% performed before requiring a blocking call.
--spec max_hashtree_tokens() -> pos_integer().
-max_hashtree_tokens() ->
- %% Use same max as riak_kv
- app_helper:get_env(riak_kv, anti_entropy_max_async, 90).
-
%% @doc Write a value
-spec put(any(), binary(), binary(), binary(), string()) -> ok.
put(Client, Bucket, Key, Value, ContentType) ->
@@ -324,50 +302,6 @@ put(Client, Bucket, Key, Value, ContentType) ->
check_flag(Flag) ->
true == erlang:get(Flag).
-%% @private
-%%
-%% @doc General cleanup for non-sibling-permitted documents,
-%% setting up a delete operation of the bkey if the field
-%% contains a tombstone.
-%% e.g. for allow_mult=false/lww=true/datatype/sc
--spec cleanup([doc()], bkey()) -> [{bkey, bkey()}].
-cleanup([], _BKey) ->
- [];
-cleanup([{doc, Fields}|T], BKey) ->
- case proplists:is_defined(tombstone, Fields) of
- true -> [{bkey, BKey}];
- false -> cleanup(T, BKey)
- end.
-
-%% @private
-%% @doc Cleanup for siblings-permitted objects.
-%% `Last-case' if single-document and has no tombstones is to cleanup
-%% possible siblings on reconcilation.
-%%
-%% If there's a tombstone, remove w/ bkey Query.
-%%
-%% For Docs > 1, we start w/ & remove the original doc_id (ODocID) in order
-%% to preserve vtag-based ids used for sibling writes, but just remove
-%% via BKey if there's at least 1 tombstone.
-%%
-%% Tombstone tuples come through as {tombstone, <<>>}.
--spec cleanup_for_sibs([doc()], bkey(), binary()|[{id, binary()}])
- -> [{id, binary()}|{siblings, bkey()}].
-cleanup_for_sibs(Docs, BKey, ODocID) when is_binary(ODocID) ->
- case length(Docs) > 1 of
- true -> cleanup_for_sibs(Docs, BKey, [{id, ODocID}]);
- false -> cleanup_for_sibs(Docs, BKey, [])
- end;
-cleanup_for_sibs([{doc, Fields}|T], BKey, Ops) ->
- case proplists:is_defined(tombstone, Fields) of
- true -> [{bkey, BKey}];
- false -> cleanup_for_sibs(T, BKey, Ops)
- end;
-cleanup_for_sibs([], BKey, []) ->
- [{siblings, BKey}];
-cleanup_for_sibs([], _BKey, Ops) ->
- Ops.
-
%% @private
%%
%% @doc Get first partition from a preflist.
@@ -493,21 +427,6 @@ siblings_permitted(Obj, BProps) when is_list(BProps) ->
end;
siblings_permitted(_, _) -> true.
-%% @private
-%%
-%% @doc Set yz_solr:index delete operation(s).
-%% If object relates to lww=true/allow_mult=false/datatype/sc
-%% do cleanup of tombstones only.
--spec delete_operation(riak_kv_bucket:props(), obj(), [doc()],
- bkey(), lp()) -> delops().
-delete_operation(BProps, Obj, Docs, BKey, LP) ->
- case siblings_permitted(Obj, BProps) of
- true -> cleanup_for_sibs(Docs,
- BKey,
- yz_doc:doc_id(Obj, ?INT_TO_BIN(LP)));
- _ -> cleanup(Docs, BKey)
- end.
-
%% @private
%%
%% @doc Merge siblings for objects that shouldn't have them.
diff --git a/src/yz_misc.erl b/src/yz_misc.erl
index 90cb426f..4fab9093 100644
--- a/src/yz_misc.erl
+++ b/src/yz_misc.erl
@@ -230,7 +230,7 @@ owned_and_next_partitions(Node, Ring) ->
%% @doc Filter out all entries for partitions that are not currently owned or
%% this node is a future owner of.
--spec filter_out_fallbacks(ordset(p), solr_entries()) -> [{bkey(), obj(),
+-spec filter_out_fallbacks(ordset(p), solr_entries()) -> [{bkey(), object_pair(),
write_reason(), p()}].
filter_out_fallbacks(OwnedAndNext, Entries) ->
lists:filter(fun({_Bkey, _Obj, _Reason, P}) ->
diff --git a/src/yz_pb_admin.erl b/src/yz_pb_admin.erl
index b3df4a67..529d75de 100644
--- a/src/yz_pb_admin.erl
+++ b/src/yz_pb_admin.erl
@@ -109,9 +109,10 @@ process(#rpbyokozunaindexputreq{
index = #rpbyokozunaindex{
name = IndexName,
schema = SchemaName,
- n_val = Nval}}, State) ->
- Timeout = app_helper:get_env(?YZ_APP_NAME, index_put_timeout_ms,
- ?DEFAULT_IDX_CREATE_TIMEOUT),
+ n_val = Nval}
+ }, State) ->
+ Timeout = app_helper:get_env(yokozuna, index_put_timeout_ms,
+ ?DEFAULT_IDX_CREATE_TIMEOUT),
case maybe_create_index(IndexName, SchemaName, Nval, Timeout) of
ok ->
diff --git a/src/yz_pb_search.erl b/src/yz_pb_search.erl
index 3e2b5c81..ce6fd98f 100644
--- a/src/yz_pb_search.erl
+++ b/src/yz_pb_search.erl
@@ -1,8 +1,6 @@
%% -------------------------------------------------------------------
%%
-%% yz_pb_search: PB Service for Yokozuna queries
-%%
-%% Copyright (c) 2013 Basho Technologies, Inc. All Rights Reserved.
+%% Copyright (c) 2013-2016 Basho Technologies, Inc.
%%
%% This file is provided to you under the Apache License,
%% Version 2.0 (the "License"); you may not use this file
diff --git a/src/yz_solr.erl b/src/yz_solr.erl
index b3ba0c2f..b600ffb9 100644
--- a/src/yz_solr.erl
+++ b/src/yz_solr.erl
@@ -22,7 +22,30 @@
%% All interaction with Solr should go through this API.
-module(yz_solr).
--compile(export_all).
+-export([build_mapping/1,
+ commit/1,
+ core/2,
+ core/3,
+ cores/0,
+ delete/2,
+ dist_search/2,
+ dist_search/3,
+ encode_delete/1,
+ encode_doc/1,
+ entropy_data/2,
+ get_doc_pairs/1,
+ get_ibrowse_config/0,
+ get_response/1,
+ index_batch/2,
+ is_up/0,
+ jmx_port/0,
+ mbeans_and_stats/1,
+ partition_list/1,
+ ping/1,
+ port/0,
+ prepare_json/1,
+ set_ibrowse_config/1,
+ search/3]).
-include_lib("riak_core/include/riak_core_bucket_type.hrl").
-include("yokozuna.hrl").
@@ -39,7 +62,7 @@
-type delete_op() :: {id, binary()}
| {bkey, bkey()}
- | {siblings, bkey()}
+ | {bkey, bkey(), lp()}
| {'query', binary()}.
-type ibrowse_config_key() :: max_sessions | max_pipeline_size.
@@ -49,6 +72,15 @@
-export_type([delete_op/0]).
-export_type([ibrowse_config_key/0, ibrowse_config_value/0, ibrowse_config/0]).
+-ifdef(EQC).
+%% -define(EQC_DEBUG(S, F), eqc:format(S, F)).
+-define(EQC_DEBUG(S, F), ok).
+-else.
+-define(EQC_DEBUG(S, F), ok).
+-endif.
+
+
+
%%%===================================================================
%%% API
%%%===================================================================
@@ -63,12 +95,6 @@ build_mapping(Nodes) ->
|| Node <- Nodes],
P /= unknown].
--spec build_partition_delete_query(ordset(lp())) -> term().
-build_partition_delete_query(LPartitions) ->
- Deletes = [{delete, ?QUERY(<>)}
- || LP <- LPartitions],
- mochijson2:encode({struct, Deletes}).
-
-spec commit(index_name()) -> ok.
commit(Core) ->
JSON = encode_commit(),
@@ -190,42 +216,39 @@ entropy_data(Core, Filter) ->
{error, X}
end.
-%% @doc Index the given `Docs'.
-index(Core, Docs) ->
- index(Core, Docs, []).
-
--spec index(index_name(), list(), [delete_op()]) -> ok.
-index(Core, Docs, DelOps) ->
- Ops = {struct,
- [{delete, encode_delete(Op)} || Op <- DelOps] ++
- [{add, encode_doc(D)} || D <- Docs]},
- JSON = mochijson2:encode(Ops),
+index_batch(Core, Ops) ->
+ ?EQC_DEBUG("index_batch: About to send entries. ~p~n", [Ops]),
+ JSON = encode_json(Ops),
+ ?EQC_DEBUG("index_batch: About to send JSON. ~p~n", [JSON]),
+ maybe_make_http_request(Core, JSON).
+
+maybe_make_http_request(_Core, {error, _} = Error) ->
+ ?EQC_DEBUG("Not making HTTP request due to error: ~p", [Error]),
+ Error;
+maybe_make_http_request(Core, JSON) ->
URL = ?FMT("~s/~s/update", [base_url(), Core]),
Headers = [{content_type, "application/json"}],
Opts = [{response_format, binary}],
-
- case ibrowse:send_req(URL, Headers, post, JSON, Opts,
+ ?EQC_DEBUG("About to send_req: ~p", [[URL, Headers, post, JSON, opts, ?YZ_SOLR_REQUEST_TIMEOUT]]),
+ Res = case ibrowse:send_req(URL, Headers, post, JSON, Opts,
?YZ_SOLR_REQUEST_TIMEOUT) of
{ok, "200", _, _} -> ok;
- Err -> throw({"Failed to index docs", Err})
- end.
+ {ok, "400", _, ErrBody} ->
+ {error, {badrequest, ErrBody}};
+ Err ->
+ {error, {other, Err}}
+ end,
+ %% ?PULSE_DEBUG("send_req result: ~p", [Res]),
+ Res.
-index_batch(Core, Ops) ->
+encode_json(Ops) ->
JSON = try
mochijson2:encode({struct, Ops})
catch _:E ->
- throw({"Failed to encode ops", bad_data, E})
+ {error, {bad_data, E}}
end,
- URL = ?FMT("~s/~s/update", [base_url(), Core]),
- Headers = [{content_type, "application/json"}],
- Opts = [{response_format, binary}],
- case ibrowse:send_req(URL, Headers, post, JSON, Opts,
- ?YZ_SOLR_REQUEST_TIMEOUT) of
- {ok, "200", _, _} -> ok;
- {ok, "400", _, ErrBody} -> throw({"Failed to index docs", badrequest,
- ErrBody});
- Err -> throw({"Failed to index docs", other, Err})
- end.
+ ?EQC_DEBUG("Encoded JSON: ~p", [JSON]),
+ JSON.
%% @doc Determine if Solr is running.
-spec is_up() -> boolean().
@@ -271,7 +294,7 @@ partition_list(Core) ->
-spec ping(index_name()) -> boolean()|error.
ping(Core) ->
URL = ?FMT("~s/~s/admin/ping", [base_url(), Core]),
- Response = ibrowse:send_req(URL, [], get),
+ Response = ibrowse:send_req(URL, [], head),
Result = case Response of
{ok, "200", _, _} -> true;
{ok, "404", _, _} -> false;
@@ -414,24 +437,28 @@ encode_commit() ->
%%
%% @doc Encode a delete operation into a mochijson2 compatiable term.
-spec encode_delete(delete_op()) -> {struct, [{atom(), binary()}]}.
+encode_delete({bkey, {{Type, Bucket},Key}, LP}) ->
+ PNQ = encode_field_query(?YZ_PN_FIELD_B, escape_special_chars(?INT_TO_BIN(LP))),
+ TypeQ = encode_field_query(?YZ_RT_FIELD_B, escape_special_chars(Type)),
+ BucketQ = encode_field_query(?YZ_RB_FIELD_B, escape_special_chars(Bucket)),
+ KeyQ = encode_field_query(?YZ_RK_FIELD_B, escape_special_chars(Key)),
+ ?QUERY(<>);
+%% NOTE: Used for testing only. This deletes _all_ documents for _all_ partitions
+%% which may cause extra docs to be deleted if a fallback partition exists
+%% on the same node as a primary, or if handoff is still ongoing in a newly-built
+%% cluster
encode_delete({bkey,{{Type, Bucket},Key}}) ->
- TypeQ = encode_nested_query(?YZ_RT_FIELD_B, escape_special_chars(Type)),
- BucketQ = encode_nested_query(?YZ_RB_FIELD_B, escape_special_chars(Bucket)),
- KeyQ = encode_nested_query(?YZ_RK_FIELD_B, escape_special_chars(Key)),
+ TypeQ = encode_field_query(?YZ_RT_FIELD_B, escape_special_chars(Type)),
+ BucketQ = encode_field_query(?YZ_RB_FIELD_B, escape_special_chars(Bucket)),
+ KeyQ = encode_field_query(?YZ_RK_FIELD_B, escape_special_chars(Key)),
?QUERY(<>);
+%% NOTE: Also only used for testing
encode_delete({bkey,{Bucket,Key}}) ->
%% Need to take legacy (pre 2.0.0) objects into account.
- encode_delete({bkey,{{?DEFAULT_TYPE,Bucket},Key}});
-encode_delete({siblings,{{Type,Bucket},Key}}) ->
- VTagQ = <>,
- TypeQ = encode_nested_query(?YZ_RT_FIELD_B, escape_special_chars(Type)),
- BucketQ = encode_nested_query(?YZ_RB_FIELD_B, escape_special_chars(Bucket)),
- KeyQ = encode_nested_query(?YZ_RK_FIELD_B, escape_special_chars(Key)),
- ?QUERY(<>);
-encode_delete({siblings,{Bucket,Key}}) ->
+ encode_delete({bkey,{{?DEFAULT_TYPE, Bucket}, Key}});
+encode_delete({bkey, {Bucket,Key}, LP}) ->
%% Need to take legacy (pre 2.0.0) objects into account.
- encode_delete({siblings,{{?DEFAULT_TYPE,Bucket},Key}});
+ encode_delete({bkey, {{?DEFAULT_TYPE, Bucket}, Key}, LP});
encode_delete({'query', Query}) ->
?QUERY(Query);
encode_delete({id, Id}) ->
@@ -450,18 +477,17 @@ encode_field({Name,Value}) ->
%% @private
%%
-%% @doc Encode a field and query into a Solr nested query using the
-%% term query parser.
--spec encode_nested_query(binary(), binary()) -> binary().
-encode_nested_query(Field, Query) ->
- <<"_query_:\"{!term f=",Field/binary,"}",Query/binary,"\"">>.
+%% @doc Encode a field and query.
+-spec encode_field_query(binary(), binary()) -> binary().
+encode_field_query(Field, Query) ->
+ <>.
%% @private
%%
%% @doc Escape the backslash and double quote chars to prevent from
%% being improperly interpreted by Solr's query parser.
-spec escape_special_chars(binary()) ->binary().
-escape_special_chars(Bin) ->
+escape_special_chars(Bin) when is_binary(Bin)->
Bin2 = binary:replace(Bin, <<"\\">>, <<"\\\\">>, [global]),
binary:replace(Bin2, <<"\"">>, <<"\\\"">>, [global]).
diff --git a/src/yz_solr_proc.erl b/src/yz_solr_proc.erl
index d9d3da90..a0616a8b 100644
--- a/src/yz_solr_proc.erl
+++ b/src/yz_solr_proc.erl
@@ -33,14 +33,17 @@
handle_cast/2,
handle_info/2,
terminate/2,
- getpid/0]).
+ getpid/0,
+ get_dist_query/0,
+ set_dist_query/1]).
-record(state, {
dir=exit(dir_undefined),
port=exit(port_undefined),
solr_port=exit(solr_port_undefined),
solr_jmx_port=exit(solr_jmx_port_undefined),
- is_up=false
+ is_up=false,
+ enable_dist_query=exit(enable_dist_query_undefined)
}).
-define(SHUTDOWN_MSG, "INT\n").
@@ -51,6 +54,12 @@
-define(S_PORT(S), S#state.port).
-define(YZ_DEFAULT_SOLR_JVM_OPTS, "").
+-define(SOLRCONFIG_2_0_HASH, 64816669).
+-define(LUCENE_MATCH_4_7_VERSION, "4.7").
+-define(LUCENE_MATCH_4_10_4_VERSION, "4.10.4").
+
+-type path() :: string().
+
%%%===================================================================
%%% API
%%%===================================================================
@@ -65,9 +74,15 @@ start_link(Dir, TempDir, SolrPort, SolrJMXPort) ->
getpid() ->
gen_server:call(?MODULE, getpid).
--spec is_up() -> boolean().
-is_up() ->
- gen_server:call(?MODULE, is_up).
+-spec get_dist_query() -> boolean().
+get_dist_query() ->
+ gen_server:call(?MODULE, enabled).
+
+-spec set_dist_query(boolean()) -> {ok, PreviousValue :: boolean()} | {error, Reason :: term()}.
+set_dist_query(Enabled) when is_boolean(Enabled) ->
+ {ok, gen_server:call(?MODULE, {enabled, Enabled})};
+set_dist_query(Value) ->
+ {error, {bad_type, Value}}.
%%%===================================================================
%%% Callbacks
@@ -89,6 +104,7 @@ is_up() ->
init([Dir, TempDir, SolrPort, SolrJMXPort]) ->
ensure_data_dir(Dir),
ensure_temp_dir(TempDir),
+ check_solr_index_versions(Dir),
case get_java_path() of
undefined ->
%% This logging call is needed because the stop reason
@@ -107,13 +123,26 @@ init([Dir, TempDir, SolrPort, SolrJMXPort]) ->
dir=Dir,
port=Port,
solr_port=SolrPort,
- solr_jmx_port=SolrJMXPort
+ solr_jmx_port=SolrJMXPort,
+ enable_dist_query=?YZ_ENABLE_DIST_QUERY
},
{ok, S}
end.
handle_call(getpid, _, S) ->
{reply, get_pid(?S_PORT(S)), S};
+handle_call(enabled, _, #state{enable_dist_query=EnableDistQuery} = S) ->
+ {reply, EnableDistQuery, S};
+handle_call({enabled, NewValue}, _, #state{enable_dist_query=OldValue} = S) ->
+ case {yz_solr:is_up(), OldValue, NewValue} of
+ {true, false, true} ->
+ riak_core_node_watcher:service_up(yokozuna, self());
+ {_, true, false} ->
+ riak_core_node_watcher:service_down(yokozuna);
+ _ ->
+ ok
+ end,
+ {reply, OldValue, S#state{enable_dist_query=NewValue}};
handle_call(Req, _, S) ->
?WARN("unexpected request ~p", [Req]),
{noreply, S}.
@@ -123,29 +152,28 @@ handle_cast(Req, S) ->
{noreply, S}.
handle_info({check_solr, WaitTimeSecs}, S=?S_MATCH) ->
- case yz_solr:is_up() of
- true ->
- %% solr finished its startup, be merry
- ?INFO("solr is up", []),
+ case {yz_solr:is_up(), ?YZ_ENABLE_DIST_QUERY} of
+ {true, true} ->
riak_core_node_watcher:service_up(yokozuna, self()),
- schedule_tick(),
- {noreply, S#state{is_up=true}};
- false when WaitTimeSecs > 0 ->
+ solr_is_up(S);
+ {true, false} ->
+ solr_is_up(S);
+ {false, _} when WaitTimeSecs > 0 ->
%% solr hasn't finished yet, keep waiting
schedule_solr_check(WaitTimeSecs),
{noreply, S};
- false ->
+ {false, _} ->
%% solr did not finish startup quickly enough, or has just
%% crashed and the exit message is on its way, shutdown
{stop, "solr didn't start in alloted time", S}
end;
-handle_info(tick, #state{is_up=IsUp} = S) ->
+handle_info(tick, #state{is_up=IsUp, enable_dist_query=EnableDistQuery} = S) ->
NewIsUp = yz_solr:is_up(),
- case {IsUp, NewIsUp} of
- {false, true} ->
+ case {IsUp, NewIsUp, EnableDistQuery} of
+ {false, true, true} ->
?INFO("Solr was down but is now up. Notifying cluster.", []),
riak_core_node_watcher:service_up(yokozuna, self());
- {true, false} ->
+ {true, false, _} ->
?INFO("Solr was up but is now down. Notifying cluster.", []),
riak_core_node_watcher:service_down(yokozuna);
_ -> ok
@@ -163,7 +191,13 @@ handle_info({'EXIT', _Port, Reason}, S=?S_MATCH) ->
{stop, normal, S};
_ ->
{stop, {port_exit, Reason}, S}
- end.
+ end;
+
+%% ibrowse does not protect from late replies - handle them here
+handle_info({Ref, _Msg} = Message, State)
+ when is_reference(Ref) ->
+ lager:info("Received late reply: ~p", [Message]),
+ {noreply, State}.
code_change(_, S, _) ->
{ok, S}.
@@ -246,13 +280,35 @@ ensure_data_dir(Dir) ->
ok
end.
+%% @private
+%%
+%% @doc Recursively delete target directory. Equivalent to
+%% `rm -rf $DIR` at the shell
+-spec del_dir_recursive(string()) -> ok | {error, enotdir}.
+del_dir_recursive(Path) ->
+ del_dir_recursive(Path, filelib:is_dir(Path)).
+
+-spec del_dir_recursive(string(), boolean()) -> ok | {error, enotdir}.
+del_dir_recursive(DirPath, true) ->
+ FullPaths = [filename:join(DirPath, FilePath) ||
+ FilePath <- filelib:wildcard("**", DirPath)],
+ {Dirs, Files} = lists:partition(fun filelib:is_dir/1, FullPaths),
+ lists:foreach(fun file:delete/1, Files),
+ %% Delete directories sorted longest to shortest to ensure we delete leaf
+ %% directories first.
+ lists:foreach(fun file:del_dir/1, lists:sort(fun (A,B) -> A > B end, Dirs)),
+ file:del_dir(DirPath);
+del_dir_recursive(_DirPath, _Exists = false) ->
+ {error, enotdir}.
+
%% @private
%%
%% @doc Make sure that the temp directory (passed in as `TempDir')
-%% exists
+%% exists and is new
-spec ensure_temp_dir(string()) -> ok.
ensure_temp_dir(TempDir) ->
- filelib:ensure_dir(filename:join(TempDir, empty)).
+ del_dir_recursive(filename:join(TempDir, "solr-webapp")),
+ ok = filelib:ensure_dir(filename:join(TempDir, empty)).
%% @private
%%
@@ -309,3 +365,93 @@ solr_jvm_opts() ->
app_helper:get_env(?YZ_APP_NAME,
solr_jvm_opts,
?YZ_DEFAULT_SOLR_JVM_OPTS).
+
+-spec solr_is_up(#state{}) -> {noreply, #state{}}.
+solr_is_up(S) ->
+ %% solr finished its startup, be merry
+ ?INFO("solr is up", []),
+ schedule_tick(),
+ {noreply, S#state{is_up=true}}.
+
+-spec check_solr_index_versions(path()) -> ok.
+check_solr_index_versions(YZRootDir) ->
+ DefaultSolrConfigPath = filename:join([?YZ_PRIV, "conf", "solrconfig.xml"]),
+ DefaultSolrConfigHash = hash_file_contents(DefaultSolrConfigPath),
+ SolrConfigIndexPaths = get_index_solrconfig_paths(YZRootDir),
+ [check_index_solrconfig(SolrConfigIndexPath, DefaultSolrConfigPath, DefaultSolrConfigHash) ||
+ SolrConfigIndexPath <- SolrConfigIndexPaths],
+ ok.
+
+-spec check_index_solrconfig(path(), path(), non_neg_integer()) -> ok.
+check_index_solrconfig(SolrConfigIndexPath, DefaultSolrConfigPath, DefaultSolrConfigHash) ->
+ case hash_file_contents(SolrConfigIndexPath) of
+ DefaultSolrConfigHash ->
+ ok;
+ ?SOLRCONFIG_2_0_HASH ->
+ yz_misc:copy_files([DefaultSolrConfigPath], filename:dirname(SolrConfigIndexPath)),
+ lager:info(
+ "Upgraded ~s to the latest version.", [SolrConfigIndexPath]
+ );
+ _ ->
+ check_index_solrconfig_version(SolrConfigIndexPath)
+ end.
+
+-spec check_index_solrconfig_version(path()) -> ok.
+check_index_solrconfig_version(SolrConfigIndexPath) ->
+ case get_lucene_match_version(SolrConfigIndexPath) of
+ ?LUCENE_MATCH_4_7_VERSION ->
+ lager:warning(
+ "The Solr configuration file ~s has been modified by the user and contains"
+ " an outdated version (~s) under the luceneMatchVersion XML tag."
+ " Please consider reverting your changes or upgrading the luceneMatchVersion"
+ " XML tag to ~s and restarting. Note: In order to take full advantage of"
+ " the changes, you should also reindex any data in this Solr core.",
+ [SolrConfigIndexPath, ?LUCENE_MATCH_4_7_VERSION, ?LUCENE_MATCH_4_10_4_VERSION]);
+ ?LUCENE_MATCH_4_10_4_VERSION ->
+ ok;
+ {error, no_lucene_match_version} ->
+ lager:error(
+ "The Solr configuration file ~s does not contain a luceneMatchVersion"
+ " XML tag!",
+ [SolrConfigIndexPath]);
+ UnexpectedVersion ->
+ lager:error(
+ "The Solr configuration file ~s contains a luceneMatchVersion"
+ " XML tag with an unexpected value: ~s",
+ [SolrConfigIndexPath, UnexpectedVersion])
+ end.
+
+-spec get_index_solrconfig_paths(path()) -> [path()].
+get_index_solrconfig_paths(YZRootDir) ->
+ {ok, Files} = file:list_dir(YZRootDir),
+ [YZRootDir ++ "/" ++ File ++ "/conf/solrconfig.xml" ||
+ File <- Files,
+ filelib:is_dir(YZRootDir ++ "/" ++ File)
+ andalso has_solr_config(YZRootDir ++ "/" ++ File)].
+
+-spec has_solr_config(path()) -> boolean().
+has_solr_config(RootPath) ->
+ SolrConfigPath = RootPath ++ "/conf/solrconfig.xml",
+ filelib:is_file(SolrConfigPath) andalso not filelib:is_dir(SolrConfigPath).
+
+-spec get_lucene_match_version(path()) -> string() | {error, no_lucene_match_version}.
+get_lucene_match_version(SolrConfigIndexPath) ->
+ {#xmlElement{content=Contents}, _Rest} = xmerl_scan:file(SolrConfigIndexPath),
+ lucene_match_version(Contents).
+
+-spec lucene_match_version([xmlElement()]) -> string() | {error, no_lucene_match_version}.
+lucene_match_version([]) ->
+ {error, no_lucene_match_version};
+lucene_match_version(
+ [#xmlElement{
+ name=luceneMatchVersion,
+ content=[#xmlText{value=Version}]
+ } | _Rest]) ->
+ Version;
+lucene_match_version([_Element | Rest]) ->
+ lucene_match_version(Rest).
+
+-spec hash_file_contents(path()) -> non_neg_integer().
+hash_file_contents(Path) ->
+ {ok, Data} = file:read_file(Path),
+ erlang:phash2(Data).
diff --git a/src/yz_solrq.erl b/src/yz_solrq.erl
index ad97437b..3818f44c 100644
--- a/src/yz_solrq.erl
+++ b/src/yz_solrq.erl
@@ -18,10 +18,9 @@
%% -------------------------------------------------------------------
-module(yz_solrq).
--export([index/5, worker_regname/1, helper_regname/1,
- random_helper/0,
- num_worker_specs/0,
- num_helper_specs/0,
+-export([index/5,
+ worker_regname/2,
+ helper_regname/2,
set_hwm/1,
set_index/4,
set_purge_strategy/1,
@@ -29,19 +28,19 @@
blown_fuse/1,
healed_fuse/1,
solrq_worker_names/0,
- solrq_helper_names/0,
- queue_total_length/0]).
+ queue_total_length/0,
+ get_max_batch_size/0,
+ get_min_batch_size/0,
+ get_flush_interval/0,
+ all_solrq_workers/0,
+ solrq_workers_for_partition/1,
+ solrq_worker_pairs_for_index/1]).
-include("yokozuna.hrl").
-% exported for yz_solrq_sup
--export([set_solrq_worker_tuple/1, set_solrq_helper_tuple/1,
- get_solrq_worker_tuple/0, get_solrq_helper_tuple/0]).
-
% for debugging only
-export([status/0]).
--type phash() :: integer().
-type regname() :: atom().
-type size_resps() :: same_size | {shrank, non_neg_integer()} |
{grew, non_neg_integer()}.
@@ -55,97 +54,68 @@
%%% API functions
%%%===================================================================
--spec index(index_name(), bkey(), obj(), write_reason(), p()) -> ok.
-index(Index, BKey, Obj, Reason, P) ->
- Hash = erlang:phash2({Index, BKey}),
- Worker = yz_solrq:worker_regname(Hash),
- yz_solrq_worker:index(Worker, Index, BKey, Obj, Reason, P).
+-spec index(index_name(), bkey(), object_pair(), write_reason(), p()) -> ok.
+index(Index, BKey, ObjectPair, Reason, P) ->
+ WorkerName = yz_solrq:worker_regname(Index, P),
+ ok = ensure_worker(Index, P),
+ yz_solrq_worker:index(WorkerName, BKey, ObjectPair, Reason, P).
%% @doc From the hash, return the registered name of a queue
--spec worker_regname(phash()) -> regname().
-worker_regname(Hash) ->
- case get_solrq_worker_tuple() of
- undefined ->
- error(solrq_sup_not_started);
- Names ->
- Index = 1 + (Hash rem size(Names)),
- element(Index, Names)
- end.
+-spec worker_regname(index_name(), p()) -> regname().
+worker_regname(Index, Partition) ->
+ make_regname("yz_solrq_worker_", Partition, Index).
-%% @doc From the hash, return the registered name of a helper
--spec helper_regname(phash()) -> regname().
-helper_regname(Hash) ->
- case get_solrq_helper_tuple() of
- undefined ->
- error(solrq_sup_not_started);
- Names ->
- Index = 1 + (Hash rem size(Names)),
- element(Index, Names)
- end.
-
-%% @doc return a random helper
--spec random_helper() -> regname().
-random_helper() ->
- case get_solrq_helper_tuple() of
- undefined ->
- error(solrq_sup_not_started);
- Names ->
- Index = random:uniform(size(Names)),
- element(Index, Names)
- end.
-
-%% @doc Active queue count
--spec num_worker_specs() -> non_neg_integer().
-num_worker_specs() ->
- yz_solrq_sup:child_count(yz_solrq_worker).
-
-%% @doc Active helper count
--spec num_helper_specs() -> non_neg_integer().
-num_helper_specs() ->
- yz_solrq_sup:child_count(yz_solrq_helper).
+make_regname(Prefix, Partition, Index) ->
+ list_to_atom(Prefix ++
+ integer_to_list(Partition) ++ "_" ++
+ binary_to_list(Index)).
+%% @doc From the hash, return the registered name of a helper
+-spec helper_regname(index_name(), p()) -> regname().
+helper_regname(Index, Partition) ->
+ make_regname("yz_solrq_helper_", Partition, Index).
%% @doc Set the high water mark on all queues
--spec set_hwm(solrq_hwm()) -> [{atom(),
+-spec set_hwm(solrq_hwm()) -> [{{index_name(), p()},
{ok | error,
solrq_hwm() | bad_hwm_value}}].
set_hwm(HWM) ->
- [{Name, yz_solrq_worker:set_hwm(Name, HWM)} ||
- Name <- tuple_to_list(get_solrq_worker_tuple())].
+ [{IndexPartition, yz_solrq_worker:set_hwm(Index, Partition, HWM)} ||
+ {Index, Partition} = IndexPartition <- yz_solrq_sup:active_queues()].
%% @doc Set the index parameters for all queues (note, index goes back to appenv
%% queue is empty).
-spec set_index(index_name(), solrq_batch_min(), solrq_batch_max(),
solrq_batch_flush_interval()) ->
- [{atom(), {ok | error, {Params :: number()} |
+ [{{index_name(), p()}, {ok | error, {Params :: number()} |
bad_index_params}}].
set_index(Index, Min, Max, DelayMsMax) ->
- [{Name, yz_solrq_worker:set_index(Name, Index, Min, Max, DelayMsMax)} ||
- Name <- tuple_to_list(get_solrq_worker_tuple())].
+ [{IndexPartition, yz_solrq_worker:set_index(Index, Partition, Min, Max, DelayMsMax)} ||
+ {_Index, Partition} = IndexPartition <- solrq_worker_pairs_for_index(Index)].
%% @doc Set the purge strategy on all queues
-spec set_purge_strategy(purge_strategy()) ->
- [{atom(), {ok | error,
+ [{{index_name(), p()}, {ok | error,
purge_strategy()
| bad_purge_strategy}}].
set_purge_strategy(PurgeStrategy) ->
- [{Name, yz_solrq_worker:set_purge_strategy(Name, PurgeStrategy)} ||
- Name <- tuple_to_list(get_solrq_worker_tuple())].
+ [{IndexPartition, yz_solrq_worker:set_purge_strategy(Index, Partition, PurgeStrategy)} ||
+ {Index, Partition} =IndexPartition <- yz_solrq_sup:active_queues()].
%% @doc Request each solrq reloads from appenv - currently only affects HWM
--spec reload_appenv() -> [{atom(), ok}].
+-spec reload_appenv() -> [{{index_name(), p()}, ok}].
reload_appenv() ->
- [{Name, yz_solrq_worker:reload_appenv(Name)} ||
- Name <- tuple_to_list(get_solrq_worker_tuple())].
+ [{IndexPartition, yz_solrq_worker:reload_appenv(Index, Partition)} ||
+ {Index, Partition} = IndexPartition <- yz_solrq_sup:active_queues()].
%% @doc Signal to all Solrqs that a fuse has blown for the the specified index.
-spec blown_fuse(index_name()) -> ok.
blown_fuse(Index) ->
lists:foreach(
fun(Name) ->
- yz_solrq_worker:blown_fuse(Name, Index)
+ yz_solrq_worker:blown_fuse(Name)
end,
- solrq_worker_names()
+ solrq_workers_for_index(Index)
).
%% @doc Signal to all Solrqs that a fuse has healed for the the specified index.
@@ -153,9 +123,9 @@ blown_fuse(Index) ->
healed_fuse(Index) ->
lists:foreach(
fun(Name) ->
- yz_solrq_worker:healed_fuse(Name, Index)
+ yz_solrq_worker:healed_fuse(Name)
end,
- solrq_worker_names()
+ solrq_workers_for_index(Index)
).
%% @doc Return the status of all solrq workers.
@@ -166,42 +136,58 @@ status() ->
%% @doc Return the list of solrq names registered with this supervisor
-spec solrq_worker_names() -> [atom()].
solrq_worker_names() ->
- tuple_to_list(get_solrq_worker_tuple()).
-
-%% @doc Return the list of solrq names registered with this supervisor
--spec solrq_helper_names() -> [atom()].
-solrq_helper_names() ->
- tuple_to_list(get_solrq_helper_tuple()).
+ [yz_solrq:worker_regname(Index, Partition) ||
+ {Index, Partition} <- yz_solrq_sup:active_queues()].
+
+-spec all_solrq_workers() -> [atom()].
+all_solrq_workers() ->
+ [yz_solrq:worker_regname(Index, Partition) ||
+ {Index, Partition} <- yz_solrq_sup:active_queues()].
+
+-spec solrq_workers_for_partition(Partition :: p()) -> [atom()].
+solrq_workers_for_partition(Partition) ->
+ [yz_solrq:worker_regname(Index, Partition) ||
+ {Index, WorkerPartition} <- yz_solrq_sup:active_queues(),
+ Partition == WorkerPartition].
+
+-spec solrq_worker_pairs_for_index(Index :: index_name()) -> [{index_name(), p()}].
+solrq_worker_pairs_for_index(Index) ->
+ [{Index, Partition} ||
+ {WorkerIndex, Partition} <- yz_solrq_sup:active_queues(),
+ Index == WorkerIndex].
+
+-spec solrq_workers_for_index(Index :: index_name()) -> [atom()].
+solrq_workers_for_index(Index) ->
+ [yz_solrq:worker_regname(Index, Partition) ||
+ {WorkerIndex, Partition} <- yz_solrq_sup:active_queues(),
+ Index == WorkerIndex].
%% @doc return the total length of all solrq workers on the node.
-spec queue_total_length() -> non_neg_integer().
queue_total_length() ->
- lists:sum([yz_solrq_worker:all_queue_len(Name) || Name <- tuple_to_list(get_solrq_worker_tuple())]).
+ lists:sum([yz_solrq_worker:all_queue_len(Index, Partition) || {Index, Partition} <- yz_solrq_sup:active_queues()]).
+
+get_max_batch_size() ->
+ app_helper:get_env(?YZ_APP_NAME, ?SOLRQ_BATCH_MAX, ?SOLRQ_BATCH_MAX_DEFAULT).
+
+get_min_batch_size() ->
+ app_helper:get_env(?YZ_APP_NAME, ?SOLRQ_BATCH_MIN, ?SOLRQ_BATCH_MIN_DEFAULT).
+get_flush_interval() ->
+ app_helper:get_env(?YZ_APP_NAME, ?SOLRQ_BATCH_FLUSH_INTERVAL,
+ ?SOLRQ_BATCH_FLUSH_INTERVAL_DEFAULT).
%%%===================================================================
%%% Internal functions
%%%===================================================================
-get_solrq_worker_tuple() ->
- mochiglobal:get(?SOLRQS_TUPLE_KEY).
-
-set_solrq_worker_tuple(Size) ->
- mochiglobal:put(?SOLRQS_TUPLE_KEY, solrq_workers_tuple(Size)).
-
-get_solrq_helper_tuple() ->
- mochiglobal:get(?SOLRQ_HELPERS_TUPLE_KEY).
-
-set_solrq_helper_tuple(Size) ->
- mochiglobal:put(?SOLRQ_HELPERS_TUPLE_KEY, solrq_helpers_tuple(Size)).
-
-solrq_workers_tuple(Queues) ->
- list_to_tuple([int_to_worker_regname(I) || I <- lists:seq(1, Queues)]).
-
-solrq_helpers_tuple(Helpers) ->
- list_to_tuple([int_to_helper_regname(I) || I <- lists:seq(1, Helpers)]).
-
-int_to_worker_regname(I) ->
- list_to_atom(lists:flatten(io_lib:format("yz_solrq_worker_~4..0b", [I]))).
-
-int_to_helper_regname(I) ->
- list_to_atom(lists:flatten(io_lib:format("yz_solrq_helper_~4..0b", [I]))).
+ensure_worker(Index, Partition) ->
+ WorkerName = yz_solrq:worker_regname(Index, Partition),
+ case whereis(WorkerName) of
+ undefined ->
+ %% Two processes may both get here at once. It's ok to ignore the
+ %% return value here, as we would just ignore the already_started
+ %% error anyway.
+ ok = yz_solrq_sup:start_queue_pair(Index, Partition);
+ _Pid ->
+ ok
+ end.
diff --git a/src/yz_solrq_drain_fsm.erl b/src/yz_solrq_drain_fsm.erl
index 0e508fef..c5dc4b73 100644
--- a/src/yz_solrq_drain_fsm.erl
+++ b/src/yz_solrq_drain_fsm.erl
@@ -21,7 +21,7 @@
-behaviour(gen_fsm).
%% API
--export([start_link/0, start_link/1, cancel/1]).
+-export([start_link/0, start_link/1, cancel/2]).
%% gen_fsm callbacks
-export([init/1,
@@ -31,13 +31,20 @@
terminate/3,
code_change/4]).
--export([start_prepare/0, prepare/2, wait/2, drain_complete/2]).
+%% gen_fsm states
+-export([
+ prepare/2,
+ wait_for_drain_complete/2,
+ wait_for_snapshot_complete/2,
+ wait_for_yz_hashtree_updated/2
+]).
--ifdef(PULSE).
--compile(export_all).
--compile({parse_transform, pulse_instrument}).
--compile({pulse_replace_module, [{gen_fsm, pulse_gen_fsm}]}).
--endif.
+%% API
+-export([
+ start_prepare/1,
+ drain_complete/2,
+ drain_already_in_progress/2,
+ resume_workers/1]).
-include("yokozuna.hrl").
-define(SERVER, ?MODULE).
@@ -47,7 +54,8 @@
exchange_fsm_pid,
yz_index_hashtree_update_params,
partition,
- time_start
+ time_start,
+ owner_pid
}).
%%%===================================================================
@@ -71,7 +79,7 @@ start_link() ->
%%
-spec(start_link(drain_params()) -> {ok, pid()} | ignore | {error, Reason :: term()}).
start_link(Params) ->
- gen_fsm:start_link({local, ?SERVER}, ?MODULE, Params, []).
+ gen_fsm:start_link(?MODULE, Params, []).
%% @doc Notify the drain FSM identified by DPid that the solrq associated
%% with the specified Token has completed draining. Note that the solrq
@@ -85,6 +93,16 @@ start_link(Params) ->
drain_complete(DPid, Token) ->
gen_fsm:send_event(DPid, {drain_complete, Token}).
+%% @doc Notify the drain FSM identified by DPid that the solrq associated
+%% with the specified Token has an existing drain request in progress and
+%% cannot perform another drain at this time.
+%%
+%% NB. This function is typically called from each solrq.
+%% @end
+%%
+drain_already_in_progress(DPid, Token) ->
+ gen_fsm:send_event(DPid, {drain_already_in_progress, Token}).
+
%% @doc Start draining. This operation will send a start message to this
%% FSM with a start message, which in turn will initiate drains on all of
%% the solrqs.
@@ -92,18 +110,18 @@ drain_complete(DPid, Token) ->
%% NB. This function is typically called from the drain manager.
%% @end
%%
--spec start_prepare() -> no_proc | timeout | term().
-start_prepare() ->
- gen_fsm:send_event(?SERVER, start).
+-spec start_prepare(pid()) -> no_proc | timeout | term().
+start_prepare(DPid) ->
+ gen_fsm:send_event(DPid, start).
%% @doc Cancel a drain. This operation will result in sending a cancel
%% message to each of the solrqs, putting them back into a batching state.
%% @end
%%
--spec cancel(non_neg_integer()) -> ok | no_proc | timeout.
-cancel(Timeout) ->
+-spec cancel(pid(), non_neg_integer()) -> ok | no_proc | timeout.
+cancel(DPid, Timeout) ->
try
- gen_fsm:sync_send_all_state_event(?SERVER, cancel, Timeout)
+ gen_fsm:sync_send_all_state_event(DPid, cancel, Timeout)
catch
_:{normal, _} ->
%% It's possible that the drain FSM terminates "naturally"
@@ -116,6 +134,15 @@ cancel(Timeout) ->
timeout
end.
+%%
+%% Resume workers, typically called from the callback supplied to
+%% yz_index_hashtree. We are declaring this one-liner as a public
+%% API function in order to have an effective intecept in riak_test
+%% C.f., yz_solrq_test:confirm_drain_fsm_timeout
+%%
+resume_workers(Pid) ->
+ gen_fsm:send_event(Pid, resume_workers).
+
%%%===================================================================
%%% gen_fsm callbacks
%%%===================================================================
@@ -124,7 +151,8 @@ init(Params) ->
{ok, prepare, #state{
exchange_fsm_pid = proplists:get_value(?EXCHANGE_FSM_PID, Params),
yz_index_hashtree_update_params = proplists:get_value(?YZ_INDEX_HASHTREE_PARAMS, Params),
- partition = proplists:get_value(?DRAIN_PARTITION, Params)
+ partition = proplists:get_value(?DRAIN_PARTITION, Params),
+ owner_pid = proplists:get_value(owner_pid, Params)
}}.
@@ -134,44 +162,87 @@ init(Params) ->
%% @end
%%
prepare(start, #state{partition = P} = State) ->
- lager:debug("Starting a drain for partition ~p", [P]),
- SolrqIds = yz_solrq:solrq_worker_names(),
- TS = os:timestamp(),
- Tokens = [yz_solrq_worker:drain(SolrqId, P) || SolrqId <- SolrqIds],
- {next_state, wait, State#state{tokens = Tokens, time_start=TS}}.
+ lager:debug("Solrq drain starting for partition ~p", [P]),
+ SolrqIds = get_solrq_ids(P),
+ maybe_send_drain_messages(P, SolrqIds, State).
%% @doc While in the wait state, we wait for drain_complete messages with accompanying
%% tokens. When we have received all of the tokens, we are done, and the FSM terminates
%% normally. Otherwise, we keep waiting.
%% @end
%%
-wait({drain_complete, Token},
+wait_for_drain_complete({drain_complete, Token},
#state{
tokens = Tokens,
exchange_fsm_pid = ExchangeFSMPid,
yz_index_hashtree_update_params = YZIndexHashtreeUpdateParams,
- time_start = StartTS
+ time_start = StartTS,
+ partition = Partition
} = State) ->
Tokens2 = lists:delete(Token, Tokens),
NewState = State#state{tokens = Tokens2},
case Tokens2 of
[] ->
- lager:debug("Drain completed for all workers. Resuming batching."),
+ lager:debug("Solrq drain completed for all workers for partition ~p. Resuming batching.", [Partition]),
yz_stat:drain_end(?YZ_TIME_ELAPSED(StartTS)),
- maybe_update_yz_index_hashtree(
- ExchangeFSMPid, YZIndexHashtreeUpdateParams
+ Self = self(),
+ %%
+ %% This callback function will be called from within the
+ %% yz_index_hashtree, after the hashtree we are exchanging
+ %% is snapshotted, but before it is updated. In this callback
+ %% we let the workers know they can resume normal operations,
+ %% and we inform ourself that workers have been resumed.
+ %%
+ SnapshotCompleteCallback = fun() ->
+ resume_workers(Self)
+ end,
+ spawn_link(
+ fun() ->
+ maybe_update_yz_index_hashtree(
+ ExchangeFSMPid, YZIndexHashtreeUpdateParams, SnapshotCompleteCallback
+ ),
+ gen_fsm:send_event(Self, yz_hashtree_updated)
+ end
),
- [yz_solrq_worker:drain_complete(Name) || Name <- yz_solrq:solrq_worker_names()],
- {stop, normal, NewState};
+ {next_state, wait_for_snapshot_complete, NewState};
_ ->
- {next_state, wait, NewState}
- end.
+ {next_state, wait_for_drain_complete, NewState}
+ end;
+%% If a drain is already in progress, but we are draining all queues,
+%% Try again until the previous drain completes.
+wait_for_drain_complete({drain_already_in_progress, Token, QPid},
+ #state{partition=undefined,
+ tokens = Tokens0}=State) ->
+ NewToken = yz_solrq_worker:drain(QPid, undefined),
+ Tokens1 = lists:delete(Token, Tokens0),
+ Tokens2 = [NewToken | Tokens1],
+ {next_state, wait_for_drain_complete, State#state{tokens = Tokens2}};
+
+%% In the case of a single-partition drain, just stop and let
+%% the calling process handle retries
+wait_for_drain_complete({drain_already_in_progress, _Token}, State) ->
+ {stop, overlapping_drain_requested, State}.
+
+%% The workers have resumed normal operations. Draining is now "complete",
+%% but we need to wait for the yz_index_hashtree to update its inner hashes,
+%% which can take some time. In the meantime, notify the process waiting
+%% for drains to complete
+wait_for_snapshot_complete(resume_workers, #state{partition=Partition, owner_pid=OwnerPid} = State) ->
+ lists:foreach(
+ fun yz_solrq_worker:drain_complete/1,
+ get_solrq_ids(Partition)
+ ),
+ notify_workers_resumed(OwnerPid),
+ {next_state, wait_for_yz_hashtree_updated, State}.
+
+wait_for_yz_hashtree_updated(yz_hashtree_updated, State) ->
+ {stop, normal, State}.
handle_event(_Event, StateName, State) ->
{next_state, StateName, State}.
-handle_sync_event(cancel, _From, _StateName, State) ->
- [yz_solrq_worker:cancel_drain(Name) || Name <- yz_solrq:solrq_worker_names()],
+handle_sync_event(cancel, _From, _StateName, #state{partition=Partition} = State) ->
+ [yz_solrq_worker:cancel_drain(Name) || Name <- get_solrq_ids(Partition)],
{stop, normal, ok, State};
handle_sync_event(_Event, _From, StateName, State) ->
@@ -190,7 +261,48 @@ code_change(_OldVsn, StateName, State, _Extra) ->
%%% Internal functions
%%%===================================================================
-maybe_update_yz_index_hashtree(undefined, undefined) ->
+notify_workers_resumed(OwnerPid) ->
+ OwnerPid ! workers_resumed.
+
+maybe_update_yz_index_hashtree(undefined, undefined, Callback) ->
+ maybe_callback(Callback),
+ ok;
+maybe_update_yz_index_hashtree(Pid, {YZTree, Index, IndexN}, Callback) ->
+ yz_exchange_fsm:update_yz_index_hashtree(Pid, YZTree, Index, IndexN, Callback).
+
+
+%%
+%% @doc There were no queues to drain for this index/partition
+%% Just update the hashtree and stop rather than waiting
+%% for messages that will never arrive.
+maybe_send_drain_messages(_P, [], #state{
+ exchange_fsm_pid = ExchangeFSMPid,
+ yz_index_hashtree_update_params = YZIndexHashtreeUpdateParams } = State) ->
+ maybe_update_yz_index_hashtree(
+ ExchangeFSMPid, YZIndexHashtreeUpdateParams, undefined
+ ),
+ {stop, normal, State};
+%%
+%% @doc one or more queues need to be drained - send
+%% drain messages and move to wait state
+maybe_send_drain_messages(P, SolrqIds, State) ->
+ TS = os:timestamp(),
+ Tokens = [yz_solrq_worker:drain(SolrqId, P) || SolrqId <- SolrqIds],
+ {next_state, wait_for_drain_complete, State#state{tokens = Tokens, time_start = TS}}.
+
+%%
+%% @doc if partition is `undefined` then drain all queues.
+%% This is used in the yz_solrq_drain_mgr:drain/0 case,
+%% usually during yokozuna's `yz_app:prep_stop`
+get_solrq_ids(undefined) ->
+ yz_solrq:all_solrq_workers();
+%%
+%% @doc drain all queues for a particular partition, `P`.
+get_solrq_ids(P) ->
+ yz_solrq:solrq_workers_for_partition(P).
+
+maybe_callback(undefined) ->
ok;
-maybe_update_yz_index_hashtree(Pid, {YZTree, Index, IndexN}) ->
- yz_exchange_fsm:update_yz_index_hashtree(Pid, YZTree, Index, IndexN).
+maybe_callback(Callback) ->
+ Callback().
+
diff --git a/src/yz_solrq_drain_mgr.erl b/src/yz_solrq_drain_mgr.erl
index b0b084c3..33819520 100644
--- a/src/yz_solrq_drain_mgr.erl
+++ b/src/yz_solrq_drain_mgr.erl
@@ -36,7 +36,7 @@
-define(SERVER, ?MODULE).
--record(state, {draining = false :: boolean()}).
+-record(state, {draining = [] :: [p()]}).
%%%===================================================================
%%% API
@@ -65,35 +65,46 @@ init([]) ->
schedule_tick(),
{ok, #state{}}.
-handle_call({drain, Params}, _From, #state{draining=true} = State) ->
+handle_call({drain, Params}, From, #state{draining=Draining} = State) ->
ExchangeFSMPid = proplists:get_value(
?EXCHANGE_FSM_PID, Params, undefined
),
- lager:debug("Drain in progress."),
- maybe_exchange_fsm_drain_error(ExchangeFSMPid, {error, in_progress}),
- {reply, {error, in_progress}, State};
-handle_call({drain, Params}, From, State) ->
- ExchangeFSMPid = proplists:get_value(
- ?EXCHANGE_FSM_PID, Params, undefined
- ),
- spawn_link(
- fun() ->
- Result = try
- maybe_drain(enabled(), ExchangeFSMPid, Params)
- catch
- _:E ->
- lager:debug("An error occurred draining: ~p", [E]),
- maybe_exchange_fsm_drain_error(ExchangeFSMPid, E),
- {error, E}
- end,
- gen_server:cast(?SERVER, drain_complete),
- gen_server:reply(From, Result)
- end
+ PartitionToDrain = proplists:get_value(
+ ?DRAIN_PARTITION, Params, undefined
),
- {noreply, State#state{draining = true}}.
+ AlreadyDraining = lists:member(PartitionToDrain, Draining),
+ case AlreadyDraining of
+ true ->
+ lager:debug("Drain in progress."),
+ maybe_exchange_fsm_drain_error(ExchangeFSMPid, {error, in_progress}),
+ {reply, {error, in_progress}, State};
+ _ ->
+ lager:debug("Solrq drain starting for partition ~p", [PartitionToDrain]),
+ ExchangeFSMPid = proplists:get_value(
+ ?EXCHANGE_FSM_PID, Params, undefined
+ ),
+ spawn_link(
+ fun() ->
+ Result = try
+ maybe_drain(enabled(), ExchangeFSMPid, Params)
+ catch
+ _:E ->
+ lager:info("An error occurred draining: ~p", [E]),
+ maybe_exchange_fsm_drain_error(ExchangeFSMPid, E),
+ {error, E}
+ end,
+ lager:debug("Solrq drain about to send compelte message for partition ~p.", [PartitionToDrain]),
+ gen_server:cast(?SERVER, {drain_complete, PartitionToDrain}),
+ gen_server:reply(From, Result)
+ end
+ ),
+ {noreply, State#state{draining = Draining ++ [PartitionToDrain]}}
+ end.
-handle_cast(drain_complete, State) ->
- {noreply, State#state{draining = false}}.
+handle_cast({drain_complete, Partition}, #state{draining = Draining} = State) ->
+ lager:debug("Solrq drain completed for partition ~p.", [Partition]),
+ NewDraining = lists:delete(Partition, Draining),
+ {noreply, State#state{draining = NewDraining}}.
%% Handle race conditions in monitor/receive timeout
handle_info({'DOWN', _Ref, _, _Obj, _Status}, State) ->
@@ -129,38 +140,61 @@ maybe_drain(false, ExchangeFSMPid, Params) ->
actual_drain(Params, ExchangeFSMPid) ->
DrainTimeout = application:get_env(?YZ_APP_NAME,
- ?SOLRQ_DRAIN_TIMEOUT, 60000),
- {ok, Pid} = yz_solrq_sup:start_drain_fsm(Params),
+ ?SOLRQ_DRAIN_TIMEOUT, ?SOLRQ_DRAIN_TIMEOUT_DEFAULT),
+ Params1 = [{owner_pid, self()} | Params],
+ {ok, Pid} = yz_solrq_sup:start_drain_fsm(Params1),
Reference = erlang:monitor(process, Pid),
- yz_solrq_drain_fsm:start_prepare(),
+ yz_solrq_drain_fsm:start_prepare(Pid),
try
- receive
- {'DOWN', Reference, process, Pid, normal} ->
- lager:debug("Drain ~p completed normally.", [Pid]),
- ok;
- {'DOWN', Reference, process, Pid, Reason} ->
- lager:debug("Drain ~p failed with reason ~p", [Pid, Reason]),
- yz_stat:drain_fail(),
- maybe_exchange_fsm_drain_error(ExchangeFSMPid, Reason),
+ WorkersResumed = wait_for_workers_resumed_or_crash(DrainTimeout, Reference, Pid, ExchangeFSMPid),
+ case WorkersResumed of
+ ok ->
+ wait_for_exit(Reference, Pid, ExchangeFSMPid);
+ {error, Reason} ->
{error, Reason}
- after DrainTimeout ->
- lager:debug("Drain ~p timed out. Cancelling...", [Pid]),
- yz_stat:drain_timeout(),
- _ = cancel(Reference, Pid),
- maybe_exchange_fsm_drain_error(ExchangeFSMPid, timeout),
- {error, timeout}
end
after
erlang:demonitor(Reference)
end.
+wait_for_workers_resumed_or_crash(DrainTimeout, Reference, Pid, ExchangeFSMPid) ->
+ receive
+ workers_resumed ->
+ lager:debug("Workers resumed."),
+ ok;
+ {'DOWN', Reference, process, Pid, Reason} ->
+ lager:error("Drain ~p exited prematurely.", [Pid]),
+ handle_drain_fsm_pid_crash(Reason, ExchangeFSMPid)
+ after DrainTimeout ->
+ lager:debug("Drain ~p timed out. Cancelling...", [Pid]),
+ yz_stat:drain_timeout(),
+ _ = cancel(Reference, Pid),
+ maybe_exchange_fsm_drain_error(ExchangeFSMPid, timeout),
+ {error, timeout}
+ end.
+
+wait_for_exit(Reference, Pid, ExchangeFSMPid) ->
+ receive
+ {'DOWN', Reference, process, Pid, normal} ->
+ lager:debug("Drain ~p completed normally.", [Pid]),
+ ok;
+ {'DOWN', Reference, process, Pid, Reason} ->
+ lager:error("Drain ~p crashed with reason ~p.", [Pid, Reason]),
+ handle_drain_fsm_pid_crash(Reason, ExchangeFSMPid)
+ end.
+
+handle_drain_fsm_pid_crash(Reason, ExchangeFSMPid) ->
+ yz_stat:drain_fail(),
+ maybe_exchange_fsm_drain_error(ExchangeFSMPid, Reason),
+ {error, Reason}.
+
enabled() ->
- application:get_env(?YZ_APP_NAME, ?SOLRQ_DRAIN_ENABLE, true).
+ application:get_env(?YZ_APP_NAME, ?SOLRQ_DRAIN_ENABLE, ?SOLRQ_DRAIN_ENABLE_DEFAULT).
cancel(Reference, Pid) ->
CancelTimeout = application:get_env(
- ?YZ_APP_NAME, ?SOLRQ_DRAIN_CANCEL_TIMEOUT, 5000),
- case yz_solrq_drain_fsm:cancel(CancelTimeout) of
+ ?YZ_APP_NAME, ?SOLRQ_DRAIN_CANCEL_TIMEOUT, ?SOLRQ_DRAIN_CANCEL_TIMEOUT_DEFAULT),
+ case yz_solrq_drain_fsm:cancel(Pid, CancelTimeout) of
timeout ->
lager:warning("Drain cancel timed out. Killing FSM pid ~p...", [Pid]),
yz_stat:drain_cancel_timeout(),
@@ -181,8 +215,7 @@ unlink_and_kill(Reference, Pid) ->
-spec schedule_tick() -> reference().
schedule_tick() ->
- erlang:send_after(5000, ?MODULE, tick).
-
+ erlang:send_after(5000, ?SERVER, tick).
maybe_exchange_fsm_drain_error(undefined, _Reason) ->
ok;
@@ -192,4 +225,4 @@ maybe_exchange_fsm_drain_error(Pid, Reason) ->
maybe_update_yz_index_hashtree(undefined, undefined) ->
ok;
maybe_update_yz_index_hashtree(Pid, {YZTree, Index, IndexN}) ->
- yz_exchange_fsm:update_yz_index_hashtree(Pid, YZTree, Index, IndexN).
+ yz_exchange_fsm:update_yz_index_hashtree(Pid, YZTree, Index, IndexN, undefined).
diff --git a/src/yz_solrq_helper.erl b/src/yz_solrq_helper.erl
index eb2f2d2d..e7b486e8 100644
--- a/src/yz_solrq_helper.erl
+++ b/src/yz_solrq_helper.erl
@@ -23,26 +23,24 @@
-behavior(gen_server).
%% api
--export([start_link/1, status/1, status/2]).
+-export([start_link/2, status/1, status/2]).
%% gen_server callbacks
-export([init/1, handle_call/3, handle_cast/2, handle_info/2,
terminate/2, code_change/3]).
% solrq/helper interface
--export([index_ready/3, index_ready/2, index_batch/5]).
+-export([index_batch/5]).
%% TODO: Dynamically pulse_instrument.
--ifdef(PULSE).
--compile(export_all).
--compile({parse_transform, pulse_instrument}).
--compile({pulse_replace_module, [{gen_server, pulse_gen_server}]}).
-
--define(PULSE_DEBUG(S,F), pulse:format(S,F)).
+-ifdef(EQC).
+-define(EQC_DEBUG(S, F), _=element(1, {S, F}), ok).
+%%-define(EQC_DEBUG(S, F), eqc:format(S, F)).
+%%-define(EQC_DEBUG(S, F), io:fwrite(user, S, F)).
debug_entries(Entries) ->
[erlang:element(1, Entry) || Entry <- Entries].
-else.
--define(PULSE_DEBUG(S,F), ok).
+-define(EQC_DEBUG(S, F), ok).
-endif.
-record(state, {}).
@@ -63,8 +61,8 @@ debug_entries(Entries) ->
%%% API
%%%===================================================================
-start_link(Name) ->
- gen_server:start_link({local, Name}, ?MODULE, [], []).
+start_link(Index, Partition) ->
+ gen_server:start_link({local, yz_solrq:helper_regname(Index, Partition)}, ?MODULE, [], []).
status(Pid) ->
status(Pid, 60000). % solr can block, long timeout by default
@@ -72,21 +70,6 @@ status(Pid) ->
status(Pid, Timeout) ->
gen_server:call(Pid, status, Timeout).
--spec index_ready(index_name(), solrq_id()) -> ok.
-index_ready(Index, QPid) ->
- HPid = yz_solrq:random_helper(),
- index_ready(HPid, Index, QPid).
-
-%% @doc Mark the index as ready. Separating into a two phase
-%% rather than just blindly sending from the solrq adds the
-%% backpressure on the KV vnode.
--spec index_ready(solrq_helper_id(), index_name(), solrq_id()) -> ok.
-index_ready(HPid, Index, QPid) when is_atom(HPid); is_pid(HPid) ->
- gen_server:cast(HPid, {ready, Index, QPid});
-index_ready(Hash, Index, QPid) ->
- HPid = yz_solrq:helper_regname(Hash),
- index_ready(HPid, Index, QPid).
-
%% @doc Index a batch
-spec index_batch(solrq_helper_id(),
index_name(),
@@ -108,21 +91,18 @@ handle_call(status, _From, State) ->
handle_call(BadMsg, _From, State) ->
{reply, {error, {unknown, BadMsg}}, State}.
-handle_cast({ready, Index, QPid}, State) ->
- yz_solrq_worker:request_batch(QPid, Index, self()),
- {noreply, State};
handle_cast({batch, Index, BatchMax, QPid, Entries}, State) ->
- ?PULSE_DEBUG("Handling batch for index ~p. Entries: ~p~n", [Index, debug_entries(Entries)]),
+ ?EQC_DEBUG("Handling batch for index ~p. Entries: ~p~n", [Index, debug_entries(Entries)]),
Message = case do_batches(Index, BatchMax, [], Entries) of
ok ->
{length(Entries), ok};
{ok, Delivered} ->
{length(Delivered), {retry, remove(Delivered, Entries)}};
{error, Undelivered} ->
- ?PULSE_DEBUG("Error handling batch for index ~p. Undelivered: ~p~n", [Index, debug_entries(Undelivered)]),
+ %% ?EQC_DEBUG("Error handling batch for index ~p. Undelivered: ~p~n", [Index, debug_entries(Undelivered)]),
{length(Entries) - length(Undelivered), {retry, Undelivered}}
end,
- yz_solrq_worker:batch_complete(QPid, Index, Message),
+ yz_solrq_worker:batch_complete(QPid, Message),
{noreply, State}.
%%%===================================================================
@@ -152,6 +132,7 @@ do_batches(Index, BatchMax, Delivered, Entries) ->
{ok, DeliveredInBatch} ->
{ok, DeliveredInBatch ++ Delivered};
{error, _Reason} ->
+ %% ?EQC_DEBUG("Error handling batch:~p", [_Reason]),
{error, Entries}
end.
@@ -160,118 +141,137 @@ do_batches(Index, BatchMax, Delivered, Entries) ->
| {ok, Delivered :: solr_entries()} % a strict subset of entries were delivered
| {error, Reason :: term()}. % an error occurred; retry all of them
do_batch(Index, Entries0) ->
- try
- %% TODO: use ibrowse http worker
- %% TODO: batch updates to YZ AAE
- %% TODO: move the owned/next partition logic back up
- %% to yz_kv:index/3 once we efficiently cache
- %% owned and next rather than calculating per-object.
- Ring = yz_misc:get_ring(transformed),
- LI = yz_cover:logical_index(Ring),
- OwnedAndNext = yz_misc:owned_and_next_partitions(node(), Ring),
-
- Entries1 = [{BKey, Obj, Reason, P,
- riak_kv_util:get_index_n(BKey), yz_kv:hash_object(Obj)} ||
- {BKey, Obj, Reason, P} <-
- yz_misc:filter_out_fallbacks(OwnedAndNext, Entries0)],
- case update_solr(Index, LI, Entries1) of
- ok ->
- update_aae_and_repair_stats(Entries1),
- ok;
- {ok, Entries2} ->
- update_aae_and_repair_stats(Entries2),
- {ok, [{BKey, Obj, Reason, P} || {BKey, Obj, Reason, P, _, _} <- Entries2]};
- {error, Reason} ->
- {error, Reason}
- end
- catch
- _:Err ->
- yz_stat:index_fail(),
- Trace = erlang:get_stacktrace(),
- ?DEBUG("index ~p failed - ~p\nat: ~p", [Index, Err, Trace]),
- {error, Err}
+ %% TODO: use ibrowse http worker
+ %% TODO: batch updates to YZ AAE
+ %% TODO: move the owned/next partition logic back up
+ %% to yz_kv:index/3 once we efficiently cache
+ %% owned and next rather than calculating per-object.
+ Ring = yz_misc:get_ring(transformed),
+ LI = yz_cover:logical_index(Ring),
+ OwnedAndNext = yz_misc:owned_and_next_partitions(node(), Ring),
+
+ Entries1 = [{BKey, {Obj, _OldObj}, Reason, P,
+ riak_kv_util:get_index_n(BKey), yz_kv:hash_object(Obj)} ||
+ {BKey, {Obj, _OldObj}, Reason, P} <-
+ yz_misc:filter_out_fallbacks(OwnedAndNext, Entries0)],
+ case update_solr(Index, LI, Entries1) of
+ ok ->
+ update_aae_and_repair_stats(Entries1),
+ ok;
+ {ok, Entries2} ->
+ update_aae_and_repair_stats(Entries2),
+ {ok, [{BKey, Objects, Reason, P} || {BKey, Objects, Reason, P, _, _} <- Entries2]};
+ {error, Reason} ->
+ {error, Reason}
end.
-%% @doc Entries is [{Index, BKey, Obj, Reason, P, ShortPL, Hash}]
+%% @doc Entries is [{BKey, Obj, Reason, P, ShortPL, Hash}]
-spec update_solr(index_name(), logical_idx(), solr_entries()) ->
ok | {ok, SuccessEntries :: solr_entries()} |
{error, fuse_blown} | {error, tuple()}.
update_solr(_Index, _LI, []) -> % nothing left after filtering fallbacks
ok;
-update_solr(Index, LI, Entries) ->
- case yz_kv:should_index(Index) of
- false ->
- ok; % No need to send anything to SOLR, still need for AAE.
+update_solr(Index, LI, Entries0) when ?YZ_SHOULD_INDEX(Index) ->
+ case yz_fuse:check(Index) of
+ blown ->
+ %% ?EQC_DEBUG( "Fuse Blown: can't currently send solr "
+ %% "operations for index ~s", [Index]),
+ {error, fuse_blown};
_ ->
- case yz_fuse:check(Index) of
- ok ->
- send_solr_ops_for_entries(Index, solr_ops(LI, Entries),
- Entries);
- blown ->
- ?DEBUG("Fuse Blown: can't currently send solr "
- "operations for index ~s", [Index]),
- {error, fuse_blown};
- _ ->
- %% fuse table creation is idempotent and occurs on
- %% yz_index:add_index/1 on 1st creation or diff-check.
- %% We send entries until we can ask again for
- %% ok | error, as we wait for the tick.
- send_solr_ops_for_entries(Index, solr_ops(LI, Entries),
- Entries)
- end
- end.
+ %% fuse table creation is idempotent and occurs on
+ %% yz_index:add_index/1 on 1st creation or diff-check.
+ %% We send entries until we can ask again for
+ %% ok | error, as we wait for the tick.
+ Ops = solr_ops(LI, Entries0),
+ send_solr_ops_for_entries(Index, Ops, Entries0)
+ end;
+update_solr(_Index, _LI, _Entries) ->
+ ok.
%% @doc Build the SOLR query
-spec solr_ops(logical_idx(), solr_entries()) -> solr_ops().
solr_ops(LI, Entries) ->
- [get_ops_for_entry(Entry, LI) || Entry <- Entries].
+ [get_ops_for_entry(Entry, LI) || Entry <- Entries].
-spec get_ops_for_entry(solr_entry(), logical_idx()) -> solr_ops().
-get_ops_for_entry({BKey, Obj0, Reason, P, ShortPL, Hash}, LI) ->
+get_ops_for_entry({BKey, {Obj0, _OldObj}=Objects, Reason, P, ShortPL, Hash}, LI) ->
{Bucket, _} = BKey,
BProps = riak_core_bucket:get_bucket(Bucket),
Obj = yz_kv:maybe_merge_siblings(BProps, Obj0),
ObjValues = riak_object:get_values(Obj),
Action = get_reason_action(Reason),
- get_ops_for_entry_action(Action, ObjValues, LI, P, Obj, BKey, ShortPL,
+ get_ops_for_entry_action(Action, ObjValues, LI, P, Objects, BKey, ShortPL,
Hash, BProps).
-spec get_ops_for_entry_action(write_action(), [riak_object:value()],
- logical_idx(), p(), obj(), bkey(), short_preflist(), hash(),
+ logical_idx(), p(), object_pair(), bkey(), short_preflist(), hash(),
riak_core_bucket:properties()) -> solr_ops().
-get_ops_for_entry_action(_Action, [notfound], _LI, _P, _Obj, BKey,
- _ShortPL, _Hash, _BProps) ->
- [{delete, yz_solr:encode_delete({bkey, BKey})}];
-get_ops_for_entry_action(anti_entropy_delete, _ObjValues, LI, P, Obj, _BKey,
- _ShortPL, _Hash, _BProps) ->
+get_ops_for_entry_action(_Action, [notfound], LI, P, _Objects, BKey,
+ _ShortPL, _Hash, _BProps) ->
+ LP = yz_cover:logical_partition(LI, P),
+ [{delete, yz_solr:encode_delete({bkey, BKey, LP})}];
+get_ops_for_entry_action(anti_entropy_delete, _ObjValues, LI, P, _FakeObjects, BKey,
+ _ShortPL, _Hash, _BProps) ->
+ %% anti-entropy is the "case of last resort" and at this point
+ %% we need to do a cleanup of _any_ documents that may be
+ %% floating around.
+ get_ops_for_object_cleanup(BKey, LI, P);
+get_ops_for_entry_action(anti_entropy, _ObjValues, LI, P, {Obj, _OldObj}, BKey,
+ ShortPL, Hash, _BProps) ->
+ %% anti-entropy is the "case of last resort" and at this point
+ %% we need to do a cleanup of _any_ documents that may be
+ %% floating around.
+ DeleteOpsForEntry = get_ops_for_object_cleanup(BKey, LI, P),
+ AddOpsForEntry = get_ops_for_add(LI, ShortPL, P, Obj, Hash),
+ [DeleteOpsForEntry, AddOpsForEntry];
+get_ops_for_entry_action(delete, _ObjValues, LI, P, {Obj, _OldObj}, _BKey,
+ _ShortPL, _Hash, BProps) ->
+ [get_ops_for_deletes(LI, P, Obj, BProps)];
+
+get_ops_for_entry_action(Action, _ObjValues, LI, P, {Obj, OldObj}, _BKey,
+ ShortPL, Hash, BProps) when
+ Action == handoff;
+ Action == put ->
+ DeleteOps = get_ops_for_deletes(LI, P, OldObj, BProps),
+ AddOps = get_ops_for_add(LI, ShortPL, P, Obj, Hash),
+ [DeleteOps, AddOps].
+
+get_ops_for_object_cleanup(BKey, LI, P) ->
+ LP = yz_cover:logical_partition(LI, P),
+ [[{delete, yz_solr:encode_delete({bkey, BKey, LP})}]].
+
+get_ops_for_add(LI, ShortPL, P, Obj, Hash) ->
+ LFPN = yz_cover:logical_partition(LI, element(1, ShortPL)),
+ LP = yz_cover:logical_partition(LI, P),
+ Docs = yz_doc:make_docs(Obj, Hash, ?INT_TO_BIN(LFPN),
+ ?INT_TO_BIN(LP)),
+ AddOps = yz_doc:adding_docs(Docs),
+ [{add, yz_solr:encode_doc(Doc)} ||
+ Doc <- AddOps].
+
+
+get_ops_for_deletes(_LI, _P, no_old_object, _BProps) ->
+ [];
+get_ops_for_deletes(LI, P, Obj, BProps) ->
+ case yz_kv:siblings_permitted(Obj, BProps) of
+ true ->
+ get_ops_for_sibling_deletes(LI, P, Obj);
+ _ ->
+ get_ops_for_no_sibling_deletes(LI, P, Obj)
+ end.
+
+get_ops_for_no_sibling_deletes(LI, P, Obj) ->
+ LP = yz_cover:logical_partition(LI, P),
+ DocId = yz_doc:doc_id(Obj, ?INT_TO_BIN(LP)),
+ [{delete, yz_solr:encode_delete({id, DocId})}].
+
+get_ops_for_sibling_deletes(LI, P, Obj) ->
LP = yz_cover:logical_partition(LI, P),
DocIds = yz_doc:doc_ids(Obj, ?INT_TO_BIN(LP)),
- DeleteOps =
- [{delete, yz_solr:encode_delete({id, DocId})}
- || DocId <- DocIds],
- [DeleteOps];
-get_ops_for_entry_action(delete, _ObjValues, _LI, _P, _Obj, BKey,
- _ShortPL, _Hash, _BProps) ->
- [{delete, yz_solr:encode_delete({bkey, BKey})}];
-get_ops_for_entry_action(Action, _ObjValues, LI, P, Obj, BKey,
- ShortPL, Hash, BProps) when Action == handoff;
- Action == put;
- Action == anti_entropy ->
- LFPN = yz_cover:logical_partition(LI, element(1, ShortPL)),
- LP = yz_cover:logical_partition(LI, P),
- Docs = yz_doc:make_docs(Obj, Hash, ?INT_TO_BIN(LFPN),
- ?INT_TO_BIN(LP)),
- AddOps = yz_doc:adding_docs(Docs),
- DeleteOps = yz_kv:delete_operation(BProps, Obj, Docs, BKey,
- LP),
-
- OpsForEntry = [[{delete, yz_solr:encode_delete(DeleteOp)} ||
- DeleteOp <- DeleteOps],
- [{add, yz_solr:encode_doc(Doc)}
- || Doc <- AddOps]
- ],
- [OpsForEntry].
+ DeleteOps = [{delete, yz_solr:encode_delete({id, DocId})}
+ || DocId <- DocIds],
+ [DeleteOps].
%% @doc A function that takes in an `Index', a list of `Ops' and the list
%% of `Entries', and attempts to batch_index them into Solr.
@@ -285,23 +285,23 @@ get_ops_for_entry_action(Action, _ObjValues, LI, P, Obj, BKey,
{ok, SuccessEntries :: solr_entries()} |
{error, tuple()}.
send_solr_ops_for_entries(Index, Ops, Entries) ->
- try
- T1 = os:timestamp(),
- ok = yz_solr:index_batch(Index, prepare_ops_for_batch(Ops)),
- yz_stat:index_end(Index, length(Ops), ?YZ_TIME_ELAPSED(T1)),
- ok
- catch _:Err ->
- yz_stat:index_fail(),
- Trace = erlang:get_stacktrace(),
+ T1 = os:timestamp(),
+ PreparedOps = prepare_ops_for_batch(Index, Ops),
+ %% ?EQC_DEBUG("send_solr_ops_for_entries: About to send entries. ~p", Entries),
+ case yz_solr:index_batch(Index, PreparedOps) of
+ ok ->
+ yz_stat:index_end(Index, length(Ops), ?YZ_TIME_ELAPSED(T1)),
+ ok;
+ {error, {Reason, _Detail}} = Err when Reason =:= badrequest; Reason =:= bad_data ->
?DEBUG("batch for index ~s failed. Error: ~p~n", [Index, Err]),
- case Err of
- {_, Reason, _} when Reason =:= badrequest; Reason =:= bad_data ->
- handle_bad_entries(Index, Ops, Entries);
- _ ->
- ?ERROR("Updating a batch of Solr operations failed for index ~p with error ~p", [Index, Err]),
- yz_fuse:melt(Index),
- {error, {Err, Trace}}
- end
+ %yz_stat:index_fail(),
+ handle_bad_entries(Index, Ops, Entries);
+ Err ->
+ ?DEBUG("batch for index ~s failed. Error: ~p~n", [Index, Err]),
+ ?ERROR("Updating a batch of Solr operations failed for index ~p with error ~p", [Index, Err]),
+ yz_fuse:melt(Index),
+ Trace = erlang:get_stacktrace(),
+ {error, {Err, Trace}}
end.
handle_bad_entries(Index, Ops, Entries) ->
@@ -321,52 +321,73 @@ handle_bad_entries(Index, Ops, Entries) ->
%% successful ops and applying side-effects to Solr.
-spec send_solr_single_ops(index_name(), solr_ops()) -> GoodOps :: solr_ops().
send_solr_single_ops(Index, Ops) ->
- lists:takewhile(
- fun(Op) ->
- try
- T1 = os:timestamp(),
- ok = yz_solr:index_batch(Index, prepare_ops_for_batch([Op])),
- yz_stat:index_end(Index, length(Ops), ?YZ_TIME_ELAPSED(T1)),
- true
- catch _:Err ->
- %% TODO This results in double counting index failures when
- %% we get a bad request back from Solr.
- %% We should probably refine our stats so that
- %% they differentiate between bad data and Solr going wonky
- yz_stat:index_fail(),
- case Err of
- {_, Reason, _} when Reason =:= badrequest; Reason =:= bad_data ->
- ?ERROR("Updating a single Solr operation failed for index ~p with bad request.", [Index]),
- true;
- _ ->
- ?ERROR("Updating a single Solr operation failed for index ~p with error ~p", [Index, Err]),
- yz_fuse:melt(Index),
- false
- end
- end
- end, Ops).
+ lists:takewhile(fun(Op) ->
+ single_op_batch(Index, Op)
+ end,
+ Ops).
+
+single_op_batch(Index, Op) ->
+ Ops = prepare_ops_for_batch(Index, [Op]),
+ case yz_solr:index_batch(Index, Ops) of
+ ok ->
+ T1 = os:timestamp(),
+ yz_stat:index_end(Index, length(Ops), ?YZ_TIME_ELAPSED(T1)),
+ true;
+ %% TODO This results in double counting index failures when
+ %% we get a bad request back from Solr.
+ %% We should probably refine our stats so that
+ %% they differentiate between bad data and Solr going wonky
+ {error, {Reason, _Details}} when Reason =:= badrequest; Reason =:= bad_data ->
+ yz_stat:index_bad_entry(),
+ ?ERROR("Updating a single Solr operation failed for index ~p with bad request.", [Index]),
+ true;
+ Err ->
+ yz_stat:index_fail(),
+ ?ERROR("Updating a single Solr operation failed for index ~p with error ~p", [Index, Err]),
+ yz_fuse:melt(Index),
+ false
+ end.
-spec update_aae_and_repair_stats(solr_entries()) -> ok.
update_aae_and_repair_stats(Entries) ->
- Repairs = lists:foldl(
- fun({BKey, _Obj, Reason, P, ShortPL, Hash}, StatsD) ->
- ReasonAction = get_reason_action(Reason),
- Action = hashtree_action(ReasonAction, Hash),
- yz_kv:update_hashtree(Action, P, ShortPL, BKey),
- gather_counts({P, ShortPL, Reason}, StatsD)
- end, dict:new(), Entries),
- dict:map(fun({Index, IndexN}, Count) ->
- case Count of
- 0 ->
- ok;
- Count ->
- lager:debug("Repaired ~b keys during active anti-entropy "
- "exchange of partition ~p for preflist ~p",
- [Count, Index, IndexN]),
- yz_kv:update_aae_exchange_stats(Index, IndexN, Count)
- end
- end, Repairs),
- ok.
+ update_hashtree_for_entries(Entries),
+ update_repair_stats(Entries).
+
+-spec update_repair_stats(solr_entries()) -> ok.
+update_repair_stats(Entries) ->
+ Repairs = calculate_repair_counts(Entries),
+ update_stats(Repairs).
+
+-spec calculate_repair_counts(solr_entries()) -> yz_dict().
+calculate_repair_counts(Entries) ->
+ Repairs = lists:foldl(fun increment_repair_count/2, dict:new(), Entries),
+ Repairs.
+
+-spec update_hashtree_for_entries(solr_entries()) -> ok.
+update_hashtree_for_entries(Entries) ->
+ lists:foreach(fun update_hashtree_for_entry/1, Entries).
+
+-spec update_stats(yz_dict()) -> ok.
+update_stats(RepairsDict) ->
+ Repairs = dict:to_list(RepairsDict),
+ lists:foreach(fun update_stat/1, Repairs).
+
+-spec update_stat({{p(), short_preflist()}, non_neg_integer()}) -> ok.
+update_stat({_, 0}) ->
+ ok;
+update_stat({{Partition, ShortPL}, Count}) ->
+ lager:debug("Repaired ~b keys during active anti-entropy "
+ "exchange of partition ~p for preflist ~p",
+ [Count, Partition, ShortPL]),
+ yz_kv:update_aae_exchange_stats(Partition, ShortPL, Count).
+
+increment_repair_count({_BKey, _Obj, Reason, P, ShortPL, _Hash}, StatsD) ->
+ maybe_add_repair_to_counts({P, ShortPL, Reason}, StatsD).
+
+update_hashtree_for_entry({BKey, _Obj, Reason, P, ShortPL, Hash}) ->
+ ReasonAction = get_reason_action(Reason),
+ Action = hashtree_action(ReasonAction, Hash),
+ yz_kv:update_hashtree(Action, P, ShortPL, BKey).
-spec hashtree_action(write_action(), hash()) ->
delete | {insert, hash()}.
@@ -379,13 +400,11 @@ hashtree_action(Action, Hash) when Action == put;
Action == anti_entropy ->
{insert, Hash}.
--spec gather_counts({p(), {p(), n()}, write_reason()}, yz_dict()) -> yz_dict().
-gather_counts({Index, IndexN, Reason}, StatsD) ->
- case Reason of
- {_, full_repair} ->
- dict:update_counter({Index, IndexN}, 1, StatsD);
- _ -> dict:update_counter({Index, IndexN}, 0, StatsD)
- end.
+-spec maybe_add_repair_to_counts({p(), short_preflist(), write_reason()}, yz_dict()) -> yz_dict().
+maybe_add_repair_to_counts({Index, ShortPL, {_, full_repair}}, StatsD) ->
+ dict:update_counter({Index, ShortPL}, 1, StatsD);
+maybe_add_repair_to_counts({_Index, _ShortPL, _Reason}, StatsD) ->
+ StatsD.
-spec get_reason_action(write_reason()) -> write_action().
get_reason_action(Reason) when is_tuple(Reason) ->
@@ -393,8 +412,10 @@ get_reason_action(Reason) when is_tuple(Reason) ->
get_reason_action(Reason) ->
Reason.
--spec prepare_ops_for_batch(solr_ops()) -> solr_entries().
-prepare_ops_for_batch(Ops) ->
+-define(QUERY(Bin), {struct, [{'query', Bin}]}).
+
+-spec prepare_ops_for_batch(index_name(), solr_ops()) -> solr_entries().
+prepare_ops_for_batch(_Index, Ops) ->
%% Flatten combined operators for a batch.
lists:flatten(Ops).
diff --git a/src/yz_solrq_queue_pair_sup.erl b/src/yz_solrq_queue_pair_sup.erl
new file mode 100644
index 00000000..142dda64
--- /dev/null
+++ b/src/yz_solrq_queue_pair_sup.erl
@@ -0,0 +1,92 @@
+%% -------------------------------------------------------------------
+%% Copyright (c) 2016 Basho Technologies, Inc. All Rights Reserved.
+%%
+%% This file is provided 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.
+%%
+%% -------------------------------------------------------------------
+-module(yz_solrq_queue_pair_sup).
+
+-include("yokozuna.hrl").
+
+-behaviour(supervisor).
+
+%% API
+-export([start_link/2]).
+
+%% Supervisor callbacks
+-export([init/1]).
+
+-define(SERVER, ?MODULE).
+
+%%%===================================================================
+%%% API functions
+%%%===================================================================
+
+%%--------------------------------------------------------------------
+%% @doc
+%% Starts the supervisor
+%%
+%% @end
+%%--------------------------------------------------------------------
+-spec(start_link(index_name(), p()) ->
+ {ok, Pid :: pid()} | {error, Reason :: term()}).
+start_link(Index, Partition) ->
+ supervisor:start_link(?MODULE, [Index, Partition]).
+
+%%%===================================================================
+%%% Supervisor callbacks
+%%%===================================================================
+
+%%--------------------------------------------------------------------
+%% @private
+%% @doc
+%% Whenever a supervisor is started using supervisor:start_link/[2,3],
+%% this function is called by the new process to find out about
+%% restart strategy, maximum restart frequency and child
+%% specifications.
+%%
+%% @end
+%%--------------------------------------------------------------------
+-spec(init(Args :: term()) ->
+ {ok, {SupFlags :: {RestartStrategy :: supervisor:strategy(),
+ MaxR :: non_neg_integer(), MaxT :: non_neg_integer()},
+ [ChildSpec :: supervisor:child_spec()]
+ }}).
+init([Index, Partition]) ->
+ RestartStrategy = one_for_all,
+ MaxRestarts = 10,
+ MaxSecondsBetweenRestarts = 10,
+
+ SupFlags = {RestartStrategy, MaxRestarts, MaxSecondsBetweenRestarts},
+
+ HelperChild = helper_spec(Index, Partition),
+ WorkerChild = worker_spec(Index, Partition),
+
+ {ok, {SupFlags, [HelperChild, WorkerChild]}}.
+
+%%%===================================================================
+%%% Internal functions
+%%%===================================================================
+
+
+helper_spec(Index, Partition) ->
+ child_spec(helper, yz_solrq_helper, Index, Partition).
+
+worker_spec(Index, Partition) ->
+ child_spec(worker, yz_solrq_worker, Index, Partition).
+
+child_spec(Type, Module, Index, Partition) ->
+ {{Type, Index, Partition}, {Module, start_link, [Index, Partition]},
+ permanent, 5000, worker, [Module]}.
diff --git a/src/yz_solrq_sup.erl b/src/yz_solrq_sup.erl
index 51635fcd..064ed12d 100644
--- a/src/yz_solrq_sup.erl
+++ b/src/yz_solrq_sup.erl
@@ -20,70 +20,125 @@
-behaviour(supervisor).
--export([start_link/0, start_link/2, start_drain_fsm/1, child_count/1]).
+-export([start_link/0, start_drain_fsm/1, child_count/1, start_queue_pair/2, active_queues/0, sync_active_queue_pairs/0]).
-include("yokozuna.hrl").
-export([init/1]).
-%% used only by yz_perf on resize (manual operation)
--export([child_spec/2]).
%%%===================================================================
%%% API functions
%%%===================================================================
start_link() ->
- start_link(worker_procs(), helper_procs()).
-
-start_link(NumQueues, NumHelpers) ->
- supervisor:start_link({local, ?MODULE}, ?MODULE, [NumQueues, NumHelpers]).
+ supervisor:start_link({local, ?MODULE}, ?MODULE, []).
%% @doc Start the drain FSM, under this supervisor
-spec start_drain_fsm(proplist()) -> {ok, pid()} | {error, Reason :: term()}.
-start_drain_fsm(CallbackList) ->
+start_drain_fsm(Parameters) ->
+ PartitionToDrain = proplists:get_value(
+ ?DRAIN_PARTITION, Parameters, undefined
+ ),
supervisor:start_child(
?MODULE,
- {yz_solrq_drain_fsm, {yz_solrq_drain_fsm, start_link, [CallbackList]}, temporary, 5000, worker, []}
+ {PartitionToDrain, {yz_solrq_drain_fsm, start_link, [Parameters]}, temporary, 5000, worker, []}
).
-
+-spec start_queue_pair(Index::index_name(), Partition::p()) -> ok.
+start_queue_pair(Index, Partition) ->
+ lager:info(
+ "Starting solrq supervisor for index ~p and partition ~p",
+ [Index, Partition]
+ ),
+ validate_child_started(
+ supervisor:start_child(?MODULE, queue_pair_spec({Index, Partition}))).
+
+-spec active_queues() -> [{index_name(), p()}].
+active_queues() ->
+ PairSups = find_pair_supervisors(),
+ PairChildren = lists:flatten([supervisor:which_children(Sup) || Sup <- PairSups]),
+ Workers = [{Index, Partition} ||
+ {{worker, Index, Partition}, _Child, _Type, Modules} <- PairChildren,
+ Modules == [yz_solrq_worker]],
+ Workers.
+
+-spec find_pair_supervisors() -> [PairSupPid::pid()].
+find_pair_supervisors() ->
+ AllChildren = supervisor:which_children(yz_solrq_sup),
+ PairSups = [SupPid ||
+ {_IndexPartition, SupPid, _Type, Modules} <- AllChildren,
+ Modules == [yz_solrq_queue_pair_sup]],
+ PairSups.
%%%===================================================================
%%% Supervisor callbacks
%%%===================================================================
-init([NumQueues, NumHelpers]) ->
- yz_solrq:set_solrq_worker_tuple(NumQueues),
- yz_solrq:set_solrq_helper_tuple(NumHelpers),
+init([]) ->
+
DrainMgrSpec = {yz_solrq_drain_mgr, {yz_solrq_drain_mgr, start_link, []}, permanent, 5000, worker, [yz_drain_mgr]},
- QueueChildren = [child_spec(worker, Name) ||
- Name <- tuple_to_list(yz_solrq:get_solrq_worker_tuple())],
- HelperChildren = [child_spec(helper, Name) ||
- Name <- tuple_to_list(yz_solrq:get_solrq_helper_tuple())],
- %% Using a one_for_all restart strategy as we write data to a hashed worker,
- %% which then uses a random helper to send data to Solr itself - if we want to
- %% make this one_for_one we will need to do more work monitoring the processes
- %% and responding to crashes more carefully.
- {ok, {{one_for_all, 10, 10}, [DrainMgrSpec | HelperChildren ++ QueueChildren]}}.
+
+ QueuePairSupervisors = [queue_pair_spec(IndexPartition) ||
+ IndexPartition <- required_queues()],
+ {ok, {{one_for_one, 10, 10}, [DrainMgrSpec | QueuePairSupervisors]}}.
%%%===================================================================
%%% Internal functions
%%%===================================================================
-worker_procs() ->
- application:get_env(?YZ_APP_NAME, ?SOLRQ_WORKER_COUNT, 10).
-
-helper_procs() ->
- application:get_env(?YZ_APP_NAME, ?SOLRQ_HELPER_COUNT, 10).
-
-child_spec(helper, Name) ->
- {Name, {yz_solrq_helper, start_link, [Name]}, permanent, 5000, worker, [yz_solrq_helper]};
-
-child_spec(worker, Name) ->
- {Name, {yz_solrq_worker, start_link, [Name]}, permanent, 5000, worker, [yz_solrq_worker]}.
-
-spec child_count(atom()) -> non_neg_integer().
child_count(ChildType) ->
length([true || {_,_,_,[Type]} <- supervisor:which_children(?MODULE),
- Type == ChildType]).
\ No newline at end of file
+ Type == ChildType]).
+
+validate_child_started({ok, _Child}) ->
+ ok;
+validate_child_started({ok, _Child, _Info}) ->
+ ok;
+validate_child_started({error, already_present}) ->
+ ok;
+validate_child_started({error, {already_started, _Child}}) ->
+ ok;
+validate_child_started(Error) ->
+ throw(Error).
+
+required_queues() ->
+ AllVnodes = riak_core_vnode_manager:all_vnodes(riak_kv_vnode),
+ Partitions = [Idx || {_Mod, Idx, _Pid} <- AllVnodes],
+ %% Indexes includes ?YZ_INDEX_TOMBSTONE because we need to write the entries
+ %% for non-indexed data to the YZ AAE tree. Excluding them makes this process
+ %% constantly start and stop these queues.
+ Indexes = yz_index:get_indexes_from_meta() ++ [?YZ_INDEX_TOMBSTONE],
+ CalculatedQueues = [{Index, Partition} ||
+ Partition <- Partitions,
+ Index <- Indexes],
+ CalculatedQueues.
+ %% TODO: we shouldn't need ?YZ_INDEX_TOMBSTONE if we just update the YZ AAE tree
+ %% when we call index rather than pushing the value all the way to the solrq
+
+sync_active_queue_pairs() ->
+ ActiveQueues = active_queues(),
+ RequiredQueues = required_queues(),
+ QueuePairsToStop = ActiveQueues -- RequiredQueues,
+ lists:foreach(fun({Index, Partition}) -> stop_queue_pair(Index, Partition) end, QueuePairsToStop),
+ MissingWorkers = RequiredQueues -- ActiveQueues,
+ lists:foreach(fun({Index, Partition}) -> start_queue_pair(Index, Partition) end, MissingWorkers),
+ ok.
+
+stop_queue_pair(Index, Partition) ->
+ lager:info(
+ "Stopping solrq supervisor for index ~p and partition ~p",
+ [Index, Partition]
+ ),
+ SupId = {Index, Partition},
+ case supervisor:terminate_child(?MODULE, SupId) of
+ ok ->
+ _ = supervisor:delete_child(?MODULE, SupId);
+ _ ->
+ ok
+ end.
+
+queue_pair_spec({Index, Partition} = Id) ->
+ Id = {Index, Partition},
+ {Id, {yz_solrq_queue_pair_sup, start_link, [Index, Partition]}, permanent, 5000, supervisor, [yz_solrq_queue_pair_sup]}.
diff --git a/src/yz_solrq_worker.erl b/src/yz_solrq_worker.erl
index b36354a9..9e418d61 100644
--- a/src/yz_solrq_worker.erl
+++ b/src/yz_solrq_worker.erl
@@ -23,35 +23,41 @@
-behavior(gen_server).
%% api
--export([start_link/1, status/1, index/6, set_hwm/2, get_hwm/1, set_index/5,
- reload_appenv/1, blown_fuse/2, healed_fuse/2, cancel_drain/1,
- all_queue_len/1, set_purge_strategy/2]).
+-export([start_link/2, status/1, index/5, set_hwm/3, get_hwm/1, set_index/5,
+ reload_appenv/2, blown_fuse/1, healed_fuse/1, cancel_drain/1,
+ all_queue_len/2, set_purge_strategy/3, stop/2]).
%% gen_server callbacks
-export([init/1, handle_call/3, handle_cast/2, handle_info/2,
terminate/2, code_change/3]).
-% solrq/helper interface
--export([request_batch/3, drain/2, drain_complete/1, batch_complete/3]).
+%% solrq/helper interface
+-export([batch_complete/2]).
+
+%% drain manager interface
+-export([drain/2, drain_complete/1]).
%% TODO: Dynamically pulse_instrument. See test/pulseh.erl
--ifdef(PULSE).
--compile(export_all).
--compile({parse_transform, pulse_instrument}).
--compile({pulse_replace_module, [{gen_server, pulse_gen_server}]}).
--define(PULSE_DEBUG(S,F), pulse:format(S,F)).
+-ifdef(EQC).
+-define(EQC_DEBUG(S, F), ok).
+%% -define(EQC_DEBUG(S, F), eqc:format(S, F)).
-else.
--define(PULSE_DEBUG(S,F), ok).
+-define(EQC_DEBUG(S, F), ok).
-endif.
+-define(COUNT_PER_REPORT, 20).
-type solrq_message() :: tuple(). % {BKey, Docs, Reason, P}.
--type pending_vnodes() :: [{pid(), atom()}].
--type drain_info() :: {pid(), reference(), [solrq_message()]} | undefined.
--type indexq_status() ::
- {queue, yz_queue()}
+-type pending_processes() :: {pid(), atom()}.
+-type drain_info() :: {pid(), Token::reference(), DrainFSMMonitor::reference()} | undefined.
+-type worker_state_status() ::
+ {all_queue_len, non_neg_integer()}
+ | {queue_hwm, non_neg_integer()}
+ | {pending_processes, pending_processes()}
+ | {drain_info, drain_info()}
+ | {purge_strategy, purge_strategy()
+ | {queue, yz_queue()}
| {queue_len, non_neg_integer()}
| {timer_ref, reference() | undefined}
- | {pending_helper, boolean()}
| {batch_min, solrq_batch_min()}
| {batch_max, solrq_batch_max()}
| {delayms_max, solrq_batch_flush_interval()}
@@ -59,44 +65,33 @@
| {draining, boolean() | wait_for_drain_complete}
| {fuse_blown, boolean()}
| {in_flight_len, non_neg_integer()}
- | {batch_start, timestamp() | undefined}.
--type worker_state_status() ::
- {all_queue_len, non_neg_integer()}
- | {queue_hwm, non_neg_integer()}
- | {pending_vnodes, pending_vnodes()}
- | {drain_info, drain_info()}
- | {purge_strategy, purge_strategy()}
- | {indexqs, [indexq_status()]}.
+ | {batch_start, timestamp() | undefined}}.
-type status() :: [worker_state_status()].
-export_type([status/0]).
--record(
- indexq, {
- queue = queue:new() :: yz_queue(), % solrq_message()
- queue_len = 0 :: non_neg_integer(),
- timer_ref = undefined :: reference()|undefined,
- pending_helper = false :: boolean(),
- batch_min = 1 :: solrq_batch_min(),
- batch_max = 100 :: solrq_batch_max(),
- delayms_max = 1000 :: solrq_batch_flush_interval(),
- aux_queue = queue:new() :: yz_queue(),
- draining = false :: boolean() | wait_for_drain_complete,
- fuse_blown = false :: boolean(),
- in_flight_len = 0 :: non_neg_integer(),
- batch_start :: timestamp() | undefined
- }
-).
-
-record(
state, {
- indexqs = dict:new() :: yz_dict(),
- all_queue_len = 0 :: non_neg_integer(),
- queue_hwm = 1000 :: non_neg_integer(),
- pending_vnodes = [] :: pending_vnodes(),
- drain_info = undefined :: drain_info(),
- purge_strategy :: purge_strategy()
- }
-).
+ index :: index_name(),
+ partition :: p(),
+ queue_hwm = 1000 :: non_neg_integer(),
+ %% Both the vnode and the yz_exchange_fsm can call into `index`.
+ %% Therefore, we need a list of processes, not just a single vnode PID
+ pending_processes = [] :: [pending_processes()],
+ drain_info = undefined :: drain_info(),
+ purge_strategy :: purge_strategy(),
+ helper_pid = undefined :: pid() | undefined,
+ queue = queue:new() :: yz_queue(), % solrq_message()
+ flush_timer_ref = undefined :: reference()|undefined,
+ batch_min = yz_solrq:get_min_batch_size() :: solrq_batch_min(),
+ batch_max = yz_solrq:get_max_batch_size() :: solrq_batch_max(),
+ delayms_max = yz_solrq:get_flush_interval() :: solrq_batch_flush_interval(),
+ aux_queue = queue:new() :: yz_queue(),
+ draining = false :: boolean() | wait_for_drain_complete,
+ fuse_blown = false :: boolean(),
+ in_flight_len = 0 :: non_neg_integer(),
+ batch_start :: timestamp() | undefined
+ }).
+-type state() :: #state{}.
%% `record_info' only exists during compilation, so we'll continue to leave
@@ -108,8 +103,9 @@
%%% API
%%%===================================================================
-start_link(Name) ->
- gen_server:start_link({local, Name}, ?MODULE, [], []).
+start_link(Index, Partition) ->
+ Name = yz_solrq:worker_regname(Index, Partition),
+ gen_server:start_link({local, Name}, ?MODULE, [Index, Partition], []).
-spec status(solrq_id()) -> status().
status(QPid) ->
@@ -119,42 +115,45 @@ status(QPid) ->
status(QPid, Timeout) ->
gen_server:call(QPid, status, Timeout).
--spec index(solrq_id(), index_name(), bkey(), obj(), write_reason(), p()) -> ok.
-index(QPid, Index, BKey, Obj, Reason, P) ->
- gen_server:call(QPid, {index, Index, {BKey, Obj, Reason, P}}, infinity).
+-spec index(solrq_id(), bkey(), object_pair(), write_reason(), p()) -> ok.
+index(QPid, BKey, Objects, Reason, P) ->
+ gen_server:call(QPid, {index, {BKey, Objects, Reason, P}}, infinity).
--spec set_hwm(solrq_id(), HWM :: solrq_hwm()) ->
+-spec set_hwm(Index :: index_name(), Partition :: p(), HWM :: solrq_hwm()) ->
{ok, OldHWM :: solrq_hwm()} | {error, bad_hwm_value}.
-set_hwm(QPid, HWM) when HWM >= 0 ->
- gen_server:call(QPid, {set_hwm, HWM});
-set_hwm(_, _) ->
+set_hwm(Index, Partition, HWM) when HWM >= 0 ->
+ gen_server:call(yz_solrq:worker_regname(Index, Partition), {set_hwm, HWM});
+set_hwm(_, _, _) ->
{error, bad_hwm_value}.
--spec set_index(solrq_id(), index_name(), solrq_batch_min(), solrq_batch_max(),
+-spec set_index(index_name(), p(), solrq_batch_min(), solrq_batch_max(),
solrq_batch_flush_interval()) ->
{ok, {solrq_batch_min(),
solrq_batch_max(),
solrq_batch_flush_interval()}}.
-set_index(QPid, Index, Min, Max, DelayMax)
+set_index(Index, Partition, Min, Max, DelayMax)
when Min > 0, Min =< Max, DelayMax >= 0 orelse DelayMax == infinity ->
- gen_server:call(QPid, {set_index, Index, Min, Max, DelayMax});
+ gen_server:call(yz_solrq:worker_regname(Index, Partition), {set_index, Min, Max, DelayMax});
set_index(_, _, _, _, _) ->
{error, bad_index_params}.
--spec set_purge_strategy(solrq_id(), purge_strategy()) ->
+-spec set_purge_strategy(index_name(), p(), purge_strategy()) ->
{ok, OldPurgeStrategy :: purge_strategy()} | {error, bad_purge_strategy}.
-set_purge_strategy(QPid, PurgeStrategy)
+set_purge_strategy(Index, Partition, PurgeStrategy)
when PurgeStrategy == ?PURGE_NONE
orelse PurgeStrategy == ?PURGE_ONE
- orelse PurgeStrategy == ?PURGE_IDX
- orelse PurgeStrategy == ?PURGE_ALL ->
- gen_server:call(QPid, {set_purge_strategy, PurgeStrategy});
-set_purge_strategy(_QPid, _PurgeStrategy) ->
+ orelse PurgeStrategy == ?PURGE_IDX ->
+ gen_server:call(yz_solrq:worker_regname(Index, Partition), {set_purge_strategy, PurgeStrategy});
+set_purge_strategy(_Index, _Partition, _PurgeStrategy) ->
{error, bad_purge_strategy}.
--spec reload_appenv(solrq_id()) -> ok.
-reload_appenv(QPid) ->
- gen_server:call(QPid, reload_appenv).
+-spec stop(Index::index_name(), Partition::p()) -> any().
+stop(Index, Partition) ->
+ gen_server:cast(yz_solrq:worker_regname(Index, Partition), stop).
+
+-spec reload_appenv(index_name(), p()) -> ok.
+reload_appenv(Index, Partition) ->
+ gen_server:call(yz_solrq:worker_regname(Index, Partition), reload_appenv).
-spec drain(solrq_id(), p() | undefined) -> reference().
drain(QPid, Partition) ->
@@ -171,14 +170,14 @@ cancel_drain(QPid) ->
gen_server:call(QPid, cancel_drain).
%% @doc Signal to the solrq that a fuse has blown for the the specified index.
--spec blown_fuse(solrq_id(), index_name()) -> ok.
-blown_fuse(QPid, Index) ->
- gen_server:cast(QPid, {blown_fuse, Index}).
+-spec blown_fuse(solrq_id()) -> ok.
+blown_fuse(QPid) ->
+ gen_server:cast(QPid, blown_fuse).
%% @doc Signal to the solrq that a fuse has healed for the the specified index.
--spec healed_fuse(solrq_id(), index_name()) -> ok.
-healed_fuse(QPid, Index) ->
- gen_server:cast(QPid, {healed_fuse, Index}).
+-spec healed_fuse(solrq_id()) -> ok.
+healed_fuse(QPid) ->
+ gen_server:cast(QPid, healed_fuse).
%% @doc return the sum of the length of all queues in each indexq
-spec get_hwm(solrq_id()) -> HWM :: non_neg_integer().
@@ -186,74 +185,71 @@ get_hwm(QPid) ->
gen_server:call(QPid, get_hwm).
%% @doc return the sum of the length of all queues in each indexq
--spec all_queue_len(solrq_id()) -> non_neg_integer().
-all_queue_len(QPid) ->
- gen_server:call(QPid, all_queue_len).
+-spec all_queue_len(Index:: index_name(), Partition::p()) -> non_neg_integer().
+all_queue_len(Index, Partition) ->
+ gen_server:call(yz_solrq:worker_regname(Index, Partition), all_queue_len).
%%%===================================================================
%%% solrq/helper interface
%%%===================================================================
--spec request_batch(solrq_id(), index_name(), solrq_helper_id()) -> ok.
-request_batch(QPid, Index, HPid) ->
- gen_server:cast(QPid, {request_batch, Index, HPid}).
-
-spec batch_complete(
solrq_id(),
- index_name(),
{NumDelivered :: non_neg_integer(), ok} |
{NumDelivered :: non_neg_integer(),
{retry, [Undelivered :: solrq_message()]}}) -> ok.
-batch_complete(QPid, Index, Message) ->
- gen_server:cast(QPid, {batch_complete, Index, Message}).
+batch_complete(QPid, Message) ->
+ gen_server:cast(QPid, {batch_complete, Message}).
%%%===================================================================
%%% gen_server callbacks
%%%===================================================================
%% @private
-init([]) ->
- {ok, read_appenv(#state{})} .
-
-handle_call({index, Index, E}, From, State) ->
- ?PULSE_DEBUG("index. State: ~p~n", [debug_state(State)]),
- State2 = inc_qlen_and_maybe_unblock_vnode(From, State),
- IndexQ = enqueue(E, get_indexq(Index, State2)),
- IndexQ2 = maybe_request_worker(Index, IndexQ),
- IndexQ3 = maybe_start_timer(Index, IndexQ2),
- NewState = update_indexq(Index, IndexQ3, State2),
- ?PULSE_DEBUG("index. NewState: ~p~n", [debug_state(NewState)]),
- {noreply, NewState};
+init([Index, Partition]) ->
+ State0 = read_appenv(#state{index = Index, partition = Partition}),
+ State1 = get_helper(Index, Partition, State0),
+ {ok, State1}.
+
+handle_call({index, E}, From, State0) ->
+ ?EQC_DEBUG("index. State: ~p~n", [debug_state(State0)]),
+ State1 = maybe_purge(State0),
+ State2 = maybe_send_reply(From, State1),
+ State3 = enqueue(E, State2),
+ State4 = maybe_send_batch_to_helper(State3),
+ FinalState = maybe_start_flush_timer(State4),
+ ?EQC_DEBUG("index. NewState: ~p~n", [debug_state(FinalState)]),
+ {noreply, FinalState};
handle_call(status, _From, #state{} = State) ->
{reply, internal_status(State), State};
handle_call({set_hwm, NewHWM}, _From, #state{queue_hwm = OldHWM} = State) ->
- {reply, {ok, OldHWM}, maybe_unblock_vnodes(State#state{queue_hwm = NewHWM})};
+ {reply, {ok, OldHWM}, maybe_unblock_processes(State#state{queue_hwm = NewHWM})};
handle_call(get_hwm, _From, #state{queue_hwm = HWM} = State) ->
{reply, HWM, State};
-handle_call({set_index, Index, Min, Max, DelayMS}, _From, State) ->
- IndexQ = get_indexq(Index, State),
- IndexQ2 = maybe_request_worker(Index,
- IndexQ#indexq{batch_min = Min,
- batch_max = Max,
- delayms_max = DelayMS}),
- OldParams = {IndexQ#indexq.batch_min,
- IndexQ#indexq.batch_max,
- IndexQ#indexq.delayms_max},
- {reply, {ok, OldParams}, update_indexq(Index, IndexQ2, State)};
+handle_call({set_index, Min, Max, DelayMS}, _From, State0) ->
+ State1 = State0#state{batch_min = Min,
+ batch_max = Max,
+ delayms_max = DelayMS},
+ State2 = maybe_send_batch_to_helper(State1),
+ OldParams = {State2#state.batch_min,
+ State2#state.batch_max,
+ State2#state.delayms_max},
+ {reply, {ok, OldParams}, State2};
handle_call({set_purge_strategy, NewPurgeStrategy},
_From,
#state{purge_strategy=OldPurgeStrategy} = State) ->
{reply, {ok, OldPurgeStrategy}, State#state{purge_strategy=NewPurgeStrategy}};
handle_call(cancel_drain, _From, State) ->
- {noreply, NewState} = handle_cast(drain_complete, State),
+ NewState = stop_draining(State),
{reply, ok, NewState};
-handle_call(all_queue_len, _From, #state{all_queue_len=Len} = State) ->
+handle_call(all_queue_len, _From, #state{queue = Queue} = State) ->
+ Len = queue:len(Queue),
{reply, Len, State};
handle_call(reload_appenv, _From, State) ->
{reply, ok, read_appenv(State)}.
-handle_cast({request_batch, Index, HPid}, State) ->
- State2 = send_entries(HPid, Index, State),
- {noreply, State2};
+
+handle_cast(stop, State) ->
+ {stop, normal, State};
%%
%% @doc Handle the drain message.
@@ -263,59 +259,53 @@ handle_cast({request_batch, Index, HPid}, State) ->
%% during its prepare state, typically as the result of
%% a request to drain the queues.
%%
-%% This handler will iterate over all IndexQs in the
-%% solrq, and initiate a drain on the queue, if it is currently
+%% This handler initiate a drain on the queue, if it is currently
%% non-empty.
%% @end
%%
-handle_cast({drain, DPid, Token, Partition}, #state{indexqs = IndexQs} = State) ->
- ?PULSE_DEBUG("drain{~p=DPid, ~p=Token, ~p=Partition}. State: ~p~n", [debug_state(State), DPid, Token, Partition]),
- {Remaining, NewIndexQs} = dict:fold(
- fun(Index, IndexQ0, {RemainingAccum, IndexQsAccum}) ->
- IndexQ = partition(IndexQ0, Partition),
- case {IndexQ#indexq.queue_len, IndexQ#indexq.in_flight_len} of
- {0, 0} ->
- {RemainingAccum, dict:store(Index, IndexQ#indexq{draining = wait_for_drain_complete}, IndexQsAccum)};
- {0, _InFlightLen} ->
- {[Index | RemainingAccum], dict:store(Index, IndexQ#indexq{draining = true}, IndexQsAccum)};
- _ ->
- {[Index | RemainingAccum], drain(Index, IndexQ, IndexQsAccum)}
- end
- end,
- {[], IndexQs},
- IndexQs
- ),
- NewState =
- case Remaining of
- [] ->
- yz_solrq_drain_fsm:drain_complete(DPid, Token),
- State;
- _ ->
- State#state{indexqs = NewIndexQs, drain_info = {DPid, Token, Remaining}}
-
- end,
- ?PULSE_DEBUG("drain. NewState: ~p~n", [debug_state(NewState)]),
+handle_cast({drain, DPid, Token, _TargetPartition},
+ #state{draining = Draining}=State) when Draining =/= false ->
+ %% Drain already in progress - notify caller and continue
+ yz_solrq_drain_fsm:drain_already_in_progress(DPid, Token),
+ {noreply, State};
+
+handle_cast({drain, DPid, Token, TargetPartition},
+ #state{queue = Queue,
+ in_flight_len = InFlightLen,
+ partition = QueueParitition} = State)
+ when TargetPartition == undefined; TargetPartition == QueueParitition ->
+ ?EQC_DEBUG("drain{~p=DPid, ~p=Token, ~p=Partition}. State: ~p~n",
+ [DPid, Token, TargetPartition, internal_status(State)]),
+ NewState0 = monitor_draining_process(DPid, State, Token),
+ NewState = maybe_drain_queue(Queue, InFlightLen, NewState0),
+ maybe_send_drain_complete(NewState#state.draining, DPid, Token),
+ ?EQC_DEBUG("drain. NewState: ~p~n", [internal_status(NewState)]),
{noreply, NewState};
+%% Totally ignore drain if it's not our partition
+handle_cast({drain, DPid, Token, _Partition}, State) ->
+ yz_solrq_drain_fsm:drain_complete(DPid, Token),
+ {noreply, State};
+
%% @doc The fuse for the specified index has blown.
-handle_cast({blown_fuse, Index}, State) ->
- {noreply, handle_blown_fuse(Index, State)};
+handle_cast(blown_fuse, State) ->
+ {noreply, handle_blown_fuse(State)};
%%
%% @doc Clear the fuse_blown state on the IndexQ associated with the supplied Index
%% Resume any batches that may need to proceed.
%% @end
%%
-handle_cast({healed_fuse, Index}, State) ->
- IndexQ = get_indexq(Index, State),
- NewIndexQ = maybe_request_worker(Index, maybe_start_timer(Index, IndexQ#indexq{fuse_blown = false})),
- {noreply, update_indexq(Index, NewIndexQ, State)};
+handle_cast(healed_fuse, #state{} = State) ->
+ State1 = maybe_start_flush_timer(State#state{fuse_blown = false}),
+ State2 = maybe_send_batch_to_helper(State1),
+ {noreply, State2};
%%
%% @doc Handle the batch_complete message.
%%
-%% The batch_complete message is sent via the batch_complete/3 function
+%% The batch_complete message is sent via the batch_complete/2 function
%% in this module, which is called by a solrq_helper when a batch has
%% been delivered to Solr (or has failed to have been delivered).
%% If the batch was successfully sent to Solr, the Result field of the
@@ -326,17 +316,19 @@ handle_cast({healed_fuse, Index}, State) ->
%% that have not been delivered (a subset of what was sent for delivery)
%% and should be retried. This handler will decrement the all_queues_len
%% field on the solrq state record by the supplied NumDelievered value
-%% thus potentially unblocking any vnodes waiting on this solrq instance,
+%% thus potentially unblocking any processes waiting on this solrq instance,
%% if the number of queued messages are above the high water mark.
%% @end
%%
-handle_cast({batch_complete, Index, {NumDelivered, Result}}, #state{all_queue_len = AQL} = State) ->
- ?PULSE_DEBUG("batch_complete. State: ~p~n", [debug_state(State)]),
- IndexQ = get_indexq(Index, State),
- State1 = handle_batch(Index, IndexQ#indexq{pending_helper = false, in_flight_len = 0}, Result, State),
- NewState = maybe_unblock_vnodes(State1#state{all_queue_len = AQL - NumDelivered}),
- ?PULSE_DEBUG("batch_complete. NewState: ~p~n", [debug_state(NewState)]),
- {noreply, NewState};
+handle_cast({batch_complete, {_NumDelivered, Result}},
+ #state{} = State) ->
+ ?EQC_DEBUG("batch_complete. State: ~p~n", [debug_state(State)]),
+ State1 = handle_batch(Result, State#state{in_flight_len = 0}),
+ State2 = maybe_unblock_processes(State1),
+ State3 = maybe_send_batch_to_helper(State2),
+ State4 = maybe_start_flush_timer(State3),
+ ?EQC_DEBUG("batch_complete. NewState: ~p~n", [debug_state(State4)]),
+ {noreply, State4};
%%
%% @doc Handle the drain_complete message.
@@ -347,40 +339,55 @@ handle_cast({batch_complete, Index, {NumDelivered, Result}}, #state{all_queue_le
%% flag to false (finally), and kick-starting the indexq.
%% @end
%%
-handle_cast(drain_complete, #state{indexqs = IndexQs} = State) ->
- NewIndexQs = dict:fold(
- fun(Index, #indexq{queue = Queue, queue_len = QueueLen, aux_queue = AuxQueue} = IndexQ, IndexQAccum) ->
- NewIndexQ = IndexQ#indexq{
- queue = queue:join(Queue, AuxQueue),
- queue_len = QueueLen + queue:len(AuxQueue),
- aux_queue = queue:new(),
- draining = false
- },
- dict:store(Index, maybe_start_timer(Index, maybe_request_worker(Index, NewIndexQ)), IndexQAccum)
- end,
- dict:new(),
- IndexQs
- ),
- {noreply, State#state{indexqs = NewIndexQs, drain_info = undefined}}.
-
-
-%% @doc Timer has fired - request a worker.
-handle_info({timeout, _TimerRef, {flush, Index}}, State) ->
- case find_indexq(Index, State) of
- undefined ->
- {noreply, State};
- IndexQ ->
- case timer_running(IndexQ) of
- true ->
- {noreply,
- flush(Index,
- IndexQ#indexq{timer_ref = undefined},
- State)};
- false -> % out of date
- {noreply, State}
- end
- end;
-handle_info(_Msg, State) ->
+handle_cast(drain_complete, State) ->
+ State1 = stop_draining(State),
+ {noreply, State1}.
+
+monitor_draining_process(DPid, State, Token) ->
+ MonitorRef = erlang:monitor(process, DPid),
+ State#state{drain_info = {DPid, Token, MonitorRef}}.
+
+maybe_drain_queue(Queue, InFlightLen, State) ->
+ case {queue:is_empty(Queue), InFlightLen} of
+ {true, 0} ->
+ %% If our queue is empty, and we have no in-flight messages,
+ %% skip the `draining=true` state and go
+ %% straight to `wait_for_drain_complete`
+ State#state{draining = wait_for_drain_complete};
+ _ ->
+ drain_queue(State)
+ end.
+
+stop_draining(#state{queue = Queue,
+ aux_queue = AuxQueue,
+ drain_info = DrainInfo} = State) ->
+ maybe_demonitor_draining_process(DrainInfo),
+ State1 = State#state{
+ queue = queue:join(Queue, AuxQueue),
+ aux_queue = queue:new(),
+ draining = false,
+ drain_info=undefined},
+ State2 = maybe_send_batch_to_helper(State1),
+ maybe_start_flush_timer(State2).
+
+
+%% @doc Timer has fired - request a helper.
+handle_info({timeout, TimerRef, flush},
+ #state{flush_timer_ref = TimerRef} = State) ->
+ {noreply, flush(State#state{flush_timer_ref = undefined})};
+
+handle_info({timeout, _TimerRef, flush}, State) ->
+ lager:debug("Received timeout from stale Timer Reference"),
+ {noreply, State};
+
+handle_info({'DOWN', MonitorRef, process, DPid, Info},
+ #state{draining=Draining, drain_info = {DPid, _Token, MonitorRef}} = State)
+ when Draining =:= true; Draining =:= wait_for_drain_complete ->
+ lager:info("Drain FSM terminated for reason ~p without notifying of drain_complete. Resuming normal operations.", [Info]),
+ NewState = stop_draining(State),
+ {noreply, NewState};
+
+handle_info({'DOWN', _MonitorRef, process, _DPid, _Info}, State) ->
{noreply, State}.
terminate(_Reason, _State) ->
@@ -393,110 +400,57 @@ code_change(_OldVsn, State, _Extra) ->
%%% Private
%%%===================================================================
-%% @doc Check if timer (erlang:start_timer/3) ref is defined.
--spec timer_running(#indexq{}) -> boolean().
-timer_running(#indexq{timer_ref = undefined}) -> false;
-timer_running(_) -> true.
+%%
+%% @doc
+%% When we have to retry it does not matter what draining state we are in,
+%% we simply have to requeue and continue.
+%% end
+%%
+handle_batch({retry, Undelivered}, State) ->
+ State1 = requeue_undelivered(Undelivered, State),
+ State1;
%%
%% @doc
-%% A drain is not in progress. There are two cases to consider:
-%% 1. The result is ok; This means all batched messages were delivered.
-%% 2. The solrq_helper returned some undelivered messages; pre-pend these to the queue
-%% for this index, and request a new worker, if we are over the requested minimum.
+%% A drain is not in progress and the result is ok; This means all batched messages were delivered.
%% @end
%%
-handle_batch(
- Index,
- #indexq{draining = false, batch_start = T1} = IndexQ0,
- Result,
- State
-) ->
- IndexQ1 =
- case Result of
- ok ->
- yz_stat:batch_end(?YZ_TIME_ELAPSED(T1)),
- IndexQ0#indexq{batch_start = undefined};
- {retry, Undelivered} ->
- requeue_undelivered(Undelivered, IndexQ0)
- end,
- IndexQ2 = maybe_request_worker(Index, IndexQ1),
- IndexQ3 = maybe_start_timer(Index, IndexQ2),
- update_indexq(Index, IndexQ3, State);
+handle_batch(ok, #state{draining = false, batch_start = T1} = State) ->
+ yz_stat:batch_end(?YZ_TIME_ELAPSED(T1)),
+ State1 = State#state{batch_start = undefined},
+ State1;
%%
%% @doc
-%% This queue is being drained. There are two cases to consider:
-%% 1. The batch succeeded. If there is nothing left in the queue,
-%% then remove ourselves from the list
-%% of remaining indices that need to be flushed, and
-%% wait for t a drain_complete message to come back from the
-%% drain FSM.
-%% 2. The batch did not succeed. In this case, we got back a list
-%% of undelivered messages. Put the undelivered messages back onto
-%% the queue and request another worker.
-%%
-%% If there are no remaining indexqs to drain, then send the drain FSM
-%% a drain_complete message for this solrq.
+%% This queue is being drained and the batch succeeded.
+%% If there is nothing left in the queue,
+%% wait for the drain_complete message to come back from the
+%% drain FSM.
%% @end
%%
-handle_batch(
- Index,
- #indexq{queue_len = QueueLen, draining = _Draining, batch_start = T1} = IndexQ,
- Result,
- #state{drain_info = DrainInfo} = State) ->
- {DPid, Token, Remaining} = DrainInfo,
- {NewRemaining, NewIndexQ} =
- case Result of
- ok ->
- case QueueLen of
- 0 ->
- yz_stat:batch_end(?YZ_TIME_ELAPSED(T1)),
- {
- lists:delete(Index, Remaining),
- IndexQ#indexq{
- draining = wait_for_drain_complete,
- batch_start = undefined
- }
- };
- _ ->
- {Remaining, request_worker(Index, IndexQ)}
- end;
- {retry, Undelivered} ->
- {Remaining, requeue_undelivered(Undelivered, request_worker(Index, IndexQ))}
- end,
- %%
- %% If there are no remaining indexqs to be flushed, send the drain FSM
- %% a drain complete for this solrq instance.
- %%
- case NewRemaining of
- [] ->
- yz_solrq_drain_fsm:drain_complete(DPid, Token);
- _ ->
- ok
- end,
- update_indexq(
- Index,
- maybe_start_timer(Index, NewIndexQ),
- State#state{drain_info = {DPid, Token, NewRemaining}}
- ).
-
-partition(IndexQ, undefined) ->
- IndexQ;
-partition(IndexQ, Partition) ->
- {NewQueue, NewAuxQueue} = lists:partition(
- fun({_BKey, _Obj, _Reason, P}) ->
- P =:= Partition
- end,
- queue:to_list(IndexQ#indexq.queue)
- ),
- IndexQ#indexq{
- queue = queue:from_list(NewQueue),
- queue_len = length(NewQueue),
- aux_queue = queue:join(queue:from_list(NewAuxQueue), IndexQ#indexq.aux_queue)
- }.
-
-drain(Index, IndexQ, IndexQs) ->
+handle_batch(ok, #state{draining = true,
+ queue = Queue}=State) ->
+ case queue:is_empty(Queue) of
+ true ->
+ #state{drain_info={DPid, Token, _MonitorRef}, batch_start = T1} = State,
+ yz_stat:batch_end(?YZ_TIME_ELAPSED(T1)),
+ yz_solrq_drain_fsm:drain_complete(DPid, Token),
+ State#state{draining = wait_for_drain_complete,
+ batch_start = undefined };
+ false ->
+ State
+ end;
+
+%%
+%% @doc
+%% We are waiting for a drain_complete message to come from the
+%% yz_solrq_drain_fsm. Do nothing as we had already emptied
+%% our queue.
+handle_batch(ok, #state{draining = wait_for_drain_complete} = State) ->
+ lager:debug("DBG: Received a batch_complete message while waiting for drain complete"),
+ State.
+
+drain_queue(#state{in_flight_len = 0} = State) ->
%% NB. A drain request may occur while a helper is pending
%% (and hence while a batch may be "in flight" to Solr). If
%% so, then no worker will be requested. However, the draining
@@ -506,106 +460,92 @@ drain(Index, IndexQ, IndexQs) ->
%% will remain in the draining state, the result being that the
%% indexq will eventually get drained, once the current in-flight
%% batch completes.
- IndexQ2 = request_worker(Index, IndexQ),
- dict:store(Index, IndexQ2#indexq{draining = true}, IndexQs).
-
--spec internal_status(#state{}) -> status().
-internal_status(#state{indexqs = IndexQs} = State) ->
- [{F, V} || {F, V} <- ?REC_PAIRS(state, State), F /= indexqs] ++
- [{indexqs,
- lists:sort(
- dict:fold(
- fun(Index, IndexQ, Acc) ->
- [{Index, [{F, V} || {F, V} <- ?REC_PAIRS(indexq, IndexQ),
- F /= queue]} | Acc]
- end, [], IndexQs))}].
-
-%% @doc Increment the aggregated queue length and see if the vnode needs to be
-%% stalled.
-inc_qlen_and_maybe_unblock_vnode(From, #state{all_queue_len = AQL,
- pending_vnodes = PendingVnodes}
- = State) ->
- State2 = State#state{all_queue_len = AQL + 1},
- case over_hwm(State2) of
+ State2 = send_batch_to_helper(State),
+ State2#state{draining = true};
+drain_queue(State) ->
+ State#state{draining = true}.
+
+%% @todo: after collapsing the records we might want to reconsider how we do this formatting...
+-spec internal_status(state()) -> status().
+internal_status(#state{queue = Queue, aux_queue = AuxQueue} = State) ->
+ [{F, V} || {F, V} <- ?REC_PAIRS(state, State), F /= queue, F /= aux_queue]
+ ++ [{queue_len, queue:len(Queue)}, {aux_queue_len, queue:len(AuxQueue)}].
+
+%% @doc Check HWM, if we are not over it send a reply.
+maybe_send_reply(From, #state{pending_processes = PendingProcesses} = State) ->
+ case over_hwm(State) of
true ->
- State3 = maybe_purge_blown_indices(State2),
- case over_hwm(State3) of
- true ->
- yz_stat:blocked_vnode(From),
- State3#state{pending_vnodes = [From | PendingVnodes]};
- false ->
- gen_server:reply(From, ok),
- State3
- end;
+ log_blocked_process(From, State),
+ yz_stat:blocked_vnode(From),
+ State#state{pending_processes = [From | PendingProcesses] };
false ->
gen_server:reply(From, ok),
- State2
+ State
end.
+log_blocked_process(From, State) ->
+ lager:info("Blocking process ~p due to SolrQ ~p exceeding HWM of ~p", [
+ From,
+ self(),
+ State#state.queue_hwm]),
+ From.
+
%% @doc Enqueue the entry and return updated state.
-enqueue(E, #indexq{queue = Q, queue_len = L, aux_queue = A, draining = Draining} = IndexQ) ->
- case Draining of
- false ->
- IndexQ#indexq{queue = queue:in(E, Q), queue_len = L + 1};
- _ ->
- IndexQ#indexq{aux_queue = queue:in(E, A)}
- end.
+enqueue(E, #state{draining = false,
+ queue = Q} = State) ->
+ lager:debug("enqueue to queue, len=~p",
+ [queue:len(Q)+1]),
+ State#state{queue = queue:in(E, Q)};
+enqueue(E, #state{draining = _Draining,
+ aux_queue = A} = State) ->
+ lager:debug("enqueue to aux_queue, len=~p",
+ [queue:len(A)+1]),
+ State#state{aux_queue = queue:in(E, A)}.
%% @doc Re-enqueue undelivered items as part of updated indexq.
-requeue_undelivered(Undelivered, #indexq{queue = Queue, queue_len = QueueLen} = IndexQ) ->
+requeue_undelivered(Undelivered,
+ #state{queue = Queue} = State) ->
NewQueue = queue:join(queue:from_list(Undelivered), Queue),
- IndexQ#indexq{
- queue = NewQueue,
- queue_len = QueueLen + erlang:length(Undelivered)
- }.
+ State#state{queue = NewQueue}.
%% @doc Trigger a flush and return state
-flush(Index, IndexQ, State) ->
- IndexQ2 = request_worker(Index, IndexQ),
- update_indexq(Index, IndexQ2, State).
+flush(#state{in_flight_len = 0} = State) ->
+ State2 = send_batch_to_helper(State),
+ State2;
+flush(State) ->
+ State.
%% @doc handle a blown fuse by setting the fuse_blown flag,
%% purging any data if required, and unblocking any vnodes
%% if required.
-handle_blown_fuse(Index, #state{purge_strategy=PBIStrategy} = State) ->
- IndexQ = get_indexq(Index, State),
- State1 = update_indexq(Index, IndexQ#indexq{fuse_blown = true}, State),
- State2 = maybe_purge([{Index, IndexQ}], PBIStrategy, State1),
- maybe_unblock_vnodes(State2).
+-spec handle_blown_fuse(state()) -> state().
+handle_blown_fuse(#state{} = State) ->
+ State1 = State#state{fuse_blown = true},
+ State2 = maybe_purge(State1),
+ State3 = maybe_unblock_processes(State2),
+ State3.
%% @doc purge entries depending on purge strategy if we are over the HWM
-maybe_purge(Indices, PBIStrategy, State) ->
- case over_hwm(State) of
+maybe_purge(#state{fuse_blown = false}=State) ->
+ State;
+maybe_purge(State) ->
+ case queue_has_items(State) andalso over_hwm(State) of
true ->
- purge(Indices, PBIStrategy, State);
+ purge(State);
_ ->
State
end.
%% @doc Return true if queue is over the high water mark
-over_hwm(#state{all_queue_len = L, queue_hwm = HWM}) ->
- L > HWM.
+%% Note that there is one "in-flight" message so we use >=, not >
+over_hwm(#state{queue = Queue, aux_queue = AuxQueue, queue_hwm = HWM, in_flight_len = InFlightLen}) ->
+ queue:len(Queue) + queue:len(AuxQueue) + InFlightLen >= HWM.
-maybe_purge_blown_indices(#state{purge_strategy=?PURGE_NONE} = State) ->
- State;
-maybe_purge_blown_indices(#state{indexqs=IndexQs,
- purge_strategy=PBIStrategy}
- = State) ->
- BlownIndices = [{Index, IndexQ} ||
- {Index,
- #indexq{fuse_blown=true,
- queue_len=QueueLen,
- aux_queue=AuxQueue} = IndexQ} <-
- dict:to_list(IndexQs),
- QueueLen > 0 orelse queue:len(AuxQueue) > 0
- ],
- case BlownIndices of
- [] ->
- State;
- _ ->
- purge(BlownIndices, PBIStrategy, State)
- end.
+queue_has_items(#state{queue = Queue,
+ aux_queue = AuxQueue}) ->
+ not (queue:is_empty(Queue) andalso queue:is_empty(AuxQueue)).
+%% @todo: should we move this comment? Or is it fine to have it here??
%% @doc If we hit the high water mark, we will attempt to purge previously
%% enqueued entries based on a purge strategy, which may be one of:
%%
@@ -613,13 +553,10 @@ maybe_purge_blown_indices(#state{indexqs=IndexQs,
%% to purge from the aux_queue, if it is not empty.
%% purge_index: Purge all entries (both in the regular queue and in the
%% aux_queue) from a randomly blown indexq
-%% purge_all: Purge all entries (both in the regular queue and in the
-%% aux_queue) from all blown indexqs
%%
--spec purge([{index_name(), #indexq{}}], purge_strategy(), #state{}) ->
- #state{}.
-purge(BlownIndices, PurgeStrategy, State) ->
- {NewState, NumPurged} = purge_internal(BlownIndices, PurgeStrategy, State),
+-spec purge(state()) -> state().
+purge(State) ->
+ {NewState, NumPurged} = purge_idx(State),
case NumPurged of
0 ->
ok;
@@ -628,62 +565,42 @@ purge(BlownIndices, PurgeStrategy, State) ->
end,
NewState.
--spec purge_internal([{index_name(), #indexq{}}], purge_strategy(), #state{}) ->
- {#state{}, TotalNumPurged :: non_neg_integer()}.
-purge_internal(BlownIndices, ?PURGE_ALL, State) ->
- lists:foldl(fun purge_idx/2, {State, 0}, BlownIndices);
-purge_internal(BlownIndices, PurgeStrategy, State) ->
- I = random:uniform(length(BlownIndices)),
- BlownIndex = lists:nth(I, BlownIndices),
- purge_idx(BlownIndex, PurgeStrategy, State).
-
--spec purge_idx(BlownIndex :: {index_name(), #indexq{}},
- {#state{}, NumPurgedAccum :: non_neg_integer()}) ->
- {#state{}, TotalNumPurged :: non_neg_integer()}.
-purge_idx(BlownIndex, {State, PurgeNumAccum}) ->
- {NewState, NumPurged} = purge_idx(BlownIndex, ?PURGE_IDX, State),
- {NewState, PurgeNumAccum + NumPurged}.
-
--spec purge_idx(BlownIndex :: {index_name(), #indexq{}},
- purge_strategy(),
- #state{}) -> {#state{}, NumPurged :: non_neg_integer()}.
-purge_idx(_, ?PURGE_NONE, State) ->
+-spec purge_idx(state()) -> {state(), NumPurged :: non_neg_integer()}.
+purge_idx(#state{purge_strategy = ?PURGE_NONE} = State) ->
{State, 0};
-purge_idx(
- {Index,
- #indexq{queue=OldQueue, aux_queue=OldAuxQueue}=IndexQ
- },
- ?PURGE_ONE,
- #state{all_queue_len=AllQueueLen} = State) ->
- OldAuxQueueLen = queue:len(OldAuxQueue),
- NewIndexQ =
- case OldAuxQueueLen > 0 of
- true ->
+purge_idx(#state{purge_strategy = ?PURGE_ONE,
+ queue=OldQueue,
+ aux_queue=OldAuxQueue,
+ index = Index}=State) ->
+ NewState =
+ case queue:is_empty(OldAuxQueue) of
+ false ->
NewAuxQueue = maybe_pop_queue(OldAuxQueue),
- IndexQ#indexq{aux_queue=NewAuxQueue};
- _ ->
+ State#state{aux_queue=NewAuxQueue};
+ true ->
NewQueue = maybe_pop_queue(OldQueue),
- IndexQ#indexq{queue=NewQueue, queue_len=queue:len(NewQueue)}
+ State#state{queue=NewQueue}
end,
?DEBUG("Removing item from queue because we have hit the high"
" watermark and the fuse is blown for index ~p, using"
" purge_strategy ~p", [Index, ?PURGE_ONE]),
- {update_indexq(
- Index, NewIndexQ, State#state{all_queue_len=AllQueueLen - 1}),
- 1};
-purge_idx({Index, #indexq{queue_len=QueueLen, aux_queue=AuxQueue}=IndexQ},
- ?PURGE_IDX,
- #state{all_queue_len=AllQueueLen} = State) ->
+ {NewState, 1};
+purge_idx(#state{purge_strategy = ?PURGE_IDX,
+ queue = Queue,
+ aux_queue = AuxQueue,
+ index = Index}=State) ->
+ QueueLen = queue:len(Queue),
AuxQueueLen = queue:len(AuxQueue),
+
NumPurged = QueueLen + AuxQueueLen,
- NewIndexQ = IndexQ#indexq{queue=queue:new(), aux_queue=queue:new(),
- queue_len=0},
+ NewState = State#state{ queue = queue:new(),
+ aux_queue=queue:new()},
?DEBUG("Removing index-queue because we have hit the high"
" watermark and the fuse is blown for index ~p, using"
" purge_strategy ~p", [Index, ?PURGE_IDX]),
- {update_indexq(
- Index, NewIndexQ, State#state{all_queue_len = AllQueueLen - NumPurged}),
- NumPurged}.
+ %% @todo: do we really need NumPurged now that we don't update the length
+ %% in the state all the time?
+ {NewState, NumPurged}.
maybe_pop_queue(Queue) ->
case queue:out(Queue) of
@@ -691,138 +608,117 @@ maybe_pop_queue(Queue) ->
{empty, _Queue} -> Queue
end.
-%% @doc Request a worker to pull the queue
-maybe_request_worker(Index, #indexq{batch_min = Min} = IndexQ) ->
- maybe_request_worker(Index, Min, IndexQ).
-%% @doc Request a worker to pulll the queue with a provided minimum,
+%% The fuse is blown - just return for now
+maybe_send_batch_to_helper(#state{fuse_blown = true} = State) ->
+ State;
+
+%% @doc Send a batch to the helper to pull the queue with a provided minimum,
%% as long as one has not already been requested.
-maybe_request_worker(Index, Min, #indexq{pending_helper = false,
- fuse_blown = false,
- queue_len = L} = IndexQ) when L >= Min ->
- request_worker(Index, IndexQ);
-maybe_request_worker(_Index, _Min, IndexQ) ->
- IndexQ.
-
-%% @doc Notify the solrq workers the index is ready to be pulled.
-request_worker(Index, #indexq{pending_helper = false, fuse_blown = false} = IndexQ) ->
- yz_solrq_helper:index_ready(Index, self()),
- IndexQ#indexq{pending_helper = true};
-request_worker(_Index, IndexQ) ->
- IndexQ.
-
-%% @doc Send a batch of entries, reply to any blocked vnodes and
-%% return updated state
-send_entries(HPid, Index, State) ->
- IndexQ = get_indexq(Index, State),
- #indexq{batch_max = BatchMax} = IndexQ,
- {Batch, _BatchLen, IndexQ2} = get_batch(IndexQ),
+maybe_send_batch_to_helper(#state{in_flight_len = 0,
+ draining = false,
+ batch_min = Min,
+ queue = Queue } = State) ->
+ case queue:len(Queue) >= Min of
+ true ->
+ send_batch_to_helper(State);
+ %% We're not above the min, so this is a no-op
+ false ->
+ State
+ end;
+
+%% @doc when we are draining we send a batch even if the batch size is less
+%% than the batch_min.
+maybe_send_batch_to_helper(#state{draining = true, in_flight_len = 0} = State) ->
+ send_batch_to_helper(State);
+
+%% @doc if we didn't take one of the other paths, our in_fight_len is > 0
+%% just return State for now
+maybe_send_batch_to_helper(State) ->
+ State.
+
+%% @doc Push a batch to the helper.
+send_batch_to_helper(#state{index = Index,
+ batch_max = BatchMax,
+ helper_pid = HPid} = State) ->
+ State1 = maybe_cancel_timer(State),
+ {Batch, State2} = prepare_for_batching(State1),
yz_solrq_helper:index_batch(HPid, Index, BatchMax, self(), Batch),
- case IndexQ2#indexq.queue_len of
- 0 ->
+ case queue:is_empty(State2#state.queue) of
+ true ->
% all the messages have been sent
- update_indexq(Index, IndexQ2, State);
- _ ->
+ State2;
+ false ->
% may be another full batch
- IndexQ3 = maybe_request_worker(Index, IndexQ2),
- % if entries left, restart timer if batch not full
- IndexQ4 = maybe_start_timer(Index, IndexQ3),
- update_indexq(Index, IndexQ4, State)
+ maybe_start_flush_timer(State2)
end.
-%% @doc Get up to batch_max entries and reset the pending worker/timer ref.
-get_batch(#indexq{queue = Q, queue_len = L, batch_max = Max,
- draining = Draining} = IndexQ0) ->
- IndexQ1 = maybe_cancel_timer(IndexQ0),
- {BatchQ, RestQ} =
- case Draining of
- true ->
- {Q, queue:new()};
- _ ->
- queue:split(min(L,Max), Q)
- end,
+%% @doc Get up to batch_max entries.
+prepare_for_batching(#state{queue = Q, batch_max = Max} = State) ->
+ L = queue:len(Q),
+ {BatchQ, RestQ} = queue:split(min(L,Max), Q),
Batch = queue:to_list(BatchQ),
BatchLen = length(Batch),
- IndexQ2 = IndexQ1#indexq{queue = RestQ, queue_len = L - BatchLen,
- in_flight_len = BatchLen,
- batch_start = os:timestamp()},
- {Batch, BatchLen, IndexQ2}.
+ State1 = State#state{
+ batch_start = os:timestamp(),
+ queue = RestQ,
+ in_flight_len = BatchLen
+ },
+ {Batch, State1}.
%% If previous `TimerRef' was set, but not yet triggered, cancel it
-%% first as it's invalid for the next batch of this IndexQ.
-maybe_cancel_timer(#indexq{timer_ref=undefined} = IndexQ) ->
- IndexQ;
-maybe_cancel_timer(#indexq{timer_ref=TimerRef} = IndexQ) ->
+%% first as it's invalid for the next batch of this queue.
+maybe_cancel_timer(#state{flush_timer_ref =undefined} = State) ->
+ State;
+maybe_cancel_timer(#state{flush_timer_ref =TimerRef} = State) ->
erlang:cancel_timer(TimerRef),
- IndexQ#indexq{timer_ref=undefined}.
+ State#state{flush_timer_ref =undefined}.
-%% @doc Send replies to blocked vnodes if under the high water mark
+%% @doc Send replies to blocked processes if under the high water mark
%% and return updated state
-maybe_unblock_vnodes(#state{pending_vnodes = []} = State) ->
+-spec maybe_unblock_processes(state()) -> state().
+maybe_unblock_processes(#state{pending_processes = []} = State) ->
State;
-maybe_unblock_vnodes(#state{pending_vnodes = PendingVnodes} = State) ->
+maybe_unblock_processes(#state{pending_processes = PendingProcesses, queue_hwm = QueueHWM} = State) ->
case over_hwm(State) of
true ->
State;
_ ->
- _ = [gen_server:reply(From, ok) || From <- PendingVnodes],
- State#state{pending_vnodes = []}
- end.
-
-maybe_start_timer(Index, #indexq{timer_ref = undefined, queue_len = L,
- pending_helper = false,
- fuse_blown = false,
- delayms_max = DelayMS} = IndexQ) when L > 0 ->
- TimerRef =
- case DelayMS of
- infinity ->
- lager:debug("Infinite delay, will not start timer and flush."),
- undefined;
- _ ->
- erlang:start_timer(DelayMS, self(), {flush, Index})
- end,
- IndexQ#indexq{timer_ref = TimerRef};
-maybe_start_timer(_Index, IndexQ) ->
- IndexQ.
-
-find_indexq(Index, #state{indexqs = IndexQs}) ->
- case dict:find(Index, IndexQs) of
- {ok, IndexQ} ->
- IndexQ;
- _ ->
- undefined
- end.
-
-get_indexq(Index, #state{indexqs = IndexQs}) ->
- case dict:find(Index, IndexQs) of
- {ok, IndexQ} ->
- IndexQ;
- _ ->
- new_indexq()
+ lists:foreach(fun(PendingProcess) ->
+ unblock_process(PendingProcess, QueueHWM) end,
+ PendingProcesses),
+ State#state{pending_processes = []}
end.
-set_new_index(Min, Max, DelayMS)
- when Min > 0, Min =< Max, DelayMS >= 0 orelse DelayMS == infinity ->
- #indexq{batch_min = Min,
- batch_max = Max,
- delayms_max = DelayMS};
-set_new_index(_, _, _) ->
- #indexq{}.
-
-new_indexq() ->
- BatchMin = app_helper:get_env(?YZ_APP_NAME, ?SOLRQ_BATCH_MIN, 1),
- BatchMax = app_helper:get_env(?YZ_APP_NAME, ?SOLRQ_BATCH_MAX, 100),
- DelayMS = app_helper:get_env(?YZ_APP_NAME, ?SOLRQ_BATCH_FLUSH_INTERVAL,
- 1000),
- set_new_index(BatchMin, BatchMax, DelayMS).
-
-update_indexq(Index, IndexQ, #state{indexqs = IndexQs} = State) ->
- State#state{indexqs = dict:store(Index, IndexQ, IndexQs)}.
+unblock_process(PendingProcess, HWM) ->
+ lager:debug("Unblocking process ~p due to SolrQ ~p going below HWM of ~p", [
+ PendingProcess,
+ self(),
+ HWM]),
+ gen_server:reply(PendingProcess, ok).
+
+maybe_start_flush_timer(#state{delayms_max = infinity} = State) ->
+ lager:debug("Infinite delay, will not start timer and flush."),
+ State#state{flush_timer_ref = undefined};
+maybe_start_flush_timer(#state{flush_timer_ref = undefined,
+ fuse_blown = false,
+ delayms_max = DelayMS,
+ queue = Queue} = State) ->
+ case queue:is_empty(Queue) of
+ true ->
+ State;
+ false ->
+ TimerRef = erlang:start_timer(DelayMS, self(), flush),
+ State#state{flush_timer_ref = TimerRef}
+ end;
+%% timer already running or blown fuse, so don't start a new timer.
+maybe_start_flush_timer(State) ->
+ State.
%% @doc Read settings from the application environment
%% TODO: Update HWM for each Index when Ring-Resize occurrs
read_appenv(State) ->
- HWM = app_helper:get_env(?YZ_APP_NAME, ?SOLRQ_HWM, 1),
+ HWM = app_helper:get_env(?YZ_APP_NAME, ?SOLRQ_HWM, ?SOLRQ_HWM_DEFAULT),
PBIStrategy = application:get_env(?YZ_APP_NAME, ?SOLRQ_HWM_PURGE_STRATEGY,
?PURGE_ONE),
State#state{queue_hwm = HWM,
@@ -832,11 +728,17 @@ read_appenv(State) ->
%% debugging
%%
--ifdef(PULSE).
+-compile([{nowarn_unused_function, [
+ {debug_queue,1 },
+ {debug_indexq, 1},
+ {debug_state, 1}]}]).
+
debug_queue(Queue) ->
[erlang:element(1, Entry) || Entry <- queue:to_list(Queue)].
-debug_indexq(#indexq{queue = Queue, queue_len = QueueLen, aux_queue = AuxQueue, draining = Draining, fuse_blown = FuseBlown, in_flight_len = InFlightLen}) ->
+debug_indexq(#state{queue = Queue, aux_queue = AuxQueue, draining = Draining,
+ fuse_blown = FuseBlown, in_flight_len = InFlightLen}) ->
+ QueueLen = queue:len(Queue),
[
{queue, debug_queue(Queue)},
{queue_len, QueueLen},
@@ -846,9 +748,29 @@ debug_indexq(#indexq{queue = Queue, queue_len = QueueLen, aux_queue = AuxQueue,
{in_flight_len, InFlightLen}
].
debug_state(State) ->
+ Len = queue:len(State#state.queue),
[
- {indexqs, [{Index, debug_indexq(IndexQ)} || {Index, IndexQ} <- dict:to_list(State#state.indexqs)]},
- {all_queue_len, State#state.all_queue_len},
- {drain_info, State#state.drain_info}
- ].
--endif.
+ {index, State#state.index},
+ {all_queue_len, Len},
+ {drain_info, State#state.drain_info},
+ {batch_start, State#state.batch_start},
+ {timer_ref, State#state.flush_timer_ref}
+ ] ++ debug_indexq(State).
+
+get_helper(Index, Partition, State0) ->
+ HelperName = yz_solrq:helper_regname(Index, Partition),
+ HPid = whereis(HelperName),
+ State0#state{helper_pid = HPid}.
+
+maybe_send_drain_complete(wait_for_drain_complete, DPid, Token) ->
+ yz_solrq_drain_fsm:drain_complete(DPid, Token);
+maybe_send_drain_complete(_Draining, _DPid, _Token) ->
+ ok.
+
+maybe_demonitor_draining_process({_DPid, _TokenRef, MonRef})
+ when is_reference(MonRef) ->
+ %% Demonitor and remove a potential 'DOWN'
+ %% message from this MonRef in our mailbox
+ erlang:demonitor(MonRef, [flush]);
+maybe_demonitor_draining_process(undefined) ->
+ ok.
diff --git a/src/yz_stat.erl b/src/yz_stat.erl
index 6a827bff..e3728135 100644
--- a/src/yz_stat.erl
+++ b/src/yz_stat.erl
@@ -31,7 +31,11 @@
%% Public API
-export([start_link/0,
get_stats/0,
+ get_stat/1,
+ get_stat/2,
index_fail/0,
+ index_bad_entry/0,
+ index_extract_fail/0,
index_end/3,
drain_end/1,
batch_end/1,
@@ -82,6 +86,20 @@ register_stats() ->
get_stats() ->
riak_core_stat:get_stats(?APP).
+%% @doc Return the value for the stat with the given `Name', or `undefined' if
+%% not found.
+-spec get_stat([atom(), ...]) -> undefined | term().
+get_stat(Name) when is_list(Name) ->
+ get_stat(Name, value).
+get_stat(Name, SummaryName) ->
+ Path = [?PFX, ?APP] ++ Name,
+ case exometer:get_value(Path, SummaryName) of
+ {ok, [{SummaryName, Value}]} ->
+ Value;
+ _ ->
+ undefined
+ end.
+
%% @doc Initialize the Fuse stats subsystem for a given fuse.
-spec initialize_fuse_stats(Name :: atom()) -> ok.
initialize_fuse_stats(Name) ->
@@ -92,6 +110,16 @@ initialize_fuse_stats(Name) ->
index_fail() ->
update(index_fail).
+%% @doc Send stat updates for an index failure.
+-spec index_bad_entry() -> ok.
+index_bad_entry() ->
+ update(index_bad_entry).
+
+%% @doc Send stat updates for an index failure.
+-spec index_extract_fail() -> ok.
+index_extract_fail() ->
+ update(index_extract_fail).
+
%% @doc Send stat updates for an index completion. `ElapsedTime'
%% should be microseconds.
-spec index_end(index_name(),
@@ -197,6 +225,10 @@ perform_update({index_end, BatchSize, Time}) ->
exometer:update([?PFX, ?APP, queue, batchsize], BatchSize);
perform_update(index_fail) ->
exometer:update([?PFX, ?APP, index, fail], 1);
+perform_update(index_bad_entry) ->
+ exometer:update([?PFX, ?APP, index, bad_entry], 1);
+perform_update(index_extract_fail) ->
+ exometer:update([?PFX, ?APP, index, extract, fail], 1);
perform_update({batch_end, Time}) ->
exometer:update([?PFX, ?APP, queue, batch, latency], Time);
perform_update(blockedvnode) ->
@@ -311,6 +343,14 @@ stats() -> [
{count, search_index_fail_count},
{one, search_index_fail_one}
]},
+ {[index, bad_entry], spiral, [], [
+ {count, search_index_bad_entry_count},
+ {one, search_index_bad_entry_one}
+ ]},
+ {[index, extract, fail], spiral, [], [
+ {count, search_index_extract_fail_count},
+ {one, search_index_extract_fail_one}
+ ]},
{[index, latency], histogram, [], [
{min, search_index_latency_min},
{max, search_index_latency_max},
diff --git a/src/yz_wm_search.erl b/src/yz_wm_search.erl
index f4697a8f..704b3059 100644
--- a/src/yz_wm_search.erl
+++ b/src/yz_wm_search.erl
@@ -1,6 +1,6 @@
%% -------------------------------------------------------------------
%%
-%% Copyright (c) 2012 Basho Technologies, Inc. All Rights Reserved.
+%% Copyright (c) 2012-2016 Basho Technologies, Inc.
%%
%% This file is provided to you under the Apache License,
%% Version 2.0 (the "License"); you may not use this file
@@ -92,7 +92,7 @@ resource_forbidden(RD, Ctx=#ctx{security=Security}, Permission,
{false, RD, Ctx}
end.
-
+-spec forbidden(#wm_reqdata{}, #ctx{}) -> {boolean(), #wm_reqdata{}, #ctx{}}.
%% @doc Uses the riak_kv,secure_referer_check setting rather
%% as opposed to a special yokozuna-specific config
forbidden(RD, Ctx=#ctx{security=undefined}) ->
diff --git a/test/yokozuna_schema_tests.erl b/test/yokozuna_schema_tests.erl
index 6664b1fc..9edcefc6 100644
--- a/test/yokozuna_schema_tests.erl
+++ b/test/yokozuna_schema_tests.erl
@@ -1,3 +1,23 @@
+%% -------------------------------------------------------------------
+%%
+%% Copyright (c) 2013-2016 Basho Technologies, Inc.
+%%
+%% This file is provided 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.
+%%
+%% -------------------------------------------------------------------
+
-module(yokozuna_schema_tests).
-include_lib("eunit/include/eunit.hrl").
-compile(export_all).
@@ -38,18 +58,17 @@ basic_schema_test() ->
async_dirty),
cuttlefish_unit:assert_config(Config, "yokozuna.solrq_hwm_purge_strategy",
purge_one),
- cuttlefish_unit:assert_config(Config, "yokozuna.solrq_batch_min", 1),
- cuttlefish_unit:assert_config(Config, "yokozuna.solrq_batch_max", 100),
+ cuttlefish_unit:assert_config(Config, "yokozuna.solrq_batch_min", 10),
+ cuttlefish_unit:assert_config(Config, "yokozuna.solrq_batch_max", 500),
cuttlefish_unit:assert_config(Config, "yokozuna.solrq_batch_flush_interval",
- 1000),
- cuttlefish_unit:assert_config(Config, "yokozuna.solrq_hwm", 10),
- cuttlefish_unit:assert_config(Config, "yokozuna.solrq_worker_count", 10),
- cuttlefish_unit:assert_config(Config, "yokozuna.solrq_helper_count", 10),
+ 500),
+ cuttlefish_unit:assert_config(Config, "yokozuna.solrq_hwm", 1000),
cuttlefish_unit:assert_config(Config, "yokozuna.solrq_drain_timeout", 60000),
cuttlefish_unit:assert_config(Config, "yokozuna.solrq_drain_cancel_timeout", 5000),
- cuttlefish_unit:assert_config(Config, "yokozuna.solrq_drain_enable", false),
+ cuttlefish_unit:assert_config(Config, "yokozuna.solrq_drain_enable", true),
cuttlefish_unit:assert_config(Config, "yokozuna.ibrowse_max_sessions", 100),
cuttlefish_unit:assert_config(Config, "yokozuna.ibrowse_max_pipeline_size", 1),
+ cuttlefish_unit:assert_config(Config, "yokozuna.enable_dist_query", true),
ok.
override_schema_test() ->
@@ -72,20 +91,18 @@ override_schema_test() ->
{["search", "index", "error_threshold", "failure_interval"], "10s"},
{["search", "index", "error_threshold", "reset_interval"], "60s"},
{["search", "fuse_context"], "sync"},
- {["search", "queue", "batch", "minimum"], 10},
+ {["search", "queue", "batch", "minimum"], 100},
{["search", "queue", "batch", "maximum"], 10000},
{["search", "queue", "batch", "flush_interval"], infinity},
{["search", "queue", "high_watermark"], 100000},
- {["search", "queue", "worker_count"], 5},
- {["search", "queue", "helper_count"], 20},
- {["search", "queue", "high_watermark", "purge_strategy"],
- "purge_all"},
+ {["search", "queue", "high_watermark", "purge_strategy"], "purge_index"},
{["search", "queue", "drain", "enable"], "off"},
{["search", "queue", "drain", "timeout"], "2m"},
{["search", "queue", "drain", "cancel", "timeout"], "10ms"},
{["search", "ibrowse_max_sessions"], 101},
- {["search", "ibrowse_max_pipeline_size"], 11}
- ],
+ {["search", "ibrowse_max_pipeline_size"], 11},
+ {["search", "dist_query"], "off"}
+ ],
Config = cuttlefish_unit:generate_templated_config(
"../priv/yokozuna.schema", Conf, context(), predefined_schema()),
@@ -112,14 +129,12 @@ override_schema_test() ->
cuttlefish_unit:assert_config(Config, "yokozuna.fuse_ctx",
sync),
cuttlefish_unit:assert_config(Config, "yokozuna.solrq_hwm_purge_strategy",
- purge_all),
- cuttlefish_unit:assert_config(Config, "yokozuna.solrq_batch_min", 10),
+ purge_index),
+ cuttlefish_unit:assert_config(Config, "yokozuna.solrq_batch_min", 100),
cuttlefish_unit:assert_config(Config, "yokozuna.solrq_batch_max", 10000),
cuttlefish_unit:assert_config(Config, "yokozuna.solrq_batch_flush_interval",
infinity),
cuttlefish_unit:assert_config(Config, "yokozuna.solrq_hwm", 100000),
- cuttlefish_unit:assert_config(Config, "yokozuna.solrq_worker_count", 5),
- cuttlefish_unit:assert_config(Config, "yokozuna.solrq_helper_count", 20),
cuttlefish_unit:assert_config(Config, "yokozuna.solrq_drain_timeout",
120000),
cuttlefish_unit:assert_config(Config, "yokozuna.solrq_drain_cancel_timeout",
@@ -127,6 +142,7 @@ override_schema_test() ->
cuttlefish_unit:assert_config(Config, "yokozuna.solrq_drain_enable", false),
cuttlefish_unit:assert_config(Config, "yokozuna.ibrowse_max_sessions", 101),
cuttlefish_unit:assert_config(Config, "yokozuna.ibrowse_max_pipeline_size", 11),
+ cuttlefish_unit:assert_config(Config, "yokozuna.enable_dist_query", false),
ok.
validations_test() ->
@@ -138,9 +154,7 @@ validations_test() ->
{["search", "index", "error_threshold", "failure_count"], 0},
{["search", "queue", "batch", "minimum"], -1},
{["search", "queue", "high_watermark"], -1},
- {["search", "queue", "batch", "maximum"], -10},
- {["search", "queue", "worker_count"], 0},
- {["search", "queue", "helper_count"], 20}
+ {["search", "queue", "batch", "maximum"], -10}
],
Config = cuttlefish_unit:generate_templated_config(
"../priv/yokozuna.schema", Conf, context(), predefined_schema()),
@@ -153,9 +167,7 @@ validations_test() ->
{"search.queue.batch.minimum",
"must be a positive integer > 0"},
{"search.queue.high_watermark",
- "must be an integer >= 0"},
- {"search.queue.worker_count",
- "must be a positive integer > 0"}],
+ "must be an integer >= 0"}],
ListOfErrors2),
ok.
diff --git a/test/yz_component_tests.erl b/test/yz_component_tests.erl
index 41b31ca6..7faf6871 100644
--- a/test/yz_component_tests.erl
+++ b/test/yz_component_tests.erl
@@ -6,7 +6,7 @@
disable_index_test()->
yokozuna:disable(index),
- ?assertEqual(yz_kv:index({},delete,{}), ok).
+ ?assertEqual(yz_kv:index({riak_object:new({<<"type">>, <<"bucket">>}, <<"key">>, <<"value">>), no_old_object}, delete, {}), ok).
disable_search_test()->
yokozuna:disable(search),
diff --git a/test/yz_solrq_eqc.erl b/test/yz_solrq_eqc.erl
index 3aa46265..ae13563e 100644
--- a/test/yz_solrq_eqc.erl
+++ b/test/yz_solrq_eqc.erl
@@ -8,13 +8,8 @@
-ifdef(EQC).
-include_lib("eqc/include/eqc.hrl").
-
-include_lib("eunit/include/eunit.hrl").
-
--ifdef(PULSE).
--include_lib("pulse/include/pulse.hrl").
--compile([export_all, {parse_transform, pulse_instrument}]).
--compile({pulse_replace_module, [{gen_server, pulse_gen_server}]}).
+-compile([export_all]).
-define(QC_OUT(P),
eqc:on_output(fun(Str, Args) -> io:format(user, Str, Args) end, P)).
@@ -26,15 +21,15 @@
solrq_test_() ->
{setup,
fun() ->
- error_logger:tty(false),
- pulse:start()
+ ok %error_logger:tty(false)
end,
fun(_) ->
- pulse:stop(),
+ unlink_kill(yz_solrq_sup),
error_logger:tty(true)
end,
{timeout, 300,
fun() ->
+ %% pulse:verbose([format]),
?assert(eqc:quickcheck(?QC_OUT(eqc:testing_time(120, prop_ok()))))
end
}
@@ -109,12 +104,6 @@ prop_ok() ->
{Entries0, {HWM, Min, Max}},
{gen_entries(), gen_params()},
begin
- true = lists:member({'PULSE-REPLACE-MODULE',1},
- ?MODULE:module_info(exports)),
- true = lists:member({'PULSE-REPLACE-MODULE',1},
- yz_solrq_worker:module_info(exports)),
- true = lists:member({'PULSE-REPLACE-MODULE',1},
- yz_solrq_helper:module_info(exports)),
%% Reset the solrq/solrq helper processes
application:set_env(?YZ_APP_NAME, ?SOLRQ_HWM, HWM),
@@ -128,7 +117,8 @@ prop_ok() ->
Entries = add_keys(Entries0),
KeyRes = make_keyres(Entries),
PE = entries_by_vnode(Entries),
- Partitions = partitions(Entries),
+ Partitions = partitions(Entries0),
+ Indexes = indexes(Entries0),
meck:expect(
ibrowse, send_req,
@@ -137,34 +127,23 @@ prop_ok() ->
{Keys, Res} = yz_solrq_eqc_ibrowse:get_response(B),
solr_responses:record(Keys, Res),
Res
- end
- ),
-
- ?PULSE(
- {SolrQ, Helper, IBrowseKeys, MeltsByIndex},
- begin
- reset(), % restart the processes
- unlink_kill(yz_solrq_worker_0001),
- unlink_kill(yz_solrq_helper_0001),
- unlink_kill(yz_solrq_eqc_fuse),
- unlink_kill(yz_solrq_eqc_ibrowse),
- {ok, SolrQ} = yz_solrq_worker:start_link(yz_solrq_worker_0001),
- {ok, Helper} = yz_solrq_helper:start_link(yz_solrq_helper_0001),
- {ok, _} = yz_solrq_eqc_fuse:start_link(),
- {ok, _} = yz_solrq_eqc_ibrowse:start_link(KeyRes),
-
- %% Issue the requests under pulse
- Pids = ?MODULE:send_entries(PE),
- start_drains([undefind|Partitions]),
- wait_for_vnodes(Pids, timer:seconds(20)),
- timer:sleep(500),
- catch yz_solrq_eqc_ibrowse:wait(expected_keys(Entries)),
- {SolrQ, Helper, yz_solrq_eqc_ibrowse:keys(), melts_by_index(Entries)}
- end,
- ?WHENFAIL(
+ end),
+ reset(), % restart the processes
+ unlink_kill(yz_solrq_eqc_fuse),
+ unlink_kill(yz_solrq_eqc_ibrowse),
+ start_solrqs(Partitions, Indexes),
+ {ok, _} = yz_solrq_eqc_fuse:start_link(),
+ {ok, _} = yz_solrq_eqc_ibrowse:start_link(KeyRes),
+
+ %% Issue the requests under pulse
+ Pids = ?MODULE:send_entries(PE),
+ yz_solrq_drain_mgr:drain(),
+ wait_for_vnodes(Pids, timer:seconds(20)),
+ catch yz_solrq_eqc_ibrowse:wait(expected_keys(Entries)),
+ IBrowseKeys = yz_solrq_eqc_ibrowse:keys(),
+ MeltsByIndex = melts_by_index(Entries),
+ ?WHENFAIL(
begin
- eqc:format("SolrQ: ~p\n", [SolrQ]),
- eqc:format("Helper: ~p\n", [Helper]),
eqc:format("KeyRes: ~p\n", [KeyRes]),
eqc:format("keys(): ~p\n", [IBrowseKeys]),
eqc:format("expected_entry_keys: ~p\n", [expected_entry_keys(PE)]),
@@ -172,7 +151,7 @@ prop_ok() ->
eqc:format("melts_by_index: ~p~n", [MeltsByIndex]),
eqc:format("Entries: ~p\n", [Entries]),
%debug_history([ibrowse, solr_responses, yz_kv])
- debug_history([solr_responses])
+ eqc:format("debug history: ~p\n", [debug_history([solr_responses, ibrowse])])
end,
begin
%% For each vnode, spawn a process and start sending
@@ -181,9 +160,9 @@ prop_ok() ->
%Expect = lists:sort(lists:flatten([Es || {_P,Es} <- PE])),
%equals(Expect, lists:sort(ibrowse_requests()))
- HttpRespByKey = http_response_by_key(),
- HashtreeHistory = hashtree_history(),
- HashtreeExpect = hashtree_expect(Entries, HttpRespByKey),
+ %HttpRespByKey = http_response_by_key(),
+ %HashtreeHistory = hashtree_history(),
+ %HashtreeExpect = hashtree_expect(Entries, HttpRespByKey),
%eqc:collect({hwm, HWM},
% eqc:collect({batch_min, Min},
@@ -192,11 +171,11 @@ prop_ok() ->
{solr, equals(
lists:sort(IBrowseKeys),
solr_expect(Entries))},
- {hashtree, equals(HashtreeHistory, HashtreeExpect)},
+ %{hashtree, equals(HashtreeHistory, HashtreeExpect)},
%% TODO Modify ordering test to center around key order, NOT partition order.
%% requires a fairly significant change to the test structure, becuase currently
%% all keys are unique.
- {insert_order, ordered(expected_entry_keys(PE), IBrowseKeys)},
+ %% {insert_order, ordered(expected_entry_keys(PE), IBrowseKeys)},
{melts, equals(MeltsByIndex, errors_by_index(Entries))}
])
% )
@@ -204,8 +183,6 @@ prop_ok() ->
%)
end
)
-
- )
end
)
).
@@ -221,13 +198,13 @@ setup() ->
fun(_, _, Fmt, Args) ->
io:format(user, "LAGER: " ++ Fmt, Args)
end),
+ %% Force stats updates to not go through SideJob by saying we don't have the resource
+ meck:new(sidejob, [passthrough]),
+ meck:expect(sidejob, resource_exists, fun(_Resource) -> false end),
application:start(syntax_tools),
application:start(compiler),
application:start(goldrush),
- yz_solrq:set_solrq_worker_tuple(1), % for yz_solrq_sup:regname
- yz_solrq:set_solrq_helper_tuple(1), % for yz_solrq_helper_sup:regname
-
meck:new(ibrowse),
%% meck:expect(ibrowse, send_req, fun(_A, _B, _C, _D, _E, _F) ->
%% io:format("REQ: ~p\n", [{_A,_B,_C,_D,_E,_F}]),
@@ -270,6 +247,12 @@ setup() ->
meck:new(solr_responses, [non_strict]),
meck:expect(solr_responses, record, fun(_Keys, _Response) -> ok end),
+
+ meck:new(riak_core_ring_manager, [passthrough]),
+ meck:new(riak_core_ring, [passthrough]),
+ meck:new(yz_index, [passthrough]),
+
+
%% Apply the pulse transform to the modules in the test
%% Pulse compile solrq/solrq helper
%% TODO dynamically pulse_instrument
@@ -280,17 +263,13 @@ setup() ->
% yz_pulseh:compile(yz_solrq, Opts),
% yz_pulseh:compile(yz_solrq_helper, Opts),
- %% And start up supervisors to own the solrq/solrq helper
- %% {ok, SolrqSup} = yz_solrq_sup:start_link(1),
- %% {ok, HelperSup} = yz_solrq_helper_sup:start_link(1),
- %% io:format(user, "SolrqSup = ~p HelperSup = ~p\n", [SolrqSup, HelperSup]),
ok.
cleanup() ->
meck:unload(),
%% unlink_kill(yz_solrq_helper_sup),
- %% unlink_kill(yz_solrq_sup),
+ unlink_kill(yz_solrq_sup),
catch application:stop(fuse),
@@ -306,6 +285,9 @@ reset() ->
meck:reset(ibrowse),
meck:reset(solr_responses),
meck:reset(yz_kv),
+ meck:reset(riak_core_ring_manager),
+ meck:reset(riak_core_ring),
+ meck:reset(yz_index),
ok.
@@ -509,7 +491,16 @@ unlink_kill(Name) ->
end.
partitions(Entries) ->
- [P || {P, _Index, _Bucket, _Reason, _Result} <- Entries].
+ _PartitionList = [P || {P, _Index, _Bucket, _Reason, _Result} <- Entries].
+ %% unique_entries(PartitionList).
+
+indexes(Entries) ->
+ _IndexList = [Index || {_P, Index, _Bucket, _Reason, _Result} <- Entries].
+ %% unique_entries(IndexList).
+
+unique_entries(List) ->
+ Set = sets:from_list(List),
+ sets:to_list(Set).
add_keys(Entries) ->
[{P, Index, Bucket, make_key(Seq), Reason, Result} ||
@@ -524,7 +515,7 @@ make_keyres(Entries) ->
expected_keys(Entries) ->
[Key || {_P, Index, _Bucket, Key, _Reason, Result} <- Entries, Index /= ?YZ_INDEX_TOMBSTONE, Result /= {ok, "400", bad, request}].
- entries_by_vnode(Entries) ->
+entries_by_vnode(Entries) ->
lists:foldl(fun({P, Index, Bucket, Key, Reason, Result}, Acc) ->
orddict:append_list(P, [{Index, Bucket, Key, Reason, Result}], Acc)
end, orddict:new(), Entries).
@@ -536,7 +527,8 @@ send_entries(PE) ->
%% Send the entries for a vnode
send_vnode_entries(Runner, P, Events) ->
self() ! {ohai, length(Events)},
- [yz_solrq:index(Index, {Bucket, Key}, make_obj(Bucket, Key), Reason, P) || {Index, Bucket, Key, Reason, _Result} <- Events],
+ [yz_solrq:index(Index, {Bucket, Key}, make_obj(Bucket, Key), Reason, P)
+ || {Index, Bucket, Key, Reason, _Result} <- Events],
receive
{ohai, _Len} ->
ok
@@ -544,50 +536,17 @@ send_vnode_entries(Runner, P, Events) ->
Runner ! {self(), done}.
make_obj(B,K) ->
- riak_object:new(B, K, K, "application/yz_solrq_eqc"). % Set Key as value
-
-
-
-start_drains(Partitions) ->
- spawn_link(fun() -> drain(Partitions) end).
-
-drain([]) ->
- ok;
-drain([P | Rest] = _Partitions) ->
- %% TODO fix this so that drain can be called (requires support for yz_solrq_sup
- %ok = yz_solrq_sup:drain(),
- try
- {ok, Pid} = yz_solrq_drain_fsm:start_link([{partition, P}]),
- Reference = erlang:monitor(process, Pid),
- yz_solrq_drain_fsm:start_prepare(),
- receive
- {'DOWN', Reference, _Type, _Object, normal} ->
- ok;
- {'DOWN', Reference, _Type, _Object, Info} ->
- {error, Info}
- after 10000 ->
- erlang:demonitor(Reference),
- %lager:error("Warning! Drain timed out. Cancelling..."),
- yz_solrq_drain_fsm:cancel(),
- {error, timeout}
- end
- catch
- _:badarg ->
- %lager:error("Error! Drain in progress."),
- {error, in_progress}
- end,
- timer:sleep(500),
- drain(Rest).
+ {riak_object:new(B, K, K, "application/yz_solrq_eqc"), no_old_object}. % Set Key as value
%% Wait for send_entries - should probably set a global timeout and
%% and look for that instead
-wait_for_vnodes(Pids, _Timeout) ->
+wait_for_vnodes(Pids, Timeout) ->
RRef = make_ref(),
- %% TRef = erlang:send_after(Timeout, self(), {timeout, RRef}),
+ TRef = erlang:send_after(Timeout, self(), {timeout, RRef}),
wait_for_vnodes_msgs(Pids, RRef),
- %% erlang:cancel_timer(TRef),
+ erlang:cancel_timer(TRef),
receive
- {timeout, _TRef} -> %todo - remove underscore if renable timeout
+ {timeout, TRef} -> %todo - remove underscore if renable timeout
ok
after
0 ->
@@ -607,6 +566,17 @@ wait_for_vnodes_msgs([Pid | Pids], Ref) ->
wait_for_vnodes_msgs([Pid|Pids], Ref)
end.
+start_solrqs(Partitions, Indexes) ->
+ %% Ring retrieval for required workers
+ UniquePartitions = unique_entries(Partitions),
+ meck:expect(riak_core_vnode_manager, all_vnodes, fun(riak_kv_vnode) ->
+ [{riak_kv_vnode, Idx, fake_pid} || Idx <- UniquePartitions]
+ end),
+ meck:expect(yz_index, get_indexes_from_meta, fun() -> unique_entries(Indexes) -- [?YZ_INDEX_TOMBSTONE] end),
+ %% And start up supervisors to own the solrq/solrq helper
+ _ = yz_solrq_sup:start_link(),
+ _ = yz_solrq_sup:sync_active_queue_pairs().
+
%% ibrowse_requests() ->
%% [ibrowse_call_extract(Args, Res) || {_Pid, {ibrowse, send_req, Args, Res}} <- meck:history(ibrowse)].
@@ -614,16 +584,7 @@ wait_for_vnodes_msgs([Pid | Pids], Ref) ->
%% {parse_solr_url(Url), parse_solr_reqs(mochijson2:decode(JsonIolist))}.
debug_history(Mods) ->
- [io:format("~p\n====\n~p\n\n", [Mod, meck:history(Mod)]) || Mod <- Mods],
- ok.
-
--else. %% PULSE is not defined
-
-pulse_warning_test() ->
- ?debugMsg("WARNING: PULSE is not defined. Run `make pulse` to execute this test."),
- ok.
-
--endif. % PULSE
+ [{Mod, meck:history(Mod)} || Mod <- Mods].
-else. %% EQC is not defined
diff --git a/test/yz_solrq_eqc_fuse.erl b/test/yz_solrq_eqc_fuse.erl
index cd7db251..fb6ef1ee 100644
--- a/test/yz_solrq_eqc_fuse.erl
+++ b/test/yz_solrq_eqc_fuse.erl
@@ -33,12 +33,6 @@
code_change/3
]).
--ifdef(PULSE).
--include_lib("pulse/include/pulse.hrl").
--compile([export_all, {parse_transform, pulse_instrument}]).
--compile({pulse_replace_module, [{gen_server, pulse_gen_server}]}).
--endif.
-
-define(SERVER, ?MODULE).
-record(state, {
diff --git a/test/yz_solrq_eqc_ibrowse.erl b/test/yz_solrq_eqc_ibrowse.erl
index 4613b8b0..60e2c1e1 100644
--- a/test/yz_solrq_eqc_ibrowse.erl
+++ b/test/yz_solrq_eqc_ibrowse.erl
@@ -31,13 +31,11 @@
terminate/2,
code_change/3]).
--ifdef(PULSE).
--include_lib("pulse/include/pulse.hrl").
--compile([export_all, {parse_transform, pulse_instrument}]).
--compile({pulse_replace_module, [{gen_server, pulse_gen_server}]}).
--define(PULSE_DEBUG(S,F), pulse:format(S,F)).
+-ifdef(EQC).
+%%-define(EQC_DEBUG(S, F), io:fwrite(user, S, F)).
+-define(EQC_DEBUG(S, F), ok).
-else.
--define(PULSE_DEBUG(S,F), ok).
+-define(EQC_DEBUG(S, F), ok).
-endif.
-define(SERVER, ?MODULE).
@@ -85,6 +83,7 @@ handle_call(
expected=Expected
} = State
) ->
+ ?EQC_DEBUG("In get_response~n", []),
SolrReqs = parse_solr_reqs(mochijson2:decode(B)),
{Keys, Res, NewFailed} = get_response(SolrReqs, KeyRes, AlredyFailed),
WrittenKeys = case Res of
@@ -100,7 +99,7 @@ handle_call(
_ ->
proceed
end,
- ?PULSE_DEBUG("yz_solrq_eqc_ibrowse: response: ~p~n", [{Keys, Res}]),
+ ?EQC_DEBUG("yz_solrq_eqc_ibrowse: response: ~p~n", [{Keys, Res}]),
{reply, {Keys, Res}, State#state{written = NewWritten, failed = NewFailed}};
handle_call(keys, _From, #state{written = Written} = State) ->
@@ -113,10 +112,9 @@ handle_call({wait, Keys}, From, #state{written=Written} = State) ->
_ ->
case lists:usort(Written) == lists:usort(Keys) of
true ->
- timer:sleep(500),
{reply, ok, State};
_ ->
- ?PULSE_DEBUG("Process ~p waiting for keys...: ~p", [From, Keys]),
+ ?EQC_DEBUG("Process ~p waiting for keys...: ~p~n", [From, Keys]),
{noreply, State#state{root=From, expected=lists:usort(Keys)}}
end
end;
@@ -148,6 +146,8 @@ parse_solr_req({<<"add">>, {struct, [{<<"doc">>, Doc}]}}) ->
{add, find_key_field(Doc)};
parse_solr_req({<<"delete">>, {struct, [{<<"query">>, Query}]}}) ->
{delete, parse_delete_query(Query)};
+parse_solr_req({<<"delete">>, {struct, [{<<"id">>, Id}]}}) ->
+ {delete, parse_delete_id(Id)};
parse_solr_req({delete, _Query}) ->
{delete, could_parse_bkey};
parse_solr_req({Other, Thing}) ->
@@ -160,6 +160,10 @@ parse_delete_query(Query) ->
{match, [Key]} = re:run(Query, "(XKEYX[0-9]+)",[{capture,[1],binary}]),
Key.
+parse_delete_id(Id) ->
+ {match, [Key]} = re:run(Id, "(XKEYX[0-9]+)",[{capture,[1],binary}]),
+ Key.
+
%% Decide what to return for the request... If any of the seq
%% numbers had failures generated, apply to all of them, but
@@ -199,4 +203,4 @@ maybe_reply(undefined) ->
ok;
maybe_reply(Root) ->
timer:sleep(500),
- gen_server:reply(Root, ok).
\ No newline at end of file
+ gen_server:reply(Root, ok).